[jbosscache-commits] JBoss Cache SVN: r7633 - in core/branches/flat/src: main/java/org/horizon/eviction/algorithms and 10 other directories.

jbosscache-commits at lists.jboss.org jbosscache-commits at lists.jboss.org
Tue Feb 3 16:28:24 EST 2009


Author: manik.surtani at jboss.com
Date: 2009-02-03 16:28:24 -0500 (Tue, 03 Feb 2009)
New Revision: 7633

Added:
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseEvictionQueue.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/BaseAlgorithmTest.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/BaseQueueTest.java
Removed:
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseSortedEvictionAlgorithm.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/SortedEvictionQueue.java
Modified:
   core/branches/flat/src/main/java/org/horizon/eviction/EntryEvictionData.java
   core/branches/flat/src/main/java/org/horizon/eviction/EvictionAlgorithmConfigBase.java
   core/branches/flat/src/main/java/org/horizon/eviction/EvictionEvent.java
   core/branches/flat/src/main/java/org/horizon/eviction/EvictionQueue.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseEvictionAlgorithm.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOAlgorithm.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOAlgorithmConfig.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOQueue.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUAlgorithm.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUAlgorithmConfig.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUQueue.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUAlgorithm.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUAlgorithmConfig.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUQueue.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUAlgorithm.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUAlgorithmConfig.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUQueue.java
   core/branches/flat/src/main/java/org/horizon/eviction/algorithms/nullalgo/NullEvictionQueue.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/fifo/FifoAlgorithmTest.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/fifo/FifoQueueTest.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lfu/LfuAlgorithmTest.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lfu/LfuQueueTest.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lru/LruAlgorithmTest.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lru/LruQueueTest.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/mru/MruAlgorithmTest.java
   core/branches/flat/src/test/java/org/horizon/eviction/algorithms/mru/MruQueueTest.java
Log:
Eviction overhaul, phase 2

Modified: core/branches/flat/src/main/java/org/horizon/eviction/EntryEvictionData.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/EntryEvictionData.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/EntryEvictionData.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -110,10 +110,6 @@
       this.numberOfVisits = numberOfVisits;
    }
 
-   public void incerementNumberOfNodeVisits() {
-      this.numberOfVisits++;
-   }
-
    public Object getKey() {
       return key;
    }
@@ -164,4 +160,8 @@
       output.append(" CurrentlyInUse: ").append(this.isCurrentlyInUse());
       return output.toString();
    }
+
+   public void incrementNumberOfVisits() {
+      numberOfVisits++;
+   }
 }

Modified: core/branches/flat/src/main/java/org/horizon/eviction/EvictionAlgorithmConfigBase.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/EvictionAlgorithmConfigBase.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/EvictionAlgorithmConfigBase.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -39,10 +39,17 @@
 
    protected String evictionAlgorithmClassName;
    @Dynamic
-   protected int maxEntries = 0;
+   protected int maxEntries = -1;
    @Dynamic
-   protected long minTimeToLive;
+   protected int minEntries = -1;
+   @Dynamic
+   protected long minTimeToLive = -1;
+   @Dynamic
+   private long timeToLive = -1;
+   @Dynamic
+   private long maxAge = -1;
 
+
    /**
     * Can only be instantiated by a subclass.
     */
@@ -57,12 +64,34 @@
       return maxEntries;
    }
 
+   /**
+    * @param maxEntries max entries to hold in the cache. 0 denotes immediate expiry and -1 denotes unlimited entries.
+    *                   -1 is the default
+    */
    public void setMaxEntries(int maxEntries) {
       testImmutability("maxEntries");
       this.maxEntries = maxEntries;
    }
 
+   public int getMinEntries() {
+      return minEntries;
+   }
+
    /**
+    * This specifies the minimum entries to prune down to when maxExtries has been hit.  -1 is the default value, which
+    * means this feature is effectively unset, and eviction algorithms would be expected to evict until the cache
+    * contains no more than maxEntries.  Any other value means that if a pruning process starts, it will not stop until
+    * minEntries has been reached.  So, for example, minEntries of 0 would mean that the cache is emptied the moment
+    * maxEntries is exceeded.
+    *
+    * @param minEntries minEntries value
+    */
+   public void setMinEntries(int minEntries) {
+      testImmutability("minEntries");
+      this.minEntries = minEntries;
+   }
+
+   /**
     * @return The minimum time to live, in milliseconds.
     */
    public long getMinTimeToLive() {
@@ -70,7 +99,8 @@
    }
 
    /**
-    * @param minTimeToLive time to live, in milliseconds
+    * @param minTimeToLive time to live, in milliseconds.  This defaults to -1, meaning that it is excluded from
+    *                      calculations.
     */
    public void setMinTimeToLive(long minTimeToLive) {
       testImmutability("minTimeToLive");
@@ -82,42 +112,100 @@
       minTimeToLive = timeUnit.toMillis(time);
    }
 
+   public long getTimeToLive() {
+      return timeToLive;
+   }
+
+   /**
+    * -1 means time to live is unused, 0 means the time to live is 0 (considered for immediate eviction)
+    *
+    * @param timeToLive the time to live, since last modification, of any entry in the cache.  Defaults to -1, meaning
+    *                   that this is unused.
+    */
+   public void setTimeToLive(long timeToLive) {
+      testImmutability("timeToLive");
+      this.timeToLive = timeToLive;
+   }
+
+   public void setTimeToLive(long timeToLive, TimeUnit unit) {
+      setTimeToLive(unit.toMillis(timeToLive));
+   }
+
+   public long getMaxAge() {
+      return maxAge;
+   }
+
+   /**
+    * -1 means max age is not considered, 0 means entries are considered for eviction immediately.
+    *
+    * @param maxAge the maximum age of entries in the cache, after which they will be removed.
+    */
+   public void setMaxAge(long maxAge) {
+      testImmutability("maxAge");
+      this.maxAge = maxAge;
+   }
+
+   public void setMaxAge(long maxAge, TimeUnit unit) {
+      setMaxAge(unit.toMillis(maxAge));
+   }
+
    public void validate() throws ConfigurationException {
       if (evictionAlgorithmClassName == null)
          throw new ConfigurationException("Eviction algorithm class name cannot be null!");
    }
 
+   @Override
    public boolean equals(Object o) {
       if (this == o) return true;
-      if (!(o instanceof EvictionAlgorithmConfigBase)) return false;
+      if (o == null || getClass() != o.getClass()) return false;
 
       EvictionAlgorithmConfigBase that = (EvictionAlgorithmConfigBase) o;
 
+      if (maxAge != that.maxAge) return false;
       if (maxEntries != that.maxEntries) return false;
+      if (minEntries != that.minEntries) return false;
       if (minTimeToLive != that.minTimeToLive) return false;
+      if (timeToLive != that.timeToLive) return false;
       if (evictionAlgorithmClassName != null ? !evictionAlgorithmClassName.equals(that.evictionAlgorithmClassName) : that.evictionAlgorithmClassName != null)
          return false;
 
       return true;
    }
 
+   @Override
    public int hashCode() {
-      int result;
-      result = (evictionAlgorithmClassName != null ? evictionAlgorithmClassName.hashCode() : 0);
+      int result = evictionAlgorithmClassName != null ? evictionAlgorithmClassName.hashCode() : 0;
       result = 31 * result + maxEntries;
-      result = (int) (31 * result + minTimeToLive);
+      result = 31 * result + minEntries;
       result = 31 * result + (int) (minTimeToLive ^ (minTimeToLive >>> 32));
+      result = 31 * result + (int) (timeToLive ^ (timeToLive >>> 32));
+      result = 31 * result + (int) (maxAge ^ (maxAge >>> 32));
       return result;
    }
 
+   @Override
+   public String toString() {
+      return getClass().getSimpleName() + "{" +
+            "evictionAlgorithmClassName='" + evictionAlgorithmClassName + '\'' +
+            ", maxEntries=" + maxEntries +
+            ", minEntries=" + minEntries +
+            ", minTimeToLive=" + minTimeToLive +
+            ", timeToLive=" + timeToLive +
+            ", maxAge=" + maxAge +
+            '}';
+   }
+
    public void reset() {
-      maxEntries = 0;
-      minTimeToLive = 0;
+      maxEntries = -1;
+      minEntries = -1;
+      minTimeToLive = -1;
+      timeToLive = -1;
+      maxAge = -1;
    }
 
-   public EvictionAlgorithmConfig clone() {
+   public EvictionAlgorithmConfigBase clone() {
       try {
-         return (EvictionAlgorithmConfig) super.clone();
+         return (EvictionAlgorithmConfigBase) super.clone();
       } catch (CloneNotSupportedException e) {
          throw new RuntimeException("Should never happen", e);
       }

Modified: core/branches/flat/src/main/java/org/horizon/eviction/EvictionEvent.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/EvictionEvent.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/EvictionEvent.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -52,10 +52,20 @@
       this.creationTimestamp = System.currentTimeMillis();
    }
 
+   public EvictionEvent(Object key, Type type, long creationTimestamp) {
+      this.key = key;
+      this.type = type;
+      this.creationTimestamp = creationTimestamp;
+   }
+
    public long getCreationTimestamp() {
       return creationTimestamp;
    }
 
+   public void setCreationTimestamp(long creationTimestamp) {
+      this.creationTimestamp = creationTimestamp;
+   }
+
    public long getInUseTimeout() {
       return inUseTimeout;
    }

Modified: core/branches/flat/src/main/java/org/horizon/eviction/EvictionQueue.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/EvictionQueue.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/EvictionQueue.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -31,17 +31,14 @@
  * @since 1.0
  */
 public interface EvictionQueue extends Iterable<EntryEvictionData> {
-//   /**
-//    * Get the first entry in the queue.
-//    * <p/>
-//    * If there are no entries in queue, this method will return null.
-//    * <p/>
-//    * The first node returned is expected to be the first node to evict.
-//    *
-//    * @return first NodeEntry in queue.
-//    */
-//   EntryEvictionData getFirstNodeEntry();
 
+   /**
+    * Tests whether queue is empty
+    *
+    * @return true if the queue is empty; false otherwise
+    */
+   boolean isEmpty();
+
    //
    /**
     * Retrieve eviction entry data representing a specific key
@@ -57,17 +54,17 @@
    /**
     * Check if eviction entry data exists in the queue
     *
-    * @param entryEvictionData to check for
+    * @param key to check for
     * @return true if the entry exists, false otherwise
     */
-   boolean contains(EntryEvictionData entryEvictionData);
+   boolean contains(Object key);
 
    /**
     * Remove eviction entry data from the queue.  A no-op if the specified object does not exist.
     *
-    * @param entryEvictionData to remove
+    * @param key to remove
     */
-   void remove(EntryEvictionData entryEvictionData);
+   void remove(Object key);
 
    /**
     * Add entry eviction data to the queue

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseEvictionAlgorithm.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseEvictionAlgorithm.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseEvictionAlgorithm.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -26,6 +26,7 @@
 import org.horizon.eviction.EntryEvictionData;
 import org.horizon.eviction.EvictionAction;
 import org.horizon.eviction.EvictionAlgorithm;
+import org.horizon.eviction.EvictionAlgorithmConfigBase;
 import org.horizon.eviction.EvictionEvent;
 import org.horizon.eviction.EvictionEvent.Type;
 import org.horizon.eviction.EvictionException;
@@ -33,6 +34,7 @@
 import org.horizon.logging.Log;
 import org.horizon.logging.LogFactory;
 
+import java.util.Iterator;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
@@ -52,7 +54,7 @@
    private static final boolean trace = log.isTraceEnabled();
 
    protected EvictionAction evictionAction;
-   protected EvictionAlgorithmConfig evictionAlgorithmConfig;
+   protected EvictionAlgorithmConfigBase evictionAlgorithmConfig;
    protected BlockingQueue recycleQueue;
    protected EvictionQueue evictionQueue;
    protected Cache cache;
@@ -66,13 +68,37 @@
    protected abstract EvictionQueue setupEvictionQueue() throws EvictionException;
 
    /**
-    * This method will check whether the given node should be evicted or not.
+    * This method will check whether the given entry should be evicted or not.
     *
-    * @param ne NodeEntry to test eviction for.
-    * @return True if the given node should be evicted. False if the given node should not be evicted.
+    * @param entry entry to consider
+    * @return true if the entry is to be evicted, false otherwise
     */
-   protected abstract boolean shouldEvictNode(EntryEvictionData ne);
+   protected boolean shouldEvictEntry(EntryEvictionData entry, int originalQueueSize) {
+      // first test max and min entries
+      if (evictionAlgorithmConfig.getMaxEntries() > -1) {
+         int currentSize = evictionQueue.size();
+         // entry count eviction is enabled!
+         if (evictionAlgorithmConfig.getMaxEntries() < currentSize) return true;
+         if (evictionAlgorithmConfig.getMaxEntries() < originalQueueSize &&
+               evictionAlgorithmConfig.getMinEntries() > -1 &&
+               evictionAlgorithmConfig.getMinEntries() < currentSize)
+            return true;
+      }
 
+      // else, let's check if the entry has expired due to TTL or maxAge
+      if (evictionAlgorithmConfig.getTimeToLive() > -1) {
+         if (System.currentTimeMillis() > evictionAlgorithmConfig.getTimeToLive() + entry.getModifiedTimeStamp())
+            return true;
+      }
+
+      if (evictionAlgorithmConfig.getMaxAge() > -1) {
+         if (System.currentTimeMillis() > evictionAlgorithmConfig.getMaxAge() + entry.getCreationTimeStamp())
+            return true;
+      }
+
+      return false;
+   }
+
    protected BaseEvictionAlgorithm() {
       recycleQueue = new LinkedBlockingQueue(500000);
    }
@@ -89,7 +115,7 @@
 
    public void assignToCache(Cache<?, ?> cache, EvictionAlgorithmConfig evictionAlgorithmConfig) {
       this.cache = cache;
-      this.evictionAlgorithmConfig = evictionAlgorithmConfig;
+      this.evictionAlgorithmConfig = (EvictionAlgorithmConfigBase) evictionAlgorithmConfig;
    }
 
    public boolean canIgnoreEvent(Type eventType) {
@@ -108,9 +134,9 @@
    public void process(BlockingQueue<EvictionEvent> eventQueue) throws EvictionException {
       if (trace) log.trace("processing eviction event queue");
       initialize();
-      this.processQueues(eventQueue);
-      this.emptyRecycleQueue();
-      this.prune();
+      processQueues(eventQueue);
+      emptyRecycleQueue();
+      prune();
    }
 
    public void resetEvictionQueue() {
@@ -154,19 +180,19 @@
          count++;
          switch (event.getEventType()) {
             case ADD_ENTRY_EVENT:
-               this.processAddedNodes(event);
+               processAddedEntries(event);
                break;
             case REMOVE_ENTRY_EVENT:
-               this.processRemovedNodes(event);
+               processRemovedEntries(event);
                break;
             case VISIT_ENTRY_EVENT:
-               this.processVisitedNodes(event);
+               processVisitedEntries(event);
                break;
             case MARK_IN_USE_EVENT:
-               this.processMarkInUseNodes(event.getKey(), event.getInUseTimeout());
+               processMarkInUseNodes(event.getKey(), event.getInUseTimeout());
                break;
             case UNMARK_IN_USE_EVENT:
-               this.processUnmarkInUseNodes(event.getKey());
+               processUnmarkInUseNodes(event.getKey());
                break;
             default:
                throw new RuntimeException("Illegal Eviction Event type " + event.getEventType());
@@ -175,9 +201,9 @@
       if (trace) log.trace("processed {0} eviction events", count);
    }
 
-   protected void evict(EntryEvictionData data) {
+   protected boolean evict(EntryEvictionData data) {
       if (data != null) {
-         evictionQueue.remove(data);
+         log.trace("Attempting to evict {0}", data);
          if (!evictionAction.evict(data.getKey())) {
             try {
                boolean result = recycleQueue.offer(data.getKey(), 5, TimeUnit.SECONDS);
@@ -192,6 +218,9 @@
                log.debug("InterruptedException", e);
             }
          }
+         return true;
+      } else {
+         return false;
       }
    }
 
@@ -213,23 +242,24 @@
       }
    }
 
-   protected void processAddedNodes(EvictionEvent evictedEventNode) throws EvictionException {
+   protected void processAddedEntries(EvictionEvent evictedEventNode) throws EvictionException {
       Object key = evictedEventNode.getKey();
       if (trace) log.trace("Adding entry {0} to eviction queue", key);
       EntryEvictionData data = evictionQueue.get(key);
       if (data != null) {
          data.setModifiedTimeStamp(evictedEventNode.getCreationTimestamp());
-         data.incerementNumberOfNodeVisits();
+         data.incrementNumberOfVisits();
          if (trace) log.trace("Queue already contains key.  Processing it as visited.");
-         processVisitedNodes(evictedEventNode);
-         return;
+         processVisitedEntries(evictedEventNode);
+      } else {
+         data = new EntryEvictionData(1, evictedEventNode.getCreationTimestamp(), key);
+         data.setCreationTimeStamp(evictedEventNode.getCreationTimestamp());
+         evictionQueue.add(data);
+         if (trace) log.trace("Added successfully to eviction queue");
       }
-      data = new EntryEvictionData(1, evictedEventNode.getCreationTimestamp(), key);
-      evictionQueue.add(data);
-      if (trace) log.trace("Added successfully to eviction queue");
    }
 
-   protected void processRemovedNodes(EvictionEvent evictedEventNode) throws EvictionException {
+   protected void processRemovedEntries(EvictionEvent evictedEventNode) throws EvictionException {
       Object key = evictedEventNode.getKey();
 
       if (trace) log.trace("Removing key {0} from eviction queue and attempting eviction", key);
@@ -258,19 +288,19 @@
     *
     * @throws EvictionException
     */
-   protected void processVisitedNodes(EvictionEvent evictedEventNode) throws EvictionException {
-      Object key = evictedEventNode.getKey();
+   protected void processVisitedEntries(EvictionEvent evictionEvent) throws EvictionException {
+      Object key = evictionEvent.getKey();
       EntryEvictionData data = evictionQueue.get(key);
       if (data == null) {
          if (trace) log.trace("Visiting entry {0} that has not added to eviction queues before.", key);
-         this.processAddedNodes(evictedEventNode);
+         this.processAddedEntries(evictionEvent);
          return;
       }
       // note this method will visit and modify the node statistics by reference!
       // if a collection is only guaranteed sort order by adding to the collection,
       // this implementation will not guarantee sort order.
-      data.incerementNumberOfNodeVisits();
-      data.setModifiedTimeStamp(evictedEventNode.getCreationTimestamp());
+      data.incrementNumberOfVisits();
+      data.setModifiedTimeStamp(evictionEvent.getCreationTimestamp());
    }
 
    /**
@@ -312,9 +342,11 @@
    }
 
    protected void prune() throws EvictionException {
-      for (EntryEvictionData data : evictionQueue) {
-         if (data != null && shouldEvictNode(data)) {
-            evict(data);
+      int originalQueueSize = evictionQueue.size();
+      for (Iterator<EntryEvictionData> it = evictionQueue.iterator(); it.hasNext();) {
+         EntryEvictionData data = it.next();
+         if (data != null && shouldEvictEntry(data, originalQueueSize)) {
+            if (shouldNotOverrideEviction(data) && evict(data)) it.remove();
          } else {
             break; // assume the rest won't need to be evicted either
          }
@@ -330,13 +362,13 @@
    }
 
    /**
-    * A helper for implementations that support the minimul time to live property
+    * A helper for implementations that support the minimum time to live property
     *
-    * @param minTTL minimum time to live, in millis, for all entries in the cache
-    * @param data   eviction data on the entry to consider
+    * @param data eviction data on the entry to consider
     * @return true if the entry is younger than the minimum time to live, false otherwise.
     */
-   protected boolean isYoungerThanMinimumTimeToLive(long minTTL, EntryEvictionData data) {
-      return minTTL >= 1 && (data.getModifiedTimeStamp() + minTTL > System.currentTimeMillis());
+   protected boolean shouldNotOverrideEviction(EntryEvictionData data) {
+      long minTTL = evictionAlgorithmConfig.getMinTimeToLive();
+      return minTTL < 1 || (data.getModifiedTimeStamp() + minTTL < System.currentTimeMillis());
    }
 }

Added: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseEvictionQueue.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseEvictionQueue.java	                        (rev 0)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseEvictionQueue.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -0,0 +1,9 @@
+package org.horizon.eviction.algorithms;
+
+import org.horizon.eviction.EvictionQueue;
+
+public abstract class BaseEvictionQueue implements EvictionQueue {
+   public boolean isEmpty() {
+      return size() == 0;
+   }
+}

Deleted: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseSortedEvictionAlgorithm.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseSortedEvictionAlgorithm.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/BaseSortedEvictionAlgorithm.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,92 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * Copyright 2000 - 2008, Red Hat Middleware LLC, and individual contributors
- * as indicated by the @author tags. See the copyright.txt file in the
- * distribution for a full listing of individual contributors.
- *
- * This is free software; you can redistribute it and/or modify it
- * under the terms of the GNU Lesser General Public License as
- * published by the Free Software Foundation; either version 2.1 of
- * the License, or (at your option) any later version.
- *
- * This software is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- * Lesser General Public License for more details.
- *
- * You should have received a copy of the GNU Lesser General Public
- * License along with this software; if not, write to the Free
- * Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
- * 02110-1301 USA, or see the FSF site: http://www.fsf.org.
- */
-package org.horizon.eviction.algorithms;
-
-import org.horizon.eviction.EvictionEvent;
-import org.horizon.eviction.EvictionException;
-import org.horizon.logging.Log;
-import org.horizon.logging.LogFactory;
-
-import java.util.concurrent.BlockingQueue;
-
-
-/**
- * An abstract SortedEvictionAlgorithm.
- * <p/>
- * This class supports early termination of the eviction queue processing. Because the eviction queue is sorted by first
- * to evict to last to evict, when iterating the eviction queue, the first time a node is encountered that does not
- * require eviction will terminate the loop early. This way we don't incur the full breadth of the O(n) = n operation
- * everytime we need to check for eviction (defined by eviction poll time interval).
- *
- * @author Daniel Huang - dhuang at jboss.org - 10/2005
- * @since 1.0
- */
-public abstract class BaseSortedEvictionAlgorithm extends BaseEvictionAlgorithm {
-   private static final Log log = LogFactory.getLog(BaseSortedEvictionAlgorithm.class);
-   private static final boolean trace = log.isTraceEnabled();
-
-   @Override
-   protected void processQueues(BlockingQueue<EvictionEvent> queue) throws EvictionException {
-      boolean evictionQueueRequiresSort = false;
-
-      EvictionEvent event;
-      int count = 0;
-      while ((event = getNextInQueue(queue)) != null) {
-         count++;
-         switch (event.getEventType()) {
-            case ADD_ENTRY_EVENT:
-               this.processAddedNodes(event);
-               evictionQueueRequiresSort = true;
-               break;
-            case REMOVE_ENTRY_EVENT:
-               this.processRemovedNodes(event);
-               break;
-            case VISIT_ENTRY_EVENT:
-               this.processVisitedNodes(event);
-               evictionQueueRequiresSort = true;
-               break;
-            default:
-               throw new RuntimeException("Illegal Eviction Event type " + event.getEventType());
-         }
-      }
-
-      if (evictionQueueRequiresSort) {
-         if (trace) log.trace("Eviction queue requires re-sort.  Re-sorting.");
-         resortEvictionQueue();
-      }
-
-      if (trace) log.trace("processed {0} eviction events", count);
-   }
-
-   /**
-    * This method is called to resort the queue after add or visit events have occurred.
-    */
-   protected void resortEvictionQueue() {
-      long begin = System.currentTimeMillis();
-      ((SortedEvictionQueue) evictionQueue).reSortEvictionQueue();
-
-      if (trace) {
-         log.trace("Took {0} millis to re-sort queue with {1} elements",
-                   System.currentTimeMillis() - begin, getEvictionQueue().size());
-      }
-   }
-}

Deleted: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/SortedEvictionQueue.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/SortedEvictionQueue.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/SortedEvictionQueue.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,37 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * Copyright 2000 - 2008, Red Hat Middleware LLC, and individual contributors
- * as indicated by the @author tags. See the copyright.txt file in the
- * distribution for a full listing of individual contributors.
- *
- * This is free software; you can redistribute it and/or modify it
- * under the terms of the GNU Lesser General Public License as
- * published by the Free Software Foundation; either version 2.1 of
- * the License, or (at your option) any later version.
- *
- * This software is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- * Lesser General Public License for more details.
- *
- * You should have received a copy of the GNU Lesser General Public
- * License along with this software; if not, write to the Free
- * Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
- * 02110-1301 USA, or see the FSF site: http://www.fsf.org.
- */
-package org.horizon.eviction.algorithms;
-
-import org.horizon.eviction.EvictionQueue;
-
-/**
- * Sorted Eviction Queue implementation.
- *
- * @author Daniel Huang (dhuang at jboss.org)
- * @since 1.0
- */
-public interface SortedEvictionQueue extends EvictionQueue {
-   /**
-    * Provide contract to re-sort a sorted queue.
-    */
-   void reSortEvictionQueue();
-}

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOAlgorithm.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOAlgorithm.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOAlgorithm.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -21,8 +21,8 @@
  */
 package org.horizon.eviction.algorithms.fifo;
 
+import net.jcip.annotations.NotThreadSafe;
 import org.horizon.config.EvictionAlgorithmConfig;
-import org.horizon.eviction.EntryEvictionData;
 import org.horizon.eviction.EvictionException;
 import org.horizon.eviction.EvictionQueue;
 import org.horizon.eviction.algorithms.BaseEvictionAlgorithm;
@@ -30,45 +30,22 @@
 import org.horizon.logging.LogFactory;
 
 /**
- * First-in-first-out algorithm used to evict nodes.
+ * First-in-first-out algorithm used to evict cache entries
  *
- * @author Daniel Huang - dhuang at jboss.org
- * @author Morten Kvistgaard
+ * @author Manik Surtani
  * @since 1.0
  */
+ at NotThreadSafe
 public class FIFOAlgorithm extends BaseEvictionAlgorithm {
    private static final Log log = LogFactory.getLog(FIFOAlgorithm.class);
-   private static final boolean trace = log.isTraceEnabled();
-   private FIFOAlgorithmConfig fifoAlgorithmConfig;
 
    @Override
    protected EvictionQueue setupEvictionQueue() throws EvictionException {
       return new FIFOQueue();
    }
 
-   /**
-    * For FIFO, a node should be evicted if the queue size is >= to the configured maxNodes size.
-    */
-   @Override
-   protected boolean shouldEvictNode(EntryEvictionData data) {
-      // check the minimum time to live and see if we should not evict the node.  This check will
-      // ensure that, if configured, nodes are kept alive for at least a minimum period of time.
-      if (isYoungerThanMinimumTimeToLive(fifoAlgorithmConfig.getMinTimeToLive(), data)) return false;
-
-      if (trace) log.trace("Deciding whether node in queue {0} requires eviction.", data.getKey());
-
-      int size = this.getEvictionQueue().size();
-      return fifoAlgorithmConfig.getMaxEntries() != 0 && size > fifoAlgorithmConfig.getMaxEntries();
-   }
-
    public Class<? extends EvictionAlgorithmConfig> getConfigurationClass() {
       return FIFOAlgorithmConfig.class;
    }
-
-   @Override
-   public void initialize() {
-      super.initialize();
-      fifoAlgorithmConfig = (FIFOAlgorithmConfig) evictionAlgorithmConfig;
-   }
 }
 

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOAlgorithmConfig.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOAlgorithmConfig.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOAlgorithmConfig.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -21,13 +21,10 @@
  */
 package org.horizon.eviction.algorithms.fifo;
 
-import org.horizon.config.ConfigurationException;
 import org.horizon.eviction.EvictionAlgorithmConfigBase;
 
 /**
  * Configuration for {@link FIFOAlgorithm}.
- * <p/>
- * Requires a "maxNodes" attribute otherwise a ConfigurationException is thrown.
  *
  * @author Manik Surtani
  * @since 1.0
@@ -40,48 +37,9 @@
 
    public FIFOAlgorithmConfig() {
       evictionAlgorithmClassName = FIFOAlgorithm.class.getName();
-      // We require that maxEntries is set
-      setMaxEntries(-1);
    }
 
-   public FIFOAlgorithmConfig(int maxNodes) {
-      evictionAlgorithmClassName = FIFOAlgorithm.class.getName();
-      // We require that maxEntries is set
-      setMaxEntries(maxNodes);
-   }
-
-   /**
-    * Requires a positive maxNodes value or ConfigurationException is thrown.
-    */
    @Override
-   public void validate() throws ConfigurationException {
-      super.validate();
-      if (getMaxEntries() < 0) {
-         throw new ConfigurationException("maxEntries must be must be " +
-               "configured to a value greater than or equal to 0");
-      }
-   }
-
-   @Override
-   public String toString() {
-      StringBuilder ret = new StringBuilder();
-      ret.append("FIFOAlgorithmConfig: maxEntries = ").append(getMaxEntries());
-      return ret.toString();
-   }
-
-   @Override
-   public boolean equals(Object obj) {
-      return (obj instanceof FIFOAlgorithmConfig && super.equals(obj));
-   }
-
-   @Override
-   public void reset() {
-      super.reset();
-      setMaxEntries(-1);
-      evictionAlgorithmClassName = FIFOAlgorithm.class.getName();
-   }
-
-   @Override
    public FIFOAlgorithmConfig clone() {
       return (FIFOAlgorithmConfig) super.clone();
    }

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOQueue.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOQueue.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/fifo/FIFOQueue.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -22,7 +22,7 @@
 package org.horizon.eviction.algorithms.fifo;
 
 import org.horizon.eviction.EntryEvictionData;
-import org.horizon.eviction.EvictionQueue;
+import org.horizon.eviction.algorithms.BaseEvictionQueue;
 
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -34,7 +34,7 @@
  * @author Daniel Huang (dhuang at jboss.org)
  * @since 1.0
  */
-public class FIFOQueue implements EvictionQueue {
+public class FIFOQueue extends BaseEvictionQueue {
    private Map<Object, EntryEvictionData> keyMap;
 
    protected FIFOQueue() {
@@ -47,12 +47,12 @@
       return keyMap.get(key);
    }
 
-   public boolean contains(EntryEvictionData entryEvictionData) {
-      return keyMap.containsKey(entryEvictionData.getKey());
+   public boolean contains(Object key) {
+      return keyMap.containsKey(key);
    }
 
-   public void remove(EntryEvictionData entryEvictionData) {
-      keyMap.remove(entryEvictionData.getKey());
+   public void remove(Object key) {
+      keyMap.remove(key);
    }
 
    public void add(EntryEvictionData entryEvictionData) {

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUAlgorithm.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUAlgorithm.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUAlgorithm.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -21,14 +21,17 @@
  */
 package org.horizon.eviction.algorithms.lfu;
 
+import net.jcip.annotations.NotThreadSafe;
 import org.horizon.config.EvictionAlgorithmConfig;
-import org.horizon.eviction.EntryEvictionData;
+import org.horizon.eviction.EvictionEvent;
 import org.horizon.eviction.EvictionException;
 import org.horizon.eviction.EvictionQueue;
-import org.horizon.eviction.algorithms.BaseSortedEvictionAlgorithm;
+import org.horizon.eviction.algorithms.BaseEvictionAlgorithm;
 import org.horizon.logging.Log;
 import org.horizon.logging.LogFactory;
 
+import java.util.concurrent.BlockingQueue;
+
 /**
  * Least Frequently Used algorithm for cache eviction. Note that this algorithm is not thread-safe.
  * <p/>
@@ -43,30 +46,15 @@
  * This algorithm uses a sorted eviction queue. The eviction queue is sorted in ascending order based on the number of
  * times a node is visited. The more frequently a node is visited, the less likely it will be evicted.
  *
- * @author Daniel Huang - dhuang at jboss.org 10/2005
+ * @author Manik Surtani
  * @since 1.0
  */
-public class LFUAlgorithm extends BaseSortedEvictionAlgorithm {
+ at NotThreadSafe
+public class LFUAlgorithm extends BaseEvictionAlgorithm {
    private static final Log log = LogFactory.getLog(LFUAlgorithm.class);
    private static final boolean trace = log.isTraceEnabled();
-   private LFUAlgorithmConfig lfuAlgorithmConfig;
+   boolean evictionQueueRequiresSort = false;
 
-   @Override
-   protected boolean shouldEvictNode(EntryEvictionData data) {
-      // check the minimum time to live and see if we should not evict the node.  This check will
-      // ensure that, if configured, nodes are kept alive for at least a minimum period of time.
-      if (isYoungerThanMinimumTimeToLive(lfuAlgorithmConfig.getMinTimeToLive(), data)) return false;
-
-      int size = this.getEvictionQueue().size();
-      if (lfuAlgorithmConfig.getMaxEntries() > -1 && size > lfuAlgorithmConfig.getMaxEntries()) {
-         return true;
-      } else if (size > lfuAlgorithmConfig.getMinEntries()) {
-         return true;
-      }
-
-      return false;
-   }
-
    /**
     * Will create a LFUQueue to be used as the underlying eviction queue.
     *
@@ -84,7 +72,7 @@
       super.prune();
 
       // clean up the Queue's eviction removals
-      ((LFUQueue) this.evictionQueue).prune();
+      ((LFUQueue) this.evictionQueue).doBatchRemove();
    }
 
    public Class<? extends EvictionAlgorithmConfig> getConfigurationClass() {
@@ -92,8 +80,41 @@
    }
 
    @Override
-   public void initialize() {
-      super.initialize();
-      lfuAlgorithmConfig = (LFUAlgorithmConfig) evictionAlgorithmConfig;
+   protected void processQueues(BlockingQueue<EvictionEvent> queue) throws EvictionException {
+      evictionQueueRequiresSort = false;
+      try {
+         super.processQueues(queue);
+         if (evictionQueueRequiresSort) {
+            if (trace) log.trace("Eviction queue requires re-sort.  Re-sorting.");
+            resortEvictionQueue();
+         }
+      } finally {
+         evictionQueueRequiresSort = false;
+      }
    }
+
+   @Override
+   protected void processAddedEntries(EvictionEvent event) {
+      evictionQueueRequiresSort = true;
+      super.processAddedEntries(event);
+   }
+
+   @Override
+   protected void processVisitedEntries(EvictionEvent event) {
+      evictionQueueRequiresSort = true;
+      super.processVisitedEntries(event);
+   }
+
+   /**
+    * This method is called to resort the queue after add or visit events have occurred.
+    */
+   protected void resortEvictionQueue() {
+      long begin = System.currentTimeMillis();
+      ((LFUQueue) evictionQueue).reSortEvictionQueue();
+
+      if (trace) {
+         log.trace("Took {0} millis to re-sort queue with {1} elements",
+                   System.currentTimeMillis() - begin, getEvictionQueue().size());
+      }
+   }
 }

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUAlgorithmConfig.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUAlgorithmConfig.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUAlgorithmConfig.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -21,7 +21,6 @@
  */
 package org.horizon.eviction.algorithms.lfu;
 
-import org.horizon.config.Dynamic;
 import org.horizon.eviction.EvictionAlgorithmConfigBase;
 
 /**
@@ -36,59 +35,12 @@
     */
    private static final long serialVersionUID = 1865801530398969179L;
 
-   @Dynamic
-   private int minEntries;
-
    public LFUAlgorithmConfig() {
       evictionAlgorithmClassName = LFUAlgorithm.class.getName();
    }
 
-   public LFUAlgorithmConfig(int maxNodes, int minEntries) {
-      this();
-      setMaxEntries(maxNodes);
-      setMinEntries(minEntries);
-   }
-
-   public int getMinEntries() {
-      return minEntries;
-   }
-
-   public void setMinEntries(int minEntries) {
-      testImmutability("minEntries");
-      this.minEntries = minEntries;
-   }
-
    @Override
-   public String toString() {
-      StringBuilder ret = new StringBuilder();
-      ret.append("LFUAlgorithmConfig: maxEntries = ").append(getMaxEntries()).append(" minEntries = ").append(getMinEntries());
-      return ret.toString();
-   }
-
-   @Override
-   public boolean equals(Object obj) {
-      if (obj instanceof LFUAlgorithmConfig && super.equals(obj)) {
-         return (this.minEntries == ((LFUAlgorithmConfig) obj).minEntries);
-      }
-      return false;
-   }
-
-   @Override
-   public int hashCode() {
-      int result = super.hashCode();
-      result = 31 * result + minEntries;
-      return result;
-   }
-
-   @Override
    public LFUAlgorithmConfig clone() {
       return (LFUAlgorithmConfig) super.clone();
    }
-
-   @Override
-   public void reset() {
-      super.reset();
-      minEntries = -1;
-      evictionAlgorithmClassName = LFUAlgorithm.class.getName();
-   }
 }
\ No newline at end of file

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUQueue.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUQueue.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lfu/LFUQueue.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -22,7 +22,7 @@
 package org.horizon.eviction.algorithms.lfu;
 
 import org.horizon.eviction.EntryEvictionData;
-import org.horizon.eviction.algorithms.SortedEvictionQueue;
+import org.horizon.eviction.algorithms.BaseEvictionQueue;
 
 import java.util.Collections;
 import java.util.Comparator;
@@ -41,10 +41,10 @@
  * @author Daniel Huang (dhuang at jboss.org)
  * @since 1.0
  */
-public class LFUQueue implements SortedEvictionQueue {
-   private Map<Object, EntryEvictionData> keyMap;
-   private LinkedList<EntryEvictionData> evictionList;
-   private Set<EntryEvictionData> removalQueue;
+public class LFUQueue extends BaseEvictionQueue {
+   Map<Object, EntryEvictionData> keyMap;
+   LinkedList<EntryEvictionData> evictionList;
+   Set<EntryEvictionData> removalQueue;
    private Comparator<EntryEvictionData> comparator;
 
    protected LFUQueue() {
@@ -58,12 +58,12 @@
       return keyMap.get(key);
    }
 
-   public boolean contains(EntryEvictionData entryEvictionData) {
-      return keyMap.containsKey(entryEvictionData.getKey());
+   public boolean contains(Object key) {
+      return keyMap.containsKey(key);
    }
 
-   public void remove(EntryEvictionData entryEvictionData) {
-      EntryEvictionData data = keyMap.remove(entryEvictionData.getKey());
+   public void remove(Object key) {
+      EntryEvictionData data = keyMap.remove(key);
       if (data != null) {
          // don't remove directly from the LinkedList otherwise we will incur a O(n) = n
          // performance penalty for every removal! In the prune method for LFU, we will iterate the LinkedList through ONCE
@@ -96,15 +96,36 @@
       Collections.sort(evictionList, comparator);
    }
 
-   protected void prune() {
-      Iterator<EntryEvictionData> it = this.iterator();
+   /**
+    * Anything removed using {@link #remove(Object)} or removing using the iterator does not actually get removed until
+    * this method is called.
+    */
+   public void doBatchRemove() {
+      Iterator<EntryEvictionData> it = evictionList.iterator();
       while (it.hasNext() && removalQueue.size() > 0) {
          if (removalQueue.remove(it.next())) it.remove();
       }
    }
 
    public Iterator<EntryEvictionData> iterator() {
-      return evictionList.iterator();
+      //return evictionList.iterator();
+      return new Iterator<EntryEvictionData>() {
+         Iterator<EntryEvictionData> delegate = evictionList.iterator();
+         EntryEvictionData current;
+
+         public boolean hasNext() {
+            return delegate.hasNext();
+         }
+
+         public EntryEvictionData next() {
+            current = delegate.next();
+            return current;
+         }
+
+         public void remove() {
+            if (current != null) LFUQueue.this.remove(current.getKey());
+         }
+      };
    }
 
    /**

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUAlgorithm.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUAlgorithm.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUAlgorithm.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -21,6 +21,7 @@
  */
 package org.horizon.eviction.algorithms.lru;
 
+import net.jcip.annotations.NotThreadSafe;
 import org.horizon.config.EvictionAlgorithmConfig;
 import org.horizon.eviction.EntryEvictionData;
 import org.horizon.eviction.EvictionException;
@@ -32,90 +33,32 @@
 import java.util.Iterator;
 
 /**
- * Least recently Used algorithm to purge old data. Note that this algorithm is not thread-safe.
+ * Least recently Used algorithm to evict cache entries
  *
- * @author Ben Wang 02-2004
- * @author Daniel Huang - dhuang at jboss.org
+ * @author Manik Surtani
  * @since 1.0
  */
+ at NotThreadSafe
 public class LRUAlgorithm extends BaseEvictionAlgorithm {
    private static final Log log = LogFactory.getLog(LRUAlgorithm.class);
    private static final boolean trace = log.isTraceEnabled();
-   private LRUAlgorithmConfig lruAlgorithmConfig;
 
-
    @Override
    protected EvictionQueue setupEvictionQueue() throws EvictionException {
       return new LRUQueue();
    }
 
    @Override
-   protected boolean shouldEvictNode(EntryEvictionData entryEvictionData) {
-      // check the minimum time to live and see if we should not evict the node.  This check will
-      // ensure that, if configured, nodes are kept alive for at least a minimum period of time.
-      if (isYoungerThanMinimumTimeToLive(lruAlgorithmConfig.getMinTimeToLive(), entryEvictionData)) return false;
-
-      // no idle or max time limit
-      if (lruAlgorithmConfig.getTimeToLive() < 0 && lruAlgorithmConfig.getMaxAge() < 0) return false;
-
-      long currentTime = System.currentTimeMillis();
-      if (lruAlgorithmConfig.getTimeToLive() > -1) {
-         long idleTime = currentTime - entryEvictionData.getModifiedTimeStamp();
-         if (trace) {
-            log.trace("Node " + entryEvictionData.getKey() + " has been idle for " + idleTime + "ms");
-         }
-         if ((idleTime >= (lruAlgorithmConfig.getTimeToLive()))) {
-            if (trace) {
-               log.trace("Node " + entryEvictionData.getKey() + " should be evicted because of idle time");
-               log.trace("Time to live in millies is: " + (lruAlgorithmConfig.getTimeToLive()));
-               log.trace("Config instance is: " + System.identityHashCode(lruAlgorithmConfig));
-            }
-            return true;
-         }
-      }
-
-      if (lruAlgorithmConfig.getMaxAge() > -1) {
-         long objectLifeTime = currentTime - entryEvictionData.getCreationTimeStamp();
-         if (trace) {
-            log.trace("Node " + entryEvictionData.getKey() + " has been alive for " + objectLifeTime + "ms");
-         }
-         if ((objectLifeTime >= lruAlgorithmConfig.getMaxAge())) {
-            if (trace) {
-               log.trace("Node " + entryEvictionData.getKey() + " should be evicted because of max age");
-            }
-            return true;
-         }
-      }
-
-      if (trace) {
-         log.trace("Node " + entryEvictionData.getKey() + " should not be evicted");
-      }
-      return false;
-   }
-
-   @Override
-   protected void evict(EntryEvictionData data) {
-      if (data != null) {
-         if (!evictionAction.evict(data.getKey())) {
-            try {
-               recycleQueue.put(data.getKey());
-            }
-            catch (InterruptedException e) {
-               log.debug("InterruptedException", e);
-            }
-         }
-      }
-   }
-
-   @Override
    protected void prune() throws EvictionException {
       LRUQueue lruQueue = (LRUQueue) evictionQueue;
+      int origSize = evictionQueue.size();
+
       EntryEvictionData data;
       Iterator<EntryEvictionData> it = lruQueue.iterateLRUQueue();
       while (it.hasNext()) {
          data = it.next();
          if (data.isNodeInUseAndNotTimedOut()) continue;
-         if (this.shouldEvictNode(data)) {
+         if (shouldEvictEntry(data, origSize) && shouldNotOverrideEviction(data)) {
             it.remove();
             lruQueue.removeNodeEntryFromMaxAge(data);
             this.evict(data);
@@ -128,7 +71,7 @@
       while (it.hasNext()) {
          data = it.next();
          if (data.isNodeInUseAndNotTimedOut()) continue;
-         if (this.shouldEvictNode(data)) {
+         if (shouldEvictEntry(data, origSize) && shouldNotOverrideEviction(data)) {
             it.remove();
             lruQueue.removeNodeEntryFromLRU(data);
             this.evict(data);
@@ -137,20 +80,20 @@
          }
       }
 
-      int maxNodes = ((LRUAlgorithmConfig) evictionAlgorithmConfig).getMaxEntries();
+      int maxNodes = evictionAlgorithmConfig.getMaxEntries();
       if (maxNodes <= 0) {
          return;
       }
 
       it = lruQueue.iterateLRUQueue();
-      while (evictionQueue.size() > maxNodes) {
+      while (evictionQueue.size() > maxNodes && it.hasNext()) {
          data = it.next();
          if (trace) {
             log.trace("Node " + data.getKey() + " will be evicted because of exceeding the maxNode limit." +
                   " maxNode: " + maxNodes + " but current queue size is: " + evictionQueue.size());
          }
 
-         if (!data.isNodeInUseAndNotTimedOut()) {
+         if (!data.isNodeInUseAndNotTimedOut() && shouldNotOverrideEviction(data)) {
             it.remove();
             lruQueue.removeNodeEntryFromMaxAge(data);
             this.evict(data);
@@ -161,10 +104,4 @@
    public Class<? extends EvictionAlgorithmConfig> getConfigurationClass() {
       return LRUAlgorithmConfig.class;
    }
-
-   @Override
-   public void initialize() {
-      super.initialize();
-      lruAlgorithmConfig = (LRUAlgorithmConfig) evictionAlgorithmConfig;
-   }
 }

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUAlgorithmConfig.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUAlgorithmConfig.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUAlgorithmConfig.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -21,12 +21,8 @@
  */
 package org.horizon.eviction.algorithms.lru;
 
-import org.horizon.config.ConfigurationException;
-import org.horizon.config.Dynamic;
 import org.horizon.eviction.EvictionAlgorithmConfigBase;
 
-import java.util.concurrent.TimeUnit;
-
 /**
  * Configuration implementation for {@link LRUAlgorithm}.
  * <p/>
@@ -40,129 +36,11 @@
     */
    private static final long serialVersionUID = -3426716488271559729L;
 
-   /**
-    * value expressed in millis
-    */
-   @Dynamic
-   private long timeToLive;
-
-   /**
-    * value expressed in millis
-    */
-   @Dynamic
-   private long maxAge;
-
    public LRUAlgorithmConfig() {
       evictionAlgorithmClassName = LRUAlgorithm.class.getName();
-      // Force config of ttls
-      setTimeToLive(-1);
-      setMaxAge(-1);
    }
 
-   public LRUAlgorithmConfig(long timeToLive, long maxAge) {
-      this();
-      this.timeToLive = timeToLive;
-      this.maxAge = maxAge;
-   }
-
-   public LRUAlgorithmConfig(long timeToLive, int maxAge, int maxNodes) {
-      this(timeToLive, maxAge);
-      this.maxEntries = maxNodes;
-   }
-
-   /**
-    * @return the time to live, in milliseconds
-    */
-   public long getTimeToLive() {
-      return timeToLive;
-   }
-
-   /**
-    * Sets the time to live
-    *
-    * @param timeToLive the time to live, in milliseconds
-    */
-   public void setTimeToLive(long timeToLive) {
-      testImmutability("timeToLive");
-      this.timeToLive = timeToLive;
-   }
-
-   public void setTimeToLive(long timeToLive, TimeUnit timeUnit) {
-      testImmutability("timeToLive");
-      this.timeToLive = timeUnit.toMillis(timeToLive);
-   }
-
-   /**
-    * @return the max age per element, in milliseconds
-    */
-   public long getMaxAge() {
-      return maxAge;
-   }
-
-   /**
-    * Sets the max age per element
-    *
-    * @param maxAge value in milliseconds
-    */
-   public void setMaxAge(long maxAge) {
-      testImmutability("maxAge");
-      this.maxAge = maxAge;
-   }
-
-   public void setMaxAge(long maxAge, TimeUnit timeUnit) {
-      testImmutability("maxAge");
-      this.maxAge = timeUnit.toMillis(maxAge);
-   }
-
-   /**
-    * Requires a positive timeToLive value or ConfigurationException is thrown.
-    */
    @Override
-   public void validate() throws ConfigurationException {
-      super.validate();
-      if (timeToLive < -1) {
-         throw new ConfigurationException("timeToLive must be " +
-               "configured to a value greater than or equal to 0 (or -1 for unlimited time to live) for " + getEvictionAlgorithmClassName());
-      }
-   }
-
-   public String toString() {
-      return "LRUAlgorithmConfig {" +
-            "algorithmClassName=" + evictionAlgorithmClassName +
-            ", timeToLive=" + timeToLive +
-            ", maxAge=" + maxAge +
-            '}';
-   }
-
-   public boolean equals(Object o) {
-      if (this == o) return true;
-      if (!(o instanceof LRUAlgorithmConfig)) return false;
-      if (!super.equals(o)) return false;
-
-      LRUAlgorithmConfig that = (LRUAlgorithmConfig) o;
-
-      if (maxAge != that.maxAge) return false;
-      if (timeToLive != that.timeToLive) return false;
-
-      return true;
-   }
-
-   public int hashCode() {
-      int result = super.hashCode();
-      result = 31 * result + (int) (timeToLive ^ (timeToLive >>> 32));
-      result = 31 * result + (int) (maxAge ^ (maxAge >>> 32));
-      return result;
-   }
-
-   @Override
-   public void reset() {
-      super.reset();
-      setTimeToLive(-1);
-      setMaxAge(-1);
-      evictionAlgorithmClassName = LRUAlgorithm.class.getName();
-   }
-
-   @Override
    public LRUAlgorithmConfig clone() {
       return (LRUAlgorithmConfig) super.clone();
    }

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUQueue.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUQueue.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/lru/LRUQueue.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -22,7 +22,7 @@
 package org.horizon.eviction.algorithms.lru;
 
 import org.horizon.eviction.EntryEvictionData;
-import org.horizon.eviction.EvictionQueue;
+import org.horizon.eviction.algorithms.BaseEvictionQueue;
 
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -37,9 +37,9 @@
  * @author Daniel Huang (dhuang at jboss.org)
  * @since 1.0
  */
-public class LRUQueue implements EvictionQueue {
-   private Map<Object, EntryEvictionData> maxAgeQueue;
-   private Map<Object, EntryEvictionData> lruQueue;
+public class LRUQueue extends BaseEvictionQueue {
+   Map<Object, EntryEvictionData> maxAgeQueue;
+   Map<Object, EntryEvictionData> lruQueue;
 
    protected LRUQueue() {
       maxAgeQueue = new LinkedHashMap<Object, EntryEvictionData>();
@@ -50,13 +50,12 @@
       return lruQueue.get(key);
    }
 
-   public boolean contains(EntryEvictionData entryEvictionData) {
-      return this.maxAgeQueue.containsKey(entryEvictionData.getKey());
+   public boolean contains(Object key) {
+      return maxAgeQueue.containsKey(key);
    }
 
-   public void remove(EntryEvictionData entryEvictionData) {
-      if (contains(entryEvictionData)) {
-         Object key = entryEvictionData.getKey();
+   public void remove(Object key) {
+      if (contains(key)) {
          EntryEvictionData data1 = lruQueue.remove(key);
          EntryEvictionData data2 = maxAgeQueue.remove(key);
          if (data1 == null || data2 == null) {
@@ -83,11 +82,11 @@
    }
 
    public Iterator<EntryEvictionData> iterator() {
-      return lruQueue.values().iterator();
+      return new LRUQueueIterator(lruQueue, maxAgeQueue);
    }
 
    public Iterator<EntryEvictionData> iterateLRUQueue() {
-      return lruQueue.values().iterator();
+      return new LRUQueueIterator(lruQueue, maxAgeQueue);
    }
 
    public void removeNodeEntryFromMaxAge(EntryEvictionData data) {
@@ -95,10 +94,37 @@
    }
 
    public Iterator<EntryEvictionData> iterateMaxAgeQueue() {
-      return maxAgeQueue.values().iterator();
+      return new LRUQueueIterator(maxAgeQueue, lruQueue);
    }
 
    public void removeNodeEntryFromLRU(EntryEvictionData data) {
       lruQueue.remove(data.getKey());
    }
+
+   private class LRUQueueIterator implements Iterator<EntryEvictionData> {
+      Iterator<EntryEvictionData> delegate;
+      Map<Object, EntryEvictionData> secondaryStructure;
+      EntryEvictionData current;
+
+      private LRUQueueIterator(Map<Object, EntryEvictionData> primaryStructure, Map<Object, EntryEvictionData> secondaryStructure) {
+         delegate = primaryStructure.values().iterator();
+         this.secondaryStructure = secondaryStructure;
+      }
+
+      public boolean hasNext() {
+         return delegate.hasNext();
+      }
+
+      public EntryEvictionData next() {
+         current = delegate.next();
+         return current;
+      }
+
+      public void remove() {
+         delegate.remove();
+         if (current != null) {
+            maxAgeQueue.remove(current.getKey());
+         }
+      }
+   }
 }

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUAlgorithm.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUAlgorithm.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUAlgorithm.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -21,8 +21,8 @@
  */
 package org.horizon.eviction.algorithms.mru;
 
+import net.jcip.annotations.NotThreadSafe;
 import org.horizon.config.EvictionAlgorithmConfig;
-import org.horizon.eviction.EntryEvictionData;
 import org.horizon.eviction.EvictionEvent;
 import org.horizon.eviction.EvictionException;
 import org.horizon.eviction.EvictionQueue;
@@ -32,43 +32,24 @@
  * Most Recently Used Algorithm.
  * <p/>
  * This algorithm will evict the most recently used cache entries from cache.
- * <p/>
- * Note: None of the Eviction classes are thread safe. It is assumed that an individual instance of an EvictionPolicy/
- * EvictionAlgorithm/EvictionQueue/EvictionConfiguration are only operated on by one thread at any given time.
  *
- * @author Daniel Huang (dhuang at jboss.org)
+ * @author Manik Surtani
  * @since 1.0
  */
+ at NotThreadSafe
 public class MRUAlgorithm extends BaseEvictionAlgorithm {
-
-   MRUAlgorithmConfig mruAlgorithmConfig;
-
    @Override
    protected EvictionQueue setupEvictionQueue() throws EvictionException {
       return new MRUQueue();
    }
 
    @Override
-   protected boolean shouldEvictNode(EntryEvictionData data) {
-      // check the minimum time to live and see if we should not evict the node.  This check will
-      // ensure that, if configured, nodes are kept alive for at least a minimum period of time.
-      return !isYoungerThanMinimumTimeToLive(mruAlgorithmConfig.getMinTimeToLive(), data) &&
-            evictionQueue.size() > mruAlgorithmConfig.getMaxEntries();
-   }
-
-   @Override
-   protected void processVisitedNodes(EvictionEvent evictedEventNode) throws EvictionException {
-      super.processVisitedNodes(evictedEventNode);
+   protected void processVisitedEntries(EvictionEvent evictedEventNode) throws EvictionException {
+      super.processVisitedEntries(evictedEventNode);
       ((MRUQueue) evictionQueue).moveToTopOfStack(evictedEventNode.getKey());
    }
 
    public Class<? extends EvictionAlgorithmConfig> getConfigurationClass() {
       return MRUAlgorithmConfig.class;
    }
-
-   @Override
-   public void initialize() {
-      super.initialize();
-      mruAlgorithmConfig = (MRUAlgorithmConfig) evictionAlgorithmConfig;
-   }
 }

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUAlgorithmConfig.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUAlgorithmConfig.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUAlgorithmConfig.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -21,13 +21,10 @@
  */
 package org.horizon.eviction.algorithms.mru;
 
-import org.horizon.config.ConfigurationException;
 import org.horizon.eviction.EvictionAlgorithmConfigBase;
 
 /**
  * Configuration for {@link MRUAlgorithm}.
- * <p/>
- * Requires a "maxEntries" attribute otherwise a ConfigurationException is thrown.
  *
  * @author Manik Surtani
  * @since 1.0
@@ -40,48 +37,10 @@
 
    public MRUAlgorithmConfig() {
       evictionAlgorithmClassName = MRUAlgorithm.class.getName();
-      // We require that maxEntries is set
-      setMaxEntries(-1);
    }
 
-   public MRUAlgorithmConfig(int maxNodes) {
-      evictionAlgorithmClassName = MRUAlgorithm.class.getName();
-      setMaxEntries(maxNodes);
-   }
-
-   /**
-    * Requires a positive maxNodes value or ConfigurationException is thrown.
-    */
    @Override
-   public void validate() throws ConfigurationException {
-      super.validate();
-      if (getMaxEntries() < 0)
-         throw new ConfigurationException("maxEntries not configured");
-   }
-
-   @Override
-   public String toString() {
-      StringBuilder str = new StringBuilder();
-      str.append("MRUAlgorithmConfig: ").
-            append(" maxEntries =").append(getMaxEntries());
-      return str.toString();
-   }
-
-   @Override
-   public boolean equals(Object obj) {
-      return (obj instanceof MRUAlgorithmConfig && super.equals(obj));
-   }
-
-   @Override
-   public void reset() {
-      super.reset();
-      setMaxEntries(-1);
-      evictionAlgorithmClassName = MRUAlgorithm.class.getName();
-   }
-
-   @Override
    public MRUAlgorithmConfig clone() {
       return (MRUAlgorithmConfig) super.clone();
    }
-
 }
\ No newline at end of file

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUQueue.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUQueue.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/mru/MRUQueue.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -22,7 +22,7 @@
 package org.horizon.eviction.algorithms.mru;
 
 import org.horizon.eviction.EntryEvictionData;
-import org.horizon.eviction.EvictionQueue;
+import org.horizon.eviction.algorithms.BaseEvictionQueue;
 
 import java.util.HashMap;
 import java.util.Iterator;
@@ -39,12 +39,12 @@
  * @author Daniel Huang (dhuang at jboss.org)
  * @since 1.0
  */
-public class MRUQueue implements EvictionQueue {
+public class MRUQueue extends BaseEvictionQueue {
    // we use our own Stack/Linked List implementation here because it guarantees O(n) = 1 for add, remove, get and
    // we can sort it in order of MRU implicitly while still getting O(n) = 1 access time
    // throughout.
    Map<Object, EntryEvictionData> keyMap;
-   private LinkedList<EntryEvictionData> list;
+   LinkedList<EntryEvictionData> list;
 
    protected MRUQueue() {
       keyMap = new HashMap<Object, EntryEvictionData>();
@@ -70,12 +70,12 @@
       return keyMap.get(key);
    }
 
-   public boolean contains(EntryEvictionData entryEvictionData) {
-      return keyMap.containsKey(entryEvictionData.getKey());
+   public boolean contains(Object key) {
+      return keyMap.containsKey(key);
    }
 
-   public void remove(EntryEvictionData entryEvictionData) {
-      EntryEvictionData data = keyMap.remove(entryEvictionData.getKey());
+   public void remove(Object key) {
+      EntryEvictionData data = keyMap.remove(key);
       if (data != null) list.remove(data);
    }
 
@@ -96,7 +96,28 @@
    }
 
    public Iterator<EntryEvictionData> iterator() {
-      return list.iterator();
+      return new Iterator<EntryEvictionData>() {
+         Iterator<EntryEvictionData> delegate = list.iterator();
+         EntryEvictionData current;
+
+         public boolean hasNext() {
+            return delegate.hasNext();
+         }
+
+         public EntryEvictionData next() {
+            current = delegate.next();
+            return current;
+         }
+
+         public void remove() {
+            // this will remove it from the list
+            delegate.remove();
+            // now to take care of the key map
+            if (current != null) {
+               keyMap.remove(current.getKey());
+            }
+         }
+      };
    }
 
    @Override

Modified: core/branches/flat/src/main/java/org/horizon/eviction/algorithms/nullalgo/NullEvictionQueue.java
===================================================================
--- core/branches/flat/src/main/java/org/horizon/eviction/algorithms/nullalgo/NullEvictionQueue.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/main/java/org/horizon/eviction/algorithms/nullalgo/NullEvictionQueue.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -22,7 +22,7 @@
 package org.horizon.eviction.algorithms.nullalgo;
 
 import org.horizon.eviction.EntryEvictionData;
-import org.horizon.eviction.EvictionQueue;
+import org.horizon.eviction.algorithms.BaseEvictionQueue;
 
 import java.util.Iterator;
 import java.util.NoSuchElementException;
@@ -33,7 +33,7 @@
  * @author Brian Stansberry
  * @since 1.0
  */
-public class NullEvictionQueue implements EvictionQueue {
+public class NullEvictionQueue extends BaseEvictionQueue {
    /**
     * Singleton instance of this class.
     */
@@ -66,7 +66,7 @@
    /**
     * Returns <code>false</code>
     */
-   public boolean contains(EntryEvictionData entryEvictionData) {
+   public boolean contains(Object key) {
       return false;
    }
 
@@ -87,7 +87,7 @@
    /**
     * No-op
     */
-   public void remove(EntryEvictionData entryEvictionData) {
+   public void remove(Object key) {
       // no-op
    }
 

Added: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/BaseAlgorithmTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/BaseAlgorithmTest.java	                        (rev 0)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/BaseAlgorithmTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -0,0 +1,260 @@
+package org.horizon.eviction.algorithms;
+
+import org.easymock.EasyMock;
+import static org.easymock.EasyMock.*;
+import org.horizon.config.EvictionAlgorithmConfig;
+import org.horizon.eviction.EntryEvictionData;
+import org.horizon.eviction.EvictionAction;
+import org.horizon.eviction.EvictionAlgorithm;
+import org.horizon.eviction.EvictionAlgorithmConfigBase;
+import org.horizon.eviction.EvictionEvent;
+import static org.horizon.eviction.EvictionEvent.Type.ADD_ENTRY_EVENT;
+import org.horizon.util.Util;
+import org.testng.annotations.Test;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+ at Test(groups = "unit")
+public abstract class BaseAlgorithmTest {
+
+   protected abstract EvictionAlgorithmConfigBase getNewEvictionAlgorithmConfig();
+
+   protected EvictionAlgorithm createAndInit(EvictionAlgorithmConfig cfg) throws Exception {
+      EvictionAlgorithm a = (EvictionAlgorithm) Util.getInstance(cfg.getEvictionAlgorithmClassName());
+      a.assignToCache(null, cfg);
+      return a;
+   }
+
+   public void testMinTimeToLive() throws Exception {
+      EvictionAlgorithmConfigBase cfg = getNewEvictionAlgorithmConfig();
+      cfg.setMinTimeToLive(2 * 60 * 60 * 1000); // something enormous - 2 hrs
+      cfg.setMaxEntries(5);
+      EvictionAlgorithm a = createAndInit(cfg);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      a.setEvictionAction(mockAction);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      for (int i = 0; i < 10; i++) eventQueue.put(new EvictionEvent(i, EvictionEvent.Type.ADD_ENTRY_EVENT));
+
+      assert eventQueue.size() == 10;
+
+      // what do we expect to happen on the eviction action class?
+      // nothing at this stage.
+      replay(mockAction);
+      a.process(eventQueue);
+      verify(mockAction);
+
+      for (EntryEvictionData data : a.getEvictionQueue()) {
+         // change the creation timestamp to before 2 hrs in the past
+         // for all even keys
+         Integer key = (Integer) data.getKey();
+
+         if (key % 2 == 0) {
+            data.setCreationTimeStamp(1);
+            data.setModifiedTimeStamp(1);
+            EvictionEvent e = new EvictionEvent(key, EvictionEvent.Type.VISIT_ENTRY_EVENT);
+            e.setCreationTimestamp(1);
+            eventQueue.put(e);
+         } else {
+            eventQueue.put(new EvictionEvent(key, EvictionEvent.Type.VISIT_ENTRY_EVENT));
+         }
+      }
+
+      assert eventQueue.size() == 10;
+
+      // this time we expect all even numbered keys to get evicted.
+      reset(mockAction);
+      expect(mockAction.evict(eq(0))).andReturn(true).once();
+      expect(mockAction.evict(eq(2))).andReturn(true).once();
+      expect(mockAction.evict(eq(4))).andReturn(true).once();
+      expect(mockAction.evict(eq(6))).andReturn(true).once();
+      expect(mockAction.evict(eq(8))).andReturn(true).once();
+      replay(mockAction);
+      a.process(eventQueue);
+      verify(mockAction);
+   }
+
+   public void testNumEntries1() throws Exception {
+      EvictionAlgorithmConfigBase config = getNewEvictionAlgorithmConfig();
+      config.setMaxEntries(4);
+      config.setMinEntries(2);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      eventQueue.put(new EvictionEvent("one", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("two", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("three", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("four", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("five", ADD_ENTRY_EVENT));
+      EvictionAlgorithm algo = createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      // should prune down to 2 entries
+      EasyMock.expect(mockAction.evict(eq("one"))).andReturn(true).once();
+      EasyMock.expect(mockAction.evict(eq("two"))).andReturn(true).once();
+      EasyMock.expect(mockAction.evict(eq("three"))).andReturn(true).once();
+      EasyMock.replay(mockAction);
+      algo.process(eventQueue);
+      EasyMock.verify(mockAction);
+
+   }
+
+   public void testNumEntries2() throws Exception {
+      EvictionAlgorithmConfigBase config = getNewEvictionAlgorithmConfig();
+      config.setMaxEntries(0);
+      config.setMinEntries(20);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      eventQueue.put(new EvictionEvent("one", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("two", ADD_ENTRY_EVENT));
+      EvictionAlgorithm algo = createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      // should prune down to 2 entries
+      EasyMock.expect(mockAction.evict(eq("one"))).andReturn(true).once();
+      EasyMock.expect(mockAction.evict(eq("two"))).andReturn(true).once();
+      EasyMock.replay(mockAction);
+      algo.process(eventQueue);
+      EasyMock.verify(mockAction);
+   }
+
+   public void testNumEntries3() throws Exception {
+      EvictionAlgorithmConfigBase config = getNewEvictionAlgorithmConfig();
+      config.setMaxEntries(3);
+      config.setMinEntries(20);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      eventQueue.put(new EvictionEvent("one", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("two", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("three", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("four", ADD_ENTRY_EVENT));
+      EvictionAlgorithm algo = createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      // should prune down to equal to maxEntries
+      EasyMock.expect(mockAction.evict(eq("one"))).andReturn(true).once();
+      EasyMock.replay(mockAction);
+      algo.process(eventQueue);
+      EasyMock.verify(mockAction);
+   }
+
+   public void testNumEntries4() throws Exception {
+      EvictionAlgorithmConfigBase config = getNewEvictionAlgorithmConfig();
+      config.setMaxEntries(-1);
+      config.setMinEntries(2);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      eventQueue.put(new EvictionEvent("one", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("two", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("three", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("four", ADD_ENTRY_EVENT));
+      EvictionAlgorithm algo = createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      // expecting nothing to be evicted
+      EasyMock.replay(mockAction);
+      algo.process(eventQueue);
+      EasyMock.verify(mockAction);
+   }
+
+   public void testNumEntries5() throws Exception {
+      EvictionAlgorithmConfigBase config = getNewEvictionAlgorithmConfig();
+      config.setMaxEntries(3);
+      config.setMinEntries(-1);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      eventQueue.put(new EvictionEvent("one", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("two", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("three", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("four", ADD_ENTRY_EVENT));
+      EvictionAlgorithm algo = createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      // should prune down to equal to maxEntries
+      EasyMock.expect(mockAction.evict(eq("one"))).andReturn(true).once();
+      EasyMock.replay(mockAction);
+      algo.process(eventQueue);
+      EasyMock.verify(mockAction);
+   }
+
+   public void testTimeToLive1() throws Exception {
+      EvictionAlgorithmConfigBase config = getNewEvictionAlgorithmConfig();
+      config.setTimeToLive(-1);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      eventQueue.put(new EvictionEvent("one", ADD_ENTRY_EVENT, 1));
+      eventQueue.put(new EvictionEvent("two", ADD_ENTRY_EVENT, 1));
+      eventQueue.put(new EvictionEvent("three", ADD_ENTRY_EVENT, 1));
+      eventQueue.put(new EvictionEvent("four", ADD_ENTRY_EVENT, 1));
+      EvictionAlgorithm algo = createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      // Should not select any
+      EasyMock.replay(mockAction);
+      algo.process(eventQueue);
+      EasyMock.verify(mockAction);
+
+   }
+
+   public void testTimeToLive2() throws Exception {
+      EvictionAlgorithmConfigBase config = getNewEvictionAlgorithmConfig();
+      config.setTimeToLive(0);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      eventQueue.put(new EvictionEvent("one", ADD_ENTRY_EVENT, 1));
+      eventQueue.put(new EvictionEvent("two", ADD_ENTRY_EVENT, System.currentTimeMillis() * 2));
+      eventQueue.put(new EvictionEvent("three", ADD_ENTRY_EVENT, 1));
+      eventQueue.put(new EvictionEvent("four", ADD_ENTRY_EVENT, System.currentTimeMillis() * 2));
+      EvictionAlgorithm algo = createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      // only the first one is selected since the others are not considered since max entries has not been hit.
+      EasyMock.expect(mockAction.evict(eq("one"))).andReturn(true).once();
+      EasyMock.replay(mockAction);
+      algo.process(eventQueue);
+      EasyMock.verify(mockAction);
+   }
+
+   public void testMaxAge1() throws Exception {
+      EvictionAlgorithmConfigBase config = getNewEvictionAlgorithmConfig();
+      config.setMaxAge(-1);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      eventQueue.put(new EvictionEvent("one", ADD_ENTRY_EVENT, 1));
+      eventQueue.put(new EvictionEvent("two", ADD_ENTRY_EVENT, 1));
+      eventQueue.put(new EvictionEvent("three", ADD_ENTRY_EVENT, 1));
+      eventQueue.put(new EvictionEvent("four", ADD_ENTRY_EVENT, 1));
+      EvictionAlgorithm algo = createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      // Should not select any
+      EasyMock.replay(mockAction);
+      algo.process(eventQueue);
+      EasyMock.verify(mockAction);
+
+   }
+
+   public void testMaxAge2() throws Exception {
+      EvictionAlgorithmConfigBase config = getNewEvictionAlgorithmConfig();
+      config.setMaxAge(10);
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      eventQueue.put(new EvictionEvent("one", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("two", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("three", ADD_ENTRY_EVENT));
+      eventQueue.put(new EvictionEvent("four", ADD_ENTRY_EVENT));
+      Thread.sleep(10);
+      EvictionAlgorithm algo = createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      // all of them should have crossed max age by now
+      EasyMock.expect(mockAction.evict(eq("one"))).andReturn(true).once();
+      EasyMock.expect(mockAction.evict(eq("two"))).andReturn(true).once();
+      EasyMock.expect(mockAction.evict(eq("three"))).andReturn(true).once();
+      EasyMock.expect(mockAction.evict(eq("four"))).andReturn(true).once();
+      EasyMock.replay(mockAction);
+      algo.process(eventQueue);
+      EasyMock.verify(mockAction);
+   }
+
+
+}

Added: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/BaseQueueTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/BaseQueueTest.java	                        (rev 0)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/BaseQueueTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -0,0 +1,50 @@
+package org.horizon.eviction.algorithms;
+
+import org.horizon.eviction.EntryEvictionData;
+import org.horizon.eviction.EvictionQueue;
+import org.testng.annotations.Test;
+
+ at Test(groups = "unit")
+public abstract class BaseQueueTest {
+   protected EntryEvictionData getFirstEntry(EvictionQueue q) {
+      return q.iterator().next();
+   }
+
+   protected Object getFirstEntryKey(EvictionQueue q) {
+      return q.iterator().next().getKey();
+   }
+
+   protected void fillQueue(EvictionQueue q, int numEntries) {
+      for (int i = 0; i < numEntries; i++) q.add(new EntryEvictionData(i));
+   }
+
+   protected abstract EvictionQueue getNewEvictionQueue();
+
+   public void testSizingAndContents() {
+      EvictionQueue q = getNewEvictionQueue();
+
+      assert q.isEmpty();
+      assert q.size() == 0;
+
+      fillQueue(q, 1000);
+
+      assert !q.isEmpty();
+      assert 1000 == q.size();
+
+      assert q.contains(1);
+      assert q.contains(999);
+      assert !q.contains(1000);
+
+      for (int i = 0; i < 1000; i++) {
+         assert q.get(i).getKey().equals(i);
+      }
+
+      for (int i = 0; i < 1000; i++) {
+         assert q.size() == 1000 - i;
+         q.remove(i);
+      }
+
+      assert q.isEmpty();
+      assert q.size() == 0;
+   }
+}

Modified: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/fifo/FifoAlgorithmTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/fifo/FifoAlgorithmTest.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/fifo/FifoAlgorithmTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,13 +1,63 @@
 package org.horizon.eviction.algorithms.fifo;
 
+import org.easymock.EasyMock;
+import static org.easymock.EasyMock.*;
+import org.horizon.eviction.EntryEvictionData;
+import org.horizon.eviction.EvictionAction;
+import org.horizon.eviction.EvictionEvent;
+import org.horizon.eviction.algorithms.BaseAlgorithmTest;
 import org.testng.annotations.Test;
 
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
 @Test(groups = "unit")
-public class FifoAlgorithmTest {
-   public void testAlgorithm() {
-      FIFOAlgorithm algo = new FIFOAlgorithm();
-      algo.initialize();
-      assert algo.getConfigurationClass().equals(FIFOAlgorithmConfig.class);
-//      assert algo.process();
+public class FifoAlgorithmTest extends BaseAlgorithmTest {
+
+   public void testMaxEntries() throws Exception {
+      FIFOAlgorithmConfig config = new FIFOAlgorithmConfig();
+      config.setMaxEntries(5);
+
+      FIFOAlgorithm algo = (FIFOAlgorithm) createAndInit(config);
+      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+      algo.setEvictionAction(mockAction);
+
+      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+      for (int i = 0; i < 10; i++) eventQueue.put(new EvictionEvent(i, EvictionEvent.Type.ADD_ENTRY_EVENT));
+
+      for (int i = 0; i < 5; i++) expect(mockAction.evict(eq(i))).andReturn(true).once();
+      replay(mockAction);
+      algo.process(eventQueue);
+      verify(mockAction);
+
+      // now verify the order.
+      int index = 5;
+      for (EntryEvictionData data : algo.getEvictionQueue()) {
+         assert data.getKey().equals(index);
+         index++;
+      }
+
+      // now verify the same order still exists after visiting the entries
+      for (int i = 5; i < 10; i++) {
+         eventQueue.put(new EvictionEvent(i, EvictionEvent.Type.VISIT_ENTRY_EVENT));
+      }
+      for (int i = 5; i < 10; i++) {
+         eventQueue.put(new EvictionEvent(i, EvictionEvent.Type.VISIT_ENTRY_EVENT));
+      }
+
+      algo.process(eventQueue);
+
+      assert 5 == algo.getEvictionQueue().size();
+
+      // now verify the order.
+      index = 5;
+      for (EntryEvictionData data : algo.getEvictionQueue()) {
+         assert data.getKey().equals(index);
+         index++;
+      }
    }
+
+   protected FIFOAlgorithmConfig getNewEvictionAlgorithmConfig() {
+      return new FIFOAlgorithmConfig();
+   }
 }

Modified: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/fifo/FifoQueueTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/fifo/FifoQueueTest.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/fifo/FifoQueueTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,7 +1,50 @@
 package org.horizon.eviction.algorithms.fifo;
 
+import org.horizon.eviction.EntryEvictionData;
+import org.horizon.eviction.EvictionQueue;
+import org.horizon.eviction.algorithms.BaseQueueTest;
 import org.testng.annotations.Test;
 
+import java.util.Iterator;
+
 @Test(groups = "unit")
-public class FifoQueueTest {
+public class FifoQueueTest extends BaseQueueTest {
+   public void testOrder() throws Exception {
+      FIFOQueue queue = new FIFOQueue();
+
+      fillQueue(queue, 5000);
+
+      assert getFirstEntryKey(queue).equals(0);
+
+      // now make sure the ordering is correct.
+      int k = 0;
+      for (Iterator<EntryEvictionData> i = queue.iterator(); i.hasNext();) {
+         EntryEvictionData data = i.next();
+         assert data.getKey().equals(k);
+         i.remove();
+         k++;
+         if (k == 2500) break;
+      }
+
+      assert getFirstEntryKey(queue).equals(2500);
+
+      assert 2500 == queue.size();
+      assert !queue.isEmpty();
+
+      k = 2500;
+
+      for (Iterator<EntryEvictionData> i = queue.iterator(); i.hasNext();) {
+         EntryEvictionData data = i.next();
+         assert data.getKey().equals(k);
+         i.remove();
+         k++;
+      }
+
+      assert 0 == queue.size();
+      assert queue.isEmpty();
+   }
+
+   protected EvictionQueue getNewEvictionQueue() {
+      return new FIFOQueue();
+   }
 }

Modified: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lfu/LfuAlgorithmTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lfu/LfuAlgorithmTest.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lfu/LfuAlgorithmTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,7 +1,60 @@
 package org.horizon.eviction.algorithms.lfu;
 
+import org.horizon.eviction.algorithms.BaseAlgorithmTest;
 import org.testng.annotations.Test;
 
 @Test(groups = "unit")
-public class LfuAlgorithmTest {
+public class LfuAlgorithmTest extends BaseAlgorithmTest {
+//   public void testMinTimeToLive() throws Exception {
+//      LFUAlgorithmConfig cfg = getNewEvictionAlgorithmConfig();
+//      cfg.setMinTimeToLive(2 * 60 * 60 * 1000); // something enormous - 2 hrs
+//      cfg.setMaxEntries(5);
+//      EvictionAlgorithm a = createAndInit(cfg);
+//      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+//      a.setEvictionAction(mockAction);
+//      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+//      for (int i = 0; i < 10; i++) eventQueue.put(new EvictionEvent(i, ADD_ENTRY_EVENT));
+//
+//      assert eventQueue.size() == 10;
+//
+//      // what do we expect to happen on the eviction action class?
+//      // nothing at this stage.
+//      replay(mockAction);
+//      a.process(eventQueue);
+//      verify(mockAction);
+//
+//      for (EntryEvictionData data : a.getEvictionQueue()) {
+//         // change the creation timestamp to before 2 hrs in the past
+//         // for all even keys
+//         Integer key = (Integer) data.getKey();
+//
+//         if (key % 2 == 0) {
+//            EvictionEvent e = new EvictionEvent(key, EvictionEvent.Type.VISIT_ENTRY_EVENT);
+//            e.setCreationTimestamp(1);
+//            data.setCreationTimeStamp(1);
+//            data.setModifiedTimeStamp(1);
+//            eventQueue.put(e);
+//         } else {
+//            eventQueue.put(new EvictionEvent(key, EvictionEvent.Type.VISIT_ENTRY_EVENT));
+//         }
+//      }
+//
+//      assert eventQueue.size() == 10;
+//
+//      // this time we expect all even numbered keys to get evicted, since they are least frequently visited
+//      reset(mockAction);
+//      expect(mockAction.evict(eq(0))).andReturn(true).once();
+//      expect(mockAction.evict(eq(2))).andReturn(true).once();
+//      expect(mockAction.evict(eq(4))).andReturn(true).once();
+//      expect(mockAction.evict(eq(6))).andReturn(true).once();
+//      expect(mockAction.evict(eq(8))).andReturn(true).once();
+//      replay(mockAction);
+//      a.process(eventQueue);
+//      verify(mockAction);
+//   }
+
+
+   protected LFUAlgorithmConfig getNewEvictionAlgorithmConfig() {
+      return new LFUAlgorithmConfig();
+   }
 }

Modified: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lfu/LfuQueueTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lfu/LfuQueueTest.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lfu/LfuQueueTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,7 +1,109 @@
 package org.horizon.eviction.algorithms.lfu;
 
+import org.horizon.eviction.EntryEvictionData;
+import org.horizon.eviction.EvictionQueue;
+import org.horizon.eviction.algorithms.BaseQueueTest;
 import org.testng.annotations.Test;
 
+import java.util.Iterator;
+
 @Test(groups = "unit")
-public class LfuQueueTest {
+public class LfuQueueTest extends BaseQueueTest {
+   public void testOrder() throws Exception {
+      LFUQueue queue = (LFUQueue) getNewEvictionQueue();
+      fillQueue(queue, 500);
+      assert 500 == queue.size();
+
+      queue.reSortEvictionQueue();
+
+      assert 500 == queue.size();
+
+      int k = 0;
+      for (EntryEvictionData entry : queue) {
+         assert entry.getKey().equals(k);
+         if (k % 2 == 0) entry.incrementNumberOfVisits();
+         k++;
+      }
+
+      queue.reSortEvictionQueue();
+
+      assert getFirstEntryKey(queue).equals(1);
+
+      // now check the sort order.
+      k = 0;
+      for (EntryEvictionData entry : queue) {
+         if (k < 250)
+            assert 0 == entry.getNumberOfVisits();
+         else
+            assert 1 == entry.getNumberOfVisits();
+         k++;
+      }
+
+      k = 0;
+      for (Iterator<EntryEvictionData> it = queue.iterator(); it.hasNext() && it.next() != null;) {
+         if (k == 250) break;
+         it.remove();
+         k++;
+      }
+
+      queue.doBatchRemove();
+
+      assert 250 == queue.size();
+      assert !queue.contains(275);
+
+      for (int i = 0; i < 500; i++) {
+         if (i % 2 == 0) {
+            EntryEvictionData data = queue.get(i);
+            assert 1 == data.getNumberOfVisits();
+            if (i > 250) data.incrementNumberOfVisits();
+         }
+      }
+
+      queue.reSortEvictionQueue();
+      assert 250 == queue.size();
+
+      k = 0;
+      for (EntryEvictionData entry : queue) {
+         if (k <= 125)
+            assert 1 == entry.getNumberOfVisits();
+         else
+            assert 2 == entry.getNumberOfVisits();
+         k++;
+      }
+   }
+
+   public void testPrune() throws Exception {
+      LFUQueue queue = (LFUQueue) getNewEvictionQueue();
+      fillQueue(queue, 500);
+
+      assert 500 == queue.size();
+      assert 500 == queue.evictionList.size();
+      assert 500 == queue.keyMap.size();
+
+      int i = 0;
+      for (Iterator it = queue.iterator(); it.hasNext() && it.next() != null;) {
+         if (i % 2 == 0) it.remove();
+         i++;
+      }
+      assert 250 == queue.size();
+
+      for (EntryEvictionData data : queue.removalQueue) {
+         int currentIndex = (Integer) data.getKey();
+         assert 0 == currentIndex % 2;
+
+         assert !queue.contains(currentIndex);
+         assert queue.evictionList.contains(data);
+      }
+
+      assert 500 == queue.evictionList.size();
+
+      queue.doBatchRemove();
+
+      assert 0 == queue.removalQueue.size();
+      assert 250 == queue.evictionList.size();
+   }
+
+   protected EvictionQueue getNewEvictionQueue() {
+      return new LFUQueue();
+   }
 }

Modified: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lru/LruAlgorithmTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lru/LruAlgorithmTest.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lru/LruAlgorithmTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,7 +1,60 @@
 package org.horizon.eviction.algorithms.lru;
 
+import org.horizon.eviction.algorithms.BaseAlgorithmTest;
 import org.testng.annotations.Test;
 
 @Test(groups = "unit")
-public class LruAlgorithmTest {
+public class LruAlgorithmTest extends BaseAlgorithmTest {
+
+   protected LRUAlgorithmConfig getNewEvictionAlgorithmConfig() {
+      return new LRUAlgorithmConfig();
+   }
+//
+//   public void testMinTimeToLive() throws Exception {
+//      LRUAlgorithmConfig cfg = getNewEvictionAlgorithmConfig();
+//      cfg.setMinTimeToLive(2 * 60 * 60 * 1000); // something enormous - 2 hrs
+//      cfg.setMaxEntries(5);
+//      EvictionAlgorithm a = createAndInit(cfg);
+//      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+//      a.setEvictionAction(mockAction);
+//      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+//      for (int i = 0; i < 10; i++) eventQueue.put(new EvictionEvent(i, EvictionEvent.Type.ADD_ENTRY_EVENT));
+//
+//      assert eventQueue.size() == 10;
+//
+//      // what do we expect to happen on the eviction action class?
+//      // nothing at this stage.
+//      replay(mockAction);
+//      a.process(eventQueue);
+//      verify(mockAction);
+//
+//      for (EntryEvictionData data : a.getEvictionQueue()) {
+//         // change the creation timestamp to before 2 hrs in the past
+//         // for all even keys
+//         Integer key = (Integer) data.getKey();
+//
+//         if (key % 2 == 0) {
+//            data.setCreationTimeStamp(1);
+//            data.setModifiedTimeStamp(1);
+//            EvictionEvent e = new EvictionEvent(key, EvictionEvent.Type.VISIT_ENTRY_EVENT);
+//            e.setCreationTimestamp(1);
+//            eventQueue.put(e);
+//         } else {
+//            eventQueue.put(new EvictionEvent(key, EvictionEvent.Type.VISIT_ENTRY_EVENT));
+//         }
+//      }
+//
+//      assert eventQueue.size() == 10;
+//
+//      // this time we expect all even numbered keys to get evicted.
+//      reset(mockAction);
+//      expect(mockAction.evict(eq(0))).andReturn(true).once();
+//      expect(mockAction.evict(eq(2))).andReturn(true).once();
+//      expect(mockAction.evict(eq(4))).andReturn(true).once();
+//      expect(mockAction.evict(eq(6))).andReturn(true).once();
+//      expect(mockAction.evict(eq(8))).andReturn(true).once();
+//      replay(mockAction);
+//      a.process(eventQueue);
+//      verify(mockAction);
+//   }
 }

Modified: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lru/LruQueueTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lru/LruQueueTest.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/lru/LruQueueTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,7 +1,139 @@
 package org.horizon.eviction.algorithms.lru;
 
+import org.horizon.eviction.EntryEvictionData;
+import org.horizon.eviction.EvictionQueue;
+import org.horizon.eviction.algorithms.BaseQueueTest;
 import org.testng.annotations.Test;
 
+import java.util.Iterator;
+
 @Test(groups = "unit")
-public class LruQueueTest {
+public class LruQueueTest extends BaseQueueTest {
+   protected EvictionQueue getNewEvictionQueue() {
+      return new LRUQueue();
+   }
+
+   public void testOrder() throws Exception {
+      LRUQueue queue = new LRUQueue();
+      fillQueue(queue, 500);
+
+      for (int i = 0; i < 500; i++) {
+         if ((i < 100) || (i >= 300 && i < 400)) {
+            // visit the entries from 0-99 and the entries from 300 - 399
+            reorderByLRU(queue, i);
+         }
+      }
+
+      // visiting the nodes should have no affect on the maxAgeQueue.
+      Iterator<EntryEvictionData> maxAgeIt = queue.iterateMaxAgeQueue();
+      int count = 0;
+      long lastTs = 0;
+      while (maxAgeIt.hasNext()) {
+         EntryEvictionData data = maxAgeIt.next();
+         assert lastTs <= data.getCreationTimeStamp();
+         lastTs = data.getCreationTimeStamp();
+         count++;
+      }
+
+      Iterator<EntryEvictionData> lruIt = queue.iterateLRUQueue();
+      count = 0;
+      while (lruIt.hasNext()) {
+         EntryEvictionData data = lruIt.next();
+         int index = (Integer) data.getKey();
+
+         // the last 200 in the list should be the visisted LRU ones.
+         if (count >= 300 && count < 400)
+            assert count - 300 == index;
+         else if (count >= 400 && count < 500)
+            assert count - 100 == index;
+         else if (count < 200)
+            assert count + 100 == index;
+         else if (count >= 200 && count < 300)
+            assert count + 200 == index;
+
+         count++;
+      }
+
+      Object key = getFirstEntryKey(queue);
+      queue.remove(key);
+      assert 499 == queue.size();
+
+      EntryEvictionData data = queue.iterateLRUQueue().next();
+      queue.remove(data.getKey());
+      assert 498 == queue.size();
+   }
+
+   public void testEmptyingInternalCollections() {
+      LRUQueue queue = new LRUQueue();
+      fillQueue(queue, 1000);
+      assert queue.size() == 1000;
+      assert queue.maxAgeQueue.size() == 1000;
+      assert queue.lruQueue.size() == 1000;
+
+      for (Iterator<EntryEvictionData> it = queue.iterator(); it.hasNext() && it.next() != null;) it.remove();
+
+      assert queue.isEmpty();
+      assert queue.maxAgeQueue.isEmpty();
+      assert queue.lruQueue.isEmpty();
+   }
+
+   public void testGetFirstLRUNodeEntry() throws Exception {
+      LRUQueue queue = new LRUQueue();
+      fillQueue(queue, 100);
+
+      for (int i = 0; i < 100; i++) {
+         // this should move all the even numbered entries to the bottom of the lruQueue.
+         // maxAgeQueue should be unaffected.
+         if (i % 2 == 0) reorderByLRU(queue, i);
+      }
+
+      assert 100 == queue.size();
+
+      int count = 0;
+      for (Iterator<EntryEvictionData> it = queue.iterateLRUQueue(); it.hasNext();) {
+         int nodeIndex = (Integer) it.next().getKey();
+
+         if (count < 50) {
+            // the top 50 should be all odds in the lruQueue/
+            assert nodeIndex % 2 != 0;
+         } else {
+            // the bottom fifty should all be even #'s (and 0)
+            assert nodeIndex % 2 == 0;
+         }
+         it.remove();
+         count++;
+      }
+      assert queue.isEmpty();
+   }
+
+   public void testGetFirstMaxAgeNodeEntriy() throws Exception {
+      LRUQueue queue = new LRUQueue();
+      fillQueue(queue, 100);
+
+      for (int i = 0; i < 100; i++) {
+         // this should move all the even numbered NodeEntries to the bottom of the lruQueue.
+         // maxAgeQueue should be unaffected.
+         if (i % 2 == 0) {
+            reorderByLRU(queue, i);
+         }
+      }
+
+      int count = 0;
+      for (Iterator<EntryEvictionData> it = queue.iterateMaxAgeQueue(); it.hasNext();) {
+         int nodeIndex = (Integer) it.next().getKey();
+         assert count == nodeIndex;
+         it.remove();
+         count++;
+      }
+
+      assert queue.isEmpty();
+   }
+
+   private void reorderByLRU(LRUQueue queue, Object key) {
+      // leave the max age queue alone - it is like a fifo.
+
+      // the lru queue is access ordered. meaning the most recently read item is moved to the bottom of the queue.
+      // simply calling get against it visits it and will cause LinkedHashMap to move it to the bottom of the queue.
+      queue.lruQueue.get(key);
+   }
 }

Modified: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/mru/MruAlgorithmTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/mru/MruAlgorithmTest.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/mru/MruAlgorithmTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,7 +1,73 @@
 package org.horizon.eviction.algorithms.mru;
 
+import org.horizon.eviction.algorithms.BaseAlgorithmTest;
 import org.testng.annotations.Test;
 
 @Test(groups = "unit")
-public class MruAlgorithmTest {
+public class MruAlgorithmTest extends BaseAlgorithmTest {
+   protected MRUAlgorithmConfig getNewEvictionAlgorithmConfig() {
+      return new MRUAlgorithmConfig();
+   }
+//
+//   public void testMinTimeToLive() throws Exception {
+//      MRUAlgorithmConfig cfg = getNewEvictionAlgorithmConfig();
+//      cfg.setMinTimeToLive(2 * 60 * 60 * 1000); // something enormous - 2 hrs
+//      cfg.setMaxEntries(5);
+//      EvictionAlgorithm a = createAndInit(cfg);
+//      EvictionAction mockAction = EasyMock.createMock(EvictionAction.class);
+//      a.setEvictionAction(mockAction);
+//      BlockingQueue<EvictionEvent> eventQueue = new LinkedBlockingQueue<EvictionEvent>();
+//      for (int i = 0; i < 10; i++) eventQueue.put(new EvictionEvent(i, EvictionEvent.Type.ADD_ENTRY_EVENT));
+//
+//      assert eventQueue.size() == 10;
+//
+//      // what do we expect to happen on the eviction action class?
+//      // nothing at this stage.
+//      replay(mockAction);
+//      a.process(eventQueue);
+//      verify(mockAction);
+//
+//
+//      // the MRU queue doesn't actually work on timestamps, but instead considers the most recently accessed, and uses a stack.
+//      for (EntryEvictionData data : a.getEvictionQueue()) {
+//         Integer key = (Integer) data.getKey();
+//
+//         if (key % 2 == 1) {
+//            // so first touch all of the odd keys.
+//            data.setCreationTimeStamp(1);
+//            data.setModifiedTimeStamp(1);
+//            EvictionEvent e = new EvictionEvent(key, EvictionEvent.Type.VISIT_ENTRY_EVENT);
+//            e.setCreationTimestamp(1);
+//            eventQueue.put(e);
+//         }
+//      }
+//      // and now the event ones
+//      for (EntryEvictionData data : a.getEvictionQueue()) {
+//         Integer key = (Integer) data.getKey();
+//
+//         if (key % 2 == 0) {
+//            // so first touch all of the odd keys.
+//            data.setCreationTimeStamp(1);
+//            data.setModifiedTimeStamp(1);
+//            EvictionEvent e = new EvictionEvent(key, EvictionEvent.Type.VISIT_ENTRY_EVENT);
+//            e.setCreationTimestamp(1);
+//            eventQueue.put(e);
+//         }
+//      }
+//
+//
+//     // so that the even keys will be the most recently used.
+//      assert eventQueue.size() == 10;
+//
+//      // this time we expect all even numbered keys to get evicted.
+//      reset(mockAction);
+//      expect(mockAction.evict(eq(0))).andReturn(true).once();
+//      expect(mockAction.evict(eq(2))).andReturn(true).once();
+//      expect(mockAction.evict(eq(4))).andReturn(true).once();
+//      expect(mockAction.evict(eq(6))).andReturn(true).once();
+//      expect(mockAction.evict(eq(8))).andReturn(true).once();
+//      replay(mockAction);
+//      a.process(eventQueue);
+//      verify(mockAction);
+//   }
 }

Modified: core/branches/flat/src/test/java/org/horizon/eviction/algorithms/mru/MruQueueTest.java
===================================================================
--- core/branches/flat/src/test/java/org/horizon/eviction/algorithms/mru/MruQueueTest.java	2009-02-03 13:51:43 UTC (rev 7632)
+++ core/branches/flat/src/test/java/org/horizon/eviction/algorithms/mru/MruQueueTest.java	2009-02-03 21:28:24 UTC (rev 7633)
@@ -1,7 +1,43 @@
 package org.horizon.eviction.algorithms.mru;
 
+import org.horizon.eviction.EntryEvictionData;
+import org.horizon.eviction.EvictionQueue;
+import org.horizon.eviction.algorithms.BaseQueueTest;
 import org.testng.annotations.Test;
 
+import java.util.Iterator;
+
 @Test(groups = "unit")
-public class MruQueueTest {
+public class MruQueueTest extends BaseQueueTest {
+   protected EvictionQueue getNewEvictionQueue() {
+      return new MRUQueue();
+   }
+
+   public void testOrder() throws Exception {
+      MRUQueue queue = new MRUQueue();
+      fillQueue(queue, 100);
+      for (int i = 0; i < 100; i++) {
+         if (i % 2 == 0) {
+            EntryEvictionData data = queue.get(i);
+            data.setModifiedTimeStamp(System.currentTimeMillis());
+            queue.moveToTopOfStack(i);
+         }
+      }
+
+      int count = 0;
+      for (Iterator<EntryEvictionData> it = queue.iterator(); it.hasNext();) {
+         EntryEvictionData data = it.next();
+         if (count < 50) {
+            assert data.getModifiedTimeStamp() > 0;
+         } else {
+            assert data.getModifiedTimeStamp() == 0;
+         }
+         it.remove();
+         count++;
+      }
+
+      assert queue.isEmpty();
+      assert queue.keyMap.isEmpty();
+      assert queue.list.isEmpty();
+   }
 }




More information about the jbosscache-commits mailing list