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 2006/02/13 20:29:19 UTC

svn commit: r377472 - in /jakarta/jcs/trunk/src: conf/ java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/ java/org/apache/jcs/auxiliary/remote/ java/org/apache/jcs/auxiliary/remote/behavior/ java/org/apache/jcs/auxiliary/remote/server/ java/or...

Author: asmuts
Date: Mon Feb 13 11:29:17 2006
New Revision: 377472

URL: http://svn.apache.org/viewcvs?rev=377472&view=rev
Log:
Changed remote cache client so that it sends a serialized 
form of the object as the value.  This way the remote cache server
does not need the code of the classes that are being cached.  I added
a new serializer interface and a standard implementation.  This will
allows us to include additional serialization mechanisms, such as xstream.
Added initial unit tests for all.

Added:
    jakarta/jcs/trunk/src/conf/cacheRC1.ccf
    jakarta/jcs/trunk/src/conf/cacheRC2.ccf
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/CacheElementSerialized.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICacheElementSerialized.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IElementSerializer.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObservable.java
      - copied, changed from r292446, jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObservable.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObserver.java
      - copied, changed from r292446, jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObserver.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/SerializationConversionUtil.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/StandardSerializer.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/RemoteCacheListenerUnitTest.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/SerializationConversionUtilUnitTest.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/StandardSerializerUnitTest.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/SortedPrefArrayUnitTest.java
      - copied, changed from r295072, jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/TestSortedPrefArray.java
Removed:
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObservable.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObserver.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/TestSortedPrefArray.java
Modified:
    jakarta/jcs/trunk/src/conf/log4j.properties
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryReceiver.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryService.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCache.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheAttributes.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheListener.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheManager.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheAttributes.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServer.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerAttributes.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerListener.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/CompositeCacheManager.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPFilterRemoveHashCode.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestTCPLateral.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/TestRemoteCache.java

Added: jakarta/jcs/trunk/src/conf/cacheRC1.ccf
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/conf/cacheRC1.ccf?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/conf/cacheRC1.ccf (added)
+++ jakarta/jcs/trunk/src/conf/cacheRC1.ccf Mon Feb 13 11:29:17 2006
@@ -0,0 +1,63 @@
+##############################################################
+################## DEFAULT CACHE REGION  #####################
+# sets the default aux value for any non configured caches
+jcs.default=DC
+jcs.default.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.default.cacheattributes.MaxObjects=200000
+jcs.default.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+
+##############################################################
+################## CACHE REGIONS AVAILABLE ###################
+jcs.region.testCache1=RC
+jcs.region.testCache1.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.region.testCache1.cacheattributes.MaxObjects=200000
+jcs.region.testCache1.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+jcs.region.testCache1.elementattributes.IsLateral=true
+jcs.region.testCache1.elementattributes.IsRemote=true
+
+##############################################################
+################## AUXILIARY CACHES AVAILABLE ################
+# standard disk cache
+jcs.auxiliary.DC=org.apache.jcs.auxiliary.disk.indexed.IndexedDiskCacheFactory
+jcs.auxiliary.DC.attributes=org.apache.jcs.auxiliary.disk.indexed.IndexedDiskCacheAttributes
+jcs.auxiliary.DC.attributes.DiskPath=${user.dir}/raf1
+
+# need to make put or invalidate an option
+# just a remove lock to add
+jcs.auxiliary.RC=org.apache.jcs.auxiliary.remote.RemoteCacheFactory
+jcs.auxiliary.RC.attributes=org.apache.jcs.auxiliary.remote.RemoteCacheAttributes
+jcs.auxiliary.RC.attributes.FailoverServers=localhost:1101
+jcs.auxiliary.RC.attributes.LocalPort=1201
+jcs.auxiliary.RC.attributes.RemoveUponRemotePut=false
+jcs.auxiliary.RC.attributes.GetTimeoutMillis=5000
+jcs.auxiliary.RC.attributes.ThreadPoolName=remote_cache_client
+jcs.auxiliary.RC.attributes.GetOnly=false
+
+##############################################################
+################## THREAD POOL CONFIGURATION ###################
+# Default thread pool config
+thread_pool.default.boundarySize=2000
+thread_pool.default.maximumPoolSize=150
+thread_pool.default.minimumPoolSize=4
+thread_pool.default.keepAliveTime=350000
+#RUN ABORT WAIT BLOCK DISCARDOLDEST
+thread_pool.default.whenBlockedPolicy=RUN
+thread_pool.default.startUpSize=4
+
+# Default Cache Event Queue thread pool config, used by auxiliaries
+thread_pool.cache_event_queue.useBoundary=false
+#thread_pool.cache_event_queue.boundarySize=2000
+#thread_pool.cache_event_queue.maximumPoolSize=10
+thread_pool.cache_event_queue.minimumPoolSize=5
+thread_pool.cache_event_queue.keepAliveTime=3500
+#thread_pool.cache_event_queue.whenBlockedPolicy=RUN
+thread_pool.cache_event_queue.startUpSize=5
+
+# 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.whenBlockedPolicy=RUN
+thread_pool.remote_cache_client.startUpSize=4
+

Added: jakarta/jcs/trunk/src/conf/cacheRC2.ccf
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/conf/cacheRC2.ccf?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/conf/cacheRC2.ccf (added)
+++ jakarta/jcs/trunk/src/conf/cacheRC2.ccf Mon Feb 13 11:29:17 2006
@@ -0,0 +1,63 @@
+##############################################################
+################## DEFAULT CACHE REGION  #####################
+# sets the default aux value for any non configured caches
+jcs.default=DC
+jcs.default.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.default.cacheattributes.MaxObjects=200000
+jcs.default.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+
+##############################################################
+################## CACHE REGIONS AVAILABLE ###################
+jcs.region.testCache1=RC
+jcs.region.testCache1.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.region.testCache1.cacheattributes.MaxObjects=200000
+jcs.region.testCache1.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+jcs.region.testCache1.elementattributes.IsLateral=true
+jcs.region.testCache1.elementattributes.IsRemote=true
+
+##############################################################
+################## AUXILIARY CACHES AVAILABLE ################
+# standard disk cache
+jcs.auxiliary.DC=org.apache.jcs.auxiliary.disk.indexed.IndexedDiskCacheFactory
+jcs.auxiliary.DC.attributes=org.apache.jcs.auxiliary.disk.indexed.IndexedDiskCacheAttributes
+jcs.auxiliary.DC.attributes.DiskPath=${user.dir}/raf2
+
+# need to make put or invalidate an option
+# just a remove lock to add
+jcs.auxiliary.RC=org.apache.jcs.auxiliary.remote.RemoteCacheFactory
+jcs.auxiliary.RC.attributes=org.apache.jcs.auxiliary.remote.RemoteCacheAttributes
+jcs.auxiliary.RC.attributes.FailoverServers=localhost:1101
+jcs.auxiliary.RC.attributes.LocalPort=1202
+jcs.auxiliary.RC.attributes.RemoveUponRemotePut=false
+jcs.auxiliary.RC.attributes.GetTimeoutMillis=5000
+jcs.auxiliary.RC.attributes.ThreadPoolName=remote_cache_client
+jcs.auxiliary.RC.attributes.GetOnly=false
+
+##############################################################
+################## THREAD POOL CONFIGURATION ###################
+# Default thread pool config
+thread_pool.default.boundarySize=2000
+thread_pool.default.maximumPoolSize=150
+thread_pool.default.minimumPoolSize=4
+thread_pool.default.keepAliveTime=350000
+#RUN ABORT WAIT BLOCK DISCARDOLDEST
+thread_pool.default.whenBlockedPolicy=RUN
+thread_pool.default.startUpSize=4
+
+# Default Cache Event Queue thread pool config, used by auxiliaries
+thread_pool.cache_event_queue.useBoundary=false
+#thread_pool.cache_event_queue.boundarySize=2000
+#thread_pool.cache_event_queue.maximumPoolSize=10
+thread_pool.cache_event_queue.minimumPoolSize=5
+thread_pool.cache_event_queue.keepAliveTime=3500
+#thread_pool.cache_event_queue.whenBlockedPolicy=RUN
+thread_pool.cache_event_queue.startUpSize=5
+
+# 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.whenBlockedPolicy=RUN
+thread_pool.remote_cache_client.startUpSize=4
+

Modified: jakarta/jcs/trunk/src/conf/log4j.properties
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/conf/log4j.properties?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/conf/log4j.properties (original)
+++ jakarta/jcs/trunk/src/conf/log4j.properties Mon Feb 13 11:29:17 2006
@@ -41,7 +41,7 @@
 #log4j.category.org.apache.jcs.auxiliary.lateral.xmlrpc=INFO
 log4j.category.org.apache.jcs.auxiliary.remote=INFO
 #log4j.category.org.apache.jcs.auxiliary.remote.RemoteCacheFailoverRunner=INFO
-#log4j.category.org.apache.jcs.auxiliary.remote.RemoteCacheListener=INFO
+#log4j.category.org.apache.jcs.auxiliary.remote.RemoteCacheListener=DEBUG
 #log4j.category.org.apache.jcs.auxiliary.remote.RemoteCacheManager=INFO
 #log4j.category.org.apache.jcs.auxiliary.remote.server=INFO
 #log4j.category.org.apache.jcs.utils=WARN,WF

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryReceiver.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryReceiver.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryReceiver.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryReceiver.java Mon Feb 13 11:29:17 2006
@@ -35,7 +35,7 @@
 import org.apache.jcs.auxiliary.lateral.socket.tcp.behavior.ITCPLateralCacheAttributes;
 import org.apache.jcs.engine.behavior.ICache;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
-import org.apache.jcs.engine.behavior.ShutdownObserver;
+import org.apache.jcs.engine.behavior.IShutdownObserver;
 
 import EDU.oswego.cs.dl.util.concurrent.BoundedBuffer;
 import EDU.oswego.cs.dl.util.concurrent.PooledExecutor;
@@ -45,7 +45,7 @@
  * Receives UDP Discovery messages.
  */
 public class UDPDiscoveryReceiver
-    implements Runnable, ShutdownObserver
+    implements Runnable, IShutdownObserver
 {
     private final static Log log = LogFactory.getLog( UDPDiscoveryReceiver.class );
 

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryService.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryService.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryService.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/discovery/UDPDiscoveryService.java Mon Feb 13 11:29:17 2006
@@ -14,8 +14,8 @@
 import org.apache.jcs.auxiliary.lateral.LateralCacheNoWaitFacade;
 import org.apache.jcs.auxiliary.lateral.socket.tcp.behavior.ITCPLateralCacheAttributes;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
-import org.apache.jcs.engine.behavior.ShutdownObservable;
-import org.apache.jcs.engine.behavior.ShutdownObserver;
+import org.apache.jcs.engine.behavior.IShutdownObservable;
+import org.apache.jcs.engine.behavior.IShutdownObserver;
 
 import EDU.oswego.cs.dl.util.concurrent.ClockDaemon;
 import EDU.oswego.cs.dl.util.concurrent.ThreadFactory;
@@ -34,7 +34,7 @@
  *  
  */
 public class UDPDiscoveryService
-    implements ShutdownObserver
+    implements IShutdownObserver
 {
 
     private final static Log log = LogFactory.getLog( UDPDiscoveryService.class );
@@ -68,7 +68,7 @@
     public UDPDiscoveryService( ITCPLateralCacheAttributes lca, ICompositeCacheManager cacheMgr )
     {
         // register for shutdown notification
-        ( (ShutdownObservable) cacheMgr ).registerShutdownObserver( this );
+        ( (IShutdownObservable) cacheMgr ).registerShutdownObserver( this );
 
         this.setTcpLateralCacheAttributes( lca );
 

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCache.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCache.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCache.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCache.java Mon Feb 13 11:29:17 2006
@@ -24,11 +24,6 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
-import EDU.oswego.cs.dl.util.concurrent.Callable;
-import EDU.oswego.cs.dl.util.concurrent.FutureResult;
-import EDU.oswego.cs.dl.util.concurrent.ThreadFactory;
-
 import org.apache.jcs.access.exception.ObjectNotFoundException;
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheAttributes;
@@ -37,18 +32,26 @@
 import org.apache.jcs.engine.CacheConstants;
 import org.apache.jcs.engine.behavior.ICache;
 import org.apache.jcs.engine.behavior.ICacheElement;
+import org.apache.jcs.engine.behavior.ICacheElementSerialized;
 import org.apache.jcs.engine.behavior.IElementAttributes;
+import org.apache.jcs.engine.behavior.IElementSerializer;
 import org.apache.jcs.engine.behavior.IZombie;
 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.serialization.SerializationConversionUtil;
+import org.apache.jcs.utils.serialization.StandardSerializer;
 import org.apache.jcs.utils.threadpool.ThreadPool;
 import org.apache.jcs.utils.threadpool.ThreadPoolManager;
 
+import EDU.oswego.cs.dl.util.concurrent.Callable;
+import EDU.oswego.cs.dl.util.concurrent.FutureResult;
+import EDU.oswego.cs.dl.util.concurrent.ThreadFactory;
+
 /**
  * Client proxy for an RMI remote cache.
- *  
+ * 
  */
 public class RemoteCache
     implements ICache
@@ -71,6 +74,8 @@
 
     private boolean usePoolForGet = false;
 
+    private IElementSerializer elementSerializer = new StandardSerializer();
+
     /**
      * Constructor for the RemoteCache object. This object communicates with a
      * remote cache server. One of these exists for each region. This also holds
@@ -158,7 +163,11 @@
                     {
                         log.debug( "sending item to remote server" );
                     }
-                    remote.update( ce, getListenerId() );
+                    
+                    // convert so we don't have to know about the object on the other end.
+                    ICacheElementSerialized serialized = SerializationConversionUtil.getSerializedCacheElement( ce, this.elementSerializer );                    
+                    
+                    remote.update( serialized, getListenerId() );
                 }
                 catch ( NullPointerException npe )
                 {
@@ -206,6 +215,14 @@
             {
                 retVal = remote.get( cacheName, sanitized( key ) );
             }
+
+            // Eventually the instance of will not be necessary.
+            if ( retVal != null && retVal instanceof ICacheElementSerialized )
+            {
+                retVal = SerializationConversionUtil.getDeSerializedCacheElement( (ICacheElementSerialized) retVal,
+                                                                                 this.elementSerializer );
+            }
+
         }
         catch ( ObjectNotFoundException one )
         {
@@ -215,7 +232,6 @@
         catch ( Exception ex )
         {
             handleException( ex, "Failed to get " + key + " from " + cacheName );
-            // never executes; just keep the compiler happy.
         }
 
         return retVal;
@@ -317,7 +333,7 @@
         // we again wrap
         // it into a new MarsahlledObject for "escape" purposes during the get
         // operation.
-        //return s.getClass().getName().startsWith("java.") && !(s instanceof
+        // return s.getClass().getName().startsWith("java.") && !(s instanceof
         // MarshalledObject) ? s : new MarshalledObject(s);
 
         // avoid this step for now, [problem with group id wrapper]
@@ -520,7 +536,7 @@
     {
         log.error( "Disabling remote cache due to error " + msg );
         log.error( ex );
-        //log.error( ex.toString() );
+        // log.error( ex.toString() );
 
         remote = new ZombieRemoteCacheService();
         // may want to flush if region specifies
@@ -618,6 +634,23 @@
     }
 
     /**
+     * @param elementSerializer
+     *            The elementSerializer to set.
+     */
+    public void setElementSerializer( IElementSerializer elementSerializer )
+    {
+        this.elementSerializer = elementSerializer;
+    }
+
+    /**
+     * @return Returns the elementSerializer.
+     */
+    public IElementSerializer getElementSerializer()
+    {
+        return elementSerializer;
+    }
+
+    /**
      * Debugging info.
      * 
      * @return basic info about the RemoteCache
@@ -631,7 +664,7 @@
      * Allows us to set the daemon status on the clockdaemon
      * 
      * @author aaronsm
-     *  
+     * 
      */
     class MyThreadFactory
         implements ThreadFactory

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheAttributes.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheAttributes.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheAttributes.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheAttributes.java Mon Feb 13 11:29:17 2006
@@ -22,7 +22,7 @@
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheConstants;
 
 /**
- * Description of the Class
+ * These objects are used to configure the remote cache client.  
  *  
  */
 public class RemoteCacheAttributes
@@ -37,7 +37,7 @@
 
     private int remotePort;
 
-    /*
+    /**
      * failover servers will be used by local caches one at a time. Listeners
      * will be registered with all cluster servers. If we add a get from cluster
      * attribute we will have the ability to chain clusters and have them get
@@ -68,7 +68,7 @@
     // must be greater than 0 for a pool to be used.
     private int getTimeoutMillis = -1;
 
-    /** Constructor for the RemoteCacheAttributes object */
+    /** Default constructor for the RemoteCacheAttributes object */
     public RemoteCacheAttributes()
     {
         super();
@@ -428,15 +428,16 @@
     public String toString()
     {
         StringBuffer buf = new StringBuffer();
-        buf.append( "\n remoteHost = " + this.remoteHost );
-        buf.append( "\n remotePort = " + this.remotePort );
-        buf.append( "\n cacheName = " + this.cacheName );
-        buf.append( "\n removeUponRemotePut = " + this.removeUponRemotePut );
-        buf.append( "\n getOnly = " + getOnly );
-        buf.append( "\n getTimeoutMillis = " + getTimeoutMillis );
-        buf.append( "\n threadPoolName = " + threadPoolName );
-        buf.append( "\n remoteType = " + remoteType );
-        buf.append( "\n localClusterConsistency = " + localClusterConsistency );
+        buf.append( "\n RemoteCacheAttributes " );
+        buf.append( "\n remoteHost = [" + this.remoteHost + "]" );
+        buf.append( "\n remotePort = [" + this.remotePort + "]" );
+        buf.append( "\n cacheName = [" + this.cacheName + "]" );
+        buf.append( "\n removeUponRemotePut = [" + this.removeUponRemotePut + "]" );
+        buf.append( "\n getOnly = [" + getOnly + "]" );
+        buf.append( "\n getTimeoutMillis = [" + getGetTimeoutMillis() + "]" );
+        buf.append( "\n threadPoolName = [" + getThreadPoolName() + "]" );
+        buf.append( "\n remoteType = [" + remoteType + "]" );
+        buf.append( "\n localClusterConsistency = [" + getLocalClusterConsistency() + "]" );
         return buf.toString();
     }
 

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheListener.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheListener.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheListener.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheListener.java Mon Feb 13 11:29:17 2006
@@ -27,14 +27,21 @@
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheConstants;
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheListener;
 import org.apache.jcs.engine.behavior.ICacheElement;
+import org.apache.jcs.engine.behavior.ICacheElementSerialized;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
+import org.apache.jcs.engine.behavior.IElementSerializer;
 import org.apache.jcs.engine.control.CompositeCache;
 import org.apache.jcs.engine.control.CompositeCacheManager;
+import org.apache.jcs.utils.serialization.SerializationConversionUtil;
+import org.apache.jcs.utils.serialization.StandardSerializer;
 
 /**
  * Registered with RemoteCache server. The server updates the local caches via
  * this listener. Each server asings a unique listener id for a listener.
- *  
+ * <p>
+ * One listener is used per remote cache server. The same listener is used for
+ * all the regions that talk to a particular server.
+ * 
  */
 public class RemoteCacheListener
     implements IRemoteCacheListener, IRemoteCacheConstants, Serializable
@@ -52,21 +59,21 @@
     /** The remote cache configuration object. */
     protected IRemoteCacheAttributes irca;
 
-    /** Number of put requests received */
+    /** Number of put requests received. For debugging only. */
     protected int puts = 0;
 
-    /** Number of remove requests received */
+    /** Number of remove requests received. For debugging only. */
     protected int removes = 0;
 
-    /**
-     * This is set by the remote cache server.
-     */
+    /** This is set by the remote cache server. */
     protected long listenerId = 0;
 
+    private transient IElementSerializer elementSerializer = new StandardSerializer();
+    
     /**
      * Only need one since it does work for all regions, just reference by
      * multiple region names.
-     * 
+     * <p>
      * The constructor exports this object, making it available to receive
      * incoming calls. The calback port is anonymous unless a local port vlaue
      * was specified in the configurtion.
@@ -79,7 +86,7 @@
         this.irca = irca;
 
         this.cacheMgr = cacheMgr;
-        
+
         // Export this remote object to make it available to receive incoming
         // calls,
         // using an anonymous port unless the local port is specified.
@@ -157,13 +164,16 @@
         return irca.getRemoteType();
     }
 
-    //////////////////////////// implements the IRemoteCacheListener interface.
+    // ////////////////////////// implements the IRemoteCacheListener interface.
     // //////////////
     /**
-     * Just remove the element since it has been updated elsewhere cd should be
-     * incomplete for faster transmission. We don't want to pass data only
-     * invalidation. The next time it is used the local cache will get the new
-     * version from the remote store.
+     * If this is configured to remove on put, then remove the element since it
+     * has been updated elsewhere. cd should be incomplete for faster
+     * transmission. We don't want to pass data only invalidation. The next time
+     * it is used the local cache will get the new version from the remote
+     * store.
+     * <p>
+     * If remove on put is not ocnfigured, then update the item.
      * 
      * @param cb
      * @throws IOException
@@ -181,21 +191,47 @@
         }
         else
         {
+            puts++;
             if ( log.isDebugEnabled() )
             {
                 log.debug( "PUTTING ELEMENT FROM REMOTE, ( updating ) " );
                 log.debug( "cb = " + cb );
 
-                puts++;
                 if ( puts % 100 == 0 )
                 {
                     log.debug( "puts = " + puts );
                 }
             }
 
-            getCacheManager();
+            ensureCacheManager();
             CompositeCache cache = cacheMgr.getCache( cb.getCacheName() );
 
+            // Eventually the instance of will not be necessary.
+            if ( cb != null && cb instanceof ICacheElementSerialized )
+            {
+                if ( log.isDebugEnabled() )
+                {
+                    log.debug( "Object needs to be deserialized." );
+                }
+                try
+                {
+                    cb = SerializationConversionUtil.getDeSerializedCacheElement( (ICacheElementSerialized) cb,
+                                                                                     this.elementSerializer );
+                    if ( log.isDebugEnabled() )
+                    {
+                        log.debug( "Deserialized result = " + cb );
+                    }
+                }
+                catch ( IOException e )
+                {
+                    throw e;
+                }
+                catch ( ClassNotFoundException e )
+                {
+                    log.error( "Received a serialized version of a class that we don't know about.", e );
+                }
+            }            
+            
             cache.localUpdate( cb );
         }
 
@@ -211,9 +247,9 @@
     public void handleRemove( String cacheName, Serializable key )
         throws IOException
     {
+        removes++;
         if ( log.isDebugEnabled() )
         {
-            removes++;
             if ( removes % 100 == 0 )
             {
                 log.debug( "removes = " + removes );
@@ -222,7 +258,7 @@
             log.debug( "handleRemove> cacheName=" + cacheName + ", key=" + key );
         }
 
-        getCacheManager();
+        ensureCacheManager();
         CompositeCache cache = cacheMgr.getCache( cacheName );
 
         cache.localRemove( key );
@@ -240,7 +276,7 @@
         {
             log.debug( "handleRemoveAll> cacheName=" + cacheName );
         }
-        getCacheManager();
+        ensureCacheManager();
         CompositeCache cache = cacheMgr.getCache( cacheName );
         cache.localRemoveAll();
     }
@@ -260,14 +296,14 @@
         // TODO consider what to do here, we really don't want to
         // dispose, we just want to disconnect.
         // just allow the cache to go into error recovery mode.
-        //getCacheManager().freeCache( cacheName, true );
+        // getCacheManager().freeCache( cacheName, true );
     }
 
     /**
      * Gets the cacheManager attribute of the RemoteCacheListener object. This
      * is one of the few places that force the cache to be a singleton.
      */
-    protected void getCacheManager()
+    protected void ensureCacheManager()
     {
         if ( cacheMgr == null )
         {

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheManager.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheManager.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheManager.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheManager.java Mon Feb 13 11:29:17 2006
@@ -302,8 +302,9 @@
 
     /**
      * Gets a RemoteCacheNoWait from the RemoteCacheManager. The
-     * RemoteCacheNoWait are identified by the cache name value of the
+     * RemoteCacheNoWait objects are identified by the cache name value of the
      * RemoteCacheAttributes object.
+     * 
      * @param cattr
      * 
      * @return The cache value

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheAttributes.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheAttributes.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheAttributes.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheAttributes.java Mon Feb 13 11:29:17 2006
@@ -19,20 +19,21 @@
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 
 /**
- * Description of the Interface
- *  
+ * This specifies what a remote cache configuration object should look like.
+ * 
  */
 public interface IRemoteCacheAttributes
     extends AuxiliaryCacheAttributes
 {
 
-    /*
+    /**
      * A remote cache is either a local cache or a cluster cache.
      */
-    /** Description of the Field */
     public static int LOCAL = 0;
 
-    /** Description of the Field */
+    /**
+     * A remote cache is either a local cache or a cluster cache.
+     */
     public static int CLUSTER = 1;
 
     /**
@@ -65,11 +66,12 @@
      */
     public void setRemoteType( int p );
 
-    // specifies which server in the list we are listening to
-    // if the number is greater than 0 we will try to move to 0 position
-    // the primary is added as position 1 if it is present
     /**
-     * Gets the failoverIndex attribute of the IRemoteCacheAttributes object
+     * Gets the failoverIndex attribute of the IRemoteCacheAttributes object.
+     * <p>
+     * This specifies which server in the list we are listening to if the number
+     * is greater than 0 we will try to move to 0 position the primary is added
+     * as position 1 if it is present
      * 
      * @return The failoverIndex value
      */

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServer.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServer.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServer.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServer.java Mon Feb 13 11:29:17 2006
@@ -72,10 +72,6 @@
     /** timing -- if we should record operation times. */
     protected final static boolean timing = true;
 
-    //true;
-    /** Description of the Field */
-    public String className;
-
     private int puts = 0;
 
     // Maps cache name to CacheListeners object.
@@ -121,10 +117,6 @@
         throws IOException, NotBoundException
     {
 
-        String s = this.getClass().getName();
-        int idx = s.lastIndexOf( "." );
-        this.className = s.substring( idx + 1 );
-
         cacheManager = createCacheManager( prop );
 
         // cacheManager would have created a number of ICache objects.
@@ -256,12 +248,12 @@
      * An update can come from either a local cache's remote auxiliary, or it
      * can come from a remote server. A remote server is considered a a source
      * of type cluster.
-     * 
+     * <p>
      * If the update came from a cluster, then we should tell the cache manager
      * that this was a remote put. This way, any lateral and remote auxiliaries
      * configured for the region will not be updated. This is basically how a
      * remote listener works when plugged into a local cache.
-     * 
+     * <p>
      * If the cluster is configured to keep local cluster consistency, then all
      * listeners will be updated. This allows cluster server A to update cluster
      * server B and then B to update its clients if it is told to keep local
@@ -271,7 +263,7 @@
      * the leavess. The remote cluster, with local cluster consistency, allows
      * you to update leaves. This basically allows you to have a failover remote
      * server.
-     * 
+     * <p>
      * Since currently a cluster will not try to get from other cluster servers,
      * you can scale a bit with a cluster configuration. Puts and removes will
      * be broadcasted to all clients, but the get load on a remote server can be
@@ -370,7 +362,6 @@
 
                 // UPDATE LOCALS IF A REQUEST COMES FROM A CLUSTER
                 // IF LOCAL CLUSTER CONSISTENCY IS CONFIGURED
-
                 if ( !fromCluster || ( fromCluster && rcsa.getLocalClusterConsistency() ) )
                 {
 
@@ -793,7 +784,7 @@
                 if ( !q.isWorking() )
                 {
                     itr.remove();
-                    p1( "Cache event queue " + q + " is not working and removed from cache server." );
+                    print( "Cache event queue " + q + " is not working and removed from cache server." );
                 }
             }
         }
@@ -857,7 +848,7 @@
                         id = listenerIdB;
                         // in case it needs synchronization
                         log.info( "adding vm listener under new id = " + listenerIdB );
-                        p1( "adding vm listener under new id = " + listenerIdB );
+                        print( "adding vm listener under new id = " + listenerIdB );
                     }
                     else
                     {
@@ -1099,7 +1090,7 @@
      * 
      * @param s
      */
-    private static void p1( String s )
+    private static void print( String s )
     {
         System.out.println( "RemoteCacheServer:" + s + " >" + Thread.currentThread().getName() );
     }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerAttributes.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerAttributes.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerAttributes.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerAttributes.java Mon Feb 13 11:29:17 2006
@@ -23,7 +23,7 @@
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheConstants;
 
 /**
- * Description of the Class
+ * These attributes are used to configure the remote cache server.
  *  
  */
 public class RemoteCacheServerAttributes
@@ -39,8 +39,8 @@
 
     private int remotePort;
 
-    /*
-     * failover servers will be used by local caches one at a time. Listeners
+    /**
+     * Failover servers will be used by local caches one at a time. Listeners
      * will be registered with all cluster servers. If we add a get from cluster
      * attribute we will have the ability to chain clusters and have them get
      * from each other.
@@ -63,7 +63,7 @@
 
     private String configFileName = "";
 
-    /** Constructor for the RemoteCacheAttributes object */
+    /** Default constructor for the RemoteCacheAttributes object */
     public RemoteCacheServerAttributes()
     {
         super();

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerListener.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerListener.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerListener.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/server/RemoteCacheServerListener.java Mon Feb 13 11:29:17 2006
@@ -31,10 +31,9 @@
 import org.apache.jcs.engine.control.CompositeCache;
 import org.apache.jcs.engine.control.CompositeCacheManager;
 
-
 /**
- * This listener class is for inter cache commumication.
- *  
+ * This listener class is for inter remote cache commumication.
+ * 
  */
 public class RemoteCacheServerListener
     implements IRemoteCacheListener, IRemoteCacheConstants, Serializable
@@ -43,25 +42,29 @@
 
     private final static Log log = LogFactory.getLog( RemoteCacheServerListener.class );
 
-    /** cacheMgr */
+    /** The hub, the cache manager, the master controller . . . */
     protected static transient CompositeCacheManager cacheMgr;
 
-    /** Singleton instance of the listner */
+    /** Singleton instance of the listener */
     protected static IRemoteCacheListener instance;
 
-    /** configuration attributes. */
+    /** Remote cache configuration attributes. */
     protected IRemoteCacheAttributes irca;
 
-    /** Description of the Field */
+    /** How many times update was called. Used for debugging only. */
     private int puts = 0;
 
+    /** The number of times removed was called. Used for debugging only. */
     private int removes = 0;
 
     /**
      * Only need one since it does work for all regions, just reference by
      * multiple region names.
+     * <p>
+     * Exports this remote object to make it available to receive incoming
+     * calls, using an anonymous port if none is specified.
      * 
-     * @param irca
+     * @param irca the atttributes, the LocalPort specifies the receiving port.
      */
     protected RemoteCacheServerListener( IRemoteCacheAttributes irca )
     {
@@ -70,14 +73,13 @@
         // may need to add to ICacheManager interface to handle
         // the source arument extended update and remove methods
 
-        // causes circular reference, unfortunate, becasue the
+        // causes circular reference, unfortunately, becasue the
         // the overhead is higer
         // will need to pass a refernce thru
-        //cacheMgr = CacheManagerFactory.getInstance();
+        // cacheMgr = CacheManagerFactory.getInstance();
 
         // Export this remote object to make it available to receive incoming
-        // calls,
-        // using an anonymous port.
+        // calls, using an anonymous port if none is specified.
         try
         {
             if ( irca.getLocalPort() != 0 )
@@ -98,7 +100,7 @@
     }
 
     /**
-     * let the remote cache set a listener_id. Since there is only one listerenr
+     * Let the remote cache set a listener_id. Since there is only one listerenr
      * for all the regions and every region gets registered? the id shouldn't be
      * set if it isn't zero. If it is we assume that it is a reconnect.
      * 
@@ -127,9 +129,8 @@
     {
 
         // set the manager since we are in use
-        //getCacheManager();
+        // getCacheManager();
 
-        //p( "get listenerId" );
         if ( log.isDebugEnabled() )
         {
             log.debug( "get listenerId = " + RemoteCacheServerInfo.listenerId );
@@ -155,13 +156,14 @@
 
     /**
      * Gets the instance attribute of the RemoteCacheServerListener class
+     * 
      * @param irca
      * 
      * @return The instance value
      */
     public static IRemoteCacheListener getInstance( IRemoteCacheAttributes irca )
     {
-        //throws IOException, NotBoundException
+        // throws IOException, NotBoundException
         if ( instance == null )
         {
             synchronized ( RemoteCacheServerListener.class )
@@ -172,26 +174,28 @@
                 }
             }
         }
-        //instance.incrementClients();
+        // instance.incrementClients();
         return instance;
     }
 
-    //////////////////////////// implements the IRemoteCacheListener interface.
+    // ////////////////////////// implements the IRemoteCacheListener interface.
     // //////////////
     /**
      * Just remove the element since it has been updated elsewhere cd should be
      * incomplete for faster transmission. We don't want to pass data only
      * invalidation. The next time it is used the local cache will get the new
      * version from the remote store
+     * 
      * @param cb
      * @throws IOException
      */
     public void handlePut( ICacheElement cb )
         throws IOException
     {
+        // not a thread safe increment, but that doesn't matter yet.
+        puts++;
         if ( log.isInfoEnabled() )
         {
-            puts++;
             if ( puts % 100 == 0 )
             {
                 log.info( "puts from remote server = " + puts );
@@ -202,9 +206,9 @@
         cache.localUpdate( cb );
     }
 
-    /** 
+    /**
      * Calls remove for the key on the named cache
-     *  
+     * 
      * @param cacheName
      * @param key
      * @throws IOException
@@ -212,9 +216,10 @@
     public void handleRemove( String cacheName, Serializable key )
         throws IOException
     {
+        // not a thread safe increment, but that doesn't matter yet.
+        removes++;
         if ( log.isInfoEnabled() )
         {
-            removes++;
             if ( removes % 100 == 0 )
             {
                 log.info( "removes = " + removes );
@@ -233,7 +238,7 @@
         cache.localRemove( key );
     }
 
-    /** 
+    /**
      * Calls removeAl on the named cache region.
      * 
      * @param cacheName
@@ -251,7 +256,7 @@
         cache.removeAll();
     }
 
-    /** 
+    /**
      * Calls freeCache on the named region.
      * 
      * @param cacheName

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/engine/CacheElementSerialized.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/engine/CacheElementSerialized.java?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/engine/CacheElementSerialized.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/engine/CacheElementSerialized.java Mon Feb 13 11:29:17 2006
@@ -0,0 +1,130 @@
+package org.apache.jcs.engine;
+
+import java.io.Serializable;
+
+import org.apache.jcs.engine.behavior.ICacheElementSerialized;
+import org.apache.jcs.engine.behavior.IElementAttributes;
+
+/*
+ * 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.
+ */
+
+/**
+ * Either serialized value or the value should be null;
+ */
+public class CacheElementSerialized
+    implements ICacheElementSerialized
+{
+
+    private static final long serialVersionUID = -7265084818647601874L;
+
+    /** The name of the cache region. This is a namespace. */
+    private final String cacheName;
+
+    /** This is the cache key by which the value can be referenced. */
+    private final Serializable key;
+
+    private final byte[] serializedValue;
+
+    /**
+     * These attributes hold information about the element and what it is
+     * allowed to do.
+     */
+    private IElementAttributes elementAttributes;
+
+    /**
+     * 
+     * @param cacheNameArg
+     * @param keyArg
+     * @param serializedValueArg
+     * @param elementAttributesArg
+     */
+    public CacheElementSerialized( String cacheNameArg, Serializable keyArg, byte[] serializedValueArg,
+                                  IElementAttributes elementAttributesArg )
+    {
+        this.cacheName = cacheNameArg;
+        this.key = keyArg;
+        this.serializedValue = serializedValueArg;
+        this.elementAttributes = elementAttributesArg;
+    }
+
+    public String getCacheName()
+    {
+        return this.cacheName;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.jcs.engine.behavior.ICacheElement#getKey()
+     */
+    public Serializable getKey()
+    {
+        return this.key;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.jcs.engine.behavior.ICacheElementSerialized#getSerializedValue()
+     */
+    public byte[] getSerializedValue()
+    {
+        return this.serializedValue;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.jcs.engine.behavior.ICacheElement#getElementAttributes()
+     */
+    public IElementAttributes getElementAttributes()
+    {
+        return this.elementAttributes;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see org.apache.jcs.engine.behavior.ICacheElement#setElementAttributes(org.apache.jcs.engine.behavior.IElementAttributes)
+     */
+    public void setElementAttributes( IElementAttributes attr )
+    {
+        this.elementAttributes = attr;
+    }
+
+    /**
+     * Backward compatibility.
+     */
+    public Serializable getVal()
+    {
+        return null;
+    }
+
+    /**
+     * For debugging only.
+     */
+    public String toString()
+    {
+        StringBuffer buf = new StringBuffer();
+        buf.append( "\n CacheElementSerialized: " );
+        buf.append( "\n CacheName = [" + getCacheName() + "]" );
+        buf.append( "\n Key = [" + getKey() + "]" );
+        buf.append( "\n SerializedValue = " + getSerializedValue() );
+        buf.append( "\n ElementAttributes = " + getElementAttributes() );
+        return buf.toString();
+    }
+
+}

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICacheElementSerialized.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICacheElementSerialized.java?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICacheElementSerialized.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICacheElementSerialized.java Mon Feb 13 11:29:17 2006
@@ -0,0 +1,75 @@
+package org.apache.jcs.engine.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 behavior of the serialized element wrapper.
+ * <p>
+ * The value is stored as a byte array. This should allow for a variety of
+ * serialization mechanisms.
+ * <p>
+ * This currently extends ICacheElement for backward compatibility.
+ * 
+ * @author Aaron Smuts
+ * 
+ */
+public interface ICacheElementSerialized
+    extends ICacheElement
+{
+
+    /**
+     * Gets the cacheName attribute of the ICacheElement object. The cacheName
+     * is also known as the region name.
+     * 
+     * @return The cacheName value
+     */
+    public String getCacheName();
+
+    /**
+     * Gets the key attribute of the ICacheElementSerialized object. This is the
+     * standard key that the value can be reference by.
+     * 
+     * @return The key value
+     */
+    public Serializable getKey();
+    
+    /**
+     * Gets the value attribute of the ICacheElementSerialized object. This is
+     * the value the client cached seriailzed by some mechanism.
+     * 
+     * @return The serialized value
+     */
+    public byte[] getSerializedValue();
+
+    /**
+     * Gets the attributes attribute of the ICacheElement object
+     * 
+     * @return The attributes value
+     */
+    public IElementAttributes getElementAttributes();
+
+    /**
+     * Sets the attributes attribute of the ICacheElement object
+     * 
+     * @param attr
+     *            The new attributes value
+     */
+    public void setElementAttributes( IElementAttributes attr );
+
+}

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IElementSerializer.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IElementSerializer.java?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IElementSerializer.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IElementSerializer.java Mon Feb 13 11:29:17 2006
@@ -0,0 +1,37 @@
+package org.apache.jcs.engine.behavior;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * Defines the behavior for cache element serializers. This layer of abstraction
+ * allows us to plug in different serialization mechanisms, such as XStream.
+ * 
+ * @author Aaron Smuts
+ * 
+ */
+public interface IElementSerializer
+{
+
+    /**
+     * Turns an object into a byte array.
+     * 
+     * @param obj
+     * @return
+     * @throws IOException
+     */
+    public abstract byte[] serialize( Serializable obj )
+        throws IOException;
+
+    /**
+     * Turns a byte array into an object.
+     * 
+     * @param bytes
+     * @return
+     * @throws IOException
+     * @throws ClassNotFoundException thrown if we don't know the object.
+     */
+    public abstract Object deSerialize( byte[] bytes )
+        throws IOException, ClassNotFoundException;
+
+}

Copied: jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObservable.java (from r292446, jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObservable.java)
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObservable.java?p2=jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObservable.java&p1=jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObservable.java&r1=292446&r2=377472&rev=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObservable.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObservable.java Mon Feb 13 11:29:17 2006
@@ -34,20 +34,20 @@
  * @author Aaron Smuts
  *
  */
-public interface ShutdownObservable
+public interface IShutdownObservable
 {
 
     /**
      * Registers an observer with the observable object.
      * @param observer
      */
-    abstract void registerShutdownObserver( ShutdownObserver observer );
+    abstract void registerShutdownObserver( IShutdownObserver observer );
 
     /**
      * Deregisters the observer with the observable.
      * 
      * @param observer
      */
-    abstract void deregisterShutdownObserver( ShutdownObserver observer );
+    abstract void deregisterShutdownObserver( IShutdownObserver observer );
 
 }

Copied: jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObserver.java (from r292446, jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObserver.java)
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObserver.java?p2=jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObserver.java&p1=jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObserver.java&r1=292446&r2=377472&rev=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ShutdownObserver.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/IShutdownObserver.java Mon Feb 13 11:29:17 2006
@@ -28,7 +28,7 @@
  * @author Aaron Smuts
  *
  */
-public interface ShutdownObserver
+public interface IShutdownObserver
 {
 
     /**

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/CompositeCacheManager.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/CompositeCacheManager.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/CompositeCacheManager.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/CompositeCacheManager.java Mon Feb 13 11:29:17 2006
@@ -39,8 +39,8 @@
 import org.apache.jcs.engine.behavior.ICompositeCacheAttributes;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementAttributes;
-import org.apache.jcs.engine.behavior.ShutdownObservable;
-import org.apache.jcs.engine.behavior.ShutdownObserver;
+import org.apache.jcs.engine.behavior.IShutdownObservable;
+import org.apache.jcs.engine.behavior.IShutdownObserver;
 import org.apache.jcs.engine.stats.CacheStats;
 import org.apache.jcs.engine.stats.behavior.ICacheStats;
 import org.apache.jcs.utils.threadpool.ThreadPoolManager;
@@ -50,7 +50,7 @@
  * way to shutdown the caching system as a whole.
  */
 public class CompositeCacheManager
-    implements IRemoteCacheConstants, Serializable, ICompositeCacheManager, ShutdownObservable
+    implements IRemoteCacheConstants, Serializable, ICompositeCacheManager, IShutdownObservable
 {
     private static final long serialVersionUID = 7598584393134401756L;
 
@@ -469,7 +469,7 @@
             Iterator it = shutdownObservers.iterator();
             while ( it.hasNext() )
             {
-                ShutdownObserver observer = (ShutdownObserver)it.next();
+                IShutdownObserver observer = (IShutdownObserver)it.next();
                 observer.shutdown();
             }
         }
@@ -653,7 +653,7 @@
      * 
      * @param observer
      */
-    public void registerShutdownObserver( ShutdownObserver observer )
+    public void registerShutdownObserver( IShutdownObserver observer )
     {
         // synchronized to take care of iteration safety
         // during shutdown.
@@ -669,7 +669,7 @@
      * 
      * @see org.apache.jcs.engine.behavior.ShutdownObservable#deregisterShutdownObserver(org.apache.jcs.engine.behavior.ShutdownObserver)
      */
-    public void deregisterShutdownObserver( ShutdownObserver observer )
+    public void deregisterShutdownObserver( IShutdownObserver observer )
     {
         synchronized ( shutdownObservers )
         {

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/SerializationConversionUtil.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/SerializationConversionUtil.java?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/SerializationConversionUtil.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/SerializationConversionUtil.java Mon Feb 13 11:29:17 2006
@@ -0,0 +1,122 @@
+package org.apache.jcs.utils.serialization;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.jcs.engine.CacheElement;
+import org.apache.jcs.engine.CacheElementSerialized;
+import org.apache.jcs.engine.behavior.ICacheElement;
+import org.apache.jcs.engine.behavior.ICacheElementSerialized;
+import org.apache.jcs.engine.behavior.IElementSerializer;
+
+/**
+ * This uses a supplied Serialer to convert to and from cache elements.
+ * 
+ * @author Aaron Smuts
+ * 
+ */
+public class SerializationConversionUtil
+{
+
+    private final static Log log = LogFactory.getLog( SerializationConversionUtil.class );
+
+    /**
+     * This returns a wrapper that has a serialized version of the value instead
+     * of the value.
+     * 
+     * @param element
+     * @param elementSerializer
+     *            the serializer to be used.
+     * @return null for null;
+     * @throws IOException
+     */
+    public static ICacheElementSerialized getSerializedCacheElement( ICacheElement element,
+                                                                    IElementSerializer elementSerializer )
+        throws IOException
+    {
+        if ( element == null )
+        {
+            return null;
+        }
+
+        byte[] serialzedValue = null;
+
+        if ( elementSerializer != null )
+        {
+            try
+            {
+                serialzedValue = elementSerializer.serialize( element.getVal() );
+            }
+            catch ( IOException e )
+            {
+                log.error( "Problem serializing object.", e );
+                throw e;
+            }
+        }
+        else
+        {
+            // we could just use the default.
+            log.warn( "ElementSerializer is null.  Could not serialize object." );
+            throw new IOException( "Could not serialize object.  The ElementSerializer is null." );
+        }
+        ICacheElementSerialized serialized = new CacheElementSerialized( element.getCacheName(), element.getKey(),
+                                                                         serialzedValue, element.getElementAttributes() );
+
+        return serialized;
+    }
+
+    /**
+     * This returns a wrapper that has a de-serialized version of the value
+     * instead of the serialized value.
+     * 
+     * @param serialized
+     * @param elementSerializer
+     *            the serializer to be used.
+     * @return null for null;
+     * @throws IOException
+     * @throws ClassNotFoundException
+     */
+    public static ICacheElement getDeSerializedCacheElement( ICacheElementSerialized serialized,
+                                                            IElementSerializer elementSerializer )
+        throws IOException, ClassNotFoundException
+    {
+        if ( serialized == null )
+        {
+            return null;
+        }
+
+        Object deSerialzedValue = null;
+
+        if ( elementSerializer != null )
+        {
+            try
+            {
+                try
+                {
+                    deSerialzedValue = elementSerializer.deSerialize( serialized.getSerializedValue() );
+                }
+                catch ( ClassNotFoundException e )
+                {
+                    log.error( "Problem de-serializing object.", e );
+                    throw e;
+                }
+            }
+            catch ( IOException e )
+            {
+                log.error( "Problem de-serializing object.", e );
+                throw e;
+            }
+        }
+        else
+        {
+            // we could just use the default.
+            log.warn( "ElementSerializer is null.  Could not serialize object." );
+        }
+        ICacheElement deSerialized = new CacheElement( serialized.getCacheName(), serialized.getKey(), deSerialzedValue );
+        deSerialized.setElementAttributes( serialized.getElementAttributes() );
+
+        return deSerialized;
+    }
+
+}

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/StandardSerializer.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/StandardSerializer.java?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/StandardSerializer.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/utils/serialization/StandardSerializer.java Mon Feb 13 11:29:17 2006
@@ -0,0 +1,75 @@
+package org.apache.jcs.utils.serialization;
+
+import java.io.BufferedInputStream;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+
+import org.apache.jcs.engine.behavior.IElementSerializer;
+
+/**
+ * Performs default serialization and de-serialization.
+ * 
+ *
+ * @author Aaron Smuts
+ *
+ */
+public class StandardSerializer
+    implements IElementSerializer
+{
+
+    /**
+     * Serializes an object using default serilaization.
+     * 
+     */
+    public byte[] serialize( Serializable obj )
+        throws IOException
+    {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        ObjectOutputStream oos = new ObjectOutputStream( baos );
+        try
+        {
+            oos.writeObject( obj );
+        }
+        finally
+        {
+            oos.close();
+        }
+        return baos.toByteArray();
+    }
+
+    /**
+     * Uses default de-serialization to turn a byte array into an object.  All exceptions are converted into 
+     * IOExceptions.
+     */
+    public Object deSerialize( byte[] data )
+        throws IOException, ClassNotFoundException
+    {
+        ByteArrayInputStream bais = new ByteArrayInputStream( data );
+        BufferedInputStream bis = new BufferedInputStream( bais );
+        ObjectInputStream ois = new ObjectInputStream( bis );
+        try
+        {
+            try
+            {
+                return ois.readObject();
+            }
+            catch ( IOException e )
+            {
+                throw e;
+            }
+            catch ( ClassNotFoundException e )
+            {
+                throw e;
+            }
+        }
+        finally
+        {
+            ois.close();
+        }
+    }
+
+}

Modified: jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPFilterRemoveHashCode.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPFilterRemoveHashCode.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPFilterRemoveHashCode.java (original)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPFilterRemoveHashCode.java Mon Feb 13 11:29:17 2006
@@ -167,12 +167,5 @@
         {
             System.out.println( s );
         }
-        else
-        {
-            //if ( log.isInfoEnabled() )
-            //{
-            //    log.info( s );
-            //}
-        }
     }
 }

Modified: jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestTCPLateral.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestTCPLateral.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestTCPLateral.java (original)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestTCPLateral.java Mon Feb 13 11:29:17 2006
@@ -40,7 +40,8 @@
         throws Exception
     {
 
-        JCS jcs = JCS.getInstance( "test" );
+        // force initialization
+        JCS.getInstance( "test" );
 
         TCPLateralCacheAttributes lac = new TCPLateralCacheAttributes();
         lac.setTransmissionType( LateralCacheAttributes.TCP );
@@ -73,6 +74,10 @@
 
     }
 
+    /**
+     * 
+     * @throws Exception
+     */
     public void testReceive()
         throws Exception
     {
@@ -82,7 +87,8 @@
         CompositeCacheManagerMockImpl cacheMgr = new CompositeCacheManagerMockImpl();
         System.out.println( "mock cache = " + cacheMgr.getCache( "test" ) );
 
-        LateralTCPCacheFactory fact = new LateralTCPCacheFactory();
+        // force initialization
+        //LateralTCPCacheFactory fact = new LateralTCPCacheFactory();
         //.getInstance( lattr, cacheMgr );
         //LateralCacheNoWait nwait1 = (LateralCacheNoWait)lcMgr1.getCache(
         // "test" );
@@ -91,7 +97,8 @@
         //nowait1.update( );
 
         // start the listener
-        LateralTCPListener listener = (LateralTCPListener) LateralTCPListener.getInstance( lattr, cacheMgr );
+        //LateralTCPListener listener = (LateralTCPListener) 
+        LateralTCPListener.getInstance( lattr, cacheMgr );
 
         TCPLateralCacheAttributes lattr2 = new TCPLateralCacheAttributes();
         lattr2.setTcpListenerPort( 1102 );
@@ -130,7 +137,8 @@
 
         // get the listener started
         // give it our mock cache manager
-        LateralTCPListener listener = (LateralTCPListener) LateralTCPListener.getInstance( lattr, cacheMgr );
+        //LateralTCPListener listener = (LateralTCPListener) 
+        LateralTCPListener.getInstance( lattr, cacheMgr );
 
         TCPLateralCacheAttributes lattr2 = new TCPLateralCacheAttributes();
         lattr2.setTcpListenerPort( 1102 );
@@ -171,7 +179,8 @@
 
         // get the listener started
         // give it our mock cache manager
-        LateralTCPListener listener = (LateralTCPListener) LateralTCPListener.getInstance( lattr, cacheMgr );
+        //LateralTCPListener listener = (LateralTCPListener) 
+        LateralTCPListener.getInstance( lattr, cacheMgr );
 
         TCPLateralCacheAttributes lattr2 = new TCPLateralCacheAttributes();
         lattr2.setTcpListenerPort( 1102 );

Added: jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/RemoteCacheListenerUnitTest.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/RemoteCacheListenerUnitTest.java?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/RemoteCacheListenerUnitTest.java (added)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/RemoteCacheListenerUnitTest.java Mon Feb 13 11:29:17 2006
@@ -0,0 +1,95 @@
+package org.apache.jcs.auxiliary.remote;
+
+import junit.framework.TestCase;
+
+import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheAttributes;
+import org.apache.jcs.engine.CacheElementSerialized;
+import org.apache.jcs.engine.ElementAttributes;
+import org.apache.jcs.engine.behavior.ICacheElement;
+import org.apache.jcs.engine.behavior.ICacheElementSerialized;
+import org.apache.jcs.engine.behavior.ICompositeCacheManager;
+import org.apache.jcs.engine.behavior.IElementAttributes;
+import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.control.CompositeCacheManagerMockImpl;
+import org.apache.jcs.utils.serialization.StandardSerializer;
+
+/**
+ * Tests for the remote cache listener.
+ * 
+ * @author Aaron Smuts
+ * 
+ */
+public class RemoteCacheListenerUnitTest
+    extends TestCase
+{
+
+    /**
+     * Create a RemoteCacheListener with a mock cache manager.  Set remove on put to false.
+     * Create a serialized element.  Call put on the listener.
+     * Verify that the deserialized element is in the cache.  
+     * 
+     * @throws Exception 
+     */
+    public void testUpdate()
+        throws Exception
+    {
+        IRemoteCacheAttributes irca = new RemoteCacheAttributes();
+        irca.setRemoveUponRemotePut( false );
+        ICompositeCacheManager cacheMgr = new CompositeCacheManagerMockImpl();
+        RemoteCacheListener listener = new RemoteCacheListener( irca, cacheMgr );
+
+        String cacheName = "testName";
+        String key = "key";
+        String value = "value fdsadf dsafdsa fdsaf dsafdsaf dsafdsaf dsaf dsaf dsaf dsafa dsaf dsaf dsafdsaf";
+        IElementAttributes attr = new ElementAttributes();
+        attr.setMaxLifeSeconds( 34 );
+
+        IElementSerializer elementSerializer = new StandardSerializer();
+
+        ICacheElementSerialized element = new CacheElementSerialized( cacheName, key, elementSerializer
+            .serialize( value ), attr );
+        listener.handlePut( element );
+        
+        ICacheElement after = cacheMgr.getCache( cacheName ).get( key );
+        
+        assertNotNull( "Should have a deserialized object.", after );
+        assertEquals( "Values should be the same.", value, after.getVal() );
+        assertEquals( "Attributes should be the same.", attr.getMaxLifeSeconds(), after
+            .getElementAttributes().getMaxLifeSeconds() );
+        assertEquals( "Keys should be the same.", key, after.getKey() );
+        assertEquals( "Cache name should be the same.", cacheName, after.getCacheName() );                        
+    }
+
+    /**
+     * Create a RemoteCacheListener with a mock cache manager.  Set remove on put to false.
+     * Create a serialized element.  Call put on the listener.
+     * Verify that the deserialized element is in the cache.  
+     * 
+     * @throws Exception 
+     */
+    public void testUpdate_RemoveOnPut()
+        throws Exception
+    {
+        IRemoteCacheAttributes irca = new RemoteCacheAttributes();
+        irca.setRemoveUponRemotePut( true );
+        ICompositeCacheManager cacheMgr = new CompositeCacheManagerMockImpl();
+        RemoteCacheListener listener = new RemoteCacheListener( irca, cacheMgr );
+
+        String cacheName = "testName";
+        String key = "key";
+        String value = "value fdsadf dsafdsa fdsaf dsafdsaf dsafdsaf dsaf dsaf dsaf dsafa dsaf dsaf dsafdsaf";
+        IElementAttributes attr = new ElementAttributes();
+        attr.setMaxLifeSeconds( 34 );
+
+        IElementSerializer elementSerializer = new StandardSerializer();
+
+        ICacheElementSerialized element = new CacheElementSerialized( cacheName, key, elementSerializer
+            .serialize( value ), attr );
+        listener.handlePut( element );
+        
+        ICacheElement after = cacheMgr.getCache( cacheName ).get( key );
+        
+        assertNull( "Should not have a deserialized object since remove on put is true.", after );
+    }
+
+}

Modified: jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/TestRemoteCache.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/TestRemoteCache.java?rev=377472&r1=377471&r2=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/TestRemoteCache.java (original)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/TestRemoteCache.java Mon Feb 13 11:29:17 2006
@@ -53,6 +53,7 @@
     }
     
     /**
+     * @throws Exception 
      * 
      *
      */
@@ -75,6 +76,9 @@
     }
     
     
+    /**
+     * @throws Exception
+     */
     public void testService() throws Exception 
     {
 

Added: jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/SerializationConversionUtilUnitTest.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/SerializationConversionUtilUnitTest.java?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/SerializationConversionUtilUnitTest.java (added)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/SerializationConversionUtilUnitTest.java Mon Feb 13 11:29:17 2006
@@ -0,0 +1,91 @@
+package org.apache.jcs.utils.serialization;
+
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+import org.apache.jcs.engine.CacheElement;
+import org.apache.jcs.engine.ElementAttributes;
+import org.apache.jcs.engine.behavior.ICacheElement;
+import org.apache.jcs.engine.behavior.ICacheElementSerialized;
+import org.apache.jcs.engine.behavior.IElementAttributes;
+import org.apache.jcs.engine.behavior.IElementSerializer;
+
+/**
+ * Tests the serialization conversion util.
+ * 
+ * @author Aaron Smuts
+ * 
+ */
+public class SerializationConversionUtilUnitTest
+    extends TestCase
+{
+
+    /**
+     * Verify that we can go back and forth with the simplest of objects.
+     * 
+     * @throws Exception
+     */
+    public void testSimpleConversion()
+        throws Exception
+    {
+        String cacheName = "testName";
+        String key = "key";
+        String value = "value fdsadf dsafdsa fdsaf dsafdsaf dsafdsaf dsaf dsaf dsaf dsafa dsaf dsaf dsafdsaf";
+
+        IElementSerializer elementSerializer = new StandardSerializer();
+
+        IElementAttributes attr = new ElementAttributes();
+        attr.setMaxLifeSeconds( 34 );
+
+        ICacheElement before = new CacheElement( cacheName, key, value );
+        before.setElementAttributes( attr );
+
+        ICacheElementSerialized serialized = SerializationConversionUtil.getSerializedCacheElement( before,
+                                                                                                    elementSerializer );
+        assertNotNull( "Should have a serialized object.", serialized );
+        System.out.println( "testSimpleConversion, " + serialized );
+
+        ICacheElement after = SerializationConversionUtil.getDeSerializedCacheElement( serialized, elementSerializer );
+
+        assertNotNull( "Should have a deserialized object.", after );
+        assertEquals( "Values should be the same.", before.getVal(), after.getVal() );
+        assertEquals( "Attributes should be the same.", before.getElementAttributes().getMaxLifeSeconds(), after
+            .getElementAttributes().getMaxLifeSeconds() );
+        assertEquals( "Keys should be the same.", before.getKey(), after.getKey() );
+        assertEquals( "Cache name should be the same.", before.getCacheName(), after.getCacheName() );
+
+    }
+
+    /**
+     * Verify that we get an IOException for a null serializer.
+     * 
+     * @throws Exception
+     */
+    public void testNullSerializerConversion()
+    {
+        String cacheName = "testName";
+        String key = "key";
+        String value = "value fdsadf dsafdsa fdsaf dsafdsaf dsafdsaf dsaf dsaf dsaf dsafa dsaf dsaf dsafdsaf";
+
+        IElementSerializer elementSerializer = null;// new StandardSerializer();
+
+        IElementAttributes attr = new ElementAttributes();
+        attr.setMaxLifeSeconds( 34 );
+
+        ICacheElement before = new CacheElement( cacheName, key, value );
+        before.setElementAttributes( attr );
+
+        try
+        {
+            SerializationConversionUtil.getSerializedCacheElement( before, elementSerializer );
+            fail( "We should have received an IOException." );
+        }
+        catch ( IOException e )
+        {
+            // expected
+        }
+
+    }
+
+}

Added: jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/StandardSerializerUnitTest.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/StandardSerializerUnitTest.java?rev=377472&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/StandardSerializerUnitTest.java (added)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/utils/serialization/StandardSerializerUnitTest.java Mon Feb 13 11:29:17 2006
@@ -0,0 +1,57 @@
+package org.apache.jcs.utils.serialization;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests the standard serializer.
+ * 
+ * @author Aaron Smuts
+ * 
+ */
+public class StandardSerializerUnitTest
+    extends TestCase
+{
+
+    /**
+     * Test simple back and forth with a string.
+     * 
+     * @throws Exception
+     * 
+     */
+    public void testSimpleBackAndForth()
+        throws Exception
+    {
+        StandardSerializer serializer = new StandardSerializer();
+
+        String before = "adsfdsafdsafdsafdsafdsafdsafdsagfdsafdsafdsfdsafdsafsa333 31231";
+
+        String after = (String) serializer.deSerialize( serializer.serialize( before ) );
+
+        assertEquals( "Before and after should be the same.", before, after );
+    }
+
+    /**
+     * Test serialization with a null object.  Verify that we don't get an error.
+     * 
+     * @throws Exception
+     * 
+     */
+    public void testNullInput()
+        throws Exception
+    {
+        StandardSerializer serializer = new StandardSerializer();
+
+        String before = null;
+
+        byte[] serialized = serializer.serialize( before );
+
+        System.out.println( "testNullInput " + serialized );
+        
+        String after = (String) serializer.deSerialize( serialized );
+
+        System.out.println( "testNullInput " + after );
+
+        assertNull( "Should have nothing.", after );
+
+    }
+}

Copied: jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/SortedPrefArrayUnitTest.java (from r295072, jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/TestSortedPrefArray.java)
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/SortedPrefArrayUnitTest.java?p2=jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/SortedPrefArrayUnitTest.java&p1=jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/TestSortedPrefArray.java&r1=295072&r2=377472&rev=377472&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/TestSortedPrefArray.java (original)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/utils/struct/SortedPrefArrayUnitTest.java Mon Feb 13 11:29:17 2006
@@ -24,7 +24,7 @@
  * @author aaronsm
  *  
  */
-public class TestSortedPrefArray
+public class SortedPrefArrayUnitTest
     extends TestCase
 {
 
@@ -34,7 +34,7 @@
      * @param testName
      *            Description of the Parameter
      */
-    public TestSortedPrefArray( String testName )
+    public SortedPrefArrayUnitTest( String testName )
     {
         super( testName );
     }
@@ -47,7 +47,7 @@
      */
     public static void main( String args[] )
     {
-        String[] testCaseName = { TestSortedPrefArray.class.getName() };
+        String[] testCaseName = { SortedPrefArrayUnitTest.class.getName() };
         junit.textui.TestRunner.main( testCaseName );
     }
 



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