You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@geode.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/12/07 19:08:10 UTC

[jira] [Commented] (GEODE-3622) The first HeapLRU evictions on large region can consume high amounts of CPU

    [ https://issues.apache.org/jira/browse/GEODE-3622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16282345#comment-16282345 ] 

ASF GitHub Bot commented on GEODE-3622:
---------------------------------------

dschneider-pivotal closed pull request #1017: GEODE-3622: Implement new algorithm for eviction
URL: https://github.com/apache/geode/pull/1017
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession.java b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession.java
index 7ef9807732..f69016acc4 100644
--- a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession.java
+++ b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession.java
@@ -45,7 +45,7 @@
 import org.apache.geode.Delta;
 import org.apache.geode.InvalidDeltaException;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.lru.Sizeable;
+import org.apache.geode.internal.size.Sizeable;
 import org.apache.geode.internal.util.BlobHelper;
 import org.apache.geode.modules.gatewaydelta.GatewayDelta;
 import org.apache.geode.modules.gatewaydelta.GatewayDeltaEvent;
diff --git a/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java b/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
index bdc3e0fbf8..0a68251962 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
@@ -122,7 +122,8 @@
  * any callbacks, ({@link CacheListener}, {@link CacheWriter}, and {@link CacheLoader} are shared by
  * the parent and all its dynamic children so make sure the callback is thread-safe and that its
  * {@link CacheCallback#close} implementation does not stop it from functioning. However the
- * products LRUAlgorithm instances will be cloned so that each dynamic Region has its own callback.
+ * products EvictionAlgorithm instances will be cloned so that each dynamic Region has its own
+ * callback.
  *
  * <li>The root Region name "DynamicRegions" is reserved. The factory creates a root Region of that
  * name and uses it to keep track of what dynamic Regions exist. Applications should not directly
diff --git a/geode-core/src/main/java/org/apache/geode/cache/EvictionAlgorithm.java b/geode-core/src/main/java/org/apache/geode/cache/EvictionAlgorithm.java
index 48b7ad9441..8cd6c18a0c 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/EvictionAlgorithm.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/EvictionAlgorithm.java
@@ -52,7 +52,7 @@
    * An algorithm that considers the number of Entries in the Region before invoking its
    * {@link EvictionAction}
    *
-   * @deprecated
+   * @deprecated For internal use only.
    */
   public static final EvictionAlgorithm LIFO_ENTRY = new EvictionAlgorithm(4);
 
@@ -60,7 +60,7 @@
    * An algorithm that considers the amount of bytes consumed by the Region before invoking its
    * {@link EvictionAction}
    *
-   * @deprecated
+   * @deprecated For internal use only.
    */
   public static final EvictionAlgorithm LIFO_MEMORY = new EvictionAlgorithm(5);
 
@@ -133,7 +133,7 @@ public boolean isNone() {
   }
 
   /**
-   * @deprecated
+   * @deprecated For internal use only.
    */
   public boolean isLIFO() {
     return this == LIFO_ENTRY || this == LIFO_MEMORY;
diff --git a/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java b/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java
index ea5d720a4d..8ef0d484ba 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/EvictionAttributes.java
@@ -482,6 +482,7 @@ public String toString() {
   /**
    * @return an EvictionAttributes for the LIFOCapacityController
    * @since GemFire 5.7
+   * @deprecated For internal use only.
    */
   public static EvictionAttributes createLIFOEntryAttributes(int maximumEntries,
       EvictionAction evictionAction) {
@@ -492,6 +493,7 @@ public static EvictionAttributes createLIFOEntryAttributes(int maximumEntries,
   /**
    * @return an EvictionAttributes for the MemLIFOCapacityController
    * @since GemFire 5.7
+   * @deprecated For internal use only.
    */
   public static EvictionAttributes createLIFOMemoryAttributes(int maximumMegabytes,
       EvictionAction evictionAction) {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/OverflowQueueWithDMStats.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/OverflowQueueWithDMStats.java
index 5d455f28d2..8f57d897a1 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/OverflowQueueWithDMStats.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/OverflowQueueWithDMStats.java
@@ -23,8 +23,9 @@
  * A LinkedBlockingQueue that supports stats. Named OverflowQueue for historical reasons.
  *
  */
-public class OverflowQueueWithDMStats extends LinkedBlockingQueue {
+public class OverflowQueueWithDMStats<E> extends LinkedBlockingQueue {
   private static final long serialVersionUID = -1846248853494394996L;
+
   protected final QueueStatHelper stats;
 
   /** Creates new OverflowQueueWithDMStats */
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractBucketRegionQueue.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractBucketRegionQueue.java
index 58bdadd6c1..5fd48f58ef 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractBucketRegionQueue.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractBucketRegionQueue.java
@@ -23,7 +23,7 @@
 
 import org.apache.geode.cache.*;
 import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.internal.cache.versions.RegionVersionVector;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
@@ -49,7 +49,7 @@
   private final long throttleTime =
       Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "GATEWAY_QUEUE_THROTTLE_TIME_MS", 100);
 
-  private final LRUStatistics stats;
+  private final EvictionStatistics stats;
 
   private final ReentrantReadWriteLock initializationLock = new ReentrantReadWriteLock();
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractDiskRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractDiskRegion.java
index bc7d2e6e65..e51af1f241 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractDiskRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractDiskRegion.java
@@ -1047,4 +1047,9 @@ public void oplogRecovered(long oplogId) {
   public String toString() {
     return getClass().getSimpleName() + ":" + getName();
   }
+
+  @Override
+  public void incRecentlyUsed() {
+    entries.incRecentlyUsed();
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractLRURegionMap.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractLRURegionMap.java
index a2f6e001d9..62158c6cc8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractLRURegionMap.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractLRURegionMap.java
@@ -27,15 +27,15 @@
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.entries.DiskEntry;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
-import org.apache.geode.internal.cache.lru.HeapLRUCapacityController;
-import org.apache.geode.internal.cache.lru.LRUAlgorithm;
-import org.apache.geode.internal.cache.lru.LRUEntry;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
-import org.apache.geode.internal.cache.lru.MemLRUCapacityController;
-import org.apache.geode.internal.cache.lru.NewLIFOClockHand;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.CachedDeserializableValueWrapper;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionList;
+import org.apache.geode.internal.cache.eviction.EvictionListBuilder;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
+import org.apache.geode.internal.cache.eviction.HeapEvictor;
+import org.apache.geode.internal.cache.eviction.HeapLRUController;
+import org.apache.geode.internal.cache.eviction.MemoryLRUController;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
 import org.apache.geode.internal.cache.versions.RegionVersionVector;
 import org.apache.geode.internal.cache.versions.VersionSource;
@@ -54,62 +54,56 @@
 public abstract class AbstractLRURegionMap extends AbstractRegionMap {
   private static final Logger logger = LogService.getLogger();
 
-  protected abstract void _setCCHelper(EnableLRU ccHelper);
+  private EvictionController evictionController;
 
-  protected abstract EnableLRU _getCCHelper();
-
-  protected abstract void _setLruList(NewLRUClockHand lruList);
+  protected AbstractLRURegionMap(InternalRegionArguments internalRegionArgs) {
+    super(internalRegionArgs);
+  }
 
-  protected abstract NewLRUClockHand _getLruList();
+  protected abstract void setEvictionList(EvictionList evictionList);
 
-  private LRUAlgorithm evictionController;
+  public abstract EvictionList getEvictionList();
 
-  protected AbstractLRURegionMap(InternalRegionArguments internalRegionArgs) {
-    super(internalRegionArgs);
+  protected EvictionController getEvictionController() {
+    return evictionController;
   }
 
   protected void initialize(Object owner, Attributes attr,
       InternalRegionArguments internalRegionArgs) {
+    super.initialize(owner, attr, internalRegionArgs, true);
+    createEvictionController(owner, internalRegionArgs);
+  }
 
-    super.initialize(owner, attr, internalRegionArgs, true/* isLRU */);
-
-    EvictionAlgorithm ea;
-    LRUAlgorithm ec;
+  private void createEvictionController(Object owner, InternalRegionArguments internalRegionArgs) {
+    EvictionAlgorithm evictionAlgorithm;
+    EvictionController evictionController;
     if (owner instanceof LocalRegion) {
-      ea = ((LocalRegion) owner).getEvictionAttributes().getAlgorithm();
-      ec = ((LocalRegion) owner).getEvictionController();
+      evictionAlgorithm = ((LocalRegion) owner).getEvictionAttributes().getAlgorithm();
+      evictionController = ((LocalRegion) owner).getEvictionController();
     } else if (owner instanceof PlaceHolderDiskRegion) {
       PlaceHolderDiskRegion phdr = (PlaceHolderDiskRegion) owner;
-      ea = phdr.getActualLruAlgorithm();
-      ec = phdr.getEvictionAttributes().createEvictionController(null, phdr.getOffHeap());
+      evictionAlgorithm = phdr.getActualLruAlgorithm();
+      evictionController =
+          phdr.getEvictionAttributes().createEvictionController(null, phdr.getOffHeap());
     } else {
       throw new IllegalStateException("expected LocalRegion or PlaceHolderDiskRegion");
     }
-    this.evictionController = ec;
 
-    if (ea.isLRUMemory()) {
-      ((MemLRUCapacityController) ec).setEntryOverHead(getEntryOverHead());
+    if (evictionAlgorithm.isLRUMemory()) {
+      ((MemoryLRUController) evictionController).setEntryOverHead(getEntryOverHead());
     }
-    if (ea.isLRUHeap()) {
-      ((HeapLRUCapacityController) ec).setEntryOverHead(getEntryOverHead());
-    }
-    _setCCHelper(getHelper(ec));
-
-    /*
-     * modification for LIFO Logic incubation
-     *
-     */
-    if (ea == EvictionAlgorithm.LIFO_ENTRY || ea == EvictionAlgorithm.LIFO_MEMORY) {
-      _setLruList(new NewLIFOClockHand(owner, _getCCHelper(), internalRegionArgs));
-    } else {
-      _setLruList(new NewLRUClockHand(owner, _getCCHelper(), internalRegionArgs));
+    if (evictionAlgorithm.isLRUHeap()) {
+      ((HeapLRUController) evictionController).setEntryOverHead(getEntryOverHead());
     }
+    this.evictionController = evictionController;
+    setEvictionList(new EvictionListBuilder(evictionAlgorithm).withArgs(internalRegionArgs)
+        .withRegion(owner).withEvictionController(getEvictionController()).create());
   }
 
   @Override
   public void changeOwner(LocalRegion r) {
     super.changeOwner(r);
-    _getLruList().setBucketRegion(r);
+    getEvictionList().setBucketRegion(r);
     this.evictionController.setBucketRegion(r);
   }
 
@@ -148,23 +142,6 @@ private void setDelta(int delta) {
     lruDelta.set(Integer.valueOf(delta));
   }
 
-
-  /**
-   * Marker class to indicate that the wrapped value is owned by a CachedDeserializable and its form
-   * is changing from serialized to deserialized.
-   */
-  public static class CDValueWrapper {
-    private final Object v;
-
-    CDValueWrapper(Object v) {
-      this.v = v;
-    }
-
-    public Object getValue() {
-      return this.v;
-    }
-  }
-
   /**
    * Used when a CachedDeserializable's value changes form. PRECONDITION: caller has le synced
    *
@@ -174,11 +151,10 @@ public Object getValue() {
    * @return true if finishExpandValue needs to be called
    * @since GemFire 6.1.2.9
    */
-  public boolean beginChangeValueForm(LRUEntry le, CachedDeserializable cd, Object v) {
+  public boolean beginChangeValueForm(EvictableEntry le, CachedDeserializable cd, Object v) {
     // make sure this cached deserializable is still in the entry
-    // @todo what if a clear is done and this entry is no longer in the region?
     {
-      if (_getCCHelper().getEvictionAlgorithm().isLRUEntry()) {
+      if (getEvictionController().getEvictionAlgorithm().isLRUEntry()) {
         // no need to worry about the value changing form with entry LRU.
         return false;
       }
@@ -194,7 +170,8 @@ public boolean beginChangeValueForm(LRUEntry le, CachedDeserializable cd, Object
       }
     }
     boolean result = false;
-    int delta = le.updateEntrySize(_getCCHelper(), new CDValueWrapper(v));
+    int delta =
+        le.updateEntrySize(getEvictionController(), new CachedDeserializableValueWrapper(v));
     if (delta != 0) {
       result = true;
       boolean disabledLURCallbacks = disableLruUpdateCallback();
@@ -208,7 +185,7 @@ public boolean beginChangeValueForm(LRUEntry le, CachedDeserializable cd, Object
       }
     }
     // fix for bug 42090
-    if (_getCCHelper().getEvictionAlgorithm().isLRUHeap() && _isOwnerALocalRegion()
+    if (getEvictionController().getEvictionAlgorithm().isLRUHeap() && _isOwnerALocalRegion()
         && _getOwner() instanceof BucketRegion
         && HeapEvictor.EVICT_HIGH_ENTRY_COUNT_BUCKETS_FIRST) {
       result = false;
@@ -259,26 +236,20 @@ public int getEntryOverHead() {
     }
   }
 
-  /** unsafe audit code. */
-  public void audit() {
-    if (logger.isTraceEnabled(LogMarker.LRU)) {
-      logger.trace(LogMarker.LRU, "Size of LRUMap = {}", sizeInVM());
-    }
-    _getLruList().audit();
-  }
-
   /**
    * Evicts the given entry from the cache. Returns the total number of bytes evicted. 1. For action
    * local destroy, returns size(key + value) 2. For action evict to disk, returns size(value)
    *
    * @return number of bytes evicted, zero if no eviction took place
    */
-  protected int evictEntry(LRUEntry entry, LRUStatistics stats) throws RegionClearedException {
-    EvictionAction action = _getCCHelper().getEvictionAction();
+  protected int evictEntry(EvictableEntry entry, EvictionStatistics stats)
+      throws RegionClearedException {
+    EvictionAction action = getEvictionController().getEvictionAction();
     LocalRegion region = _getOwner();
     if (action.isLocalDestroy()) {
       int size = entry.getEntrySize();
       if (region.evictDestroy(entry)) {
+        stats.incDestroys();
         return size;
       } else {
         return 0;
@@ -314,7 +285,8 @@ protected int evictEntry(LRUEntry entry, LRUStatistics stats) throws RegionClear
           return 0;
         }
         entry.setEvicted();
-        change = DiskEntry.Helper.overflowToDisk((DiskEntry) entry, region, _getCCHelper());
+        change =
+            DiskEntry.Helper.overflowToDisk((DiskEntry) entry, region, getEvictionController());
       }
       boolean result = change < 0;
       if (result) {
@@ -357,7 +329,7 @@ protected void changeTotalEntrySize(int delta) {
         bucketRegion.updateCounter(delta);
       }
     }
-    _getLruList().stats().updateCounter(delta);
+    getEvictionList().getStatistics().updateCounter(delta);
 
     if (delta > 0) {
       if (logger.isTraceEnabled(LogMarker.LRU)) {
@@ -366,16 +338,6 @@ protected void changeTotalEntrySize(int delta) {
     }
   }
 
-  /**
-   * access the getHelper method on the eviction controller to initialize the ccHelper field.
-   *
-   * @param ec The governing eviction controller.
-   * @return the helper instance from the eviction controller.
-   */
-  private static EnableLRU getHelper(LRUAlgorithm ec) {
-    return ec.getLRUHelper();
-  }
-
   @Override
   public void evictValue(Object key) {
     throw new IllegalStateException(
@@ -392,11 +354,11 @@ protected long getLimit() {
       BucketRegion bucketRegion = (BucketRegion) _getOwner();
       return bucketRegion.getLimit();
     }
-    return _getLruList().stats().getLimit();
+    return getEvictionController().getStatistics().getLimit();
   }
 
-  public LRUStatistics getLRUStatistics() {
-    return _getLruList().stats();
+  public EvictionStatistics getLRUStatistics() {
+    return getEvictionController().getStatistics();
   }
 
 
@@ -410,7 +372,7 @@ protected long getTotalEntrySize() {
       BucketRegion bucketRegion = (BucketRegion) _getOwner();
       return bucketRegion.getCounter();
     }
-    return _getLruList().stats().getCounter();
+    return getEvictionController().getStatistics().getCounter();
   }
 
   @Override
@@ -426,17 +388,18 @@ public void lruUpdateCallback() {
     if (isDebugEnabled_LRU && _isOwnerALocalRegion()) {
       logger.trace(LogMarker.LRU,
           "lruUpdateCallback; list size is: {}; actual size is: {}; map size is: {}; delta is: {}; limit is: {}; tombstone count={}",
-          getTotalEntrySize(), this._getLruList().getExpensiveListCount(), size(), delta,
-          getLimit(), _getOwner().getTombstoneCount());
+          getTotalEntrySize(), this.getEvictionList().size(), size(), delta, getLimit(),
+          _getOwner().getTombstoneCount());
     }
-    LRUStatistics stats = _getLruList().stats();
+    EvictionStatistics stats = getEvictionList().getStatistics();
     if (!_isOwnerALocalRegion()) {
       changeTotalEntrySize(delta);
       // instead of evicting we just quit faulting values in
-    } else if (_getCCHelper().getEvictionAlgorithm().isLRUHeap()) {
+    } else if (getEvictionController().getEvictionAlgorithm().isLRUHeap()) {
       changeTotalEntrySize(delta);
       try {
-        while (bytesToEvict > 0 && _getCCHelper().mustEvict(stats, _getOwner(), bytesToEvict)) {
+        while (bytesToEvict > 0
+            && getEvictionController().mustEvict(stats, _getOwner(), bytesToEvict)) {
           boolean evictFromThisRegion = true;
           if (HeapEvictor.EVICT_HIGH_ENTRY_COUNT_BUCKETS_FIRST
               && _getOwner() instanceof BucketRegion) {
@@ -479,7 +442,7 @@ public void lruUpdateCallback() {
             }
           }
           if (evictFromThisRegion) {
-            LRUEntry removalEntry = (LRUEntry) _getLruList().getLRUEntry();
+            EvictableEntry removalEntry = getEvictionList().getEvictableEntry();
             if (removalEntry != null) {
               int sizeOfValue = evictEntry(removalEntry, stats);
               if (sizeOfValue != 0) {
@@ -498,7 +461,6 @@ public void lruUpdateCallback() {
                 if (isDebugEnabled_LRU) {
                   logger.trace(LogMarker.LRU, "evictions={}", stats.getEvictions());
                 }
-                _getCCHelper().afterEviction();
               }
 
             } else {
@@ -512,21 +474,21 @@ public void lruUpdateCallback() {
           }
         }
       } catch (RegionClearedException e) {
-        // TODO Auto-generated catch block
         if (isDebugEnabled_LRU) {
-          logger.trace(LogMarker.LRU, "exception ={}", e.getCause());
+          logger.trace(LogMarker.LRU, "exception ={}", e.getCause().getMessage(), e.getCause());
         }
       }
     } else {
       try {
         // to fix bug 48285 do no evict if bytesToEvict <= 0.
-        while (bytesToEvict > 0 && _getCCHelper().mustEvict(stats, _getOwner(), bytesToEvict)) {
-          LRUEntry removalEntry = (LRUEntry) _getLruList().getLRUEntry();
+        while (bytesToEvict > 0
+            && getEvictionController().mustEvict(stats, _getOwner(), bytesToEvict)) {
+          EvictableEntry removalEntry = (EvictableEntry) getEvictionList().getEvictableEntry();
           if (removalEntry != null) {
             if (evictEntry(removalEntry, stats) != 0) {
               if (isDebugEnabled_LRU) {
                 logger.trace(LogMarker.LRU,
-                    "evicted entry key(2)={} total entry size is now: {} bytesToEvict :",
+                    "evicted entry key(2)={} total entry size is now: {} bytesToEvict: {}",
                     removalEntry.getKey(), getTotalEntrySize(), bytesToEvict);
               }
               stats.incEvictions();
@@ -538,7 +500,6 @@ public void lruUpdateCallback() {
               if (isDebugEnabled_LRU) {
                 logger.trace(LogMarker.LRU, "evictions={}", stats.getEvictions());
               }
-              _getCCHelper().afterEviction();
 
             }
 
@@ -553,15 +514,14 @@ public void lruUpdateCallback() {
         }
         changeTotalEntrySize(delta);
       } catch (RegionClearedException e) {
-        // TODO Auto-generated catch block
         if (isDebugEnabled_LRU) {
-          logger.debug("exception ={}", e.getCause());
+          logger.debug("exception ={}", e.getCause().getMessage(), e.getCause());
         }
       }
     }
     if (isDebugEnabled_LRU) {
       logger.trace(LogMarker.LRU, "callback complete.  LRU size is now {}",
-          _getLruList().stats().getCounter());
+          getEvictionController().getStatistics().getCounter());
     }
     // If in transaction context (either local or message)
     // reset the tx thread local
@@ -587,10 +547,10 @@ public int centralizedLruUpdateCallback() {
       logger.trace(LogMarker.LRU, "centralLruUpdateCallback: lru size is now {}, limit is: {}",
           getTotalEntrySize(), getLimit());
     }
-    LRUStatistics stats = _getLruList().stats();
+    EvictionStatistics stats = getEvictionList().getStatistics();
     try {
       while (mustEvict() && evictedBytes == 0) {
-        LRUEntry removalEntry = (LRUEntry) _getLruList().getLRUEntry();
+        EvictableEntry removalEntry = getEvictionList().getEvictableEntry();
         if (removalEntry != null) {
           evictedBytes = evictEntry(removalEntry, stats);
           if (evictedBytes != 0) {
@@ -601,7 +561,6 @@ public int centralizedLruUpdateCallback() {
             if (isDebugEnabled_LRU) {
               logger.debug("evictions={}", stats.getEvictions());
             }
-            _getCCHelper().afterEviction();
           }
         } else {
           if (getTotalEntrySize() != 0) {
@@ -612,10 +571,10 @@ public int centralizedLruUpdateCallback() {
           break;
         }
       }
-    } catch (RegionClearedException rce) {
+    } catch (RegionClearedException e) {
       // Ignore
       if (isDebugEnabled_LRU) {
-        logger.trace(LogMarker.LRU, "exception ={}", rce.getCause());
+        logger.trace(LogMarker.LRU, "exception ={}", e.getCause().getMessage(), e.getCause());
       }
     }
     if (isDebugEnabled_LRU) {
@@ -633,8 +592,6 @@ public int centralizedLruUpdateCallback() {
    *
    * @since GemFire 5.7
    */
-  // TODO this method acts as LRUupdateCallbacks
-  // do we need to put it here are insert one level up
   public void updateStats() {
     final int delta = getDelta();
     resetThreadLocals();
@@ -663,16 +620,6 @@ public void enableLruUpdateCallback() {
     setCallbackDisabled(false);
   }
 
-  // TODO rebalancing these methods are new on the
-  // rebalancing branch but never used???
-  public void disableLruUpdateCallbackForInline() {
-    setCallbackDisabled(true);
-  }
-
-  public void enableLruUpdateCallbackForInline() {
-    setCallbackDisabled(false);
-  }
-
   @Override
   public void resetThreadLocals() {
     mustRemove.set(null);
@@ -682,7 +629,7 @@ public void resetThreadLocals() {
 
   @Override
   public Set<VersionSource> clear(RegionVersionVector rvv) {
-    _getLruList().clear(rvv);
+    getEvictionList().clear(rvv);
     return super.clear(rvv);
   }
 
@@ -723,17 +670,17 @@ public void resetThreadLocals() {
    */
   @Override
   protected void lruEntryCreate(RegionEntry re) {
-    LRUEntry e = (LRUEntry) re;
+    EvictableEntry e = (EvictableEntry) re;
     // Assert.assertFalse(e._getValue() instanceof DiskEntry.RecoveredEntry)
     if (logger.isTraceEnabled(LogMarker.LRU)) {
       logger.trace(LogMarker.LRU,
           "lruEntryCreate for key={}; list size is: {}; actual size is: {}; map size is: {}; entry size: {}; in lru clock: {}",
-          re.getKey(), getTotalEntrySize(), this._getLruList().getExpensiveListCount(), size(),
-          e.getEntrySize(), !e.testEvicted());
+          re.getKey(), getTotalEntrySize(), this.getEvictionList().size(), size(), e.getEntrySize(),
+          !e.isEvicted());
     }
     // this.lruCreatedKey = re.getKey(); // [ bruce ] for DEBUGGING only
     e.unsetEvicted();
-    NewLRUClockHand lruList = _getLruList();
+    EvictionList lruList = getEvictionList();
     DiskRegion disk = _getOwner().getDiskRegion();
     boolean possibleClear = disk != null && disk.didClearCountChange();
     if (!possibleClear || this._getOwner().basicGetEntry(re.getKey()) == re) {
@@ -744,18 +691,18 @@ protected void lruEntryCreate(RegionEntry re) {
 
   @Override
   protected void lruEntryUpdate(RegionEntry re) {
-    final LRUEntry e = (LRUEntry) re;
-    setDelta(e.updateEntrySize(_getCCHelper()));
+    final EvictableEntry e = (EvictableEntry) re;
+    setDelta(e.updateEntrySize(getEvictionController()));
     if (logger.isDebugEnabled()) {
       logger.debug("lruEntryUpdate for key={} size={}", re.getKey(), e.getEntrySize());
     }
-    NewLRUClockHand lruList = _getLruList();
+    EvictionList lruList = getEvictionList();
     if (_isOwnerALocalRegion()) {
       DiskRegion disk = _getOwner().getDiskRegion();
       boolean possibleClear = disk != null && disk.didClearCountChange();
       if (!possibleClear || this._getOwner().basicGetEntry(re.getKey()) == re) {
         if (e instanceof DiskEntry) {
-          if (!e.testEvicted()) {
+          if (!e.isEvicted()) {
             lruList.appendEntry(e);
           }
         }
@@ -778,7 +725,7 @@ protected void lruEntryUpdate(RegionEntry re) {
       // We are recovering the region so it is a DiskEntry.
       // Also clear is not yet possible and this entry will be in the region.
       // No need to call resetRefCount since tx are not yet possible.
-      if (!e.testEvicted()) {
+      if (!e.isEvicted()) {
         lruList.appendEntry(e);
       }
     }
@@ -786,27 +733,28 @@ protected void lruEntryUpdate(RegionEntry re) {
 
   @Override
   protected void lruEntryDestroy(RegionEntry re) {
-    final LRUEntry e = (LRUEntry) re;
+    final EvictableEntry e = (EvictableEntry) re;
     if (logger.isTraceEnabled(LogMarker.LRU)) {
       logger.trace(LogMarker.LRU,
           "lruEntryDestroy for key={}; list size is: {}; actual size is: {}; map size is: {}; entry size: {}; in lru clock: {}",
-          re.getKey(), getTotalEntrySize(), this._getLruList().getExpensiveListCount(), size(),
-          e.getEntrySize(), !e.testEvicted());
+          re.getKey(), getTotalEntrySize(), this.getEvictionList().size(), size(), e.getEntrySize(),
+          !e.isEvicted());
     }
 
     // if (this.lruCreatedKey == re.getKey()) {
     // String method = Thread.currentThread().getStackTrace()[5].getMethodName();
     // }
     // boolean wasEvicted = e.testEvicted();
-    /* boolean removed = */_getLruList().unlinkEntry(e);
+    /* boolean removed = */
+    getEvictionList().destroyEntry(e);
     // if (removed || wasEvicted) { // evicted entries have already been removed from the list
     changeTotalEntrySize(-1 * e.getEntrySize());// subtract the size.
     Token vTok = re.getValueAsToken();
-    if (vTok == Token.DESTROYED || vTok == Token.TOMBSTONE) { // OFFHEAP noop TODO: use
-                                                              // re.isDestroyedOrTombstone
+    if (vTok == Token.DESTROYED || vTok == Token.TOMBSTONE) {
+      // OFFHEAP noop TODO: use re.isDestroyedOrTombstone
       // if in token mode we need to recalculate the size of the entry since it's
       // staying in the map and may be resurrected
-      e.updateEntrySize(_getCCHelper());
+      e.updateEntrySize(getEvictionController());
     }
     // } else if (debug) {
     // debugLogging("entry not removed from LRU list");
@@ -818,11 +766,11 @@ protected void lruEntryDestroy(RegionEntry re) {
    * Called by DiskEntry.Helper.faultInValue
    */
   @Override
-  public void lruEntryFaultIn(LRUEntry e) {
+  public void lruEntryFaultIn(EvictableEntry e) {
     if (logger.isDebugEnabled()) {
       logger.debug("lruEntryFaultIn for key={} size={}", e.getKey(), e.getEntrySize());
     }
-    NewLRUClockHand lruList = _getLruList();
+    EvictionList lruList = getEvictionList();
     if (_isOwnerALocalRegion()) {
       DiskRegion disk = _getOwner().getDiskRegion();
       boolean possibleClear = disk != null && disk.didClearCountChange();
@@ -843,23 +791,24 @@ public void decTxRefCount(RegionEntry re) {
     if (_isOwnerALocalRegion()) {
       lr = _getOwner();
     }
-    ((LRUEntry) re).decRefCount(_getLruList(), lr);
+    re.decRefCount(getEvictionList(), lr);
   }
 
   @Override
-  public boolean lruLimitExceeded(DiskRegionView drv) {
-    return _getCCHelper().lruLimitExceeded(_getLruList().stats(), drv);
+  public boolean lruLimitExceeded(DiskRegionView diskRegionView) {
+    return getEvictionController().lruLimitExceeded(getEvictionController().getStatistics(),
+        diskRegionView);
   }
 
   @Override
   public void lruCloseStats() {
-    _getLruList().closeStats();
+    getEvictionList().closeStats();
   }
 
   @Override
   boolean confirmEvictionDestroy(RegionEntry re) {
     // We assume here that a LRURegionMap contains LRUEntries
-    LRUEntry lruRe = (LRUEntry) re;
+    EvictableEntry lruRe = (EvictableEntry) re;
     if (lruRe.isInUseByTransaction() || lruRe.isDestroyed()) {
       lruRe.unsetEvicted();
       return false;
@@ -867,4 +816,14 @@ boolean confirmEvictionDestroy(RegionEntry re) {
       return true;
     }
   }
+
+  @Override
+  public long getEvictions() {
+    return this.getEvictionController().getStatistics().getEvictions();
+  }
+
+  @Override
+  public void incRecentlyUsed() {
+    getEvictionList().incrementRecentlyUsed();
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
index 3ffc7e132d..bead7ce5f5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
@@ -83,10 +83,10 @@
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.DataSerializableFixedID;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.extension.Extensible;
 import org.apache.geode.internal.cache.extension.ExtensionPoint;
 import org.apache.geode.internal.cache.extension.SimpleExtensionPoint;
-import org.apache.geode.internal.cache.lru.LRUAlgorithm;
 import org.apache.geode.internal.cache.snapshot.RegionSnapshotServiceImpl;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
@@ -127,7 +127,7 @@
 
   private volatile CacheWriter cacheWriter;
 
-  private LRUAlgorithm evictionController;
+  private EvictionController evictionController;
 
   protected int entryIdleTimeout;
 
@@ -1699,11 +1699,11 @@ public EvictionAttributesMutator getEvictionAttributesMutator() {
     return this.evictionAttributes;
   }
 
-  private void setEvictionController(LRUAlgorithm evictionController) {
+  private void setEvictionController(EvictionController evictionController) {
     this.evictionController = evictionController;
   }
 
-  public LRUAlgorithm getEvictionController() {
+  public EvictionController getEvictionController() {
     return this.evictionController;
   }
 
@@ -1759,6 +1759,7 @@ public InternalCache getCache() {
     return this.cache;
   }
 
+  @Override
   public long cacheTimeMillis() {
     return this.cache.getInternalDistributedSystem().getClock().cacheTimeMillis();
   }
@@ -1841,7 +1842,13 @@ public boolean getOffHeap() {
     return this.offHeap;
   }
 
+  @Override
   public boolean isConcurrencyChecksEnabled() {
     return this.concurrencyChecksEnabled;
   }
+
+  @Override
+  public void incRecentlyUsed() {
+    // nothing
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegionMap.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegionMap.java
index 25b0df08d1..404488b36b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegionMap.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegionMap.java
@@ -50,9 +50,9 @@
 import org.apache.geode.internal.cache.entries.AbstractRegionEntry;
 import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.internal.cache.entries.OffHeapRegionEntry;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
 import org.apache.geode.internal.cache.ha.HAContainerWrapper;
 import org.apache.geode.internal.cache.ha.HARegionQueue;
-import org.apache.geode.internal.cache.lru.LRUEntry;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
 import org.apache.geode.internal.cache.region.entry.RegionEntryFactoryBuilder;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
@@ -465,11 +465,6 @@ public void lruUpdateCallback(boolean b) {
     // By default do nothing; LRU maps needs to override this method
   }
 
-  @Override
-  public void lruUpdateCallback(int i) {
-    // By default do nothing; LRU maps needs to override this method
-  }
-
   @Override
   public boolean disableLruUpdateCallback() {
     // By default do nothing; LRU maps needs to override this method
@@ -517,7 +512,7 @@ public void decTxRefCount(RegionEntry e) {
   }
 
   @Override
-  public boolean lruLimitExceeded(DiskRegionView drv) {
+  public boolean lruLimitExceeded(DiskRegionView diskRegionView) {
     return false;
   }
 
@@ -527,7 +522,7 @@ public void lruCloseStats() {
   }
 
   @Override
-  public void lruEntryFaultIn(LRUEntry entry) {
+  public void lruEntryFaultIn(EvictableEntry entry) {
     // do nothing by default
   }
 
@@ -3921,6 +3916,16 @@ public boolean verifyTombstoneCount(AtomicInteger numTombstones) {
     return true;
   }
 
+  @Override
+  public long getEvictions() {
+    return 0;
+  }
+
+  @Override
+  public void incRecentlyUsed() {
+    // nothing by default
+  }
+
   public interface ARMLockTestHook {
     public void beforeBulkLock(LocalRegion region);
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionEvictior.java b/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionEvictior.java
deleted file mode 100755
index f74d59606d..0000000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionEvictior.java
+++ /dev/null
@@ -1,60 +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.geode.internal.cache;
-
-import java.util.concurrent.Callable;
-
-import org.apache.geode.cache.Region;
-
-/**
- *
- * Takes delta to be evicted and tries to evict the least no of LRU entry which would make
- * evictedBytes more than or equal to the delta
- *
- * @since GemFire 6.0
- *
- */
-public class BucketRegionEvictior implements Callable<Object> {
-  private LocalRegion region;
-
-  private int bytesToEvict;
-
-  public BucketRegionEvictior(LocalRegion region, int bytesToEvict) {
-    this.bytesToEvict = bytesToEvict;
-    this.region = region;
-  }
-
-  public Region getRegion() {
-    return this.region;
-  }
-
-  public int getDelta() {
-    return this.bytesToEvict;
-  }
-
-  public void setRegion(Region reg) {
-    this.region = (LocalRegion) reg;
-  }
-
-  public void setDelta(int bytes) {
-    this.bytesToEvict = bytes;
-  }
-
-  public Object call() throws Exception {
-    ((AbstractLRURegionMap) region.entries).lruUpdateCallback(bytesToEvict);
-    return null;
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/CachedDeserializable.java b/geode-core/src/main/java/org/apache/geode/internal/cache/CachedDeserializable.java
index 8027ddf349..066c5b8a61 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/CachedDeserializable.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/CachedDeserializable.java
@@ -20,8 +20,7 @@
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.DataSerializableFixedID;
-import org.apache.geode.internal.cache.lru.Sizeable;
+import org.apache.geode.internal.size.Sizeable;
 import org.apache.geode.internal.util.BlobHelper;
 
 /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/CachedDeserializableFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/CachedDeserializableFactory.java
index bf5b784fcf..2ceb616adb 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/CachedDeserializableFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/CachedDeserializableFactory.java
@@ -23,8 +23,8 @@
 import org.apache.geode.internal.DSCODE;
 import org.apache.geode.internal.HeapDataOutputStream;
 import org.apache.geode.internal.NullDataOutputStream;
-import org.apache.geode.internal.cache.lru.Sizeable;
 import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.size.Sizeable;
 import org.apache.geode.pdx.PdxInstance;
 
 /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
index 3bf13ea784..e3701e0a1c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
@@ -14,7 +14,9 @@
  */
 package org.apache.geode.internal.cache;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.distributed.ConfigurationProperties.CACHE_XML_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.apache.geode.internal.cache.entries.DiskEntry.Helper.readRawValue;
 
 import java.io.File;
@@ -85,8 +87,8 @@
 import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.internal.cache.entries.DiskEntry.Helper.ValueWrapper;
 import org.apache.geode.internal.cache.entries.DiskEntry.RecoveredEntry;
-import org.apache.geode.internal.cache.lru.LRUAlgorithm;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.internal.cache.persistence.BytesAndBits;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
@@ -3915,7 +3917,8 @@ private void offlineCompact() {
     }
   }
 
-  private final HashMap<String, LRUStatistics> prlruStatMap = new HashMap<String, LRUStatistics>();
+  private final HashMap<String, EvictionStatistics> prlruStatMap =
+      new HashMap<String, EvictionStatistics>();
 
   /**
    * Lock used to synchronize access to the init file. This is a lock rather than a synchronized
@@ -3927,16 +3930,16 @@ public BackupLock getBackupLock() {
     return backupLock;
   }
 
-  LRUStatistics getOrCreatePRLRUStats(PlaceHolderDiskRegion dr) {
+  EvictionStatistics getOrCreatePRLRUStats(PlaceHolderDiskRegion dr) {
     String prName = dr.getPrName();
-    LRUStatistics result = null;
+    EvictionStatistics result = null;
     synchronized (this.prlruStatMap) {
       result = this.prlruStatMap.get(prName);
       if (result == null) {
         EvictionAttributesImpl ea = dr.getEvictionAttributes();
-        LRUAlgorithm ec = ea.createEvictionController(null, dr.getOffHeap());
+        EvictionController ec = ea.createEvictionController(null, dr.getOffHeap());
         StatisticsFactory sf = cache.getDistributedSystem();
-        result = ec.getLRUHelper().initStats(dr, sf);
+        result = ec.initStats(dr, sf);
         this.prlruStatMap.put(prName, result);
       }
     }
@@ -3944,12 +3947,12 @@ LRUStatistics getOrCreatePRLRUStats(PlaceHolderDiskRegion dr) {
   }
 
   /**
-   * If we have recovered a bucket earlier for the given pr then we will have an LRUStatistics to
-   * return for it. Otherwise return null.
+   * If we have recovered a bucket earlier for the given pr then we will have an EvictionStatistics
+   * to return for it. Otherwise return null.
    */
-  LRUStatistics getPRLRUStats(PartitionedRegion pr) {
+  EvictionStatistics getPRLRUStats(PartitionedRegion pr) {
     String prName = pr.getFullPath();
-    LRUStatistics result = null;
+    EvictionStatistics result = null;
     synchronized (this.prlruStatMap) {
       result = this.prlruStatMap.get(prName);
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
index 0d64f36326..8ead35d4cc 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
@@ -48,7 +48,6 @@
 import org.apache.geode.cache.CacheWriterException;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.DiskAccessException;
-import org.apache.geode.cache.EntryExistsException;
 import org.apache.geode.cache.EntryNotFoundException;
 import org.apache.geode.cache.LossAction;
 import org.apache.geode.cache.MembershipAttributes;
@@ -95,6 +94,7 @@
 import org.apache.geode.internal.cache.control.MemoryEvent;
 import org.apache.geode.internal.cache.event.DistributedEventTracker;
 import org.apache.geode.internal.cache.event.EventTracker;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
 import org.apache.geode.internal.cache.execute.DistributedRegionFunctionExecutor;
 import org.apache.geode.internal.cache.execute.DistributedRegionFunctionResultSender;
 import org.apache.geode.internal.cache.execute.DistributedRegionFunctionResultWaiter;
@@ -102,7 +102,6 @@
 import org.apache.geode.internal.cache.execute.LocalResultCollector;
 import org.apache.geode.internal.cache.execute.RegionFunctionContextImpl;
 import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
-import org.apache.geode.internal.cache.lru.LRUEntry;
 import org.apache.geode.internal.cache.partitioned.RemoteSizeMessage;
 import org.apache.geode.internal.cache.persistence.CreatePersistentRegionProcessor;
 import org.apache.geode.internal.cache.persistence.PersistenceAdvisor;
@@ -1645,7 +1644,7 @@ void distributeDestroy(EntryEventImpl event, Object expectedOldValue) {
   }
 
   @Override
-  boolean evictDestroy(LRUEntry entry) {
+  boolean evictDestroy(EvictableEntry entry) {
     boolean evictDestroyWasDone = super.evictDestroy(entry);
     if (evictDestroyWasDone) {
       if (this.scope.isGlobal()) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java
index 0ea0c87a74..bac23739a5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java
@@ -61,7 +61,6 @@
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.FilterRoutingInfo.FilterInfo;
 import org.apache.geode.internal.cache.entries.OffHeapRegionEntry;
-import org.apache.geode.internal.cache.lru.Sizeable;
 import org.apache.geode.internal.cache.partitioned.PartitionMessage;
 import org.apache.geode.internal.cache.partitioned.PutMessage;
 import org.apache.geode.internal.cache.tier.sockets.CacheServerHelper;
@@ -82,6 +81,7 @@
 import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Retained;
 import org.apache.geode.internal.offheap.annotations.Unretained;
+import org.apache.geode.internal.size.Sizeable;
 import org.apache.geode.internal.util.ArrayUtils;
 import org.apache.geode.internal.util.BlobHelper;
 import org.apache.geode.pdx.internal.PeerTypeRegistration;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/EvictionAttributesImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/EvictionAttributesImpl.java
index 4ace4f6a77..267be481e9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/EvictionAttributesImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/EvictionAttributesImpl.java
@@ -26,10 +26,10 @@
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.util.ObjectSizer;
 import org.apache.geode.internal.InternalDataSerializer;
-import org.apache.geode.internal.cache.lru.HeapLRUCapacityController;
-import org.apache.geode.internal.cache.lru.LRUAlgorithm;
-import org.apache.geode.internal.cache.lru.LRUCapacityController;
-import org.apache.geode.internal.cache.lru.MemLRUCapacityController;
+import org.apache.geode.internal.cache.eviction.CountLRUEviction;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.HeapLRUController;
+import org.apache.geode.internal.cache.eviction.MemoryLRUController;
 
 /**
  * Defines the attributes for configuring the eviction controller associated with a
@@ -51,7 +51,8 @@
     implements EvictionAttributesMutator {
   private static final long serialVersionUID = -6404395520499379715L;
 
-  private EvictionAlgorithm algorithm = EvictionAlgorithm.NONE;
+  private org.apache.geode.cache.EvictionAlgorithm algorithm =
+      org.apache.geode.cache.EvictionAlgorithm.NONE;
 
   private ObjectSizer sizer;
 
@@ -63,7 +64,7 @@
    * The Eviction Controller instance generated as a result of processing this instance Typically
    * used for any mutation operations
    */
-  private volatile LRUAlgorithm evictionController;
+  private volatile EvictionController evictionController;
 
   public EvictionAttributesImpl() {}
 
@@ -82,7 +83,7 @@ public EvictionAttributesImpl(EvictionAttributesImpl other) {
     // this.evictionController = null;
   }
 
-  public EvictionAttributesImpl setAlgorithm(EvictionAlgorithm algorithm) {
+  public EvictionAttributesImpl setAlgorithm(org.apache.geode.cache.EvictionAlgorithm algorithm) {
     this.algorithm = algorithm;
     return this;
   }
@@ -108,10 +109,11 @@ public ObjectSizer getObjectSizer() {
    * @see org.apache.geode.cache.EvictionAttributes#getAlgorithm()
    */
   @Override
-  public EvictionAlgorithm getAlgorithm() {
+  public org.apache.geode.cache.EvictionAlgorithm getAlgorithm() {
     return this.algorithm;
   }
 
+  @Override
   public void setMaximum(int maximum) {
     this.maximum = maximum;
     if (this.evictionController != null) {
@@ -120,7 +122,7 @@ public void setMaximum(int maximum) {
   }
 
   /**
-   * @param maximum parameter for the given {@link EvictionAlgorithm}
+   * @param maximum parameter for the given {@link org.apache.geode.cache.EvictionAlgorithm}
    * @return the instance of {@link EvictionAttributesImpl} on which this method was called
    */
   public EvictionAttributesImpl internalSetMaximum(int maximum) {
@@ -143,7 +145,7 @@ public int getMaximum() {
 
   /**
    * Sets the {@link EvictionAction} on the {@link EvictionAttributesImpl} that the given
-   * {@link EvictionAlgorithm} uses to perform the eviction.
+   * {@link org.apache.geode.cache.EvictionAlgorithm} uses to perform the eviction.
    *
    * @param action the {@link EvictionAction} used by the {@link EvictionAction}
    * @return the instance of {@link EvictionAttributesMutator} on which this method was called
@@ -174,19 +176,19 @@ public EvictionAction getAction() {
    *
    * @see EvictionAttributes
    */
-  public LRUAlgorithm createEvictionController(Region region, boolean isOffHeap) {
+  public EvictionController createEvictionController(Region region, boolean isOffHeap) {
     if (this.algorithm == EvictionAlgorithm.LRU_ENTRY) {
-      this.evictionController = new LRUCapacityController(this.maximum, this.action, region);
+      this.evictionController = new CountLRUEviction(this.maximum, this.action, region);
     } else if (this.algorithm == EvictionAlgorithm.LRU_HEAP) {
-      this.evictionController = new HeapLRUCapacityController(this.sizer, this.action, region);
+      this.evictionController = new HeapLRUController(this.action, region, this.sizer);
     } else if (this.algorithm == EvictionAlgorithm.LRU_MEMORY) {
       this.evictionController =
-          new MemLRUCapacityController(this.maximum, this.sizer, this.action, region, isOffHeap);
+          new MemoryLRUController(this.maximum, this.sizer, this.action, region, isOffHeap);
     } else if (this.algorithm == EvictionAlgorithm.LIFO_ENTRY) {
-      this.evictionController = new LRUCapacityController(this.maximum, this.action, region);
+      this.evictionController = new CountLRUEviction(this.maximum, this.action, region);
     } else if (this.algorithm == EvictionAlgorithm.LIFO_MEMORY) {
       this.evictionController =
-          new MemLRUCapacityController(this.maximum, this.sizer, this.action, region, isOffHeap);
+          new MemoryLRUController(this.maximum, this.sizer, this.action, region, isOffHeap);
     } else {
       // for all other algorithms, return null
       this.evictionController = null;
@@ -194,16 +196,18 @@ public LRUAlgorithm createEvictionController(Region region, boolean isOffHeap) {
     return this.evictionController;
   }
 
+  @Override
   public void toData(DataOutput out) throws IOException {
     out.writeInt(this.maximum);
     DataSerializer.writeObject(this.action, out);
     DataSerializer.writeObject(this.algorithm, out);
   }
 
+  @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
     this.maximum = in.readInt();
-    this.action = (EvictionAction) DataSerializer.readObject(in);
-    this.algorithm = (EvictionAlgorithm) DataSerializer.readObject(in);
+    this.action = DataSerializer.readObject(in);
+    this.algorithm = DataSerializer.readObject(in);
   }
 
   public static EvictionAttributesImpl createFromData(DataInput in)
@@ -223,11 +227,11 @@ public boolean isLIFO() {
   }
 
   public boolean isLIFOEntry() {
-    return this.algorithm == EvictionAlgorithm.LIFO_ENTRY;
+    return this.algorithm == org.apache.geode.cache.EvictionAlgorithm.LIFO_ENTRY;
   }
 
   public boolean isLIFOMemory() {
-    return this.algorithm == EvictionAlgorithm.LIFO_MEMORY;
+    return this.algorithm == org.apache.geode.cache.EvictionAlgorithm.LIFO_MEMORY;
   }
 
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index fd8625c59d..93241c0318 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -177,14 +177,14 @@
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
 import org.apache.geode.internal.cache.control.ResourceAdvisor;
 import org.apache.geode.internal.cache.event.EventTrackerExpiryTask;
+import org.apache.geode.internal.cache.eviction.HeapEvictor;
+import org.apache.geode.internal.cache.eviction.OffHeapEvictor;
 import org.apache.geode.internal.cache.execute.util.FindRestEnabledServersFunction;
 import org.apache.geode.internal.cache.extension.Extensible;
 import org.apache.geode.internal.cache.extension.ExtensionPoint;
 import org.apache.geode.internal.cache.extension.SimpleExtensionPoint;
 import org.apache.geode.internal.cache.ha.HARegionQueue;
 import org.apache.geode.internal.cache.locks.TXLockService;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
-import org.apache.geode.internal.cache.lru.OffHeapEvictor;
 import org.apache.geode.internal.cache.partitioned.RedundancyAlreadyMetException;
 import org.apache.geode.internal.cache.persistence.PersistentMemberID;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/HasDiskRegion.java~d2263ebc2... Create HasDiskRegion interface b/geode-core/src/main/java/org/apache/geode/internal/cache/HasDiskRegion.java~d2263ebc2... Create HasDiskRegion interface
new file mode 100644
index 0000000000..dbf55db74d
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/HasDiskRegion.java~d2263ebc2... Create HasDiskRegion interface	
@@ -0,0 +1,22 @@
+/*
+ * 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.geode.internal.cache;
+
+public interface HasDiskRegion {
+
+  DiskRegion getDiskRegion();
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
index 43a59bd4f2..a4be9354c1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
@@ -14,11 +14,14 @@
  */
 package org.apache.geode.internal.cache;
 
+import java.io.IOException;
+
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.cache.CacheWriterException;
 import org.apache.geode.cache.EntryNotFoundException;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.RegionExistsException;
 import org.apache.geode.cache.TimeoutException;
 import org.apache.geode.cache.client.internal.ServerRegionProxy;
 import org.apache.geode.cache.query.internal.index.IndexManager;
@@ -115,6 +118,12 @@ void invokeDestroyCallbacks(final EnumListenerEvent eventType, final EntryEventI
   void invokeInvalidateCallbacks(final EnumListenerEvent eventType, final EntryEventImpl event,
       final boolean callDispatchListenerEvent);
 
+  long getEvictions();
+
+  Region createSubregion(String subregionName, RegionAttributes attrs,
+      InternalRegionArguments internalRegionArgs)
+      throws RegionExistsException, TimeoutException, IOException, ClassNotFoundException;
+
   void addCacheServiceProfile(CacheServiceProfile profile);
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index 3687d3f669..0d87b1d22e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -173,13 +173,13 @@
 import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.internal.cache.event.EventTracker;
 import org.apache.geode.internal.cache.event.NonDistributedEventTracker;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
 import org.apache.geode.internal.cache.execute.DistributedRegionFunctionExecutor;
 import org.apache.geode.internal.cache.execute.DistributedRegionFunctionResultSender;
 import org.apache.geode.internal.cache.execute.LocalResultCollector;
 import org.apache.geode.internal.cache.execute.RegionFunctionContextImpl;
 import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
 import org.apache.geode.internal.cache.ha.ThreadIdentifier;
-import org.apache.geode.internal.cache.lru.LRUEntry;
 import org.apache.geode.internal.cache.partitioned.Bucket;
 import org.apache.geode.internal.cache.partitioned.RedundancyAlreadyMetException;
 import org.apache.geode.internal.cache.persistence.DiskExceptionHandler;
@@ -231,7 +231,7 @@
  * distribution behavior.
  */
 @SuppressWarnings("deprecation")
-public class LocalRegion extends AbstractRegion implements InternalRegion, LoaderHelperFactory,
+public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
     ResourceListener<MemoryEvent>, DiskExceptionHandler, DiskRecoveryStore {
 
   // package-private to avoid synthetic accessor
@@ -854,6 +854,7 @@ public VersionSource getVersionMember() {
   }
 
   // TODO: createSubregion method is too complex for IDE to analyze
+  @Override
   public Region createSubregion(String subregionName, RegionAttributes attrs,
       InternalRegionArguments internalRegionArgs)
       throws RegionExistsException, TimeoutException, IOException, ClassNotFoundException {
@@ -6598,7 +6599,7 @@ protected EntryEventImpl generateEvictDestroyEvent(final Object key) {
   /**
    * @return true if the evict destroy was done; false if it was not needed
    */
-  boolean evictDestroy(LRUEntry entry) {
+  boolean evictDestroy(EvictableEntry entry) {
     checkReadiness();
 
     @Released
@@ -7098,7 +7099,7 @@ public long updateStatsForPut(RegionEntry entry, long lastModified, boolean lruR
     long lastAccessed = cacheTimeMillis();
     if (lruRecentUse) {
       // fix for bug 31102
-      entry.setRecentlyUsed();
+      entry.setRecentlyUsed(this);
     }
     if (lastModified == 0L) {
       lastModified = lastAccessed;
@@ -8272,9 +8273,9 @@ TXEntryState createReadEntry(TXRegionState txRegionState, KeyInfo keyInfo,
       try {
         synchronized (regionEntry) {
           if (!regionEntry.isRemoved()) {
-            if (regionEntry instanceof DiskEntry && regionEntry instanceof LRUEntry) {
-              LRUEntry le = (LRUEntry) regionEntry;
-              if (le.testEvicted()) {
+            if (regionEntry instanceof DiskEntry && regionEntry instanceof EvictableEntry) {
+              EvictableEntry le = (EvictableEntry) regionEntry;
+              if (le.isEvicted()) {
                 // Handle the case where we fault in a disk entry
                 txLRUStart();
                 needsLRUCleanup = true;
@@ -8408,9 +8409,11 @@ List debugGetSubregionNames() {
     return names;
   }
 
-  /*****************************************************************************
-   * INNER CLASSES
-   ****************************************************************************/
+  @Override
+  public void incRecentlyUsed() {
+    // nothing
+    this.entries.incRecentlyUsed();
+  }
 
   // package-private to avoid synthetic accessor
   static void dispatchEvent(LocalRegion region, InternalCacheEvent event,
@@ -8466,6 +8469,10 @@ static void dispatchEvent(LocalRegion region, InternalCacheEvent event,
     }
   }
 
+  /*****************************************************************************
+   * INNER CLASSES
+   ****************************************************************************/
+
   class EventDispatcher implements Runnable {
     /**
      * released by the release method
@@ -10064,7 +10071,7 @@ private void basicEntryRemoveAll(Object key, DistributedRemoveAllOperation op, i
       discoverJTA();
 
       /*
-       * If this is tx, do removeEntry, unless it is a local region?
+       * If this is tx, do destroyEntry, unless it is a local region?
        */
       try {
         performRemoveAllEntry(event);
@@ -12074,4 +12081,12 @@ public long getLatestLastAccessTimeFromOthers(Object key) {
     return 0L;
   }
 
+  /**
+   * Returns the number of LRU evictions done by this region.
+   */
+  @Override
+  public long getEvictions() {
+    return this.entries.getEvictions();
+  }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/NonLocalRegionEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/NonLocalRegionEntry.java
index 32c06c77d8..6e0dc067ac 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/NonLocalRegionEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/NonLocalRegionEntry.java
@@ -31,18 +31,8 @@
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.ByteArrayDataInput;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.cache.CachedDeserializable;
-import org.apache.geode.internal.cache.CachedDeserializableFactory;
-import org.apache.geode.internal.cache.DistributedRegion;
-import org.apache.geode.internal.cache.EntryEventImpl;
-import org.apache.geode.internal.cache.EntrySnapshot;
 import org.apache.geode.internal.cache.InitialImageOperation.Entry;
-import org.apache.geode.internal.cache.InternalRegion;
-import org.apache.geode.internal.cache.RegionClearedException;
-import org.apache.geode.internal.cache.RegionEntry;
-import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionList;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
 import org.apache.geode.internal.cache.versions.VersionTag;
@@ -237,7 +227,7 @@ public Object getValueRetain(RegionEntryContext context) {
   }
 
   /** update the value held in this non-local region entry */
-  public void setCachedValue(Object newValue) {
+  void setCachedValue(Object newValue) {
     this.value = newValue;
   }
 
@@ -250,7 +240,7 @@ public void updateStatsForPut(long lastModifiedTime, long lastAccessedTime) {
   }
 
   @Override
-  public void setRecentlyUsed() {
+  public void setRecentlyUsed(RegionEntryContext context) {
     throw new UnsupportedOperationException(
         LocalizedStrings.PartitionedRegion_NOT_APPROPRIATE_FOR_PARTITIONEDREGIONNONLOCALREGIONENTRY
             .toLocalizedString());
@@ -558,8 +548,7 @@ public boolean isDestroyedOrRemovedButNotTombstone() {
 
   @Override
   public void returnToPool() {
-    // TODO Auto-generated method stub
-
+    // nothing
   }
 
   @Override
@@ -572,14 +561,12 @@ public void setValueWithTombstoneCheck(Object value, EntryEvent event)
 
   @Override
   public boolean isCacheListenerInvocationInProgress() {
-    // TODO Auto-generated method stub
     return false;
   }
 
   @Override
   public void setCacheListenerInvocationInProgress(boolean isListenerInvoked) {
-    // TODO Auto-generated method stub
-
+    // nothing
   }
 
   @Override
@@ -595,23 +582,18 @@ public boolean isInUseByTransaction() {
     return false;
   }
 
-  @Override
-  public void setInUseByTransaction(boolean inUseByTransaction) {
-    // nothing
-  }
-
   @Override
   public void incRefCount() {
     // nothing
   }
 
   @Override
-  public void decRefCount(NewLRUClockHand lruList, InternalRegion region) {
+  public void decRefCount(EvictionList lruList, InternalRegion region) {
     // nothing
   }
 
   @Override
-  public void resetRefCount(NewLRUClockHand lruList) {
+  public void resetRefCount(EvictionList lruList) {
     // nothing
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java b/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
index 9b4ed10237..9bcffa869b 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
@@ -85,8 +85,8 @@
 import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.internal.cache.entries.DiskEntry.Helper.Flushable;
 import org.apache.geode.internal.cache.entries.DiskEntry.Helper.ValueWrapper;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionList;
 import org.apache.geode.internal.cache.persistence.BytesAndBits;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
@@ -7074,7 +7074,7 @@ public boolean isRemovedFromDisk() {
     }
 
     @Override
-    public int updateAsyncEntrySize(EnableLRU capacityController) {
+    public int updateAsyncEntrySize(EvictionController capacityController) {
       throw new IllegalStateException();
     }
 
@@ -7150,7 +7150,7 @@ public boolean dispatchListenerEvents(EntryEventImpl event) throws InterruptedEx
     }
 
     @Override
-    public void setRecentlyUsed() {
+    public void setRecentlyUsed(RegionEntryContext context) {
       // nothing
     }
 
@@ -7362,23 +7362,18 @@ public boolean isInUseByTransaction() {
       return false;
     }
 
-    @Override
-    public void setInUseByTransaction(boolean inUseByTransaction) {
-      // nothing
-    }
-
     @Override
     public void incRefCount() {
       // nothing
     }
 
     @Override
-    public void decRefCount(NewLRUClockHand lruList, InternalRegion region) {
+    public void decRefCount(EvictionList lruList, InternalRegion region) {
       // nothing
     }
 
     @Override
-    public void resetRefCount(NewLRUClockHand lruList) {
+    public void resetRefCount(EvictionList lruList) {
       // nothing
     }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
index 6e0c0cd2b5..a0b003cb8c 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
@@ -166,6 +166,8 @@
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
 import org.apache.geode.internal.cache.control.MemoryEvent;
 import org.apache.geode.internal.cache.control.MemoryThresholds;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
+import org.apache.geode.internal.cache.eviction.HeapEvictor;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
 import org.apache.geode.internal.cache.execute.FunctionExecutionNodePruner;
 import org.apache.geode.internal.cache.execute.FunctionRemoteContext;
@@ -177,9 +179,6 @@
 import org.apache.geode.internal.cache.execute.RegionFunctionContextImpl;
 import org.apache.geode.internal.cache.execute.ServerToClientFunctionResultSender;
 import org.apache.geode.internal.cache.ha.ThreadIdentifier;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
-import org.apache.geode.internal.cache.lru.Sizeable;
 import org.apache.geode.internal.cache.partitioned.ContainsKeyValueMessage;
 import org.apache.geode.internal.cache.partitioned.ContainsKeyValueMessage.ContainsKeyValueResponse;
 import org.apache.geode.internal.cache.partitioned.DestroyMessage;
@@ -249,6 +248,7 @@
 import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Unretained;
 import org.apache.geode.internal.sequencelog.RegionLogger;
+import org.apache.geode.internal.size.Sizeable;
 import org.apache.geode.internal.util.TransformUtils;
 import org.apache.geode.internal.util.concurrent.StoppableCountDownLatch;
 
@@ -510,11 +510,11 @@ public byte getMetadataVersion() {
   }
 
   /**
-   * Returns the LRUStatistics for this PR. This is needed to find the single instance of
-   * LRUStatistics created early for a PR when it is recovered from disk. This fixes bug 41938
+   * Returns the EvictionStatistics for this PR. This is needed to find the single instance of
+   * EvictionStatistics created early for a PR when it is recovered from disk. This fixes bug 41938
    */
-  public LRUStatistics getPRLRUStatsDuringInitialization() {
-    LRUStatistics result = null;
+  public EvictionStatistics getPRLRUStatsDuringInitialization() {
+    EvictionStatistics result = null;
     if (getDiskStore() != null) {
       result = getDiskStore().getPRLRUStats(this);
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PlaceHolderDiskRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PlaceHolderDiskRegion.java
index 2f1db58a34..fcb9e69855 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PlaceHolderDiskRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PlaceHolderDiskRegion.java
@@ -18,7 +18,7 @@
 import org.apache.geode.InternalGemFireError;
 import org.apache.geode.cache.DiskAccessException;
 import org.apache.geode.internal.cache.entries.DiskEntry;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
 import org.apache.geode.internal.cache.persistence.DiskStoreID;
@@ -53,9 +53,9 @@
     this.name = drv.getName();
   }
 
-  LRUStatistics prlruStats;
+  EvictionStatistics prlruStats;
 
-  public LRUStatistics getPRLRUStats() {
+  public EvictionStatistics getPRLRUStats() {
     return this.prlruStats;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ProxyRegionMap.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ProxyRegionMap.java
index ec6441106d..d8270ab563 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ProxyRegionMap.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ProxyRegionMap.java
@@ -36,8 +36,8 @@
 import org.apache.geode.internal.cache.AbstractRegionMap.ARMLockTestHook;
 import org.apache.geode.internal.cache.InitialImageOperation.Entry;
 import org.apache.geode.internal.cache.entries.DiskEntry;
-import org.apache.geode.internal.cache.lru.LRUEntry;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
+import org.apache.geode.internal.cache.eviction.EvictionList;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.versions.RegionVersionVector;
@@ -394,7 +394,7 @@ public void decTxRefCount(RegionEntry e) {
   }
 
   @Override
-  public boolean lruLimitExceeded(DiskRegionView drv) {
+  public boolean lruLimitExceeded(DiskRegionView diskRegionView) {
     return false;
   }
 
@@ -484,7 +484,7 @@ public void updateStatsForPut(long lastModifiedTime, long lastAccessedTime) {
     }
 
     @Override
-    public void setRecentlyUsed() {
+    public void setRecentlyUsed(RegionEntryContext context) {
       // do nothing; called by LocalRegion.updateStatsForPut
     }
 
@@ -805,20 +805,17 @@ public boolean isInUseByTransaction() {
     }
 
     @Override
-    public void setInUseByTransaction(boolean inUseByTransaction) {
+    public void incRefCount() {
       // nothing
     }
 
     @Override
-    public void incRefCount() {}
-
-    @Override
-    public void decRefCount(NewLRUClockHand lruList, InternalRegion region) {
+    public void decRefCount(EvictionList lruList, InternalRegion region) {
       // nothing
     }
 
     @Override
-    public void resetRefCount(NewLRUClockHand lruList) {
+    public void resetRefCount(EvictionList lruList) {
       // nothing
     }
 
@@ -830,12 +827,7 @@ public Object prepareValueForCache(RegionEntryContext context, Object value,
   }
 
   @Override
-  public void lruUpdateCallback(int n) {
-    // do nothing
-  }
-
-  @Override
-  public void lruEntryFaultIn(LRUEntry entry) {
+  public void lruEntryFaultIn(EvictableEntry entry) {
     // do nothing.
 
   }
@@ -882,10 +874,22 @@ public int sizeInVM() {
   }
 
   @Override
-  public void close() {}
+  public void close() {
+    // nothing
+  }
 
   @Override
   public ARMLockTestHook getARMLockTestHook() {
     return null;
   }
+
+  @Override
+  public long getEvictions() {
+    return 0;
+  }
+
+  @Override
+  public void incRecentlyUsed() {
+    // nothing
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEntry.java
index 7834da9f3d..44ee787e1b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEntry.java
@@ -26,7 +26,7 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.InitialImageOperation.Entry;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionList;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
 import org.apache.geode.internal.cache.versions.VersionTag;
@@ -105,7 +105,7 @@ VersionTag generateVersionTag(VersionSource member, boolean withDelta, InternalR
   /**
    * Used to mark an LRU entry as having been recently used.
    */
-  void setRecentlyUsed();
+  void setRecentlyUsed(RegionEntryContext context); // same method as on EvictionNode interface
 
   void updateStatsForGet(boolean hit, long time);
 
@@ -437,8 +437,6 @@ boolean destroy(InternalRegion region, EntryEventImpl event, boolean inTokenMode
 
   boolean isInUseByTransaction();
 
-  void setInUseByTransaction(final boolean inUseByTransaction);
-
   /**
    * Increments the number of transactions that are currently referencing this node.
    */
@@ -449,13 +447,13 @@ boolean destroy(InternalRegion region, EntryEventImpl event, boolean inTokenMode
    *
    * @param region the local region that owns this region entry; null if no local region owner
    */
-  void decRefCount(NewLRUClockHand lruList, InternalRegion region);
+  void decRefCount(EvictionList lruList, InternalRegion region);
 
   /**
    * Clear the number of transactions that are currently referencing this node and returns to LRU
    * list
    */
-  void resetRefCount(NewLRUClockHand lruList);
+  void resetRefCount(EvictionList lruList);
 
   @Retained(ABSTRACT_REGION_ENTRY_PREPARE_VALUE_FOR_CACHE)
   Object prepareValueForCache(RegionEntryContext context, Object value, boolean isEntryUpdate);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEntryContext.java b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEntryContext.java
index 1e1aa072cb..2b2d7f91a2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEntryContext.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEntryContext.java
@@ -23,18 +23,20 @@
  * @since GemFire 8.0
  */
 public interface RegionEntryContext extends HasCachePerfStats {
-  public static final String DEFAULT_COMPRESSION_PROVIDER =
-      "org.apache.geode.compression.SnappyCompressor";
+
+  String DEFAULT_COMPRESSION_PROVIDER = "org.apache.geode.compression.SnappyCompressor";
 
   /**
    * Returns the compressor to be used by this region entry when storing the entry value.
    *
    * @return null if no compressor is assigned or available for the entry.
    */
-  public Compressor getCompressor();
+  Compressor getCompressor();
 
   /**
    * Returns true if region entries are stored off heap.
    */
-  public boolean getOffHeap();
+  boolean getOffHeap();
+
+  void incRecentlyUsed();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEvictorTask.java b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEvictorTask.java
deleted file mode 100755
index 72eb05b55d..0000000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionEvictorTask.java
+++ /dev/null
@@ -1,141 +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.geode.internal.cache;
-
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.Callable;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.log4j.LocalizedMessage;
-
-/**
- * Takes delta to be evicted and tries to evict the least no of LRU entry which would make
- * evictedBytes more than or equal to the delta
- *
- * @since GemFire 6.0
- */
-public class RegionEvictorTask implements Runnable {
-
-  private static final Logger logger = LogService.getLogger();
-
-  private static final int EVICTION_BURST_PAUSE_TIME_MILLIS;
-
-  public static int TEST_EVICTION_BURST_PAUSE_TIME_MILLIS = Integer.MAX_VALUE;
-
-  static {
-    EVICTION_BURST_PAUSE_TIME_MILLIS = Integer
-        .getInteger(DistributionConfig.GEMFIRE_PREFIX + "evictionBurstPauseTimeMillis", 1000);
-  }
-
-  private static volatile long lastTaskCompletionTime = 0;
-
-  public static void setLastTaskCompletionTime(long v) {
-    lastTaskCompletionTime = v;
-  }
-
-  public static long getLastTaskCompletionTime() {
-    return lastTaskCompletionTime;
-  }
-
-  private List<LocalRegion> regionSet;
-
-  private final HeapEvictor evictor;
-
-  private final long bytesToEvictPerTask;
-
-  public RegionEvictorTask(List<LocalRegion> regionSet, HeapEvictor evictor,
-      long bytesToEvictPerTask) {
-    this.evictor = evictor;
-    this.regionSet = regionSet;
-    this.bytesToEvictPerTask = bytesToEvictPerTask;
-  }
-
-
-  public List<LocalRegion> getRegionList() {
-    synchronized (this.regionSet) {
-      return this.regionSet;
-    }
-  }
-
-  private InternalCache getInternalCache() {
-    return getHeapEvictor().getGemFireCache();
-  }
-
-  private HeapEvictor getHeapEvictor() {
-    return this.evictor;
-  }
-
-  @Override
-  public void run() {
-    getInternalCache().getCachePerfStats().incEvictorJobsStarted();
-    long bytesEvicted = 0;
-    long totalBytesEvicted = 0;
-    try {
-      while (true) {
-        getInternalCache().getCachePerfStats();
-        final long start = CachePerfStats.getStatTime();
-        synchronized (this.regionSet) {
-          if (this.regionSet.isEmpty()) {
-            lastTaskCompletionTime = System.currentTimeMillis();
-            return;
-          }
-          // TODO: Yogesh : try Fisher-Yates shuffle algorithm
-          Iterator<LocalRegion> iter = regionSet.iterator();
-          while (iter.hasNext()) {
-            LocalRegion region = iter.next();
-            try {
-              bytesEvicted = ((AbstractLRURegionMap) region.entries).centralizedLruUpdateCallback();
-              if (bytesEvicted == 0) {
-                iter.remove();
-              }
-              totalBytesEvicted += bytesEvicted;
-              if (totalBytesEvicted >= bytesToEvictPerTask || !getHeapEvictor().mustEvict()
-                  || this.regionSet.size() == 0) {
-                lastTaskCompletionTime = System.currentTimeMillis();
-                return;
-              }
-            } catch (RegionDestroyedException rd) {
-              region.cache.getCancelCriterion().checkCancelInProgress(rd);
-            } catch (Exception e) {
-              region.cache.getCancelCriterion().checkCancelInProgress(e);
-              logger.warn(LocalizedMessage.create(LocalizedStrings.Eviction_EVICTOR_TASK_EXCEPTION,
-                  new Object[] {e.getMessage()}), e);
-            } finally {
-              getInternalCache().getCachePerfStats();
-              long end = CachePerfStats.getStatTime();
-              getInternalCache().getCachePerfStats().incEvictWorkTime(end - start);
-            }
-          }
-        }
-      }
-    } finally {
-      getInternalCache().getCachePerfStats().incEvictorJobsCompleted();
-    }
-  }
-
-  public static int getEvictionBurstPauseTimeMillis() {
-    if (TEST_EVICTION_BURST_PAUSE_TIME_MILLIS != Integer.MAX_VALUE) {
-      return TEST_EVICTION_BURST_PAUSE_TIME_MILLIS;
-    }
-    return EVICTION_BURST_PAUSE_TIME_MILLIS;
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionMap.java b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionMap.java
index 80ad81bb69..0db5d84225 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionMap.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionMap.java
@@ -27,7 +27,7 @@
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.AbstractRegionMap.ARMLockTestHook;
 import org.apache.geode.internal.cache.entries.DiskEntry;
-import org.apache.geode.internal.cache.lru.LRUMapCallbacks;
+import org.apache.geode.internal.cache.eviction.EvictableMap;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.versions.RegionVersionVector;
 import org.apache.geode.internal.cache.versions.VersionHolder;
@@ -40,7 +40,7 @@
  *
  * @since GemFire 3.5.1
  */
-public interface RegionMap extends LRUMapCallbacks {
+public interface RegionMap extends EvictableMap {
 
   /**
    * Parameter object used to facilitate construction of an EntriesMap. Modification of fields after
@@ -353,4 +353,8 @@ default void lockRegionForAtomicTX(LocalRegion r) {}
   default void unlockRegionForAtomicTX(LocalRegion r) {}
 
   ARMLockTestHook getARMLockTestHook();
+
+  long getEvictions();
+
+  void incRecentlyUsed();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionStatus.java b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionStatus.java
index 3126a60fe3..a5c6f2296f 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionStatus.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionStatus.java
@@ -19,7 +19,7 @@
 
 import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 
 /**
  * Class <code>RegionStatus</code> provides information about <code>Region</code>s. This class is
@@ -62,7 +62,8 @@ private void initialize(Region region) {
     EvictionAttributes ea = region.getAttributes().getEvictionAttributes();
     if (ea != null && ea.getAlgorithm().isLRUMemory()) {
       LocalRegion lr = (LocalRegion) region;
-      LRUStatistics stats = ((AbstractLRURegionMap) lr.getRegionMap())._getLruList().stats();
+      EvictionStatistics stats =
+          ((AbstractLRURegionMap) lr.getRegionMap()).getEvictionController().getStatistics();
       setHeapSize(stats.getCounter());
     } else {
       setHeapSize(-1);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/VMCachedDeserializable.java b/geode-core/src/main/java/org/apache/geode/internal/cache/VMCachedDeserializable.java
index faf153654e..ef47b8e0d5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/VMCachedDeserializable.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/VMCachedDeserializable.java
@@ -24,7 +24,7 @@
 import org.apache.geode.cache.Region;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.cache.lru.LRUEntry;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.pdx.PdxInstance;
@@ -87,11 +87,11 @@ public Object getDeserializedValue(Region r, RegionEntry re) {
     if (v instanceof byte[]) {
       // org.apache.geode.internal.cache.GemFireCache.getInstance().getLogger().info("DEBUG
       // getDeserializedValue r=" + r + " re=" + re, new RuntimeException("STACK"));
-      LRUEntry le = null;
+      EvictableEntry le = null;
       if (re != null) {
         assert r != null;
-        if (re instanceof LRUEntry) {
-          le = (LRUEntry) re;
+        if (re instanceof EvictableEntry) {
+          le = (EvictableEntry) re;
         }
       }
       if (le != null) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/VMLRURegionMap.java b/geode-core/src/main/java/org/apache/geode/internal/cache/VMLRURegionMap.java
index 32426f22f0..5c4544b280 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/VMLRURegionMap.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/VMLRURegionMap.java
@@ -12,18 +12,15 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.internal.cache;
 
-import org.apache.geode.internal.cache.lru.*;
+import org.apache.geode.internal.cache.eviction.EvictionList;
 
 /**
  * Internal implementation of {@link RegionMap} for regions stored in normal VM memory that maintain
  * an LRU.
  *
  * @since GemFire 3.5.1
- *
- *
  */
 class VMLRURegionMap extends AbstractLRURegionMap {
 
@@ -32,31 +29,16 @@
     initialize(owner, attr, internalRegionArgs);
   }
 
-  // LRU fields and accessors
-  /**
-   * A tool from the eviction controller for sizing entries and expressing limits.
-   */
-  private EnableLRU ccHelper;
   /** The list of nodes in LRU order */
-  private NewLRUClockHand lruList;
-
-  @Override
-  protected void _setCCHelper(EnableLRU ccHelper) {
-    this.ccHelper = ccHelper;
-  }
-
-  @Override
-  protected EnableLRU _getCCHelper() {
-    return this.ccHelper;
-  }
+  private EvictionList lruList;
 
   @Override
-  protected void _setLruList(NewLRUClockHand lruList) {
-    this.lruList = lruList;
+  protected void setEvictionList(EvictionList evictionList) {
+    this.lruList = evictionList;
   }
 
   @Override
-  public NewLRUClockHand _getLruList() {
+  public EvictionList getEvictionList() {
     return this.lruList;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ValidatingDiskRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ValidatingDiskRegion.java
index 730f4cd8da..31c448c1ca 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ValidatingDiskRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ValidatingDiskRegion.java
@@ -31,8 +31,8 @@
 import org.apache.geode.internal.cache.DistributedRegion.DiskPosition;
 import org.apache.geode.internal.cache.InitialImageOperation.Entry;
 import org.apache.geode.internal.cache.entries.DiskEntry;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionList;
 import org.apache.geode.internal.cache.persistence.DiskExceptionHandler;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
@@ -229,7 +229,7 @@ public long getLastModified() {
     }
 
     @Override
-    public int updateAsyncEntrySize(EnableLRU capacityController) {
+    public int updateAsyncEntrySize(EvictionController capacityController) {
       throw new IllegalStateException("should never be called");
     }
 
@@ -305,7 +305,7 @@ public boolean dispatchListenerEvents(EntryEventImpl event) throws InterruptedEx
     }
 
     @Override
-    public void setRecentlyUsed() {
+    public void setRecentlyUsed(RegionEntryContext context) {
       // nothing
     }
 
@@ -517,23 +517,18 @@ public boolean isInUseByTransaction() {
       return false;
     }
 
-    @Override
-    public void setInUseByTransaction(boolean inUseByTransaction) {
-      // nothing
-    }
-
     @Override
     public void incRefCount() {
       // nothing
     }
 
     @Override
-    public void decRefCount(NewLRUClockHand lruList, InternalRegion region) {
+    public void decRefCount(EvictionList lruList, InternalRegion region) {
       // nothing
     }
 
     @Override
-    public void resetRefCount(NewLRUClockHand lruList) {
+    public void resetRefCount(EvictionList lruList) {
       // nothing
     }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/compression/CompressedCachedDeserializable.java b/geode-core/src/main/java/org/apache/geode/internal/cache/compression/CompressedCachedDeserializable.java
index 00149fea7d..62863e2d3d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/compression/CompressedCachedDeserializable.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/compression/CompressedCachedDeserializable.java
@@ -28,9 +28,9 @@
 import org.apache.geode.internal.cache.CachedDeserializableFactory;
 import org.apache.geode.internal.cache.EntryEventImpl;
 import org.apache.geode.internal.cache.RegionEntry;
-import org.apache.geode.internal.cache.lru.Sizeable;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.internal.size.Sizeable;
 
 /**
  * An abstract implementation of {@link CachedDeserializable} that prefers serialization and
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractDiskLRURegionEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractDiskLRURegionEntry.java
similarity index 86%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/AbstractDiskLRURegionEntry.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractDiskLRURegionEntry.java
index ad4d46ba17..ccfdafc5ae 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractDiskLRURegionEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractDiskLRURegionEntry.java
@@ -12,23 +12,19 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.internal.cache.entries;
 
-package org.apache.geode.internal.cache;
-
-
-import org.apache.geode.internal.cache.entries.AbstractOplogDiskRegionEntry;
-import org.apache.geode.internal.cache.lru.LRUEntry;
+import org.apache.geode.internal.cache.RegionEntryContext;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
 
 /**
  * Abstract implementation class of RegionEntry interface. This adds LRU support behavior to entries
  * that already have Disk support.
  *
  * @since GemFire 3.5.1
- *
- *
  */
 public abstract class AbstractDiskLRURegionEntry extends AbstractOplogDiskRegionEntry
-    implements LRUEntry {
+    implements EvictableEntry {
   protected AbstractDiskLRURegionEntry(RegionEntryContext context, Object value) {
     super(context, value);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractDiskRegionEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractDiskRegionEntry.java
index d6ebed95ce..a20f53ac8b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractDiskRegionEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractDiskRegionEntry.java
@@ -38,7 +38,7 @@ public void setValue(RegionEntryContext context, Object v) throws RegionClearedE
   public void setValue(RegionEntryContext context, Object value, EntryEventImpl event)
       throws RegionClearedException {
     Helper.update(this, (LocalRegion) context, value, event);
-    setRecentlyUsed(); // fix for bug #42284 - entry just put into the cache is evicted
+    setRecentlyUsed(context); // fix for bug #42284 - entry just put into the cache is evicted
   }
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractLRURegionEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractLRURegionEntry.java
index 2e36181c9d..0d168668cb 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractLRURegionEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractLRURegionEntry.java
@@ -12,30 +12,28 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.internal.cache.entries;
 
-
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.entries.AbstractRegionEntry;
-import org.apache.geode.internal.cache.lru.LRUEntry;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
+import org.apache.geode.internal.cache.eviction.EvictionList;
 
 /**
  * Abstract implementation class of RegionEntry interface. This adds LRU support behaviour
  *
  * @since GemFire 3.5.1
- *
- *
  */
-public abstract class AbstractLRURegionEntry extends AbstractRegionEntry implements LRUEntry {
+public abstract class AbstractLRURegionEntry extends AbstractRegionEntry implements EvictableEntry {
   protected AbstractLRURegionEntry(RegionEntryContext context, Object value) {
     super(context, value);
   }
 
-  /////////////////////////////////////////////////////////////////////
-  ////////////////////////// instance methods /////////////////////////
-  /////////////////////////////////////////////////////////////////////
-
   // Do not add any instance fields to this class.
   // Instead add them to the LRU section of LeafRegionEntry.cpp.
+
+  protected void appendToEvictionList(EvictionList evictionList) {
+    if (evictionList != null) {
+      evictionList.appendEntry(this);
+    }
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractOplogDiskRegionEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractOplogDiskRegionEntry.java
index f7b76c8f6f..a769e5b60c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractOplogDiskRegionEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractOplogDiskRegionEntry.java
@@ -142,7 +142,7 @@ public void setNext(DiskEntry v) {
   public void persistConflictingTag(InternalRegion region, VersionTag tag) {
     // only persist region needs to persist conflict tag
     Helper.updateVersionOnly(this, region, tag);
-    setRecentlyUsed();
+    setRecentlyUsed(region);
   }
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractRegionEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractRegionEntry.java
index 66677afe51..8451d27bab 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractRegionEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/AbstractRegionEntry.java
@@ -63,8 +63,7 @@
 import org.apache.geode.internal.cache.TimestampedEntryEventImpl;
 import org.apache.geode.internal.cache.Token;
 import org.apache.geode.internal.cache.TombstoneService;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionList;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.persistence.DiskStoreID;
 import org.apache.geode.internal.cache.versions.ConcurrentCacheModificationException;
@@ -232,7 +231,7 @@ public void updateStatsForPut(long lastModifiedTime, long lastAccessedTime) {
   }
 
   @Override
-  public void setRecentlyUsed() {
+  public void setRecentlyUsed(RegionEntryContext context) {
     // do nothing by default; only needed for LRU
   }
 
@@ -278,7 +277,7 @@ public void makeTombstone(InternalRegion region, VersionTag version)
         // unschedule the old tombstone
         region.unscheduleTombstone(this);
       }
-      setRecentlyUsed();
+      setRecentlyUsed(region);
       boolean newEntry = getValueAsToken() == Token.REMOVED_PHASE1;
       basicMakeTombstone(region);
       region.scheduleTombstone(this, version);
@@ -463,7 +462,7 @@ public Object getValue(RegionEntryContext context) {
     } else {
       result = OffHeapHelper.copyAndReleaseIfNeeded(result);
       ReferenceCountHelper.setReferenceCountOwner(null);
-      setRecentlyUsed();
+      setRecentlyUsed(context);
       return result;
     }
   }
@@ -476,7 +475,7 @@ public Object getValueRetain(RegionEntryContext context) {
     if (Token.isRemoved(result)) {
       return null;
     } else {
-      setRecentlyUsed();
+      setRecentlyUsed(context);
       return result;
     }
   }
@@ -503,7 +502,7 @@ protected void setValue(RegionEntryContext context, @Unretained Object value,
     _setValue(value);
     releaseOffHeapRefIfRegionBeingClosedOrDestroyed(context, value);
     if (recentlyUsed) {
-      setRecentlyUsed();
+      setRecentlyUsed(context);
     }
   }
 
@@ -1453,12 +1452,11 @@ public void setCacheListenerInvocationInProgress(final boolean isListenerInvoked
   }
 
   @Override
-  public boolean isInUseByTransaction() {
+  public synchronized boolean isInUseByTransaction() {
     return areAnyBitsSet(IN_USE_BY_TX);
   }
 
-  @Override
-  public void setInUseByTransaction(final boolean v) {
+  private void setInUseByTransaction(final boolean v) {
     if (v) {
       setBits(IN_USE_BY_TX);
     } else {
@@ -1473,14 +1471,11 @@ public synchronized void incRefCount() {
   }
 
   @Override
-  public synchronized void decRefCount(NewLRUClockHand lruList, InternalRegion region) {
+  public synchronized void decRefCount(EvictionList evictionList, InternalRegion region) {
     if (TXManagerImpl.decRefCount(this)) {
       if (isInUseByTransaction()) {
         setInUseByTransaction(false);
-        if (lruList != null) {
-          // No more transactions, place in lru list
-          lruList.appendEntry((LRUClockNode) this);
-        }
+        appendToEvictionList(evictionList);
         if (region != null && region.isEntryExpiryPossible()) {
           region.addExpiryTaskIfAbsent(this);
         }
@@ -1489,15 +1484,17 @@ public synchronized void decRefCount(NewLRUClockHand lruList, InternalRegion reg
   }
 
   @Override
-  public synchronized void resetRefCount(NewLRUClockHand lruList) {
+  public synchronized void resetRefCount(EvictionList evictionList) {
     if (isInUseByTransaction()) {
       setInUseByTransaction(false);
-      if (lruList != null) {
-        lruList.appendEntry((LRUClockNode) this);
-      }
+      appendToEvictionList(evictionList);
     }
   }
 
+  protected void appendToEvictionList(EvictionList evictionList) {
+    // nothing
+  }
+
   void _setValue(Object val) {
     setValueField(val);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/DiskEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/DiskEntry.java
index 43aae36ec0..8210564ed2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/DiskEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/DiskEntry.java
@@ -45,10 +45,8 @@
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.RegionMap;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.LRUEntry;
-import org.apache.geode.internal.cache.partitioned.Bucket;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.BytesAndBits;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
@@ -106,7 +104,7 @@
    */
   public DiskId getDiskId();
 
-  public int updateAsyncEntrySize(EnableLRU capacityController);
+  public int updateAsyncEntrySize(EvictionController capacityController);
 
   public DiskEntry getPrev();
 
@@ -984,8 +982,8 @@ private static AsyncDiskEntry basicUpdate(DiskEntry entry, InternalRegion region
           }
         }
       }
-      if (entry instanceof LRUEntry) {
-        LRUEntry le = (LRUEntry) entry;
+      if (entry instanceof EvictableEntry) {
+        EvictableEntry le = (EvictableEntry) entry;
         le.unsetEvicted();
       }
       return result;
@@ -1054,7 +1052,7 @@ private static Object faultInValue(DiskEntry entry, InternalRegion region,
       boolean lruFaultedIn = false;
       boolean done = false;
       try {
-        if (entry instanceof LRUEntry && !dr.isSync()) {
+        if (entry instanceof EvictableEntry && !dr.isSync()) {
           synchronized (entry) {
             DiskId did = entry.getDiskId();
             if (did != null && did.isPendingAsync()) {
@@ -1062,7 +1060,7 @@ private static Object faultInValue(DiskEntry entry, InternalRegion region,
               // See if it is pending async because of a faultOut.
               // If so then if we are not a backup then we can unschedule the pending async.
               // In either case we need to do the lruFaultIn logic.
-              boolean evicted = ((LRUClockNode) entry).testEvicted();
+              boolean evicted = ((EvictableEntry) entry).isEvicted();
               if (evicted) {
                 if (!dr.isBackup()) {
                   // @todo do we also need a bit that tells us if it is in the async queue?
@@ -1070,7 +1068,7 @@ private static Object faultInValue(DiskEntry entry, InternalRegion region,
                   did.setPendingAsync(false);
                 }
               }
-              lruEntryFaultIn((LRUEntry) entry, (DiskRecoveryStore) region);
+              lruEntryFaultIn((EvictableEntry) entry, (DiskRecoveryStore) region);
               lruFaultedIn = true;
             }
           }
@@ -1081,9 +1079,9 @@ private static Object faultInValue(DiskEntry entry, InternalRegion region,
 
             if (v == null) {
               v = readValueFromDisk(entry, (DiskRecoveryStore) region);
-              if (entry instanceof LRUEntry) {
+              if (entry instanceof EvictableEntry) {
                 if (v != null && !Token.isInvalid(v)) {
-                  lruEntryFaultIn((LRUEntry) entry, (DiskRecoveryStore) region);
+                  lruEntryFaultIn((EvictableEntry) entry, (DiskRecoveryStore) region);
 
                   lruFaultedIn = true;
                 }
@@ -1101,7 +1099,7 @@ private static Object faultInValue(DiskEntry entry, InternalRegion region,
         // fix for bug 31800
         v = null;
       } else {
-        ((RegionEntry) entry).setRecentlyUsed();
+        entry.setRecentlyUsed(region);
       }
       if (lruFaultedIn) {
         lruUpdateCallback((DiskRecoveryStore) region);
@@ -1132,9 +1130,9 @@ public static void recoverValue(DiskEntry entry, long oplogId, DiskRecoveryStore
             } finally {
               dr.releaseReadLock();
             }
-            if (entry instanceof LRUEntry) {
+            if (entry instanceof EvictableEntry) {
               if (value != null && !Token.isInvalid(value)) {
-                lruEntryFaultIn((LRUEntry) entry, recoveryStore);
+                lruEntryFaultIn((EvictableEntry) entry, recoveryStore);
                 lruFaultedIn = true;
               }
             }
@@ -1195,10 +1193,10 @@ private static void lruUpdateCallback(DiskRecoveryStore recoveryStore) {
       }
     }
 
-    private static void lruEntryFaultIn(LRUEntry entry, DiskRecoveryStore recoveryStore) {
+    private static void lruEntryFaultIn(EvictableEntry entry, DiskRecoveryStore recoveryStore) {
       RegionMap rm = (RegionMap) recoveryStore.getRegionMap();
       try {
-        rm.lruEntryFaultIn((LRUEntry) entry);
+        rm.lruEntryFaultIn((EvictableEntry) entry);
       } catch (DiskAccessException dae) {
         recoveryStore.handleDiskAccessException(dae);
         throw dae;
@@ -1311,14 +1309,14 @@ public static void updateStats(DiskRegionView drv, Object owner, int entriesInVm
      * <p>
      * Caller must synchronize on entry and it is assumed the entry is evicted
      */
-    public static int overflowToDisk(DiskEntry entry, InternalRegion region, EnableLRU ccHelper)
-        throws RegionClearedException {
+    public static int overflowToDisk(DiskEntry entry, InternalRegion region,
+        EvictionController ccHelper) throws RegionClearedException {
       DiskRegion dr = region.getDiskRegion();
       final int oldSize = ((DiskRecoveryStore) region).calculateRegionEntryValueSize(entry);
       // Get diskID . If it is null, it implies it is overflow only mode.
       DiskId did = entry.getDiskId();
       if (did == null) {
-        ((LRUEntry) entry).setDelayedDiskId((DiskRecoveryStore) region);
+        ((EvictableEntry) entry).setDelayedDiskId((DiskRecoveryStore) region);
         did = entry.getDiskId();
       }
 
@@ -1356,7 +1354,7 @@ public static int overflowToDisk(DiskEntry entry, InternalRegion region, EnableL
             region.updateSizeOnEvict(entry.getKey(), oldSize);
             entry.handleValueOverflow(region);
             entry.setValueWithContext(region, null);
-            change = ((LRUClockNode) entry).updateEntrySize(ccHelper);
+            change = ((EvictableEntry) entry).updateEntrySize(ccHelper);
             // the caller checked to make sure we had something to overflow
             // so dec inVM and inc onDisk
             updateStats(dr, region, -1/* InVM */, 1/* OnDisk */, getValueLength(did));
@@ -1459,7 +1457,8 @@ private static void writeEntryToDisk(DiskEntry entry, InternalRegion region, Ver
                     // Only setValue to null if this was an evict.
                     // We could just be a backup that is writing async.
                     if (!Token.isInvalid(entryVal) && (entryVal != Token.TOMBSTONE)
-                        && entry instanceof LRUEntry && ((LRUClockNode) entry).testEvicted()) {
+                        && entry instanceof EvictableEntry
+                        && ((EvictableEntry) entry).isEvicted()) {
                       // Moved this here to fix bug 40116.
                       region.updateSizeOnEvict(entry.getKey(), entryValSize);
                       updateStats(dr, region, -1/* InVM */, 1/* OnDisk */, did.getValueLength());
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/LeafRegionEntry.cpp b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/LeafRegionEntry.cpp
index 158b56dc43..83304ca7b8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/LeafRegionEntry.cpp
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/LeafRegionEntry.cpp
@@ -42,7 +42,7 @@ import org.apache.geode.internal.cache.InternalRegion;
 #endif
 import org.apache.geode.internal.cache.RegionEntryContext;
 #if defined(DISK) || defined(LRU)
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 #endif
 #ifdef DISK
@@ -56,8 +56,7 @@ import org.apache.geode.internal.InternalStatisticsDisabledException;
 #endif
 #ifdef LRU
 import org.apache.geode.internal.cache.InternalRegion;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 #endif
 #ifdef VERSIONED
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
@@ -172,7 +171,7 @@ public class LEAF_CLASS extends PARENT_CLASS {
   private byte distributedSystemId;
 #endif
 
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
 
 #ifdef KEY_OBJECT
@@ -211,7 +210,7 @@ public class LEAF_CLASS extends PARENT_CLASS {
       , final boolean byteEncode
 #endif
       ) {
-    super(context, 
+    super(context,
 #ifdef DISK
           (value instanceof RecoveredEntry ? null : value)
 #else
@@ -291,7 +290,7 @@ public class LEAF_CLASS extends PARENT_CLASS {
   public Token getValueAsToken() {
     return OffHeapRegionEntryHelper.getValueAsToken(this);
   }
-  
+
   @Override
   protected Object getValueField() {
     return OffHeapRegionEntryHelper._getValue(this);
@@ -326,7 +325,7 @@ public class LEAF_CLASS extends PARENT_CLASS {
   public boolean setAddress(final long expectedAddress, long newAddress) {
     return OFF_HEAP_ADDRESS_UPDATER.compareAndSet(this, expectedAddress, newAddress);
   }
-  
+
   @Override
 #ifdef OFFHEAP
   @Released
@@ -334,7 +333,7 @@ public class LEAF_CLASS extends PARENT_CLASS {
   public void release() {
     OffHeapRegionEntryHelper.releaseEntry(this);
   }
-  
+
   @Override
   public void returnToPool() {
     // never implemented
@@ -402,9 +401,9 @@ public class LEAF_CLASS extends PARENT_CLASS {
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -415,13 +414,13 @@ public class LEAF_CLASS extends PARENT_CLASS {
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 #endif
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  
+
   @Override
   public DiskId getDiskId() {
     return this.id;
@@ -441,7 +440,7 @@ public class LEAF_CLASS extends PARENT_CLASS {
     Helper.initialize(this, diskRecoveryStore, value);
   }
 #endif
-  
+
 #ifdef LRU
   // --------------------------------------- eviction code ----------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
@@ -458,30 +457,33 @@ public class LEAF_CLASS extends PARENT_CLASS {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
-  
+
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  
+
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController, final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -490,7 +492,7 @@ public class LEAF_CLASS extends PARENT_CLASS {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -506,28 +508,28 @@ public class LEAF_CLASS extends PARENT_CLASS {
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
 
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntry.java
index 76a78faaf0..c6924b99a9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntry.java
@@ -15,7 +15,6 @@
 
 package org.apache.geode.internal.cache.entries;
 
-import org.apache.geode.internal.cache.AbstractDiskLRURegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 
 /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapIntKey.java
index e9a767d2a3..8d93e20732 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -67,7 +68,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryHeapIntKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryHeapIntKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryHeapIntKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -138,9 +139,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -176,29 +177,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -207,7 +212,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -222,28 +227,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapLongKey.java
index 0c9e67cf87..60057e7134 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -67,7 +68,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryHeapLongKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryHeapLongKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryHeapLongKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -138,9 +139,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -176,29 +177,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -207,7 +212,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -222,28 +227,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapObjectKey.java
index fc98bf39af..924c68754a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -70,7 +71,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryHeapObjectKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryHeapObjectKey.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -141,9 +142,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -179,29 +180,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -210,7 +215,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -225,28 +230,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapStringKey1.java
index 944a0f691e..1eab2da2f8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -70,7 +71,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryHeapStringKey1> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryHeapStringKey1.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -158,9 +159,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -196,29 +197,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -227,7 +232,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -242,28 +247,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapStringKey2.java
index a6621f7953..54d8d08382 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -70,7 +71,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryHeapStringKey2> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryHeapStringKey2.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -179,9 +180,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -217,29 +218,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -248,7 +253,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -263,28 +268,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapUUIDKey.java
index 7d38d0ccdd..0319c0768d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -68,7 +69,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryHeapUUIDKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryHeapUUIDKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryHeapUUIDKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -141,9 +142,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -179,29 +180,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -210,7 +215,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -225,28 +230,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapIntKey.java
index dbdd9ffbda..3b0891f812 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -95,7 +96,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryOffHeapIntKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryOffHeapIntKey.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -200,9 +201,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -238,29 +239,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -269,7 +274,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -284,28 +289,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapLongKey.java
index 0c9a2bc6d0..f5215c78a6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -95,7 +96,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryOffHeapLongKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryOffHeapLongKey.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -200,9 +201,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -238,29 +239,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -269,7 +274,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -284,28 +289,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapObjectKey.java
index a962596bd2..ca9e6fec9c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -95,7 +96,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryOffHeapObjectKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryOffHeapObjectKey.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -200,9 +201,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -238,29 +239,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -269,7 +274,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -284,28 +289,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapStringKey1.java
index 679238e9ea..a70f0cdb9a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -95,7 +96,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryOffHeapStringKey1> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryOffHeapStringKey1.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -217,9 +218,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -255,29 +256,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -286,7 +291,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -301,28 +306,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapStringKey2.java
index 519c361820..834c8c59fd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -95,7 +96,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryOffHeapStringKey2> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryOffHeapStringKey2.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -238,9 +239,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -276,29 +277,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -307,7 +312,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -322,28 +327,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapUUIDKey.java
index b6d746195c..bbdce74ad0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskLRURegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -96,7 +97,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskLRURegionEntryOffHeapUUIDKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskLRURegionEntryOffHeapUUIDKey.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -203,9 +204,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -241,29 +242,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -272,7 +277,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -287,28 +292,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapIntKey.java
index a48d75257c..8ee0907c2c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,7 +26,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -64,7 +66,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapIntKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryHeapIntKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapIntKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -124,7 +126,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapLongKey.java
index 01fd925f02..c509feb29d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,7 +26,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -64,7 +66,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapLongKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryHeapLongKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapLongKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -124,7 +126,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapObjectKey.java
index f6d64ad32f..668b69935f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,7 +26,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -64,7 +66,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapObjectKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryHeapObjectKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapObjectKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -124,7 +126,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapStringKey1.java
index 1d9be0f663..d3e6b4a3df 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,7 +26,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -64,7 +66,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapStringKey1.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryHeapStringKey1> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapStringKey1.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -141,7 +143,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapStringKey2.java
index c985fe6fa4..250ee2e1ec 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,7 +26,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -64,7 +66,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapStringKey2.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryHeapStringKey2> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapStringKey2.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -162,7 +164,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapUUIDKey.java
index c3c5b59aed..5260e68997 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -65,7 +67,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapUUIDKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryHeapUUIDKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryHeapUUIDKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -127,7 +129,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapIntKey.java
index d2d5fa60cd..c6cd0d9244 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -89,7 +91,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryOffHeapIntKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryOffHeapIntKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryOffHeapIntKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -183,7 +185,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapLongKey.java
index 1ad4b32019..0ab36f8260 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -89,7 +91,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryOffHeapLongKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryOffHeapLongKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryOffHeapLongKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -183,7 +185,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapObjectKey.java
index 7fa131716e..e51b8298cf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -92,7 +94,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryOffHeapObjectKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryOffHeapObjectKey.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -186,7 +188,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapStringKey1.java
index faade3d685..a265fd8851 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -92,7 +94,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryOffHeapStringKey1> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryOffHeapStringKey1.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -203,7 +205,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapStringKey2.java
index 4de67a211f..96cbde040e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -92,7 +94,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryOffHeapStringKey2> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryOffHeapStringKey2.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -224,7 +226,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapUUIDKey.java
index 41407a824c..31c0400d20 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsDiskRegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,7 +28,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -90,7 +92,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryOffHeapUUIDKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsDiskRegionEntryOffHeapUUIDKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsDiskRegionEntryOffHeapUUIDKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -186,7 +188,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapIntKey.java
index 25e9587d70..667c4f5683 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -58,7 +59,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapIntKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryHeapIntKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapIntKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -118,29 +119,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -149,7 +154,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -164,28 +169,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapLongKey.java
index a27415e7e9..199e7a8b35 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -58,7 +59,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapLongKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryHeapLongKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapLongKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -118,29 +119,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -149,7 +154,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -164,28 +169,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapObjectKey.java
index 9141f66c2f..718c93eeb9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -58,7 +59,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapObjectKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryHeapObjectKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapObjectKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -118,29 +119,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -149,7 +154,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -164,28 +169,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapStringKey1.java
index d6157e9375..54a4a6e074 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -58,7 +59,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapStringKey1.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryHeapStringKey1> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapStringKey1.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -135,29 +136,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -166,7 +171,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -181,28 +186,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapStringKey2.java
index 99ed6310cb..8d61a2d408 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -58,7 +59,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapStringKey2.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryHeapStringKey2> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapStringKey2.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -156,29 +157,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -187,7 +192,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -202,28 +207,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapUUIDKey.java
index 2d90e813fa..0b57d58419 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -59,7 +60,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapUUIDKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryHeapUUIDKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryHeapUUIDKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -121,29 +122,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -152,7 +157,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -167,28 +172,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapIntKey.java
index cc8f353852..9e7d661f75 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -82,7 +83,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryOffHeapIntKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryOffHeapIntKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryOffHeapIntKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -176,29 +177,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -207,7 +212,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -222,28 +227,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapLongKey.java
index 6853300f96..31d2e00385 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -83,7 +84,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryOffHeapLongKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryOffHeapLongKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryOffHeapLongKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -177,29 +178,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -208,7 +213,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -223,28 +228,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapObjectKey.java
index 6030f1a231..be65af3020 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -85,7 +86,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryOffHeapObjectKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryOffHeapObjectKey.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -179,29 +180,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -210,7 +215,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -225,28 +230,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapStringKey1.java
index 24f4249a81..ae18456151 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -86,7 +87,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryOffHeapStringKey1> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryOffHeapStringKey1.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -197,29 +198,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -228,7 +233,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -243,28 +248,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapStringKey2.java
index bc76c51ec8..45e8395daf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -86,7 +87,7 @@
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryOffHeapStringKey2> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryOffHeapStringKey2.class,
           "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -218,29 +219,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -249,7 +254,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -264,28 +269,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapUUIDKey.java
index 659ea63b28..7478ffb98f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsLRURegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -84,7 +85,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryOffHeapUUIDKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsLRURegionEntryOffHeapUUIDKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsLRURegionEntryOffHeapUUIDKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -180,29 +181,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -211,7 +216,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -226,28 +231,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapIntKey.java
index 654fcac84e..9f8128ef36 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -53,7 +55,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapIntKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryHeapIntKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapIntKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapLongKey.java
index 5f05bf9eb6..219967d35b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -53,7 +55,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapLongKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryHeapLongKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapLongKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapObjectKey.java
index 19f8495c02..e6828bbdaa 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -53,7 +55,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapObjectKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryHeapObjectKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapObjectKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapStringKey1.java
index 9a997b5224..c0349aa8af 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -53,7 +55,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapStringKey1.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryHeapStringKey1> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapStringKey1.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapStringKey2.java
index 9862a9e29d..50c343b6d2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -53,7 +55,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapStringKey2.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryHeapStringKey2> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapStringKey2.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapUUIDKey.java
index 6ddfe2f38b..e540dc216e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -54,7 +56,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapUUIDKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryHeapUUIDKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryHeapUUIDKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapIntKey.java
index 10de41b218..69f793f02f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -77,7 +79,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapIntKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryOffHeapIntKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapIntKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapLongKey.java
index b8b588c4a9..c2ced956cd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -77,7 +79,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapLongKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryOffHeapLongKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapLongKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapObjectKey.java
index 9085102c5c..b4fca6bd32 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -77,7 +79,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapObjectKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryOffHeapObjectKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapObjectKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapStringKey1.java
index c0270065dc..88d4ca5f52 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -78,7 +80,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapStringKey1.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryOffHeapStringKey1> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapStringKey1.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapStringKey2.java
index 8e211ad9e2..729bbee696 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -78,7 +80,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapStringKey2.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryOffHeapStringKey2> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapStringKey2.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapUUIDKey.java
index 617e43d0f3..ad528bcedd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMStatsRegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -78,7 +80,7 @@
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapUUIDKey.class, "hitCount");
   private static final AtomicIntegerFieldUpdater<VMStatsRegionEntryOffHeapUUIDKey> MISS_COUNT_UPDATER =
       AtomicIntegerFieldUpdater.newUpdater(VMStatsRegionEntryOffHeapUUIDKey.class, "missCount");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntry.java
index 35c0bde68a..133e1b6cea 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntry.java
@@ -16,7 +16,6 @@
 package org.apache.geode.internal.cache.entries;
 
 
-import org.apache.geode.internal.cache.AbstractDiskLRURegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 
 /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapIntKey.java
index 0766608635..9d4c137997 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -58,7 +59,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -129,9 +130,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -167,29 +168,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -198,7 +203,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -213,28 +218,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapLongKey.java
index 7626f2f04b..0d97466c83 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -58,7 +59,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -129,9 +130,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -167,29 +168,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -198,7 +203,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -213,28 +218,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapObjectKey.java
index beb91ee553..2d51cb317f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -59,7 +60,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -130,9 +131,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -168,29 +169,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -199,7 +204,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -214,28 +219,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapStringKey1.java
index bac481c59d..8adeea0d33 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -59,7 +60,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -147,9 +148,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -185,29 +186,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -216,7 +221,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -231,28 +236,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapStringKey2.java
index 4cbd3df3b3..2dbd9f2da1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -59,7 +60,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -168,9 +169,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -206,29 +207,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -237,7 +242,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -252,28 +257,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapUUIDKey.java
index 7b33604123..ce0e51b838 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -59,7 +60,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -132,9 +133,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -170,29 +171,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -201,7 +206,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -216,28 +221,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapIntKey.java
index 0870d81726..4bedab8ba4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -84,7 +85,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -189,9 +190,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -227,29 +228,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -258,7 +263,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -273,28 +278,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapLongKey.java
index 253f280ce0..c6b673702e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -84,7 +85,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -189,9 +190,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -227,29 +228,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -258,7 +263,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -273,28 +278,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapObjectKey.java
index 61418af58c..04c369e423 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -84,7 +85,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -189,9 +190,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -227,29 +228,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -258,7 +263,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -273,28 +278,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapStringKey1.java
index b234501b72..afabf22284 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -84,7 +85,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -206,9 +207,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -244,29 +245,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -275,7 +280,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -290,28 +295,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapStringKey2.java
index e54e6c69a8..c5033b4ec0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -84,7 +85,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -227,9 +228,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -265,29 +266,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -296,7 +301,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -311,28 +316,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapUUIDKey.java
index 03c1fbbe2d..bf2d1b92c9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskLRURegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -85,7 +86,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -192,9 +193,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -230,29 +231,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -261,7 +266,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -276,28 +281,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapIntKey.java
index 2b7a5505c1..92363f96a7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,7 +24,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -54,7 +56,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -114,7 +116,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapLongKey.java
index 36aec3977c..21ff52ae47 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,7 +24,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -54,7 +56,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -114,7 +116,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapObjectKey.java
index 8c14aa2c09..708c1fa38a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,7 +24,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -54,7 +56,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -114,7 +116,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapStringKey1.java
index 7fcbf1676d..dfad7cb524 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,7 +24,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -54,7 +56,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -131,7 +133,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapStringKey2.java
index 1b91394e3e..413bdad921 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,7 +24,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -54,7 +56,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -152,7 +154,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapUUIDKey.java
index 99e1dee94e..87254709ca 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,7 +25,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -55,7 +57,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -117,7 +119,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapIntKey.java
index cf1cbf6c1d..177adb3d76 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,7 +25,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -78,7 +80,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -172,7 +174,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapLongKey.java
index 14b142ca97..321b087d91 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,7 +25,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -79,7 +81,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -173,7 +175,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapObjectKey.java
index e746ad7afb..1397902be0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,7 +25,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -80,7 +82,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -174,7 +176,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapStringKey1.java
index 7f215fbc74..5b882ae6bf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,7 +25,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -80,7 +82,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -191,7 +193,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapStringKey2.java
index e669d0c409..a3b072000c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,7 +25,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -80,7 +82,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -212,7 +214,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapUUIDKey.java
index e526fd25c0..5c869f81e1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinDiskRegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,7 +26,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -80,7 +82,7 @@
    * @since GemFire 5.1
    */
   protected DiskId id;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -176,7 +178,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapIntKey.java
index f7846a108c..1edf46b1dd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -20,9 +22,8 @@
 
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -49,7 +50,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -109,29 +110,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -140,7 +145,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -155,28 +160,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapLongKey.java
index 0680506635..9ac83d8485 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -20,9 +22,8 @@
 
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -49,7 +50,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -109,29 +110,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -140,7 +145,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -155,28 +160,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapObjectKey.java
index 13eb190e35..fac31fc83e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -20,9 +22,8 @@
 
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -49,7 +50,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -109,29 +110,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -140,7 +145,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -155,28 +160,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapStringKey1.java
index 6ec2996aef..574713a82c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -20,9 +22,8 @@
 
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -49,7 +50,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -126,29 +127,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -157,7 +162,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -172,28 +177,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapStringKey2.java
index 3c756d73f2..bc7b1a2d7f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -20,9 +22,8 @@
 
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -49,7 +50,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -147,29 +148,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -178,7 +183,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -193,28 +198,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapUUIDKey.java
index 1ec0b07ee1..3236506d7c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 
@@ -50,7 +51,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -112,29 +113,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -143,7 +148,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -158,28 +163,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapIntKey.java
index 6fce080d2a..c4295c121b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -73,7 +74,7 @@
    */
   private static final AtomicLongFieldUpdater<VMThinLRURegionEntryOffHeapIntKey> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinLRURegionEntryOffHeapIntKey.class, "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -167,29 +168,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -198,7 +203,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -213,28 +218,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapLongKey.java
index 8113b75f5e..cbf1893211 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -73,7 +74,7 @@
    */
   private static final AtomicLongFieldUpdater<VMThinLRURegionEntryOffHeapLongKey> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinLRURegionEntryOffHeapLongKey.class, "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -167,29 +168,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -198,7 +203,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -213,28 +218,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapObjectKey.java
index 940d769cee..7cac8b0da9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -74,7 +75,7 @@
   private static final AtomicLongFieldUpdater<VMThinLRURegionEntryOffHeapObjectKey> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinLRURegionEntryOffHeapObjectKey.class,
           "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -168,29 +169,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -199,7 +204,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -214,28 +219,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapStringKey1.java
index 2c339a3c17..aef89d96ab 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -75,7 +76,7 @@
   private static final AtomicLongFieldUpdater<VMThinLRURegionEntryOffHeapStringKey1> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinLRURegionEntryOffHeapStringKey1.class,
           "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -186,29 +187,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -217,7 +222,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -232,28 +237,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapStringKey2.java
index f09ec39f54..d0820564c8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -21,9 +23,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -75,7 +76,7 @@
   private static final AtomicLongFieldUpdater<VMThinLRURegionEntryOffHeapStringKey2> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinLRURegionEntryOffHeapStringKey2.class,
           "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -207,29 +208,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -238,7 +243,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -253,28 +258,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapUUIDKey.java
index 8194f186db..c41c3b4231 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinLRURegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
@@ -74,7 +75,7 @@
    */
   private static final AtomicLongFieldUpdater<VMThinLRURegionEntryOffHeapUUIDKey> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinLRURegionEntryOffHeapUUIDKey.class, "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -170,29 +171,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -201,7 +206,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -216,28 +221,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapIntKey.java
index 449c5ce489..5042fafc50 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -43,7 +45,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapLongKey.java
index 53a3027f8f..52463b7ba5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -43,7 +45,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapObjectKey.java
index 586f9dd344..0bc963336d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -43,7 +45,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapStringKey1.java
index 929f26a050..41547f29ce 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -43,7 +45,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapStringKey2.java
index f413bdb824..c922b82960 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -43,7 +45,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapUUIDKey.java
index f9d411cc41..f2850bd856 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -44,7 +46,7 @@
   @SuppressWarnings("unused")
   private volatile long lastModified;
   private volatile Object value;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapIntKey.java
index c44847d5e5..3b8238ae6c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -67,7 +69,7 @@
    */
   private static final AtomicLongFieldUpdater<VMThinRegionEntryOffHeapIntKey> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinRegionEntryOffHeapIntKey.class, "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapLongKey.java
index a92ccf7cd6..130b28c659 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -67,7 +69,7 @@
    */
   private static final AtomicLongFieldUpdater<VMThinRegionEntryOffHeapLongKey> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinRegionEntryOffHeapLongKey.class, "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapObjectKey.java
index d9b061ea14..06351cead5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -67,7 +69,7 @@
    */
   private static final AtomicLongFieldUpdater<VMThinRegionEntryOffHeapObjectKey> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinRegionEntryOffHeapObjectKey.class, "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapStringKey1.java
index b1dd066c2a..df5be7c830 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -67,7 +69,7 @@
    */
   private static final AtomicLongFieldUpdater<VMThinRegionEntryOffHeapStringKey1> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinRegionEntryOffHeapStringKey1.class, "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapStringKey2.java
index cec58b4a86..0e01273ae6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -67,7 +69,7 @@
    */
   private static final AtomicLongFieldUpdater<VMThinRegionEntryOffHeapStringKey2> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinRegionEntryOffHeapStringKey2.class, "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapUUIDKey.java
index cb605dcd7d..4898c47c00 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VMThinRegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -68,7 +70,7 @@
    */
   private static final AtomicLongFieldUpdater<VMThinRegionEntryOffHeapUUIDKey> OFF_HEAP_ADDRESS_UPDATER =
       AtomicLongFieldUpdater.newUpdater(VMThinRegionEntryOffHeapUUIDKey.class, "offHeapAddress");
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapIntKey.java
index a631a2fcaa..951bae5f36 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -84,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -155,9 +156,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -193,29 +194,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -224,7 +229,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -239,28 +244,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapLongKey.java
index d5c517daa4..57abee53fc 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -84,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -155,9 +156,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -193,29 +194,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -224,7 +229,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -239,28 +244,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapObjectKey.java
index 0748f15644..8ff66ec7cf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -84,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -155,9 +156,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -193,29 +194,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -224,7 +229,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -239,28 +244,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapStringKey1.java
index 29659bcd27..6489508b46 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -84,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -172,9 +173,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -210,29 +211,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -241,7 +246,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -256,28 +261,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapStringKey2.java
index 50909d56ab..0d0173e0ca 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -84,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -193,9 +194,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -231,29 +232,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -262,7 +267,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -277,28 +282,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapUUIDKey.java
index bbb7986b2c..7bd0b680c1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,9 +30,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -85,7 +86,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -158,9 +159,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -196,29 +197,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -227,7 +232,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -242,28 +247,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapIntKey.java
index 86175bf737..ca43b11cda 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,9 +30,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -109,7 +110,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -214,9 +215,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -252,29 +253,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -283,7 +288,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -298,28 +303,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapLongKey.java
index f5d5575bb7..ec1b67508d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,9 +30,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -109,7 +110,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -214,9 +215,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -252,29 +253,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -283,7 +288,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -298,28 +303,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapObjectKey.java
index b99b0a401e..01f8feb10b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,9 +30,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -109,7 +110,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -214,9 +215,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -252,29 +253,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -283,7 +288,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -298,28 +303,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapStringKey1.java
index 2a39b671f5..8de9909156 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,9 +30,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -109,7 +110,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -231,9 +232,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -269,29 +270,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -300,7 +305,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -315,28 +320,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapStringKey2.java
index c66ec95140..167a1c926c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,9 +30,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -109,7 +110,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -252,9 +253,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -290,29 +291,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -321,7 +326,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -336,28 +341,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapUUIDKey.java
index 61ced60230..16ee36d8ba 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskLRURegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -29,9 +31,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -110,7 +111,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -217,9 +218,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -255,29 +256,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -286,7 +291,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -301,28 +306,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapIntKey.java
index 3bfb4b9b3f..414ae044cf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,7 +29,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -81,7 +83,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -141,7 +143,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapLongKey.java
index 532e9863b7..09f6ad1448 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,7 +29,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -81,7 +83,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -141,7 +143,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapObjectKey.java
index 94889277d9..adc497806d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,7 +29,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -81,7 +83,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -141,7 +143,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapStringKey1.java
index 3901644832..9ca60b2d23 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,7 +29,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -81,7 +83,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -158,7 +160,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapStringKey2.java
index d4d4fb5eaa..bc1a5eee54 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,7 +29,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -81,7 +83,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -179,7 +181,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapUUIDKey.java
index 7b45756594..957110c243 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,7 +30,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -82,7 +84,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -144,7 +146,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapIntKey.java
index 3f8fa9b962..1dfe0ede76 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,7 +30,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -107,7 +109,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -201,7 +203,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapLongKey.java
index 8feeecded8..37f5a55ed1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,7 +30,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -107,7 +109,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -201,7 +203,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapObjectKey.java
index da0306dd9d..111f54c244 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,7 +30,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -107,7 +109,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -201,7 +203,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapStringKey1.java
index e2c648dc83..3a2701657e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,7 +30,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -107,7 +109,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -218,7 +220,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapStringKey2.java
index d166840e34..f02899e917 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,7 +30,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -107,7 +109,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -239,7 +241,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapUUIDKey.java
index 193cd63348..83e587bc1d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsDiskRegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -29,7 +31,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -108,7 +110,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -204,7 +206,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapIntKey.java
index 3fb9a6d8ad..2649bb3ee3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -74,7 +75,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -134,29 +135,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -165,7 +170,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -180,28 +185,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapLongKey.java
index 8d64aaa706..468cb3afca 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -74,7 +75,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -134,29 +135,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -165,7 +170,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -180,28 +185,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapObjectKey.java
index 186aa120f3..f3de3b5f2e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -74,7 +75,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -134,29 +135,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -165,7 +170,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -180,28 +185,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapStringKey1.java
index 1b157afc99..fa2fe8f4c1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -74,7 +75,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -151,29 +152,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -182,7 +187,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -197,28 +202,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapStringKey2.java
index 4c430948d8..6a0a1be2bd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -74,7 +75,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -172,29 +173,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -203,7 +208,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -218,28 +223,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapUUIDKey.java
index d360bcf14d..d9e74eb4a8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.InternalStatisticsDisabledException;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -75,7 +76,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -137,29 +138,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -168,7 +173,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -183,28 +188,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapIntKey.java
index e36cc217db..0aa3cf583b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -99,7 +100,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -193,29 +194,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -224,7 +229,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -239,28 +244,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapLongKey.java
index 1c5e17122f..53b9e21ffe 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -100,7 +101,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -194,29 +195,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -225,7 +230,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -240,28 +245,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapObjectKey.java
index 2a604d104a..d1496ea172 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -100,7 +101,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -194,29 +195,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -225,7 +230,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -240,28 +245,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapStringKey1.java
index 3f9b10da7b..dafc3032c7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -100,7 +101,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -211,29 +212,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -242,7 +247,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -257,28 +262,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapStringKey2.java
index d7fd503a70..854b40c6cc 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -100,7 +101,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -232,29 +233,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -263,7 +268,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -278,28 +283,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapUUIDKey.java
index 42c1267e87..fb2a043a91 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsLRURegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,9 +27,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -101,7 +102,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -197,29 +198,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -228,7 +233,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -243,28 +248,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapIntKey.java
index 9b004437f6..da7f5d4574 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -67,7 +69,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapLongKey.java
index 7634039891..05a794b7dd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -67,7 +69,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapObjectKey.java
index ad51953088..d5ab0df68c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -70,7 +72,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapStringKey1.java
index 85363ba1a5..047e620b9c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -70,7 +72,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapStringKey2.java
index 6abc6b0e0c..ed5549e8fd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -70,7 +72,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapUUIDKey.java
index df23aaec02..e074215c5e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -68,7 +70,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapIntKey.java
index 2d933defdc..836311fa84 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -95,7 +97,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapLongKey.java
index c3fbc247fa..8c4c6ae3f4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -95,7 +97,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapObjectKey.java
index 6338abd5af..d851bd6da7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -95,7 +97,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapStringKey1.java
index 5111387cb4..5e5c48e647 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -95,7 +97,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapStringKey2.java
index 536c0072de..b9d579f9e3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -95,7 +97,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapUUIDKey.java
index b6c25d3401..fafd735dea 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedStatsRegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -96,7 +98,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapIntKey.java
index 2254063c27..9c5c92ee54 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -72,7 +73,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -143,9 +144,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -181,29 +182,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -212,7 +217,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -227,28 +232,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapLongKey.java
index 5c86762898..f9318eb22b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -73,7 +74,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -144,9 +145,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -182,29 +183,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -213,7 +218,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -228,28 +233,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapObjectKey.java
index 93f35d7056..cdd2e585ac 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -73,7 +74,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -144,9 +145,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -182,29 +183,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -213,7 +218,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -228,28 +233,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapStringKey1.java
index cf05b69cba..30e954cc3a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -73,7 +74,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -161,9 +162,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -199,29 +200,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -230,7 +235,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -245,28 +250,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapStringKey2.java
index 75448e6134..26b08a07c8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,9 +28,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -73,7 +74,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -182,9 +183,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -220,29 +221,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -251,7 +256,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -266,28 +271,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapUUIDKey.java
index 108098ba48..415fcfb531 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -74,7 +75,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -147,9 +148,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -185,29 +186,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -216,7 +221,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -231,28 +236,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapIntKey.java
index 78d5775aeb..ed0b563cd4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -98,7 +99,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -203,9 +204,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -241,29 +242,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -272,7 +277,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -287,28 +292,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapLongKey.java
index 0bc1a377fb..ef33cc93df 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -98,7 +99,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -203,9 +204,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -241,29 +242,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -272,7 +277,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -287,28 +292,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapObjectKey.java
index 4c5a942873..5183ea4614 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -98,7 +99,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -203,9 +204,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -241,29 +242,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -272,7 +277,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -287,28 +292,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapStringKey1.java
index 4cd61f645a..09c113b7fe 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -98,7 +99,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -220,9 +221,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -258,29 +259,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -289,7 +294,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -304,28 +309,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapStringKey2.java
index ee7b623f82..4b6e322d54 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,9 +29,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -98,7 +99,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -241,9 +242,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -279,29 +280,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -310,7 +315,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -325,28 +330,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapUUIDKey.java
index fbec474898..149f5a868d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskLRURegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -28,9 +30,8 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -99,7 +100,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -206,9 +207,9 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public synchronized int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateAsyncEntrySize(final EvictionController evictionController) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), null);
+    int newSize = evictionController.entrySize(getKeyForSizing(), null);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
@@ -244,29 +245,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -275,7 +280,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -290,28 +295,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapIntKey.java
index 69404ecf29..20c9e8e0b9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -69,7 +71,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -129,7 +131,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapLongKey.java
index 4272c6f1f5..66cd4d9eba 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -69,7 +71,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -129,7 +131,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapObjectKey.java
index 7d3ec1e0db..37449e0063 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -69,7 +71,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -129,7 +131,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapStringKey1.java
index 48bc237cbe..38ba70ca39 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -69,7 +71,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -146,7 +148,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapStringKey2.java
index 0c0f93ddfc..99fef20290 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -25,7 +27,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -69,7 +71,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -167,7 +169,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapUUIDKey.java
index ededc52fe0..7e58c64798 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,7 +28,7 @@
 import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -70,7 +72,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -132,7 +134,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapIntKey.java
index 6d050bd74a..c9f93d023b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,7 +28,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -94,7 +96,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -188,7 +190,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapLongKey.java
index 43c81e3265..b740d6e3bf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,7 +28,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -95,7 +97,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -189,7 +191,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapObjectKey.java
index 36a18db443..459e8de819 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,7 +28,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -95,7 +97,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -189,7 +191,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapStringKey1.java
index a9e90cf58c..64d88bd2d9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,7 +28,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -95,7 +97,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -206,7 +208,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapStringKey2.java
index 71bff72b5a..599fe296e3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -26,7 +28,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -95,7 +97,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -227,7 +229,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapUUIDKey.java
index 6e639b761a..a46ea081b6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinDiskRegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -27,7 +29,7 @@
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -96,7 +98,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -192,7 +194,7 @@ protected void initialize(final RegionEntryContext context, final Object value)
   }
 
   @Override
-  public int updateAsyncEntrySize(final EnableLRU capacityController) {
+  public int updateAsyncEntrySize(final EvictionController evictionController) {
     throw new IllegalStateException("should never be called");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapIntKey.java
index f8797e6c13..7438107c45 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -63,7 +64,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -123,29 +124,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -154,7 +159,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -169,28 +174,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapLongKey.java
index 5853733291..8808d6ed97 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -63,7 +64,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -123,29 +124,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -154,7 +159,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -169,28 +174,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapObjectKey.java
index 84d33765bc..d9c5aba5e8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -63,7 +64,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -123,29 +124,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -154,7 +159,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -169,28 +174,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapStringKey1.java
index 444c403ddb..405d69f8ee 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -63,7 +64,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -140,29 +141,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -171,7 +176,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -186,28 +191,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapStringKey2.java
index ec56423d04..09acff2889 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -22,9 +24,8 @@
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -63,7 +64,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -161,29 +162,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -192,7 +197,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -207,28 +212,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapUUIDKey.java
index ed75c5cfb5..a1c13086b1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -64,7 +65,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -126,29 +127,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -157,7 +162,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -172,28 +177,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapIntKey.java
index b2f30e1c89..df3862987e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -88,7 +89,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
@@ -182,29 +183,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -213,7 +218,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -228,28 +233,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapLongKey.java
index dbfdadec1c..a4cd27e68d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -88,7 +89,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
@@ -182,29 +183,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -213,7 +218,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -228,28 +233,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapObjectKey.java
index 7154a9551f..498339d759 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -89,7 +90,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
@@ -183,29 +184,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -214,7 +219,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -229,28 +234,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapStringKey1.java
index c290bfdbc1..150d998d7e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -89,7 +90,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
@@ -200,29 +201,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -231,7 +236,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -246,28 +251,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapStringKey2.java
index 55c631519d..584a7b0960 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -23,9 +25,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -89,7 +90,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
@@ -221,29 +222,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -252,7 +257,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -267,28 +272,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapUUIDKey.java
index 0f7365f8cc..1452be2d55 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinLRURegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -24,9 +26,8 @@
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionEntryContext;
 import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
@@ -89,7 +90,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
@@ -185,29 +186,33 @@ public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
   }
 
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController) {
+  public synchronized int updateEntrySize(final EvictionController evictionController) {
     // OFFHEAP: getValue ok w/o incing refcount because we are synced and only getting the size
-    return updateEntrySize(capacityController, getValue());
+    return updateEntrySize(evictionController, getValue());
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   @Override
-  public synchronized int updateEntrySize(final EnableLRU capacityController, final Object value) {
+  public synchronized int updateEntrySize(final EvictionController evictionController,
+      final Object value) {
     int oldSize = getEntrySize();
-    int newSize = capacityController.entrySize(getKeyForSizing(), value);
+    int newSize = evictionController.entrySize(getKeyForSizing(), value);
     setEntrySize(newSize);
     int delta = newSize - oldSize;
     return delta;
   }
 
   @Override
-  public boolean testRecentlyUsed() {
+  public boolean isRecentlyUsed() {
     return areAnyBitsSet(RECENTLY_USED);
   }
 
   @Override
-  public void setRecentlyUsed() {
-    setBits(RECENTLY_USED);
+  public void setRecentlyUsed(RegionEntryContext context) {
+    if (!isRecentlyUsed()) {
+      setBits(RECENTLY_USED);
+      context.incRecentlyUsed();
+    }
   }
 
   @Override
@@ -216,7 +221,7 @@ public void unsetRecentlyUsed() {
   }
 
   @Override
-  public boolean testEvicted() {
+  public boolean isEvicted() {
     return areAnyBitsSet(EVICTED);
   }
 
@@ -231,28 +236,28 @@ public void unsetEvicted() {
   }
 
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
-  private LRUClockNode nextLRU;
-  private LRUClockNode previousLRU;
+  private EvictionNode nextEvictionNode;
+  private EvictionNode previousEvictionNode;
   private int size;
 
   @Override
-  public void setNextLRUNode(final LRUClockNode nextLRU) {
-    this.nextLRU = nextLRU;
+  public void setNext(final EvictionNode nextEvictionNode) {
+    this.nextEvictionNode = nextEvictionNode;
   }
 
   @Override
-  public LRUClockNode nextLRUNode() {
-    return this.nextLRU;
+  public EvictionNode next() {
+    return this.nextEvictionNode;
   }
 
   @Override
-  public void setPrevLRUNode(final LRUClockNode previousLRU) {
-    this.previousLRU = previousLRU;
+  public void setPrevious(final EvictionNode previousEvictionNode) {
+    this.previousEvictionNode = previousEvictionNode;
   }
 
   @Override
-  public LRUClockNode prevLRUNode() {
-    return this.previousLRU;
+  public EvictionNode previous() {
+    return this.previousEvictionNode;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapIntKey.java
index 4468abe992..74f31b15f5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -57,7 +59,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapLongKey.java
index ce8d77671d..4ef8cb087b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -57,7 +59,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapObjectKey.java
index 8adb1ba1a8..db5207742f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -58,7 +60,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapStringKey1.java
index 7da13392bd..1c7c559b20 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -58,7 +60,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapStringKey2.java
index 910b225236..9faae87f02 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -58,7 +60,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapUUIDKey.java
index 7670287875..9f78478fee 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -58,7 +60,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapIntKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapIntKey.java
index a032c0171b..e388a7be15 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapIntKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapIntKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -83,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final int key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapLongKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapLongKey.java
index d15e4167b1..0e4a959880 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapLongKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapLongKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -83,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapObjectKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapObjectKey.java
index 0a9ca08b8e..9f8a557213 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapObjectKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapObjectKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -83,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final Object key;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapStringKey1.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapStringKey1.java
index 9b68091412..6a1e1e339a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapStringKey1.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapStringKey1.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -83,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long bits1;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapStringKey2.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapStringKey2.java
index c633430b76..54424f63eb 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapStringKey2.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapStringKey2.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -83,7 +85,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   /**
    * strlen is encoded in lowest 6 bits (max strlen is 63)<br>
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapUUIDKey.java b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapUUIDKey.java
index 71046becd7..b3115c73c7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapUUIDKey.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/entries/VersionedThinRegionEntryOffHeapUUIDKey.java
@@ -1,3 +1,5 @@
+
+
 /*
  * 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
@@ -84,7 +86,7 @@
   private int regionVersionLowBytes;
   private byte entryVersionHighByte;
   private byte distributedSystemId;
-  // ----------------------------------------- key code -------------------------------------------
+  // --------------------------------------- key fields -------------------------------------------
   // DO NOT modify this class. It was generated from LeafRegionEntry.cpp
   private final long keyMostSigBits;
   private final long keyLeastSigBits;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/AbstractEvictionController.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/AbstractEvictionController.java
new file mode 100644
index 0000000000..ce02f9bd80
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/AbstractEvictionController.java
@@ -0,0 +1,270 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+
+import org.apache.geode.InternalGemFireException;
+import org.apache.geode.StatisticsFactory;
+import org.apache.geode.cache.EvictionAction;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+
+/**
+ * Eviction controllers that extend this class evict the least recently used (LRU) entry in the
+ * region whose capacity they controller. In order to provide an efficient computation of the LRU
+ * entry, GemFire uses special internal data structures for managing the contents of a region. As a
+ * result, there are several restrictions that are placed on regions whose capacity is governed by
+ * an LRU algorithm.
+ *
+ * <ul>
+ * <li>If the capacity of a region is to be controlled by an LRU algorithm, then the region must be
+ * <b>created</b> with {@link org.apache.geode.cache.EvictionAttributes}
+ * <li>The eviction controller of a region governed by an LRU algorithm cannot be changed.</li>
+ * <li>An LRU algorithm cannot be applied to a region after the region has been created.</li>
+ * </ul>
+ *
+ * <p>
+ * LRU algorithms also specify what {@linkplain org.apache.geode.cache.EvictionAction action} should
+ * be performed upon the least recently used entry when the capacity is reached. Currently, there
+ * are two supported actions: {@linkplain org.apache.geode.cache.EvictionAction#LOCAL_DESTROY
+ * locally destroying} the entry (which is the
+ * {@linkplain org.apache.geode.cache.EvictionAction#DEFAULT_EVICTION_ACTION default}), thus freeing
+ * up space in the VM, and {@linkplain org.apache.geode.cache.EvictionAction#OVERFLOW_TO_DISK
+ * overflowing} the value of the entry to disk.
+ *
+ * <p>
+ * {@link org.apache.geode.cache.EvictionAttributes Eviction controllers} that use an LRU algorithm
+ * maintain certain region-dependent state (such as the maximum number of entries allowed in the
+ * region). As a result, an instance of {@code AbstractEvictionController} cannot be shared among
+ * multiple regions. Attempts to create a region with a LRU-based capacity controller that has
+ * already been used to create another region will result in an {@link IllegalStateException} being
+ * thrown.
+ *
+ * @since GemFire 3.2
+ */
+abstract class AbstractEvictionController implements EvictionController, Serializable, Cloneable {
+
+  /**
+   * The key for setting the {@code eviction-action} property of an
+   * {@code AbstractEvictionController}
+   */
+  protected static final String EVICTION_ACTION = "eviction-action";
+
+  private static final int DESTROYS_LIMIT = 1000;
+
+  /**
+   * What to do upon eviction
+   */
+  protected EvictionAction evictionAction;
+
+  /**
+   * Used to dynamically track the changing region limit.
+   */
+  protected transient InternalEvictionStatistics stats;
+
+  protected BucketRegion bucketRegion;
+
+  /** The region whose capacity is controller by this eviction controller */
+  private transient volatile String regionName;
+
+  /**
+   * Creates a new {@code AbstractEvictionController} with the given {@linkplain EvictionAction
+   * eviction action}.
+   */
+  protected AbstractEvictionController(EvictionAction evictionAction, Region region) {
+    bucketRegion = (BucketRegion) (region instanceof BucketRegion ? region : null);
+    setEvictionAction(evictionAction);
+  }
+
+  /**
+   * Force subclasses to have a reasonable {@code toString}
+   *
+   * @since GemFire 4.0
+   */
+  @Override
+  public abstract String toString();
+
+  /**
+   * Used to hook up a bucketRegion late during disk recovery.
+   */
+  @Override
+  public void setBucketRegion(Region region) {
+    if (region instanceof BucketRegion) {
+      this.bucketRegion = (BucketRegion) region;
+      this.bucketRegion.setLimit(getLimit());
+    }
+  }
+
+  /**
+   * Gets the action that is performed on the least recently used entry when it is evicted from the
+   * VM.
+   *
+   * @return one of the following constants: {@link EvictionAction#LOCAL_DESTROY},
+   *         {@link EvictionAction#OVERFLOW_TO_DISK}
+   */
+  @Override
+  public EvictionAction getEvictionAction() {
+    return this.evictionAction;
+  }
+
+  @Override
+  public synchronized EvictionStatistics getStatistics() {
+    // Synchronize with readObject/writeObject to avoid race
+    // conditions with copy sharing. See bug 31047.
+    return stats;
+  }
+
+  /**
+   * Releases resources obtained by this {@code AbstractEvictionController}
+   */
+  @Override
+  public void close() {
+    if (this.stats != null) {
+      if (bucketRegion != null) {
+        this.stats.incEvictions(bucketRegion.getEvictions() * -1);
+        this.stats.decrementCounter(bucketRegion.getCounter());
+        bucketRegion.close();
+      } else {
+        this.stats.close();
+      }
+    }
+  }
+
+  /**
+   * Returns a copy of this LRU-based eviction controller. This method is a artifact when capacity
+   * controllers were used on a {@code Region}
+   */
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    synchronized (this) {
+      AbstractEvictionController clone = (AbstractEvictionController) super.clone();
+      clone.stats = null;
+      return clone;
+    }
+  }
+
+  /**
+   * Return true if the specified capacity controller is compatible with this
+   */
+  @Override
+  public boolean equals(Object cc) {
+    if (cc == null) {
+      return false;
+    }
+    if (!getClass().isAssignableFrom(cc.getClass())) {
+      return false;
+    }
+    AbstractEvictionController other = (AbstractEvictionController) cc;
+    if (!other.evictionAction.equals(this.evictionAction)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Note that we just need to make sure that equal objects return equal hashcodes; nothing really
+   * elaborate is done here.
+   */
+  @Override
+  public int hashCode() {
+    return this.evictionAction.hashCode();
+  }
+
+  @Override
+  public long limit() {
+    if (stats == null) {
+      throw new InternalGemFireException(
+          LocalizedStrings.LRUAlgorithm_LRU_STATS_IN_EVICTION_CONTROLLER_INSTANCE_SHOULD_NOT_BE_NULL
+              .toLocalizedString());
+    }
+    if (bucketRegion != null) {
+      return bucketRegion.getLimit();
+    }
+    return stats.getLimit();
+  }
+
+  @Override
+  public EvictionStatistics initStats(Object region, StatisticsFactory statsFactory) {
+    setRegionName(region);
+    InternalEvictionStatistics stats =
+        new EvictionStatisticsImpl(statsFactory, getRegionName(), this);
+    stats.setLimit(AbstractEvictionController.this.getLimit());
+    stats.setDestroysLimit(DESTROYS_LIMIT);
+    setStatistics(stats);
+    return stats;
+  }
+
+  /**
+   * Sets the action that is performed on the least recently used entry when it is evicted from the
+   * VM.
+   *
+   * @throws IllegalArgumentException If {@code evictionAction} specifies an unknown eviction
+   *         action.
+   * @see EvictionAction
+   */
+  protected void setEvictionAction(EvictionAction evictionAction) {
+    this.evictionAction = evictionAction;
+  }
+
+  protected String getRegionName() {
+    return this.regionName;
+  }
+
+  protected void setRegionName(Object region) {
+    String fullPathName;
+    if (region instanceof Region) {
+      fullPathName = ((Region) region).getFullPath();
+    } else if (region instanceof PlaceHolderDiskRegion) {
+      PlaceHolderDiskRegion placeHolderDiskRegion = (PlaceHolderDiskRegion) region;
+      if (placeHolderDiskRegion.isBucket()) {
+        fullPathName = placeHolderDiskRegion.getPrName();
+      } else {
+        fullPathName = placeHolderDiskRegion.getName();
+      }
+    } else {
+      throw new IllegalStateException("expected Region or PlaceHolderDiskRegion");
+    }
+
+    if (this.regionName != null && !this.regionName.equals(fullPathName)) {
+      throw new IllegalArgumentException(
+          LocalizedStrings.LRUAlgorithm_LRU_EVICTION_CONTROLLER_0_ALREADY_CONTROLS_THE_CAPACITY_OF_1_IT_CANNOT_ALSO_CONTROL_THE_CAPACITY_OF_REGION_2
+              .toLocalizedString(AbstractEvictionController.this, this.regionName, fullPathName));
+    }
+    this.regionName = fullPathName; // store the name not the region since
+    // region is not fully constructed yet
+  }
+
+  protected void setStatistics(InternalEvictionStatistics stats) {
+    this.stats = stats;
+  }
+
+  private void writeObject(ObjectOutputStream out) throws IOException {
+    synchronized (this) { // See bug 31047
+      out.writeObject(this.evictionAction);
+    }
+  }
+
+  private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+    synchronized (this) { // See bug 31047
+      this.evictionAction = (EvictionAction) in.readObject();
+    }
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/AbstractEvictionList.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/AbstractEvictionList.java
new file mode 100644
index 0000000000..b9ed8580be
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/AbstractEvictionList.java
@@ -0,0 +1,212 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.versions.RegionVersionVector;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.logging.log4j.LocalizedMessage;
+import org.apache.geode.internal.logging.log4j.LogMarker;
+
+abstract class AbstractEvictionList implements EvictionList {
+  private static final Logger logger = LogService.getLogger();
+
+  /** The last node in the LRU list after which all new nodes are added */
+  protected final EvictionNode tail = new GuardNode();
+
+  /** The starting point in the LRU list for searching for the LRU node */
+  protected final EvictionNode head = new GuardNode();
+
+  /** Description of the Field */
+  protected final InternalEvictionStatistics stats;
+
+  /** Counter for the size of the LRU list */
+  private final AtomicInteger size = new AtomicInteger();
+
+  private BucketRegion bucketRegion;
+
+  AbstractEvictionList(InternalEvictionStatistics stats, BucketRegion region) {
+    if (stats == null) {
+      throw new IllegalArgumentException("EvictionStatistics must not be null");
+    }
+    this.stats = stats;
+    this.bucketRegion = region;
+    initEmptyList();
+  }
+
+  @Override
+  public int size() {
+    return size.get();
+  }
+
+  private void incrementSize() {
+    size.incrementAndGet();
+  }
+
+  void decrementSize() {
+    // Size could go negative if there is a concurrent clear and
+    // cache updates are in progress.
+    size.decrementAndGet();
+  }
+
+  @Override
+  public void closeStats() {
+    stats.close();
+  }
+
+  @Override
+  public EvictionStatistics getStatistics() {
+    return stats;
+  }
+
+  @Override
+  public void setBucketRegion(Object region) {
+    if (region instanceof BucketRegion) {
+      this.bucketRegion = (BucketRegion) region; // see bug 41388
+    }
+  }
+
+  @Override
+  public void clear(RegionVersionVector regionVersionVector) {
+    if (regionVersionVector != null) {
+      return; // when concurrency checks are enabled the clear operation removes entries iteratively
+    }
+
+    synchronized (this) {
+      if (bucketRegion != null) {
+        stats.decrementCounter(bucketRegion.getCounter());
+        bucketRegion.resetCounter();
+      } else {
+        stats.resetCounter();
+      }
+      initEmptyList();
+    }
+  }
+
+  private synchronized void initEmptyList() {
+    size.set(0);
+    head.setNext(tail);
+    tail.setPrevious(head);
+  }
+
+  /**
+   * Adds an lru node to the tail of the list.
+   */
+  @Override
+  public synchronized void appendEntry(final EvictionNode evictionNode) {
+    if (evictionNode.next() != null) {
+      // already in the list
+      return;
+    }
+
+    evictionNode.unsetRecentlyUsed();
+
+    if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+      logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
+          .create(LocalizedStrings.NewLRUClockHand_ADDING_ANODE_TO_LRU_LIST, evictionNode));
+    }
+
+    evictionNode.setNext(tail);
+    tail.previous().setNext(evictionNode);
+    evictionNode.setPrevious(tail.previous());
+    tail.setPrevious(evictionNode);
+
+    incrementSize();
+  }
+
+  @Override
+  public synchronized void destroyEntry(EvictionNode evictionNode) {
+    if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+      logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
+          .create(LocalizedStrings.NewLRUClockHand_UNLINKENTRY_CALLED, evictionNode));
+    }
+
+    if (removeEntry(evictionNode)) {
+      getStatistics().incDestroys();
+    }
+
+  }
+
+  protected synchronized boolean removeEntry(EvictionNode evictionNode) {
+    if (evictionNode.next() == null) {
+      // not in the list anymore.
+      return false;
+    }
+
+    unlinkEntry(evictionNode);
+    return true;
+  }
+
+  protected synchronized void unlinkEntry(EvictionNode evictionNode) {
+    EvictionNode next = evictionNode.next();
+    EvictionNode previous = evictionNode.previous();
+    next.setPrevious(previous);
+    previous.setNext(next);
+    evictionNode.setNext(null);
+    evictionNode.setPrevious(null);
+    decrementSize();
+  }
+
+  protected synchronized EvictionNode unlinkTailEntry() {
+    EvictionNode evictionNode = tail.previous();
+    if (evictionNode == head) {
+      return null; // end of eviction list
+    }
+
+    unlinkEntry(evictionNode);
+    return evictionNode;
+  }
+
+  /**
+   * Remove and return the head entry in the list
+   */
+  protected synchronized EvictionNode unlinkHeadEntry() {
+    EvictionNode evictionNode = head.next();
+    if (evictionNode == tail) {
+      return null; // end of list
+    }
+
+    unlinkEntry(evictionNode);
+    return evictionNode;
+  }
+
+  protected boolean isEvictable(EvictionNode evictionNode) {
+    if (evictionNode.isEvicted()) {
+      if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+        logger.trace(LogMarker.LRU_CLOCK,
+            LocalizedMessage.create(LocalizedStrings.NewLRUClockHand_DISCARDING_EVICTED_ENTRY));
+      }
+      return false;
+    }
+
+    // If this Entry is part of a transaction, skip it since
+    // eviction should not cause commit conflicts
+    synchronized (evictionNode) {
+      if (evictionNode.isInUseByTransaction()) {
+        if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+          logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage.create(
+              LocalizedStrings.NewLRUClockHand_REMOVING_TRANSACTIONAL_ENTRY_FROM_CONSIDERATION));
+        }
+        return false;
+      }
+    }
+    return true;
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/lru/LRUAlgorithmTest.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/CachedDeserializableValueWrapper.java
similarity index 55%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/lru/LRUAlgorithmTest.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/CachedDeserializableValueWrapper.java
index 2a49712789..49e661b766 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/lru/LRUAlgorithmTest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/CachedDeserializableValueWrapper.java
@@ -12,24 +12,21 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.lru;
+package org.apache.geode.internal.cache.eviction;
 
-import static org.assertj.core.api.Assertions.*;
-import static org.mockito.Mockito.*;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+/**
+ * Marker class to indicate that the wrapped value is owned by a CachedDeserializable and its form
+ * is changing from serialized to deserialized.
+ */
+public class CachedDeserializableValueWrapper {
 
-import org.apache.geode.test.junit.categories.UnitTest;
+  private final Object value;
 
-@Category(UnitTest.class)
-public class LRUAlgorithmTest {
+  public CachedDeserializableValueWrapper(Object value) {
+    this.value = value;
+  }
 
-  @Test
-  public void shouldBeMockable() throws Exception {
-    LRUAlgorithm mockLRUAlgorithm = mock(LRUAlgorithm.class);
-    EnableLRU mockEnableLRU = mock(EnableLRU.class);
-    when(mockLRUAlgorithm.getLRUHelper()).thenReturn(mockEnableLRU);
-    assertThat(mockLRUAlgorithm.getLRUHelper()).isEqualTo(mockEnableLRU);
+  public Object getValue() {
+    return value;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUCapacityController.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/CountLRUEviction.java
similarity index 59%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUCapacityController.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/CountLRUEviction.java
index 7103dc3b6a..3d7bd9ca3e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUCapacityController.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/CountLRUEviction.java
@@ -12,33 +12,41 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.lru;
-
-import java.util.*;
-
-import org.apache.geode.*;
-import org.apache.geode.cache.*;
-import org.apache.geode.internal.cache.*;
+package org.apache.geode.internal.cache.eviction;
+
+import java.util.Properties;
+
+import org.apache.geode.StatisticDescriptor;
+import org.apache.geode.StatisticsType;
+import org.apache.geode.StatisticsTypeFactory;
+import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.Declarable;
+import org.apache.geode.cache.EvictionAction;
+import org.apache.geode.cache.EvictionAttributes;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.InternalRegion;
+import org.apache.geode.internal.cache.Token;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
 /**
- * A <code>CapacityController</code> that will remove the least recently used (LRU) entry from a
- * region once the region reaches a certain capacity. The entry is locally destroyed when evicted by
- * the capacity controller.
- * <P>
+ * A {@code CapacityController} that will remove the least recently used (LRU) entry from a region
+ * once the region reaches a certain capacity. The entry is locally destroyed when evicted by the
+ * capacity controller.
+ *
+ * <p>
  * This is not supported as the capacity controller of a region with mirroring enabled.
  *
- * <P>
- * LRUCapacityController must be set in the RegionAttributes before the region is created. A Region
- * with LRUCapacityController set will throw an IllegalStateException if an attempt is made to
- * replace the Region's capacity controller. While the capacity controller cannot be replaced, it
- * does support changing the limit with the setMaximumEntries method.
+ * <p>
+ * CountLRUEviction must be set in the RegionAttributes before the region is created. A Region with
+ * CountLRUEviction set will throw an IllegalStateException if an attempt is made to replace the
+ * Region's capacity controller. While the capacity controller cannot be replaced, it does support
+ * changing the limit with the setMaximumEntries method.
  *
- * <P>
- * If you are using a <code>cache.xml</code> file to create a JCache region declaratively, you can
- * include the following to associate a <code>LRUCapacityController</code> with a region:
+ * <p>
+ * If you are using a {@code cache.xml} file to create a JCache region declaratively, you can
+ * include the following to associate a {@code CountLRUEviction} with a region:
  *
  * <pre>
  *  &lt;region-attributes&gt;
@@ -51,7 +59,7 @@
  *
  * @since GemFire 2.0.2
  */
-public class LRUCapacityController extends LRUAlgorithm implements Declarable {
+public class CountLRUEviction extends AbstractEvictionController implements Declarable {
 
   private static final long serialVersionUID = -4383074909189355938L;
 
@@ -95,21 +103,17 @@
 
   }
 
-  ////////////////////// Instance Fields /////////////////////
-
   /** The maximum number entries allowed by this controller */
   private volatile int maximumEntries;
 
-  /////////////////////// Constructors ///////////////////////
-
   /**
    * Creates an LRU capacity controller that allows the {@link #DEFAULT_MAXIMUM_ENTRIES default}
    * maximum number of entries and the
    * {@link org.apache.geode.cache.EvictionAction#DEFAULT_EVICTION_ACTION default} eviction action.
    *
-   * @see #LRUCapacityController(int,Region)
+   * @see #CountLRUEviction(int,Region)
    */
-  public LRUCapacityController(Region region) {
+  public CountLRUEviction(Region region) {
     this(DEFAULT_MAXIMUM_ENTRIES, EvictionAction.DEFAULT_EVICTION_ACTION, region);
   }
 
@@ -118,16 +122,16 @@ public LRUCapacityController(Region region) {
    * default eviction action.
    *
    * @param maximumEntries The maximum number of entries allowed in the region whose capacity this
-   *        controller controls. Once there are <code>capacity</code> entries in a region, this
+   *        controller controls. Once there are {@code capacity} entries in a region, this
    *        controller will remove the least recently used entry.<br>
    *        <p>
    *        For a region with {@link DataPolicy#PARTITION}, the maximum number of entries allowed in
    *        the region, collectively for its primary buckets and redundant copies for this VM. After
-   *        there are <code>capacity</code> entries in the region's primary buckets and redundant
-   *        copies for this VM, this controller will remove the least recently used entry from the
-   *        bucket in which the subsequent <code>put</code> takes place.
+   *        there are {@code capacity} entries in the region's primary buckets and redundant copies
+   *        for this VM, this controller will remove the least recently used entry from the bucket
+   *        in which the subsequent {@code put} takes place.
    */
-  public LRUCapacityController(int maximumEntries, Region region) {
+  public CountLRUEviction(int maximumEntries, Region region) {
     this(maximumEntries, EvictionAction.DEFAULT_EVICTION_ACTION, region);
   }
 
@@ -135,18 +139,18 @@ public LRUCapacityController(int maximumEntries, Region region) {
    * Creates an LRU capacity controller that allows the given number of maximum entries.
    *
    * @param maximumEntries The maximum number of entries allowed in the region whose capacity this
-   *        controller controls. Once there are <code>capacity</code> entries in a region, this
+   *        controller controls. Once there are {@code capacity} entries in a region, this
    *        controller will remove the least recently used entry.<br>
    *        <p>
    *        For a region with {@link DataPolicy#PARTITION}, the maximum number of entries allowed in
    *        the region, collectively for its primary buckets and redundant copies for this VM. After
-   *        there are <code>capacity</code> entries in the region's primary buckets and redundant
-   *        copies for this VM, this controller will remove the least recently used entry from the
-   *        bucket in which the subsequent <code>put</code> takes place.
+   *        there are {@code capacity} entries in the region's primary buckets and redundant copies
+   *        for this VM, this controller will remove the least recently used entry from the bucket
+   *        in which the subsequent {@code put} takes place.
    * @param evictionAction The action to perform upon the least recently used entry. See
    *        {@link #EVICTION_ACTION}.
    */
-  public LRUCapacityController(int maximumEntries, EvictionAction evictionAction, Region region) {
+  public CountLRUEviction(int maximumEntries, EvictionAction evictionAction, Region region) {
     super(evictionAction, region);
     setMaximumEntries(maximumEntries);
   }
@@ -176,17 +180,18 @@ public void setLimit(int max) {
   ////////////////////// Instance Methods /////////////////////
 
   /**
-   * Because an <code>LRUCapacityController</code> is {@link Declarable}, it can be initialized with
+   * Because an {@code LRUCapacityController} is {@link Declarable}, it can be initialized with
    * properties. The {@link #MAXIMUM_ENTRIES "maximum-entries"} (case-sensitive) property can be
    * used to specify the capacity allowed by this controller. Other properties in props are ignored.
    * The {@link #EVICTION_ACTION "eviction-action"} property specifies the action to be taken when
    * the region has reached its capacity.
    *
-   * @throws NumberFormatException The <code>maximum-entries</code> property cannot be parsed as an
+   * @throws NumberFormatException The {@code maximum-entries} property cannot be parsed as an
    *         integer
-   * @throws IllegalArgumentException The value of the <code>eviction-action</code> property is not
+   * @throws IllegalArgumentException The value of the {@code eviction-action} property is not
    *         recoginzed.
    */
+  @Override
   public void init(Properties props) throws NumberFormatException {
     String prop = null;
     if ((prop = props.getProperty(MAXIMUM_ENTRIES)) != null) {
@@ -198,102 +203,94 @@ public void init(Properties props) throws NumberFormatException {
     }
   }
 
-  // Candidate for removal since capacity controller no longer part of
-  // cache.xml
   @Override
-  public Properties getProperties() {
-    Properties props = new Properties();
-    if (this.evictionAction != EvictionAction.DEFAULT_EVICTION_ACTION) {
-      props.setProperty(EVICTION_ACTION, this.evictionAction.toString());
+  public long getLimit() {
+    return this.maximumEntries;
+  }
+
+  /**
+   * Indicate what kind of {@code AbstractEvictionController} this helper implements
+   */
+  @Override
+  public org.apache.geode.cache.EvictionAlgorithm getEvictionAlgorithm() {
+    return org.apache.geode.cache.EvictionAlgorithm.LRU_ENTRY;
+  }
+
+  /**
+   * All entries for the LRUCapacityController are considered to be of size 1.
+   */
+  @Override
+  public int entrySize(Object key, Object value) throws IllegalArgumentException {
+
+    if (Token.isRemoved(value) /* && (value != Token.TOMBSTONE) */) { // un-comment to make
+                                                                      // tombstones visible
+      // bug #42228 - lruEntryDestroy removes an entry from the LRU, but if
+      // it is subsequently resurrected we want the new entry to generate a delta
+      return 0;
     }
-    if (this.maximumEntries != DEFAULT_MAXIMUM_ENTRIES) {
-      props.setProperty(MAXIMUM_ENTRIES, String.valueOf(this.maximumEntries));
+    if ((value == null /* overflow to disk */ || value == Token.INVALID
+        || value == Token.LOCAL_INVALID) && getEvictionAction().isOverflowToDisk()) {
+      // Don't count this guys toward LRU
+      return 0;
+
+    } else {
+      return 1;
     }
-    return props;
   }
 
   @Override
-  public long getLimit() {
-    return this.maximumEntries;
+  public StatisticsType getStatisticsType() {
+    return statType;
+  }
+
+  @Override
+  public String getStatisticsName() {
+    return "LRUStatistics";
+  }
+
+  @Override
+  public int getLimitStatId() {
+    return statType.nameToId("entriesAllowed");
+  }
+
+  @Override
+  public int getCountStatId() {
+    return statType.nameToId("entryCount");
+  }
+
+  @Override
+  public int getEvictionsStatId() {
+    return statType.nameToId("lruEvictions");
+  }
+
+  @Override
+  public int getDestroysStatId() {
+    return statType.nameToId("lruDestroys");
+  }
+
+  @Override
+  public int getDestroysLimitStatId() {
+    return statType.nameToId("lruDestroysLimit");
+  }
+
+  @Override
+  public int getEvaluationsStatId() {
+    return statType.nameToId("lruEvaluations");
+  }
+
+  @Override
+  public int getGreedyReturnsStatId() {
+    return statType.nameToId("lruGreedyReturns");
+  }
+
+  @Override
+  public boolean mustEvict(EvictionStatistics stats, InternalRegion region, int delta) {
+    return stats.getCounter() + delta > stats.getLimit();
   }
 
   @Override
-  protected EnableLRU createLRUHelper() {
-    return new AbstractEnableLRU() {
-
-      /**
-       * Indicate what kind of <code>EvictionAlgorithm</code> this helper implements
-       */
-      public EvictionAlgorithm getEvictionAlgorithm() {
-        return EvictionAlgorithm.LRU_ENTRY;
-      }
-
-      /**
-       * All entries for the LRUCapacityController are considered to be of size 1.
-       */
-      public int entrySize(Object key, Object value) throws IllegalArgumentException {
-
-        if (Token.isRemoved(value) /* && (value != Token.TOMBSTONE) */) { // un-comment to make
-                                                                          // tombstones visible
-          // bug #42228 - lruEntryDestroy removes an entry from the LRU, but if
-          // it is subsequently resurrected we want the new entry to generate a delta
-          return 0;
-        }
-        if ((value == null /* overflow to disk */ || value == Token.INVALID
-            || value == Token.LOCAL_INVALID) && getEvictionAction().isOverflowToDisk()) {
-          // Don't count this guys toward LRU
-          return 0;
-
-        } else {
-          return 1;
-        }
-      }
-
-      public StatisticsType getStatisticsType() {
-        return statType;
-      }
-
-      public String getStatisticsName() {
-        return "LRUStatistics";
-      }
-
-      public int getLimitStatId() {
-        return statType.nameToId("entriesAllowed");
-      }
-
-      public int getCountStatId() {
-        return statType.nameToId("entryCount");
-      }
-
-      public int getEvictionsStatId() {
-        return statType.nameToId("lruEvictions");
-      }
-
-      public int getDestroysStatId() {
-        return statType.nameToId("lruDestroys");
-      }
-
-      public int getDestroysLimitStatId() {
-        return statType.nameToId("lruDestroysLimit");
-      }
-
-      public int getEvaluationsStatId() {
-        return statType.nameToId("lruEvaluations");
-      }
-
-      public int getGreedyReturnsStatId() {
-        return statType.nameToId("lruGreedyReturns");
-      }
-
-      public boolean mustEvict(LRUStatistics stats, Region region, int delta) {
-        return stats.getCounter() + delta > stats.getLimit();
-      }
-
-      @Override
-      public boolean lruLimitExceeded(LRUStatistics lruStatistics, DiskRegionView drv) {
-        return lruStatistics.getCounter() > lruStatistics.getLimit();
-      }
-    };
+  public boolean lruLimitExceeded(EvictionStatistics stats, DiskRegionView diskRegionView) {
+    return stats.getCounter() > stats.getLimit();
   }
 
 
@@ -301,9 +298,9 @@ public boolean lruLimitExceeded(LRUStatistics lruStatistics, DiskRegionView drv)
   public boolean equals(Object cc) {
     if (!super.equals(cc))
       return false;
-    if (!(cc instanceof LRUCapacityController))
+    if (!(cc instanceof CountLRUEviction))
       return false;
-    LRUCapacityController other = (LRUCapacityController) cc;
+    CountLRUEviction other = (CountLRUEviction) cc;
     if (this.maximumEntries != other.maximumEntries)
       return false;
     return true;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/DisabledEvictionStatistics.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/DisabledEvictionStatistics.java
new file mode 100644
index 0000000000..84bf2438f4
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/DisabledEvictionStatistics.java
@@ -0,0 +1,105 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import org.apache.geode.Statistics;
+
+class DisabledEvictionStatistics implements InternalEvictionStatistics {
+
+  @Override
+  public void close() {
+    // nothing
+  }
+
+  @Override
+  public void setLimit(long newValue) {
+    // nothing
+  }
+
+  @Override
+  public void setDestroysLimit(long newValue) {
+    // nothing
+  }
+
+  @Override
+  public long getDestroysLimit() {
+    return 0;
+  }
+
+  @Override
+  public void resetCounter() {
+    // nothing
+  }
+
+  @Override
+  public void decrementCounter(long delta) {
+    // nothing
+  }
+
+  @Override
+  public void incEvictions(long delta) {
+    // nothing
+  }
+
+  @Override
+  public long getDestroys() {
+    return 0;
+  }
+
+  @Override
+  public void incEvaluations(long evaluations) {
+    // nothing
+  }
+
+  @Override
+  public void incGreedyReturns(long greedyReturns) {
+    // nothing
+  }
+
+  @Override
+  public void incEvictions() {
+    // nothing
+  }
+
+  @Override
+  public long getCounter() {
+    return 0;
+  }
+
+  @Override
+  public long getLimit() {
+    return 0;
+  }
+
+  @Override
+  public void updateCounter(long delta) {
+    // nothing
+  }
+
+  @Override
+  public long getEvictions() {
+    return 0;
+  }
+
+  @Override
+  public Statistics getStats() {
+    return null;
+  }
+
+  @Override
+  public void incDestroys() {
+    // nothing
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUEntry.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictableEntry.java
similarity index 86%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUEntry.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictableEntry.java
index 6ba00952f0..9c0436073e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUEntry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictableEntry.java
@@ -12,15 +12,15 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.lru;
+package org.apache.geode.internal.cache.eviction;
 
 import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 
 /**
- * Represents an entry in an LRU map
+ * Represents an entry in a Region map implementing eviction
  */
-public interface LRUEntry extends LRUClockNode, RegionEntry {
+public interface EvictableEntry extends EvictionNode, RegionEntry {
   /**
    * If the key is stored as an Object then returns that object; but if the key is stored as
    * primitives then returns null.
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUMapCallbacks.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictableMap.java
similarity index 67%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUMapCallbacks.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictableMap.java
index de9a6890fd..790601c6f4 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUMapCallbacks.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictableMap.java
@@ -12,56 +12,56 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.lru;
+package org.apache.geode.internal.cache.eviction;
 
-import org.apache.geode.internal.cache.RegionEntry;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
 
 /**
- * The lru action on the map for evicting items must be called while the current thread is free of
- * any map synchronizations.
+ * A RegionMap should implement EvictableMap if it needs to support Eviction.
+ *
+ * <p>
+ * The eviction action on the map for evicting items must be called while the current thread is free
+ * of any map synchronizations.
  */
-public interface LRUMapCallbacks {
+public interface EvictableMap {
 
   /**
    * to be called by LocalRegion after any synchronization surrounding a map.put or map.replace call
-   * is made. This will then perform the lru removals, or region.localDestroy() calls to make up for
-   * the recent addition.
+   * is made. This will then perform the eviction removals, or region.localDestroy() calls to make
+   * up for the recent addition.
    */
-  public void lruUpdateCallback();
-
-  public void lruUpdateCallback(int n);
+  void lruUpdateCallback();
 
   /**
    * Disables lruUpdateCallback in calling thread
    *
    * @return false if it's already disabled
    */
-  public boolean disableLruUpdateCallback();
+  boolean disableLruUpdateCallback();
 
   /**
    * Enables lruUpdateCallback in calling thread
    */
-  public void enableLruUpdateCallback();
+  void enableLruUpdateCallback();
 
   /**
    * if an exception occurs between an LRUEntriesMap put and the call to lruUpdateCallback, then
    * this must be called to allow the thread to continue to work with other regions.
    */
-  public void resetThreadLocals();
+  void resetThreadLocals();
 
   /**
    * Return true if the lru has exceeded its limit and needs to evict. Note that this method is
    * currently used to prevent disk recovery from faulting in values once the limit is exceeded.
    *
-   * @param drv the disk region whose limit is checked
+   * @param diskRegionView the disk region whose limit is checked
    */
-  public boolean lruLimitExceeded(DiskRegionView drv);
+  boolean lruLimitExceeded(DiskRegionView diskRegionView);
 
-  public void lruCloseStats();
+  void lruCloseStats();
 
   /**
    * Called when an entry is faulted in from disk.
    */
-  public void lruEntryFaultIn(LRUEntry entry);
+  void lruEntryFaultIn(EvictableEntry entry);
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/EnableLRU.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionController.java
similarity index 65%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/lru/EnableLRU.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionController.java
index 69f7e8b533..7743a7044a 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/EnableLRU.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionController.java
@@ -12,103 +12,110 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
-package org.apache.geode.internal.cache.lru;
+package org.apache.geode.internal.cache.eviction;
 
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.StatisticsType;
+import org.apache.geode.cache.CacheCallback;
 import org.apache.geode.cache.EvictionAction;
 import org.apache.geode.cache.EvictionAlgorithm;
 import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
 
 /**
- * Marker interface to eviction controller that determines if LRU list maintainance is required.
+ * Marker interface to eviction controller that determines if LRU list maintenance is required.
  */
-public interface EnableLRU {
+public interface EvictionController extends CacheCallback {
 
   /**
    * return the size of an entry or its worth when constraining the size of an LRU EntriesMap.
    */
-  public int entrySize(Object key, Object value) throws IllegalArgumentException;
+  int entrySize(Object key, Object value) throws IllegalArgumentException;
 
   /**
    * return the limit for the accumulated entrySize which triggers disposal.
    */
-  public long limit();
+  long limit();
 
   /** setup stats for this LRU type, if reset is true, initialize counter on stats to zero. */
-  public LRUStatistics initStats(Object region, StatisticsFactory sf);
+  EvictionStatistics initStats(Object region, StatisticsFactory statsFactory);
 
   /** return the eviction controller instance this came from */
-  public EvictionAlgorithm getEvictionAlgorithm();
+  EvictionAlgorithm getEvictionAlgorithm();
 
   /** return the stats object for this eviction controller */
-  public LRUStatistics getStats();
+  EvictionStatistics getStatistics();
 
   /**
-   * Returns the {@linkplain EvictionAction action} to take when the LRU entry is evicted.
+   * Returns the {@code EvictionAction} to take when the LRU entry is evicted.
    */
-  public EvictionAction getEvictionAction();
+  EvictionAction getEvictionAction();
 
   /**
    * Returns the statistics for this LRU algorithm
    */
-  public StatisticsType getStatisticsType();
+  StatisticsType getStatisticsType();
 
   /**
    * Returns the name of the statistics for this LRU algorithm
    */
-  public String getStatisticsName();
+  String getStatisticsName();
 
   /**
    * Returns the id of the "limit" statistic for this LRU algorithm's statistics
    */
-  public int getLimitStatId();
+  int getLimitStatId();
 
   /**
    * Returns the id of the "counter" statistic for this LRU algorithm's statistics.
    */
-  public int getCountStatId();
+  int getCountStatId();
 
   /**
    * Returns the id of the "evictions" statistic for this LRU algorithm's statistics.
    */
-  public int getEvictionsStatId();
+  int getEvictionsStatId();
 
   /**
    * Returns the id of the "destroys" statistic for this LRU algorithm's statistics.
    */
-  public int getDestroysStatId();
+  int getDestroysStatId();
 
   /**
    * Returns the id of the "destroysLimit" statistic for this LRU algorithm's statistics
    */
-  public int getDestroysLimitStatId();
+  int getDestroysLimitStatId();
 
   /**
    * Returns the id of the "evaluations" statistic for this LRU algorithm's statistics.
    */
-  public int getEvaluationsStatId();
+  int getEvaluationsStatId();
 
   /**
-   * Returns the id of the "greedyReturns" statistic for this LRU algorith'ms statistics
+   * Returns the id of the "greedyReturns" statistic for this LRU algorithm's statistics
    *
    * @return the id
    */
-  public int getGreedyReturnsStatId();
+  int getGreedyReturnsStatId();
 
   /**
    * Returns whether or not there is enough room to accommodate data of the given size based on the
-   * given <code>LRUStatistics</code>.
+   * given {@code EvictionStatistics}.
    */
-  public boolean mustEvict(LRUStatistics stats, Region region, int delta);
+  boolean mustEvict(EvictionStatistics stats, InternalRegion region, int delta);
+
+  boolean lruLimitExceeded(EvictionStatistics stats, DiskRegionView diskRegionView);
+
+  void setBucketRegion(Region region);
 
   /**
-   * Envoked after an entry has been evicted
+   * Returns the "limit" as defined by this LRU algorithm
    */
-  public void afterEviction();
-
-  public boolean lruLimitExceeded(LRUStatistics lruStatistics, DiskRegionView drv);
+  long getLimit();
 
+  /**
+   * Set the limiting parameter used to determine when eviction is needed.
+   */
+  void setLimit(int maximum);
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionList.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionList.java
new file mode 100644
index 0000000000..6aa19650e5
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionList.java
@@ -0,0 +1,59 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import org.apache.geode.internal.cache.versions.RegionVersionVector;
+
+public interface EvictionList {
+
+  void setBucketRegion(Object region);
+
+  void closeStats();
+
+  /**
+   * Adds a new eviction node for the entry between the current tail and head of the list.
+   */
+  void appendEntry(EvictionNode evictionNode);
+
+  /**
+   * Returns the Entry that is considered least recently used. The entry will no longer be in the
+   * pipe (unless it is the last empty marker).
+   */
+  EvictableEntry getEvictableEntry();
+
+  /**
+   * remove an entry from the list
+   */
+  void destroyEntry(EvictionNode evictionNode);
+
+  /**
+   * Get the modifier for lru based statistics.
+   *
+   * @return The EvictionStatistics for this Clock hand's region.
+   */
+  EvictionStatistics getStatistics();
+
+  /**
+   * called when an LRU map is cleared... resets stats and releases prev and next.
+   */
+  void clear(RegionVersionVector regionVersionVector);
+
+  /**
+   * Returns the number of EvictionNodes in the EvictionList.
+   */
+  int size();
+
+  void incrementRecentlyUsed();
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionListBuilder.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionListBuilder.java
new file mode 100644
index 0000000000..677be499d1
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionListBuilder.java
@@ -0,0 +1,102 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import org.apache.geode.StatisticsFactory;
+import org.apache.geode.cache.EvictionAlgorithm;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalRegionArguments;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
+import org.apache.geode.internal.lang.SystemPropertyHelper;
+
+public class EvictionListBuilder {
+
+  private final boolean EVICTION_SCAN_ASYNC =
+      SystemPropertyHelper.getProductBooleanProperty(SystemPropertyHelper.EVICTION_SCAN_ASYNC);
+
+  private EvictionAlgorithm algorithm;
+  private Object region;
+  private EvictionController controller;
+  private InternalRegionArguments args;
+
+  public EvictionListBuilder(EvictionAlgorithm algorithm) {
+    this.algorithm = algorithm;
+  }
+
+  /**
+   * @param region PlaceHolderDiskRegion during disk recovery or LocalRegion
+   */
+  public EvictionListBuilder withRegion(Object region) {
+    this.region = region;
+    return this;
+  }
+
+  public EvictionListBuilder withEvictionController(EvictionController evictionController) {
+    this.controller = evictionController;
+    return this;
+  }
+
+  public EvictionListBuilder withArgs(InternalRegionArguments args) {
+    this.args = args;
+    return this;
+  }
+
+  public EvictionList create() {
+    if (algorithm.isLIFO()) {
+      return new LIFOList(getEvictionStats(), getBucketRegion());
+    } else {
+      if (EVICTION_SCAN_ASYNC) {
+        return new LRUListWithAsyncSorting(getEvictionStats(), getBucketRegion());
+      } else {
+        return new LRUListWithSyncSorting(getEvictionStats(), getBucketRegion());
+      }
+    }
+  }
+
+  private InternalEvictionStatistics getEvictionStats() {
+    InternalEvictionStatistics statistics = null;
+    if (region != null) {
+      if (region instanceof BucketRegion) {
+        if (args != null && args.getPartitionedRegion() != null) {
+          statistics = (InternalEvictionStatistics) args.getPartitionedRegion()
+              .getEvictionController().getStatistics();
+        } else {
+          statistics = new DisabledEvictionStatistics();
+        }
+      } else if (region instanceof PlaceHolderDiskRegion) {
+        statistics = (InternalEvictionStatistics) ((PlaceHolderDiskRegion) region).getPRLRUStats();
+      } else if (region instanceof PartitionedRegion) {
+        statistics = (InternalEvictionStatistics) ((PartitionedRegion) region)
+            .getPRLRUStatsDuringInitialization();
+        if (statistics != null) {
+          PartitionedRegion partitionedRegion = (PartitionedRegion) region;
+          EvictionController evictionController = partitionedRegion.getEvictionController();
+          ((AbstractEvictionController) evictionController).setStatistics(statistics);
+        }
+      }
+    }
+    if (statistics == null) {
+      StatisticsFactory sf = GemFireCacheImpl.getExisting("").getDistributedSystem();
+      statistics = (InternalEvictionStatistics) controller.initStats(region, sf);
+    }
+    return statistics;
+  }
+
+  private BucketRegion getBucketRegion() {
+    return region instanceof BucketRegion ? (BucketRegion) region : null;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUClockNode.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionNode.java
similarity index 63%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUClockNode.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionNode.java
index e7836265ca..cb6d676f15 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUClockNode.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionNode.java
@@ -12,21 +12,22 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.internal.cache.eviction;
 
-package org.apache.geode.internal.cache.lru;
+import org.apache.geode.internal.cache.RegionEntryContext;
 
-public interface LRUClockNode {
+public interface EvictionNode {
 
-  public void setNextLRUNode(LRUClockNode next);
+  void setNext(EvictionNode next);
 
-  public void setPrevLRUNode(LRUClockNode prev);
+  void setPrevious(EvictionNode previous);
 
-  public LRUClockNode nextLRUNode();
+  EvictionNode next();
 
-  public LRUClockNode prevLRUNode();
+  EvictionNode previous();
 
   /** compute the new entry size and return the delta from the previous entry size */
-  public int updateEntrySize(EnableLRU ccHelper);
+  int updateEntrySize(EvictionController ccHelper);
 
   /**
    * compute the new entry size and return the delta from the previous entry size
@@ -34,19 +35,21 @@
    * @param value then entry's value
    * @since GemFire 6.1.2.9
    */
-  public int updateEntrySize(EnableLRU ccHelper, Object value);
+  int updateEntrySize(EvictionController ccHelper, Object value);
 
-  public int getEntrySize();
+  int getEntrySize();
 
-  public boolean testRecentlyUsed();
+  boolean isRecentlyUsed();
 
-  public void setRecentlyUsed();
+  void setRecentlyUsed(RegionEntryContext context);
 
-  public void unsetRecentlyUsed();
+  void unsetRecentlyUsed();
 
-  public void setEvicted();
+  void setEvicted();
 
-  public void unsetEvicted();
+  void unsetEvicted();
 
-  public boolean testEvicted();
+  boolean isEvicted();
+
+  boolean isInUseByTransaction();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionStatistics.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionStatistics.java
new file mode 100755
index 0000000000..11d0fe4f50
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionStatistics.java
@@ -0,0 +1,34 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import org.apache.geode.Statistics;
+
+public interface EvictionStatistics {
+
+  void incEvictions();
+
+  long getCounter();
+
+  long getLimit();
+
+  void updateCounter(long delta);
+
+  long getEvictions();
+
+  Statistics getStats();
+
+  void incDestroys();
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUStatistics.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionStatisticsImpl.java
old mode 100755
new mode 100644
similarity index 69%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUStatistics.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionStatisticsImpl.java
index 04504fd145..e50733f75d
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUStatistics.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/EvictionStatisticsImpl.java
@@ -12,40 +12,36 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
-package org.apache.geode.internal.cache.lru;
+package org.apache.geode.internal.cache.eviction;
 
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsFactory;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.Assert;
 
-/**
- * Statistics for both the LocalLRUClockHand. Note that all its instance fields are
- * <code>final</code>. Thus, we do not need to worry about refreshing an instance when it resides in
- * shared memory.
- */
-public class LRUStatistics {
+class EvictionStatisticsImpl implements InternalEvictionStatistics {
 
   /** The Statistics object that we delegate most behavior to */
   private final Statistics stats;
-  protected int limitId;
+  private final int limitId;
+
   /**
    * the number of destroys that must occur before a list scan is initiated to to remove unlinked
    * entries.
    */
-  protected int destroysLimitId;
-  protected int counterId;
+  private final int destroysLimitId;
+  private final int counterId;
+
   /** entries that have been evicted from the LRU list */
-  protected int evictionsId;
+  private final int evictionsId;
+
   /** entries that have been destroyed, but not yet evicted from the LRU list */
-  protected int destroysId;
-  protected int evaluationsId;
-  protected int greedyReturnsId;
+  private final int destroysId;
+  private final int evaluationsId;
+  private final int greedyReturnsId;
 
-  // Note: the following atomics have been added so that the LRU code
+  // Note: the following atomics have been added so that the eviction code
   // does not depend on the value of a statistic for its operations.
   // In particular they optimize the "get" methods for these items.
   // Striped stats optimize inc but cause set and get to be more expensive.
@@ -55,17 +51,12 @@
   private final AtomicLong destroys = new AtomicLong();
   private final AtomicLong evictions = new AtomicLong();
 
-  ///////////////////////// Constructors /////////////////////////
-
-  /**
-   * Constructor for the LRUStatistics object
-   *
-   * @param name Description of the Parameter
-   */
-  public LRUStatistics(StatisticsFactory factory, String name, EnableLRU helper) {
+  public EvictionStatisticsImpl(StatisticsFactory factory, String name, EvictionController helper) {
     String statName = helper.getStatisticsName() + "-" + name;
     stats = factory.createAtomicStatistics(helper.getStatisticsType(), statName);
-    if (!helper.getEvictionAlgorithm().isLRUHeap()) {
+    if (helper.getEvictionAlgorithm().isLRUHeap()) {
+      limitId = 0;
+    } else {
       limitId = helper.getLimitStatId();
     }
     destroysLimitId = helper.getDestroysLimitStatId();
@@ -76,27 +67,53 @@ public LRUStatistics(StatisticsFactory factory, String name, EnableLRU helper) {
     this.greedyReturnsId = helper.getGreedyReturnsStatId();
   }
 
-  public LRUStatistics(StatisticsFactory factory, String name, StatisticsType statisticsType) {
-    stats = factory.createAtomicStatistics(statisticsType, name);
-    limitId = 0;
-    destroysLimitId = 0;
-    counterId = 0;
-    evictionsId = 0;
-    destroysId = 0;
-    this.evaluationsId = 0;
-    this.greedyReturnsId = 0;
-  }
-
-  public void close() {
-    stats.close();
+  @Override
+  public void incEvictions() {
+    this.evictions.getAndAdd(1);
+    stats.incLong(evictionsId, 1);
   }
 
-  /** common counter for different lru types */
+  /** common counter for different eviction types */
+  @Override
   public long getCounter() {
     return this.counter.get();
   }
 
-  /** limit */
+  @Override
+  public long getLimit() {
+    return this.limit.get();
+  }
+
+  @Override
+  public void updateCounter(long delta) {
+    if (delta != 0) {
+      this.counter.getAndAdd(delta);
+      stats.incLong(counterId, delta);
+    }
+  }
+
+  @Override
+  public long getEvictions() {
+    return this.evictions.get();
+  }
+
+  @Override
+  public Statistics getStats() {
+    return this.stats;
+  }
+
+  @Override
+  public void incDestroys() {
+    this.destroys.getAndAdd(1);
+    stats.incLong(destroysId, 1);
+  }
+
+  @Override
+  public void close() {
+    stats.close();
+  }
+
+  @Override
   public void setLimit(long newValue) {
     Assert.assertTrue(newValue > 0L,
         "limit must be positive, an attempt was made to set it to: " + newValue);
@@ -108,6 +125,7 @@ public void setLimit(long newValue) {
   }
 
   /** destroy limit */
+  @Override
   public void setDestroysLimit(long newValue) {
     Assert.assertTrue(newValue > 0L,
         "destroys limit must be positive, an attempt was made to set it to: " + newValue);
@@ -118,21 +136,12 @@ public void setDestroysLimit(long newValue) {
     }
   }
 
-  public long getLimit() {
-    return this.limit.get();
-  }
-
+  @Override
   public long getDestroysLimit() {
     return this.destroysLimit.get();
   }
 
-  public void updateCounter(long delta) {
-    if (delta != 0) {
-      this.counter.getAndAdd(delta);
-      stats.incLong(counterId, delta);
-    }
-  }
-
+  @Override
   public void resetCounter() {
     if (this.counter.get() != 0) {
       this.counter.set(0);
@@ -140,14 +149,7 @@ public void resetCounter() {
     }
   }
 
-  // public void setCounter(long newValue) {
-  // long oldValue = this.counter.get();
-  // if (oldValue != newValue) {
-  // this.counter.set(oldValue+newValue);
-  // stats.setLong(counterId, newValue);
-  // }
-  // }
-  //
+  @Override
   public void decrementCounter(long delta) {
     if (delta != 0) {
       this.counter.addAndGet(-delta);
@@ -155,47 +157,24 @@ public void decrementCounter(long delta) {
     }
   }
 
-  public void incEvictions() {
-    this.evictions.getAndAdd(1);
-    stats.incLong(evictionsId, 1);
-  }
-
+  @Override
   public void incEvictions(long delta) {
     this.evictions.getAndAdd(delta);
     stats.incLong(evictionsId, delta);
   }
 
-  public long getEvictions() {
-    return this.evictions.get();
-  }
-
-  public void resetDestroys() {
-    if (this.destroys.get() != 0) {
-      this.destroys.set(0);
-      stats.setLong(destroysId, 0);
-    }
-  }
-
-  public void incDestroys() {
-    this.destroys.getAndAdd(1);
-    stats.incLong(destroysId, 1);
-  }
-
+  @Override
   public long getDestroys() {
     return this.destroys.get();
   }
 
-  public void incEvaluations(long numEvals) {
-    stats.incLong(evaluationsId, numEvals);
+  @Override
+  public void incEvaluations(long evaluations) {
+    stats.incLong(evaluationsId, evaluations);
   }
 
-  public void incGreedyReturns(long numEvals) {
-    stats.incLong(greedyReturnsId, numEvals);
+  @Override
+  public void incGreedyReturns(long greedyReturns) {
+    stats.incLong(greedyReturnsId, greedyReturns);
   }
-
-
-  public Statistics getStats() {
-    return this.stats;
-  }
-
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/GuardNode.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/GuardNode.java
new file mode 100644
index 0000000000..b830314984
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/GuardNode.java
@@ -0,0 +1,94 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import org.apache.geode.internal.cache.RegionEntryContext;
+
+class GuardNode implements EvictionNode {
+
+  private EvictionNode next;
+
+  private EvictionNode previous;
+
+  @Override
+  public int getEntrySize() {
+    return 0;
+  }
+
+  @Override
+  public EvictionNode next() {
+    return next;
+  }
+
+  @Override
+  public EvictionNode previous() {
+    return previous;
+  }
+
+  @Override
+  public void setEvicted() {
+    // nothing
+  }
+
+  @Override
+  public void setNext(EvictionNode next) {
+    this.next = next;
+  }
+
+  @Override
+  public void setPrevious(EvictionNode previous) {
+    this.previous = previous;
+  }
+
+  @Override
+  public void setRecentlyUsed(RegionEntryContext context) {
+    // nothing
+  }
+
+  @Override
+  public boolean isEvicted() {
+    return false;
+  }
+
+  @Override
+  public boolean isRecentlyUsed() {
+    return false;
+  }
+
+  @Override
+  public void unsetEvicted() {
+    // nothing
+  }
+
+  @Override
+  public void unsetRecentlyUsed() {
+    // nothing
+  }
+
+  @Override
+  public int updateEntrySize(EvictionController ccHelper) {
+    return 0;
+  }
+
+  @Override
+  public int updateEntrySize(EvictionController ccHelper, Object value) {
+    return 0;
+  }
+
+  @Override
+  public boolean isInUseByTransaction() {
+    return false;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapEvictor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapEvictor.java
similarity index 54%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapEvictor.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapEvictor.java
index 212ba5cdd9..3040ba57e2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapEvictor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapEvictor.java
@@ -12,21 +12,35 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.lru;
+package org.apache.geode.internal.cache.eviction;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.geode.distributed.internal.DistributionConfig.GEMFIRE_PREFIX;
 
 import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 import java.util.concurrent.*;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
 import org.apache.logging.log4j.Logger;
 
-import org.apache.geode.LogWriter;
-import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.OverflowQueueWithDMStats;
-import org.apache.geode.internal.cache.*;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
@@ -45,167 +59,169 @@
 public class HeapEvictor implements ResourceListener<MemoryEvent> {
   private static final Logger logger = LogService.getLogger();
 
-
   // Add 1 for the management task that's putting more eviction tasks on the queue
-  public static final int MAX_EVICTOR_THREADS = Integer.getInteger(
-      DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.MAX_EVICTOR_THREADS",
-      (Runtime.getRuntime().availableProcessors() * 4)) + 1;
-
-  public static final boolean DISABLE_HEAP_EVICTIOR_THREAD_POOL =
-      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX
-          + "HeapLRUCapacityController.DISABLE_HEAP_EVICTIOR_THREAD_POOL");
+  public static final int MAX_EVICTOR_THREADS =
+      Integer.getInteger(GEMFIRE_PREFIX + "HeapLRUCapacityController.MAX_EVICTOR_THREADS",
+          Runtime.getRuntime().availableProcessors() * 4) + 1;
 
   public static final boolean EVICT_HIGH_ENTRY_COUNT_BUCKETS_FIRST =
-      Boolean
-          .valueOf(System.getProperty(DistributionConfig.GEMFIRE_PREFIX
-              + "HeapLRUCapacityController.evictHighEntryCountBucketsFirst", "true"))
-          .booleanValue();
+      Boolean.valueOf(System.getProperty(
+          GEMFIRE_PREFIX + "HeapLRUCapacityController.evictHighEntryCountBucketsFirst", "true"));
 
-  public static final int MINIMUM_ENTRIES_PER_BUCKET = Integer.getInteger(
-      DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.inlineEvictionThreshold", 0);
+  public static final int MINIMUM_ENTRIES_PER_BUCKET =
+      Integer.getInteger(GEMFIRE_PREFIX + "HeapLRUCapacityController.inlineEvictionThreshold", 0);
 
-  public static final long TOTAL_BYTES_TO_EVICT_FROM_HEAP;
+  public static final int BUCKET_SORTING_INTERVAL = Integer.getInteger(
+      GEMFIRE_PREFIX + "HeapLRUCapacityController.higherEntryCountBucketCalculationInterval", 100);
 
-  public static final int BUCKET_SORTING_INTERVAL =
-      Integer
-          .getInteger(DistributionConfig.GEMFIRE_PREFIX
-              + "HeapLRUCapacityController.higherEntryCountBucketCalculationInterval", 100)
-          .intValue();
+  private static final boolean DISABLE_HEAP_EVICTOR_THREAD_POOL = Boolean
+      .getBoolean(GEMFIRE_PREFIX + "HeapLRUCapacityController.DISABLE_HEAP_EVICTOR_THREAD_POOL");
+
+  private static final long TOTAL_BYTES_TO_EVICT_FROM_HEAP = setTotalBytesToEvictFromHeap();
 
   private static final String EVICTOR_THREAD_GROUP_NAME = "EvictorThreadGroup";
 
   private static final String EVICTOR_THREAD_NAME = "EvictorThread";
 
-  static {
-    float evictionBurstPercentage = Float.parseFloat(System.getProperty(
-        DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.evictionBurstPercentage",
-        "0.4"));
-    long maxTenuredBytes = HeapMemoryMonitor.getTenuredPoolMaxMemory();
-    TOTAL_BYTES_TO_EVICT_FROM_HEAP = (long) (maxTenuredBytes * 0.01 * evictionBurstPercentage);
-  }
-
-  private ThreadPoolExecutor evictorThreadPool = null;
+  private final Object evictionLock = new Object();
 
-  private AtomicBoolean mustEvict = new AtomicBoolean(false);
+  private final AtomicBoolean mustEvict = new AtomicBoolean(false);
 
-  protected final InternalCache cache;
+  private final List<Integer> testTaskSetSizes = new ArrayList<>();
 
-  private final ArrayList<Integer> testTaskSetSizes = new ArrayList<>();
-  public volatile int testAbortAfterLoopCount = Integer.MAX_VALUE;
+  private final ThreadPoolExecutor evictorThreadPool;
 
-  private BlockingQueue<Runnable> poolQueue;
+  private final InternalCache cache;
 
   private final AtomicBoolean isRunning = new AtomicBoolean(true);
 
-  public HeapEvictor(InternalCache gemFireCache) {
-    this.cache = gemFireCache;
-    initializeEvictorThreadPool();
+  private volatile int testAbortAfterLoopCount = Integer.MAX_VALUE;
+
+  /*
+   * Since the amount of memory used is to a large degree dependent upon when garbage collection is
+   * run, it's difficult to determine when to stop evicting. So, an initial calculation is done to
+   * determine the number of evictions that are likely needed in order to bring memory usage below
+   * the eviction threshold. This number is stored in 'numFastLoops' and we quickly loop through
+   * this number performing evictions. We then continue to evict, but at a progressively slower rate
+   * waiting either for an event which indicates we've dropped below the eviction threshold or
+   * another eviction event with an updated "number of bytes used". If we get another eviction event
+   * with an updated "number of bytes used" then 'numFastLoops' is recalculated and we start over.
+   */
+  private volatile int numEvictionLoopsCompleted = 0;
+  private volatile int numFastLoops;
+
+  public HeapEvictor(final InternalCache cache) {
+    this(cache, EVICTOR_THREAD_GROUP_NAME, EVICTOR_THREAD_NAME);
+  }
+
+  public HeapEvictor(final InternalCache cache, final String threadGroupName,
+      final String threadName) {
+    this.cache = cache;
+
+    ThreadGroup evictorThreadGroup = LoggingThreadGroup.createThreadGroup(threadGroupName, logger);
+    ThreadFactory evictorThreadFactory = new ThreadFactory() {
+      private final AtomicInteger next = new AtomicInteger(0);
+
+      @Override
+      public Thread newThread(Runnable r) {
+        Thread thread = new Thread(evictorThreadGroup, r, threadName + next.incrementAndGet());
+        thread.setDaemon(true);
+        return thread;
+      }
+    };
+
+    if (!DISABLE_HEAP_EVICTOR_THREAD_POOL) {
+      BlockingQueue<Runnable> poolQueue =
+          new OverflowQueueWithDMStats(this.cache.getCachePerfStats().getEvictionQueueStatHelper());
+      this.evictorThreadPool = new ThreadPoolExecutor(MAX_EVICTOR_THREADS, MAX_EVICTOR_THREADS, 15,
+          SECONDS, poolQueue, evictorThreadFactory);
+    } else {
+      // disabled
+      this.evictorThreadPool = null;
+    }
+  }
+
+  protected InternalCache cache() {
+    return this.cache;
   }
 
   protected boolean includePartitionedRegion(PartitionedRegion region) {
-    return (region.getEvictionAttributes().getAlgorithm().isLRUHeap()
-        && (region.getDataStore() != null) && !region.getAttributes().getOffHeap());
+    return region.getEvictionAttributes().getAlgorithm().isLRUHeap()
+        && region.getDataStore() != null && !region.getAttributes().getOffHeap();
   }
 
   protected boolean includeLocalRegion(LocalRegion region) {
-    return (region.getEvictionAttributes().getAlgorithm().isLRUHeap()
-        && !region.getAttributes().getOffHeap());
+    return region.getEvictionAttributes().getAlgorithm().isLRUHeap()
+        && !region.getAttributes().getOffHeap();
   }
 
   private List<LocalRegion> getAllRegionList() {
-    List<LocalRegion> allRegionList = new ArrayList<LocalRegion>();
-    InternalResourceManager irm = (InternalResourceManager) cache.getResourceManager();
+    List<LocalRegion> allRegionsList = new ArrayList<>();
+    InternalResourceManager resourceManager = (InternalResourceManager) cache.getResourceManager();
 
-    for (ResourceListener<MemoryEvent> listener : irm.getResourceListeners(getResourceType())) {
+    for (ResourceListener<MemoryEvent> listener : resourceManager
+        .getResourceListeners(getResourceType())) {
       if (listener instanceof PartitionedRegion) {
-        PartitionedRegion pr = (PartitionedRegion) listener;
-        if (includePartitionedRegion(pr)) {
-          allRegionList.addAll(pr.getDataStore().getAllLocalBucketRegions());
+        PartitionedRegion partitionedRegion = (PartitionedRegion) listener;
+        if (includePartitionedRegion(partitionedRegion)) {
+          allRegionsList.addAll(partitionedRegion.getDataStore().getAllLocalBucketRegions());
         }
       } else if (listener instanceof LocalRegion) {
         LocalRegion lr = (LocalRegion) listener;
         if (includeLocalRegion(lr)) {
-          allRegionList.add(lr);
+          allRegionsList.add(lr);
         }
       }
     }
 
     if (HeapEvictor.MINIMUM_ENTRIES_PER_BUCKET > 0) {
-      Iterator<LocalRegion> iter = allRegionList.iterator();
-      while (iter.hasNext()) {
-        LocalRegion lr = iter.next();
-        if (lr instanceof BucketRegion) {
-          if (((BucketRegion) lr).getNumEntriesInVM() <= HeapEvictor.MINIMUM_ENTRIES_PER_BUCKET) {
-            iter.remove();
+      Iterator<LocalRegion> iterator = allRegionsList.iterator();
+      while (iterator.hasNext()) {
+        LocalRegion region = iterator.next();
+        if (region instanceof BucketRegion) {
+          if (((BucketRegion) region)
+              .getNumEntriesInVM() <= HeapEvictor.MINIMUM_ENTRIES_PER_BUCKET) {
+            iterator.remove();
           }
         }
       }
     }
-    return allRegionList;
+    return allRegionsList;
   }
 
   private List<LocalRegion> getAllSortedRegionList() {
     List<LocalRegion> allRegionList = getAllRegionList();
 
     // Capture the sizes so that they do not change while sorting
-    final Object2LongOpenHashMap sizes = new Object2LongOpenHashMap(allRegionList.size());
-    for (LocalRegion r : allRegionList) {
-      long size = r instanceof BucketRegion ? ((BucketRegion) r).getSizeForEviction() : r.size();
-      sizes.put(r, size);
+    final Object2LongOpenHashMap<LocalRegion> sizes =
+        new Object2LongOpenHashMap<>(allRegionList.size());
+    for (LocalRegion region : allRegionList) {
+      long size = region instanceof BucketRegion ? ((BucketRegion) region).getSizeForEviction()
+          : region.size();
+      sizes.put(region, size);
     }
 
     // Sort with respect to other PR buckets also in case of multiple PRs
-    Collections.sort(allRegionList, new Comparator<LocalRegion>() {
-      public int compare(LocalRegion r1, LocalRegion r2) {
-        long numEntries1 = sizes.get(r1);
-        long numEntries2 = sizes.get(r2);
-        if (numEntries1 > numEntries2) {
-          return -1;
-        } else if (numEntries1 < numEntries2) {
-          return 1;
-        }
-        return 0;
+    allRegionList.sort((region1, region2) -> {
+      long numEntries1 = sizes.get(region1);
+      long numEntries2 = sizes.get(region2);
+      if (numEntries1 > numEntries2) {
+        return -1;
+      } else if (numEntries1 < numEntries2) {
+        return 1;
       }
+      return 0;
     });
     return allRegionList;
   }
 
-  public InternalCache getGemFireCache() {
-    return (InternalCache) this.cache;
-  }
-
-  private void initializeEvictorThreadPool() {
-
-    final ThreadGroup evictorThreadGroup =
-        LoggingThreadGroup.createThreadGroup(getEvictorThreadGroupName(), logger);
-    ThreadFactory evictorThreadFactory = new ThreadFactory() {
-      private int next = 0;
-
-      public Thread newThread(Runnable command) {
-        Thread t = new Thread(evictorThreadGroup, command, getEvictorThreadName() + next++);
-        t.setDaemon(true);
-        return t;
-      }
-    };
-
-    if (!DISABLE_HEAP_EVICTIOR_THREAD_POOL) {
-      this.poolQueue = new OverflowQueueWithDMStats(
-          getGemFireCache().getCachePerfStats().getEvictionQueueStatHelper());
-      this.evictorThreadPool = new ThreadPoolExecutor(MAX_EVICTOR_THREADS, MAX_EVICTOR_THREADS, 15,
-          TimeUnit.SECONDS, this.poolQueue, evictorThreadFactory);
-    }
-  }
-
-  /**
-   * The task(i.e the region on which eviction needs to be performed) is assigned to the threadpool.
-   */
   private void executeInThreadPool(Runnable task) {
     try {
       evictorThreadPool.execute(task);
-    } catch (RejectedExecutionException ex) {
+    } catch (RejectedExecutionException e) {
       // ignore rejection if evictor no longer running
       if (isRunning()) {
-        throw ex;
+        throw e;
       }
     }
   }
@@ -217,41 +233,32 @@ public ThreadPoolExecutor getEvictorThreadPool() {
     return null;
   }
 
-  /**
-   * returns the total number of tasks that are currently being executed or queued for execution
-   *
-   * @return sum of scheduled and running tasks
-   */
-  public int getRunningAndScheduledTasks() {
-    if (isRunning()) {
-      return this.evictorThreadPool.getActiveCount() + this.evictorThreadPool.getQueue().size();
-    }
-    return -1;
-  }
-
   private void createAndSubmitWeightedRegionEvictionTasks() {
     List<LocalRegion> allRegionList = getAllSortedRegionList();
-    float numEntriesInVm = 0;
-    for (LocalRegion lr : allRegionList) {
-      if (lr instanceof BucketRegion) {
-        numEntriesInVm = numEntriesInVm + ((BucketRegion) lr).getSizeForEviction();
+    float numEntriesInVM = 0;
+    for (LocalRegion region : allRegionList) {
+      if (region instanceof BucketRegion) {
+        numEntriesInVM += ((BucketRegion) region).getSizeForEviction();
       } else {
-        numEntriesInVm = numEntriesInVm + lr.getRegionMap().sizeInVM();
+        numEntriesInVM += region.getRegionMap().sizeInVM();
       }
     }
-    for (LocalRegion lr : allRegionList) {
-      List<LocalRegion> regionsForSingleTask = new ArrayList<LocalRegion>(1);
-      float regionEntryCnt = 0;
-      if (lr instanceof BucketRegion) {
-        regionEntryCnt = ((BucketRegion) lr).getSizeForEviction();
+
+    for (LocalRegion region : allRegionList) {
+      float regionEntryCount;
+      if (region instanceof BucketRegion) {
+        regionEntryCount = ((BucketRegion) region).getSizeForEviction();
       } else {
-        regionEntryCnt = lr.getRegionMap().sizeInVM();
+        regionEntryCount = region.getRegionMap().sizeInVM();
       }
-      float percentage = (regionEntryCnt / numEntriesInVm);
+
+      float percentage = regionEntryCount / numEntriesInVM;
       long bytesToEvictPerTask = (long) (getTotalBytesToEvict() * percentage);
-      regionsForSingleTask.add(lr);
+      List<LocalRegion> regionsForSingleTask = new ArrayList<>(1);
+      regionsForSingleTask.add(region);
       if (mustEvict()) {
-        executeInThreadPool(new RegionEvictorTask(regionsForSingleTask, this, bytesToEvictPerTask));
+        executeInThreadPool(new RegionEvictorTask(cache.getCachePerfStats(), regionsForSingleTask,
+            this, bytesToEvictPerTask));
       } else {
         break;
       }
@@ -263,21 +270,24 @@ private void createAndSubmitWeightedRegionEvictionTasks() {
     if (pool == null) {
       return Collections.emptySet();
     }
+
     int threadsAvailable = pool.getCorePoolSize();
     long bytesToEvictPerTask = getTotalBytesToEvict() / threadsAvailable;
     List<LocalRegion> allRegionList = getAllRegionList();
     if (allRegionList.isEmpty()) {
       return Collections.emptySet();
     }
+
     // This shuffling is not required when eviction triggered for the first time
     Collections.shuffle(allRegionList);
     int allRegionSetSize = allRegionList.size();
     Set<RegionEvictorTask> evictorTaskSet = new HashSet<>();
     if (allRegionSetSize <= threadsAvailable) {
       for (LocalRegion region : allRegionList) {
-        List<LocalRegion> regionList = new ArrayList<LocalRegion>(1);
+        List<LocalRegion> regionList = new ArrayList<>(1);
         regionList.add(region);
-        RegionEvictorTask task = new RegionEvictorTask(regionList, this, bytesToEvictPerTask);
+        RegionEvictorTask task =
+            new RegionEvictorTask(cache.getCachePerfStats(), regionList, this, bytesToEvictPerTask);
         evictorTaskSet.add(task);
       }
       for (RegionEvictorTask regionEvictorTask : evictorTaskSet) {
@@ -285,23 +295,26 @@ private void createAndSubmitWeightedRegionEvictionTasks() {
       }
       return evictorTaskSet;
     }
+
     int numRegionsInTask = allRegionSetSize / threadsAvailable;
     List<LocalRegion> regionsForSingleTask = null;
-    Iterator<LocalRegion> itr = allRegionList.iterator();
+    Iterator<LocalRegion> regionIterator = allRegionList.iterator();
     for (int i = 0; i < threadsAvailable; i++) {
+      regionsForSingleTask = new ArrayList<>(numRegionsInTask);
       int count = 1;
-      regionsForSingleTask = new ArrayList<LocalRegion>(numRegionsInTask);
       while (count <= numRegionsInTask) {
-        if (itr.hasNext()) {
-          regionsForSingleTask.add(itr.next());
+        if (regionIterator.hasNext()) {
+          regionsForSingleTask.add(regionIterator.next());
         }
         count++;
       }
-      evictorTaskSet.add(new RegionEvictorTask(regionsForSingleTask, this, bytesToEvictPerTask));
+      evictorTaskSet.add(new RegionEvictorTask(cache.getCachePerfStats(), regionsForSingleTask,
+          this, bytesToEvictPerTask));
     }
+
     // Add leftover regions to last task
-    while (itr.hasNext()) {
-      regionsForSingleTask.add(itr.next());
+    while (regionIterator.hasNext()) {
+      regionsForSingleTask.add(regionIterator.next());
     }
 
     for (RegionEvictorTask regionEvictorTask : evictorTaskSet) {
@@ -310,25 +323,9 @@ private void createAndSubmitWeightedRegionEvictionTasks() {
     return evictorTaskSet;
   }
 
-  // Since the amount of memory used is to a large degree dependent upon when
-  // garbage collection is run, it's difficult to determine when to stop
-  // evicting. So, an initial calculation is done to determine the number of
-  // evictions that are likely needed in order to bring memory usage below the
-  // eviction threshold. This number is stored in 'numFastLoops' and we
-  // quickly loop through this number performing evictions. We then continue
-  // to evict, but at a progressively slower rate waiting either for an event
-  // which indicates we've dropped below the eviction threshold or another
-  // eviction event with an updated "number of bytes used". If we get another
-  // eviction event with an updated "number of bytes used" then 'numFastLoops'
-  // is recalculated and we start over.
-  protected volatile int numEvictionLoopsCompleted = 0;
-  protected volatile int numFastLoops;
-  private long previousBytesUsed;
-  private final Object evictionLock = new Object();
-
   @Override
   public void onEvent(final MemoryEvent event) {
-    if (DISABLE_HEAP_EVICTIOR_THREAD_POOL) {
+    if (DISABLE_HEAP_EVICTOR_THREAD_POOL) {
       return;
     }
 
@@ -336,13 +333,10 @@ public void onEvent(final MemoryEvent event) {
     // in this VM ...
     if (isRunning() && event.isLocal()) {
       if (event.getState().isEviction()) {
-        final LogWriter logWriter = cache.getLogger();
-
         // Have we previously received an eviction event and already started eviction ...
-        if (this.mustEvict.get() == true) {
-          if (logWriter.fineEnabled()) {
-            logWriter.fine("Updating eviction in response to memory event: " + event
-                + ". previousBytesUsed=" + previousBytesUsed);
+        if (this.mustEvict.get()) {
+          if (logger.isDebugEnabled()) {
+            logger.debug("Updating eviction in response to memory event: {}", event);
           }
 
           // We lock here to make sure that the thread that was previously
@@ -370,8 +364,8 @@ public void onEvent(final MemoryEvent event) {
         numFastLoops =
             (int) ((event.getBytesUsed() - event.getThresholds().getEvictionThresholdClearBytes()
                 + getTotalBytesToEvict()) / getTotalBytesToEvict());
-        if (logWriter.fineEnabled()) {
-          logWriter.fine("Starting eviction in response to memory event: " + event);
+        if (logger.isDebugEnabled()) {
+          logger.debug("Starting eviction in response to memory event: {}", event);
         }
 
         // The new thread which will run in a loop performing evictions
@@ -379,7 +373,7 @@ public void onEvent(final MemoryEvent event) {
           @Override
           public void run() {
             // Has the test hook been set which will cause eviction to abort early
-            if (numEvictionLoopsCompleted < testAbortAfterLoopCount) {
+            if (numEvictionLoopsCompleted < getTestAbortAfterLoopCount()) {
               try {
                 // Submit tasks into the queue to do the evictions
                 if (EVICT_HIGH_ENTRY_COUNT_BUCKETS_FIRST) {
@@ -389,16 +383,15 @@ public void run() {
                     executeInThreadPool(task);
                   }
                 }
-                RegionEvictorTask.setLastTaskCompletionTime(System.currentTimeMillis());
 
                 // Make sure that another thread isn't processing a new eviction event
                 // and changing the number of fast loops to perform.
                 synchronized (evictionLock) {
                   int delayTime = getEvictionLoopDelayTime();
-                  if (logWriter.fineEnabled()) {
-                    logWriter.fine("Eviction loop delay time calculated to be " + delayTime
-                        + " milliseconds. Fast Loops=" + numFastLoops + ", Loop #="
-                        + numEvictionLoopsCompleted + 1);
+                  if (logger.isDebugEnabled()) {
+                    logger.debug(
+                        "Eviction loop delay time calculated to be {} milliseconds. Fast Loops={}, Loop #={}",
+                        delayTime, numFastLoops, numEvictionLoopsCompleted + 1);
                   }
                   numEvictionLoopsCompleted++;
                   try {
@@ -406,7 +399,7 @@ public void run() {
                     // needed can be updated by another thread processing a new
                     // eviction event.
                     evictionLock.wait(delayTime);
-                  } catch (InterruptedException iex) {
+                  } catch (InterruptedException ignored) {
                     // Loop and try again
                   }
                 }
@@ -417,7 +410,7 @@ public void run() {
                   // another pass at eviction.
                   executeInThreadPool(this);
                 }
-              } catch (RegionDestroyedException e) {
+              } catch (RegionDestroyedException ignored) {
                 // A region destroyed exception might be thrown for Region.size() when a bucket
                 // moves due to rebalancing. retry submitting the eviction task without
                 // logging an error message. fixes bug 48162
@@ -440,7 +433,7 @@ public void run() {
 
   protected int getEvictionLoopDelayTime() {
     int delayTime = 850; // The waiting period when running fast loops
-    if ((numEvictionLoopsCompleted - numFastLoops) > 2) {
+    if (numEvictionLoopsCompleted - numFastLoops > 2) {
       delayTime = 3000; // Way below the threshold
     } else if (numEvictionLoopsCompleted >= numFastLoops) {
       delayTime = (numEvictionLoopsCompleted - numFastLoops + 3) * 500; // Just below the threshold
@@ -449,7 +442,7 @@ protected int getEvictionLoopDelayTime() {
     return delayTime;
   }
 
-  public boolean mustEvict() {
+  boolean mustEvict() {
     return this.mustEvict.get();
   }
 
@@ -459,24 +452,17 @@ public void close() {
     }
   }
 
-  public boolean isRunning() {
+  private boolean isRunning() {
     return isRunning.get();
   }
 
-  public ArrayList<Integer> testOnlyGetSizeOfTasks() {
-    if (isRunning())
+  public List<Integer> testOnlyGetSizeOfTasks() {
+    if (isRunning()) {
       return testTaskSetSizes;
+    }
     return null;
   }
 
-  protected String getEvictorThreadGroupName() {
-    return HeapEvictor.EVICTOR_THREAD_GROUP_NAME;
-  }
-
-  protected String getEvictorThreadName() {
-    return HeapEvictor.EVICTOR_THREAD_NAME;
-  }
-
   public long getTotalBytesToEvict() {
     return TOTAL_BYTES_TO_EVICT_FROM_HEAP;
   }
@@ -484,4 +470,27 @@ public long getTotalBytesToEvict() {
   protected ResourceType getResourceType() {
     return ResourceType.HEAP_MEMORY;
   }
+
+  private int getTestAbortAfterLoopCount() {
+    return testAbortAfterLoopCount;
+  }
+
+  public void setTestAbortAfterLoopCount(int testAbortAfterLoopCount) {
+    this.testAbortAfterLoopCount = testAbortAfterLoopCount;
+  }
+
+  int numEvictionLoopsCompleted() {
+    return this.numEvictionLoopsCompleted;
+  }
+
+  int numFastLoops() {
+    return this.numFastLoops;
+  }
+
+  private static long setTotalBytesToEvictFromHeap() {
+    float evictionBurstPercentage = Float.parseFloat(System
+        .getProperty(GEMFIRE_PREFIX + "HeapLRUCapacityController.evictionBurstPercentage", "0.4"));
+    long maxTenuredBytes = HeapMemoryMonitor.getTenuredPoolMaxMemory();
+    return (long) (maxTenuredBytes * 0.01 * evictionBurstPercentage);
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapLRUController.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapLRUController.java
new file mode 100644
index 0000000000..9ce8956f8d
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapLRUController.java
@@ -0,0 +1,231 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import static org.apache.geode.distributed.internal.DistributionConfig.GEMFIRE_PREFIX;
+
+import org.apache.geode.StatisticDescriptor;
+import org.apache.geode.StatisticsFactory;
+import org.apache.geode.StatisticsType;
+import org.apache.geode.StatisticsTypeFactory;
+import org.apache.geode.cache.EvictionAction;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.util.ObjectSizer;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.InternalRegion;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.Token;
+import org.apache.geode.internal.cache.control.InternalResourceManager;
+import org.apache.geode.internal.cache.persistence.DiskRegionView;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
+
+/**
+ * A {@code HeapLRUController} controls the contents of {@link Region} based on the percentage of
+ * memory that is currently being used. If the percentage of memory in use exceeds the given
+ * percentage, then the least recently used entry of the region is evicted.
+ *
+ * <p>
+ * For heap regions: We have found that {@code HeapLRUController} is most useful on a JVM that is
+ * launched with both the {@code -Xmx} and {@code -Xms} switches used. Many java virtual machine
+ * implementations have additional JVM switches to control the behavior of the garbage collector. We
+ * suggest that you investigate tuning the garbage collector when using {@code HeapLRUController}.
+ * In particular, we have found that when running with Sun's HotSpot JVM, the
+ * {@code -XX:+UseConcMarkSweepGC} and {@code -XX:+UseParNewGC} options improve the behavior of
+ * {@code HeapLRUController}.
+ *
+ * @since GemFire 3.2
+ */
+public class HeapLRUController extends SizeLRUController {
+  private static final long serialVersionUID = 4970685814429530675L;
+
+  /**
+   * The default percentage of VM heap usage over which LRU eviction occurs
+   */
+  public static final String TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY =
+      GEMFIRE_PREFIX + "topUpHeapEvictionPercentage";
+
+  private static final int PER_ENTRY_OVERHEAD = 250;
+
+  private int perEntryOverhead = PER_ENTRY_OVERHEAD;
+
+  private static final StatisticsType statType;
+
+  static {
+    // create the stats type for MemLRU.
+    StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+
+    final String entryBytesDesc =
+        "The amount of memory currently used by regions configured for eviction.";
+    final String lruEvictionsDesc = "Number of total entry evictions triggered by LRU.";
+    final String lruDestroysDesc =
+        "Number of entries destroyed in the region through both destroy cache operations and eviction. Reset to zero each time it exceeds lruDestroysLimit.";
+    final String lruDestroysLimitDesc =
+        "Maximum number of entry destroys triggered by LRU before scan occurs.";
+    final String lruEvaluationsDesc = "Number of entries evaluated during LRU operations.";
+    final String lruGreedyReturnsDesc = "Number of non-LRU entries evicted during LRU operations";
+
+    statType = f.createType("HeapLRUStatistics",
+        "Statistics about byte based Least Recently Used region entry disposal",
+        new StatisticDescriptor[] {f.createLongGauge("entryBytes", entryBytesDesc, "bytes"),
+            f.createLongCounter("lruEvictions", lruEvictionsDesc, "entries"),
+            f.createLongCounter("lruDestroys", lruDestroysDesc, "entries"),
+            f.createLongGauge("lruDestroysLimit", lruDestroysLimitDesc, "entries"),
+            f.createLongCounter("lruEvaluations", lruEvaluationsDesc, "entries"),
+            f.createLongCounter("lruGreedyReturns", lruGreedyReturnsDesc, "entries"),});
+  }
+
+  public HeapLRUController(EvictionAction evictionAction, Region region, ObjectSizer sizer) {
+    super(evictionAction, region, sizer);
+  }
+
+  @Override
+  public void setLimit(int maximum) {
+    // nothing
+  }
+
+  @Override
+  public long getLimit() {
+    return 0;
+  }
+
+  /**
+   * Returns a brief description of this eviction controller.
+   *
+   * @since GemFire 4.0
+   */
+  @Override
+  public String toString() {
+    return LocalizedStrings.HeapLRUCapacityController_HEAPLRUCAPACITYCONTROLLER_WITH_A_CAPACITY_OF_0_OF_HEAP_AND_AN_THREAD_INTERVAL_OF_1_AND_EVICTION_ACTION_2
+        .toLocalizedString(getLimit(), getEvictionAction());
+  }
+
+  /**
+   * Indicate what kind of {@code AbstractEvictionController} this helper implements
+   */
+  @Override
+  public org.apache.geode.cache.EvictionAlgorithm getEvictionAlgorithm() {
+    return org.apache.geode.cache.EvictionAlgorithm.LRU_HEAP;
+  }
+
+  /**
+   * As far as we're concerned all entries have the same size
+   */
+  @Override
+  public int entrySize(Object key, Object value) throws IllegalArgumentException {
+    // value is null only after eviction occurs. A change in size is
+    // required for eviction stats, bug 30974
+
+    if (value == Token.TOMBSTONE) {
+      return 0;
+    }
+
+    int size = HeapLRUController.this.getPerEntryOverhead();
+    size += sizeof(key);
+    size += sizeof(value);
+    return size;
+  }
+
+  @Override
+  public EvictionStatistics initStats(Object region, StatisticsFactory statsFactory) {
+    setRegionName(region);
+    InternalEvictionStatistics stats =
+        new EvictionStatisticsImpl(statsFactory, getRegionName(), this);
+    setStatistics(stats);
+    return stats;
+  }
+
+  @Override
+  public StatisticsType getStatisticsType() {
+    return statType;
+  }
+
+  @Override
+  public String getStatisticsName() {
+    return "HeapLRUStatistics";
+  }
+
+  @Override
+  public int getLimitStatId() {
+    throw new UnsupportedOperationException("Limit not used with this LRU type");
+  }
+
+  @Override
+  public int getCountStatId() {
+    return statType.nameToId("entryBytes");
+  }
+
+  @Override
+  public int getEvictionsStatId() {
+    return statType.nameToId("lruEvictions");
+  }
+
+  @Override
+  public int getDestroysStatId() {
+    return statType.nameToId("lruDestroys");
+  }
+
+  @Override
+  public int getDestroysLimitStatId() {
+    return statType.nameToId("lruDestroysLimit");
+  }
+
+  @Override
+  public int getEvaluationsStatId() {
+    return statType.nameToId("lruEvaluations");
+  }
+
+  @Override
+  public int getGreedyReturnsStatId() {
+    return statType.nameToId("lruGreedyReturns");
+  }
+
+  /**
+   * Okay, deep breath. Instead of basing the LRU calculation on the number of entries in the region
+   * or on their "size" (which turned out to be incorrectly estimated in the general case), we use
+   * the amount of memory currently in use. If the amount of memory current in use
+   * {@linkplain Runtime#maxMemory} - {@linkplain Runtime#freeMemory} is greater than the overflow
+   * threshold, then we evict the LRU entry.
+   */
+  @Override
+  public boolean mustEvict(EvictionStatistics stats, InternalRegion region, int delta) {
+    InternalCache cache = (InternalCache) region.getRegionService();
+    boolean offheap = region.getAttributes().getOffHeap();
+    boolean shouldEvict =
+        cache.getInternalResourceManager().getMemoryMonitor(offheap).getState().isEviction();
+
+    if (region instanceof BucketRegion) {
+      return shouldEvict && ((BucketRegion) region).getSizeForEviction() > 0;
+    }
+    return shouldEvict && ((LocalRegion) region).getRegionMap().sizeInVM() > 0;
+  }
+
+  @Override
+  public boolean lruLimitExceeded(EvictionStatistics stats, DiskRegionView diskRegionView) {
+    InternalResourceManager resourceManager =
+        diskRegionView.getDiskStore().getCache().getInternalResourceManager();
+    return resourceManager.getMemoryMonitor(diskRegionView.getOffHeap()).getState().isEviction();
+  }
+
+
+  public int getPerEntryOverhead() {
+    return perEntryOverhead;
+  }
+
+  public void setEntryOverHead(int entryOverHead) {
+    this.perEntryOverhead = entryOverHead;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/InternalEvictionStatistics.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/InternalEvictionStatistics.java
new file mode 100644
index 0000000000..9f48571e32
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/InternalEvictionStatistics.java
@@ -0,0 +1,39 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+interface InternalEvictionStatistics extends EvictionStatistics {
+
+  void close();
+
+  void setLimit(long newValue);
+
+  /** destroy limit */
+  void setDestroysLimit(long newValue);
+
+  long getDestroysLimit();
+
+  long getDestroys();
+
+  void resetCounter();
+
+  void decrementCounter(long delta);
+
+  void incEvictions(long delta);
+
+  void incEvaluations(long evaluations);
+
+  void incGreedyReturns(long greedyReturns);
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/LIFOList.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/LIFOList.java
new file mode 100755
index 0000000000..8c726736b1
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/LIFOList.java
@@ -0,0 +1,66 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import org.apache.geode.internal.cache.BucketRegion;
+
+/**
+ * NewLIFOClockHand holds the behavior for LIFO logic , Overwriting getLRUEntry() to return most
+ * recently added Entry.
+ *
+ * @since GemFire 5.7
+ */
+public class LIFOList extends AbstractEvictionList {
+
+  LIFOList(InternalEvictionStatistics stats, BucketRegion bucketRegion) {
+    super(stats, bucketRegion);
+  }
+
+  /**
+   * Return the Entry that is considered most recently used and available to be evicted to overflow.
+   * Note that this implementation basically just returns the most recent thing added to the list.
+   * So, unlike the parent class, is does no scanning based on the recentlyUsed bit. This is a
+   * perfect implementation for our queues (gateway, client subscription) as long as they never
+   * update something already in the queue. Since updates simply set the recentlyUsed bit then the
+   * most recent node may be the one that was just updated and not moved to the tail of the list.
+   */
+  @Override
+  public EvictableEntry getEvictableEntry() {
+    long evaluations = 0;
+    EvictionNode evictionNode;
+    // search for entry to return from list
+    for (;;) {
+      evictionNode = unlinkTailEntry();
+      // end of Lifo list stop searching
+      if (evictionNode == null) {
+        break;
+      }
+      evaluations++;
+      synchronized (evictionNode) {
+        // if entry NOT used by transaction and NOT evicted return entry
+        if (!evictionNode.isInUseByTransaction() && !evictionNode.isEvicted()) {
+          break;
+        }
+      }
+    }
+    stats.incEvaluations(evaluations);
+    return (EvictableEntry) evictionNode;
+  }
+
+  @Override
+  public void incrementRecentlyUsed() {
+    // nothing
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/LRUListWithAsyncSorting.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/LRUListWithAsyncSorting.java
new file mode 100644
index 0000000000..419b0565a7
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/LRUListWithAsyncSorting.java
@@ -0,0 +1,219 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.RegionEntry;
+import org.apache.geode.internal.cache.RegionEntryContext;
+import org.apache.geode.internal.cache.versions.RegionVersionVector;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.lang.SystemPropertyHelper;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.logging.log4j.LocalizedMessage;
+import org.apache.geode.internal.logging.log4j.LogMarker;
+
+/**
+ * LRUListWithAsyncSorting holds the eviction list, and the behavior for maintaining the list and
+ * determining the next entry to be removed. Each EntriesMap that supports LRU holds one of these.
+ * Evicts are always done from the head and assume that it is the least recent entry unless if is
+ * being used by a transaction or is already evicted in which case it is removed from the list and
+ * the next item is evicted. Adds are always done to the end of the list and should not be marked
+ * recently used. An async scanner runs periodically (how often TBD), head to tail, removing entries
+ * that have been recently used, marking them as not recently used, and adding them back to the
+ * tail. Removes may unlink entries from the list.
+ */
+public class LRUListWithAsyncSorting extends AbstractEvictionList {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private static final Optional<Integer> EVICTION_SCAN_MAX_THREADS = SystemPropertyHelper
+      .getProductIntegerProperty(SystemPropertyHelper.EVICTION_SCAN_MAX_THREADS);
+
+  private static final ExecutorService SINGLETON_EXECUTOR = createExecutor();
+
+  private static final int DEFAULT_EVICTION_SCAN_THRESHOLD_PERCENT = 25;
+
+  private static final int MAX_EVICTION_ATTEMPTS = 10;
+
+  private final AtomicInteger recentlyUsedCounter = new AtomicInteger();
+
+  private final double scanThreshold;
+
+  private Future<?> currentScan;
+
+  private final ExecutorService executor;
+
+  private static ExecutorService createExecutor() {
+    int threads = EVICTION_SCAN_MAX_THREADS.orElse(0);
+    if (threads < 1) {
+      threads = Math.max((Runtime.getRuntime().availableProcessors() / 4), 1);
+    }
+    // TODO need ThreadGroup that handles thread naming and
+    // unhandled exception logging
+    return Executors.newFixedThreadPool(threads);
+  }
+
+  LRUListWithAsyncSorting(InternalEvictionStatistics stats, BucketRegion region) {
+    this(stats, region, SINGLETON_EXECUTOR);
+  }
+
+  LRUListWithAsyncSorting(InternalEvictionStatistics stats, BucketRegion region,
+      ExecutorService executor) {
+    super(stats, region);
+    this.scanThreshold = calculateScanThreshold();
+    this.executor = executor;
+  }
+
+  private double calculateScanThreshold() {
+    Optional<Integer> configuredThresholdPercent = SystemPropertyHelper
+        .getProductIntegerProperty(SystemPropertyHelper.EVICTION_SCAN_THRESHOLD_PERCENT);
+
+    int thresholdPercent =
+        configuredThresholdPercent.orElse(DEFAULT_EVICTION_SCAN_THRESHOLD_PERCENT);
+    if (thresholdPercent < 0 || thresholdPercent > 100) {
+      thresholdPercent = DEFAULT_EVICTION_SCAN_THRESHOLD_PERCENT;
+    }
+
+    return (double) thresholdPercent / 100;
+  }
+
+  @Override
+  public void clear(RegionVersionVector regionVersionVector) {
+    super.clear(regionVersionVector);
+    recentlyUsedCounter.set(0);
+  }
+
+  /**
+   * Remove and return the Entry that is considered least recently used.
+   */
+  @Override
+  public EvictableEntry getEvictableEntry() {
+    int evictionAttempts = 0;
+    for (;;) {
+      final EvictionNode evictionNode = unlinkHeadEntry();
+
+      if (evictionNode == null) {
+        // hit the end of the list
+        return null;
+      }
+
+      if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+        logger.trace(LogMarker.LRU_CLOCK, "lru considering {}", evictionNode);
+      }
+
+      if (!isEvictable(evictionNode)) {
+        continue;
+      }
+
+      if (evictionNode.isRecentlyUsed() && evictionAttempts < MAX_EVICTION_ATTEMPTS) {
+        evictionAttempts++;
+        appendEntry(evictionNode);
+        continue;
+      }
+
+      if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+        logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
+            .create(LocalizedStrings.NewLRUClockHand_RETURNING_UNUSED_ENTRY, evictionNode));
+      }
+      if (evictionNode.isRecentlyUsed()) {
+        scanIfNeeded();
+        stats.incGreedyReturns(1);
+      }
+      return (EvictableEntry) evictionNode;
+    }
+  }
+
+  private synchronized void scanIfNeeded() {
+    if (!scanInProgress()) {
+      recentlyUsedCounter.set(0);
+      currentScan = executor.submit(this::scan);
+    }
+  }
+
+
+  /**
+   * Determine who/when should invoke scan. Maybe when 10% of the RegionEntries have been dirtied by
+   * {@link RegionEntry#setRecentlyUsed(RegionEntryContext)}.
+   *
+   * Determine when to stop scanning.
+   */
+  void scan() {
+    EvictionNode evictionNode;
+    do {
+      synchronized (this) {
+        evictionNode = head.next();
+      }
+      int nodesToIterate = size();
+      while (evictionNode != null && evictionNode != tail && nodesToIterate > 0) {
+        nodesToIterate--;
+        // No need to sync on evictionNode here. If the bit is set the only one to clear it is
+        // us (i.e. the scan) or evict/remove code. If either of these happen then this will be
+        // detected by next and previous being null.
+        if (evictionNode.isRecentlyUsed()) {
+          evictionNode.unsetRecentlyUsed();
+          evictionNode = moveToTailAndGetNext(evictionNode);
+        } else {
+          synchronized (this) {
+            evictionNode = evictionNode.next();
+          }
+        }
+      }
+      // null indicates we tried to scan past a node that was concurrently removed.
+      // In that case we need to start at the beginning.
+    } while (evictionNode == null);
+  }
+
+  @Override
+  public void incrementRecentlyUsed() {
+    int recentlyUsedCount = recentlyUsedCounter.incrementAndGet();
+    if (hasThresholdBeenMet(recentlyUsedCount)) {
+      scanIfNeeded();
+    }
+  }
+
+  int getRecentlyUsedCount() {
+    return recentlyUsedCounter.get();
+  }
+
+  private boolean scanInProgress() {
+    return currentScan != null && !currentScan.isDone();
+  }
+
+  private boolean hasThresholdBeenMet(int recentlyUsedCount) {
+    return size() > 0 && (double) recentlyUsedCount / size() >= this.scanThreshold;
+  }
+
+  private synchronized EvictionNode moveToTailAndGetNext(EvictionNode evictionNode) {
+    EvictionNode next = evictionNode.next();
+    if (next != null && next != tail) {
+      EvictionNode previous = evictionNode.previous();
+      next.setPrevious(previous);
+      previous.setNext(next);
+      evictionNode.setNext(tail);
+      tail.previous().setNext(evictionNode);
+      evictionNode.setPrevious(tail.previous());
+      tail.setPrevious(evictionNode);
+    }
+    return next;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/LRUListWithSyncSorting.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/LRUListWithSyncSorting.java
new file mode 100755
index 0000000000..7f7e9ee95b
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/LRUListWithSyncSorting.java
@@ -0,0 +1,111 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import java.util.Optional;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.lang.SystemPropertyHelper;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.logging.log4j.LocalizedMessage;
+import org.apache.geode.internal.logging.log4j.LogMarker;
+
+public class LRUListWithSyncSorting extends AbstractEvictionList {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final int maxEntries;
+
+  LRUListWithSyncSorting(InternalEvictionStatistics stats, BucketRegion region) {
+    super(stats, region);
+    this.maxEntries = readMaxEntriesProperty();
+  }
+
+  private int readMaxEntriesProperty() {
+    int result = -1;
+    Optional<Integer> optionalMaxEntries = SystemPropertyHelper
+        .getProductIntegerProperty(SystemPropertyHelper.EVICTION_SEARCH_MAX_ENTRIES);
+    if (optionalMaxEntries.isPresent()) {
+      result = optionalMaxEntries.get();
+    }
+    return result;
+  }
+
+  /**
+   * return the Entry that is considered least recently used. The entry will no longer be in the
+   * pipe (unless it is the last empty marker).
+   */
+  @Override
+  public EvictableEntry getEvictableEntry() {
+    long numEvals = 0;
+
+    for (;;) {
+      EvictionNode aNode = this.unlinkHeadEntry();
+
+      if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+        logger.trace(LogMarker.LRU_CLOCK, "lru considering {}", aNode);
+      }
+
+      if (aNode == null) { // hit the end of the list
+        this.stats.incEvaluations(numEvals);
+        return null;
+      }
+
+      numEvals++;
+
+      if (!isEvictable(aNode)) {
+        continue;
+      }
+
+      // At this point we have any acceptable entry. Now
+      // use various criteria to determine if it's good enough
+      // to return, or if we need to add it back to the list.
+      if (maxEntries > 0 && numEvals > maxEntries) {
+        if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+          logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
+              .create(LocalizedStrings.NewLRUClockHand_GREEDILY_PICKING_AN_AVAILABLE_ENTRY));
+        }
+        this.stats.incGreedyReturns(1);
+        // fall through, return this node
+      } else if (aNode.isRecentlyUsed()) {
+        if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+          logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
+              .create(LocalizedStrings.NewLRUClockHand_SKIPPING_RECENTLY_USED_ENTRY, aNode));
+        }
+        appendEntry(aNode);
+        continue; // keep looking
+      } else {
+        if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
+          logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
+              .create(LocalizedStrings.NewLRUClockHand_RETURNING_UNUSED_ENTRY, aNode));
+        }
+        // fall through, return this node
+      }
+
+      // Return the current node.
+      this.stats.incEvaluations(numEvals);
+      return (EvictableEntry) aNode;
+    } // synchronized
+  } // for
+
+  @Override
+  public void incrementRecentlyUsed() {
+    // nothing needed
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/MemLRUCapacityController.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/MemoryLRUController.java
similarity index 56%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/lru/MemLRUCapacityController.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/MemoryLRUController.java
index e042d896f4..8fe33b7827 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/MemLRUCapacityController.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/MemoryLRUController.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.lru;
+package org.apache.geode.internal.cache.eviction;
 
 import java.util.Properties;
 
@@ -21,44 +21,43 @@
 import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.cache.Declarable;
 import org.apache.geode.cache.EvictionAction;
-import org.apache.geode.cache.EvictionAlgorithm;
 import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.util.ObjectSizer;
 import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.internal.cache.AbstractLRURegionMap.CDValueWrapper;
 import org.apache.geode.internal.cache.CachedDeserializableFactory;
+import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.Token;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
 import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.size.Sizeable;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
 
 /**
- * A <code>CapacityController</code> that will remove the least recently used (LRU) entry from a
- * region once the region reaches a certain byte {@linkplain #setMaximumMegabytes capacity}.
- * Capacity is determined by monitoring the size of entries added and evicted. Capacity is specified
- * in terms of megabytes. GemFire uses an efficient algorithm to determine the amount of space a
- * region entry occupies in the VM. However, this algorithm may not yield optimal results for all
- * kinds of data. The user may provide his or her own algorithm for determining the size of objects
- * by implementing an {@link ObjectSizer}.
+ * A {@code CapacityController} that will remove the least recently used (LRU) entry from a region
+ * once the region reaches a certain byte {@linkplain #setMaximumMegabytes capacity}. Capacity is
+ * determined by monitoring the size of entries added and evicted. Capacity is specified in terms of
+ * megabytes. GemFire uses an efficient algorithm to determine the amount of space a region entry
+ * occupies in the VM. However, this algorithm may not yield optimal results for all kinds of data.
+ * The user may provide his or her own algorithm for determining the size of objects by implementing
+ * an {@link ObjectSizer}.
  *
- * <P>
- * MemLRUCapacityController must be set in the {@link RegionAttributes} before the region is
- * created. A Region with MemLRUCapacityController set will throw an {@link IllegalStateException}
- * if an attempt is made to replace the Region's capacity controller. While the capacity controller
- * cannot be replaced, it does support changing the limit with the {@link #setMaximumMegabytes}
- * method.
+ * <p>
+ * MemoryLRUController must be set in the {@link RegionAttributes} before the region is created. A
+ * Region with MemoryLRUController set will throw an {@link IllegalStateException} if an attempt is
+ * made to replace the Region's capacity controller. While the capacity controller cannot be
+ * replaced, it does support changing the limit with the {@link #setMaximumMegabytes} method.
  *
- * <P>
- * If you are using a <code>cache.xml</code> file to create a JCache region declaratively, you can
- * include the following to associate a <code>MemLRUCapacityController</code> with a region:
+ * <p>
+ * If you are using a {@code cache.xml} file to create a JCache region declaratively, you can
+ * include the following to associate a {@code MemoryLRUController} with a region:
  *
  * <pre>
  *  &lt;region-attributes&gt;
  *    &lt;capacity-controller&gt;
- *      &lt;classname&gt;org.apache.geode.cache.MemLRUCapacityController&lt;/classname&gt;
+ *      &lt;classname&gt;org.apache.geode.cache.MemoryLRUController&lt;/classname&gt;
  *         &lt;parameter name="maximum-megabytes"&gt;
  *           &lt;string&gt;50&lt;/string&gt;
  *         &lt;/parameter&gt;
@@ -69,34 +68,26 @@
  *  &lt;/region-attributes&gt;
  * </pre>
  *
- * @see LRUCapacityController
- *
- *
  * @since GemFire 2.0.2
  */
-public class MemLRUCapacityController extends LRUAlgorithm implements Declarable {
+public class MemoryLRUController extends SizeLRUController implements Declarable {
 
   private static final long serialVersionUID = 6364183985590572514L;
 
   private static final int OVERHEAD_PER_ENTRY = 250;
 
-  /**
-   * The default maximum number of entries allowed by MemLRU capacity controller is 10 megabytes.
-   */
-  public static final int DEFAULT_MAXIMUM_MEGABYTES = EvictionAttributes.DEFAULT_MEMORY_MAXIMUM;
-
   /**
    * The key for setting the maximum-entries property declaratively.
    *
    * @see #init
    */
-  public static final String MAXIMUM_MEGABYTES = "maximum-megabytes";
+  private static final String MAXIMUM_MEGABYTES = "maximum-megabytes";
 
   /**
    * The {@link #init initialization} property that specifies the name of the {@link ObjectSizer}
    * implementation class.
    */
-  public static final String SIZER_IMPL = "sizer";
+  private static final String SIZER_IMPL = "sizer";
 
   private static final long ONE_MEG = 1024L * 1024L;
 
@@ -127,25 +118,19 @@
             f.createLongCounter("lruGreedyReturns", lruGreedyReturnsDesc, "entries"),});
   }
 
-  //////////////////// Instance Fields ////////////////////
-
-  private long limit = (DEFAULT_MAXIMUM_MEGABYTES) * ONE_MEG;
-
-  private ObjectSizer sizer;
+  private long limit = (EvictionAttributes.DEFAULT_MEMORY_MAXIMUM) * ONE_MEG;
 
   private int perEntryOverHead = OVERHEAD_PER_ENTRY;
 
   private final boolean isOffHeap;
 
-  /////////////////////// Constructors ///////////////////////
-
   /**
    * Create an instance of the capacity controller with default settings. The default settings are 0
-   * <code>maximum-megabytes</code> and a default <code>sizer</code>, requiring either the
-   * {@link #init} method to be called, or the {@link #setMaximumMegabytes} method.
+   * {@code maximum-megabytes} and a default {@code sizer}, requiring either the {@link #init}
+   * method to be called, or the {@link #setMaximumMegabytes} method.
    */
-  public MemLRUCapacityController(Region region) {
-    this(DEFAULT_MAXIMUM_MEGABYTES, region);
+  public MemoryLRUController(Region region) {
+    this(EvictionAttributes.DEFAULT_MEMORY_MAXIMUM, region);
   }
 
   /**
@@ -160,11 +145,10 @@ public MemLRUCapacityController(Region region) {
    *        allowed in the region, collectively for its primary buckets and redundant copies for
    *        this VM. It can be different for the same region in different VMs.
    */
-  public MemLRUCapacityController(int megabytes, Region region) {
+  public MemoryLRUController(int megabytes, Region region) {
     this(megabytes, null /* sizerImpl */, region);
   }
 
-
   /**
    * Create an instance of the capacity controller the given settings.
    *
@@ -179,7 +163,7 @@ public MemLRUCapacityController(int megabytes, Region region) {
    * @param sizerImpl classname of a class that implements ObjectSizer, used to compute object sizes
    *        for MemLRU
    */
-  public MemLRUCapacityController(int megabytes, ObjectSizer sizerImpl, Region region) {
+  public MemoryLRUController(int megabytes, ObjectSizer sizerImpl, Region region) {
     this(megabytes, sizerImpl, EvictionAction.DEFAULT_EVICTION_ACTION, region, false);
   }
 
@@ -194,20 +178,17 @@ public MemLRUCapacityController(int megabytes, ObjectSizer sizerImpl, Region reg
    *        {@link org.apache.geode.cache.PartitionAttributes}. It signifies the amount of memory
    *        allowed in the region, collectively for its primary buckets and redundant copies for
    *        this VM. It can be different for the same region in different VMs.
-   * @param sizerImpl classname of a class that implements ObjectSizer, used to compute object sizes
-   *        for MemLRU
+   * @param sizer classname of a class that implements ObjectSizer, used to compute object sizes for
+   *        MemLRU
    * @param isOffHeap true if the region that owns this cc is stored off heap
    */
-  public MemLRUCapacityController(int megabytes, ObjectSizer sizerImpl,
-      EvictionAction evictionAction, Region region, boolean isOffHeap) {
-    super(evictionAction, region);
+  public MemoryLRUController(int megabytes, ObjectSizer sizer, EvictionAction evictionAction,
+      Region region, boolean isOffHeap) {
+    super(evictionAction, region, sizer);
     this.isOffHeap = isOffHeap;
     setMaximumMegabytes(megabytes);
-    setSizer(sizerImpl);
   }
 
-  ////////////////////// Instance Methods /////////////////////
-
   /**
    * Declaratively initializes this capacity controller. Supported properties are:
    *
@@ -220,24 +201,24 @@ public MemLRUCapacityController(int megabytes, ObjectSizer sizerImpl,
    * for computing the size of region entries.</li>
    * </ul>
    *
-   * @throws NumberFormatException The <code>maximum-megabytes</code> property cannot be parsed as
-   *         an integer
-   * @throws IllegalArgumentException The value of the <code>eviction-action</code> property is not
-   *         recoginzed.
+   * @throws NumberFormatException The {@code maximum-megabytes} property cannot be parsed as an
+   *         integer
+   * @throws IllegalArgumentException The value of the {@code eviction-action} property is not
+   *         recognized.
    */
+  @Override
   public void init(Properties props) {
-    String prop = null;
-    String sizerStr = null;
+    String prop;
+    String sizerStr;
     if ((sizerStr = props.getProperty(SIZER_IMPL)) != null) {
       try {
         Class c = ClassPathLoader.getLatest().forName(sizerStr);
         setSizer((ObjectSizer) c.newInstance());
       } catch (Exception e) {
-        IllegalArgumentException ex = new IllegalArgumentException(
+        throw new IllegalArgumentException(
             LocalizedStrings.MemLRUCapacityController_COULD_NOT_CREATE_SIZER_INSTANCE_GIVEN_THE_CLASS_NAME_0
-                .toLocalizedString(sizer));
-        ex.initCause(e);
-        throw ex;
+                .toLocalizedString(sizerStr),
+            e);
       }
     }
 
@@ -250,34 +231,16 @@ public void init(Properties props) {
     }
   }
 
-  // Candidate for removal since capacity controller no longer part of
-  // cache.xml
-  @Override
-  public Properties getProperties() {
-    Properties props = new Properties();
-    if (this.evictionAction != EvictionAction.DEFAULT_EVICTION_ACTION) {
-      props.setProperty(EVICTION_ACTION, this.evictionAction.toString());
-    }
-    long megLimit = this.limit / ONE_MEG;
-    if (megLimit != DEFAULT_MAXIMUM_MEGABYTES) {
-      props.setProperty(MAXIMUM_MEGABYTES, String.valueOf(megLimit));
-    }
-    if (this.sizer != null) {
-      props.setProperty(SIZER_IMPL, this.sizer.getClass().getName());
-    }
-    return props;
-  }
-
   /**
    * Reset the maximum allowed limit on memory to use for this region. This change takes effect on
    * next region operation that could increase the region's byte size. If the region is shared, this
-   * change is seen by all vms on using the same GemFire shared memory system.
+   * change is seen by all members in the cluster.
    */
   public void setMaximumMegabytes(int megabytes) {
     if (megabytes <= 0) {
       throw new IllegalArgumentException(
           LocalizedStrings.MemLRUCapacityController_MEMLRUCONTROLLER_LIMIT_MUST_BE_POSTIVE_0
-              .toLocalizedString(Integer.valueOf(megabytes)));
+              .toLocalizedString(megabytes));
     }
     this.limit = (megabytes) * ONE_MEG;
     if (bucketRegion != null) {
@@ -300,141 +263,102 @@ public void setEntryOverHead(int entryOverHead) {
     this.perEntryOverHead = entryOverHead;
   }
 
-  // public void writeExternal(ObjectOutput out)
-  // throws IOException {
-  // super.writeExternal(out);
-  // if (this.stats != null) {
-  // long limit = this.getLRUHelper().limit();
-  // Assert.assertTrue(limit > 0);
-  // out.writeLong(limit);
-
-  // } else {
-  // Assert.assertTrue(this.limit > 0);
-  // out.writeLong(this.limit);
-  // }
-  // if (this.sizer != null) {
-  // out.writeBoolean(true);
-  // out.writeUTF(this.sizer.getClass().getName());
-
-  // } else {
-  // out.writeBoolean(false);
-  // }
-  // }
-
-  // public void readExternal(ObjectInput in)
-  // throws IOException, ClassNotFoundException {
-  // super.readExternal(in);
-  // long limit = in.readLong();
-  // setMaximumMegabytes((int) limit);
-  // if (in.readBoolean()) {
-  // String className = in.readUTF();
-  // setSizer(className);
-  // }
-  // }
+  @Override
+  public long getLimit() {
+    return this.limit;
+  }
 
   /**
-   * Sets the {@link ObjectSizer} used to calculate the size of objects placed in the cache.
-   *
-   * @param sizer The name of the sizer class
+   * Indicate what kind of {@code AbstractEvictionController} this helper implements
    */
-  private void setSizer(ObjectSizer sizer) {
-    this.sizer = sizer;
-  }
-
   @Override
-  public long getLimit() {
-    return this.limit;
+  public org.apache.geode.cache.EvictionAlgorithm getEvictionAlgorithm() {
+    return org.apache.geode.cache.EvictionAlgorithm.LRU_MEMORY;
   }
 
+
+  /**
+   * compute the size of storing a key/value pair in the cache..
+   */
   @Override
-  protected EnableLRU createLRUHelper() {
-    return new AbstractEnableLRU() {
-
-      /**
-       * Indicate what kind of <code>EvictionAlgorithm</code> this helper implements
-       */
-      public EvictionAlgorithm getEvictionAlgorithm() {
-        return EvictionAlgorithm.LRU_MEMORY;
-      }
+  public int entrySize(Object key, Object value) throws IllegalArgumentException {
 
+    if (value == Token.TOMBSTONE) {
+      return 0;
+    }
 
-      /**
-       * compute the size of storing a key/value pair in the cache..
-       */
-      public int entrySize(Object key, Object value) throws IllegalArgumentException {
-
-        if (value == Token.TOMBSTONE) {
-          return 0;
-        }
-
-        int size = 0;
-        int keySize = 0;
-        if (!MemLRUCapacityController.this.isOffHeap) {
-          size += MemLRUCapacityController.this.getPerEntryOverhead();
-          keySize = sizeof(key);
-        }
-        int valueSize = sizeof(value);
-        // org.apache.geode.internal.cache.GemFireCacheImpl.getInstance().getLogger().info("DEBUG
-        // MemLRUCC: overhead=" + size
-        // + " keySize=" + keySize
-        // + " valueSize=" + valueSize);
-        size += keySize;
-        size += valueSize;
-        return size;
-      }
+    int size = 0;
+    int keySize = 0;
+    if (!MemoryLRUController.this.isOffHeap) {
+      size += MemoryLRUController.this.getPerEntryOverhead();
+      keySize = sizeof(key);
+    }
+    int valueSize = sizeof(value);
+    size += keySize;
+    size += valueSize;
+    return size;
+  }
 
-      public StatisticsType getStatisticsType() {
-        return statType;
-      }
+  @Override
+  public StatisticsType getStatisticsType() {
+    return statType;
+  }
 
-      public String getStatisticsName() {
-        return "MemLRUStatistics";
-      }
+  @Override
+  public String getStatisticsName() {
+    return "MemLRUStatistics";
+  }
 
-      public int getLimitStatId() {
-        return statType.nameToId("bytesAllowed");
-      }
+  @Override
+  public int getLimitStatId() {
+    return statType.nameToId("bytesAllowed");
+  }
 
-      public int getCountStatId() {
-        return statType.nameToId("byteCount");
-      }
+  @Override
+  public int getCountStatId() {
+    return statType.nameToId("byteCount");
+  }
 
-      public int getEvictionsStatId() {
-        return statType.nameToId("lruEvictions");
-      }
+  @Override
+  public int getEvictionsStatId() {
+    return statType.nameToId("lruEvictions");
+  }
 
-      public int getDestroysStatId() {
-        return statType.nameToId("lruDestroys");
-      }
+  @Override
+  public int getDestroysStatId() {
+    return statType.nameToId("lruDestroys");
+  }
 
-      public int getDestroysLimitStatId() {
-        return statType.nameToId("lruDestroysLimit");
-      }
+  @Override
+  public int getDestroysLimitStatId() {
+    return statType.nameToId("lruDestroysLimit");
+  }
 
-      public int getEvaluationsStatId() {
-        return statType.nameToId("lruEvaluations");
-      }
+  @Override
+  public int getEvaluationsStatId() {
+    return statType.nameToId("lruEvaluations");
+  }
 
-      public int getGreedyReturnsStatId() {
-        return statType.nameToId("lruGreedyReturns");
-      }
+  @Override
+  public int getGreedyReturnsStatId() {
+    return statType.nameToId("lruGreedyReturns");
+  }
 
-      public boolean mustEvict(LRUStatistics stats, Region region, int delta) {
-        return stats.getCounter() + delta > stats.getLimit();
-      }
+  @Override
+  public boolean mustEvict(EvictionStatistics stats, InternalRegion region, int delta) {
+    return stats.getCounter() + delta > stats.getLimit();
+  }
 
-      @Override
-      public boolean lruLimitExceeded(LRUStatistics lruStatistics, DiskRegionView drv) {
-        return lruStatistics.getCounter() > lruStatistics.getLimit();
-      }
-    };
+  @Override
+  public boolean lruLimitExceeded(EvictionStatistics stats, DiskRegionView diskRegionView) {
+    return stats.getCounter() > stats.getLimit();
   }
 
   // added to fix bug 40718
   static int basicSizeof(Object o, ObjectSizer sizer) throws IllegalArgumentException {
-    final boolean cdChangingForm = o instanceof CDValueWrapper;
+    final boolean cdChangingForm = o instanceof CachedDeserializableValueWrapper;
     if (cdChangingForm) {
-      o = ((CDValueWrapper) o).getValue();
+      o = ((CachedDeserializableValueWrapper) o).getValue();
     }
     if (o == null || o == Token.INVALID || o == Token.LOCAL_INVALID || o == Token.DESTROYED
         || o == Token.TOMBSTONE) {
@@ -458,15 +382,6 @@ static int basicSizeof(Object o, ObjectSizer sizer) throws IllegalArgumentExcept
     return size;
   }
 
-  /**
-   * Return the size of an object as stored in GemFire... Typically this is the serialized size in
-   * bytes.. This implementation is slow.... Need to add Sizer interface and call it for customer
-   * objects.
-   */
-  protected int sizeof(Object o) throws IllegalArgumentException {
-    return basicSizeof(o, this.sizer);
-  }
-
   public int getPerEntryOverhead() {
     return perEntryOverHead;
   }
@@ -475,20 +390,12 @@ public int getPerEntryOverhead() {
   public boolean equals(Object cc) {
     if (!super.equals(cc))
       return false;
-    MemLRUCapacityController other = (MemLRUCapacityController) cc;
+    MemoryLRUController other = (MemoryLRUController) cc;
     if (this.limit != other.limit)
       return false;
     return true;
   }
 
-  /*
-   * (non-Javadoc)
-   *
-   * @see java.lang.Object#hashCode()
-   *
-   * Note that we just need to make sure that equal objects return equal hashcodes; nothing really
-   * elaborate is done here.
-   */
   @Override
   public int hashCode() {
     int result = super.hashCode();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/OffHeapEvictor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/OffHeapEvictor.java
similarity index 78%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/lru/OffHeapEvictor.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/eviction/OffHeapEvictor.java
index 058673f951..89f8e0ef9d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/OffHeapEvictor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/OffHeapEvictor.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.lru;
+package org.apache.geode.internal.cache.eviction;
 
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.cache.InternalCache;
@@ -29,23 +29,20 @@
  * @since Geode 1.0
  */
 public class OffHeapEvictor extends HeapEvictor {
+
   private static final String EVICTOR_THREAD_GROUP_NAME = "OffHeapEvictorThreadGroup";
 
   private static final String EVICTOR_THREAD_NAME = "OffHeapEvictorThread";
 
   private long bytesToEvictWithEachBurst;
 
-  public OffHeapEvictor(InternalCache gemFireCache) {
-    super(gemFireCache);
+  public OffHeapEvictor(final InternalCache cache) {
+    super(cache, EVICTOR_THREAD_GROUP_NAME, EVICTOR_THREAD_NAME);
     calculateEvictionBurst();
   }
 
   private void calculateEvictionBurst() {
-    float evictionBurstPercentage = Float.parseFloat(System.getProperty(
-        DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.evictionBurstPercentage",
-        "0.4"));
-
-    MemoryAllocator allocator = this.cache.getOffHeapStore();
+    MemoryAllocator allocator = cache().getOffHeapStore();
 
     /*
      * Bail if there is no off-heap memory to evict.
@@ -55,36 +52,35 @@ private void calculateEvictionBurst() {
           LocalizedStrings.MEMSCALE_EVICTION_INIT_FAIL.toLocalizedString());
     }
 
+    float evictionBurstPercentage = Float.parseFloat(System.getProperty(
+        DistributionConfig.GEMFIRE_PREFIX + "HeapLRUCapacityController.evictionBurstPercentage",
+        "0.4"));
     bytesToEvictWithEachBurst =
         (long) (allocator.getTotalMemory() * 0.01 * evictionBurstPercentage);
   }
 
+  @Override
   protected int getEvictionLoopDelayTime() {
-    if (numEvictionLoopsCompleted < Math.max(3, numFastLoops)) {
+    if (numEvictionLoopsCompleted() < Math.max(3, numFastLoops())) {
       return 250;
     }
 
     return 1000;
   }
 
+  @Override
   protected boolean includePartitionedRegion(PartitionedRegion region) {
-    return (region.getEvictionAttributes().getAlgorithm().isLRUHeap()
-        && (region.getDataStore() != null) && region.getAttributes().getOffHeap());
+    return region.getEvictionAttributes().getAlgorithm().isLRUHeap()
+        && region.getDataStore() != null && region.getAttributes().getOffHeap();
   }
 
+  @Override
   protected boolean includeLocalRegion(LocalRegion region) {
-    return (region.getEvictionAttributes().getAlgorithm().isLRUHeap()
-        && region.getAttributes().getOffHeap());
-  }
-
-  protected String getEvictorThreadGroupName() {
-    return OffHeapEvictor.EVICTOR_THREAD_GROUP_NAME;
-  }
-
-  protected String getEvictorThreadName() {
-    return OffHeapEvictor.EVICTOR_THREAD_NAME;
+    return region.getEvictionAttributes().getAlgorithm().isLRUHeap()
+        && region.getAttributes().getOffHeap();
   }
 
+  @Override
   public long getTotalBytesToEvict() {
     return bytesToEvictWithEachBurst;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/RegionEvictorTask.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/RegionEvictorTask.java
new file mode 100755
index 0000000000..dbb22580ad
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/RegionEvictorTask.java
@@ -0,0 +1,108 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.cache.RegionDestroyedException;
+import org.apache.geode.internal.cache.AbstractLRURegionMap;
+import org.apache.geode.internal.cache.CachePerfStats;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.logging.log4j.LocalizedMessage;
+
+/**
+ * Takes delta to be evicted and tries to evict the least no of LRU entry which would make
+ * evictedBytes more than or equal to the delta
+ *
+ * @since GemFire 6.0
+ */
+public class RegionEvictorTask implements Runnable {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final CachePerfStats stats;
+
+  private final List<LocalRegion> regions;
+
+  private final HeapEvictor evictor;
+
+  private final long bytesToEvictPerTask;
+
+  RegionEvictorTask(final CachePerfStats stats, final List<LocalRegion> regions,
+      final HeapEvictor evictor, final long bytesToEvictPerTask) {
+    this.stats = stats;
+    this.evictor = evictor;
+    this.regions = regions;
+    this.bytesToEvictPerTask = bytesToEvictPerTask;
+  }
+
+  List<LocalRegion> getRegionList() {
+    synchronized (this.regions) {
+      return this.regions;
+    }
+  }
+
+  private HeapEvictor getHeapEvictor() {
+    return this.evictor;
+  }
+
+  @Override
+  public void run() {
+    this.stats.incEvictorJobsStarted();
+    try {
+      long totalBytesEvicted = 0;
+      while (true) {
+        final long start = CachePerfStats.getStatTime();
+        synchronized (this.regions) {
+          if (this.regions.isEmpty()) {
+            return;
+          }
+          // consider trying Fisher-Yates shuffle algorithm
+          for (Iterator<LocalRegion> iterator = regions.iterator(); iterator.hasNext();) {
+            LocalRegion region = iterator.next();
+            try {
+              long bytesEvicted =
+                  ((AbstractLRURegionMap) region.entries).centralizedLruUpdateCallback();
+              if (bytesEvicted == 0) {
+                iterator.remove();
+              }
+              totalBytesEvicted += bytesEvicted;
+              if (totalBytesEvicted >= bytesToEvictPerTask || !getHeapEvictor().mustEvict()
+                  || this.regions.isEmpty()) {
+                return;
+              }
+            } catch (RegionDestroyedException e) {
+              region.getCache().getCancelCriterion().checkCancelInProgress(e);
+            } catch (RuntimeException e) {
+              region.getCache().getCancelCriterion().checkCancelInProgress(e);
+              logger.warn(LocalizedMessage.create(LocalizedStrings.Eviction_EVICTOR_TASK_EXCEPTION,
+                  new Object[] {e.getMessage()}), e);
+            } finally {
+              long end = CachePerfStats.getStatTime();
+              this.stats.incEvictWorkTime(end - start);
+            }
+          }
+        }
+      }
+    } finally {
+      this.stats.incEvictorJobsCompleted();
+    }
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/SizeLRUController.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/SizeLRUController.java
new file mode 100644
index 0000000000..3389be5664
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/SizeLRUController.java
@@ -0,0 +1,48 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import org.apache.geode.cache.EvictionAction;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.util.ObjectSizer;
+
+abstract class SizeLRUController extends AbstractEvictionController {
+
+  private ObjectSizer sizer;
+
+  SizeLRUController(EvictionAction evictionAction, Region region, ObjectSizer sizer) {
+    super(evictionAction, region);
+    this.sizer = sizer;
+  }
+
+  /**
+   * Return the size of an object as stored in GemFire. Typically this is the serialized size in
+   * bytes. This implementation is slow. Need to add Sizer interface and call it for customer
+   * objects.
+   */
+  int sizeof(Object object) throws IllegalArgumentException {
+    return MemoryLRUController.basicSizeof(object, this.sizer);
+  }
+
+  /**
+   * Sets the {@link ObjectSizer} used to calculate the size of objects placed in the cache.
+   *
+   * @param sizer The name of the sizer class
+   */
+  void setSizer(ObjectSizer sizer) {
+    this.sizer = sizer;
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapLRUCapacityController.java b/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapLRUCapacityController.java
deleted file mode 100644
index 1abedb5aeb..0000000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapLRUCapacityController.java
+++ /dev/null
@@ -1,313 +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.geode.internal.cache.lru;
-
-import java.util.Properties;
-
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.StatisticsFactory;
-import org.apache.geode.StatisticsType;
-import org.apache.geode.StatisticsTypeFactory;
-import org.apache.geode.cache.EvictionAction;
-import org.apache.geode.cache.EvictionAlgorithm;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.util.ObjectSizer;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.internal.cache.BucketRegion;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.Token;
-import org.apache.geode.internal.cache.control.InternalResourceManager;
-import org.apache.geode.internal.cache.persistence.DiskRegionView;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
-
-/**
- * A <code>HeapLRUCapacityController</code> controls the contents of {@link Region} based on the
- * percentage of memory that is currently being used. If the percentage of memory in use exceeds the
- * given percentage, then the least recently used entry of the region is evicted.
- * <p>
- * For heap regions: GemStone has found that the <code>HeapLRUCapacityController</code> has the most
- * effect on a VM that is lauched with both the <code>-Xmx</code> and <code>-Xms</code> switches
- * used. Many virtual machine implementations have additional VM switches to control the behavior of
- * the garbage collector. We suggest that you investigate tuning the garbage collector when using a
- * <code>HeapLRUCapacityController</code>. In particular, we have found that when running with Sun's
- * <A href="http://java.sun.com/docs/hotspot/gc/index.html">HotSpot</a> VM, the
- * <code>-XX:+UseConcMarkSweepGC</code> and <code>-XX:+UseParNewGC</code> options improve the
- * behavior of the <code>HeapLRUCapacityController</code>.
- *
- * @since GemFire 3.2
- */
-@SuppressWarnings("synthetic-access")
-public class HeapLRUCapacityController extends LRUAlgorithm {
-  private static final long serialVersionUID = 4970685814429530675L;
-  /**
-   * The default percentage of VM heap usage over which LRU eviction occurs
-   */
-  public static final String TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY =
-      DistributionConfig.GEMFIRE_PREFIX + "topUpHeapEvictionPercentage";
-
-  public static final float DEFAULT_TOP_UP_HEAP_EVICTION_PERCENTAGE = 4.0f;
-
-  public static final int DEFAULT_HEAP_PERCENTAGE = 75;
-
-  public static final int PER_ENTRY_OVERHEAD = 250;
-
-  private int perEntryOverhead = PER_ENTRY_OVERHEAD;
-
-
-  /**
-   * The default number of milliseconds the evictor thread should wait before evicting the LRU
-   * entry.
-   */
-  public static final int DEFAULT_EVICTOR_INTERVAL = 500;
-
-  protected static final StatisticsType statType;
-
-  static {
-    // create the stats type for MemLRU.
-    StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
-
-    final String entryBytesDesc =
-        "The amount of memory currently used by regions configured for eviction.";
-    final String lruEvictionsDesc = "Number of total entry evictions triggered by LRU.";
-    final String lruDestroysDesc =
-        "Number of entries destroyed in the region through both destroy cache operations and eviction. Reset to zero each time it exceeds lruDestroysLimit.";
-    final String lruDestroysLimitDesc =
-        "Maximum number of entry destroys triggered by LRU before scan occurs.";
-    final String lruEvaluationsDesc = "Number of entries evaluated during LRU operations.";
-    final String lruGreedyReturnsDesc = "Number of non-LRU entries evicted during LRU operations";
-
-    statType = f.createType("HeapLRUStatistics",
-        "Statistics about byte based Least Recently Used region entry disposal",
-        new StatisticDescriptor[] {f.createLongGauge("entryBytes", entryBytesDesc, "bytes"),
-            f.createLongCounter("lruEvictions", lruEvictionsDesc, "entries"),
-            f.createLongCounter("lruDestroys", lruDestroysDesc, "entries"),
-            f.createLongGauge("lruDestroysLimit", lruDestroysLimitDesc, "entries"),
-            f.createLongCounter("lruEvaluations", lruEvaluationsDesc, "entries"),
-            f.createLongCounter("lruGreedyReturns", lruGreedyReturnsDesc, "entries"),});
-  }
-
-  // //////////////////// Instance Fields /////////////////////
-
-  // ////////////////////// Constructors ///////////////////////
-
-  /**
-   * Creates a new <code>HeapLRUCapacityController</code> with the given eviction action.
-   *
-   * @param evictionAction The action that will occur when an entry is evicted
-   *
-   */
-  public HeapLRUCapacityController(EvictionAction evictionAction, Region region) {
-    super(evictionAction, region);
-  }
-
-  public HeapLRUCapacityController(ObjectSizer sizerImpl, EvictionAction evictionAction,
-      Region region) {
-    super(evictionAction, region);
-    setSizer(sizerImpl);
-  }
-  // ///////////////////// Instance Methods ///////////////////////
-
-  @Override
-  public void setLimit(int maximum) {}
-
-  // Candidate for removal since capacity controller is no longer part of
-  // cache.xml
-  @Override
-  public Properties getProperties() {
-    throw new IllegalStateException("Unused properties");
-  }
-
-  @Override
-  public void close() {
-    super.close();
-  }
-
-  @Override
-  public long getLimit() {
-    return 0;
-  }
-
-  @Override
-  public boolean equals(Object cc) {
-    if (!super.equals(cc))
-      return false;
-    return true;
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see java.lang.Object#hashCode()
-   *
-   * Note that we just need to make sure that equal objects return equal hashcodes; nothing really
-   * elaborate is done here.
-   */
-  @Override
-  public int hashCode() {
-    int result = super.hashCode();
-    return result;
-  }
-
-  /**
-   * Returns a brief description of this eviction controller.
-   *
-   * @since GemFire 4.0
-   */
-  @Override
-  public String toString() {
-    return LocalizedStrings.HeapLRUCapacityController_HEAPLRUCAPACITYCONTROLLER_WITH_A_CAPACITY_OF_0_OF_HEAP_AND_AN_THREAD_INTERVAL_OF_1_AND_EVICTION_ACTION_2
-        .toLocalizedString(new Object[] {Long.valueOf(this.getLimit()), this.getEvictionAction()});
-  }
-
-  /**
-   * Sets the {@link ObjectSizer} used to calculate the size of objects placed in the cache.
-   *
-   * @param sizer The name of the sizer class
-   */
-  private void setSizer(ObjectSizer sizer) {
-    this.sizer = sizer;
-  }
-
-  @Override
-  protected EnableLRU createLRUHelper() {
-    return new AbstractEnableLRU() {
-
-      /**
-       * Indicate what kind of <code>EvictionAlgorithm</code> this helper implements
-       */
-      public EvictionAlgorithm getEvictionAlgorithm() {
-        return EvictionAlgorithm.LRU_HEAP;
-      }
-
-      /**
-       * As far as we're concerned all entries have the same size
-       */
-      public int entrySize(Object key, Object value) throws IllegalArgumentException {
-        // value is null only after eviction occurs. A change in size is
-        // required for eviction stats, bug 30974
-        /*
-         * if (value != null) { return 1; } else { return 0; }
-         */
-        if (value == Token.TOMBSTONE) {
-          return 0;
-        }
-
-        int size = HeapLRUCapacityController.this.getPerEntryOverhead();
-        size += sizeof(key);
-        size += sizeof(value);
-        return size;
-      }
-
-      /**
-       * In addition to initializing the statistics, create an evictor thread to periodically evict
-       * the LRU entry.
-       */
-      @Override
-      public LRUStatistics initStats(Object region, StatisticsFactory sf) {
-        setRegionName(region);
-        final LRUStatistics stats = new HeapLRUStatistics(sf, getRegionName(), this);
-        setStats(stats);
-        return stats;
-      }
-
-      public StatisticsType getStatisticsType() {
-        return statType;
-      }
-
-      public String getStatisticsName() {
-        return "HeapLRUStatistics";
-      }
-
-      public int getLimitStatId() {
-        throw new UnsupportedOperationException("Limit not used with this LRU type");
-      }
-
-      public int getCountStatId() {
-        return statType.nameToId("entryBytes");
-      }
-
-      public int getEvictionsStatId() {
-        return statType.nameToId("lruEvictions");
-      }
-
-      public int getDestroysStatId() {
-        return statType.nameToId("lruDestroys");
-      }
-
-      public int getDestroysLimitStatId() {
-        return statType.nameToId("lruDestroysLimit");
-      }
-
-      public int getEvaluationsStatId() {
-        return statType.nameToId("lruEvaluations");
-      }
-
-      public int getGreedyReturnsStatId() {
-        return statType.nameToId("lruGreedyReturns");
-      }
-
-      /**
-       * Okay, deep breath. Instead of basing the LRU calculation on the number of entries in the
-       * region or on their "size" (which turned out to be incorrectly estimated in the general
-       * case), we use the amount of memory currently in use. If the amount of memory current in use
-       * {@linkplain Runtime#maxMemory max memory} - {@linkplain Runtime#freeMemory free memory} is
-       * greater than the overflow threshold, then we evict the LRU entry.
-       */
-      public boolean mustEvict(LRUStatistics stats, Region region, int delta) {
-        final InternalCache cache = (InternalCache) region.getRegionService();
-        InternalResourceManager resourceManager = cache.getInternalResourceManager();
-        boolean offheap = region.getAttributes().getOffHeap();
-        final boolean monitorStateIsEviction =
-            resourceManager.getMemoryMonitor(offheap).getState().isEviction();
-        if (region instanceof BucketRegion) {
-          return monitorStateIsEviction && ((BucketRegion) region).getSizeForEviction() > 0;
-        }
-
-        return monitorStateIsEviction && ((LocalRegion) region).getRegionMap().sizeInVM() > 0;
-      }
-
-      @Override
-      public boolean lruLimitExceeded(LRUStatistics lruStatistics, DiskRegionView drv) {
-        InternalResourceManager resourceManager =
-            drv.getDiskStore().getCache().getInternalResourceManager();
-        return resourceManager.getMemoryMonitor(drv.getOffHeap()).getState().isEviction();
-      }
-    };
-  }
-
-  // ////////////////////// Inner Classes ////////////////////////
-
-  private ObjectSizer sizer;
-
-  /**
-   * Return the size of an object as stored in GemFire... Typically this is the serialized size in
-   * bytes.. This implementation is slow.... Need to add Sizer interface and call it for customer
-   * objects.
-   */
-  protected int sizeof(Object o) throws IllegalArgumentException {
-    return MemLRUCapacityController.basicSizeof(o, this.sizer);
-  }
-
-  public int getPerEntryOverhead() {
-    return perEntryOverhead;
-  }
-
-  public void setEntryOverHead(int entryOverHead) {
-    this.perEntryOverhead = entryOverHead;
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapLRUStatistics.java b/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapLRUStatistics.java
deleted file mode 100644
index e256508d50..0000000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapLRUStatistics.java
+++ /dev/null
@@ -1,59 +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.geode.internal.cache.lru;
-
-import org.apache.geode.*;
-
-/**
- * Statistics for the HeapLRUCapacityController, which treats the counter statistic differently than
- * other flavors of <code>LRUAlgorithms</code>
- *
- * @see org.apache.geode.internal.cache.lru.MemLRUCapacityController
- * @see org.apache.geode.internal.cache.lru.LRUCapacityController
- * @since GemFire 4.0
- */
-public class HeapLRUStatistics extends LRUStatistics {
-
-  public HeapLRUStatistics(StatisticsFactory factory, String name, EnableLRU helper) {
-    super(factory, name, helper);
-  }
-
-  /**
-   * Ignore the delta value since the change isn't relevant for heap related LRU since the counter
-   * reflects a percentage of used memory. Normally the delta reflects either the number of entries
-   * that changed for the <code>LRUCapacityController</code> or the estimated amount of memory that
-   * has changed after performing a Region operation. The <code>HeapLRUCapacityController</code>
-   * however does not care about <code>Region</code> changes, it only considers heap changes and
-   * uses <code>Runtime</code> to determine how much to evict.
-   *
-   * @see org.apache.geode.internal.cache.lru.HeapLRUCapacityController#createLRUHelper
-   * @see EnableLRU#mustEvict
-   */
-  @Override
-  public void updateCounter(long delta) {
-    super.updateCounter(delta);
-  }
-
-  /**
-   * The counter for <code>HeapLRUCapacityController</code> reflects in use heap. Since you can not
-   * programatically reset the amount of heap usage (at least not directly) this method does
-   * <b>NOT</b> reset the counter value.
-   */
-  @Override
-  public void resetCounter() {
-    super.resetCounter();
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUAlgorithm.java b/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUAlgorithm.java
deleted file mode 100644
index 80e82d6780..0000000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/LRUAlgorithm.java
+++ /dev/null
@@ -1,317 +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.geode.internal.cache.lru;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Properties;
-
-import org.apache.geode.InternalGemFireException;
-import org.apache.geode.StatisticsFactory;
-import org.apache.geode.cache.CacheCallback;
-import org.apache.geode.cache.EvictionAction;
-import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.BucketRegion;
-import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-
-/**
- * Eviction controllers that extend this class evict the least recently used (LRU) entry in the
- * region whose capacity they controller. In order to provide an efficient computation of the LRU
- * entry, GemFire uses special internal data structures for managing the contents of a region. As a
- * result, there are several restrictions that are placed on regions whose capacity is governed by
- * an LRU algorithm.
- *
- * <UL>
- *
- * <LI>If the capacity of a region is to be controlled by an LRU algorithm, then the region must be
- * <b>created</b> with {@link org.apache.geode.cache.EvictionAttributes}
- *
- * <LI>The eviction controller of a region governed by an LRU algorithm cannot be changed.</LI>
- *
- * <LI>An LRU algorithm cannot be applied to a region after the region has been created.</LI>
- *
- * </UL>
- *
- * LRU algorithms also specify what {@linkplain org.apache.geode.cache.EvictionAction action} should
- * be performed upon the least recently used entry when the capacity is reached. Currently, there
- * are two supported actions: {@linkplain org.apache.geode.cache.EvictionAction#LOCAL_DESTROY
- * locally destroying} the entry (which is the
- * {@linkplain org.apache.geode.cache.EvictionAction#DEFAULT_EVICTION_ACTION default}), thus freeing
- * up space in the VM, and {@linkplain org.apache.geode.cache.EvictionAction#OVERFLOW_TO_DISK
- * overflowing} the value of the entry to disk.
- *
- * <P>
- *
- * {@link org.apache.geode.cache.EvictionAttributes Eviction controllers} that use an LRU algorithm
- * maintain certain region-dependent state (such as the maximum number of entries allowed in the
- * region). As a result, an instance of <code>LRUAlgorithm</code> cannot be shared among multiple
- * regions. Attempts to create a region with a LRU-based capacity controller that has already been
- * used to create another region will result in an {@link IllegalStateException} being thrown.
- *
- * @since GemFire 3.2
- */
-public abstract class LRUAlgorithm implements CacheCallback, Serializable, Cloneable {
-
-  /**
-   * The key for setting the <code>eviction-action</code> property of an <code>LRUAlgorithm</code>
-   */
-  public static final String EVICTION_ACTION = "eviction-action";
-
-  /** What to do upon eviction */
-  protected EvictionAction evictionAction;
-
-  /** Used to dynamically track the changing region limit. */
-  protected transient LRUStatistics stats;
-
-  /** The helper created by this LRUAlgorithm */
-  private transient EnableLRU helper;
-
-  protected BucketRegion bucketRegion;
-
-  /**
-   * Creates a new <code>LRUAlgorithm</code> with the given {@linkplain EvictionAction eviction
-   * action}.
-   */
-  protected LRUAlgorithm(EvictionAction evictionAction, Region region) {
-    bucketRegion = (BucketRegion) (region instanceof BucketRegion ? region : null);
-    setEvictionAction(evictionAction);
-    this.helper = createLRUHelper();
-  }
-
-  /**
-   * Used to hook up a bucketRegion late during disk recover.
-   */
-  public void setBucketRegion(Region r) {
-    if (r instanceof BucketRegion) {
-      this.bucketRegion = (BucketRegion) r;
-      this.bucketRegion.setLimit(getLimit());
-    }
-  }
-
-  /**
-   * Sets the action that is performed on the least recently used entry when it is evicted from the
-   * VM.
-   *
-   * @throws IllegalArgumentException If <code>evictionAction</code> specifies an unknown eviction
-   *         action.
-   *
-   * @see EvictionAction
-   */
-  protected void setEvictionAction(EvictionAction evictionAction) {
-    this.evictionAction = evictionAction;
-  }
-
-  /**
-   * Gets the action that is performed on the least recently used entry when it is evicted from the
-   * VM.
-   *
-   * @return one of the following constants: {@link EvictionAction#LOCAL_DESTROY},
-   *         {@link EvictionAction#OVERFLOW_TO_DISK}
-   */
-  public EvictionAction getEvictionAction() {
-    return this.evictionAction;
-  }
-
-  /**
-   * For internal use only. Returns a helper object used internally by the GemFire cache
-   * implementation.
-   */
-  public EnableLRU getLRUHelper() {
-    synchronized (this) {
-      // Synchronize with readObject/writeObject to avoid race
-      // conditions with copy sharing. See bug 31047.
-      return this.helper;
-    }
-  }
-
-  private void writeObject(java.io.ObjectOutputStream out) throws IOException {
-
-    synchronized (this) { // See bug 31047
-      out.writeObject(this.evictionAction);
-    }
-  }
-
-  private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
-
-    synchronized (this) { // See bug 31047
-      this.evictionAction = (EvictionAction) in.readObject();
-      this.helper = createLRUHelper();
-    }
-  }
-
-  /**
-   * Creates a new <code>LRUHelper</code> tailed for this LRU algorithm implementation.
-   */
-  protected abstract EnableLRU createLRUHelper();
-
-  /**
-   * Returns the "limit" as defined by this LRU algorithm
-   */
-  public abstract long getLimit();
-
-  /**
-   * Set the limiting parameter used to determine when eviction is needed.
-   */
-  public abstract void setLimit(int maximum);
-
-  /**
-   * This method is an artifact when eviction controllers used to called capacity controllers and
-   * were configured in the cache.xml file as <code>Declarable</code>
-   *
-   * @since GemFire 4.1.1
-   */
-  public abstract Properties getProperties();
-
-  /**
-   * Releases resources obtained by this <code>LRUAlgorithm</code>
-   */
-  public void close() {
-    if (this.stats != null) {
-      if (bucketRegion != null) {
-        this.stats.incEvictions(bucketRegion.getEvictions() * -1);
-        this.stats.decrementCounter(bucketRegion.getCounter());
-        bucketRegion.close();
-      } else {
-        this.stats.close();
-      }
-    }
-  }
-
-  /**
-   * Returns a copy of this LRU-based eviction controller. This method is a artifact when capacity
-   * controllers were used on a <code>Region</code>
-   */
-  @Override
-  public Object clone() throws CloneNotSupportedException {
-    synchronized (this) {
-      LRUAlgorithm clone = (LRUAlgorithm) super.clone();
-      clone.stats = null;
-      synchronized (clone) {
-        clone.helper = clone.createLRUHelper();
-      }
-      return clone;
-    }
-  }
-
-  /**
-   * Return true if the specified capacity controller is compatible with this
-   */
-  @Override
-  public boolean equals(Object cc) {
-    if (cc == null)
-      return false;
-    if (!getClass().isAssignableFrom(cc.getClass()))
-      return false;
-    LRUAlgorithm other = (LRUAlgorithm) cc;
-    if (!other.evictionAction.equals(this.evictionAction))
-      return false;
-    return true;
-  }
-
-  /**
-   * Note that we just need to make sure that equal objects return equal hashcodes; nothing really
-   * elaborate is done here.
-   */
-  @Override
-  public int hashCode() {
-    return this.evictionAction.hashCode();
-  }
-
-  /**
-   * Force subclasses to have a reasonable <code>toString</code>
-   *
-   * @since GemFire 4.0
-   */
-  @Override
-  public abstract String toString();
-
-  /**
-   * A partial implementation of the <code>EnableLRU</code> interface that contains code common to
-   * all <code>LRUAlgorithm</code>s.
-   */
-  protected abstract class AbstractEnableLRU implements EnableLRU {
-
-    /** The region whose capacity is controller by this eviction controller */
-    private transient volatile String regionName;
-
-    public long limit() {
-      if (stats == null) {
-        throw new InternalGemFireException(
-            LocalizedStrings.LRUAlgorithm_LRU_STATS_IN_EVICTION_CONTROLLER_INSTANCE_SHOULD_NOT_BE_NULL
-                .toLocalizedString());
-      }
-      if (bucketRegion != null) {
-        return bucketRegion.getLimit();
-      }
-      return stats.getLimit();
-    }
-
-    public String getRegionName() {
-      return this.regionName;
-    }
-
-    public void setRegionName(Object region) {
-      String fullPathName;
-      if (region instanceof Region) {
-        fullPathName = ((Region) region).getFullPath();
-      } else if (region instanceof PlaceHolderDiskRegion) {
-        PlaceHolderDiskRegion phdr = (PlaceHolderDiskRegion) region;
-        if (phdr.isBucket()) {
-          fullPathName = phdr.getPrName();
-        } else {
-          fullPathName = phdr.getName();
-        }
-      } else {
-        throw new IllegalStateException("expected Region or PlaceHolderDiskRegion");
-      }
-      if (this.regionName != null && !this.regionName.equals(fullPathName)) {
-        throw new IllegalArgumentException(
-            LocalizedStrings.LRUAlgorithm_LRU_EVICTION_CONTROLLER_0_ALREADY_CONTROLS_THE_CAPACITY_OF_1_IT_CANNOT_ALSO_CONTROL_THE_CAPACITY_OF_REGION_2
-                .toLocalizedString(
-                    new Object[] {LRUAlgorithm.this, this.regionName, fullPathName}));
-      }
-      this.regionName = fullPathName; // store the name not the region since
-      // region is not fully constructed yet
-    }
-
-    protected void setStats(LRUStatistics stats) {
-      LRUAlgorithm.this.stats = stats;
-    }
-
-    public LRUStatistics initStats(Object region, StatisticsFactory sf) {
-      setRegionName(region);
-      final LRUStatistics stats = new LRUStatistics(sf, getRegionName(), this);
-      stats.setLimit(LRUAlgorithm.this.getLimit());
-      stats.setDestroysLimit(1000);
-      setStats(stats);
-      return stats;
-    }
-
-    public LRUStatistics getStats() {
-      return LRUAlgorithm.this.stats;
-    }
-
-    public EvictionAction getEvictionAction() {
-      return LRUAlgorithm.this.evictionAction;
-    }
-
-    public void afterEviction() {
-      // Do nothing
-    }
-
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/NewLIFOClockHand.java b/geode-core/src/main/java/org/apache/geode/internal/cache/lru/NewLIFOClockHand.java
deleted file mode 100755
index eec95547c1..0000000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/NewLIFOClockHand.java
+++ /dev/null
@@ -1,94 +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.geode.internal.cache.lru;
-
-import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.InternalRegionArguments;
-import org.apache.geode.internal.cache.entries.AbstractRegionEntry;
-
-/**
- * NewLIFOClockHand holds the behavior for LIFO logic , Overwriting getLRUEntry() to return most
- * recently added Entry
- *
- * @since GemFire 5.7
- */
-
-public class NewLIFOClockHand extends NewLRUClockHand {
-  /*
-   * constructor
-   */
-  public NewLIFOClockHand(Object region, EnableLRU ccHelper,
-      InternalRegionArguments internalRegionArgs) {
-    super(region, ccHelper, internalRegionArgs);
-  }
-
-  public NewLIFOClockHand(Region region, EnableLRU ccHelper, NewLRUClockHand oldList) {
-    super(region, ccHelper, oldList);
-  }
-
-  /*
-   * Fetch the tail member which should be the last added value and remove it from the list
-   */
-  protected LRUClockNode getTailEntry() {
-    LRUClockNode aNode = null;
-    synchronized (this.lock) {
-      aNode = this.tail.prevLRUNode();
-      if (aNode == this.head) {
-        return null; // end of lru list
-      }
-      // remove entry from list
-      LRUClockNode prev = aNode.prevLRUNode();
-      prev.setNextLRUNode(this.tail);
-      this.tail.setPrevLRUNode(prev);
-      aNode.setNextLRUNode(null);
-      aNode.setPrevLRUNode(null);
-      super.size--;
-    }
-    return aNode;
-  }
-
-  /*
-   * return the Entry that is considered most recently used and available to be evicted to overflow
-   */
-  @Override
-  public LRUClockNode getLRUEntry() {
-    long numEvals = 0;
-    LRUClockNode aNode = null;
-    // search for entry to return from list
-    for (;;) {
-      aNode = getTailEntry();
-      // end of Lifo list stop searching
-      if (aNode == null) {
-        break;
-      }
-      numEvals++;
-      synchronized (aNode) {
-        // look for another entry if in transaction
-        boolean inUseByTransaction = false;
-        if (aNode instanceof AbstractRegionEntry) {
-          if (((AbstractRegionEntry) aNode).isInUseByTransaction()) {
-            inUseByTransaction = true;
-          }
-        }
-        // if entry NOT used by transaction and NOT evicted return entry
-        if (!inUseByTransaction && !aNode.testEvicted()) {
-          break;
-        }
-      }
-    }
-    this.stats().incEvaluations(numEvals);
-    return aNode;
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/NewLRUClockHand.java b/geode-core/src/main/java/org/apache/geode/internal/cache/lru/NewLRUClockHand.java
deleted file mode 100755
index 37cb2c19d4..0000000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/NewLRUClockHand.java
+++ /dev/null
@@ -1,464 +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.geode.internal.cache.lru;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.StatisticsFactory;
-import org.apache.geode.cache.Region;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.internal.cache.*;
-import org.apache.geode.internal.cache.entries.AbstractRegionEntry;
-import org.apache.geode.internal.cache.versions.RegionVersionVector;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.log4j.LocalizedMessage;
-import org.apache.geode.internal.logging.log4j.LogMarker;
-
-/**
- * AbstractLRUClockHand holds the lrulist, and the behavior for maintaining the list in a cu-pipe
- * and determining the next entry to be removed. Each EntriesMap that supports LRU holds one of
- * these.
- */
-public class NewLRUClockHand {
-  private static final Logger logger = LogService.getLogger();
-
-  private BucketRegion bucketRegion = null;
-
-  /** The last node in the LRU list after which all new nodes are added */
-  protected LRUClockNode tail = new GuardNode();
-
-  /** The starting point in the LRU list for searching for the LRU node */
-  protected LRUClockNode head = new GuardNode();
-
-  /** The object for locking the head of the cu-pipe. */
-  protected final HeadLock lock;
-
-  /** Description of the Field */
-  private final LRUStatistics stats;
-  /** Counter for the size of the LRU list */
-  protected int size = 0;
-
-  public static final boolean debug =
-      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "verbose-lru-clock");
-
-  private static final int maxEntries;
-
-  static {
-    String squelch = System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "lru.maxSearchEntries");
-    if (squelch == null)
-      maxEntries = -1;
-    else
-      maxEntries = Integer.parseInt(squelch);
-  }
-
-  /** only used by enhancer */
-  // protected NewLRUClockHand( ) { }
-
-  // private long size = 0;
-
-  public NewLRUClockHand(Object region, EnableLRU ccHelper,
-      InternalRegionArguments internalRegionArgs) {
-    setBucketRegion(region);
-    this.lock = new HeadLock();
-    // behavior relies on a single evicted node in the pipe when the pipe is empty.
-    initHeadAndTail();
-    if (this.bucketRegion != null) {
-      this.stats = internalRegionArgs.getPartitionedRegion() != null
-          ? internalRegionArgs.getPartitionedRegion().getEvictionController().stats : null;
-    } else {
-      LRUStatistics tmp = null;
-      if (region instanceof PlaceHolderDiskRegion) {
-        tmp = ((PlaceHolderDiskRegion) region).getPRLRUStats();
-      } else if (region instanceof PartitionedRegion) {
-        tmp = ((PartitionedRegion) region).getPRLRUStatsDuringInitialization(); // bug 41938
-        PartitionedRegion pr = (PartitionedRegion) region;
-        if (tmp != null) {
-          pr.getEvictionController().stats = tmp;
-        }
-      }
-      if (tmp == null) {
-        StatisticsFactory sf = GemFireCacheImpl.getExisting("").getDistributedSystem();
-        tmp = ccHelper.initStats(region, sf);
-      }
-      this.stats = tmp;
-    }
-  }
-
-  public void setBucketRegion(Object r) {
-    if (r instanceof BucketRegion) {
-      this.bucketRegion = (BucketRegion) r; // see bug 41388
-    }
-  }
-
-  public NewLRUClockHand(Region region, EnableLRU ccHelper, NewLRUClockHand oldList) {
-    setBucketRegion(region);
-    this.lock = new HeadLock();
-    // behavior relies on a single evicted node in the pipe when the pipe is empty.
-    initHeadAndTail();
-    if (oldList.stats == null) {
-      // see bug 41388
-      StatisticsFactory sf = region.getCache().getDistributedSystem();
-      this.stats = ccHelper.initStats(region, sf);
-    } else {
-      this.stats = oldList.stats;
-      if (this.bucketRegion != null) {
-        this.stats.decrementCounter(this.bucketRegion.getCounter());
-        this.bucketRegion.resetCounter();
-      } else {
-        this.stats.resetCounter();
-      }
-    }
-  }
-
-  /** Description of the Method */
-  public void close() {
-    closeStats();
-    if (bucketRegion != null)
-      bucketRegion.close();
-  }
-
-  public void closeStats() {
-    LRUStatistics ls = this.stats;
-    if (ls != null) {
-      ls.close();
-    }
-  }
-
-  /**
-   * Adds a new lru node for the entry between the current tail and head of the list.
-   *
-   * @param aNode Description of the Parameter
-   */
-  public void appendEntry(final LRUClockNode aNode) {
-    synchronized (this.lock) {
-      if (aNode.nextLRUNode() != null || aNode.prevLRUNode() != null) {
-        return;
-      }
-
-      if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
-        logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
-            .create(LocalizedStrings.NewLRUClockHand_ADDING_ANODE_TO_LRU_LIST, aNode));
-      }
-      aNode.setNextLRUNode(this.tail);
-      this.tail.prevLRUNode().setNextLRUNode(aNode);
-      aNode.setPrevLRUNode(this.tail.prevLRUNode());
-      this.tail.setPrevLRUNode(aNode);
-
-      this.size++;
-    }
-  }
-
-  /**
-   * return the head entry in the list preserving the cupipe requirement of at least one entry left
-   * in the list
-   */
-  private LRUClockNode getHeadEntry() {
-    synchronized (lock) {
-      LRUClockNode aNode = NewLRUClockHand.this.head.nextLRUNode();
-      if (aNode == this.tail) {
-        return null;
-      }
-
-      LRUClockNode next = aNode.nextLRUNode();
-      this.head.setNextLRUNode(next);
-      next.setPrevLRUNode(this.head);
-
-      aNode.setNextLRUNode(null);
-      aNode.setPrevLRUNode(null);
-      this.size--;
-      return aNode;
-    }
-  }
-
-
-  /**
-   * return the Entry that is considered least recently used. The entry will no longer be in the
-   * pipe (unless it is the last empty marker).
-   */
-  public LRUClockNode getLRUEntry() {
-    long numEvals = 0;
-
-    for (;;) {
-      LRUClockNode aNode = null;
-      aNode = getHeadEntry();
-
-      if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
-        logger.trace(LogMarker.LRU_CLOCK, "lru considering {}", aNode);
-      }
-
-      if (aNode == null) { // hit the end of the list
-        this.stats.incEvaluations(numEvals);
-        return aNode;
-      } // hit the end of the list
-
-      numEvals++;
-
-      // If this Entry is part of a transaction, skip it since
-      // eviction should not cause commit conflicts
-      synchronized (aNode) {
-        if (aNode instanceof AbstractRegionEntry) {
-          if (((AbstractRegionEntry) aNode).isInUseByTransaction()) {
-            if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
-              logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage.create(
-                  LocalizedStrings.NewLRUClockHand_REMOVING_TRANSACTIONAL_ENTRY_FROM_CONSIDERATION));
-            }
-            continue;
-          }
-        }
-        if (aNode.testEvicted()) {
-          if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
-            logger.trace(LogMarker.LRU_CLOCK,
-                LocalizedMessage.create(LocalizedStrings.NewLRUClockHand_DISCARDING_EVICTED_ENTRY));
-          }
-          continue;
-        }
-
-        // At this point we have any acceptable entry. Now
-        // use various criteria to determine if it's good enough
-        // to return, or if we need to add it back to the list.
-        if (maxEntries > 0 && numEvals > maxEntries) {
-          if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
-            logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
-                .create(LocalizedStrings.NewLRUClockHand_GREEDILY_PICKING_AN_AVAILABLE_ENTRY));
-          }
-          this.stats.incGreedyReturns(1);
-          // fall through, return this node
-        } else if (aNode.testRecentlyUsed()) {
-          // Throw it back, it's in the working set
-          aNode.unsetRecentlyUsed();
-          // aNode.setInList();
-          if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
-            logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
-                .create(LocalizedStrings.NewLRUClockHand_SKIPPING_RECENTLY_USED_ENTRY, aNode));
-          }
-          appendEntry(aNode);
-          continue; // keep looking
-        } else {
-          if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
-            logger.trace(LogMarker.LRU_CLOCK, LocalizedMessage
-                .create(LocalizedStrings.NewLRUClockHand_RETURNING_UNUSED_ENTRY, aNode));
-          }
-          // fall through, return this node
-        }
-
-        // Return the current node.
-        this.stats.incEvaluations(numEvals);
-        return aNode;
-      } // synchronized
-    } // for
-  }
-
-  public void dumpList() {
-    final boolean isDebugEnabled = logger.isTraceEnabled(LogMarker.LRU_CLOCK);
-    if (!isDebugEnabled) {
-      return;
-    }
-    synchronized (lock) {
-      int idx = 1;
-      for (LRUClockNode aNode = this.head; aNode != null; aNode = aNode.nextLRUNode()) {
-        if (isDebugEnabled) {
-          logger.trace(LogMarker.LRU_CLOCK, "  ({}) {}", (idx++), aNode);
-        }
-      }
-    }
-  }
-
-  public long getExpensiveListCount() {
-    synchronized (lock) {
-      long count = 0;
-      for (LRUClockNode aNode = this.head.nextLRUNode(); aNode != this.tail; aNode =
-          aNode.nextLRUNode()) {
-        count++;
-      }
-      return count;
-    }
-  }
-
-  public String getAuditReport() {
-    LRUClockNode h = this.head;
-    int totalNodes = 0;
-    int evictedNodes = 0;
-    int usedNodes = 0;
-    while (h != null) {
-      totalNodes++;
-      if (h.testEvicted())
-        evictedNodes++;
-      if (h.testRecentlyUsed())
-        usedNodes++;
-      h = h.nextLRUNode();
-    }
-    StringBuffer result = new StringBuffer(128);
-    result.append("LRUList Audit: listEntries = ").append(totalNodes).append(" evicted = ")
-        .append(evictedNodes).append(" used = ").append(usedNodes);
-    return result.toString();
-  }
-
-  /** unsynchronized audit...only run after activity has ceased. */
-  public void audit() {
-    System.out.println(getAuditReport());
-  }
-
-  /** remove an entry from the pipe... (marks it evicted to be skipped later) */
-  public boolean unlinkEntry(LRUClockNode entry) {
-    if (logger.isTraceEnabled(LogMarker.LRU_CLOCK)) {
-      logger.trace(LogMarker.LRU_CLOCK,
-          LocalizedMessage.create(LocalizedStrings.NewLRUClockHand_UNLINKENTRY_CALLED, entry));
-    }
-    entry.setEvicted();
-    stats().incDestroys();
-    synchronized (lock) {
-      LRUClockNode next = entry.nextLRUNode();
-      LRUClockNode prev = entry.prevLRUNode();
-      if (next == null || prev == null) {
-        // not in the list anymore.
-        return false;
-      }
-      next.setPrevLRUNode(prev);
-      prev.setNextLRUNode(next);
-      entry.setNextLRUNode(null);
-      entry.setPrevLRUNode(null);
-      this.size--;
-    }
-    return true;
-  }
-
-  /**
-   * Get the modifier for lru based statistics.
-   *
-   * @return The LRUStatistics for this Clock hand's region.
-   */
-  public LRUStatistics stats() {
-    return this.stats;
-  }
-
-  /**
-   * called when an LRU map is cleared... resets stats and releases prev and next.
-   */
-
-  public void clear(RegionVersionVector rvv) {
-    if (rvv != null) {
-      return; // when concurrency checks are enabled the clear operation removes entries iteratively
-    }
-    synchronized (this.lock) {
-      if (bucketRegion != null) {
-        this.stats.decrementCounter(bucketRegion.getCounter());
-        bucketRegion.resetCounter();
-      } else {
-        this.stats.resetCounter();
-      }
-      initHeadAndTail();
-      // LRUClockNode node = this.tail;
-      // node.setEvicted();
-      //
-      // // NYI need to walk the list and call unsetInList for each one.
-      //
-      // // tail's next should already be null.
-      // setHead( node );
-    }
-  }
-
-  private void initHeadAndTail() {
-    // I'm not sure, but I think it's important that we
-    // drop the references to the old head and tail on a region clear
-    // That will prevent any concurrent operations that are messing
-    // with existing nodes from screwing up the head and tail after
-    // the clear.
-    // Dan 9/23/09
-    this.head = new GuardNode();
-    this.tail = new GuardNode();
-    this.head.setNextLRUNode(this.tail);
-    this.tail.setPrevLRUNode(this.head);
-    this.size = 0;
-  }
-
-  /**
-   * Get size of LRU queue
-   *
-   * @return size
-   */
-  public int size() {
-    return size;
-  }
-
-  /** perform work of clear(), after subclass has properly synchronized */
-  // private void internalClear() {
-  // stats().resetCounter();
-  // LRUClockNode node = this.tail;
-  // node.setEvicted();
-  //
-  // // NYI need to walk the list and call unsetInList for each one.
-  //
-  // // tail's next should already be null.
-  // setHead( node );
-  // }
-
-  /** Marker class name to identify the lock more easily in thread dumps */
-  protected static class HeadLock extends Object {
-  }
-
-  private static class GuardNode implements LRUClockNode {
-
-    private LRUClockNode next;
-    LRUClockNode prev;
-
-    public int getEntrySize() {
-      return 0;
-    }
-
-    public LRUClockNode nextLRUNode() {
-      return next;
-    }
-
-    public LRUClockNode prevLRUNode() {
-      return prev;
-    }
-
-    public void setEvicted() {
-
-    }
-
-    public void setNextLRUNode(LRUClockNode next) {
-      this.next = next;
-    }
-
-    public void setPrevLRUNode(LRUClockNode prev) {
-      this.prev = prev;
-    }
-
-    public void setRecentlyUsed() {}
-
-    public boolean testEvicted() {
-      return false;
-    }
-
-    public boolean testRecentlyUsed() {
-      return false;
-    }
-
-    public void unsetEvicted() {}
-
-    public void unsetRecentlyUsed() {}
-
-    public int updateEntrySize(EnableLRU ccHelper) {
-      return 0;
-    }
-
-    public int updateEntrySize(EnableLRU ccHelper, Object value) {
-      return 0;
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/Sizeable.java b/geode-core/src/main/java/org/apache/geode/internal/cache/lru/Sizeable.java
deleted file mode 100644
index 8b1fd4a5bb..0000000000
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/Sizeable.java
+++ /dev/null
@@ -1,60 +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.geode.internal.cache.lru;
-
-/**
- * An interface that allows an object to define its own size.<br>
- *
- * <b>Sample Implementation</b><br>
- * <code>public int getSizeInBytes(){</code><br>
- * // The sizes of the primitive as well as object instance variables are calculated:<br>
- *
- * <code>int size = 0;</code><br>
- *
- * // Add overhead for this instance.<br>
- * <code>size += Sizeable.PER_OBJECT_OVERHEAD;</code><br>
- *
- * // Add object references (implements Sizeable)<br>
- * // value reference = 4 bytes <br>
- *
- * <code>size += 4;</code><br>
- *
- * // Add primitive instance variable size<br>
- * // byte bytePr = 1 byte<br>
- * // boolean flag = 1 byte<br>
- *
- * <code>size += 2;</code><br>
- *
- * // Add individual object size<br>
- * <code>size += (value.getSizeInBytes());</code><br>
- *
- * <code>return size;</code><br>
- * }<br>
- *
- *
- * @since GemFire 3.2
- */
-public interface Sizeable {
-
-  /** The overhead of an object in the VM in bytes */
-  public static final int PER_OBJECT_OVERHEAD = 8; // TODO for a 64bit jvm with small oops this is
-                                                   // 12; for other 64bit jvms it is 16
-
-  /**
-   * Returns the size (in bytes) of this object including the {@link #PER_OBJECT_OVERHEAD}.
-   */
-  public int getSizeInBytes();
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientUpdateMessageImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientUpdateMessageImpl.java
index 33358a3628..9acc003ea0 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientUpdateMessageImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientUpdateMessageImpl.java
@@ -24,7 +24,6 @@
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -33,29 +32,25 @@
 import org.apache.geode.DataSerializer;
 import org.apache.geode.GemFireIOException;
 import org.apache.geode.InternalGemFireError;
-import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.internal.cq.InternalCqQuery;
 import org.apache.geode.cache.util.ObjectSizer;
 import org.apache.geode.internal.DSCODE;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Sendable;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.cache.CachedDeserializable;
 import org.apache.geode.internal.cache.CachedDeserializableFactory;
 import org.apache.geode.internal.cache.EntryEventImpl;
 import org.apache.geode.internal.cache.EntryEventImpl.NewValueImporter;
-import org.apache.geode.internal.cache.EntryEventImpl.SerializedCacheValueImpl;
 import org.apache.geode.internal.cache.EnumListenerEvent;
 import org.apache.geode.internal.cache.EventID;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.Token;
 import org.apache.geode.internal.cache.WrappedCallbackArgument;
 import org.apache.geode.internal.cache.ha.HAContainerRegion;
-import org.apache.geode.internal.cache.lru.Sizeable;
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.versions.VersionTag;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.offheap.MemoryAllocatorImpl;
+import org.apache.geode.internal.size.Sizeable;
 
 
 /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/HAEventWrapper.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/HAEventWrapper.java
index 9ace480263..fc6921635f 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/HAEventWrapper.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/HAEventWrapper.java
@@ -31,12 +31,12 @@
 import org.apache.geode.internal.cache.Conflatable;
 import org.apache.geode.internal.cache.EnumListenerEvent;
 import org.apache.geode.internal.cache.EventID;
-import org.apache.geode.internal.cache.lru.Sizeable;
 import org.apache.geode.internal.cache.tier.sockets.ClientUpdateMessageImpl.ClientCqConcurrentMap;
 import org.apache.geode.internal.cache.tier.sockets.ClientUpdateMessageImpl.CqNameToOp;
 import org.apache.geode.internal.cache.tier.sockets.ClientUpdateMessageImpl.CqNameToOpHashMap;
 import org.apache.geode.internal.cache.tier.sockets.ClientUpdateMessageImpl.CqNameToOpSingleEntry;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.size.Sizeable;
 
 /**
  * This new class acts as a wrapper for the existing <code>ClientUpdateMessageImpl</code>. Now, the
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderEventImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderEventImpl.java
index 4d201b2124..5b1ba54e6a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderEventImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderEventImpl.java
@@ -24,14 +24,11 @@
 import org.apache.geode.InternalGemFireError;
 import org.apache.geode.cache.CacheEvent;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.EntryEvent;
 import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.SerializedCacheValue;
 import org.apache.geode.cache.asyncqueue.AsyncEvent;
 import org.apache.geode.cache.util.ObjectSizer;
 import org.apache.geode.cache.wan.EventSequenceID;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
@@ -45,8 +42,6 @@
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.Token;
 import org.apache.geode.internal.cache.WrappedCallbackArgument;
-import org.apache.geode.internal.cache.lru.Sizeable;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerHelper;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.offheap.OffHeapHelper;
 import org.apache.geode.internal.offheap.ReferenceCountHelper;
@@ -56,6 +51,7 @@
 import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Retained;
 import org.apache.geode.internal.offheap.annotations.Unretained;
+import org.apache.geode.internal.size.Sizeable;
 
 /**
  * Class <code>GatewaySenderEventImpl</code> represents an event sent between
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParser.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParser.java
index 5db6d53d72..a80997ccb4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParser.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheXmlParser.java
@@ -112,8 +112,7 @@
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.PartitionAttributesImpl;
 import org.apache.geode.internal.cache.PartitionedRegionHelper;
-import org.apache.geode.internal.cache.lru.LRUCapacityController;
-import org.apache.geode.internal.cache.lru.MemLRUCapacityController;
+import org.apache.geode.internal.cache.eviction.CountLRUEviction;
 import org.apache.geode.internal.datasource.ConfigProperty;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.jndi.JNDIInvoker;
@@ -2110,7 +2109,7 @@ private void endCustomExpiry() {
    */
   private void startLRUEntryCount(Attributes atts) {
     final String maximum = atts.getValue(MAXIMUM);
-    int max = LRUCapacityController.DEFAULT_MAXIMUM_ENTRIES;
+    int max = CountLRUEviction.DEFAULT_MAXIMUM_ENTRIES;
     if (maximum != null) {
       max = parseInt(maximum);
     }
@@ -2137,7 +2136,7 @@ private void startLRUMemorySize(Attributes atts) {
       action = EvictionAction.parseAction(lruAction);
     }
     String maximum = atts.getValue(MAXIMUM);
-    int max = MemLRUCapacityController.DEFAULT_MAXIMUM_MEGABYTES;
+    int max = EvictionAttributes.DEFAULT_MEMORY_MAXIMUM;
     if (maximum != null) {
       max = parseInt(maximum);
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
index e37520a928..7a3b1d5ca5 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
@@ -1305,7 +1305,7 @@
   public static final StringId NewLRUClockHand_SKIPPING_RECENTLY_USED_ENTRY =
       new StringId(1817, "skipping recently used entry {0}");
   public static final StringId NewLRUClockHand_UNLINKENTRY_CALLED =
-      new StringId(1818, "unlinkEntry called for {0}");
+      new StringId(1818, "destroyEntry called for {0}");
 
   public static final StringId Oplog_FAILED_READING_FILE_DURING_RECOVERY_FROM_0 =
       new StringId(1820, "Failed to read file during recovery from \"{0}\"");
diff --git a/geode-core/src/main/java/org/apache/geode/internal/lang/SystemPropertyHelper.java b/geode-core/src/main/java/org/apache/geode/internal/lang/SystemPropertyHelper.java
index 4063307cac..448ab21049 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/lang/SystemPropertyHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/lang/SystemPropertyHelper.java
@@ -14,6 +14,10 @@
  */
 package org.apache.geode.internal.lang;
 
+import java.util.Optional;
+
+import org.apache.geode.annotations.Experimental;
+
 /**
  * The SystemPropertyHelper class is an helper class for accessing system properties used in geode.
  * The method name to get the system property should be the same as the system property name.
@@ -25,6 +29,40 @@
   private static final String GEODE_PREFIX = "geode.";
   private static final String GEMFIRE_PREFIX = "gemfire.";
 
+
+  /**
+   * Setting this to "true" enables a new asynchronous eviction algorithm. For more details see
+   * {@link org.apache.geode.internal.cache.eviction.LRUListWithSyncSorting}.
+   *
+   * @since Geode 1.4.0
+   */
+  @Experimental
+  public static final String EVICTION_SCAN_ASYNC = "EvictionScanAsync";
+
+  /**
+   * This property allows the maximum number of threads used for asynchronous eviction scanning to
+   * be configured. It defaults to "Math.max((Runtime.getRuntime().availableProcessors() / 4), 1)".
+   * For more details see {@link org.apache.geode.internal.cache.eviction.LRUListWithSyncSorting}.
+   *
+   * @since Geode 1.4.0
+   */
+  @Experimental
+  public static final String EVICTION_SCAN_MAX_THREADS = "EvictionScanMaxThreads";
+
+  /**
+   * This property allows configuration of the threshold percentage at which an asynchronous scan is
+   * started. If the number of entries that have been recently used since the previous scan divided
+   * by total number of entries exceeds the threshold then a scan is started. The default threshold
+   * is 25. If the threshold is less than 0 or greater than 100 then the default threshold is used.
+   * For more details see {@link org.apache.geode.internal.cache.eviction.LRUListWithSyncSorting}.
+   *
+   * @since Geode 1.4.0
+   */
+  @Experimental
+  public static final String EVICTION_SCAN_THRESHOLD_PERCENT = "EvictionScanThresholdPercent";
+
+  public static final String EVICTION_SEARCH_MAX_ENTRIES = "lru.maxSearchEntries";
+
   /**
    * This method will try to look up "geode." and "gemfire." versions of the system property. It
    * will check and prefer "geode." setting first, then try to check "gemfire." setting.
@@ -32,7 +70,7 @@
    * @param name system property name set in Geode
    * @return a boolean value of the system property
    */
-  private static boolean getProductBooleanProperty(String name) {
+  public static boolean getProductBooleanProperty(String name) {
     String property = System.getProperty(GEODE_PREFIX + name);
     if (property != null) {
       return Boolean.getBoolean(GEODE_PREFIX + name);
@@ -40,6 +78,19 @@ private static boolean getProductBooleanProperty(String name) {
     return Boolean.getBoolean(GEMFIRE_PREFIX + name);
   }
 
+  public static Optional<Integer> getProductIntegerProperty(String name) {
+    Integer propertyValue = Integer.getInteger(GEODE_PREFIX + name);
+    if (propertyValue == null) {
+      propertyValue = Integer.getInteger(GEMFIRE_PREFIX + name);
+    }
+
+    if (propertyValue != null) {
+      return Optional.of(propertyValue);
+    } else {
+      return Optional.empty();
+    }
+  }
+
   /**
    * As of Geode 1.4.0, a region set operation will be in a transaction even if it is the first
    * operation in the transaction.
diff --git a/geode-core/src/main/java/org/apache/geode/internal/size/Sizeable.java b/geode-core/src/main/java/org/apache/geode/internal/size/Sizeable.java
new file mode 100644
index 0000000000..a7d3cc5623
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/size/Sizeable.java
@@ -0,0 +1,61 @@
+/*
+ * 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.geode.internal.size;
+
+/**
+ * An interface that allows an object to define its own size.
+ *
+ * <p>
+ * <b>Sample Implementation</b>
+ *
+ * <pre>
+ * public int getSizeInBytes() {
+ *   // The sizes of the primitive as well as object instance variables are calculated:
+ *   int size = 0;
+ *
+ *   // Add overhead for this instance.
+ *   size += Sizeable.PER_OBJECT_OVERHEAD;
+ *
+ *   // Add object references (implements Sizeable)
+ *   // value reference = 4 bytes
+ *   size += 4;
+ *
+ *   // Add primitive instance variable size
+ *   // byte bytePr = 1 byte
+ *   // boolean flag = 1 byte
+ *   size += 2;
+ *
+ *   // Add individual object size
+ *   size += (value.getSizeInBytes());
+ *
+ *   return size;
+ * }
+ * </pre>
+ *
+ * @since GemFire 3.2
+ */
+public interface Sizeable {
+
+  // TODO: for a 64bit jvm with small oops this is 12; for other 64bit jvms it is 16
+  /**
+   * The overhead of an object in the VM in bytes
+   */
+  int PER_OBJECT_OVERHEAD = 8;
+
+  /**
+   * Returns the size (in bytes) of this object including the {@link #PER_OBJECT_OVERHEAD}.
+   */
+  int getSizeInBytes();
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
index 2a6565ace2..26483bbeec 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
@@ -75,8 +75,8 @@
 import org.apache.geode.internal.cache.PartitionedRegionStats;
 import org.apache.geode.internal.cache.backup.BackupManager;
 import org.apache.geode.internal.cache.control.ResourceManagerStats;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.internal.cache.execute.FunctionServiceStats;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
@@ -513,7 +513,7 @@ public void addRegion(Region region) {
 
     LocalRegion l = (LocalRegion) region;
     if (l.getEvictionController() != null) {
-      LRUStatistics stats = l.getEvictionController().getLRUHelper().getStats();
+      EvictionStatistics stats = l.getEvictionController().getStatistics();
       if (stats != null) {
         addLRUStats(stats);
       }
@@ -531,7 +531,7 @@ public void addPartionRegionStats(PartitionedRegionStats parStats) {
     regionMonitor.addStatisticsToMonitor(parStats.getStats());
   }
 
-  public void addLRUStats(LRUStatistics lruStats) {
+  public void addLRUStats(EvictionStatistics lruStats) {
     regionMonitor.addStatisticsToMonitor(lruStats.getStats());
   }
 
@@ -546,7 +546,7 @@ public void removeRegion(Region region) {
 
     LocalRegion l = (LocalRegion) region;
     if (l.getEvictionController() != null) {
-      LRUStatistics stats = l.getEvictionController().getLRUHelper().getStats();
+      EvictionStatistics stats = l.getEvictionController().getStatistics();
       if (stats != null) {
         removeLRUStats(stats);
       }
@@ -564,7 +564,7 @@ public void removePartionRegionStats(PartitionedRegionStats parStats) {
     regionMonitor.removePartitionStatistics(parStats.getStats());
   }
 
-  public void removeLRUStats(LRUStatistics lruStats) {
+  public void removeLRUStats(EvictionStatistics lruStats) {
     regionMonitor.removeLRUStatistics(lruStats.getStats());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/RegionMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/RegionMBeanBridge.java
index 93da8b42e0..c7277416ea 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/RegionMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/RegionMBeanBridge.java
@@ -28,7 +28,7 @@
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.management.EvictionAttributesData;
 import org.apache.geode.management.FixedPartitionAttributesData;
 import org.apache.geode.management.MembershipAttributesData;
@@ -89,7 +89,7 @@
 
   private String member;
 
-  private LRUStatistics lruMemoryStats;
+  private EvictionStatistics lruMemoryStats;
 
   private CachePerfStats regionStats;
 
@@ -173,7 +173,7 @@ protected RegionMBeanBridge(Region<K, V> region) {
 
     LocalRegion l = (LocalRegion) region;
     if (l.getEvictionController() != null) {
-      LRUStatistics stats = l.getEvictionController().getLRUHelper().getStats();
+      EvictionStatistics stats = l.getEvictionController().getStatistics();
       if (stats != null) {
         regionMonitor.addStatisticsToMonitor(stats.getStats());
         EvictionAttributes ea = region.getAttributes().getEvictionAttributes();
diff --git a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
index e7cd276582..c4d24c6263 100644
--- a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
+++ b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
@@ -304,10 +304,6 @@ org/apache/geode/internal/cache/execute/NoResult,true,-4901369422864228848
 org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction,true,7851518767859544678
 org/apache/geode/internal/cache/ha/ThreadIdentifier$Bits,false,position:int,width:int
 org/apache/geode/internal/cache/ha/ThreadIdentifier$WanType,false
-org/apache/geode/internal/cache/lru/HeapLRUCapacityController,true,4970685814429530675,perEntryOverhead:int,sizer:org/apache/geode/cache/util/ObjectSizer
-org/apache/geode/internal/cache/lru/LRUAlgorithm,false,bucketRegion:org/apache/geode/internal/cache/BucketRegion,evictionAction:org/apache/geode/cache/EvictionAction
-org/apache/geode/internal/cache/lru/LRUCapacityController,true,-4383074909189355938,maximumEntries:int
-org/apache/geode/internal/cache/lru/MemLRUCapacityController,true,6364183985590572514,isOffHeap:boolean,limit:long,perEntryOverHead:int,sizer:org/apache/geode/cache/util/ObjectSizer
 org/apache/geode/internal/cache/partitioned/DumpB2NRegion$PrimaryInfo,true,6334695270795306178,hostToken:java/lang/String,isHosting:boolean,isPrimary:boolean
 org/apache/geode/internal/cache/partitioned/PRLocallyDestroyedException,true,-1291911181409686840
 org/apache/geode/internal/cache/partitioned/PartitionMemberInfoImpl,true,8245020687604034289,bucketCount:int,configuredMaxMemory:long,distributedMember:org/apache/geode/distributed/DistributedMember,primaryCount:int,size:long
diff --git a/geode-core/src/test/java/org/apache/geode/cache30/CacheXml66DUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache30/CacheXml66DUnitTest.java
index de56ca609e..5806c9877b 100644
--- a/geode-core/src/test/java/org/apache/geode/cache30/CacheXml66DUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache30/CacheXml66DUnitTest.java
@@ -15,7 +15,13 @@
 package org.apache.geode.cache30;
 
 import static org.apache.geode.distributed.ConfigurationProperties.ROLES;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -112,7 +118,6 @@
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PoolFactoryImpl;
 import org.apache.geode.internal.cache.functions.TestFunction;
-import org.apache.geode.internal.cache.lru.MemLRUCapacityController;
 import org.apache.geode.internal.cache.partitioned.fixed.QuarterPartitionResolver;
 import org.apache.geode.internal.cache.xmlcache.CacheCreation;
 import org.apache.geode.internal.cache.xmlcache.CacheTransactionManagerCreation;
@@ -2210,7 +2215,7 @@ public void testReplicatedRegionAttributesForMemLruWithoutMaxMem() throws Except
 
     assertEquals(ea.getAlgorithm(), EvictionAlgorithm.LRU_MEMORY);
     assertEquals(ea.getAction(), EvictionAction.OVERFLOW_TO_DISK);
-    assertEquals(ea.getMaximum(), MemLRUCapacityController.DEFAULT_MAXIMUM_MEGABYTES);
+    assertEquals(ea.getMaximum(), EvictionAttributes.DEFAULT_MEMORY_MAXIMUM);
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/cache30/DiskRegionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache30/DiskRegionDUnitTest.java
index f2fb5c7954..3e24427593 100644
--- a/geode-core/src/test/java/org/apache/geode/cache30/DiskRegionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache30/DiskRegionDUnitTest.java
@@ -48,8 +48,8 @@
 import org.apache.geode.internal.cache.DiskRegion;
 import org.apache.geode.internal.cache.DiskRegionStats;
 import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.lru.LRUCapacityController;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.CountLRUEviction;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.dunit.SerializableCallable;
@@ -94,11 +94,11 @@ public DiskRegionDUnitTest() {
   // }
 
   /**
-   * Returns the <code>LRUStatistics</code> for the given region
+   * Returns the <code>EvictionStatistics</code> for the given region
    */
-  protected LRUStatistics getLRUStats(Region region) {
+  protected EvictionStatistics getLRUStats(Region region) {
     final LocalRegion l = (LocalRegion) region;
-    return l.getEvictionController().getLRUHelper().getStats();
+    return l.getEvictionController().getStatistics();
   }
 
   //////// Test Methods
@@ -127,7 +127,7 @@ public void testDiskRegionOverflow() throws Exception {
     assertNotNull(dr);
 
     DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     assertNotNull(diskStats);
     assertNotNull(lruStats);
 
@@ -215,7 +215,7 @@ public void run2() throws CacheException {
       public void run2() throws CacheException {
         LocalRegion region = (LocalRegion) getRootRegion().getSubregion(name);
         // DiskRegion dr = region.getDiskRegion();
-        LRUStatistics lruStats = getLRUStats(region);
+        EvictionStatistics lruStats = getLRUStats(region);
         int i;
         for (i = 0; lruStats.getEvictions() <= 0; i++) {
           region.put(new Integer(i), new short[250]);
@@ -228,7 +228,7 @@ public void run2() throws CacheException {
       public void run2() throws CacheException {
         LocalRegion region = (LocalRegion) getRootRegion().getSubregion(name);
         // DiskRegion dr = region.getDiskRegion();
-        // LRUStatistics lruStats = getLRUStats(region);
+        // EvictionStatistics lruStats = getLRUStats(region);
         for (int i = 0; i < 10; i++) {
           region.put(new Integer(i), new int[250]);
         }
@@ -239,7 +239,7 @@ public void run2() throws CacheException {
       public void run2() throws CacheException {
         LocalRegion region = (LocalRegion) getRootRegion().getSubregion(name);
         // DiskRegion dr = region.getDiskRegion();
-        final LRUStatistics lruStats = getLRUStats(region);
+        final EvictionStatistics lruStats = getLRUStats(region);
         WaitCriterion ev = new WaitCriterion() {
           public boolean done() {
             return lruStats.getEvictions() > 6;
@@ -259,7 +259,7 @@ public String description() {
       public void run2() throws CacheException {
         LocalRegion region = (LocalRegion) getRootRegion().getSubregion(name);
         // DiskRegion dr = region.getDiskRegion();
-        // LRUStatistics lruStats = getLRUStats(region);
+        // EvictionStatistics lruStats = getLRUStats(region);
         for (int i = 0; i < 10000; i++) {
           region.put(String.valueOf(i), String.valueOf(i).getBytes());
         }
@@ -270,7 +270,7 @@ public void run2() throws CacheException {
       public void run2() throws CacheException {
         LocalRegion region = (LocalRegion) getRootRegion().getSubregion(name);
         // DiskRegion dr = region.getDiskRegion();
-        // LRUStatistics lruStats = getLRUStats(region);
+        // EvictionStatistics lruStats = getLRUStats(region);
         for (int i = 0; i < 10000; i++) {
           byte[] bytes = (byte[]) region.get(String.valueOf(i));
           assertEquals(String.valueOf(i), new String(bytes));
@@ -298,7 +298,7 @@ public void testNoFaults() throws Exception {
     Region region = createRegion(name, factory.create());
     DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     // Put in larger stuff until we start evicting
     int total;
@@ -362,7 +362,7 @@ public void run2() throws CacheException {
       public void run2() throws CacheException {
         LocalRegion region = (LocalRegion) getRootRegion().getSubregion(name);
         // DiskRegion dr = region.getDiskRegion();
-        LRUStatistics lruStats = getLRUStats(region);
+        EvictionStatistics lruStats = getLRUStats(region);
         for (int i = 0; lruStats.getEvictions() < 10; i++) {
           LogWriterUtils.getLogWriter().info("Put " + i);
           region.put(new Integer(i), new byte[1]);
@@ -376,7 +376,7 @@ public void run2() throws CacheException {
       public void run2() throws CacheException {
         LocalRegion region = (LocalRegion) getRootRegion().getSubregion(name);
         // DiskRegion dr = region.getDiskRegion();
-        LRUStatistics lruStats = getLRUStats(region);
+        EvictionStatistics lruStats = getLRUStats(region);
         assertEquals(10, lruStats.getEvictions());
 
         // Because we are DISTRIBUTED_ACK, we can rely on the order
@@ -413,7 +413,7 @@ public void testDestroy() throws Exception {
     Region region = createRegion(name, factory.create());
     DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     int total;
     for (total = 0; lruStats.getEvictions() < 40; total++) {
@@ -478,7 +478,7 @@ public void afterCreate2(EntryEvent event) {
     Region region = createRegion(name, factory.create());
     // DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     // DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     int total;
     for (total = 0; lruStats.getEvictions() < 20; total++) {
@@ -575,7 +575,7 @@ public void testValues() throws Exception {
     Region region = createRegion(name, factory.create());
     // DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     // DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     // Put in larger stuff until we start evicting
     int total;
@@ -860,7 +860,7 @@ public void testRegionEntryValues() throws Exception {
     Region region = createRegion(name, factory.create());
     // DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     // DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     // Put in larger stuff until we start evicting
     int total;
@@ -903,7 +903,7 @@ public void testInvalidate() throws Exception {
     Region region = createRegion(name, factory.create());
     // DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     // DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     // Put in larger stuff until we start evicting
     int total;
@@ -953,7 +953,7 @@ public void run2() throws CacheException {
       public void run2() throws CacheException {
         LocalRegion region = (LocalRegion) getRootRegion().getSubregion(name);
         // DiskRegion dr = region.getDiskRegion();
-        LRUStatistics lruStats = getLRUStats(region);
+        EvictionStatistics lruStats = getLRUStats(region);
         for (int i = 0; lruStats.getEvictions() < 10; i++) {
           LogWriterUtils.getLogWriter().info("Put " + i);
           region.put(new Integer(i), new byte[1]);
@@ -1034,7 +1034,7 @@ public void testOverflowUpdatedValue() throws Exception {
     Region region = createRegion(name, factory.create());
     // DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     // DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     // Put in larger stuff until we start evicting
     int total;
@@ -1089,7 +1089,7 @@ public void testTestHookStatistics() throws Exception {
     LocalRegion region = (LocalRegion) createRegion(name, factory.create());
     DiskRegion dr = region.getDiskRegion();
     DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     // Put in stuff until we start evicting
     int total;
@@ -1180,7 +1180,7 @@ public void testLowLevelGetMethods() throws Exception {
     LocalRegion region = (LocalRegion) createRegion(name, factory.create());
     // DiskRegion dr = region.getDiskRegion();
     // DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     // Put in larger stuff until we start evicting
     int total;
@@ -1214,7 +1214,7 @@ public void testLowLevelGetMethods() throws Exception {
   }
 
   /**
-   * Tests disk overflow with an entry-based {@link LRUCapacityController}.
+   * Tests disk overflow with an entry-based {@link CountLRUEviction}.
    */
   @Test
   public void testLRUCapacityController() throws CacheException {
@@ -1234,7 +1234,7 @@ public void testLRUCapacityController() throws CacheException {
     Region region = createRegion(name, factory.create());
     DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     flush(region);
 
@@ -1289,8 +1289,8 @@ public void testLRUCapacityController() throws CacheException {
   }
 
   /**
-   * Tests a disk-based region with an {@link LRUCapacityController} with size 1 and an eviction
-   * action of "overflow".
+   * Tests a disk-based region with an {@link CountLRUEviction} with size 1 and an eviction action
+   * of "overflow".
    */
   @Test
   public void testLRUCCSizeOne() throws CacheException {
@@ -1319,7 +1319,7 @@ public void close() {}
     factory.setDiskStoreName(ds.getName());
 
     Region region = createRegion(name, factory.create());
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     assertNotNull(lruStats);
 
     for (int i = 1; i <= 1; i++) {
diff --git a/geode-core/src/test/java/org/apache/geode/cache30/LRUEvictionControllerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache30/LRUEvictionControllerDUnitTest.java
index d01c3a3747..695e99b430 100644
--- a/geode-core/src/test/java/org/apache/geode/cache30/LRUEvictionControllerDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache30/LRUEvictionControllerDUnitTest.java
@@ -46,8 +46,8 @@
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
+import org.apache.geode.internal.cache.eviction.HeapEvictor;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.dunit.VM;
@@ -74,17 +74,17 @@ public LRUEvictionControllerDUnitTest() {
   }
 
   /**
-   * Returns the <code>LRUStatistics</code> for the given region
+   * Returns the <code>EvictionStatistics</code> for the given region
    */
-  private LRUStatistics getLRUStats(Region region) {
+  private EvictionStatistics getLRUStats(Region region) {
     final LocalRegion l = (LocalRegion) region;
-    return l.getEvictionController().getLRUHelper().getStats();
+    return l.getEvictionController().getStatistics();
   }
 
   //////// Test Methods
 
   /**
-   * Carefully verifies that region operations effect the {@link LRUStatistics} as expected.
+   * Carefully verifies that region operations effect the {@link EvictionStatistics} as expected.
    */
   @Test
   public void testRegionOperations() throws CacheException {
@@ -106,7 +106,7 @@ public void testRegionOperations() throws CacheException {
       region = createRegion(name, factory.create());
     }
 
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     assertNotNull(lruStats);
 
     for (int i = 1; i <= 10; i++) {
@@ -128,8 +128,8 @@ public void testRegionOperations() throws CacheException {
   }
 
   /**
-   * Carefully verifies that region operations effect the {@link LRUStatistics} as expected in the
-   * presense of a {@link CacheLoader}.
+   * Carefully verifies that region operations effect the {@link EvictionStatistics} as expected in
+   * the presense of a {@link CacheLoader}.
    */
   @Test
   public void testCacheLoader() throws CacheException {
@@ -159,7 +159,7 @@ public void close() {}
       region = createRegion(name, factory.create());
     }
 
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     assertNotNull(lruStats);
 
     for (int i = 1; i <= 10; i++) {
@@ -213,7 +213,7 @@ public void close() {}
       region = createRegion(name, factory.create());
     }
 
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     assertNotNull(lruStats);
 
     for (int i = 1; i <= 1; i++) {
@@ -432,10 +432,8 @@ public void run2() throws CacheException {
           assertEquals(numEntries, r[i].size());
           {
             LocalRegion lr = (LocalRegion) r[i];
-            assertEquals(maxEntries,
-                lr.getEvictionController().getLRUHelper().getStats().getLimit());
-            assertEquals(maxEntries,
-                lr.getEvictionController().getLRUHelper().getStats().getCounter());
+            assertEquals(maxEntries, lr.getEvictionController().getStatistics().getLimit());
+            assertEquals(maxEntries, lr.getEvictionController().getStatistics().getCounter());
           }
         }
       }
@@ -454,10 +452,8 @@ public void run2() throws CacheException {
           assertEquals(numEntries, r[i].size());
           {
             LocalRegion lr = (LocalRegion) r[i];
-            assertEquals(maxEntries,
-                lr.getEvictionController().getLRUHelper().getStats().getLimit());
-            assertEquals(maxEntries,
-                lr.getEvictionController().getLRUHelper().getStats().getCounter());
+            assertEquals(maxEntries, lr.getEvictionController().getStatistics().getLimit());
+            assertEquals(maxEntries, lr.getEvictionController().getStatistics().getCounter());
           }
         }
 
diff --git a/geode-core/src/test/java/org/apache/geode/cache30/MemLRUEvictionControllerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache30/MemLRUEvictionControllerDUnitTest.java
index 0c124deb60..771bdf2580 100644
--- a/geode-core/src/test/java/org/apache/geode/cache30/MemLRUEvictionControllerDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache30/MemLRUEvictionControllerDUnitTest.java
@@ -33,8 +33,8 @@
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.SharedLibrary;
 import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
-import org.apache.geode.internal.cache.lru.MemLRUCapacityController;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
+import org.apache.geode.internal.cache.eviction.MemoryLRUController;
 import org.apache.geode.internal.size.ReflectionSingleObjectSizer;
 import org.apache.geode.internal.size.WellKnownClassSizer;
 import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
@@ -60,22 +60,22 @@ public MemLRUEvictionControllerDUnitTest() {
   }
 
   /**
-   * Returns the <code>LRUStatistics</code> for the given region
+   * Returns the <code>EvictionStatistics</code> for the given region
    */
-  private LRUStatistics getLRUStats(Region region) {
+  private EvictionStatistics getLRUStats(Region region) {
     final LocalRegion l = (LocalRegion) region;
-    return l.getEvictionController().getLRUHelper().getStats();
+    return l.getEvictionController().getStatistics();
   }
 
   private int getEntryOverhead(Region region) {
     LocalRegion lRegion = (LocalRegion) region;
-    return ((MemLRUCapacityController) lRegion.getEvictionController()).getPerEntryOverhead();
+    return ((MemoryLRUController) lRegion.getEvictionController()).getPerEntryOverhead();
   }
 
   // ////// Test Methods
 
   /**
-   * Carefully verifies that region operations effect the {@link LRUStatistics} as expected.
+   * Carefully verifies that region operations effect the {@link EvictionStatistics} as expected.
    */
   @Test
   public void testRegionOperations() throws CacheException {
@@ -97,7 +97,7 @@ public void testRegionOperations() throws CacheException {
       region = createRegion(name, factory.create());
     }
 
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     assertNotNull(lruStats);
 
     String sampleKey = new String("10000");
@@ -158,7 +158,7 @@ public void testSizeClassesOnce() throws CacheException {
 
     Region region = createRegion(name, factory.create());
 
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     assertNotNull(lruStats);
 
     TestObject object = new TestObject(50);
diff --git a/geode-core/src/test/java/org/apache/geode/cache30/OffHeapLRUEvictionControllerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache30/OffHeapLRUEvictionControllerDUnitTest.java
index f30107464b..fcf1e0e5ad 100644
--- a/geode-core/src/test/java/org/apache/geode/cache30/OffHeapLRUEvictionControllerDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache30/OffHeapLRUEvictionControllerDUnitTest.java
@@ -23,7 +23,7 @@
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.OffHeapTestUtil;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
+import org.apache.geode.internal.cache.eviction.HeapEvictor;
 import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.junit.categories.DistributedTest;
diff --git a/geode-core/src/test/java/org/apache/geode/cache30/TestDiskRegion.java b/geode-core/src/test/java/org/apache/geode/cache30/TestDiskRegion.java
index c323516a80..f63b8d2719 100644
--- a/geode-core/src/test/java/org/apache/geode/cache30/TestDiskRegion.java
+++ b/geode-core/src/test/java/org/apache/geode/cache30/TestDiskRegion.java
@@ -34,7 +34,7 @@
 import org.apache.geode.internal.cache.DiskRegion;
 import org.apache.geode.internal.cache.DiskRegionStats;
 import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 
 /**
  * A little test program for testing (and debugging) disk regions.
@@ -45,11 +45,11 @@
 public class TestDiskRegion {
 
   /**
-   * Returns the <code>LRUStatistics</code> for the given region
+   * Returns the <code>EvictionStatistics</code> for the given region
    */
-  private static LRUStatistics getLRUStats(Region region) {
+  private static EvictionStatistics getLRUStats(Region region) {
     final LocalRegion l = (LocalRegion) region;
-    return l.getEvictionController().getLRUHelper().getStats();
+    return l.getEvictionController().getStatistics();
   }
 
   public static void main(String[] args) throws Exception {
@@ -70,7 +70,7 @@ public static void main(String[] args) throws Exception {
     DiskRegion dr = region.getDiskRegion();
     Assert.assertTrue(dr != null);
     DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     Assert.assertTrue(diskStats != null);
     Assert.assertTrue(lruStats != null);
 
@@ -156,7 +156,7 @@ public void afterUpdate(EntryEvent event) {
     LocalRegion region = (LocalRegion) cache.createRegion("TestDiskRegion", factory.create());
     DiskRegion dr = region.getDiskRegion();
     DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     BufferedReader br = new BufferedReader(new InputStreamReader(System.in));
     System.out.println("Hit enter to perform action");
@@ -183,7 +183,7 @@ public static void main4(String[] args) throws Exception {
     LocalRegion region = (LocalRegion) cache.createRegion("TestDiskRegion", factory.create());
     // DiskRegion dr = region.getDiskRegion();
     // DiskRegionStats diskStats = dr.getStats();
-    // LRUStatistics lruStats = getLRUStats(region);
+    // EvictionStatistics lruStats = getLRUStats(region);
 
     // int total;
     // for (total = 0; lruStats.getEvictions() > 100; total++) {
@@ -213,7 +213,7 @@ public static void main5(String[] args) throws Exception {
     LocalRegion region = (LocalRegion) cache.createRegion("TestDiskRegion", factory.create());
     // DiskRegion dr = region.getDiskRegion();
     // DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
 
     for (int i = 0; i < 10000; i++) {
       int[] array = new int[1000];
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractLRURegionMapTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractLRURegionMapTest.java
index 374b2ce6bc..871d2eb328 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractLRURegionMapTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractLRURegionMapTest.java
@@ -34,10 +34,8 @@ public void shouldBeMockable() throws Exception {
 
     when(mockAbstractLRURegionMap.centralizedLruUpdateCallback()).thenReturn(1);
 
-    mockAbstractLRURegionMap.audit();
     mockAbstractLRURegionMap.changeTotalEntrySize(1);
 
-    verify(mockAbstractLRURegionMap, times(1)).audit();
     verify(mockAbstractLRURegionMap, times(1)).changeTotalEntrySize(1);
 
     assertThat(mockAbstractLRURegionMap.centralizedLruUpdateCallback()).isEqualTo(1);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java
index de50c2c02e..af6bce5b88 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java
@@ -638,6 +638,18 @@ public void invokeInvalidateCallbacks(EnumListenerEvent eventType, EntryEventImp
       throw new UnsupportedOperationException();
     }
 
+    @Override
+    public long getEvictions() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Region createSubregion(String subregionName, RegionAttributes attrs,
+        InternalRegionArguments internalRegionArgs)
+        throws RegionExistsException, TimeoutException, IOException, ClassNotFoundException {
+      throw new UnsupportedOperationException();
+    }
+
     @Override
     protected boolean isCurrentlyLockGrantor() {
       throw new UnsupportedOperationException();
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug37244JUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/Bug37244JUnitTest.java
deleted file mode 100644
index 2ef4115391..0000000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug37244JUnitTest.java
+++ /dev/null
@@ -1,244 +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.geode.internal.cache;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.util.Properties;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.cache.*;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.internal.Assert;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-
-/**
- * Test for simluating the deadLock condition as in bug#37244
- *
- *
- */
-@Category(IntegrationTest.class)
-public class Bug37244JUnitTest {
-
-  private static Cache cache = null;
-
-  private static DistributedSystem distributedSystem = null;
-
-  protected static String regionName = "TestRegion";
-
-  /**
-   * Method for intializing the VM
-   */
-  private static void initializeVM() throws Exception {
-    Properties props = new Properties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(LOCATORS, "");
-    props.setProperty(LOG_LEVEL, "info"); // to keep diskPerf logs smaller
-    distributedSystem = DistributedSystem.connect(props);
-    cache = CacheFactory.create(distributedSystem);
-    assertNotNull(cache);
-    DiskStoreFactory dsf = cache.createDiskStoreFactory();
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-
-    File dir = new File("testingDirectoryDefault");
-    dir.mkdir();
-    dir.deleteOnExit();
-    File[] dirs = {dir};
-    dsf.setDiskDirsAndSizes(dirs, new int[] {Integer.MAX_VALUE});
-
-    dsf.setAutoCompact(false);
-    DirectoryHolder.SET_DIRECTORY_SIZE_IN_BYTES_FOR_TESTING_PURPOSES = true;
-    try {
-      factory.setDiskStoreName(dsf.create(regionName).getName());
-    } finally {
-      DirectoryHolder.SET_DIRECTORY_SIZE_IN_BYTES_FOR_TESTING_PURPOSES = false;
-    }
-    factory.setDiskSynchronous(true);
-    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
-
-    factory.setEvictionAttributes(
-        EvictionAttributes.createLRUEntryAttributes(1, EvictionAction.OVERFLOW_TO_DISK));
-    RegionAttributes attr = factory.create();
-    DistributedRegion distRegion = new DistributedRegion(regionName, attr, null,
-        (GemFireCacheImpl) cache, new InternalRegionArguments().setDestroyLockFlag(true)
-            .setRecreateFlag(false).setSnapshotInputStream(null).setImageTarget(null));
-    assertNotNull(distRegion);
-    ((AbstractLRURegionMap) distRegion.entries)._setLruList((new TestLRUClockHand(distRegion,
-        ((AbstractLRURegionMap) distRegion.entries)._getCCHelper())));
-    ((GemFireCacheImpl) cache).createVMRegion(regionName, attr,
-        new InternalRegionArguments().setInternalMetaRegion(distRegion).setDestroyLockFlag(true)
-            .setSnapshotInputStream(null).setImageTarget(null));
-
-  }
-
-  @Test
-  public void testPutWhileclear() {
-    try {
-      initializeVM();
-
-      assertNotNull(cache);
-      Region rgn = cache.getRegion(regionName);
-      assertNotNull(rgn);
-
-      // put two entries into the region
-      for (int i = 0; i < 2; i++) {
-        rgn.put(new Long(i), new Long(i));
-      }
-
-      // get an entry back
-      Long value = (Long) rgn.get(new Long(0));
-
-      // check for entry value
-      assertTrue("Test failed ", value.equals(new Long(0)));
-
-    } catch (Exception ex) {
-      ex.printStackTrace();
-      fail("Test failed");
-
-    } finally {
-      assertNotNull(cache);
-      Region rgn = cache.getRegion(regionName);
-      assertNotNull(rgn);
-      rgn.localDestroyRegion();
-      cache.close();
-
-    }
-
-  }
-
-  /**
-   * Test Implementation class of NewLRUClockHand for bug37244.
-   *
-   *
-   *
-   */
-
-  public static class TestLRUClockHand extends NewLRUClockHand {
-
-    protected static Object mutex = new Object();
-
-    // private String regionName = "TestRegion";
-
-    protected static boolean EXECUTE_AFTER_GET_CALL = false;
-
-    /**
-     * Constructor
-     *
-     * @param region
-     * @param ccHelper
-     */
-    public TestLRUClockHand(Region region, EnableLRU ccHelper) {
-      super(region, ccHelper, new InternalRegionArguments());
-
-    }
-
-    /**
-     * Overridden getLRUEntry method
-     */
-    public LRUClockNode getLRUEntry() {
-      if (EXECUTE_AFTER_GET_CALL) {
-        Cache cache = CacheFactory.getAnyInstance();
-        Assert.assertTrue(cache != null);
-        LocalRegion region = (LocalRegion) cache.getRegion(regionName);
-        Assert.assertTrue(region != null);
-        Thread clearThread = new Thread(new clearThread(region));
-        clearThread.start();
-        try {
-          synchronized (mutex) {
-            mutex.wait(10000);
-          }
-        } catch (InterruptedException ie) {
-          if (cache.getLogger().fineEnabled()) {
-            cache.getLogger().fine("TestLRUClockHand#getLRUEntry Got an interrupted Exception");
-          }
-          fail("interrupted");
-        }
-      }
-
-      LRUClockNode aNode = super.getLRUEntry();
-      return aNode;
-    }
-
-    /**
-     *
-     * clearThread
-     *
-     */
-    protected static class clearThread implements Runnable {
-      LocalRegion region = null;
-
-      clearThread(LocalRegion rgn) {
-        super();
-        this.region = rgn;
-
-      }
-
-      public void run() {
-        Cache cache = CacheFactory.getAnyInstance();
-        region.getDiskRegion().acquireWriteLock();
-        try {
-          Thread putThread = new Thread(new putThread(region));
-          putThread.start();
-          Thread.sleep(2000);
-          synchronized (mutex) {
-            mutex.notify();
-          }
-          Thread.sleep(5000);
-
-          region.clear();
-        } catch (InterruptedException e) {
-          if (cache.getLogger().fineEnabled()) {
-            cache.getLogger().fine("TestLRUClockHand#clearThread Got an interrupted Exception");
-          }
-          fail("interrupted");
-        } catch (Exception ie) {
-          fail("TestLRUClockHand#clearThread Got an Exception");
-        } finally {
-          region.getDiskRegion().releaseWriteLock();
-        }
-
-      }
-    }
-
-    /**
-     *
-     * putThread
-     *
-     */
-
-    protected static class putThread implements Runnable {
-      LocalRegion region = null;
-
-      putThread(LocalRegion rgn) {
-        super();
-        this.region = rgn;
-
-      }
-
-      public void run() {
-        region.put(new Long(1), "2");
-
-      }
-    }
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DeltaFaultInDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DeltaFaultInDUnitTest.java
index 477e339b00..6cbcef15c7 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DeltaFaultInDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DeltaFaultInDUnitTest.java
@@ -27,7 +27,6 @@
 import org.apache.geode.cache.PartitionAttributes;
 import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache30.CacheTestCase;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.dunit.SerializableRunnable;
@@ -125,7 +124,7 @@ public void run() {
         region.get(new Integer(113));
 
         long entriesEvicted = ((AbstractLRURegionMap) ((PartitionedRegion) region).entries)
-            ._getLruList().stats().getEvictions();
+            .getEvictionList().getStatistics().getEvictions();
         // assertIndexDetailsEquals(1, entriesEvicted);
 
         TestDelta result = region.get(new Integer(0));
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DeltaPropagationDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DeltaPropagationDUnitTest.java
index e934b562bc..6b9000e032 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DeltaPropagationDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DeltaPropagationDUnitTest.java
@@ -65,8 +65,8 @@
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.cache.PartitionedRegionLocalMaxMemoryDUnitTest.TestObject1;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.ha.HARegionQueue;
-import org.apache.geode.internal.cache.lru.EnableLRU;
 import org.apache.geode.internal.cache.tier.sockets.CacheServerTestUtil;
 import org.apache.geode.internal.cache.tier.sockets.ConflationDUnitTest;
 import org.apache.geode.internal.tcp.ConnectionTable;
@@ -788,13 +788,13 @@ private void assertValue(String rName, String key, Object expected) {
   }
 
   private void confirmEviction(Integer port) {
-    final EnableLRU cc = ((VMLRURegionMap) ((LocalRegion) cache.getRegion(
+    final EvictionController cc = ((VMLRURegionMap) ((LocalRegion) cache.getRegion(
         Region.SEPARATOR + CacheServerImpl.generateNameForClientMsgsRegion(port))).entries)
-            ._getCCHelper();
+            .getEvictionController();
 
     WaitCriterion wc = new WaitCriterion() {
       public boolean done() {
-        return cc.getStats().getEvictions() > 0;
+        return cc.getStatistics().getEvictions() > 0;
       }
 
       public String description() {
@@ -991,11 +991,11 @@ private void invalidateDelta() {
   }
 
   private void verifyOverflowOccurred(long evictions, int regionsize) {
-    EnableLRU cc =
-        ((VMLRURegionMap) ((LocalRegion) cache.getRegion(regionName)).entries)._getCCHelper();
-    Assert.assertTrue(cc.getStats().getEvictions() == evictions,
+    EvictionController cc = ((VMLRURegionMap) ((LocalRegion) cache.getRegion(regionName)).entries)
+        .getEvictionController();
+    Assert.assertTrue(cc.getStatistics().getEvictions() == evictions,
         "Number of evictions expected to be " + evictions + " but was "
-            + cc.getStats().getEvictions());
+            + cc.getStatistics().getEvictions());
     int rSize = ((LocalRegion) cache.getRegion(regionName)).getRegionMap().size();
     Assert.assertTrue(rSize == regionsize,
         "Region size expected to be " + regionsize + " but was " + rSize);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegOplogSwtchingAndRollerJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegOplogSwtchingAndRollerJUnitTest.java
index 4e27fa9e06..bed0ca805b 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegOplogSwtchingAndRollerJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegOplogSwtchingAndRollerJUnitTest.java
@@ -27,7 +27,7 @@
 import org.apache.geode.cache.EntryNotFoundException;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.Scope;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
 /**
@@ -617,8 +617,8 @@ public void afterHavingCompacted() {
     // region.close();
   }// end of testEntryDeletedinCurrentOplog
 
-  private LRUStatistics getLRUStats(Region region) {
-    return ((LocalRegion) region).getEvictionController().getLRUHelper().getStats();
+  private EvictionStatistics getLRUStats(Region region) {
+    return ((LocalRegion) region).getEvictionController().getStatistics();
   }
 
   /**
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java
index 7f225c0b22..6cf008e9d9 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java
@@ -44,8 +44,8 @@
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.internal.cache.entries.DiskEntry;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionList;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.internal.cache.persistence.UninterruptibleFileChannel;
 import org.apache.geode.test.dunit.ThreadUtils;
 import org.apache.geode.test.dunit.Wait;
@@ -183,8 +183,7 @@ public void testDiskRegionOverflow() throws Exception {
     assertNotNull(dr);
 
     DiskRegionStats diskStats = dr.getStats();
-    LRUStatistics lruStats =
-        ((LocalRegion) region).getEvictionController().getLRUHelper().getStats();
+    EvictionStatistics lruStats = ((LocalRegion) region).getEvictionController().getStatistics();
     assertNotNull(diskStats);
     assertNotNull(lruStats);
 
@@ -1551,11 +1550,10 @@ public void afterCreate(EntryEvent event) {
     }
 
     assertFalse(this.failureCause, this.exceptionOccurred);
-    NewLRUClockHand lruList = ((VMLRURegionMap) ((LocalRegion) region).entries)._getLruList();
+    EvictionList lruList = ((VMLRURegionMap) ((LocalRegion) region).entries).getEvictionList();
     assertEquals(region.size(), 0);
-    lruList.audit();
     assertNull("The LRU List should have been empty instead it contained a cleared entry",
-        lruList.getLRUEntry());
+        lruList.getEvictableEntry());
   }
 
   /**
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/GemFireCacheImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/GemFireCacheImplTest.java
index fdd9eadb2a..ee9ca16191 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/GemFireCacheImplTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/GemFireCacheImplTest.java
@@ -30,8 +30,8 @@
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.SystemTimer;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
-import org.apache.geode.internal.cache.lru.OffHeapEvictor;
+import org.apache.geode.internal.cache.eviction.HeapEvictor;
+import org.apache.geode.internal.cache.eviction.OffHeapEvictor;
 import org.apache.geode.pdx.internal.TypeRegistry;
 import org.apache.geode.test.fake.Fakes;
 import org.apache.geode.test.junit.categories.UnitTest;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/HAOverflowMemObjectSizerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
index ba4f8c409a..c94d8c3d95 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
@@ -39,7 +39,8 @@
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.AvailablePort;
-import org.apache.geode.internal.cache.lru.EnableLRU;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.MemoryLRUController;
 import org.apache.geode.internal.cache.tier.sockets.ClientUpdateMessageImpl;
 import org.apache.geode.internal.cache.tier.sockets.ConflationDUnitTest;
 import org.apache.geode.test.dunit.Host;
@@ -49,8 +50,8 @@
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 /**
- * Tests the size of clientUpdateMessageImpl with the size calculated by
- * {@link org.apache.geode.internal.cache.lru.MemLRUCapacityController} for HA overFlow
+ * Tests the size of clientUpdateMessageImpl with the size calculated by {@link MemoryLRUController}
+ * for HA overFlow
  *
  * @since GemFire 5.7
  */
@@ -70,7 +71,7 @@
   static String regionName = HAOverflowMemObjectSizerDUnitTest.class.getSimpleName() + "-region";
 
   /* handler for LRU capacity controller */
-  private static EnableLRU cc = null;
+  private static EvictionController cc = null;
 
   VM client = null;
 
@@ -150,7 +151,7 @@ public static Integer createCacheServer(Boolean notification) throws Exception {
      */
     cc = ((VMLRURegionMap) ((LocalRegion) cache.getRegion(
         Region.SEPARATOR + CacheServerImpl.generateNameForClientMsgsRegion(port))).entries)
-            ._getCCHelper();
+            .getEvictionController();
     return new Integer(server1.getPort());
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/LIFOEvictionAlgoEnabledRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/LIFOEvictionAlgoEnabledRegionJUnitTest.java
index 6ba39fd179..bf077261a7 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/LIFOEvictionAlgoEnabledRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/LIFOEvictionAlgoEnabledRegionJUnitTest.java
@@ -16,6 +16,7 @@
 
 import static org.apache.geode.distributed.ConfigurationProperties.*;
 import static org.junit.Assert.*;
+import static org.junit.Assert.assertNotNull;
 
 import java.io.File;
 import java.util.Properties;
@@ -28,8 +29,8 @@
 
 import org.apache.geode.cache.*;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.internal.cache.eviction.EvictionList;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
 /**
@@ -45,7 +46,7 @@
   private static Cache cache = null;
 
   /** Stores LIFO Related Statistics */
-  private static LRUStatistics lifoStats = null;
+  private static EvictionStatistics lifoStats = null;
 
   /** The distributedSystem instance */
   private static DistributedSystem distributedSystem = null;
@@ -54,7 +55,7 @@
 
   private static int capacity = 5;
 
-  private static NewLRUClockHand lifoClockHand = null;
+  private static EvictionList lifoClockHand = null;
 
   @Before
   public void setUp() throws Exception {
@@ -107,15 +108,12 @@ private static void initializeVM() throws Exception {
     RegionAttributes attr = factory.create();
 
     ((GemFireCacheImpl) cache).createRegion(regionName, attr);
-    /*
-     * NewLIFOClockHand extends NewLRUClockHand to hold on to the list reference
-     */
     lifoClockHand =
         ((VMLRURegionMap) ((LocalRegion) cache.getRegion(Region.SEPARATOR + regionName)).entries)
-            ._getLruList();
+            .getEvictionList();
 
     /* storing stats reference */
-    lifoStats = lifoClockHand.stats();
+    lifoStats = lifoClockHand.getStatistics();
 
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
index d59d60ccb6..c1ed11f72b 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
@@ -30,10 +30,21 @@
 
 import org.apache.geode.cache.*;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.internal.cache.lru.EnableLRU;
-import org.apache.geode.internal.cache.lru.LRUClockNode;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
-import org.apache.geode.internal.cache.lru.NewLRUClockHand;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.internal.ByteArrayDataInput;
+import org.apache.geode.internal.InternalStatisticsDisabledException;
+import org.apache.geode.internal.Version;
+import org.apache.geode.internal.cache.DistributedRegion.DiskPosition;
+import org.apache.geode.internal.cache.InitialImageOperation.Entry;
+import org.apache.geode.internal.cache.eviction.EvictableEntry;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionList;
+import org.apache.geode.internal.cache.eviction.EvictionNode;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
+import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
+import org.apache.geode.internal.cache.versions.VersionSource;
+import org.apache.geode.internal.cache.versions.VersionStamp;
+import org.apache.geode.internal.cache.versions.VersionTag;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
 /**
@@ -50,7 +61,7 @@
   private static Cache cache = null;
 
   /** Stores LIFO Related Statistics */
-  private static LRUStatistics lifoStats = null;
+  private static EvictionStatistics lifoStats = null;
 
   /** The distributedSystem instance */
   private static DistributedSystem distributedSystem = null;
@@ -65,7 +76,7 @@
 
   private int deltaSize = 20738;
 
-  private static NewLRUClockHand lifoClockHand = null;
+  private static EvictionList lifoClockHand = null;
 
 
   @Before
@@ -85,8 +96,6 @@ public void tearDown() throws Exception {
 
   /**
    * Method for intializing the VM and create region with LIFO attached
-   *
-   * @throws Exception
    */
   private static void initializeVM() throws Exception {
     Properties props = new Properties();
@@ -118,20 +127,17 @@ private static void initializeVM() throws Exception {
 
     /* setting LIFO MEMORY related eviction attributes */
 
-    factory.setEvictionAttributes(EvictionAttributesImpl
-        .createLIFOMemoryAttributes(maximumMegabytes, EvictionAction.OVERFLOW_TO_DISK));
+    factory.setEvictionAttributes(EvictionAttributes.createLIFOMemoryAttributes(maximumMegabytes,
+        EvictionAction.OVERFLOW_TO_DISK));
     RegionAttributes attr = factory.create();
 
     ((GemFireCacheImpl) cache).createRegion(regionName, attr);
-    /*
-     * NewLIFOClockHand extends NewLRUClockHand to hold on to the list reference
-     */
     lifoClockHand =
         ((VMLRURegionMap) ((LocalRegion) cache.getRegion(Region.SEPARATOR + regionName)).entries)
-            ._getLruList();
+            .getEvictionList();
 
     /* storing stats reference */
-    lifoStats = lifoClockHand.stats();
+    lifoStats = lifoClockHand.getStatistics();
 
   }
 
@@ -295,16 +301,16 @@ public void testLIFOQueue() {
       lifoClockHand.appendEntry(new TestLRUNode(3));
       assertTrue(lifoClockHand.size() == 3);
       // make sure data is removed in LIFO fashion
-      TestLRUNode tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
-      assertTrue("Value = " + tailValue.getValue(), tailValue.getValue() == 3);
+      TestLRUNode tailValue = (TestLRUNode) lifoClockHand.getEvictableEntry();
+      assertTrue("Value = " + tailValue.getValue(), tailValue.value == 3);
       assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 2);
-      tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
-      assertTrue("Value = " + tailValue.getValue(), tailValue.getValue() == 2);
+      tailValue = (TestLRUNode) lifoClockHand.getEvictableEntry();
+      assertTrue("Value = " + tailValue.getValue(), tailValue.value == 2);
       assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 1);
-      tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
-      assertTrue("Value = " + tailValue.getValue(), tailValue.getValue() == 1);
+      tailValue = (TestLRUNode) lifoClockHand.getEvictableEntry();
+      assertTrue("Value = " + tailValue.getValue(), tailValue.value == 1);
       assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 0);
-      tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
+      tailValue = (TestLRUNode) lifoClockHand.getEvictableEntry();
       assertTrue("No Value - null", tailValue == null);
       assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 0);
       // check that entries not available or already evicted are skipped and removed
@@ -316,10 +322,10 @@ public void testLIFOQueue() {
       testlrunode = new TestLRUNode(3);
       testlrunode.setEvicted();
       lifoClockHand.appendEntry(testlrunode);
-      tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
-      assertTrue("Value = " + tailValue.getValue(), tailValue.getValue() == 1);
+      tailValue = (TestLRUNode) lifoClockHand.getEvictableEntry();
+      assertTrue("Value = " + tailValue.getValue(), tailValue.value == 1);
       assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 0);
-      tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
+      tailValue = (TestLRUNode) lifoClockHand.getEvictableEntry();
       assertTrue("No Value - null", tailValue == null);
       assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 0);
       // TODO : need tests for data still part of transaction
@@ -329,82 +335,413 @@ public void testLIFOQueue() {
     }
   }
 
-
   // purpose to create object ,size of byteArraySize
   private Object newDummyObject(long i) {
     byte[] value = new byte[byteArraySize];
     Arrays.fill(value, (byte) i);
     return value;
   }
-}
 
+  // test class for validating LIFO queue
+  static class TestLRUNode implements EvictableEntry {
 
-// test class for validating LIFO queue
-class TestLRUNode implements LRUClockNode {
+    EvictionNode next = null;
+    EvictionNode prev = null;
+    boolean evicted = false;
+    boolean recentlyUsed = false;
+    int value = 0;
 
-  LRUClockNode next = null;
-  LRUClockNode prev = null;
-  boolean evicted = false;
-  boolean recentlyUsed = false;
-  int value = 0;
+    public TestLRUNode(int value) {
+      this.value = value;
+    }
 
-  public TestLRUNode(int value) {
-    this.value = value;
-  }
+    @Override
+    public Token getValueAsToken() {
+      return null;
+    }
 
-  public int getValue() {
-    return value;
-  }
+    @Override
+    public void setValueWithTombstoneCheck(final Object value, final EntryEvent event)
+        throws RegionClearedException {
 
-  public void setNextLRUNode(LRUClockNode next) {
-    this.next = next;
-  }
+    }
 
-  public void setPrevLRUNode(LRUClockNode prev) {
-    this.prev = prev;
-  }
+    @Override
+    public Object getTransformedValue() {
+      return null;
+    }
 
-  public LRUClockNode nextLRUNode() {
-    return next;
-  }
+    @Override
+    public Object getValueInVM(final RegionEntryContext context) {
+      return null;
+    }
 
-  public LRUClockNode prevLRUNode() {
-    return prev;
-  }
+    @Override
+    public Object getValueOnDisk(final InternalRegion region) throws EntryNotFoundException {
+      return null;
+    }
 
-  public int updateEntrySize(EnableLRU ccHelper) {
-    return 0;
-  }
+    @Override
+    public Object getValueOnDiskOrBuffer(final InternalRegion region)
+        throws EntryNotFoundException {
+      return null;
+    }
 
-  public int updateEntrySize(EnableLRU ccHelper, Object value) {
-    return 0;
-  }
+    @Override
+    public boolean initialImagePut(final InternalRegion region, final long lastModified,
+        final Object newValue, final boolean wasRecovered, final boolean acceptedVersionTag)
+        throws RegionClearedException {
+      return false;
+    }
 
-  public int getEntrySize() {
-    return 0;
-  }
+    @Override
+    public boolean initialImageInit(final InternalRegion region, final long lastModified,
+        final Object newValue, final boolean create, final boolean wasRecovered,
+        final boolean acceptedVersionTag) throws RegionClearedException {
+      return false;
+    }
 
-  public boolean testRecentlyUsed() {
-    return recentlyUsed;
-  }
+    @Override
+    public boolean destroy(final InternalRegion region, final EntryEventImpl event,
+        final boolean inTokenMode, final boolean cacheWrite, final Object expectedOldValue,
+        final boolean forceDestroy, final boolean removeRecoveredEntry) throws CacheWriterException,
+        EntryNotFoundException, TimeoutException, RegionClearedException {
+      return false;
+    }
 
-  public void setRecentlyUsed() {
-    recentlyUsed = true;
-  }
+    @Override
+    public boolean getValueWasResultOfSearch() {
+      return false;
+    }
 
-  public void unsetRecentlyUsed() {
-    recentlyUsed = false;
-  }
+    @Override
+    public void setValueResultOfSearch(final boolean value) {
 
-  public void setEvicted() {
-    evicted = true;
-  }
+    }
 
-  public void unsetEvicted() {
-    evicted = false;
-  }
+    @Override
+    public Object getSerializedValueOnDisk(final InternalRegion region) {
+      return null;
+    }
+
+    @Override
+    public Object getValueInVMOrDiskWithoutFaultIn(final InternalRegion region) {
+      return null;
+    }
+
+    @Override
+    public Object getValueOffHeapOrDiskWithoutFaultIn(final InternalRegion region) {
+      return null;
+    }
+
+    @Override
+    public boolean isUpdateInProgress() {
+      return false;
+    }
+
+    @Override
+    public void setUpdateInProgress(final boolean underUpdate) {
+
+    }
+
+    @Override
+    public boolean isCacheListenerInvocationInProgress() {
+      return false;
+    }
+
+    @Override
+    public void setCacheListenerInvocationInProgress(final boolean isListenerInvoked) {
+
+    }
+
+    @Override
+    public boolean isValueNull() {
+      return false;
+    }
+
+    @Override
+    public boolean isInvalid() {
+      return false;
+    }
+
+    @Override
+    public boolean isDestroyed() {
+      return false;
+    }
+
+    @Override
+    public boolean isDestroyedOrRemoved() {
+      return false;
+    }
+
+    @Override
+    public boolean isDestroyedOrRemovedButNotTombstone() {
+      return false;
+    }
+
+    @Override
+    public boolean isInvalidOrRemoved() {
+      return false;
+    }
+
+    @Override
+    public void setValueToNull() {
+
+    }
+
+    @Override
+    public void returnToPool() {
+
+    }
+
+    @Override
+    public void setNext(EvictionNode next) {
+      this.next = next;
+    }
+
+    @Override
+    public void setPrevious(EvictionNode previous) {
+      this.prev = previous;
+    }
+
+    @Override
+    public EvictionNode next() {
+      return next;
+    }
+
+    @Override
+    public EvictionNode previous() {
+      return prev;
+    }
+
+    @Override
+    public int updateEntrySize(EvictionController ccHelper) {
+      return 0;
+    }
+
+    @Override
+    public int updateEntrySize(EvictionController ccHelper, Object value) {
+      return 0;
+    }
+
+    @Override
+    public int getEntrySize() {
+      return 0;
+    }
+
+    @Override
+    public boolean isRecentlyUsed() {
+      return recentlyUsed;
+    }
+
+    @Override
+    public void setRecentlyUsed(final RegionEntryContext context) {
+      recentlyUsed = true;
+      context.incRecentlyUsed();
+    }
+
+    @Override
+    public long getLastModified() {
+      return 0;
+    }
+
+    @Override
+    public boolean hasStats() {
+      return false;
+    }
+
+    @Override
+    public long getLastAccessed() throws InternalStatisticsDisabledException {
+      return 0;
+    }
+
+    @Override
+    public long getHitCount() throws InternalStatisticsDisabledException {
+      return 0;
+    }
+
+    @Override
+    public long getMissCount() throws InternalStatisticsDisabledException {
+      return 0;
+    }
+
+    @Override
+    public void updateStatsForPut(final long lastModifiedTime, final long lastAccessedTime) {
+
+    }
+
+    @Override
+    public VersionStamp getVersionStamp() {
+      return null;
+    }
+
+    @Override
+    public VersionTag generateVersionTag(final VersionSource member, final boolean withDelta,
+        final InternalRegion region, final EntryEventImpl event) {
+      return null;
+    }
+
+    @Override
+    public boolean dispatchListenerEvents(final EntryEventImpl event) throws InterruptedException {
+      return false;
+    }
+
+    @Override
+    public void updateStatsForGet(final boolean hit, final long time) {
+
+    }
 
-  public boolean testEvicted() {
-    return evicted;
+    @Override
+    public void txDidDestroy(final long currentTime) {
+
+    }
+
+    @Override
+    public void resetCounts() throws InternalStatisticsDisabledException {
+
+    }
+
+    @Override
+    public void makeTombstone(final InternalRegion region, final VersionTag version)
+        throws RegionClearedException {
+
+    }
+
+    @Override
+    public void removePhase1(final InternalRegion region, final boolean clear)
+        throws RegionClearedException {
+
+    }
+
+    @Override
+    public void removePhase2() {
+
+    }
+
+    @Override
+    public boolean isRemoved() {
+      return false;
+    }
+
+    @Override
+    public boolean isRemovedPhase2() {
+      return false;
+    }
+
+    @Override
+    public boolean isTombstone() {
+      return false;
+    }
+
+    @Override
+    public boolean fillInValue(final InternalRegion region, final Entry entry,
+        final ByteArrayDataInput in, final DM distributionManager, final Version version) {
+      return false;
+    }
+
+    @Override
+    public boolean isOverflowedToDisk(final InternalRegion region,
+        final DiskPosition diskPosition) {
+      return false;
+    }
+
+    @Override
+    public Object getKey() {
+      return null;
+    }
+
+    @Override
+    public Object getValue(final RegionEntryContext context) {
+      return null;
+    }
+
+    @Override
+    public Object getValueRetain(final RegionEntryContext context) {
+      return null;
+    }
+
+    @Override
+    public void setValue(final RegionEntryContext context, final Object value)
+        throws RegionClearedException {
+
+    }
+
+    @Override
+    public void setValue(final RegionEntryContext context, final Object value,
+        final EntryEventImpl event) throws RegionClearedException {
+
+    }
+
+    @Override
+    public Object getValueRetain(final RegionEntryContext context, final boolean decompress) {
+      return null;
+    }
+
+    @Override
+    public Object getValue() {
+      return null;
+    }
+
+    @Override
+    public void unsetRecentlyUsed() {
+      recentlyUsed = false;
+    }
+
+    @Override
+    public void setEvicted() {
+      evicted = true;
+    }
+
+    @Override
+    public void unsetEvicted() {
+      evicted = false;
+    }
+
+    @Override
+    public boolean isEvicted() {
+      return evicted;
+    }
+
+    @Override
+    public boolean isInUseByTransaction() {
+      return false;
+    }
+
+    @Override
+    public void incRefCount() {
+
+    }
+
+    @Override
+    public void decRefCount(final EvictionList lruList, final InternalRegion region) {
+
+    }
+
+    @Override
+    public void resetRefCount(final EvictionList lruList) {
+
+    }
+
+    @Override
+    public Object prepareValueForCache(final RegionEntryContext context, final Object value,
+        final boolean isEntryUpdate) {
+      return null;
+    }
+
+    @Override
+    public Object prepareValueForCache(final RegionEntryContext context, final Object value,
+        final EntryEventImpl event, final boolean isEntryUpdate) {
+      return null;
+    }
+
+    @Override
+    public Object getKeyForSizing() {
+      return null;
+    }
+
+    @Override
+    public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
+
+    }
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/LRUClearWithDiskRegionOpRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/LRUClearWithDiskRegionOpRegressionTest.java
new file mode 100644
index 0000000000..981fbc8e03
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/LRUClearWithDiskRegionOpRegressionTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.geode.internal.cache;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.cache.AttributesFactory;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.DiskStoreFactory;
+import org.apache.geode.cache.EvictionAction;
+import org.apache.geode.cache.EvictionAttributes;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.internal.cache.eviction.EvictionController;
+import org.apache.geode.internal.cache.eviction.EvictionList;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
+import org.apache.geode.internal.cache.eviction.TestLRUListWithAsyncSorting;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+/**
+ * Regression test to confirm fix for TRAC #37244.
+ *
+ * <p>
+ * TRAC #37244: deadlock between a concurrent LRUList clear and a region operation where diskRegions
+ * are involved
+ *
+ * @see org.apache.geode.internal.cache.eviction.TestLRUListWithAsyncSorting
+ */
+@Category(IntegrationTest.class)
+public class LRUClearWithDiskRegionOpRegressionTest {
+
+  private InternalCache cache;
+  private Region<Integer, Integer> region;
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    String regionName = testName.getMethodName();
+    File dir = temporaryFolder.newFolder(testName.getMethodName());
+
+    cache = (InternalCache) new CacheFactory().set("locators", "").set("mcast-port", "0").create();
+
+    AttributesFactory<Integer, Integer> factory = new AttributesFactory();
+
+    DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+    diskStoreFactory.setDiskDirsAndSizes(new File[] {dir}, new int[] {Integer.MAX_VALUE});
+    diskStoreFactory.setAutoCompact(false);
+
+    DirectoryHolder.SET_DIRECTORY_SIZE_IN_BYTES_FOR_TESTING_PURPOSES = true;
+    try {
+      factory.setDiskStoreName(diskStoreFactory.create(regionName).getName());
+    } finally {
+      DirectoryHolder.SET_DIRECTORY_SIZE_IN_BYTES_FOR_TESTING_PURPOSES = false;
+    }
+
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setDiskSynchronous(true);
+    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+    factory.setEvictionAttributes(
+        EvictionAttributes.createLRUEntryAttributes(1, EvictionAction.OVERFLOW_TO_DISK));
+
+    RegionAttributes<Integer, Integer> regionAttributes = factory.create();
+
+    InternalRegionArguments args = new InternalRegionArguments().setDestroyLockFlag(true)
+        .setRecreateFlag(false).setSnapshotInputStream(null).setImageTarget(null);
+
+    DistributedRegion distributedRegion =
+        new DistributedRegion(regionName, regionAttributes, null, cache, args);
+
+    EvictionController eviction =
+        ((AbstractLRURegionMap) distributedRegion.entries).getEvictionController();
+    assertThat(eviction).isNotNull();
+
+    EvictionStatistics evictionStatistics =
+        eviction.initStats(distributedRegion, cache.getDistributedSystem());
+
+    EvictionList evictionList = new TestLRUListWithAsyncSorting(evictionStatistics, null);
+    ((AbstractLRURegionMap) distributedRegion.entries).setEvictionList(evictionList);
+
+    region = cache.createVMRegion(regionName, regionAttributes,
+        new InternalRegionArguments().setInternalMetaRegion(distributedRegion)
+            .setDestroyLockFlag(true).setSnapshotInputStream(null).setImageTarget(null));
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    cache.close();
+  }
+
+  @Test
+  public void testPutWhileClear() throws Exception {
+    // put two entries into the region
+    for (int i = 0; i < 2; i++) {
+      region.put(i, i);
+    }
+
+    // check for entry value
+    assertThat(region.get(0)).isEqualTo(0);
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/OplogJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/OplogJUnitTest.java
index 3008cda2f9..91fb5be5b5 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/OplogJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/OplogJUnitTest.java
@@ -48,6 +48,7 @@
 import org.apache.geode.cache.util.CacheWriterAdapter;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.cache.Oplog.OPLOG_TYPE;
+import org.apache.geode.internal.cache.entries.AbstractDiskLRURegionEntry;
 import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.test.dunit.ThreadUtils;
 import org.apache.geode.test.junit.categories.FlakyTest;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
index 59a6a9621d..6d3b6b5b37 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
@@ -47,8 +47,8 @@
 import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
-import org.apache.geode.internal.cache.lru.HeapLRUCapacityController;
+import org.apache.geode.internal.cache.eviction.HeapEvictor;
+import org.apache.geode.internal.cache.eviction.HeapLRUController;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.IgnoredException;
@@ -80,7 +80,7 @@ public void testHeapLRUWithOverflowToDisk() {
     final CacheSerializableRunnable create =
         new CacheSerializableRunnable("Create Heap LRU with Overflow to disk partitioned Region") {
           public void run2() {
-            System.setProperty(HeapLRUCapacityController.TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY,
+            System.setProperty(HeapLRUController.TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY,
                 Float.toString(0));
             setEvictionPercentage(heapPercentage);
             final Properties sp = System.getProperties();
@@ -191,7 +191,7 @@ public String description() {
   }
 
   protected void raiseFakeNotification() {
-    ((GemFireCacheImpl) getCache()).getHeapEvictor().testAbortAfterLoopCount = 1;
+    ((GemFireCacheImpl) getCache()).getHeapEvictor().setTestAbortAfterLoopCount(1);
     HeapMemoryMonitor.setTestDisableMemoryUpdates(true);
     System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance", "0");
 
@@ -204,7 +204,7 @@ protected void raiseFakeNotification() {
   }
 
   protected void cleanUpAfterFakeNotification() {
-    ((GemFireCacheImpl) getCache()).getHeapEvictor().testAbortAfterLoopCount = Integer.MAX_VALUE;
+    ((GemFireCacheImpl) getCache()).getHeapEvictor().setTestAbortAfterLoopCount(Integer.MAX_VALUE);
     HeapMemoryMonitor.setTestDisableMemoryUpdates(false);
     System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance");
   }
@@ -226,7 +226,7 @@ public void testHeapLRUWithLocalDestroy() {
     final SerializableRunnable create = new CacheSerializableRunnable(
         "Create Heap LRU with local destroy on a partitioned Region") {
       public void run2() {
-        System.setProperty(HeapLRUCapacityController.TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY,
+        System.setProperty(HeapLRUController.TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY,
             Float.toString(0));
         setEvictionPercentage(heapPercentage);
         final Properties sp = System.getProperties();
@@ -292,7 +292,7 @@ public Object call() throws Exception {
 
                 public boolean done() {
                   // we have a primary
-                  if (((AbstractLRURegionMap) pr.entries)._getLruList().stats()
+                  if (((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics()
                       .getEvictions() == 9) {
                     return true;
                   }
@@ -305,8 +305,8 @@ public String description() {
               };
               Wait.waitForCriterion(wc, 60000, 1000, true);
 
-              entriesEvicted =
-                  ((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions();
+              entriesEvicted = ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics()
+                  .getEvictions();
               return new Long(entriesEvicted);
             } finally {
               cleanUpAfterFakeNotification();
@@ -492,8 +492,8 @@ public Object call() throws Exception {
                 assertTrue(bucketRegion.getAttributes().getEvictionAttributes().getAction()
                     .isLocalDestroy());
               }
-              entriesEvicted =
-                  ((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions();
+              entriesEvicted = ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics()
+                  .getEvictions();
               return new Long(entriesEvicted);
             } finally {
             }
@@ -692,8 +692,8 @@ public Object call() throws Exception {
                 assertTrue(bucketRegion.getAttributes().getEvictionAttributes().getAction()
                     .isLocalDestroy());
               }
-              entriesEvicted =
-                  ((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions();
+              entriesEvicted = ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics()
+                  .getEvictions();
               return new Long(entriesEvicted);
             } finally {
             }
@@ -713,7 +713,7 @@ public Object call() throws Exception {
         RegionAttributes attrs = pr.getAttributes();
         assertNotNull(attrs);
         long entriesEvicted =
-            ((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions();
+            ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics().getEvictions();
         VerifiableCacheListener verifyMe = null;
         for (CacheListener listener : attrs.getCacheListeners()) {
           if (listener instanceof VerifiableCacheListener) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionLocalMaxMemoryDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionLocalMaxMemoryDUnitTest.java
index b12d61d021..3d0631edc4 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionLocalMaxMemoryDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionLocalMaxMemoryDUnitTest.java
@@ -34,7 +34,7 @@
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.util.ObjectSizer;
 import org.apache.geode.cache30.CacheSerializableRunnable;
-import org.apache.geode.internal.cache.lru.Sizeable;
+import org.apache.geode.internal.size.Sizeable;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.dunit.SerializableRunnable;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
index 469d22d89f..9b269aa762 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
@@ -23,7 +23,7 @@
 import org.apache.geode.cache.Region;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
 import org.apache.geode.internal.cache.control.OffHeapMemoryMonitor;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
+import org.apache.geode.internal.cache.eviction.HeapEvictor;
 import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.junit.categories.DistributedTest;
@@ -76,7 +76,7 @@ protected HeapEvictor getEvictor(Region region) {
 
   @Override
   protected void raiseFakeNotification() {
-    ((GemFireCacheImpl) getCache()).getOffHeapEvictor().testAbortAfterLoopCount = 1;
+    ((GemFireCacheImpl) getCache()).getOffHeapEvictor().setTestAbortAfterLoopCount(1);
 
     setEvictionPercentage(85);
     OffHeapMemoryMonitor ohmm =
@@ -88,6 +88,7 @@ protected void raiseFakeNotification() {
 
   @Override
   protected void cleanUpAfterFakeNotification() {
-    ((GemFireCacheImpl) getCache()).getOffHeapEvictor().testAbortAfterLoopCount = Integer.MAX_VALUE;
+    ((GemFireCacheImpl) getCache()).getOffHeapEvictor()
+        .setTestAbortAfterLoopCount(Integer.MAX_VALUE);
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/SizingFlagDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/SizingFlagDUnitTest.java
index 4a313ec986..6f54a7f4f9 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/SizingFlagDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/SizingFlagDUnitTest.java
@@ -44,9 +44,7 @@
 import org.apache.geode.cache.partition.PartitionRegionHelper;
 import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.cache.util.ObjectSizer;
-import org.apache.geode.cache30.CacheTestCase;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.test.dunit.Assert;
@@ -767,13 +765,13 @@ protected int getObjectSizerInvocations(LocalRegion region) {
   }
 
   private long getSizeFromEvictionStats(LocalRegion region) {
-    long result = region.getEvictionController().getLRUHelper().getStats().getCounter();
+    long result = region.getEvictionController().getStatistics().getCounter();
     // region.getCache().getLogger().info("DEBUG evictionSize=" + result);
     return result;
   }
 
   private long getEvictions(LocalRegion region) {
-    return region.getEvictionController().getLRUHelper().getStats().getEvictions();
+    return region.getEvictionController().getStatistics().getEvictions();
   }
 
   private void setDeltaRecalculatesSize(VM vm, final boolean shouldSizeChange) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegOverflowAsyncJUnitPerformanceTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegOverflowAsyncJUnitPerformanceTest.java
index 2fefaa35df..c65ee4a98f 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegOverflowAsyncJUnitPerformanceTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegOverflowAsyncJUnitPerformanceTest.java
@@ -27,7 +27,7 @@
 import org.apache.geode.internal.cache.DiskRegionProperties;
 import org.apache.geode.internal.cache.DiskRegionTestingBase;
 import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
 /**
@@ -73,7 +73,7 @@ protected final void postTearDown() throws Exception {
   @Test
   public void testPopulatefor5Kbwrites() {
     // RegionAttributes ra = region.getAttributes();
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     // Put in larger stuff until we start evicting
     int total;
     for (total = 0; lruStats.getEvictions() <= 0; total++) {
@@ -133,7 +133,7 @@ public void testPopulatefor5Kbwrites() {
     System.out.println("Perf Stats of get which is fauting in :" + statsGet);
   }
 
-  private LRUStatistics getLRUStats(Region region) {
-    return ((LocalRegion) region).getEvictionController().getLRUHelper().getStats();
+  private EvictionStatistics getLRUStats(Region region) {
+    return ((LocalRegion) region).getEvictionController().getStatistics();
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegOverflowSyncJUnitPerformanceTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegOverflowSyncJUnitPerformanceTest.java
index 2db54cf27f..3dbcf47041 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegOverflowSyncJUnitPerformanceTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegOverflowSyncJUnitPerformanceTest.java
@@ -27,7 +27,7 @@
 import org.apache.geode.internal.cache.DiskRegionProperties;
 import org.apache.geode.internal.cache.DiskRegionTestingBase;
 import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
 /**
@@ -75,7 +75,7 @@ protected final void postTearDown() throws Exception {
   public void testPopulatefor5Kbwrites() throws Exception {
     // RegionAttributes ra = region.getAttributes();
 
-    LRUStatistics lruStats = getLRUStats(region);
+    EvictionStatistics lruStats = getLRUStats(region);
     // Put in larger stuff until we start evicting
     int total;
     for (total = 0; lruStats.getEvictions() <= 0; total++) {
@@ -136,8 +136,8 @@ public void testPopulatefor5Kbwrites() throws Exception {
     System.out.println("Perf Stats of get which is fauting in :" + statsGet);
   }
 
-  private LRUStatistics getLRUStats(Region region) {
-    return ((LocalRegion) region).getEvictionController().getLRUHelper().getStats();
+  private EvictionStatistics getLRUStats(Region region) {
+    return ((LocalRegion) region).getEvictionController().getStatistics();
   }
 
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowAsyncRollingOpLogJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowAsyncRollingOpLogJUnitTest.java
index aa8b46ca6f..0e00f6a1b4 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowAsyncRollingOpLogJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowAsyncRollingOpLogJUnitTest.java
@@ -27,7 +27,7 @@
 import org.apache.geode.internal.cache.DiskRegionProperties;
 import org.apache.geode.internal.cache.DiskRegionTestingBase;
 import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.test.dunit.Wait;
 import org.apache.geode.test.dunit.WaitCriterion;
 import org.apache.geode.test.junit.categories.IntegrationTest;
@@ -194,8 +194,8 @@ public String description() {
    * @param region1
    * @return
    */
-  protected LRUStatistics getLRUStats(Region region1) {
-    return ((LocalRegion) region1).getEvictionController().getLRUHelper().getStats();
+  protected EvictionStatistics getLRUStats(Region region1) {
+    return ((LocalRegion) region1).getEvictionController().getStatistics();
 
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowSyncRollingOpLogJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowSyncRollingOpLogJUnitTest.java
index 14fcc0d492..169bda5a17 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowSyncRollingOpLogJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowSyncRollingOpLogJUnitTest.java
@@ -18,15 +18,13 @@
 
 import java.util.*;
 
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.*;
 import org.apache.geode.cache.*;
 import org.apache.geode.internal.cache.*;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.test.dunit.Wait;
 import org.apache.geode.test.dunit.WaitCriterion;
 import org.apache.geode.test.junit.categories.IntegrationTest;
@@ -66,7 +64,7 @@ public void testGetPerfRollingOpog() {
   public void populateFirst0k_10Kbwrites() {
     // RegionAttributes ra = region.getAttributes();
 
-    // LRUStatistics lruStats = getLRUStats(region);
+    // EvictionStatistics lruStats = getLRUStats(region);
 
     // put first 0-9999 entries
     // final String key = "K";
@@ -104,7 +102,7 @@ public void populateSecond10kto20kwrites() {
     afterHavingCompacted = false;
     // RegionAttributes ra = region.getAttributes();
 
-    // LRUStatistics lruStats = getLRUStats(region);
+    // EvictionStatistics lruStats = getLRUStats(region);
 
     setCacheObserverCallBack();
 
@@ -184,8 +182,8 @@ public String description() {
    * @param region1
    * @return
    */
-  protected LRUStatistics getLRUStats(Region region1) {
-    return ((LocalRegion) region1).getEvictionController().getLRUHelper().getStats();
+  protected EvictionStatistics getLRUStats(Region region1) {
+    return ((LocalRegion) region1).getEvictionController().getStatistics();
 
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/AbstractEvictionListTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/AbstractEvictionListTest.java
new file mode 100644
index 0000000000..d0e072ad29
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/AbstractEvictionListTest.java
@@ -0,0 +1,275 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.versions.RegionVersionVector;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class AbstractEvictionListTest {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private BucketRegion bucketRegion;
+  private InternalEvictionStatistics stats;
+
+  @Before
+  public void setup() {
+    bucketRegion = mock(BucketRegion.class);
+    stats = mock(InternalEvictionStatistics.class);
+  }
+
+  @Test
+  public void cannotInstantiateWithoutStats() {
+    thrown.expect(IllegalArgumentException.class);
+    new TestEvictionList(null, bucketRegion);
+  }
+
+  @Test
+  public void sizeIsZeroByDefault() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    assertThat(evictionList.size()).isZero();
+  }
+
+  @Test
+  public void sizeIncreasesWithAppendEntry() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+
+    evictionList.appendEntry(new LinkableEvictionNode());
+    assertThat(evictionList.size()).isEqualTo(1);
+
+    evictionList.appendEntry(new LinkableEvictionNode());
+    assertThat(evictionList.size()).isEqualTo(2);
+  }
+
+  @Test
+  public void sizeDecreasedWhenDecremented() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+
+    evictionList.appendEntry(new LinkableEvictionNode());
+    evictionList.decrementSize();
+    assertThat(evictionList.size()).isZero();
+  }
+
+  @Test
+  public void getStatisticsReturnsRightObject() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    assertThat(evictionList.getStatistics()).isSameAs(stats);
+  }
+
+  @Test
+  public void closeStats() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    evictionList.closeStats();
+    verify(stats).close();
+  }
+
+  @Test
+  public void clearWithVersionVectorDoesNotChangeStats() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+
+    evictionList.appendEntry(new LinkableEvictionNode());
+    assertThat(evictionList.size()).isEqualTo(1);
+    evictionList.clear(mock(RegionVersionVector.class));
+    assertThat(evictionList.size()).isEqualTo(1);
+  }
+
+  @Test
+  public void clearWithoutBucketRegionResetsStats() throws Exception {
+    TestEvictionList noBucketRegionEvictionList = new TestEvictionList(stats, null);
+
+    noBucketRegionEvictionList.appendEntry(new LinkableEvictionNode());
+    assertThat(noBucketRegionEvictionList.size()).isEqualTo(1);
+    noBucketRegionEvictionList.clear(null);
+    verify(stats).resetCounter();
+    assertThat(noBucketRegionEvictionList.size()).isZero();
+  }
+
+  @Test
+  public void clearWithBucketRegionResetsBucketStats() throws Exception {
+    long bucketSize = 10L;
+    when(bucketRegion.getCounter()).thenReturn(bucketSize);
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+
+    evictionList.clear(null);
+    verify(bucketRegion).resetCounter();
+    verify(stats).decrementCounter(bucketSize);
+    assertThat(evictionList.size()).isZero();
+  }
+
+  @Test
+  public void setBucketRegionWithWrongTypeDoesNothing() throws Exception {
+    TestEvictionList noBucketRegionEvictionList = new TestEvictionList(stats, null);
+
+    noBucketRegionEvictionList.appendEntry(new LinkableEvictionNode());
+    Region notABucketRegion = mock(Region.class);
+    noBucketRegionEvictionList.setBucketRegion(notABucketRegion);
+    noBucketRegionEvictionList.clear(null);
+    verifyZeroInteractions(notABucketRegion);
+  }
+
+  @Test
+  public void setBucketRegionWithBucketRegionTest() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, null);
+    evictionList.setBucketRegion(bucketRegion);
+
+    evictionList.clear(null);
+    verify(bucketRegion).resetCounter();
+  }
+
+  @Test
+  public void appendEntryAlreadyInListDoesNothing() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    EvictionNode node = mock(EvictionNode.class);
+    when(node.next()).thenReturn(mock(EvictionNode.class));
+
+    evictionList.appendEntry(node);
+    verify(node, never()).unsetRecentlyUsed();
+  }
+
+  @Test
+  public void appendingNewEntryAddsItToList() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    EvictionNode node = mock(EvictionNode.class);
+    evictionList.appendEntry(node);
+
+    verify(node).unsetRecentlyUsed();
+    verify(node).setNext(evictionList.tail);
+    verify(node).setPrevious(evictionList.head);
+    assertThat(evictionList.tail.previous()).isSameAs(node);
+    assertThat(evictionList.head.next()).isSameAs(node);
+    assertThat(evictionList.size()).isEqualTo(1);
+  }
+
+  @Test
+  public void unlinkEntryNotInListTest() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    EvictionNode node = mock(EvictionNode.class);
+
+    evictionList.destroyEntry(node);
+    assertThat(evictionList.size()).isEqualTo(0);
+  }
+
+  @Test
+  public void unlinkEntryInListTest() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    EvictionNode node = mock(EvictionNode.class);
+    when(node.next()).thenReturn(evictionList.tail);
+    when(node.previous()).thenReturn(evictionList.head);
+
+    evictionList.appendEntry(mock(EvictionNode.class));
+    assertThat(evictionList.size()).isEqualTo(1);
+
+    evictionList.destroyEntry(node);
+    assertThat(evictionList.size()).isEqualTo(0);
+    verify(stats).incDestroys();
+  }
+
+  @Test
+  public void unlinkHeadOnEmptyListReturnsNull() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    assertThat(evictionList.unlinkHeadEntry()).isNull();
+  }
+
+  @Test
+  public void unlinkTailOnEmptyListReturnsNull() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    assertThat(evictionList.unlinkTailEntry()).isNull();
+  }
+
+  @Test
+  public void unlinkHeadInListTest() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    EvictionNode node = mock(EvictionNode.class);
+    when(node.next()).thenReturn(null, evictionList.tail);
+    when(node.previous()).thenReturn(evictionList.head);
+    evictionList.appendEntry(node);
+
+    assertThat(evictionList.unlinkHeadEntry()).isSameAs(node);
+    assertThat(evictionList.size()).isEqualTo(0);
+  }
+
+  @Test
+  public void unlinkTailInListTest() throws Exception {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    EvictionNode node = mock(EvictionNode.class);
+    when(node.next()).thenReturn(null, evictionList.tail);
+    when(node.previous()).thenReturn(evictionList.head);
+    evictionList.appendEntry(node);
+
+    assertThat(evictionList.unlinkTailEntry()).isSameAs(node);
+    assertThat(evictionList.size()).isEqualTo(0);
+  }
+
+  @Test
+  public void nodeUsedByTransactionIsNotEvictable() {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    EvictionNode node = mock(EvictionNode.class);
+    when(node.isInUseByTransaction()).thenReturn(true);
+
+    assertThat(evictionList.isEvictable(node)).isFalse();
+  }
+
+  @Test
+  public void evictedNodeIsNotEvictable() {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    EvictionNode node = mock(EvictionNode.class);
+    when(node.isEvicted()).thenReturn(true);
+
+    assertThat(evictionList.isEvictable(node)).isFalse();
+  }
+
+  @Test
+  public void defaultNodeIsEvictable() {
+    TestEvictionList evictionList = new TestEvictionList(stats, bucketRegion);
+    EvictionNode node = mock(EvictionNode.class);
+
+    assertThat(evictionList.isEvictable(node)).isTrue();
+  }
+
+  private static class TestEvictionList extends AbstractEvictionList {
+
+    TestEvictionList(InternalEvictionStatistics stats, BucketRegion region) {
+      super(stats, region);
+    }
+
+    @Override
+    public EvictableEntry getEvictableEntry() {
+      return null;
+    }
+
+    @Override
+    public void incrementRecentlyUsed() {
+
+    }
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionDUnitTest.java
similarity index 77%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/EvictionDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionDUnitTest.java
index cdac80c046..8379a3571e 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionDUnitTest.java
@@ -12,26 +12,29 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.eviction;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
-import java.util.ArrayList;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 
-import org.junit.Ignore;
+import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.EvictionAlgorithm;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
-import org.apache.geode.internal.cache.lru.MemLRUCapacityController;
+import org.apache.geode.cache.EvictionAttributes;
+import org.apache.geode.internal.cache.AbstractLRURegionMap;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
-@Ignore("TODO: Test was disabled by renaming to DisabledTest")
 public class EvictionDUnitTest extends EvictionTestBase {
 
   @Test
@@ -71,7 +74,7 @@ public void testCentralizedEvictionnForDistributedRegionWithDummyEvent() {
    * has action - Overflow To Disk.
    *
    * Test Case verifies:If naturally Eviction up and eviction Down events are raised. Centralized
-   * and Inline eviction are happening.All this verificatio is done thorugh logs. It also verifies
+   * and Inline eviction are happening. All this verification is done thorugh logs. It also verifies
    * that during eviction, if one node goes down and then comes up again causing GII to take place,
    * the system doesnot throw an OOME.
    */
@@ -91,25 +94,25 @@ public void testEvictionWithNodeDown() {
   public void testEntryLruEvictions() {
     int extraEntries = 1;
     createCache();
-    maxEnteries = 3;
-    createPartitionedRegion(true, EvictionAlgorithm.LRU_ENTRY, "PR1", 4, 1, 1000, maxEnteries);
+    maxEntries = 3;
+    createPartitionedRegion(true, EvictionAlgorithm.LRU_ENTRY, "PR1", 4, 1, 1000, maxEntries);
 
     final PartitionedRegion pr = (PartitionedRegion) cache.getRegion("PR1");
     LogWriterUtils.getLogWriter().info("PR- " + pr.getEvictionAttributes().getMaximum());
 
-    for (int counter = 1; counter <= maxEnteries + extraEntries; counter++) {
+    for (int counter = 1; counter <= maxEntries + extraEntries; counter++) {
       pr.put(new Integer(counter), new byte[1 * 1024 * 1024]);
     }
 
-    assertEquals(extraEntries,
-        ((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions());
+    Assert.assertEquals(extraEntries,
+        ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics().getEvictions());
   }
 
   @Test
   public void testEntryLru() {
     createCache();
-    maxEnteries = 12;
-    createPartitionedRegion(true, EvictionAlgorithm.LRU_ENTRY, "PR1", 4, 1, 1000, maxEnteries);
+    maxEntries = 12;
+    createPartitionedRegion(true, EvictionAlgorithm.LRU_ENTRY, "PR1", 4, 1, 1000, maxEntries);
 
     final PartitionedRegion pr = (PartitionedRegion) cache.getRegion("PR1");
     LogWriterUtils.getLogWriter().info("PR- " + pr.getEvictionAttributes().getMaximum());
@@ -124,25 +127,26 @@ public void testEntryLru() {
     for (int i = 0; i < 2; i++) {
       pr.put(new Integer(i + pr.getPartitionAttributes().getTotalNumBuckets()) * 3, "value1");
     }
-    assertEquals(0, ((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions());
+    assertEquals(0,
+        ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics().getEvictions());
   }
 
   @Test
   public void testCheckEntryLruEvictionsIn1DataStore() {
     int extraEntries = 10;
     createCache();
-    maxEnteries = 20;
-    createPartitionedRegion(true, EvictionAlgorithm.LRU_ENTRY, "PR1", 5, 1, 1000, maxEnteries);
+    maxEntries = 20;
+    createPartitionedRegion(true, EvictionAlgorithm.LRU_ENTRY, "PR1", 5, 1, 1000, maxEntries);
 
     final PartitionedRegion pr = (PartitionedRegion) cache.getRegion("PR1");
     LogWriterUtils.getLogWriter().info("PR- " + pr.getEvictionAttributes().getMaximum());
 
-    for (int counter = 1; counter <= maxEnteries + extraEntries; counter++) {
+    for (int counter = 1; counter <= maxEntries + extraEntries; counter++) {
       pr.put(new Integer(counter), new byte[1 * 1024 * 1024]);
     }
 
     assertEquals(extraEntries,
-        ((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions());
+        ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics().getEvictions());
 
     for (final Iterator i = pr.getDataStore().getAllLocalBuckets().iterator(); i.hasNext();) {
       final Map.Entry entry = (Map.Entry) i.next();
@@ -158,8 +162,8 @@ public void testCheckEntryLruEvictionsIn1DataStore() {
 
   @Test
   public void testCheckEntryLruEvictionsIn2DataStore() {
-    maxEnteries = 20;
-    prepareScenario1(EvictionAlgorithm.LRU_ENTRY, maxEnteries);
+    maxEntries = 20;
+    prepareScenario1(EvictionAlgorithm.LRU_ENTRY, maxEntries);
     putData("PR1", 60, 1);
     validateNoOfEvictions("PR1", 20);
   }
@@ -173,22 +177,26 @@ public void testMemLruForPRAndDR() {
     DistributedRegion dr = (DistributedRegion) cache.getRegion("DR1");
 
     assertEquals(pr.getLocalMaxMemory(), pr.getEvictionAttributes().getMaximum());
-    assertEquals(MemLRUCapacityController.DEFAULT_MAXIMUM_MEGABYTES,
+    assertEquals(EvictionAttributes.DEFAULT_MEMORY_MAXIMUM,
         dr.getEvictionAttributes().getMaximum());
 
     for (int i = 0; i < 41; i++) {
       pr.put(new Integer(i), new byte[1 * 1024 * 1024]);
     }
 
-    assertTrue(1 <= ((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions());
-    assertTrue(((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions() <= 2);
+    assertTrue(
+        1 <= ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics().getEvictions());
+    assertTrue(
+        ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics().getEvictions() <= 2);
 
     for (int i = 0; i < 11; i++) {
       dr.put(new Integer(i), new byte[1 * 1024 * 1024]);
     }
 
-    assertTrue(1 <= ((AbstractLRURegionMap) dr.entries)._getLruList().stats().getEvictions());
-    assertTrue(((AbstractLRURegionMap) dr.entries)._getLruList().stats().getEvictions() <= 2);
+    assertTrue(
+        1 <= ((AbstractLRURegionMap) dr.entries).getEvictionList().getStatistics().getEvictions());
+    assertTrue(
+        ((AbstractLRURegionMap) dr.entries).getEvictionList().getStatistics().getEvictions() <= 2);
   }
 
   @Test
@@ -199,7 +207,7 @@ public void testEachTaskSize() {
     createPartitionedRegion(true, EvictionAlgorithm.LRU_HEAP, "PR3", 15, 1, 1000, 40);
     createDistRegion();
 
-    ArrayList<Integer> taskSetSizes = getTestTaskSetSizes();
+    List<Integer> taskSetSizes = getTestTaskSetSizes();
     if (taskSetSizes != null) {
       for (Integer size : taskSetSizes) {
         assertEquals(8, size.intValue());
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionObjectSizerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionObjectSizerDUnitTest.java
similarity index 78%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/EvictionObjectSizerDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionObjectSizerDUnitTest.java
index dd054f408c..655a7fbacc 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionObjectSizerDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionObjectSizerDUnitTest.java
@@ -12,15 +12,17 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.eviction;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 import java.io.File;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Properties;
 
+import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -36,39 +38,45 @@
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.SharedLibrary;
+import org.apache.geode.internal.cache.AbstractLRURegionMap;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.CachedDeserializableFactory;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.TestNonSizerObject;
+import org.apache.geode.internal.cache.TestObjectSizerImpl;
 import org.apache.geode.internal.cache.entries.AbstractLRURegionEntry;
-import org.apache.geode.internal.cache.lru.HeapLRUCapacityController;
-import org.apache.geode.internal.cache.lru.Sizeable;
+import org.apache.geode.internal.cache.eviction.HeapLRUController;
 import org.apache.geode.internal.size.ReflectionSingleObjectSizer;
-import org.apache.geode.test.dunit.Assert;
+import org.apache.geode.internal.size.Sizeable;
 import org.apache.geode.test.dunit.LogWriterUtils;
+import org.apache.geode.test.dunit.cache.CacheTestCase;
 import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
-public class EvictionObjectSizerDUnitTest extends JUnit4CacheTestCase {
+@SuppressWarnings("serial")
+public class EvictionObjectSizerDUnitTest extends CacheTestCase {
 
-  protected static Cache cache = null;
+  private static final int maxEnteries = 20;
+  private static final int maxSizeInMb = 20;
 
-  static int maxEnteries = 20;
+  private static Cache cache;
+  private static Region region;
 
-  static int maxSizeInMb = 20;
-
-  protected static Region region = null;
-
-  private static final long serialVersionUID = 1L;
-
-  // Removed CONCURRENT_OVERHEAD since it is included in the regionEntrySize.
-
-  public EvictionObjectSizerDUnitTest() {
-    super();
+  @After
+  public void tearDown() throws Exception {
+    if (cache != null) {
+      cache.close();
+    }
+    cache = null;
+    region = null;
   }
 
   /**
    * Without object sizer
    */
   @Test
-  public void testWithoutObjectSizerForHeapLRU() {
+  public void testWithoutObjectSizerForHeapLRU() throws Exception {
     prepareScenario(EvictionAlgorithm.LRU_HEAP, null);
 
     // Size of overhead=
@@ -78,17 +86,18 @@ public void testWithoutObjectSizerForHeapLRU() {
     // to nearest word)
     // = 1048592
     // Total Size of each entry should be= 1048592
+
     putData("PR1", 2, 1);
     int keySize = 0;
     int valueSize =
         SharedLibrary.getObjectHeaderSize() + 4 /* array length */ + (1024 * 1024) /* bytes */;
     valueSize = (int) ReflectionSingleObjectSizer.roundUpSize(valueSize);
     int entrySize = keySize + valueSize
-        + ((HeapLRUCapacityController) ((PartitionedRegion) region).getEvictionController())
+        + ((HeapLRUController) ((PartitionedRegion) region).getEvictionController())
             .getPerEntryOverhead();
     verifySize("PR1", 2, entrySize);
-    assertEquals(2 * entrySize, ((PartitionedRegion) region).getEvictionController().getLRUHelper()
-        .getStats().getCounter());
+    assertEquals(2 * entrySize,
+        ((PartitionedRegion) region).getEvictionController().getStatistics().getCounter());
   }
 
   /**
@@ -96,21 +105,23 @@ public void testWithoutObjectSizerForHeapLRU() {
    *
    */
   @Test
-  public void testObjectSizerForHeapLRU_StandardObjects() {
+  public void testObjectSizerForHeapLRU_StandardObjects() throws Exception {
     prepareScenario(EvictionAlgorithm.LRU_HEAP, new TestObjectSizerImpl());
 
     // Size of overhead= 49
     // Size of Integer key= 0(inlined)
     // Size of Byte Array(1 MB) + overhead (16 bytes)= 1048592 + 16
     // Total Size of each entry should be= 1048592
+
     putData("PR1", 2, 1);
+
     {
       int keySize = 0;
       int valueSize =
           SharedLibrary.getObjectHeaderSize() + 4 /* array length */ + (1024 * 1024) /* bytes */;
       valueSize = (int) ReflectionSingleObjectSizer.roundUpSize(valueSize);
       int entrySize = keySize + valueSize
-          + ((HeapLRUCapacityController) ((PartitionedRegion) region).getEvictionController())
+          + ((HeapLRUController) ((PartitionedRegion) region).getEvictionController())
               .getPerEntryOverhead();
       verifySize("PR1", 2, entrySize);
     }
@@ -127,7 +138,7 @@ public void testObjectSizerForHeapLRU_StandardObjects() {
           + (1024 * 1024 * 2) /* bytes */;
       valueSize = (int) ReflectionSingleObjectSizer.roundUpSize(valueSize);
       int entrySize = keySize + valueSize
-          + ((HeapLRUCapacityController) ((PartitionedRegion) region).getEvictionController())
+          + ((HeapLRUController) ((PartitionedRegion) region).getEvictionController())
               .getPerEntryOverhead();
       verifySize("PR1", 2, entrySize);
     }
@@ -138,7 +149,7 @@ public void testObjectSizerForHeapLRU_StandardObjects() {
    * TestNonSizerObject
    */
   @Test
-  public void testObjectSizerForHeapLRU_CustomizedNonSizerObject() {
+  public void testObjectSizerForHeapLRU_CustomizedNonSizerObject() throws Exception {
     prepareScenario(EvictionAlgorithm.LRU_HEAP, new TestObjectSizerImpl());
 
     // Size of overhead= 49
@@ -151,10 +162,8 @@ public void testObjectSizerForHeapLRU_CustomizedNonSizerObject() {
       int valueSize = SharedLibrary.getObjectHeaderSize() + 4 /* array length */ + 0 /* bytes */;
       valueSize = (int) ReflectionSingleObjectSizer.roundUpSize(valueSize);
       int entrySize = keySize + valueSize
-          + ((HeapLRUCapacityController) ((PartitionedRegion) region).getEvictionController())
+          + ((HeapLRUController) ((PartitionedRegion) region).getEvictionController())
               .getPerEntryOverhead();
-      LogWriterUtils.getLogWriter()
-          .info("testObjectSizerForHeapLRU_CustomizedNonSizerObject expected= " + entrySize);
       assertEquals(entrySize, getSizeOfCustomizedData(1));
     }
 
@@ -168,10 +177,8 @@ public void testObjectSizerForHeapLRU_CustomizedNonSizerObject() {
       int valueSize = SharedLibrary.getObjectHeaderSize() + 4 /* array length */ + 4 /* bytes */;
       valueSize = (int) ReflectionSingleObjectSizer.roundUpSize(valueSize);
       int entrySize = keySize + valueSize
-          + ((HeapLRUCapacityController) ((PartitionedRegion) region).getEvictionController())
+          + ((HeapLRUController) ((PartitionedRegion) region).getEvictionController())
               .getPerEntryOverhead();
-      LogWriterUtils.getLogWriter()
-          .info("testObjectSizerForHeapLRU_CustomizedNonSizerObject expected= " + entrySize);
       assertEquals(entrySize, getSizeOfCustomizedData(2));
     }
   }
@@ -181,7 +188,7 @@ public void testObjectSizerForHeapLRU_CustomizedNonSizerObject() {
    * TestObjectSizerImpl
    */
   @Test
-  public void testObjectSizerForHeapLRU_CustomizedSizerObject() {
+  public void testObjectSizerForHeapLRU_CustomizedSizerObject() throws Exception {
     prepareScenario(EvictionAlgorithm.LRU_HEAP, new TestObjectSizerImpl());
 
     // Size of overhead= 49
@@ -191,20 +198,18 @@ public void testObjectSizerForHeapLRU_CustomizedSizerObject() {
     // Total Size of entry should be= 71
     putCustomizedData(1, new TestObjectSizerImpl());
     int expected = (0 + 156 + (Sizeable.PER_OBJECT_OVERHEAD * 2)
-        + ((HeapLRUCapacityController) ((PartitionedRegion) region).getEvictionController())
+        + ((HeapLRUController) ((PartitionedRegion) region).getEvictionController())
             .getPerEntryOverhead());
-    LogWriterUtils.getLogWriter()
-        .info("testObjectSizerForHeapLRU_CustomizedSizerObject expected= " + expected);
     assertEquals(expected, getSizeOfCustomizedData(1));
-    assertEquals(expected, ((PartitionedRegion) region).getEvictionController().getLRUHelper()
-        .getStats().getCounter());
+    assertEquals(expected,
+        ((PartitionedRegion) region).getEvictionController().getStatistics().getCounter());
   }
 
   /**
    * With object sizer for customized key and value objects.
    */
   @Test
-  public void testObjectSizerForHeapLRU_CustomizedSizerObjects() {
+  public void testObjectSizerForHeapLRU_CustomizedSizerObjects() throws Exception {
     prepareScenario(EvictionAlgorithm.LRU_HEAP, new TestObjectSizerImpl());
 
     // Size of overhead= 49
@@ -214,38 +219,29 @@ public void testObjectSizerForHeapLRU_CustomizedSizerObjects() {
     // Total Size of entry should be= 72
     putCustomizedObjects(new TestNonSizerObject("1"), new TestObjectSizerImpl());
     int expected = (1 + 156 + (Sizeable.PER_OBJECT_OVERHEAD * 2)
-        + ((HeapLRUCapacityController) ((PartitionedRegion) region).getEvictionController())
+        + ((HeapLRUController) ((PartitionedRegion) region).getEvictionController())
             .getPerEntryOverhead());
-    LogWriterUtils.getLogWriter()
-        .info("testObjectSizerForHeapLRU_CustomizedSizerObjects expected= " + expected);
     assertEquals(expected, getSizeOfCustomizedObject(new TestNonSizerObject("1")));
-    assertEquals(expected, ((PartitionedRegion) region).getEvictionController().getLRUHelper()
-        .getStats().getCounter());
+    assertEquals(expected,
+        ((PartitionedRegion) region).getEvictionController().getStatistics().getCounter());
   }
 
-  public void prepareScenario(EvictionAlgorithm evictionAlgorithm, ObjectSizer sizer) {
-    createCache();
+  private void prepareScenario(EvictionAlgorithm evictionAlgorithm, ObjectSizer sizer) {
+    createMyCache();
     createPartitionedRegion(true, evictionAlgorithm, "PR1", 1, 1, 10000, sizer);
   }
 
-  public void createCache() {
-    try {
-      Properties props = new Properties();
-      DistributedSystem ds = getSystem(props);
-      assertNotNull(ds);
-      ds.disconnect();
-      ds = getSystem(props);
-      cache = CacheFactory.create(ds);
-      assertNotNull(cache);
-      LogWriterUtils.getLogWriter().info("cache= " + cache);
-      LogWriterUtils.getLogWriter().info("cache closed= " + cache.isClosed());
-      cache.getResourceManager().setEvictionHeapPercentage(50);
-    } catch (Exception e) {
-      Assert.fail("Failed while creating the cache", e);
-    }
+  private void createMyCache() {
+    Properties props = new Properties();
+    DistributedSystem ds = getSystem(props);
+    assertNotNull(ds);
+    ds.disconnect();
+    ds = getSystem(props);
+    cache = CacheFactory.create(ds);
+    cache.getResourceManager().setEvictionHeapPercentage(50);
   }
 
-  public static void createPartitionedRegion(boolean setEvictionOn,
+  private static void createPartitionedRegion(boolean setEvictionOn,
       EvictionAlgorithm evictionAlgorithm, String regionName, int totalNoOfBuckets,
       int evictionAction, int evictorInterval, ObjectSizer sizer) {
 
@@ -285,7 +281,7 @@ public static void createPartitionedRegion(boolean setEvictionOn,
   /**
    * returns data size in bytes
    */
-  public static int putData(final String regionName, final int noOfElememts,
+  private static int putData(final String regionName, final int noOfElememts,
       final int sizeOfElement) {
     int result = 0;
     final Region pr = cache.getRegion(regionName);
@@ -298,7 +294,7 @@ public static int putData(final String regionName, final int noOfElememts,
     return result;
   }
 
-  public static void verifySize(String regionName, int noOfElememts, int entrySize) {
+  private static void verifySize(String regionName, int noOfElememts, int entrySize) {
     final Region pr = cache.getRegion(regionName);
     for (final Iterator i =
         ((PartitionedRegion) pr).getDataStore().getAllLocalBuckets().iterator(); i.hasNext();) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionStatsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionStatsDUnitTest.java
similarity index 79%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/EvictionStatsDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionStatsDUnitTest.java
index cd919e6241..d364050dc0 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionStatsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionStatsDUnitTest.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.eviction;
 
 import static org.junit.Assert.*;
 
@@ -35,14 +35,14 @@
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.util.ObjectSizer;
 import org.apache.geode.cache30.CacheSerializableRunnable;
-import org.apache.geode.cache30.CacheTestCase;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.OSProcess;
+import org.apache.geode.internal.cache.AbstractLRURegionMap;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.RegionMap;
 import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
-import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
-import org.apache.geode.internal.cache.lru.HeapLRUCapacityController;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.IgnoredException;
@@ -82,6 +82,11 @@ public final void postSetUp() throws Exception {
     dataStore2 = host.getVM(1);
   }
 
+  private InternalEvictionStatistics getInternalEvictionStatistics(RegionMap regionMap) {
+    return (InternalEvictionStatistics) ((AbstractLRURegionMap) regionMap).getEvictionList()
+        .getStatistics();
+  }
+
   @Test
   public void testEntryLruLimitNDestroyLimit() {
     // Ignore this excetion as this can happen if pool is shutting down
@@ -95,15 +100,13 @@ public void testEntryLruLimitNDestroyLimit() {
       public void run2() throws CacheException {
         final PartitionedRegion pr1 = (PartitionedRegion) cache.getRegion("PR1");
         final PartitionedRegion pr2 = (PartitionedRegion) cache.getRegion("PR2");
+        Assert.assertEquals(maxEnteries,
+            ((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics().getLimit());
         assertEquals(maxEnteries,
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getLimit());
-        assertEquals(maxEnteries,
-            ((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getLimit());
+            ((AbstractLRURegionMap) pr2.entries).getEvictionList().getStatistics().getLimit());
 
-        assertEquals(1000,
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getDestroysLimit());
-        assertEquals(1000,
-            ((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getDestroysLimit());
+        assertEquals(1000, getInternalEvictionStatistics(pr1.entries).getDestroysLimit());
+        assertEquals(1000, getInternalEvictionStatistics(pr2.entries).getDestroysLimit());
       }
     });
 
@@ -113,13 +116,11 @@ public void run2() throws CacheException {
         final PartitionedRegion pr1 = (PartitionedRegion) cache.getRegion("PR1");
         final PartitionedRegion pr2 = (PartitionedRegion) cache.getRegion("PR2");
         assertEquals(maxEnteries,
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getLimit());
+            ((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics().getLimit());
         assertEquals(maxEnteries,
-            ((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getLimit());
-        assertEquals(1000,
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getDestroysLimit());
-        assertEquals(1000,
-            ((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getDestroysLimit());
+            ((AbstractLRURegionMap) pr2.entries).getEvictionList().getStatistics().getLimit());
+        assertEquals(1000, getInternalEvictionStatistics(pr1.entries).getDestroysLimit());
+        assertEquals(1000, getInternalEvictionStatistics(pr2.entries).getDestroysLimit());
       }
     });
   }
@@ -139,13 +140,13 @@ public void run2() throws CacheException {
         final PartitionedRegion pr1 = (PartitionedRegion) cache.getRegion("PR1");
         final PartitionedRegion pr2 = (PartitionedRegion) cache.getRegion("PR2");
         assertEquals(pr1.getLocalMaxMemory(),
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getLimit() / ONE_MEG);
+            ((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics().getLimit()
+                / ONE_MEG);
         assertEquals(pr2.getLocalMaxMemory(),
-            ((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getLimit() / ONE_MEG);
-        assertEquals(1000,
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getDestroysLimit());
-        assertEquals(1000,
-            ((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getDestroysLimit());
+            ((AbstractLRURegionMap) pr2.entries).getEvictionList().getStatistics().getLimit()
+                / ONE_MEG);
+        assertEquals(1000, getInternalEvictionStatistics(pr1.entries).getDestroysLimit());
+        assertEquals(1000, getInternalEvictionStatistics(pr2.entries).getDestroysLimit());
       }
     });
 
@@ -156,13 +157,13 @@ public void run2() throws CacheException {
         final PartitionedRegion pr1 = (PartitionedRegion) cache.getRegion("PR1");
         final PartitionedRegion pr2 = (PartitionedRegion) cache.getRegion("PR2");
         assertEquals(pr1.getLocalMaxMemory(),
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getLimit() / ONE_MEG);
+            ((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics().getLimit()
+                / ONE_MEG);
         assertEquals(pr2.getLocalMaxMemory(),
-            ((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getLimit() / ONE_MEG);
-        assertEquals(1000,
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getDestroysLimit());
-        assertEquals(1000,
-            ((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getDestroysLimit());
+            ((AbstractLRURegionMap) pr2.entries).getEvictionList().getStatistics().getLimit()
+                / ONE_MEG);
+        assertEquals(1000, getInternalEvictionStatistics(pr1.entries).getDestroysLimit());
+        assertEquals(1000, getInternalEvictionStatistics(pr2.entries).getDestroysLimit());
       }
     });
 
@@ -210,7 +211,7 @@ public void testHeapLruCounter() {
     IgnoredException
         .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     prepareScenario(EvictionAlgorithm.LRU_HEAP);
-    System.setProperty(HeapLRUCapacityController.TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY,
+    System.setProperty(HeapLRUController.TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY,
         Float.toString(0));
     putData("PR1", 20);
     putData("PR2", 20);
@@ -232,8 +233,8 @@ public void testEntryLruAllCounterMethods() {
     for (int counter = 1; counter <= maxEnteries; counter++) {
       region.put(new Integer(counter), new byte[(1 * 1024 * 1024) - 2]);
     }
-    long sizeOfPRegion = ((AbstractLRURegionMap) ((PartitionedRegion) region).entries)._getLruList()
-        .stats().getCounter();
+    long sizeOfPRegion = ((AbstractLRURegionMap) ((PartitionedRegion) region).entries)
+        .getEvictionList().getStatistics().getCounter();
 
     assertEquals(sizeOfPRegion, 20);
     long bucketSize = 0;
@@ -255,8 +256,8 @@ public void testEntryLruAllCounterMethods() {
     for (int counter = 1; counter <= extraEnteries; counter++) {
       region.put(new Integer(counter), new byte[(1 * 1024 * 1024) - 2]);
     }
-    sizeOfPRegion = ((AbstractLRURegionMap) ((PartitionedRegion) region).entries)._getLruList()
-        .stats().getCounter();;
+    sizeOfPRegion = ((AbstractLRURegionMap) ((PartitionedRegion) region).entries).getEvictionList()
+        .getStatistics().getCounter();;
     assertEquals(sizeOfPRegion, 20);
     for (final Iterator i =
         ((PartitionedRegion) region).getDataStore().getAllLocalBuckets().iterator(); i.hasNext();) {
@@ -286,8 +287,8 @@ public void testEntryLruAllCounterMethods() {
       assertEquals(bucketRegion.getCounter(), 0);
       break;
     }
-    sizeOfPRegion = ((AbstractLRURegionMap) ((PartitionedRegion) region).entries)._getLruList()
-        .stats().getCounter();;
+    sizeOfPRegion = ((AbstractLRURegionMap) ((PartitionedRegion) region).entries).getEvictionList()
+        .getStatistics().getCounter();;
     assertEquals(sizeOfPRegion, 10);
   }
 
@@ -307,16 +308,16 @@ public void testEntryLRUEvictionNDestroyNNumOverflowOnDiskCount() {
           @Override
           public void run2() throws CacheException {
             PartitionedRegion pr1 = (PartitionedRegion) cache.getRegion("PR1");
-            assertEquals(((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getEvictions(),
-                (extraEnteries - maxEnteries) / 2);
-            assertEquals(((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getDestroys(),
-                ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getEvictions());
+            assertEquals(((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics()
+                .getEvictions(), (extraEnteries - maxEnteries) / 2);
+            assertEquals(getInternalEvictionStatistics(pr1.entries).getDestroys(),
+                ((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics()
+                    .getEvictions());
 
             PartitionedRegion pr2 = (PartitionedRegion) cache.getRegion("PR2");
-            assertEquals(((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getEvictions(),
-                (extraEnteries - maxEnteries) / 2);
-            assertEquals(((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getDestroys(),
-                0);
+            assertEquals(((AbstractLRURegionMap) pr2.entries).getEvictionList().getStatistics()
+                .getEvictions(), (extraEnteries - maxEnteries) / 2);
+            assertEquals(getInternalEvictionStatistics(pr2.entries).getDestroys(), 0);
             assertEquals(pr2.getDiskRegionStats().getNumOverflowOnDisk(),
                 (extraEnteries - maxEnteries) / 2);
           }
@@ -327,16 +328,16 @@ public void run2() throws CacheException {
           @Override
           public void run2() throws CacheException {
             PartitionedRegion pr1 = (PartitionedRegion) cache.getRegion("PR1");
-            assertEquals(((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getEvictions(),
-                (extraEnteries - maxEnteries) / 2);
-            assertEquals(((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getDestroys(),
-                ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getEvictions());
+            assertEquals(((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics()
+                .getEvictions(), (extraEnteries - maxEnteries) / 2);
+            assertEquals(getInternalEvictionStatistics(pr1.entries).getDestroys(),
+                ((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics()
+                    .getEvictions());
 
             PartitionedRegion pr2 = (PartitionedRegion) cache.getRegion("PR2");
-            assertEquals(((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getEvictions(),
-                (extraEnteries - maxEnteries) / 2);
-            assertEquals(((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getDestroys(),
-                0);
+            assertEquals(((AbstractLRURegionMap) pr2.entries).getEvictionList().getStatistics()
+                .getEvictions(), (extraEnteries - maxEnteries) / 2);
+            assertEquals(getInternalEvictionStatistics(pr2.entries).getDestroys(), 0);
             assertEquals(pr2.getDiskRegionStats().getNumOverflowOnDisk(),
                 (extraEnteries - maxEnteries) / 2);
           }
@@ -358,25 +359,24 @@ public void testMemLRUEvictionNDestroyNNumOverflowOnDiskCount() {
       public void run2() throws CacheException {
         PartitionedRegion pr1 = (PartitionedRegion) cache.getRegion("PR1");
         LogWriterUtils.getLogWriter().info("dddd  local" + pr1.getLocalMaxMemory());
-        LogWriterUtils.getLogWriter().info("dddd  local evi"
-            + ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getEvictions());
-        LogWriterUtils.getLogWriter()
-            .info("dddd  local entries"
-                + ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getCounter()
-                    / (1024 * 1024));
+        LogWriterUtils.getLogWriter().info("dddd  local evi" + ((AbstractLRURegionMap) pr1.entries)
+            .getEvictionList().getStatistics().getEvictions());
+        LogWriterUtils.getLogWriter().info("dddd  local entries"
+            + ((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics().getCounter()
+                / (1024 * 1024));
         HeapMemoryMonitor hmm =
             ((InternalResourceManager) cache.getResourceManager()).getHeapMonitor();
         long memused = hmm.getBytesUsed() / (1024 * 1024);
         LogWriterUtils.getLogWriter().info("dddd  local memused= " + memused);
-        assertTrue(((AbstractLRURegionMap) pr1.entries)._getLruList().stats()
+        assertTrue(((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics()
             .getEvictions() >= extraEntries / 2);
-        assertEquals(((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getDestroys(),
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getEvictions());
+        assertEquals(getInternalEvictionStatistics(pr1.entries).getDestroys(),
+            ((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics().getEvictions());
 
         PartitionedRegion pr2 = (PartitionedRegion) cache.getRegion("PR2");
-        assertTrue(((AbstractLRURegionMap) pr2.entries)._getLruList().stats()
+        assertTrue(((AbstractLRURegionMap) pr2.entries).getEvictionList().getStatistics()
             .getEvictions() >= extraEntries / 2);
-        assertEquals(((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getDestroys(), 0);
+        assertEquals(getInternalEvictionStatistics(pr2.entries).getDestroys(), 0);
         assertTrue(pr2.getDiskRegionStats().getNumOverflowOnDisk() >= extraEntries / 2);
       }
     });
@@ -385,15 +385,15 @@ public void run2() throws CacheException {
       @Override
       public void run2() throws CacheException {
         PartitionedRegion pr1 = (PartitionedRegion) cache.getRegion("PR1");
-        assertTrue(((AbstractLRURegionMap) pr1.entries)._getLruList().stats()
+        assertTrue(((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics()
             .getEvictions() >= extraEntries / 2);
-        assertEquals(((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getDestroys(),
-            ((AbstractLRURegionMap) pr1.entries)._getLruList().stats().getEvictions());
+        assertEquals(getInternalEvictionStatistics(pr1.entries).getDestroys(),
+            ((AbstractLRURegionMap) pr1.entries).getEvictionList().getStatistics().getEvictions());
 
         PartitionedRegion pr2 = (PartitionedRegion) cache.getRegion("PR2");
-        assertTrue(((AbstractLRURegionMap) pr2.entries)._getLruList().stats()
+        assertTrue(((AbstractLRURegionMap) pr2.entries).getEvictionList().getStatistics()
             .getEvictions() >= extraEntries / 2);
-        assertEquals(((AbstractLRURegionMap) pr2.entries)._getLruList().stats().getDestroys(), 0);
+        assertEquals(getInternalEvictionStatistics(pr2.entries).getDestroys(), 0);
         assertTrue(pr2.getDiskRegionStats().getNumOverflowOnDisk() >= extraEntries / 2);
       }
     });
@@ -528,7 +528,7 @@ private long getPRCounter(String prRegionName) {
 
   public static long getPartionRegionCounter(String prRegionName) {
     final PartitionedRegion pr = (PartitionedRegion) cache.getRegion(prRegionName);
-    return ((AbstractLRURegionMap) pr.entries)._getLruList().stats().getCounter();
+    return ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics().getCounter();
   }
 
   private long getCounterForBucketsOfPR(String prRegionName) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionTestBase.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionTestBase.java
similarity index 90%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/EvictionTestBase.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionTestBase.java
index 2aa441b48d..830bec81a8 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionTestBase.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/EvictionTestBase.java
@@ -12,13 +12,13 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.eviction;
 
 import static org.junit.Assert.*;
 
 import java.io.File;
-import java.util.ArrayList;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
@@ -40,12 +40,17 @@
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.OSProcess;
+import org.apache.geode.internal.cache.AbstractLRURegionMap;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
 import org.apache.geode.internal.cache.control.MemoryEvent;
 import org.apache.geode.internal.cache.control.MemoryThresholds.MemoryState;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.LogWriterUtils;
@@ -72,7 +77,7 @@
 
   protected static Region region = null;
 
-  static int maxEnteries = 20;
+  static int maxEntries = 20;
 
   static int maxSizeInMb = 20;
 
@@ -100,9 +105,7 @@ public void raiseFakeNotification(VM vm, final String prName, final int noOfExpe
       @Override
       public void run2() throws CacheException {
         final LocalRegion region = (LocalRegion) cache.getRegion(prName);
-        getEvictor().testAbortAfterLoopCount = 1;
-
-        RegionEvictorTask.TEST_EVICTION_BURST_PAUSE_TIME_MILLIS = 0;
+        getEvictor().setTestAbortAfterLoopCount(1);
 
         InternalResourceManager irm = ((GemFireCacheImpl) cache).getInternalResourceManager();
         HeapMemoryMonitor hmm = irm.getHeapMonitor();
@@ -111,8 +114,8 @@ public void run2() throws CacheException {
 
         WaitCriterion wc = new WaitCriterion() {
           public boolean done() {
-            final long currentEvictions =
-                ((AbstractLRURegionMap) region.entries)._getLruList().stats().getEvictions();
+            final long currentEvictions = ((AbstractLRURegionMap) region.entries).getEvictionList()
+                .getStatistics().getEvictions();
             if (Math.abs(currentEvictions - noOfExpectedEvictions) <= 1) { // Margin of error is 1
               return true;
             } else if (currentEvictions > noOfExpectedEvictions) {
@@ -123,7 +126,8 @@ public boolean done() {
 
           public String description() {
             return "expected " + noOfExpectedEvictions + " evictions, but got "
-                + ((AbstractLRURegionMap) region.entries)._getLruList().stats().getEvictions();
+                + ((AbstractLRURegionMap) region.entries).getEvictionList().getStatistics()
+                    .getEvictions();
           }
         };
         Wait.waitForCriterion(wc, 60000, 1000, true);
@@ -197,7 +201,6 @@ public void run() {
 
   public void sendFakeNotification() {
     HeapMemoryMonitor hmm = ((InternalResourceManager) cache.getResourceManager()).getHeapMonitor();
-    RegionEvictorTask.TEST_EVICTION_BURST_PAUSE_TIME_MILLIS = 0;
     MemoryEvent event = new MemoryEvent(getResourceType(), MemoryState.NORMAL, MemoryState.EVICTION,
         cache.getDistributedSystem().getDistributedMember(), 90, true, hmm.getThresholds());
     getEvictor().onEvent(event);
@@ -283,7 +286,7 @@ public void createCache() {
     }
   }
 
-  public ArrayList<Integer> getTestTaskSetSizes() {
+  public List<Integer> getTestTaskSetSizes() {
     return getEvictor().testOnlyGetSizeOfTasks();
   }
 
@@ -381,24 +384,20 @@ public void run2() throws CacheException {
   public void validateNoOfEvictions(final String regionName, final int noOfEvictions) {
     final SerializableCallable validate = new SerializableCallable("Validate evictions") {
       public Object call() throws Exception {
+        final PartitionedRegion pr = (PartitionedRegion) cache.getRegion(regionName);
 
-        try {
-          final PartitionedRegion pr = (PartitionedRegion) cache.getRegion(regionName);
-
-          for (final Iterator i = pr.getDataStore().getAllLocalBuckets().iterator(); i.hasNext();) {
-            final Map.Entry entry = (Map.Entry) i.next();
-            final BucketRegion bucketRegion = (BucketRegion) entry.getValue();
-            if (bucketRegion == null) {
-              continue;
-            }
-            LogWriterUtils.getLogWriter().info(
-                "FINAL bucket= " + bucketRegion.getFullPath() + "size= " + bucketRegion.size());
+        for (final Iterator i = pr.getDataStore().getAllLocalBuckets().iterator(); i.hasNext();) {
+          final Map.Entry entry = (Map.Entry) i.next();
+          final BucketRegion bucketRegion = (BucketRegion) entry.getValue();
+          if (bucketRegion == null) {
+            continue;
           }
-
-          return new Long(((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions());
-
-        } finally {
+          LogWriterUtils.getLogWriter()
+              .info("FINAL bucket= " + bucketRegion.getFullPath() + "size= " + bucketRegion.size());
         }
+
+        return new Long(
+            ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics().getEvictions());
       }
     };
     long evictionsInVM1 = (Long) dataStore1.invoke(validate);
@@ -412,11 +411,8 @@ public void verifyThreadPoolTaskCount(final int taskCountToBeVerified) {
     final SerializableCallable getThreadPoolTaskCount =
         new SerializableCallable("Validate evictions") {
           public Object call() throws Exception {
-            try {
-              return getEvictor().getEvictorThreadPool() != null
-                  ? getEvictor().getEvictorThreadPool().getTaskCount() : 0;
-            } finally {
-            }
+            return getEvictor().getEvictorThreadPool() != null
+                ? getEvictor().getEvictorThreadPool().getTaskCount() : 0;
           }
         };
     Long taskCountOfVM = (Long) dataStore1.invoke(getThreadPoolTaskCount);
@@ -460,13 +456,9 @@ public void validateNoOfEvictionsInDataStore3N4(final String regionName,
       final int noOfEvictions) {
     final SerializableCallable validate = new SerializableCallable("Validate evictions") {
       public Object call() throws Exception {
-
-        try {
-          final PartitionedRegion pr = (PartitionedRegion) cache.getRegion(regionName);
-          return new Long(((AbstractLRURegionMap) pr.entries)._getLruList().stats().getEvictions());
-
-        } finally {
-        }
+        final PartitionedRegion pr = (PartitionedRegion) cache.getRegion(regionName);
+        return new Long(
+            ((AbstractLRURegionMap) pr.entries).getEvictionList().getStatistics().getEvictions());
       }
     };
     long evictionsInVM1 = (Long) dataStore3.invoke(validate);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LIFOListTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LIFOListTest.java
new file mode 100644
index 0000000000..e7c288a1ff
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LIFOListTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class LIFOListTest {
+  private BucketRegion bucketRegion;
+  private InternalEvictionStatistics stats;
+
+  @Before
+  public void setup() {
+    bucketRegion = mock(BucketRegion.class);
+    stats = mock(InternalEvictionStatistics.class);
+  }
+
+  @Test
+  public void evictingFromEmptyListTest() throws Exception {
+    LIFOList list = new LIFOList(stats, bucketRegion);
+    assertThat(list.getEvictableEntry()).isNull();
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void evictingFromNonEmptyListTest() throws Exception {
+    LIFOList list = new LIFOList(stats, bucketRegion);
+    EvictionNode node = mock(EvictableEntry.class);
+    list.appendEntry(node);
+    assertThat(list.size()).isEqualTo(1);
+
+    when(node.next()).thenReturn(list.tail);
+    when(node.previous()).thenReturn(list.head);
+    assertThat(list.getEvictableEntry()).isSameAs(node);
+    verify(stats).incEvaluations(anyLong());
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void doesNotEvictNodeInTransaction() throws Exception {
+    LIFOList list = new LIFOList(stats, bucketRegion);
+    EvictionNode nodeInTransaction = mock(EvictableEntry.class);
+    when(nodeInTransaction.isInUseByTransaction()).thenReturn(true);
+    EvictionNode nodeNotInTransaction = mock(EvictableEntry.class);
+    list.appendEntry(nodeNotInTransaction);
+    list.appendEntry(nodeInTransaction);
+    assertThat(list.size()).isEqualTo(2);
+
+    when(nodeInTransaction.next()).thenReturn(list.tail);
+    when(nodeInTransaction.previous()).thenReturn(nodeNotInTransaction);
+    when(nodeNotInTransaction.next()).thenReturn(list.tail);
+    when(nodeNotInTransaction.previous()).thenReturn(list.head);
+    assertThat(list.getEvictableEntry()).isSameAs(nodeNotInTransaction);
+    verify(stats).incEvaluations(2);
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void doesNotEvictNodeThatIsEvicted() throws Exception {
+    LIFOList list = new LIFOList(stats, bucketRegion);
+    EvictionNode evictedNode = mock(EvictableEntry.class);
+    when(evictedNode.isEvicted()).thenReturn(true);
+    EvictionNode node = mock(EvictableEntry.class);
+    list.appendEntry(node);
+    list.appendEntry(evictedNode);
+    assertThat(list.size()).isEqualTo(2);
+
+    when(evictedNode.next()).thenReturn(list.tail);
+    when(evictedNode.previous()).thenReturn(node);
+    when(node.next()).thenReturn(list.tail);
+    when(node.previous()).thenReturn(list.head);
+    assertThat(list.getEvictableEntry()).isSameAs(node);
+    verify(stats).incEvaluations(2);
+    assertThat(list.size()).isZero();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithAsyncSortingIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithAsyncSortingIntegrationTest.java
new file mode 100755
index 0000000000..5e97ccc00f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithAsyncSortingIntegrationTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.IntConsumer;
+import java.util.stream.IntStream;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.EvictionAlgorithm;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.internal.cache.InternalRegion;
+import org.apache.geode.internal.cache.InternalRegionArguments;
+import org.apache.geode.internal.lang.SystemPropertyHelper;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+/**
+ * This class tests the LRUCapacityController's core clock algorithm.
+ */
+@Category(IntegrationTest.class)
+public class LRUListWithAsyncSortingIntegrationTest {
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  private InternalRegion region;
+  private LRUListWithAsyncSorting evictionList;
+  private List<EvictionNode> nodes;
+
+  @Before
+  public void setUp() throws Exception {
+    region = createRegion();
+    evictionList = getEvictionList(region, new TestEvictionController());
+    nodes = new ArrayList<>();
+    IntStream.range(0, 10).forEach(i -> {
+      EvictionNode node = new LRUTestEntry(i);
+      nodes.add(node);
+      evictionList.appendEntry(node);
+    });
+  }
+
+  @Test
+  public void testAddEvictionList() throws Exception {
+    IntStream.range(0, 10).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("check node ids in order").isEqualTo(i);
+    });
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testEvicted() throws Exception {
+    actOnEvenNodes(i -> evictionList.destroyEntry(nodes.get(i)));
+
+    actOnOddNodes(i -> {
+      LRUTestEntry node = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(node.id()).as("check node ids in order").isEqualTo(i);
+    });
+
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testRecentlyUsed() throws Exception {
+    actOnEvenNodes(i -> nodes.get(i).setRecentlyUsed(region));
+
+    evictionList.scan();
+
+    actOnOddNodes(i -> {
+      LRUTestEntry node = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(node.id()).as("check non-recently used entries returned first").isEqualTo(i);
+    });
+
+    actOnEvenNodes(i -> {
+      LRUTestEntry node = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(node.id()).as("check recently used entries returned last").isEqualTo(i);
+    });
+
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testRemoveHead() throws Exception {
+    evictionList.destroyEntry(nodes.get(0));
+    IntStream.range(1, 10).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("all but first node should remain").isEqualTo(i);
+    });
+
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testRemoveMiddle() throws Exception {
+    evictionList.destroyEntry(nodes.get(5));
+    IntStream.range(0, 5).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("nodes before removed one should remain").isEqualTo(i);
+    });
+
+    IntStream.range(6, 10).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("nodes after removed one should remain").isEqualTo(i);
+    });
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testRemoveTail() throws Exception {
+    evictionList.destroyEntry(nodes.get(9));
+    IntStream.range(0, 9).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("all but first node should remain").isEqualTo(i);
+    });
+
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void doesNotEvictRecentlyUsedEntryIfUnderLimit() throws Exception {
+    nodes.get(0).setRecentlyUsed(region);
+    assertThat(evictionList.getEvictableEntry()).isSameAs(nodes.get(1));
+    assertThat(evictionList.size()).isEqualTo(9);
+  }
+
+  @Test
+  public void evictsRecentlyUsedNodeIfOverLimit() throws Exception {
+    EvictionNode node = new LRUTestEntry(10);
+    nodes.add(node);
+    evictionList.appendEntry(node);
+    IntStream.range(0, 11).forEach(i -> nodes.get(i).setRecentlyUsed(region));
+    assertThat(evictionList.getEvictableEntry()).isSameAs(nodes.get(10));
+  }
+
+  private LRUListWithAsyncSorting getEvictionList(Region region, EvictionController eviction) {
+    System.setProperty("geode." + SystemPropertyHelper.EVICTION_SCAN_ASYNC, "true");
+    return (LRUListWithAsyncSorting) new EvictionListBuilder(EvictionAlgorithm.LRU_HEAP)
+        .withRegion(region).withEvictionController(eviction).withArgs(new InternalRegionArguments())
+        .create();
+  }
+
+  private InternalRegion createRegion() throws Exception {
+    Cache cache = new CacheFactory().set("locators", "").set("mcast-port", "0").create();
+    return (InternalRegion) cache.createRegionFactory(RegionShortcut.PARTITION)
+        .create(testName.getMethodName());
+  }
+
+  private void actOnEvenNodes(IntConsumer s) {
+    IntStream.range(0, 10).filter(i -> i % 2 == 0).forEach(s);
+  }
+
+  private void actOnOddNodes(IntConsumer s) {
+    IntStream.range(0, 10).filter(i -> i % 2 == 1).forEach(s);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithAsyncSortingTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithAsyncSortingTest.java
new file mode 100644
index 0000000000..04ab3e9b61
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithAsyncSortingTest.java
@@ -0,0 +1,251 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.RegionEntryContext;
+import org.apache.geode.internal.lang.SystemPropertyHelper;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class LRUListWithAsyncSortingTest {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private BucketRegion bucketRegion;
+  private InternalEvictionStatistics stats;
+  private ExecutorService executor = mock(ExecutorService.class);
+
+  @Before
+  public void setup() {
+    bucketRegion = mock(BucketRegion.class);
+    stats = mock(InternalEvictionStatistics.class);
+  }
+
+  @Test
+  public void scansOnlyWhenOverThreshold() throws Exception {
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, executor);
+    for (int i = 0; i < 5; i++) {
+      list.appendEntry(mock(EvictionNode.class));
+    }
+
+    list.incrementRecentlyUsed();
+    verifyNoMoreInteractions(executor);
+
+    list.incrementRecentlyUsed();
+    verify(executor).submit(any(Runnable.class));
+  }
+
+  @Test
+  public void clearResetsRecentlyUsedCounter() throws Exception {
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, executor);
+    list.incrementRecentlyUsed();
+    assertThat(list.getRecentlyUsedCount()).isEqualTo(1);
+
+    list.clear(null);
+    assertThat(list.getRecentlyUsedCount()).isZero();
+  }
+
+  @Test
+  public void doesNotRunScanOnEmptyList() throws Exception {
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, executor);
+    list.incrementRecentlyUsed();
+    verifyNoMoreInteractions(executor);
+  }
+
+  @Test
+  public void usesSystemPropertyThresholdIfSpecified() throws Exception {
+    System.setProperty("geode." + SystemPropertyHelper.EVICTION_SCAN_THRESHOLD_PERCENT, "55");
+    try {
+      LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, executor);
+
+      list.appendEntry(mock(EvictionNode.class));
+      list.appendEntry(mock(EvictionNode.class));
+      list.incrementRecentlyUsed();
+      verifyNoMoreInteractions(executor);
+
+      list.incrementRecentlyUsed();
+      verify(executor).submit(any(Runnable.class));
+    } finally {
+      System.clearProperty("geode." + SystemPropertyHelper.EVICTION_SCAN_THRESHOLD_PERCENT);
+    }
+  }
+
+  @Test
+  public void evictingFromEmptyListTest() throws Exception {
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, executor);
+    assertThat(list.getEvictableEntry()).isNull();
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void evictingFromNonEmptyListTest() throws Exception {
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, executor);
+    EvictionNode node = mock(EvictableEntry.class);
+    list.appendEntry(node);
+    assertThat(list.size()).isEqualTo(1);
+
+    when(node.next()).thenReturn(list.tail);
+    when(node.previous()).thenReturn(list.head);
+    assertThat(list.getEvictableEntry()).isSameAs(node);
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void doesNotEvictNodeInTransaction() throws Exception {
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, executor);
+    EvictionNode nodeInTransaction = mock(EvictableEntry.class, "nodeInTransaction");
+    when(nodeInTransaction.isInUseByTransaction()).thenReturn(true);
+    EvictionNode nodeNotInTransaction = mock(EvictableEntry.class, "nodeNotInTransaction");
+    list.appendEntry(nodeInTransaction);
+    list.appendEntry(nodeNotInTransaction);
+    assertThat(list.size()).isEqualTo(2);
+
+    when(nodeInTransaction.next()).thenReturn(nodeNotInTransaction);
+    when(nodeInTransaction.previous()).thenReturn(list.head);
+    when(nodeNotInTransaction.next()).thenReturn(list.tail);
+    when(nodeNotInTransaction.previous()).thenReturn(list.head);
+    assertThat(list.getEvictableEntry()).isSameAs(nodeNotInTransaction);
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void doesNotEvictNodeThatIsEvicted() throws Exception {
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, executor);
+
+    EvictionNode evictedNode = mock(EvictableEntry.class);
+    when(evictedNode.isEvicted()).thenReturn(true);
+
+    EvictionNode node = mock(EvictableEntry.class);
+    list.appendEntry(evictedNode);
+    list.appendEntry(node);
+    assertThat(list.size()).isEqualTo(2);
+
+    when(evictedNode.next()).thenReturn(node);
+    when(evictedNode.previous()).thenReturn(list.head);
+    when(node.next()).thenReturn(list.tail);
+    when(node.previous()).thenReturn(list.head);
+    assertThat(list.getEvictableEntry()).isSameAs(node);
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void scanUnsetsRecentlyUsedOnNode() throws Exception {
+    ExecutorService realExecutor = Executors.newSingleThreadExecutor();
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, realExecutor);
+
+    EvictionNode recentlyUsedNode = mock(EvictableEntry.class);
+    when(recentlyUsedNode.previous()).thenReturn(list.head);
+    when(recentlyUsedNode.isRecentlyUsed()).thenReturn(true);
+
+    list.appendEntry(recentlyUsedNode);
+    when(recentlyUsedNode.next()).thenReturn(list.tail);
+    list.incrementRecentlyUsed();
+
+    // unsetRecentlyUsed() is called once on appendEntry(...) and once during scan
+    await().atMost(10, TimeUnit.SECONDS)
+        .until(() -> verify(recentlyUsedNode, times(2)).unsetRecentlyUsed());
+    realExecutor.shutdown();
+  }
+
+  @Test
+  public void scanEndsOnlyUpToSize() throws Exception {
+    ExecutorService realExecutor = Executors.newSingleThreadExecutor();
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, realExecutor);
+
+    EvictionNode recentlyUsedNode = mock(EvictableEntry.class);
+    when(recentlyUsedNode.previous()).thenReturn(list.head);
+    when(recentlyUsedNode.isRecentlyUsed()).thenReturn(true);
+
+    list.appendEntry(recentlyUsedNode);
+    when(recentlyUsedNode.next()).thenReturn(recentlyUsedNode);
+    list.incrementRecentlyUsed();
+
+    // unsetRecentlyUsed() is called once on appendEntry(...) and once during scan
+    await().atMost(10, TimeUnit.SECONDS)
+        .until(() -> verify(recentlyUsedNode, times(2)).unsetRecentlyUsed());
+    realExecutor.shutdown();
+  }
+
+  @Test
+  public void scanMovesRecentlyUsedNodeToTail() throws Exception {
+    ExecutorService realExecutor = Executors.newSingleThreadExecutor();
+    LRUListWithAsyncSorting list = new LRUListWithAsyncSorting(stats, bucketRegion, realExecutor);
+
+    EvictionNode recentlyUsedNode = mock(EvictableEntry.class, "first");
+    EvictionNode secondNode = mock(EvictableEntry.class, "second");
+    EvictionNode thirdNode = mock(EvictableEntry.class, "third");
+
+    list.appendEntry(recentlyUsedNode);
+    list.appendEntry(secondNode);
+    list.appendEntry(thirdNode);
+
+    when(recentlyUsedNode.next()).thenReturn(secondNode);
+    when(recentlyUsedNode.previous()).thenReturn(list.head);
+    when(secondNode.next()).thenReturn(thirdNode);
+    // The second node is moved to first. Its previous will be head.
+    when(secondNode.previous()).thenReturn(list.head);
+    when(thirdNode.next()).thenReturn(list.tail);
+    when(thirdNode.previous()).thenReturn(secondNode);
+
+    when(recentlyUsedNode.isRecentlyUsed()).thenReturn(true);
+    list.incrementRecentlyUsed();
+
+    // unsetRecentlyUsed() is called once on appendEntry(...) and once during scan
+    await().atMost(10, TimeUnit.SECONDS)
+        .until(() -> verify(recentlyUsedNode, times(2)).unsetRecentlyUsed());
+    assertThat(list.tail.previous()).isEqualTo(recentlyUsedNode);
+    realExecutor.shutdown();
+  }
+
+  @Test
+  public void startScanIfEvictableEntryIsRecentlyUsed() throws Exception {
+    List<EvictionNode> nodes = new ArrayList<>();
+    LRUListWithAsyncSorting lruEvictionList =
+        new LRUListWithAsyncSorting(stats, bucketRegion, executor);
+    IntStream.range(0, 11).forEach(i -> {
+      EvictionNode node = new LRUTestEntry(i);
+      nodes.add(node);
+      lruEvictionList.appendEntry(node);
+      node.setRecentlyUsed(mock(RegionEntryContext.class));
+    });
+
+    assertThat(lruEvictionList.getEvictableEntry().isRecentlyUsed()).isTrue();
+    verify(executor).submit(any(Runnable.class));
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithSyncSortingIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithSyncSortingIntegrationTest.java
new file mode 100644
index 0000000000..0bb28b5b2a
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithSyncSortingIntegrationTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.IntConsumer;
+import java.util.stream.IntStream;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.EvictionAlgorithm;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.internal.cache.InternalRegion;
+import org.apache.geode.internal.cache.InternalRegionArguments;
+import org.apache.geode.internal.lang.SystemPropertyHelper;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+/**
+ * This class tests the LRUCapacityController's core clock algorithm.
+ */
+@Category(IntegrationTest.class)
+public class LRUListWithSyncSortingIntegrationTest {
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  private InternalRegion region;
+  private LRUListWithSyncSorting evictionList;
+  private List<EvictionNode> nodes;
+
+  @Before
+  public void setUp() throws Exception {
+    region = createRegion();
+    evictionList = getEvictionList(region, new TestEvictionController());
+    nodes = new ArrayList<>();
+    IntStream.range(0, 10).forEach(i -> {
+      EvictionNode node = new LRUTestEntry(i);
+      nodes.add(node);
+      evictionList.appendEntry(node);
+    });
+  }
+
+  @Test
+  public void testAddEvictionList() throws Exception {
+    IntStream.range(0, 10).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("check node ids in order").isEqualTo(i);
+    });
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testEvicted() throws Exception {
+    actOnEvenNodes(i -> evictionList.destroyEntry(nodes.get(i)));
+
+    actOnOddNodes(i -> {
+      LRUTestEntry node = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(node.id()).as("check node ids in order").isEqualTo(i);
+    });
+
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testRecentlyUsed() throws Exception {
+    actOnEvenNodes(i -> nodes.get(i).setRecentlyUsed(region));
+
+    actOnOddNodes(i -> {
+      LRUTestEntry node = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(node.id()).as("check non-recently used entries returned first").isEqualTo(i);
+    });
+
+    actOnEvenNodes(i -> {
+      LRUTestEntry node = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(node.id()).as("check recently used entries returned last").isEqualTo(i);
+    });
+
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testRemoveHead() throws Exception {
+    evictionList.destroyEntry(nodes.get(0));
+    IntStream.range(1, 10).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("all but first node should remain").isEqualTo(i);
+    });
+
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testRemoveMiddle() throws Exception {
+    evictionList.destroyEntry(nodes.get(5));
+    IntStream.range(0, 5).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("nodes before removed one should remain").isEqualTo(i);
+    });
+
+    IntStream.range(6, 10).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("nodes after removed one should remain").isEqualTo(i);
+    });
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void testRemoveTail() throws Exception {
+    evictionList.destroyEntry(nodes.get(9));
+    IntStream.range(0, 9).forEach(i -> {
+      LRUTestEntry entry = (LRUTestEntry) evictionList.getEvictableEntry();
+      assertThat(entry.id()).as("all but first node should remain").isEqualTo(i);
+    });
+
+    assertThat(evictionList.getEvictableEntry()).as("check list is now empty").isNull();
+  }
+
+  @Test
+  public void doesNotEvictRecentlyUsedEntryIfUnderLimit() throws Exception {
+    nodes.get(0).setRecentlyUsed(region);
+    assertThat(evictionList.getEvictableEntry()).isSameAs(nodes.get(1));
+    assertThat(evictionList.size()).isEqualTo(9);
+  }
+
+  @Test
+  public void evictsRecentlyUsedNodeIfOverLimit() throws Exception {
+    IntStream.range(10, 16).forEach(i -> {
+      EvictionNode node = new LRUTestEntry(i);
+      nodes.add(node);
+      evictionList.appendEntry(node);
+    });
+
+    IntStream.range(0, 16).forEach(i -> nodes.get(i).setRecentlyUsed(region));
+    assertThat(evictionList.getEvictableEntry()).isSameAs(nodes.get(15));
+  }
+
+  private LRUListWithSyncSorting getEvictionList(Region region, EvictionController eviction) {
+    System.setProperty("geode." + SystemPropertyHelper.EVICTION_SCAN_ASYNC, "false");
+    System.setProperty("geode." + SystemPropertyHelper.EVICTION_SEARCH_MAX_ENTRIES, "15");
+    return (LRUListWithSyncSorting) new EvictionListBuilder(EvictionAlgorithm.LRU_HEAP)
+        .withRegion(region).withEvictionController(eviction).withArgs(new InternalRegionArguments())
+        .create();
+  }
+
+  private InternalRegion createRegion() throws Exception {
+    Cache cache = new CacheFactory().set("locators", "").set("mcast-port", "0").create();
+    return (InternalRegion) cache.createRegionFactory(RegionShortcut.PARTITION)
+        .create(testName.getMethodName());
+  }
+
+  private void actOnEvenNodes(IntConsumer s) {
+    IntStream.range(0, 10).filter(i -> i % 2 == 0).forEach(s);
+  }
+
+  private void actOnOddNodes(IntConsumer s) {
+    IntStream.range(0, 10).filter(i -> i % 2 == 1).forEach(s);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithSyncSortingTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithSyncSortingTest.java
new file mode 100644
index 0000000000..9fcb6fb7c2
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUListWithSyncSortingTest.java
@@ -0,0 +1,153 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.lang.SystemPropertyHelper;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class LRUListWithSyncSortingTest {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  private BucketRegion bucketRegion;
+  private InternalEvictionStatistics stats;
+
+  @Before
+  public void setup() {
+    bucketRegion = mock(BucketRegion.class);
+    stats = mock(InternalEvictionStatistics.class);
+  }
+
+  @Test
+  public void evictingFromEmptyListTest() throws Exception {
+    LRUListWithSyncSorting list = new LRUListWithSyncSorting(stats, bucketRegion);
+    assertThat(list.getEvictableEntry()).isNull();
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void evictingFromNonEmptyListTest() throws Exception {
+    LRUListWithSyncSorting list = new LRUListWithSyncSorting(stats, bucketRegion);
+    EvictionNode node = mock(EvictableEntry.class);
+    list.appendEntry(node);
+    assertThat(list.size()).isEqualTo(1);
+
+    when(node.next()).thenReturn(list.tail);
+    when(node.previous()).thenReturn(list.head);
+    assertThat(list.getEvictableEntry()).isSameAs(node);
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void doesNotEvictRecentlyUsed() throws Exception {
+    LRUListWithSyncSorting list = new LRUListWithSyncSorting(stats, bucketRegion);
+    EvictionNode recentlyUsedNode = mock(EvictableEntry.class);
+    list.appendEntry(recentlyUsedNode);
+    when(recentlyUsedNode.isRecentlyUsed()).thenReturn(true);
+    EvictionNode node = mock(EvictableEntry.class);
+    list.appendEntry(node);
+
+    when(recentlyUsedNode.next()).thenReturn(node).thenReturn(null);
+    when(recentlyUsedNode.previous()).thenReturn(list.head);
+    when(node.next()).thenReturn(list.tail);
+    when(node.previous()).thenReturn(recentlyUsedNode);
+
+    assertThat(list.getEvictableEntry()).isSameAs(node);
+    assertThat(list.tail.previous()).isSameAs(recentlyUsedNode);
+    verify(recentlyUsedNode, atLeast(1)).unsetRecentlyUsed();
+    assertThat(list.size()).isOne();
+  }
+
+  @Test
+  public void doesNotEvictNodeInTransaction() throws Exception {
+    LRUListWithSyncSorting list = new LRUListWithSyncSorting(stats, bucketRegion);
+    EvictionNode nodeInTransaction = mock(EvictableEntry.class, "nodeInTransaction");
+    when(nodeInTransaction.isInUseByTransaction()).thenReturn(true);
+    EvictionNode nodeNotInTransaction = mock(EvictableEntry.class, "nodeNotInTransaction");
+    list.appendEntry(nodeInTransaction);
+    list.appendEntry(nodeNotInTransaction);
+    assertThat(list.size()).isEqualTo(2);
+
+    when(nodeInTransaction.next()).thenReturn(nodeNotInTransaction);
+    when(nodeInTransaction.previous()).thenReturn(list.head);
+    when(nodeNotInTransaction.next()).thenReturn(list.tail);
+    when(nodeNotInTransaction.previous()).thenReturn(list.head);
+    assertThat(list.getEvictableEntry()).isSameAs(nodeNotInTransaction);
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void doesNotEvictNodeThatIsEvicted() throws Exception {
+    LRUListWithSyncSorting list = new LRUListWithSyncSorting(stats, bucketRegion);
+
+    EvictionNode evictedNode = mock(EvictableEntry.class);
+    when(evictedNode.isEvicted()).thenReturn(true);
+
+    EvictionNode node = mock(EvictableEntry.class);
+    list.appendEntry(evictedNode);
+    list.appendEntry(node);
+    assertThat(list.size()).isEqualTo(2);
+
+    when(evictedNode.next()).thenReturn(node);
+    when(evictedNode.previous()).thenReturn(list.head);
+    when(node.next()).thenReturn(list.tail);
+    when(node.previous()).thenReturn(list.head);
+    assertThat(list.getEvictableEntry()).isSameAs(node);
+    assertThat(list.size()).isZero();
+  }
+
+  @Test
+  public void verifyRecentlyUsedNodeIsGreedilyEvicted() throws Exception {
+    System.setProperty("geode." + SystemPropertyHelper.EVICTION_SEARCH_MAX_ENTRIES, "1");
+
+    LRUListWithSyncSorting list = new LRUListWithSyncSorting(stats, bucketRegion);
+    EvictionNode recentlyUsedNode1 = mock(EvictableEntry.class, "RecentlyUsed1");
+    list.appendEntry(recentlyUsedNode1);
+    when(recentlyUsedNode1.isRecentlyUsed()).thenReturn(true);
+    EvictionNode recentlyUsedNode2 = mock(EvictableEntry.class, "RecentlyUsed2");
+    when(recentlyUsedNode2.isRecentlyUsed()).thenReturn(true);
+    list.appendEntry(recentlyUsedNode2);
+
+    when(recentlyUsedNode1.next()).thenReturn(recentlyUsedNode2).thenReturn(null);
+    when(recentlyUsedNode1.previous()).thenReturn(list.head);
+    when(recentlyUsedNode2.next()).thenReturn(list.tail);
+    when(recentlyUsedNode2.previous()).thenReturn(recentlyUsedNode1);
+
+    assertThat(list.getEvictableEntry()).isSameAs(recentlyUsedNode2);
+    assertThat(list.tail.previous()).isSameAs(recentlyUsedNode1);
+    verify(recentlyUsedNode1, atLeast(1)).unsetRecentlyUsed();
+    assertThat(list.size()).isOne();
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUTestEntry.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUTestEntry.java
new file mode 100644
index 0000000000..51150d3d75
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LRUTestEntry.java
@@ -0,0 +1,458 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import org.apache.geode.cache.CacheWriterException;
+import org.apache.geode.cache.EntryEvent;
+import org.apache.geode.cache.EntryNotFoundException;
+import org.apache.geode.cache.TimeoutException;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.internal.ByteArrayDataInput;
+import org.apache.geode.internal.InternalStatisticsDisabledException;
+import org.apache.geode.internal.Version;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.internal.cache.EntryEventImpl;
+import org.apache.geode.internal.cache.InitialImageOperation;
+import org.apache.geode.internal.cache.InternalRegion;
+import org.apache.geode.internal.cache.RegionClearedException;
+import org.apache.geode.internal.cache.RegionEntryContext;
+import org.apache.geode.internal.cache.Token;
+import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
+import org.apache.geode.internal.cache.versions.VersionSource;
+import org.apache.geode.internal.cache.versions.VersionStamp;
+import org.apache.geode.internal.cache.versions.VersionTag;
+
+class LRUTestEntry implements EvictableEntry {
+
+  private int id;
+  private EvictionNode next;
+  private EvictionNode prev;
+  private int size;
+  private boolean recentlyUsed;
+  private boolean evicted;
+
+  public LRUTestEntry(int id) {
+    this.id = id;
+    next = null;
+    prev = null;
+    size = 0;
+    recentlyUsed = false;
+    evicted = false;
+  }
+
+  public int id() {
+    return id;
+  }
+
+  public boolean isTombstone() {
+    return false;
+  }
+
+  @Override
+  public boolean fillInValue(final InternalRegion region, final InitialImageOperation.Entry entry,
+      final ByteArrayDataInput in, final DM distributionManager, final Version version) {
+    return false;
+  }
+
+  @Override
+  public boolean isOverflowedToDisk(final InternalRegion region,
+      final DistributedRegion.DiskPosition diskPosition) {
+    return false;
+  }
+
+  @Override
+  public Object getKey() {
+    return null;
+  }
+
+  @Override
+  public Object getValue(final RegionEntryContext context) {
+    return null;
+  }
+
+  @Override
+  public Object getValueRetain(final RegionEntryContext context) {
+    return null;
+  }
+
+  @Override
+  public void setValue(final RegionEntryContext context, final Object value)
+      throws RegionClearedException {
+
+  }
+
+  @Override
+  public void setValue(final RegionEntryContext context, final Object value,
+      final EntryEventImpl event) throws RegionClearedException {
+
+  }
+
+  @Override
+  public Object getValueRetain(final RegionEntryContext context, final boolean decompress) {
+    return null;
+  }
+
+  @Override
+  public Object getValue() {
+    return null;
+  }
+
+  @Override
+  public Token getValueAsToken() {
+    return null;
+  }
+
+  @Override
+  public void setValueWithTombstoneCheck(final Object value, final EntryEvent event)
+      throws RegionClearedException {
+
+  }
+
+  @Override
+  public Object getTransformedValue() {
+    return null;
+  }
+
+  @Override
+  public Object getValueInVM(final RegionEntryContext context) {
+    return null;
+  }
+
+  @Override
+  public Object getValueOnDisk(final InternalRegion region) throws EntryNotFoundException {
+    return null;
+  }
+
+  @Override
+  public Object getValueOnDiskOrBuffer(final InternalRegion region) throws EntryNotFoundException {
+    return null;
+  }
+
+  @Override
+  public boolean initialImagePut(final InternalRegion region, final long lastModified,
+      final Object newValue, final boolean wasRecovered, final boolean acceptedVersionTag)
+      throws RegionClearedException {
+    return false;
+  }
+
+  @Override
+  public boolean initialImageInit(final InternalRegion region, final long lastModified,
+      final Object newValue, final boolean create, final boolean wasRecovered,
+      final boolean acceptedVersionTag) throws RegionClearedException {
+    return false;
+  }
+
+  @Override
+  public boolean destroy(final InternalRegion region, final EntryEventImpl event,
+      final boolean inTokenMode, final boolean cacheWrite, final Object expectedOldValue,
+      final boolean forceDestroy, final boolean removeRecoveredEntry) throws CacheWriterException,
+      EntryNotFoundException, TimeoutException, RegionClearedException {
+    return false;
+  }
+
+  @Override
+  public boolean getValueWasResultOfSearch() {
+    return false;
+  }
+
+  @Override
+  public void setValueResultOfSearch(final boolean value) {
+
+  }
+
+  @Override
+  public Object getSerializedValueOnDisk(final InternalRegion region) {
+    return null;
+  }
+
+  @Override
+  public Object getValueInVMOrDiskWithoutFaultIn(final InternalRegion region) {
+    return null;
+  }
+
+  @Override
+  public Object getValueOffHeapOrDiskWithoutFaultIn(final InternalRegion region) {
+    return null;
+  }
+
+  @Override
+  public boolean isUpdateInProgress() {
+    return false;
+  }
+
+  @Override
+  public void setUpdateInProgress(final boolean underUpdate) {
+
+  }
+
+  @Override
+  public boolean isCacheListenerInvocationInProgress() {
+    return false;
+  }
+
+  @Override
+  public void setCacheListenerInvocationInProgress(final boolean isListenerInvoked) {
+
+  }
+
+  @Override
+  public boolean isValueNull() {
+    return false;
+  }
+
+  @Override
+  public boolean isInvalid() {
+    return false;
+  }
+
+  @Override
+  public boolean isDestroyed() {
+    return false;
+  }
+
+  @Override
+  public boolean isDestroyedOrRemoved() {
+    return false;
+  }
+
+  @Override
+  public boolean isDestroyedOrRemovedButNotTombstone() {
+    return false;
+  }
+
+  @Override
+  public boolean isInvalidOrRemoved() {
+    return false;
+  }
+
+  @Override
+  public void setValueToNull() {
+
+  }
+
+  @Override
+  public void returnToPool() {
+
+  }
+
+  @Override
+  public void setNext(EvictionNode next) {
+    this.next = next;
+  }
+
+  @Override
+  public EvictionNode next() {
+    return this.next;
+  }
+
+  @Override
+  public void setPrevious(EvictionNode previous) {
+    this.prev = previous;
+  }
+
+  @Override
+  public EvictionNode previous() {
+    return this.prev;
+  }
+
+  @Override
+  public int updateEntrySize(EvictionController cc) {
+    return this.size = 1;
+  }
+
+  @Override
+  public int updateEntrySize(EvictionController cc, Object value) {
+    return this.size = 1;
+  }
+
+  @Override
+  public int getEntrySize() {
+    return this.size;
+  }
+
+  /** this should only happen with the LRUClockHand sync'ed */
+  @Override
+  public void setEvicted() {
+    evicted = true;
+  }
+
+  @Override
+  public void unsetEvicted() {
+    evicted = false;
+  }
+
+  @Override
+  public boolean isEvicted() {
+    return evicted;
+  }
+
+  @Override
+  public boolean isRecentlyUsed() {
+    return recentlyUsed;
+  }
+
+  @Override
+  public void setRecentlyUsed(final RegionEntryContext context) {
+    recentlyUsed = true;
+    context.incRecentlyUsed();
+  }
+
+  @Override
+  public long getLastModified() {
+    return 0;
+  }
+
+  @Override
+  public boolean hasStats() {
+    return false;
+  }
+
+  @Override
+  public long getLastAccessed() throws InternalStatisticsDisabledException {
+    return 0;
+  }
+
+  @Override
+  public long getHitCount() throws InternalStatisticsDisabledException {
+    return 0;
+  }
+
+  @Override
+  public long getMissCount() throws InternalStatisticsDisabledException {
+    return 0;
+  }
+
+  @Override
+  public void updateStatsForPut(final long lastModifiedTime, final long lastAccessedTime) {
+
+  }
+
+  @Override
+  public VersionStamp getVersionStamp() {
+    return null;
+  }
+
+  @Override
+  public VersionTag generateVersionTag(final VersionSource member, final boolean withDelta,
+      final InternalRegion region, final EntryEventImpl event) {
+    return null;
+  }
+
+  @Override
+  public boolean dispatchListenerEvents(final EntryEventImpl event) throws InterruptedException {
+    return false;
+  }
+
+  @Override
+  public void updateStatsForGet(final boolean hit, final long time) {
+
+  }
+
+  @Override
+  public void txDidDestroy(final long currentTime) {
+
+  }
+
+  @Override
+  public void resetCounts() throws InternalStatisticsDisabledException {
+
+  }
+
+  @Override
+  public void makeTombstone(final InternalRegion region, final VersionTag version)
+      throws RegionClearedException {
+
+  }
+
+  @Override
+  public void removePhase1(final InternalRegion region, final boolean clear)
+      throws RegionClearedException {
+
+  }
+
+  @Override
+  public void removePhase2() {
+
+  }
+
+  @Override
+  public boolean isRemoved() {
+    return false;
+  }
+
+  @Override
+  public boolean isRemovedPhase2() {
+    return false;
+  }
+
+  @Override
+  public void unsetRecentlyUsed() {
+    recentlyUsed = false;
+  }
+
+  public EvictionNode absoluteSelf() {
+    return this;
+  }
+
+  public EvictionNode clearClones() {
+    return this;
+  }
+
+  public int cloneCount() {
+    return 0;
+  }
+
+  @Override
+  public boolean isInUseByTransaction() {
+    return false;
+  }
+
+  @Override
+  public void incRefCount() {
+
+  }
+
+  @Override
+  public void decRefCount(final EvictionList lruList, final InternalRegion region) {
+
+  }
+
+  @Override
+  public void resetRefCount(final EvictionList lruList) {
+
+  }
+
+  @Override
+  public Object prepareValueForCache(final RegionEntryContext context, final Object value,
+      final boolean isEntryUpdate) {
+    return null;
+  }
+
+  @Override
+  public Object prepareValueForCache(final RegionEntryContext context, final Object value,
+      final EntryEventImpl event, final boolean isEntryUpdate) {
+    return null;
+  }
+
+  @Override
+  public Object getKeyForSizing() {
+    return null;
+  }
+
+  @Override
+  public void setDelayedDiskId(final DiskRecoveryStore diskRecoveryStore) {
+
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LinkableEvictionNode.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LinkableEvictionNode.java
new file mode 100644
index 0000000000..71a3f1396e
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/LinkableEvictionNode.java
@@ -0,0 +1,19 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+public class LinkableEvictionNode extends GuardNode {
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/OffHeapEvictionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/OffHeapEvictionDUnitTest.java
similarity index 87%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/OffHeapEvictionDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/eviction/OffHeapEvictionDUnitTest.java
index 8971c86874..0e2fae5c35 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/OffHeapEvictionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/OffHeapEvictionDUnitTest.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.eviction;
 
 import static org.apache.geode.distributed.ConfigurationProperties.*;
 import static org.junit.Assert.*;
@@ -25,8 +25,13 @@
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache30.CacheSerializableRunnable;
 import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.internal.cache.AbstractLRURegionMap;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.OffHeapTestUtil;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
-import org.apache.geode.internal.cache.lru.HeapEvictor;
+import org.apache.geode.internal.cache.eviction.EvictionDUnitTest;
+import org.apache.geode.internal.cache.eviction.HeapEvictor;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.LogWriterUtils;
@@ -96,7 +101,7 @@ public void raiseFakeNotification(VM vm, final String prName, final int noOfExpe
       @Override
       public void run2() throws CacheException {
         final LocalRegion region = (LocalRegion) cache.getRegion(prName);
-        getEvictor().testAbortAfterLoopCount = 1;
+        getEvictor().setTestAbortAfterLoopCount(1);
 
         ((GemFireCacheImpl) cache).getInternalResourceManager().getOffHeapMonitor()
             .updateStateAndSendEvent(188743680);
@@ -104,8 +109,8 @@ public void run2() throws CacheException {
         WaitCriterion wc = new WaitCriterion() {
           public boolean done() {
             // we have a primary
-            final long currentEvictions =
-                ((AbstractLRURegionMap) region.entries)._getLruList().stats().getEvictions();
+            final long currentEvictions = ((AbstractLRURegionMap) region.entries).getEvictionList()
+                .getStatistics().getEvictions();
             if (Math.abs(currentEvictions - noOfExpectedEvictions) <= 1) { // Margin of error is 1
               return true;
             } else if (currentEvictions > noOfExpectedEvictions) {
@@ -116,7 +121,8 @@ public boolean done() {
 
           public String description() {
             return "expected " + noOfExpectedEvictions + " evictions, but got "
-                + ((AbstractLRURegionMap) region.entries)._getLruList().stats().getEvictions();
+                + ((AbstractLRURegionMap) region.entries).getEvictionList().getStatistics()
+                    .getEvictions();
           }
         };
         Wait.waitForCriterion(wc, 60000, 1000, true);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/OffHeapEvictionStatsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/OffHeapEvictionStatsDUnitTest.java
similarity index 94%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/OffHeapEvictionStatsDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/eviction/OffHeapEvictionStatsDUnitTest.java
index abff35ac29..3ff316427d 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/OffHeapEvictionStatsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/OffHeapEvictionStatsDUnitTest.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.eviction;
 
 import static org.apache.geode.distributed.ConfigurationProperties.*;
 import static org.junit.Assert.*;
@@ -23,6 +23,8 @@
 
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.internal.cache.OffHeapTestUtil;
+import org.apache.geode.internal.cache.eviction.EvictionStatsDUnitTest;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.LogWriterUtils;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/TestEvictionController.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/TestEvictionController.java
new file mode 100644
index 0000000000..82c6171937
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/TestEvictionController.java
@@ -0,0 +1,177 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import org.apache.geode.StatisticDescriptor;
+import org.apache.geode.StatisticsFactory;
+import org.apache.geode.StatisticsType;
+import org.apache.geode.StatisticsTypeFactory;
+import org.apache.geode.cache.EvictionAction;
+import org.apache.geode.cache.EvictionAlgorithm;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.InternalRegion;
+import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
+import org.apache.geode.internal.cache.persistence.DiskRegionView;
+import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
+
+class TestEvictionController implements EvictionController {
+
+  private final StatisticsType statType;
+
+  {
+    // create the stats type for MemLRU.
+    StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+
+    final String bytesAllowedDesc = "Number of total bytes allowed in this region.";
+    final String byteCountDesc = "Number of bytes in region.";
+    final String lruEvictionsDesc = "Number of total entry evictions triggered by LRU.";
+    final String lruEvaluationsDesc = "Number of entries evaluated during LRU operations.";
+    final String lruGreedyReturnsDesc = "Number of non-LRU entries evicted during LRU operations";
+    final String lruDestroysDesc = "Number of entry destroys triggered by LRU.";
+    final String lruDestroysLimitDesc =
+        "Maximum number of entry destroys triggered by LRU before scan occurs.";
+
+    statType = f.createType("TestLRUStatistics",
+        "Statistics about byte based Least Recently Used region entry disposal",
+        new StatisticDescriptor[] {f.createLongGauge("bytesAllowed", bytesAllowedDesc, "bytes"),
+            f.createLongGauge("byteCount", byteCountDesc, "bytes"),
+            f.createLongCounter("lruEvictions", lruEvictionsDesc, "entries"),
+            f.createLongCounter("lruEvaluations", lruEvaluationsDesc, "entries"),
+            f.createLongCounter("lruGreedyReturns", lruGreedyReturnsDesc, "entries"),
+            f.createLongCounter("lruDestroys", lruDestroysDesc, "entries"),
+            f.createLongCounter("lruDestroysLimit", lruDestroysLimitDesc, "entries"),});
+  }
+
+  @Override
+  public int entrySize(Object key, Object value) throws IllegalArgumentException {
+    return 1;
+  }
+
+  @Override
+  public long limit() {
+    return 20;
+  }
+
+  public boolean usesMem() {
+    return false;
+  }
+
+  @Override
+  public EvictionAlgorithm getEvictionAlgorithm() {
+    return EvictionAlgorithm.LRU_ENTRY;
+  }
+
+  @Override
+  public EvictionStatistics getStatistics() {
+    return null;
+  }
+
+  @Override
+  public EvictionAction getEvictionAction() {
+    return EvictionAction.DEFAULT_EVICTION_ACTION;
+  }
+
+  @Override
+  public StatisticsType getStatisticsType() {
+    return statType;
+  }
+
+  @Override
+  public String getStatisticsName() {
+    return "TestLRUStatistics";
+  }
+
+  @Override
+  public int getLimitStatId() {
+    return statType.nameToId("bytesAllowed");
+  }
+
+  @Override
+  public int getCountStatId() {
+    return statType.nameToId("byteCount");
+  }
+
+  @Override
+  public int getEvictionsStatId() {
+    return statType.nameToId("lruEvictions");
+  }
+
+  @Override
+  public int getDestroysStatId() {
+    return statType.nameToId("lruDestroys");
+  }
+
+  @Override
+  public int getDestroysLimitStatId() {
+    return statType.nameToId("lruDestroysLimit");
+  }
+
+  @Override
+  public int getEvaluationsStatId() {
+    return statType.nameToId("lruEvaluations");
+  }
+
+  @Override
+  public int getGreedyReturnsStatId() {
+    return statType.nameToId("lruGreedyReturns");
+  }
+
+  @Override
+  public boolean mustEvict(EvictionStatistics stats, InternalRegion region, int delta) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  @Override
+  public EvictionStatistics initStats(Object region, StatisticsFactory statsFactory) {
+    String regionName;
+    if (region instanceof Region) {
+      regionName = ((Region) region).getName();
+    } else if (region instanceof PlaceHolderDiskRegion) {
+      regionName = ((PlaceHolderDiskRegion) region).getName();
+      // @todo make it shorter (I think it is the fullPath
+    } else {
+      throw new IllegalStateException("expected Region or PlaceHolderDiskRegion");
+    }
+    final InternalEvictionStatistics stats =
+        new EvictionStatisticsImpl(statsFactory, "TestLRUStatistics" + regionName, this);
+    stats.setLimit(limit());
+    return stats;
+  }
+
+  @Override
+  public boolean lruLimitExceeded(EvictionStatistics stats, DiskRegionView diskRegionView) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  @Override
+  public void setBucketRegion(Region region) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  @Override
+  public long getLimit() {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  @Override
+  public void setLimit(int maximum) {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+
+  @Override
+  public void close() {
+    throw new UnsupportedOperationException("Not implemented");
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/TestLRUListWithAsyncSorting.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/TestLRUListWithAsyncSorting.java
new file mode 100644
index 0000000000..99ad9b4161
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/TestLRUListWithAsyncSorting.java
@@ -0,0 +1,43 @@
+/*
+ * 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.geode.internal.cache.eviction;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.LRUClearWithDiskRegionOpRegressionTest;
+
+/**
+ * Test Implementation class of LRUListWithAsyncSorting for
+ * {@link LRUClearWithDiskRegionOpRegressionTest}.
+ */
+public class TestLRUListWithAsyncSorting extends LRUListWithAsyncSorting {
+
+  public TestLRUListWithAsyncSorting(EvictionStatistics stats, BucketRegion region) {
+    this((InternalEvictionStatistics) stats, region, Executors.newSingleThreadExecutor());
+  }
+
+  public TestLRUListWithAsyncSorting(EvictionStatistics stats, BucketRegion region,
+      ExecutorService executor) {
+    this((InternalEvictionStatistics) stats, region, executor);
+  }
+
+  TestLRUListWithAsyncSorting(InternalEvictionStatistics stats, BucketRegion region,
+      ExecutorService executor) {
+    super(stats, region, executor);
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/lru/TransactionsWithOverflowTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/TransactionsWithOverflowTest.java
similarity index 98%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/lru/TransactionsWithOverflowTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/eviction/TransactionsWithOverflowTest.java
index ffdf5d2bd2..37e27355fa 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/lru/TransactionsWithOverflowTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/eviction/TransactionsWithOverflowTest.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.cache.lru;
+package org.apache.geode.internal.cache.eviction;
 
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/LocalDataSetIndexingDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/LocalDataSetIndexingDUnitTest.java
index b2b0ff9b37..c6d626eed1 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/LocalDataSetIndexingDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/LocalDataSetIndexingDUnitTest.java
@@ -14,9 +14,6 @@
  */
 package org.apache.geode.internal.cache.execute;
 
-import static org.junit.Assert.*;
-
-import java.io.File;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -30,51 +27,30 @@
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.AttributesFactory;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.DiskStore;
-import org.apache.geode.cache.DiskStoreFactory;
-import org.apache.geode.cache.EntryOperation;
-import org.apache.geode.cache.EvictionAction;
-import org.apache.geode.cache.EvictionAttributes;
-import org.apache.geode.cache.PartitionAttributes;
 import org.apache.geode.cache.PartitionAttributesFactory;
-import org.apache.geode.cache.PartitionResolver;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionAdapter;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.RegionFunctionContext;
 import org.apache.geode.cache.partition.PartitionRegionHelper;
-import org.apache.geode.cache.query.CacheUtils;
-import org.apache.geode.cache.query.FunctionDomainException;
 import org.apache.geode.cache.query.Index;
 import org.apache.geode.cache.query.IndexType;
-import org.apache.geode.cache.query.NameResolutionException;
-import org.apache.geode.cache.query.QueryInvocationTargetException;
 import org.apache.geode.cache.query.QueryService;
 import org.apache.geode.cache.query.SelectResults;
-import org.apache.geode.cache.query.TypeMismatchException;
 import org.apache.geode.cache.query.internal.DefaultQuery;
 import org.apache.geode.cache.query.internal.QueryObserverAdapter;
 import org.apache.geode.cache.query.internal.QueryObserverHolder;
 import org.apache.geode.cache30.CacheSerializableRunnable;
-import org.apache.geode.cache30.CacheTestCase;
 import org.apache.geode.distributed.ConfigurationProperties;
-import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.Assert;
-import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.cache.BucketRegion;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalDataSet;
 import org.apache.geode.internal.cache.PartitionedRegion;
-import org.apache.geode.internal.cache.functions.LocalDataSetFunction;
-import org.apache.geode.internal.cache.lru.HeapLRUCapacityController;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 /**
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/lru/LRUClockJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/lru/LRUClockJUnitTest.java
deleted file mode 100755
index 2317361a3a..0000000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/lru/LRUClockJUnitTest.java
+++ /dev/null
@@ -1,516 +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.geode.internal.cache.lru;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-import java.util.Properties;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.StatisticsFactory;
-import org.apache.geode.StatisticsType;
-import org.apache.geode.StatisticsTypeFactory;
-import org.apache.geode.cache.AttributesFactory;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheExistsException;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.EvictionAction;
-import org.apache.geode.cache.EvictionAlgorithm;
-import org.apache.geode.cache.Region;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.internal.cache.InternalRegionArguments;
-import org.apache.geode.internal.cache.PlaceHolderDiskRegion;
-import org.apache.geode.internal.cache.persistence.DiskRegionView;
-import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-
-/**
- * This class tests the LRUCapacityController's core clock algorithm.
- */
-@Category(IntegrationTest.class)
-public class LRUClockJUnitTest {
-
-  private String myTestName;
-
-  static Properties sysProps = new Properties();
-
-  @Rule
-  public TestName testName = new TestName();
-
-  @Before
-  public void setUp() throws Exception {
-    sysProps = new Properties();
-    sysProps.setProperty(MCAST_PORT, "0");
-    sysProps.setProperty(LOCATORS, "");
-  }
-
-  @Test
-  public void testAddToClockFace() throws Exception {
-    NewLRUClockHand clock = getAClockHand(getARegion(), new TestEnableLRU());
-
-    // getLRUEntry( maxScan )
-
-    LRUTestEntry[] nodes = new LRUTestEntry[10];
-    int i = 0;
-    for (i = 0; i < 10; i++) {
-      nodes[i] = getANode(i);
-      clock.appendEntry(nodes[i]);
-    }
-
-    // getLRUEntry until empty... verify order of results.
-
-    for (i = 0; i < 10; i++) {
-      LRUTestEntry n = (LRUTestEntry) clock.getLRUEntry();
-      assertTrue("expected nodes[" + nodes[i].id() + "], found nodes[" + n.id() + "]",
-          n == nodes[i]);
-    }
-
-    assertTrue("expected null", clock.getLRUEntry() == null);
-  }
-
-  @Test
-  public void testFIFO() throws Exception {
-    NewLRUClockHand clock = getAClockHand(getARegion(), new TestEnableLRU());
-
-    for (int i = 0; i < 100; i++) {
-      LRUClockNode entry = getANode(i);
-      clock.appendEntry(entry);
-    }
-
-    for (int i = 100; i < 2000; i++) {
-      LRUClockNode entry = getANode(i);
-      clock.appendEntry(entry);
-      Object obj = clock.getLRUEntry();
-      if (obj instanceof LRUTestEntry) {
-        LRUTestEntry le = (LRUTestEntry) obj;
-        le.setEvicted();
-      } else {
-        assertTrue("found wrong type: " + obj.getClass().getName(), false);
-      }
-
-    }
-
-    int counter = 0;
-    LRUTestEntry le = (LRUTestEntry) clock.getLRUEntry();
-    while (le != null) {
-      counter++;
-      le = (LRUTestEntry) clock.getLRUEntry();
-    }
-    assertTrue("expected 100, found " + counter, counter == 100);
-  }
-
-  @Test
-  public void testEvicted() throws Exception {
-    NewLRUClockHand clock = getAClockHand(getARegion(), new TestEnableLRU());
-
-    // getLRUEntry( maxScan )
-
-    LRUTestEntry[] nodes = new LRUTestEntry[10];
-    int i = 0;
-    for (i = 0; i < 10; i++) {
-      nodes[i] = getANode(i);
-      clock.appendEntry(nodes[i]);
-    }
-
-    for (i = 0; i < 10; i += 2) {
-      clock.unlinkEntry(nodes[i]);
-    }
-
-    // getLRUEntry until empty... verify order of results.
-
-    for (i = 1; i < 10; i += 2) {
-      LRUTestEntry n = (LRUTestEntry) clock.getLRUEntry();
-      assertTrue("expected nodes[" + nodes[i].id() + "], found nodes[" + n.id() + "]",
-          n == nodes[i]);
-    }
-
-    assertTrue("expected null", clock.getLRUEntry() == null);
-  }
-
-  @Test
-  public void testRecentlyUsed() throws Exception {
-    NewLRUClockHand clock = getAClockHand(getARegion(), new TestEnableLRU());
-
-    // getLRUEntry( maxScan )
-
-    LRUTestEntry[] nodes = new LRUTestEntry[10];
-    int i = 0;
-    for (i = 0; i < 10; i++) {
-      nodes[i] = getANode(i);
-      clock.appendEntry(nodes[i]);
-      if (i % 2 == 0) {
-        nodes[i].setRecentlyUsed();
-      }
-    }
-
-    // getLRUEntry until empty... verify order of results.
-
-    // should find 1, 3, etc... as 0, 2, 4 etc... were marked recently used..
-    for (i = 1; i < 10; i += 2) {
-      LRUTestEntry n = (LRUTestEntry) clock.getLRUEntry();
-      assertTrue("expected nodes[" + nodes[i].id() + "], found nodes[" + n.id() + "]",
-          n == nodes[i]);
-    }
-
-    // now 0, 2, 4 should go...
-    for (i = 0; i < 10; i += 2) {
-      LRUTestEntry n = (LRUTestEntry) clock.getLRUEntry();
-      assertTrue("expected nodes[" + nodes[i].id() + "], found nodes[" + n.id() + "]",
-          n == nodes[i]);
-    }
-
-    assertTrue("expected null", clock.getLRUEntry() == null);
-  }
-
-  @Test
-  public void testRemoveHead() throws Exception {
-    NewLRUClockHand clock = getAClockHand(getARegion(), new TestEnableLRU());
-    LRUTestEntry[] nodes = new LRUTestEntry[10];
-    int i = 0;
-    for (i = 0; i < 10; i++) {
-      nodes[i] = getANode(i);
-      clock.appendEntry(nodes[i]);
-    }
-    clock.unlinkEntry(nodes[0]);
-    for (i = 1; i < 10; i++) {
-      LRUTestEntry n = (LRUTestEntry) clock.getLRUEntry();
-      assertTrue("expected nodes[" + nodes[i].id() + "], found nodes[" + n.id() + "]",
-          n == nodes[i]);
-    }
-    assertEquals(null, clock.getLRUEntry());
-  }
-
-  @Test
-  public void testRemoveMiddle() throws Exception {
-    NewLRUClockHand clock = getAClockHand(getARegion(), new TestEnableLRU());
-    LRUTestEntry[] nodes = new LRUTestEntry[10];
-    int i = 0;
-    for (i = 0; i < 10; i++) {
-      nodes[i] = getANode(i);
-      clock.appendEntry(nodes[i]);
-    }
-    clock.unlinkEntry(nodes[5]);
-    for (i = 0; i < 5; i++) {
-      LRUTestEntry n = (LRUTestEntry) clock.getLRUEntry();
-      assertTrue("expected nodes[" + nodes[i].id() + "], found nodes[" + n.id() + "]",
-          n == nodes[i]);
-    }
-    for (i = 6; i < 10; i++) {
-      LRUTestEntry n = (LRUTestEntry) clock.getLRUEntry();
-      assertTrue("expected nodes[" + nodes[i].id() + "], found nodes[" + n.id() + "]",
-          n == nodes[i]);
-    }
-    assertEquals(null, clock.getLRUEntry());
-  }
-
-  @Test
-  public void testRemoveTail() throws Exception {
-    NewLRUClockHand clock = getAClockHand(getARegion(), new TestEnableLRU());
-    LRUTestEntry[] nodes = new LRUTestEntry[10];
-    int i = 0;
-    for (i = 0; i < 10; i++) {
-      nodes[i] = getANode(i);
-      clock.appendEntry(nodes[i]);
-    }
-    clock.unlinkEntry(nodes[9]);
-
-    for (i = 0; i < 9; i++) {
-      LRUTestEntry n = (LRUTestEntry) clock.getLRUEntry();
-      assertTrue("expected nodes[" + nodes[i].id() + "], found nodes[" + n.id() + "]",
-          n == nodes[i]);
-    }
-    assertEquals(null, clock.getLRUEntry());
-  }
-
-  /** manufacture a node so that a shared type can be used by SharedLRUClockTest. */
-  private LRUTestEntry getANode(int id) {
-    return new LocalLRUTestEntry(id);
-  }
-
-  private interface LRUTestEntry extends LRUClockNode {
-    public int id();
-  }
-
-  /** test implementation of an LRUClockNode */
-  private static class LocalLRUTestEntry implements LRUTestEntry {
-
-    int id;
-    LRUClockNode next;
-    LRUClockNode prev;
-    int size;
-    boolean recentlyUsed;
-    boolean evicted;
-
-    public LocalLRUTestEntry(int id) {
-      this.id = id;
-      next = null;
-      prev = null;
-      size = 0;
-      recentlyUsed = false;
-      evicted = false;
-    }
-
-    @Override
-    public int id() {
-      return id;
-    }
-
-    public boolean isTombstone() {
-      return false;
-    }
-
-    @Override
-    public void setNextLRUNode(LRUClockNode next) {
-      this.next = next;
-    }
-
-    @Override
-    public LRUClockNode nextLRUNode() {
-      return this.next;
-    }
-
-    @Override
-    public void setPrevLRUNode(LRUClockNode prev) {
-      this.prev = prev;
-    }
-
-    @Override
-    public LRUClockNode prevLRUNode() {
-      return this.prev;
-    }
-
-    @Override
-    public int updateEntrySize(EnableLRU cc) {
-      return this.size = 1;
-    }
-
-    @Override
-    public int updateEntrySize(EnableLRU cc, Object value) {
-      return this.size = 1;
-    }
-
-    @Override
-    public int getEntrySize() {
-      return this.size;
-    }
-
-    /** this should only happen with the LRUClockHand sync'ed */
-    @Override
-    public void setEvicted() {
-      evicted = true;
-    }
-
-    @Override
-    public void unsetEvicted() {
-      evicted = false;
-    }
-
-    @Override
-    public boolean testEvicted() {
-      return evicted;
-    }
-
-    @Override
-    public boolean testRecentlyUsed() {
-      return recentlyUsed;
-    }
-
-    @Override
-    public void setRecentlyUsed() {
-      recentlyUsed = true;
-    }
-
-    @Override
-    public void unsetRecentlyUsed() {
-      recentlyUsed = false;
-    }
-
-    public LRUClockNode absoluteSelf() {
-      return this;
-    }
-
-    public LRUClockNode clearClones() {
-      return this;
-    }
-
-    public int cloneCount() {
-      return 0;
-    }
-  }
-
-  private class TestEnableLRU implements EnableLRU {
-
-    private final StatisticsType statType;
-
-    {
-      // create the stats type for MemLRU.
-      StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
-
-      final String bytesAllowedDesc = "Number of total bytes allowed in this region.";
-      final String byteCountDesc = "Number of bytes in region.";
-      final String lruEvictionsDesc = "Number of total entry evictions triggered by LRU.";
-      final String lruEvaluationsDesc = "Number of entries evaluated during LRU operations.";
-      final String lruGreedyReturnsDesc = "Number of non-LRU entries evicted during LRU operations";
-      final String lruDestroysDesc = "Number of entry destroys triggered by LRU.";
-      final String lruDestroysLimitDesc =
-          "Maximum number of entry destroys triggered by LRU before scan occurs.";
-
-      statType = f.createType("TestLRUStatistics",
-          "Statistics about byte based Least Recently Used region entry disposal",
-          new StatisticDescriptor[] {f.createLongGauge("bytesAllowed", bytesAllowedDesc, "bytes"),
-              f.createLongGauge("byteCount", byteCountDesc, "bytes"),
-              f.createLongCounter("lruEvictions", lruEvictionsDesc, "entries"),
-              f.createLongCounter("lruEvaluations", lruEvaluationsDesc, "entries"),
-              f.createLongCounter("lruGreedyReturns", lruGreedyReturnsDesc, "entries"),
-              f.createLongCounter("lruDestroys", lruDestroysDesc, "entries"),
-              f.createLongCounter("lruDestroysLimit", lruDestroysLimitDesc, "entries"),});
-    }
-
-    @Override
-    public int entrySize(Object key, Object value) throws IllegalArgumentException {
-      return 1;
-    }
-
-    @Override
-    public long limit() {
-      return 20;
-    }
-
-    public boolean usesMem() {
-      return false;
-    }
-
-    @Override
-    public EvictionAlgorithm getEvictionAlgorithm() {
-      return EvictionAlgorithm.LRU_ENTRY;
-    }
-
-    @Override
-    public LRUStatistics getStats() {
-      return null;
-    }
-
-    @Override
-    public EvictionAction getEvictionAction() {
-      return EvictionAction.DEFAULT_EVICTION_ACTION;
-    }
-
-    @Override
-    public StatisticsType getStatisticsType() {
-      return statType;
-    }
-
-    @Override
-    public String getStatisticsName() {
-      return "TestLRUStatistics";
-    }
-
-    @Override
-    public int getLimitStatId() {
-      return statType.nameToId("bytesAllowed");
-    }
-
-    @Override
-    public int getCountStatId() {
-      return statType.nameToId("byteCount");
-    }
-
-    @Override
-    public int getEvictionsStatId() {
-      return statType.nameToId("lruEvictions");
-    }
-
-    @Override
-    public int getDestroysStatId() {
-      return statType.nameToId("lruDestroys");
-    }
-
-    @Override
-    public int getDestroysLimitStatId() {
-      return statType.nameToId("lruDestroysLimit");
-    }
-
-    @Override
-    public int getEvaluationsStatId() {
-      return statType.nameToId("lruEvaluations");
-    }
-
-    @Override
-    public int getGreedyReturnsStatId() {
-      return statType.nameToId("lruGreedyReturns");
-    }
-
-    @Override
-    public boolean mustEvict(LRUStatistics stats, Region region, int delta) {
-      throw new UnsupportedOperationException("Not implemented");
-    }
-
-    @Override
-    public void afterEviction() {
-      throw new UnsupportedOperationException("Not implemented");
-    }
-
-    @Override
-    public LRUStatistics initStats(Object region, StatisticsFactory sf) {
-      String regionName;
-      if (region instanceof Region) {
-        regionName = ((Region) region).getName();
-      } else if (region instanceof PlaceHolderDiskRegion) {
-        regionName = ((PlaceHolderDiskRegion) region).getName();
-        // @todo make it shorter (I think it is the fullPath
-      } else {
-        throw new IllegalStateException("expected Region or PlaceHolderDiskRegion");
-      }
-      final LRUStatistics stats = new LRUStatistics(sf, "TestLRUStatistics" + regionName, this);
-      stats.setLimit(limit());
-      return stats;
-    }
-
-    @Override
-    public boolean lruLimitExceeded(LRUStatistics lruStatistics, DiskRegionView drv) {
-      throw new UnsupportedOperationException("Not implemented");
-    }
-  }
-
-  /** overridden in SharedLRUClockTest to test SharedLRUClockHand */
-  private NewLRUClockHand getAClockHand(Region reg, EnableLRU elru) {
-    return new NewLRUClockHand(reg, elru, new InternalRegionArguments());
-  }
-
-  private Region getARegion() throws Exception {
-    DistributedSystem ds = DistributedSystem.connect(sysProps);
-    Cache c = null;
-    try {
-      c = CacheFactory.create(ds);
-    } catch (CacheExistsException cee) {
-      c = CacheFactory.getInstance(ds);
-    }
-    AttributesFactory af = new AttributesFactory();
-    Region root = c.getRegion("root");
-    if (root == null) {
-      root = c.createRegion("root", af.create());
-    }
-    Region sub = root.createSubregion(testName.getMethodName(), af.create());
-    return sub;
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/AsyncEventQueueTestBase.java b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/AsyncEventQueueTestBase.java
index 47dbba10f8..50c5a97ec1 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/AsyncEventQueueTestBase.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/AsyncEventQueueTestBase.java
@@ -90,7 +90,7 @@
 import org.apache.geode.internal.cache.RegionQueue;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceObserver;
-import org.apache.geode.internal.cache.lru.Sizeable;
+import org.apache.geode.internal.size.Sizeable;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.IgnoredException;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
index 5b0a528cca..534b8aa3b6 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
@@ -14,8 +14,14 @@
  */
 package org.apache.geode.internal.cache.wan.parallel;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -59,7 +65,7 @@
 import org.apache.geode.internal.cache.PartitionedRegionStats;
 import org.apache.geode.internal.cache.ProxyBucketRegion;
 import org.apache.geode.internal.cache.RegionQueue;
-import org.apache.geode.internal.cache.lru.LRUAlgorithm;
+import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.partitioned.RegionAdvisor;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
 import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl;
@@ -104,9 +110,9 @@ private void createQueueRegion() {
     when(this.queueRegion.getCache()).thenReturn(this.cache);
     EvictionAttributesImpl ea = (EvictionAttributesImpl) EvictionAttributes
         .createLRUMemoryAttributes(100, null, EvictionAction.OVERFLOW_TO_DISK);
-    LRUAlgorithm algorithm = ea.createEvictionController(this.queueRegion, false);
-    algorithm.getLRUHelper().initStats(this.queueRegion, this.cache.getDistributedSystem());
-    when(this.queueRegion.getEvictionController()).thenReturn(algorithm);
+    EvictionController eviction = ea.createEvictionController(this.queueRegion, false);
+    eviction.initStats(this.queueRegion, this.cache.getDistributedSystem());
+    when(this.queueRegion.getEvictionController()).thenReturn(eviction);
   }
 
   private void createGatewaySender() {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/lang/SystemPropertyHelperJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/lang/SystemPropertyHelperJUnitTest.java
index 40fc74fb32..71df1dbe11 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/lang/SystemPropertyHelperJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/lang/SystemPropertyHelperJUnitTest.java
@@ -14,7 +14,9 @@
  */
 package org.apache.geode.internal.lang;
 
-import static org.junit.Assert.*;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -23,7 +25,7 @@
 
 @Category(UnitTest.class)
 public class SystemPropertyHelperJUnitTest {
-  String restoreSetOperationTransactionBehavior = "restoreSetOperationTransactionBehavior";
+  private String restoreSetOperationTransactionBehavior = "restoreSetOperationTransactionBehavior";
 
   @Test
   public void testRestoreSetOperationTransactionBehaviorDefaultToFalse() {
@@ -54,4 +56,30 @@ public void testRestoreSetOperationTransactionBehaviorGeodePreference() {
     System.clearProperty(gemfirePrefixProperty);
   }
 
+  @Test
+  public void getIntegerPropertyPrefersGeodePrefix() {
+    String testProperty = "testIntegerProperty";
+    String gemfirePrefixProperty = "gemfire." + testProperty;
+    String geodePrefixProperty = "geode." + testProperty;
+    System.setProperty(geodePrefixProperty, "1");
+    System.setProperty(gemfirePrefixProperty, "0");
+    assertThat(SystemPropertyHelper.getProductIntegerProperty(testProperty).get()).isEqualTo(1);
+    System.clearProperty(geodePrefixProperty);
+    System.clearProperty(gemfirePrefixProperty);
+  }
+
+  @Test
+  public void getIntegerPropertyReturnsGemfirePrefixIfGeodeMissing() {
+    String testProperty = "testIntegerProperty";
+    String gemfirePrefixProperty = "gemfire." + testProperty;
+    System.setProperty(gemfirePrefixProperty, "1");
+    assertThat(SystemPropertyHelper.getProductIntegerProperty(testProperty).get()).isEqualTo(1);
+    System.clearProperty(gemfirePrefixProperty);
+  }
+
+  @Test
+  public void getIntegerPropertyReturnsNullIfPropertiesMissing() {
+    String testProperty = "testIntegerProperty";
+    assertThat(SystemPropertyHelper.getProductIntegerProperty(testProperty).isPresent()).isFalse();
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/size/SizeOfUtil0.java b/geode-core/src/test/java/org/apache/geode/internal/size/SizeOfUtil0.java
similarity index 100%
rename from geode-core/src/main/java/org/apache/geode/internal/size/SizeOfUtil0.java
rename to geode-core/src/test/java/org/apache/geode/internal/size/SizeOfUtil0.java
diff --git a/geode-core/src/test/java/org/apache/geode/management/RegionManagementDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/RegionManagementDUnitTest.java
index 3ddb041315..1e73728085 100644
--- a/geode-core/src/test/java/org/apache/geode/management/RegionManagementDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/RegionManagementDUnitTest.java
@@ -62,7 +62,7 @@
 import org.apache.geode.internal.cache.AbstractRegion;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.TestObjectSizerImpl;
-import org.apache.geode.internal.cache.lru.LRUStatistics;
+import org.apache.geode.internal.cache.eviction.EvictionStatistics;
 import org.apache.geode.internal.cache.partitioned.fixed.SingleHopQuarterPartitionResolver;
 import org.apache.geode.management.internal.MBeanJMXAdapter;
 import org.apache.geode.management.internal.SystemManagementService;
@@ -358,8 +358,8 @@ private void createDiskRegion(final VM memberVM) {
 
       Region region = getCache().createRegion(REGION_NAME, factory.create());
 
-      LRUStatistics lruStats =
-          ((AbstractRegion) region).getEvictionController().getLRUHelper().getStats();
+      EvictionStatistics lruStats =
+          ((AbstractRegion) region).getEvictionController().getStatistics();
       assertThat(lruStats).isNotNull();
 
       RegionMXBean regionMXBean = getManagementService().getLocalRegionMBean(REGION_PATH);
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/excludedClasses.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
index abf32d66a6..7156f697a6 100644
--- a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
+++ b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
@@ -74,6 +74,11 @@ org/apache/geode/security/ResourcePermission$Target
 org/apache/geode/internal/cache/PartitionedRegion$7
 org/apache/geode/internal/cache/TXFarSideCMTracker$2
 org/apache/geode/internal/cache/TXManagerImpl$1
+org/apache/geode/internal/cache/eviction/AbstractEvictionController
+org/apache/geode/internal/cache/eviction/CountLRUEviction
+org/apache/geode/internal/cache/eviction/HeapLRUController
+org/apache/geode/internal/cache/eviction/MemoryLRUController
+org/apache/geode/internal/cache/eviction/SizeLRUController
 org/apache/geode/internal/cache/partitioned/FetchEntriesMessage$FetchEntriesResponse$1
 org/apache/geode/pdx/internal/AutoSerializableManager$1
 org/apache/geode/cache/InterestResultPolicy
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java
index 854f5c1a09..774ab5082f 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java
@@ -131,7 +131,7 @@ public void regionsWithEvictionWithOverflowMustBeAbleToCreateLuceneIndexes(
   }
 
   protected void raiseFakeNotification() {
-    ((GemFireCacheImpl) getCache()).getHeapEvictor().testAbortAfterLoopCount = 1;
+    ((GemFireCacheImpl) getCache()).getHeapEvictor().setTestAbortAfterLoopCount(1);
     HeapMemoryMonitor.setTestDisableMemoryUpdates(true);
     System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance", "0");
 
@@ -145,7 +145,7 @@ protected void raiseFakeNotification() {
   }
 
   protected void cleanUpAfterFakeNotification() {
-    ((GemFireCacheImpl) getCache()).getHeapEvictor().testAbortAfterLoopCount = Integer.MAX_VALUE;
+    ((GemFireCacheImpl) getCache()).getHeapEvictor().setTestAbortAfterLoopCount(Integer.MAX_VALUE);
     HeapMemoryMonitor.setTestDisableMemoryUpdates(false);
     System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance");
   }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> The first HeapLRU evictions on large region can consume high amounts of CPU
> ---------------------------------------------------------------------------
>
>                 Key: GEODE-3622
>                 URL: https://issues.apache.org/jira/browse/GEODE-3622
>             Project: Geode
>          Issue Type: Improvement
>          Components: eviction
>            Reporter: Darrel Schneider
>            Assignee: Kirk Lund
>
> If you have a region configured for HeapLRU eviction and are able to put a large number of entries in it before hitting the eviction threshold, then the first evictions will consume much more CPU than subsequent evictions. The reason for this is that the eviction list is not initialized until the first eviction is done. At that time a scan is done of all the existing entries marking each one as not having been recently used. This can touch a large number of memory pages and also hits some common synchronization locks. You can see that this is happening by looking at the HeapLRUStats.lruEvaluations stat.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)