You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jcs-dev@jakarta.apache.org by as...@apache.org on 2005/02/15 02:02:11 UTC

cvs commit: jakarta-turbine-jcs/src/test/org/apache/jcs/util/struct TestLRUMap.java TestLRUMapPerf.java

asmuts      2005/02/14 17:02:11

  Modified:    src/java/org/apache/jcs/engine/memory/util
                        DoubleLinkedListNode.java
                        MemoryElementDescriptor.java
               src/java/org/apache/jcs/auxiliary/disk/indexed
                        IndexedDiskCache.java
                        IndexedDiskElementDescriptor.java
               src/scripts tester.sh
  Added:       src/java/org/apache/jcs/utils/struct
                        LRUElementDescriptor.java LRUMap.java
               src/test/org/apache/jcs/util/struct TestLRUMap.java
                        TestLRUMapPerf.java
  Log:
  Pulling an LRUMap out of the memory cache.
  Duplicating some code for now.  Created some tests for the map.
  Looking to use it instead of the commons LRU map until the commons collections
  packaging stabalizes or everyone is up to 3.0.
  WIll make more tests and try to replace the commons LRU map
  used in the disk cache with this LRUMap.
  So far this implementation is faster for basic put and get operations than the
  commons version, at least for sequential keys.
  
  Revision  Changes    Path
  1.2       +12 -1     jakarta-turbine-jcs/src/java/org/apache/jcs/engine/memory/util/DoubleLinkedListNode.java
  
  Index: DoubleLinkedListNode.java
  ===================================================================
  RCS file: /home/cvs/jakarta-turbine-jcs/src/java/org/apache/jcs/engine/memory/util/DoubleLinkedListNode.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- DoubleLinkedListNode.java	13 Jul 2004 02:10:25 -0000	1.1
  +++ DoubleLinkedListNode.java	15 Feb 2005 01:02:11 -0000	1.2
  @@ -29,10 +29,21 @@
     Object payload;
   
     /** Double Linked list references */
  -  public DoubleLinkedListNode prev, next;
  +  public DoubleLinkedListNode prev;
  +  /** Double Linked list references */
  +  public DoubleLinkedListNode next;
   
     public DoubleLinkedListNode( Object payloadP )
     {
       payload = payloadP;
     }
  +  
  + /**
  +  * @return Object
  +  */
  +  public Object getPayload()
  +  {
  +      return payload;
  +  }
  +  
   }
  
  
  
  1.2       +0 -2      jakarta-turbine-jcs/src/java/org/apache/jcs/engine/memory/util/MemoryElementDescriptor.java
  
  Index: MemoryElementDescriptor.java
  ===================================================================
  RCS file: /home/cvs/jakarta-turbine-jcs/src/java/org/apache/jcs/engine/memory/util/MemoryElementDescriptor.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- MemoryElementDescriptor.java	13 Jul 2004 02:10:25 -0000	1.1
  +++ MemoryElementDescriptor.java	15 Feb 2005 01:02:11 -0000	1.2
  @@ -23,8 +23,6 @@
    */
   public class MemoryElementDescriptor extends DoubleLinkedListNode
   {
  -  /** Double Linked list references */
  -  //public MemoryElementDescriptor prev, next;
   
     /** The CacheElement wrapped by this descriptor */
     public ICacheElement ce;
  
  
  
  1.21      +11 -16    jakarta-turbine-jcs/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.java
  
  Index: IndexedDiskCache.java
  ===================================================================
  RCS file: /home/cvs/jakarta-turbine-jcs/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.java,v
  retrieving revision 1.20
  retrieving revision 1.21
  diff -u -r1.20 -r1.21
  --- IndexedDiskCache.java	1 Feb 2005 01:35:16 -0000	1.20
  +++ IndexedDiskCache.java	15 Feb 2005 01:02:11 -0000	1.21
  @@ -90,8 +90,8 @@
     private int recycleCnt = 0;
   
     /**
  -   * Each instance of a Disk cache should use this lock to synchronize reads
  -   * and writes to the underlying storage mechansism.
  +   * use this lock to synchronize reads and writes 
  +   * to the underlying storage mechansism.
      */
     protected WriterPreferenceReadWriteLock storageLock = new
         WriterPreferenceReadWriteLock();
  @@ -239,7 +239,7 @@
   
     /**
      * Check for minimal consitency between the keys and the datafile.  Makes sure
  -   * no staring positions in the keys exceed the file length.
  +   * no starting positions in the keys exceed the file length.
      * <p>
      * The caller should take the appropriate action if the keys and data
      * are not consistent.  
  @@ -298,9 +298,6 @@
                     ", key count: " + keyHash.size());
         }
   
  -      //storageLock.writeLock();
  -      //storageLock.writeLock().acquire();
  -
         try
         {
           keyFile.reset();
  @@ -315,9 +312,7 @@
         }
         finally
         {
  -        //storageLock.done();
  -        //storageLock.writeLock().release();
  -
  +          // noopt
         }
       }
       catch (Exception e)
  @@ -358,7 +353,6 @@
         byte[] data = IndexedDisk.serialize(ce);
   
         // make sure this only locks for one particular cache region
  -      //storageLock.writeLock();
         storageLock.writeLock().acquire();
   
         ded.init(dataFile.length(), data);
  @@ -419,7 +413,6 @@
         }
         finally
         {
  -        //storageLock.done();
           storageLock.writeLock().release();
         }
   
  @@ -696,7 +689,7 @@
     }
   
     /**
  -   * Description of the Method
  +   *  Remove all the items fromt he disk cache by reseting everything.
      */
     public void doRemoveAll()
     {
  @@ -714,10 +707,12 @@
       }
     }
   
  -  // end removeAll
   
     /**
  -   * handle error by last resort, force content update, or removeall
  +   * Reset effectively clears the disk cache, creating new files, recyclebins,
  +   * and keymaps.
  +   * <p>
  +   * It can be used to handle errors by last resort, force content update, or removeall.
      */
     private void reset()
     {
  @@ -800,7 +795,7 @@
       }
       else 
       {
  -      keyHash = new HashMap();// Hashtable();//HashMap();
  +      keyHash = new HashMap();
         if (log.isInfoEnabled())
         {
           log.info("Set maxKeySize to unlimited'");
  
  
  
  1.5       +4 -1      jakarta-turbine-jcs/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskElementDescriptor.java
  
  Index: IndexedDiskElementDescriptor.java
  ===================================================================
  RCS file: /home/cvs/jakarta-turbine-jcs/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskElementDescriptor.java,v
  retrieving revision 1.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- IndexedDiskElementDescriptor.java	14 Jul 2004 03:35:09 -0000	1.4
  +++ IndexedDiskElementDescriptor.java	15 Feb 2005 01:02:11 -0000	1.5
  @@ -34,7 +34,10 @@
       public int len;
   
   
  -    /** Description of the Method */
  +    /** Description of the Method 
  +     * @param pos
  +     * @param data
  +     */
       public void init( long pos, byte[] data )
       {
           this.pos = pos;
  
  
  
  1.1                  jakarta-turbine-jcs/src/java/org/apache/jcs/utils/struct/LRUElementDescriptor.java
  
  Index: LRUElementDescriptor.java
  ===================================================================
  package org.apache.jcs.utils.struct;
  
  import org.apache.jcs.engine.memory.util.DoubleLinkedListNode;
  
  /**
   * @author aaronsm
   *
   */
  public class LRUElementDescriptor extends DoubleLinkedListNode
  {
      
      /**
       * <code>key</code>
       */
      private Object key;
  
      /**
       * @param key
       * @param payloadP
       */
      public LRUElementDescriptor(Object key, Object payloadP)
      {
          super( payloadP );
          this.setKey( key );
      }
  
      /**
       * @param key The key to set.
       */
      public void setKey( Object key )
      {
          this.key = key;
      }
  
      /**
       * @return Returns the key.
       */
      public Object getKey()
      {
          return key;
      }
  
  }
  
  
  
  1.1                  jakarta-turbine-jcs/src/java/org/apache/jcs/utils/struct/LRUMap.java
  
  Index: LRUMap.java
  ===================================================================
  package org.apache.jcs.utils.struct;
  
  import java.io.Serializable;
  import java.util.ArrayList;
  import java.util.Collection;
  import java.util.Hashtable;
  import java.util.Iterator;
  import java.util.Map;
  import java.util.NoSuchElementException;
  import java.util.Set;
  
  import org.apache.commons.logging.Log;
  import org.apache.commons.logging.LogFactory;
  
  import org.apache.jcs.engine.control.group.GroupAttrName;
  import org.apache.jcs.engine.memory.util.DoubleLinkedList;
  import org.apache.jcs.engine.stats.StatElement;
  import org.apache.jcs.engine.stats.Stats;
  import org.apache.jcs.engine.stats.behavior.IStatElement;
  import org.apache.jcs.engine.stats.behavior.IStats;
  
  /**
   * This is a simple LRUMap.  It implements most of the map methods.  It is not recommended
   * that you use any but put, get, remove, and clear.
   * 
   * @author aaronsm
   *  
   */
  public class LRUMap implements Map
  {
  
      private final static Log log        = LogFactory.getLog( LRUMap.class );
  
      // double linked list for lru
      private DoubleLinkedList list;
  
      /**
       * Map where items are stored by key
       */
      protected Map            map;
  
      int                      hitCnt     = 0;
  
      int                      missCnt    = 0;
  
      int                      putCnt     = 0;
  
      int                      maxObjects = -1;
  
  
      // make configurable
      private int            chunkSize  = 1;
  
      
      /**
       * @param maxObjects
       */
      public LRUMap( int maxObjects )
      {
          this.maxObjects =  maxObjects;
          list = new DoubleLinkedList();
          map = new Hashtable();        
      }
      
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#size()
       */
      public int size()
      {
          return map.size();
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#clear()
       */
      public void clear()
      {
          map.clear();
          list.removeAll();
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#isEmpty()
       */
      public boolean isEmpty()
      {
          return map.size() == 0;
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#containsKey(java.lang.Object)
       */
      public boolean containsKey( Object key )
      {
          return map.containsKey( key );
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#containsValue(java.lang.Object)
       */
      public boolean containsValue( Object value )
      {
          return map.containsValue( value );
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#values()
       */
      public Collection values()
      {
          return map.values();
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#putAll(java.util.Map)
       */
      public void putAll( Map t )
      {
          // TODO Auto-generated method stub
  
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#entrySet()
       */
      public Set entrySet()
      {
          // todo, we should return a defensive copy
          // this is not thread safe.
          return map.entrySet();
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#keySet()
       */
      public Set keySet()
      {
          return map.keySet();
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#get(java.lang.Object)
       */
      public Object get( Object key )
      {
          Object retVal = null;
  
          if (log.isDebugEnabled())
          {
              log.debug( "getting item  for key " + key );
          }
  
          LRUElementDescriptor me = (LRUElementDescriptor) map.get( key );
  
          if (me != null)
          {
              hitCnt++;
              if (log.isDebugEnabled())
              {
                  log.debug( "LRUMap hit for " + key );
              }
  
              retVal = me.getPayload();
  
              list.makeFirst( me );
          }
          else
          {
              missCnt++;
              log.debug( "LRUMap miss for " + key );
          }
  
          //verifyCache();
          return retVal;
      }
  
      /**
       * @param key
       * @return Object
       */
      public Object getQuiet( Object key )
      {
          Object ce = null;
  
          LRUElementDescriptor me = (LRUElementDescriptor) map.get( key );
          if (me != null)
          {
              if (log.isDebugEnabled())
              {
                  log.debug( "LRUMap quiet hit for " + key );
              }
  
              ce = me.getPayload();
          }
          else if (log.isDebugEnabled())
          {
              log.debug( "LRUMap quiet miss for " + key );
          }
  
          return ce;
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#remove(java.lang.Object)
       */
      public Object remove( Object key )
      {
          if (log.isDebugEnabled())
          {
              log.debug( "removing item for key: " + key );
          }
  
          // remove single item.
          LRUElementDescriptor me = (LRUElementDescriptor) map.remove( key );
  
          if (me != null)
          {
              list.remove( me );
          }
  
          return me;
      }
  
      /*
       * (non-Javadoc)
       * 
       * @see java.util.Map#put(java.lang.Object, java.lang.Object)
       */
      public Object put( Object key, Object value )
      {
          putCnt++;
  
          LRUElementDescriptor old = null;
          synchronized (this)
          {
              // TODO address double synchronization of addFirst, use write lock
              addFirst( key, value );
              // this must be synchronized
              old = (LRUElementDescriptor) map.put( ((LRUElementDescriptor) list
                      .getFirst()).getKey(), list
                      .getFirst() );
  
              // If the node was the same as an existing node, remove it.
              if (old != null
                      && ((LRUElementDescriptor) list.getFirst()).getKey()
                              .equals( old.getKey() ))
              {
                  list.remove( old );
              }
          }
  
          int size = map.size();
          // If the element limit is reached, we need to spool
  
          if (size <= this.maxObjects)
          {
              return old;
          }
          log.debug( "In memory limit reached, spooling" );
  
          // Write the last 'chunkSize' items to disk.
          int chunkSizeCorrected = Math.min( size, getChunkSize() );
  
          if (log.isDebugEnabled())
          {
              log.debug( "About to spool to disk cache, map size: " + size
                      + ", max objects: " + this.maxObjects
                      + ", items to spool: " + chunkSizeCorrected );
          }
  
          // The spool will put them in a disk event queue, so there is no
          // need to pre-queue the queuing. This would be a bit wasteful
          // and wouldn't save much time in this synchronous call.
  
          for (int i = 0; i < chunkSizeCorrected; i++)
          {
              synchronized (this)
              {
                  if (list.getLast() != null)
                  {
                      if (((LRUElementDescriptor) list.getLast()) != null)
                      {
                          //cache.spoolToDisk( ( (LRUElementDescriptor)
                          // list.getLast()).ce);
                          if (!map.containsKey( ((LRUElementDescriptor) list
                                  .getLast()).getKey() ))
                          {
                              log.error( "update: map does not contain key: "
                                      + ((LRUElementDescriptor) list.getLast())
                                              .getKey() );
                              verifyCache();
                          }
                          if (map.remove( ((LRUElementDescriptor) list.getLast())
                                  .getKey() ) == null)
                          {
                              log.warn( "update: remove failed for key: "
                                      + ((LRUElementDescriptor) list.getLast())
                                              .getKey() );
                              verifyCache();
                          }
                      }
                      else
                      {
                          throw new Error( "update: last.ce is null!" );
                      }
                      list.removeLast();
                  }
                  else
                  {
                      verifyCache();
                      throw new Error( "update: last is null!" );
                  }
              }
          }
  
          if (log.isDebugEnabled())
          {
              log.debug( "update: After spool map size: " + map.size() );
          }
          if (map.size() != dumpCacheSize())
          {
              log.error( "update: After spool, size mismatch: map.size() = "
                      + map.size() + ", linked list size = " + dumpCacheSize() );
          }
  
          return old;
      }
  
      /**
       * Adds a new node to the end of the link list. Currently not used.
       * @param key
       * @param val The feature to be added to the Last
       * 
       */
      private void addLast( Object key, Object val )
      {
          LRUElementDescriptor me = new LRUElementDescriptor( key, val );
          list.addLast( me );
          verifyCache(me.getKey());
      }
  
      /**
       * Adds a new node to the start of the link list.
       * @param key
       * @param val The feature to be added to the First
       */
      private synchronized void addFirst( Object key, Object val )
      {
  
          LRUElementDescriptor me = new LRUElementDescriptor( key, val );
          list.addFirst( me );
          return;
      }
  
      /**
       * Returns the size of the list.
       * 
       * @return int
       */
      private int dumpCacheSize()
      {
          return list.size();
      }
  
      
      /**
       *  Dump the cache entries from first to list for debugging.
       */
      public void dumpCacheEntries()
      {
        log.debug("dumpingCacheEntries");
        for (LRUElementDescriptor me = (LRUElementDescriptor) list.getFirst();
             me != null; me = (LRUElementDescriptor) me.next)
        {
          log.debug("dumpCacheEntries> key="
                    + me.getKey() + ", val=" + me.getPayload());
        }
      }    
      
      
      /**
       * Dump the cache map for debugging.
       */
      public void dumpMap()
      {
        log.debug("dumpingMap");
        for (Iterator itr = map.entrySet().iterator(); itr.hasNext(); )
        {
          Map.Entry e = (Map.Entry) itr.next();
          LRUElementDescriptor me = (LRUElementDescriptor) e.getValue();
          log.debug("dumpMap> key=" + e.getKey() + ", val=" + me.getPayload());
        }
      }
      
      /**
       * Checks to see if all the items that should be in the cache are.
       * Checks consistency between List and map.
       *
       */
      private void verifyCache()
      {
        if (!log.isDebugEnabled())
        {
          return;
        }
  
        boolean found = false;
        log.debug("verifycache: mapContains " + map.size() +
                  " elements, linked list contains "
                  + dumpCacheSize() + " elements");
        log.debug("verifycache: checking linked list by key ");
        for (LRUElementDescriptor li = (LRUElementDescriptor) list.getFirst();
             li != null; li = (LRUElementDescriptor) li.next)
        {
          Object key = li.getKey();
          if (!map.containsKey(key))
          {
            log.error("verifycache: map does not contain key : " + li.getKey());
            log.error("li.hashcode=" + li.getKey().hashCode());
            log.error("key class=" + key.getClass());
            log.error("key hashcode=" + key.hashCode());
            log.error("key toString=" + key.toString());
            if (key instanceof GroupAttrName)
            {
              GroupAttrName name = (GroupAttrName) key;
              log.error("GroupID hashcode=" + name.groupId.hashCode());
              log.error("GroupID.class=" + name.groupId.getClass());
              log.error("AttrName hashcode=" + name.attrName.hashCode());
              log.error("AttrName.class=" + name.attrName.getClass());
            }
            dumpMap();
          }
          else if (map.get(li.getKey()) == null)
          {
            log.error("verifycache: linked list retrieval returned null for key: " +
                      li.getKey());
          }
        }
  
        log.debug("verifycache: checking linked list by value ");
        for (LRUElementDescriptor li3 = (LRUElementDescriptor) list.getFirst();
             li3 != null; li3 = (LRUElementDescriptor) li3.next)
        {
          if (map.containsValue(li3) == false)
          {
            log.error("verifycache: map does not contain value : " + li3);
            dumpMap();
          }
        }
  
        log.debug("verifycache: checking via keysets!");
        for (Iterator itr2 = map.keySet().iterator(); itr2.hasNext(); )
        {
          found = false;
          Serializable val = null;
          try
          {
            val = (Serializable) itr2.next();
          }
          catch (NoSuchElementException nse)
          {
            log.error("verifycache: no such element exception");
          }
  
          for (LRUElementDescriptor li2 = (LRUElementDescriptor) list.
               getFirst(); li2 != null; li2 = (LRUElementDescriptor) li2.next)
          {
            if (val.equals(li2.getKey()))
            {
              found = true;
              break;
            }
          }
          if (!found)
          {
            log.error("verifycache: key not found in list : " +
                      val);
            dumpCacheEntries();
            if (map.containsKey(val))
            {
              log.error("verifycache: map contains key");
            }
            else
            {
              log.error("verifycache: map does NOT contain key, what the HECK!");
            }
          }
        }
      }
  
      
      /**
       * Logs an error is an element that should be in the cache is not.
       * 
       * @param key
       */
      private void verifyCache(Object key)
      {
        if (!log.isDebugEnabled())
        {
          return;
        }
  
        boolean found = false;
  
        // go through the linked list looking for the key
        for (LRUElementDescriptor li = (LRUElementDescriptor) list.getFirst();
             li != null; li = (LRUElementDescriptor) li.next)
        {
          if (li.getKey() == key)
          {
            found = true;
            log.debug("verifycache(key) key match: " + key);
            break;
          }
        }
        if (!found)
        {
          log.error("verifycache(key), couldn't find key! : " +
                    key);
        }
      }    
      
      /**
       * The chunk size is the number of items to remove when the max is reached.  
       * By default it is 1.
       * 
       * @param chunkSize The chunkSize to set.
       */
      public void setChunkSize( int chunkSize )
      {
          this.chunkSize = chunkSize;
      }
  
      /**
       * @return Returns the chunkSize.
       */
      public int getChunkSize()
      {
          return chunkSize;
      }
  
      
      /**
       * 
       * @return IStats
       */
      public IStats getStatistics()
      {
          IStats stats = new Stats();
          stats.setTypeName( "LRUMap" );
  
          ArrayList elems = new ArrayList();
  
          IStatElement se = null;
  
          se = new StatElement();
          se.setName( "List Size" );
          se.setData( "" + list.size() );
          elems.add( se );
  
          se = new StatElement();
          se.setName( "Map Size" );
          se.setData( "" + map.size() );
          elems.add( se );
  
          se = new StatElement();
          se.setName( "Put Count" );
          se.setData( "" + putCnt );
          elems.add( se );
  
          se = new StatElement();
          se.setName( "Hit Count" );
          se.setData( "" + hitCnt );
          elems.add( se );
  
          se = new StatElement();
          se.setName( "Miss Count" );
          se.setData( "" + missCnt );
          elems.add( se );
  
          // get an array and put them in the Stats object
          IStatElement[] ses = (IStatElement[]) elems
                  .toArray( new StatElement[0] );
          stats.setStatElements( ses );
  
          return stats;
      }
  
  }
  
  
  1.3       +2 -1      jakarta-turbine-jcs/src/scripts/tester.sh
  
  Index: tester.sh
  ===================================================================
  RCS file: /home/cvs/jakarta-turbine-jcs/src/scripts/tester.sh,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- tester.sh	6 Jan 2005 01:19:51 -0000	1.2
  +++ tester.sh	15 Feb 2005 01:02:11 -0000	1.3
  @@ -14,5 +14,6 @@
   
   
   echo ${CLASSPATH}
  +#-Xrunhprof:cpu=samples,depth=6,thread=y
   
  -${JAVA_HOME}/bin/java -ms90m -mx400m -verbosegc -classpath "${CLASSPATH}" org.apache.jcs.access.TestCacheAccess /cache$argv.ccf 
  +${JAVA_HOME}/bin/java  -ms90m -mx400m -verbosegc -classpath "${CLASSPATH}" org.apache.jcs.access.TestCacheAccess /cache$argv.ccf 
  
  
  
  1.1                  jakarta-turbine-jcs/src/test/org/apache/jcs/util/struct/TestLRUMap.java
  
  Index: TestLRUMap.java
  ===================================================================
  package org.apache.jcs.util.struct;
  
  import java.util.Iterator;
  
  import junit.framework.Test;
  import junit.framework.TestCase;
  import junit.framework.TestSuite;
  
  import org.apache.jcs.utils.struct.LRUMap;
  
  /**
   * Tests the LRUMap
   * 
   * @author aaronsm
   *  
   */
  public class TestLRUMap extends TestCase
  {
  
      private static int items = 20000;
  
      /**
       * Constructor for the TestSimpleLoad object
       * 
       * @param testName
       *            Description of the Parameter
       */
      public TestLRUMap(String testName)
      {
          super( testName );
      }
  
      /**
       * Description of the Method
       * 
       * @param args
       *            Description of the Parameter
       */
      public static void main( String args[] )
      {
          String[] testCaseName = { TestLRUMap.class.getName() };
          junit.textui.TestRunner.main( testCaseName );
      }
  
      /**
       * A unit test suite for JUnit
       * 
       * @return The test suite
       */
      public static Test suite()
      {
          // run the basic tests
          TestSuite suite = new TestSuite( TestLRUMap.class );
  
          // run concurrent tests
          final LRUMap map = new LRUMap( 2000 );
          suite.addTest( new TestLRUMap( "conc1" )
          {
              public void runTest() throws Exception
              {
                  this.runConcurrentPutGetTests( map, 2000 );
              }
          } );
          suite.addTest( new TestLRUMap( "conc2" )
          {
              public void runTest() throws Exception
              {
                  this.runConcurrentPutGetTests( map, 2000 );
              }
          } );
          suite.addTest( new TestLRUMap( "conc3" )
          {
              public void runTest() throws Exception
              {
                  this.runConcurrentPutGetTests( map, 2000 );
              }
          } );
  
          // run more concurrent tests
          final int max2 = 20000;
          final LRUMap map2 = new LRUMap( max2 );
          suite.addTest( new TestLRUMap( "concB1" )
          {
              public void runTest() throws Exception
              {
                  this.runConcurrentRangeTests( map2, 10000, max2 );
              }
          } );
          suite.addTest( new TestLRUMap( "concB1" )
          {
              public void runTest() throws Exception
              {
                  this.runConcurrentRangeTests( map2, 0, 9999 );
              }
          } );
  
          return suite;
      }
  
      /**
       * Just test that we can put, get and remove as expected.
       * 
       * @exception Exception
       *                Description of the Exception
       */
      public void testSimpleLoad() throws Exception
      {
          LRUMap map = new LRUMap( items );
  
          for (int i = 0; i < items; i++)
          {
              map.put( i + ":key", "data" + i );
          }
  
          for (int i = items - 1; i >= 0; i--)
          {
              String res = (String) map.get( i + ":key" );
              if (res == null)
              {
                  assertNotNull( "[" + i + ":key] should not be null", res );
              }
          }
  
          // test removal
          map.remove( "300:key" );
          assertNull( map.get( "300:key" ) );
  
      }
  
      /**
       * Just make sure that the LRU functions int he most simple case.
       * 
       * @exception Exception
       *                Description of the Exception
       */
      public void testLRURemoval() throws Exception
      {
          int total = 10;
          LRUMap map = new LRUMap( total );
          map.setChunkSize( 1 );
  
          // put the max in
          for (int i = 0; i < total; i++)
          {
              map.put( i + ":key", "data" + i );
          }
  
          Iterator it = map.entrySet().iterator();
          while (it.hasNext())
          {
              System.out.println( it.next() );
          }
          System.out.println( map.getStatistics() );
  
          // get the max out backwards
          for (int i = total - 1; i >= 0; i--)
          {
              String res = (String) map.get( i + ":key" );
              if (res == null)
              {
                  assertNotNull( "[" + i + ":key] should not be null", res );
              }
          }
  
          System.out.println( map.getStatistics() );
  
          //since we got them backwards the total should be at the end.
          // add one confirm that total is gone.
          map.put( (total) + ":key", "data" + (total) );
          assertNull( map.get( (total - 1) + ":key" ) );
  
      }
  
      /**
       * @throws Exception
       */
      public void testLRURemovalAgain() throws Exception
      {
          int total = 10000;
          LRUMap map = new LRUMap( total );
          map.setChunkSize( 1 );
  
          // put the max in
          for (int i = 0; i < total * 2; i++)
          {
              map.put( i + ":key", "data" + i );
          }
  
          // get the total number, these shoukld be null
          for (int i = total - 1; i >= 0; i--)
          {
              assertNull( map.get( i + ":key" ) );
  
          }
  
          // get the total to total *2 items out, these should be foufn.
          for (int i = (total * 2) -1; i >= total; i--)
          {
              String res = (String) map.get( i + ":key" );
              if (res == null)
              {
                  assertNotNull( "[" + i + ":key] should not be null", res );
              }
          }
  
          System.out.println( map.getStatistics() );
  
      }
  
      /**
       * Just make sure that we can put and get concurrently
       * 
       * @param map
       * @param items
       * @throws Exception
       */
      public void runConcurrentPutGetTests( LRUMap map, int items )
              throws Exception
      {
          for (int i = 0; i < items; i++)
          {
              map.put( i + ":key", "data" + i );
          }
  
          for (int i = items - 1; i >= 0; i--)
          {
              String res = (String) map.get( i + ":key" );
              if (res == null)
              {
                  assertNotNull( "[" + i + ":key] should not be null", res );
              }
          }
  
      }
  
      /**
       * Put, get, and remove from a range. This should occur at a range that is
       * not touched by other tests.
       * 
       * @param map
       * @param start
       * @param end
       * @throws Exception
       */
      public void runConcurrentRangeTests( LRUMap map, int start, int end )
              throws Exception
      {
          for (int i = start; i < end; i++)
          {
              map.put( i + ":key", "data" + i );
          }
  
          for (int i = end - 1; i >= start; i--)
          {
              String res = (String) map.get( i + ":key" );
              if (res == null)
              {
                  assertNotNull( "[" + i + ":key] should not be null", res );
              }
          }
  
          // test removal
          map.remove( start + ":key" );
          assertNull( map.get( start + ":key" ) );
  
      }
  
  }
  
  
  1.1                  jakarta-turbine-jcs/src/test/org/apache/jcs/util/struct/TestLRUMapPerf.java
  
  Index: TestLRUMapPerf.java
  ===================================================================
  package org.apache.jcs.util.struct;
  
  import java.util.Map;
  
  import junit.framework.Test;
  import junit.framework.TestCase;
  import junit.framework.TestSuite;
  
  import org.apache.jcs.TestJCSvHashtablePerf;
  import org.apache.jcs.utils.struct.LRUMap;
  
  /**
   * This ensures that the jcs version of the LRU map is as fast as the commons version.
   * It has been testing at .6 to .7 times the commons LRU.
   * 
   * @author aaronsm
   *
   */
  public class TestLRUMapPerf extends TestCase
  {
  
      float ratioPut = 0;
  
      float ratioGet = 0;
  
      float target   = 1.0f;
  
      int   loops    = 20;
  
      int   tries    = 50000;
  
      /**
       * @param testName
       */
      public TestLRUMapPerf(String testName)
      {
          super( testName );
      }
  
      /**
       * A unit test suite for JUnit
       * 
       * @return The test suite
       */
      public static Test suite()
      {
          return new TestSuite( TestLRUMapPerf.class );
      }
  
      /**
       * A unit test for JUnit
       * 
       * @exception Exception
       *                Description of the Exception
       */
      public void testSimpleLoad() throws Exception
      {
          doWork();
          assertTrue( this.ratioPut < target );
          assertTrue( this.ratioGet < target );
      }
  
      /**
       *  
       */
      public void doWork()
      {
  
          long start = 0;
          long end = 0;
          long time = 0;
          float tPer = 0;
  
          long putTotalJCS = 0;
          long getTotalJCS = 0;
          long putTotalHashtable = 0;
          long getTotalHashtable = 0;
  
          String name = "LRUMap";        
          String cache2Name = "";
          
          try
          {
  
              Map cache = new LRUMap( tries );
                          
              for (int j = 0; j < loops; j++)
              {
  
                  name = "JCS      ";
                  start = System.currentTimeMillis();
                  for (int i = 0; i < tries; i++)
                  {
                      cache.put( "key:" + i, "data" + i );
                  }
                  end = System.currentTimeMillis();
                  time = end - start;
                  putTotalJCS += time;
                  tPer = Float.intBitsToFloat( (int) time )
                          / Float.intBitsToFloat( tries );
                  System.out.println( name + " put time for " + tries + " = "
                          + time + "; millis per = " + tPer );
  
                  start = System.currentTimeMillis();
                  for (int i = 0; i < tries; i++)
                  {
                      cache.get( "key:" + i );
                  }
                  end = System.currentTimeMillis();
                  time = end - start;
                  getTotalJCS += time;
                  tPer = Float.intBitsToFloat( (int) time )
                          / Float.intBitsToFloat( tries );
                  System.out.println( name + " get time for " + tries + " = "
                          + time + "; millis per = " + tPer );
  
                  
                  
                  ///////////////////////////////////////////////////////////////
                  cache2Name = "LRUMapJCS (commons)";
                  //or LRUMapJCS
                  Map cache2 = new org.apache.commons.collections.map.LRUMap( tries );
                  //cache2Name = "Hashtable";
                  //Hashtable cache2 = new Hashtable();
                  start = System.currentTimeMillis();
                  for (int i = 0; i < tries; i++)
                  {
                      cache2.put( "key:" + i, "data" + i );
                  }
                  end = System.currentTimeMillis();
                  time = end - start;
                  putTotalHashtable += time;
                  tPer = Float.intBitsToFloat( (int) time )
                          / Float.intBitsToFloat( tries );
                  System.out.println( cache2Name + " put time for " + tries + " = "
                          + time + "; millis per = " + tPer );
  
                  start = System.currentTimeMillis();
                  for (int i = 0; i < tries; i++)
                  {
                      cache2.get( "key:" + i );
                  }
                  end = System.currentTimeMillis();
                  time = end - start;
                  getTotalHashtable += time;
                  tPer = Float.intBitsToFloat( (int) time )
                          / Float.intBitsToFloat( tries );
                  System.out.println( cache2Name + " get time for " + tries + " = "
                          + time + "; millis per = " + tPer );
  
                  System.out.println( "\n" );
              }
  
          }
          catch (Exception e)
          {
              e.printStackTrace( System.out );
              System.out.println( e );
          }
  
          long putAvJCS = putTotalJCS / loops;
          long getAvJCS = getTotalJCS / loops;
          long putAvHashtable = putTotalHashtable / loops;
          long getAvHashtable = getTotalHashtable / loops;
  
          System.out.println( "Finished " + loops + " loops of " + tries
                  + " gets and puts" );
  
          System.out.println( "\n" );
          System.out.println( "Put average for LRUMap       = " + putAvJCS );
          System.out.println( "Put average for " + cache2Name + " = " + putAvHashtable );
          ratioPut = Float.intBitsToFloat( (int) putAvJCS )
                  / Float.intBitsToFloat( (int) putAvHashtable );
          System.out.println( name + " puts took " + ratioPut
                  + " times the " + cache2Name + ", the goal is <" + target + "x" );
  
          System.out.println( "\n" );
          System.out.println( "Get average for LRUMap       = " + getAvJCS );
          System.out.println( "Get average for " + cache2Name + " = " + getAvHashtable );
          ratioGet = Float.intBitsToFloat( (int) getAvJCS )
                  / Float.intBitsToFloat( (int) getAvHashtable );
          System.out.println( name + " gets took " + ratioGet
                  + " times the " + cache2Name + ", the goal is <" + target + "x" );
  
      }
  
      /**
       * @param args
       */
      public static void main( String args[] )
      {
          TestJCSvHashtablePerf test = new TestJCSvHashtablePerf( "command" );
          test.doWork();
      }
  
      
      
  }
  
  
  

---------------------------------------------------------------------
To unsubscribe, e-mail: turbine-jcs-dev-unsubscribe@jakarta.apache.org
For additional commands, e-mail: turbine-jcs-dev-help@jakarta.apache.org