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/01/06 02:19:21 UTC

cvs commit: jakarta-turbine-jcs/src/test je.properties

asmuts      2005/01/05 17:19:21

  Modified:    src/java/org/apache/jcs/auxiliary/disk/indexed
                        IndexedDiskCache.java
               src/java/org/apache/jcs/auxiliary/remote/behavior
                        IRemoteCacheAttributes.java
               src/java/org/apache/jcs/auxiliary/javagroups
                        JavaGroupsCache.java
               src/java/org/apache/jcs/admin JCSAdminBean.java
  Added:       src/java/org/apache/jcs/engine/stats/behavior
                        IStatElement.java IStats.java
               tempbuild jcs-1.1.3-dev.jar
  Removed:     src/test je.properties
  Log:
  1.  Put in a solution for the RMI bug.  Now, gets can be done asynchronously with a configurable timeout.
  You can specify that your remote region should timeout gets.
  
  see the cache.ccf in the src/conf for details
  
  #-1 means no timeout, this is the default
  # if the timeout is -1, no threadpool will be used.
  jcs.auxiliary.RC.attributes.GetTimeoutMillis=5000
  jcs.auxiliary.RC.attributes.ThreadPoolName=remote_cache_client
  
  Here RC is my auxiliary name.  I tell it to time out at 5 seconds and to use the thread pool named remote_cache_client.
  
  You can define pools in the file.  Below I define defaults for a thread pool and the settings for the pool I want the remote cache
  client to use.
  
  If the timeout is less than 0, no threadpool will be used.
  
  ##############################################################
  ################## THREAD POOL CONFIGURATION ###################
  # default thread pool config
  thread_pool.default.boundarySize=75
  thread_pool.default.maximumPoolSize=150
  thread_pool.default.minimumPoolSize=4
  thread_pool.default.keepAliveTime=350000
  thread_pool.default.abortWhenBlocked=false
  thread_pool.default.startUpSize=4
  
  # remote cache client thread pool config
  thread_pool.remote_cache_client.boundarySize=75
  thread_pool.remote_cache_client.maximumPoolSize=150
  thread_pool.remote_cache_client.minimumPoolSize=4
  thread_pool.remote_cache_client.keepAliveTime=350000
  thread_pool.remote_cache_client.abortWhenBlocked=false
  thread_pool.remote_cache_client.startUpSize=4
  
  2.  Changed stats gathering mechanism.  I will update the admin jsp to make use of the
  more easily formattable data.
  
  Revision  Changes    Path
  1.16      +72 -21    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.15
  retrieving revision 1.16
  diff -u -r1.15 -r1.16
  --- IndexedDiskCache.java	22 Jul 2004 13:17:49 -0000	1.15
  +++ IndexedDiskCache.java	6 Jan 2005 01:19:20 -0000	1.16
  @@ -17,18 +17,19 @@
    */
   
   import java.io.File;
  -import java.io.Serializable;
   import java.io.IOException;
  -import java.util.LinkedList;
  +import java.io.Serializable;
  +import java.util.ArrayList;
  +import java.util.Arrays;
   import java.util.ConcurrentModificationException;
  +import java.util.HashMap;
   import java.util.HashSet;
   import java.util.Iterator;
  -import java.util.HashMap;
  +import java.util.LinkedList;
  +import java.util.List;
   import java.util.Map;
   import java.util.Set;
   
  -import EDU.oswego.cs.dl.util.concurrent.WriterPreferenceReadWriteLock;
  -
   import org.apache.commons.logging.Log;
   import org.apache.commons.logging.LogFactory;
   import org.apache.jcs.auxiliary.disk.AbstractDiskCache;
  @@ -38,9 +39,14 @@
   import org.apache.jcs.engine.behavior.ICacheElement;
   import org.apache.jcs.engine.control.group.GroupAttrName;
   import org.apache.jcs.engine.control.group.GroupId;
  -//import org.apache.jcs.utils.locking.ReadWriteLock;
  +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;
   import org.apache.jcs.utils.struct.SortedPreferentialArray;
   
  +import EDU.oswego.cs.dl.util.concurrent.WriterPreferenceReadWriteLock;
  +
   /**
    * Disk cache that uses a RandomAccessFile with keys stored in memory.
    *
  @@ -1239,27 +1245,72 @@
      */
     public String getStats()
     {
  -    StringBuffer buf = new StringBuffer();
  -    buf.append("\n -------------------------");
  -    buf.append("\n Indexed Disk Cache:");
  -    buf.append("\n Key Map Size = " + this.keyHash.size());
  +    return getStatistics().toString();
  +  }
  +
  +  
  +  /*
  +   * (non-Javadoc)
  +   * 
  +   * @see org.apache.jcs.auxiliary.AuxiliaryCache#getStatistics()
  +   */
  +  public IStats getStatistics()
  +  {
  +    IStats stats = new Stats();
  +    stats.setTypeName( "Indexed Disk Cache" );
  +
  +    ArrayList elems = new ArrayList();
  +
  +    IStatElement se = null;
  +
  +    se = new StatElement();
  +    se.setName( "Key Map Size" );
  +    se.setData( "" + this.keyHash.size() );
  +    elems.add( se );
  +
       try
       {
  -      buf.append("\n Data File Length = " + this.dataFile.length());
  +      se = new StatElement();
  +      se.setName( "Data File Length" );
  +      se.setData( "" + this.dataFile.length() );
  +      elems.add( se );
       }
       catch (Exception e)
       {
  -      log.error(e);
  +      log.error( e );
       }
  -    buf.append("\n Optimize Opertaion Count = " + this.optCnt);
  -    buf.append("\n Times Optimized = " + this.timesOptimized);
  -    buf.append("\n Recycle Count = " + this.recycleCnt);
  -
  -    buf.append( super.getStats() );
  -
  -    return buf.toString();
  -  }
   
  +    
  +    se = new StatElement();
  +    se.setName( "Optimize Opertaion Count" );
  +    se.setData( "" + this.optCnt );
  +    elems.add( se );
  +
  +    se = new StatElement();
  +    se.setName( "Times Optimized" );
  +    se.setData( "" + this.timesOptimized );
  +    elems.add( se );
  +
  +    se = new StatElement();
  +    se.setName( "Recycle Count" );
  +    se.setData( "" + this.recycleCnt );
  +    elems.add( se );
  +
  +    
  +    // get the stats from the super too
  +    // get as array, convert to list, add list to our outer list
  +    IStats sStats = super.getStatistics();
  +    IStatElement[] sSEs = sStats.getStatElements();
  +    List sL = Arrays.asList( sSEs );
  +    elems.addAll( sL );
  +
  +    // get an array and put them in the Stats object
  +    IStatElement[] ses = (IStatElement[]) elems.toArray( new StatElement[0] );
  +    stats.setStatElements( ses );
  +
  +    return stats;
  +  }  
  +  
   ///////////////////////////////////////////////////////////////////////////////
   // RECYLCE INNER CLASS
     /**
  
  
  
  1.5       +35 -0     jakarta-turbine-jcs/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheAttributes.java
  
  Index: IRemoteCacheAttributes.java
  ===================================================================
  RCS file: /home/cvs/jakarta-turbine-jcs/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheAttributes.java,v
  retrieving revision 1.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- IRemoteCacheAttributes.java	15 Apr 2004 19:22:56 -0000	1.4
  +++ IRemoteCacheAttributes.java	6 Jan 2005 01:19:21 -0000	1.5
  @@ -247,5 +247,40 @@
        * @param r The new localClusterConsistency value
        */
       public void setLocalClusterConsistency( boolean r );
  +    
  +    /**
  +     * The thread pool the remote cache should use.  
  +     * At first this will only be for gets.
  +     * <p>
  +     * The default name is "remote_cache_client"
  +     * 
  +     * @return
  +     */
  +    public abstract String getThreadPoolName();
  +    
  +    /**
  +     * Set the anme of the pool to use.  Pools should be defined in the 
  +     * cache.ccf.
  +     * 
  +     * @param name
  +     */
  +    public abstract void setThreadPoolName( String name );
   
  +    /**
  +     * -1 and 0 mean no timeout, this is the default
  +     * if the timeout is -1 or 0, no threadpool will be used.
  +     * 
  +     * @return
  +     */
  +    public abstract int getGetTimeoutMillis();
  +
  +    /**
  +     * -1 means no timeout, this is the default
  +     * if the timeout is -1 or 0, no threadpool will be used.
  +     * If the timeout is greater than 0 a threadpool will be used
  +     * for get requests. 
  +     * 
  +     * @param millis
  +     */
  +    public abstract void setGetTimeoutMillis( int millis );
   }
  
  
  
  1.1                  jakarta-turbine-jcs/src/java/org/apache/jcs/engine/stats/behavior/IStatElement.java
  
  Index: IStatElement.java
  ===================================================================
  package org.apache.jcs.engine.stats.behavior;
  
  
  /*
   * Copyright 2001-2004 The Apache Software Foundation.
   *
   * Licensed under the Apache License, Version 2.0 (the "License")
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
  
  /**
   * IAuxiliaryCacheStats will hold these IStatElements.  
   * 
   * @author aaronsm
   *
   */
  public interface IStatElement {
  
  	/**
  	 * Get the name of the stat element, ex. HitCount
  	 * 
  	 * @return the stat element name
  	 */
  	public abstract String getName();
  
  	/**
  	 * 
  	 * @param name
  	 */
  	public abstract void setName( String name );
  
  	/**
  	 * Get the data, ex. for hit count you would get a String value for some
  	 * number.
  	 * 
  	 * @return String data
  	 */
  	public abstract String getData();
  
  	/**
  	 * Set the data for this element.
  	 * 
  	 * @param data
  	 */
  	public abstract void setData( String data );
  
  }
  
  
  
  1.1                  jakarta-turbine-jcs/src/java/org/apache/jcs/engine/stats/behavior/IStats.java
  
  Index: IStats.java
  ===================================================================
  package org.apache.jcs.engine.stats.behavior;
  
  /*
   * Copyright 2001-2004 The Apache Software Foundation.
   *
   * Licensed under the Apache License, Version 2.0 (the "License")
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
  
  import java.io.Serializable;
  
  /**
   * This interface defines the common behavior for a stats holder. 
   * 
   * @author aaronsm
   *
   */
  public interface IStats extends Serializable {
  
  	/**
  	 * Return generic statistical or historical data.
  	 * 
  	 * @return IStatElement[]
  	 */
  	public abstract IStatElement[] getStatElements( );
  		
  	/**
  	 * Set the generic statistical or historical data.
  	 * 
  	 * @param stats
  	 */
  	public abstract void setStatElements( IStatElement[] stats );	
  
  	/**
  	 * Get the type name, such as "LRU Memory Cache."  No formal type is defined.
  	 * 
  	 * @return String
  	 */
  	public abstract String getTypeName();
  
  	/**
  	 * Set the type name, such as "LRU Memory Cache."  No formal type is defined.
  	 * If we need formal types, we can use the cachetype param
  	 * 
  	 * @param name
  	 */
  	public abstract void setTypeName( String name );		
  }
  
  
  
  1.3       +32 -2     jakarta-turbine-jcs/src/java/org/apache/jcs/auxiliary/javagroups/JavaGroupsCache.java
  
  Index: JavaGroupsCache.java
  ===================================================================
  RCS file: /home/cvs/jakarta-turbine-jcs/src/java/org/apache/jcs/auxiliary/javagroups/JavaGroupsCache.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- JavaGroupsCache.java	16 Jul 2004 01:28:31 -0000	1.2
  +++ JavaGroupsCache.java	6 Jan 2005 01:19:21 -0000	1.3
  @@ -26,6 +26,10 @@
   import org.apache.jcs.engine.behavior.ICacheElement;
   import org.apache.jcs.engine.behavior.ICacheType;
   import org.apache.jcs.engine.control.CompositeCache;
  +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;
   import org.jgroups.Channel;
   import org.jgroups.Message;
   import org.jgroups.View;
  @@ -38,6 +42,9 @@
   
   import java.io.IOException;
   import java.io.Serializable;
  +import java.util.ArrayList;
  +import java.util.Arrays;
  +import java.util.List;
   import java.util.Set;
   import java.util.Vector;
   
  @@ -358,9 +365,32 @@
      */
     public String getStats()
     {
  -    return "";
  +    return getStatistics().toString();
     }
   
  +  /*
  +   * (non-Javadoc)
  +   * 
  +   * @see org.apache.jcs.auxiliary.AuxiliaryCache#getStatistics()
  +   */
  +  public IStats getStatistics()
  +  {
  +    IStats stats = new Stats();
  +    stats.setTypeName( "JavaGroups Cache" );
  +
  +    ArrayList elems = new ArrayList();
  +
  +    IStatElement se = null;
  +
  +    // no data gathered here
  +
  +    // get an array and put them in the Stats object
  +    IStatElement[] ses = (IStatElement[]) elems.toArray( new StatElement[0] );
  +    stats.setStatElements( ses );
  +
  +    return stats;
  +  }   
  +  
     // ---------------------------------------------------------- inner classes
   
       /**
  
  
  
  1.1                  jakarta-turbine-jcs/tempbuild/jcs-1.1.3-dev.jar
  
  	<<Binary file>>
  
  
  1.3       +43 -0     jakarta-turbine-jcs/src/java/org/apache/jcs/admin/JCSAdminBean.java
  
  Index: JCSAdminBean.java
  ===================================================================
  RCS file: /home/cvs/jakarta-turbine-jcs/src/java/org/apache/jcs/admin/JCSAdminBean.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- JCSAdminBean.java	12 Jun 2004 02:34:12 -0000	1.2
  +++ JCSAdminBean.java	6 Jan 2005 01:19:21 -0000	1.3
  @@ -45,6 +45,13 @@
   
     private CompositeCacheManager cacheHub = CompositeCacheManager.getInstance();
   
  +  /**
  +   * Builds up info about each element in a region.
  +   * 
  +   * @param cacheName
  +   * @return
  +   * @throws Exception
  +   */
     public LinkedList buildElementInfo( String cacheName ) throws Exception
     {
       CompositeCache cache =
  @@ -101,6 +108,14 @@
       return records;
     }
   
  +  /**
  +   * Builds up data on every region.
  +   * @TODO we need a most light weight method that does not count
  +   * bytes.  The byte counting can really swamp a server.
  +   * 
  +   * @return list of CacheRegionInfo objects
  +   * @throws Exception
  +   */
     public LinkedList buildCacheInfo() throws Exception
     {
       String[] cacheNames = cacheHub.getCacheNames();
  @@ -127,6 +142,16 @@
       return cacheInfo;
     }
   
  +  /**
  +   * Tries to estimate how much data is in a region.  
  +   * This is expensive.
  +   * If there are any non serializable objects in the region,
  +   * the count will stop when it encouters the first one.
  +   * 
  +   * @param cache
  +   * @return
  +   * @throws Exception
  +   */
     public int getByteCount( CompositeCache cache ) throws Exception
     {
       MemoryCache memCache = cache.getMemoryCache();
  @@ -157,6 +182,11 @@
       return counter.getCount() - 4;
     }
   
  +  /**
  +   * Clears all regions in the cache.
  +   * 
  +   * @throws IOException
  +   */
     public void clearAllRegions() throws IOException
     {
       String[] names = cacheHub.getCacheNames();
  @@ -167,11 +197,24 @@
       }
     }
   
  +  /**
  +   * Clears a particular cache region.
  +   * 
  +   * @param cacheName
  +   * @throws IOException
  +   */
     public void clearRegion( String cacheName ) throws IOException
     {
       cacheHub.getCache( cacheName ).removeAll();
     }
   
  +  /**
  +   * Removes a particular item from a particular region.
  +   * 
  +   * @param cacheName
  +   * @param key
  +   * @throws IOException
  +   */
     public void removeItem( String cacheName, String key ) throws IOException
     {
       cacheHub.getCache( cacheName ).remove( key );
  
  
  

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