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 2008/12/13 00:37:23 UTC

svn commit: r726149 [2/3] - in /jakarta/jcs/trunk: ./ src/conf/ src/java/org/apache/jcs/auxiliary/remote/ src/java/org/apache/jcs/auxiliary/remote/behavior/ src/java/org/apache/jcs/auxiliary/remote/http/behavior/ src/java/org/apache/jcs/auxiliary/remot...

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheListener.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheListener.java?rev=726149&r1=726148&r2=726149&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 Fri Dec 12 15:37:22 2008
@@ -21,7 +21,7 @@
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.net.UnknownHostException;
+import java.rmi.Remote;
 import java.rmi.RemoteException;
 import java.rmi.server.UnicastRemoteObject;
 
@@ -29,16 +29,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheAttributes;
 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.net.HostNameUtil;
-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
@@ -48,7 +39,8 @@
  * talk to a particular server.
  */
 public class RemoteCacheListener
-    implements IRemoteCacheListener, IRemoteCacheConstants, Serializable
+    extends AbsractRemoteCacheListener
+    implements IRemoteCacheConstants, Serializable, Remote
 {
     /** Don't change */
     private static final long serialVersionUID = 1L;
@@ -56,33 +48,6 @@
     /** The logger */
     private final static Log log = LogFactory.getLog( RemoteCacheListener.class );
 
-    /** The cached name of the local host.  The remote server gets this for logging purposes. */
-    private static String localHostName = null;
-
-    /** Has this client been shutdown. */
-    boolean disposed = false;
-
-    /**
-     * The cache manager used to put items in different regions. This is set lazily and should not
-     * be sent to the remote server.
-     */
-    protected transient ICompositeCacheManager cacheMgr;
-
-    /** The remote cache configuration object. */
-    protected IRemoteCacheAttributes irca;
-
-    /** Number of put requests received. For debugging only. */
-    protected int puts = 0;
-
-    /** Number of remove requests received. For debugging only. */
-    protected int removes = 0;
-
-    /** This is set by the remote cache server. */
-    protected long listenerId = 0;
-
-    /** Custom serializer.  Standard by default. */
-    private transient IElementSerializer elementSerializer = new StandardSerializer();
-
     /**
      * Only need one since it does work for all regions, just reference by multiple region names.
      * <p>
@@ -94,9 +59,7 @@
      */
     public RemoteCacheListener( IRemoteCacheAttributes irca, ICompositeCacheManager cacheMgr )
     {
-        this.irca = irca;
-
-        this.cacheMgr = cacheMgr;
+        super( irca, cacheMgr );
 
         // Export this remote object to make it available to receive incoming
         // calls, using an anonymous port unless the local port is specified.
@@ -144,249 +107,4 @@
             disposed = true;
         }
     }
-
-    /**
-     * 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.
-     * <p>
-     * @param id The new listenerId value
-     * @throws IOException
-     */
-    public void setListenerId( long id )
-        throws IOException
-    {
-        listenerId = id;
-        if ( log.isDebugEnabled() )
-        {
-            log.debug( "set listenerId = [" + id + "]" );
-        }
-    }
-
-    /**
-     * Gets the listenerId attribute of the RemoteCacheListener object. This is stored in the
-     * object. The RemoteCache object contains a reference to the listener and get the id this way.
-     * <p>
-     * @return The listenerId value
-     * @throws IOException
-     */
-    public long getListenerId()
-        throws IOException
-    {
-        if ( log.isDebugEnabled() )
-        {
-            log.debug( "get listenerId = [" + listenerId + "]" );
-        }
-        return listenerId;
-
-    }
-
-    /**
-     * Gets the remoteType attribute of the RemoteCacheListener object <p.
-     * @return The remoteType value
-     * @throws IOException
-     */
-    public int getRemoteType()
-        throws IOException
-    {
-        if ( log.isDebugEnabled() )
-        {
-            log.debug( "getRemoteType = [" + irca.getRemoteType() + "]" );
-        }
-        return irca.getRemoteType();
-    }
-
-    /**
-     * 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 configured, then update the item.
-     * @param cb
-     * @throws IOException
-     */
-    public void handlePut( ICacheElement cb )
-        throws IOException
-    {
-        if ( irca.getRemoveUponRemotePut() )
-        {
-            if ( log.isDebugEnabled() )
-            {
-                log.debug( "PUTTING ELEMENT FROM REMOTE, (  invalidating ) " );
-            }
-            handleRemove( cb.getCacheName(), cb.getKey() );
-        }
-        else
-        {
-            puts++;
-            if ( log.isDebugEnabled() )
-            {
-                log.debug( "PUTTING ELEMENT FROM REMOTE, ( updating ) " );
-                log.debug( "cb = " + cb );
-
-                if ( puts % 100 == 0 )
-                {
-                    log.debug( "puts = " + puts );
-                }
-            }
-
-            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 );
-        }
-
-        return;
-    }
-
-    /**
-     * Calls localRemove on the CompositeCache.
-     * <p>
-     * @param cacheName
-     * @param key
-     * @throws IOException
-     */
-    public void handleRemove( String cacheName, Serializable key )
-        throws IOException
-    {
-        removes++;
-        if ( log.isDebugEnabled() )
-        {
-            if ( removes % 100 == 0 )
-            {
-                log.debug( "removes = " + removes );
-            }
-
-            log.debug( "handleRemove> cacheName=" + cacheName + ", key=" + key );
-        }
-
-        ensureCacheManager();
-        CompositeCache cache = cacheMgr.getCache( cacheName );
-
-        cache.localRemove( key );
-    }
-
-    /**
-     * Calls localRemoveAll on the CompositeCache.
-     * <p>
-     * @param cacheName
-     * @throws IOException
-     */
-    public void handleRemoveAll( String cacheName )
-        throws IOException
-    {
-        if ( log.isDebugEnabled() )
-        {
-            log.debug( "handleRemoveAll> cacheName=" + cacheName );
-        }
-        ensureCacheManager();
-        CompositeCache cache = cacheMgr.getCache( cacheName );
-        cache.localRemoveAll();
-    }
-
-    /**
-     * @param cacheName
-     * @throws IOException
-     */
-    public void handleDispose( String cacheName )
-        throws IOException
-    {
-        if ( log.isDebugEnabled() )
-        {
-            log.debug( "handleDispose> cacheName=" + cacheName );
-        }
-        // 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 );
-    }
-
-    /**
-     * 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 ensureCacheManager()
-    {
-        if ( cacheMgr == null )
-        {
-            cacheMgr = CompositeCacheManager.getInstance();
-            log.debug( "had to get cacheMgr" );
-            if ( log.isDebugEnabled() )
-            {
-                log.debug( "cacheMgr = " + cacheMgr );
-            }
-        }
-        else
-        {
-            if ( log.isDebugEnabled() )
-            {
-                log.debug( "already got cacheMgr = " + cacheMgr );
-            }
-        }
-    }
-
-    /**
-     * This is for debugging. It allows the remote server to log the address of clients.
-     * <p>
-     * @return String
-     * @throws IOException
-     */
-    public synchronized String getLocalHostAddress()
-        throws IOException
-    {
-        if ( localHostName == null )
-        {
-            try
-            {
-                localHostName = HostNameUtil.getLocalHostAddress();
-            }
-            catch ( UnknownHostException uhe )
-            {
-                localHostName = "unknown";
-            }
-        }
-        return localHostName;
-    }
-
-    /**
-     * For easier debugging.
-     * <p>
-     * @return Basic info on this listener.
-     */
-    public String toString()
-    {
-        StringBuffer buf = new StringBuffer();
-        buf.append( "\n RemoteCacheListener: " );
-        buf.append( "\n RemoteHost = " + irca.getRemoteHost() );
-        buf.append( "\n RemotePort = " + irca.getRemotePort() );
-        buf.append( "\n ListenerId = " + listenerId );
-        return buf.toString();
-    }
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWaitFacade.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWaitFacade.java?rev=726149&r1=726148&r2=726149&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWaitFacade.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWaitFacade.java Fri Dec 12 15:37:22 2008
@@ -19,32 +19,12 @@
  * under the License.
  */
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.jcs.auxiliary.AbstractAuxiliaryCache;
-import org.apache.jcs.auxiliary.AuxiliaryCache;
-import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.engine.CacheConstants;
-import org.apache.jcs.engine.behavior.ICacheElement;
-import org.apache.jcs.engine.behavior.ICacheType;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
 import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
-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;
 
 /**
  * Used to provide access to multiple services under nowait protection. Factory should construct
@@ -55,46 +35,14 @@
  * RemoteCacheNoWait.
  */
 public class RemoteCacheNoWaitFacade
-    extends AbstractAuxiliaryCache
+    extends AbstractRemoteCacheNoWaitFacade
 {
     /** For serialization. Don't change. */
-    private static final long serialVersionUID = -4529970797620747110L;
+    private static final long serialVersionUID = -4529970797620747111L;
 
     /** log instance */
     private final static Log log = LogFactory.getLog( RemoteCacheNoWaitFacade.class );
 
-    /** The connection to a remote server, or a zombie. */
-    public RemoteCacheNoWait[] noWaits;
-
-    /** The cache name */
-    private String cacheName;
-
-    /** holds failover and cluster information */
-    protected RemoteCacheAttributes remoteCacheAttributes;
-
-    /** A cache manager */
-    private ICompositeCacheManager cacheMgr;
-
-    /**
-     * Gets the remoteCacheAttributes attribute of the RemoteCacheNoWaitFacade object
-     * <p>
-     * @return The remoteCacheAttributes value
-     */
-    public RemoteCacheAttributes getRemoteCacheAttributes()
-    {
-        return remoteCacheAttributes;
-    }
-
-    /**
-     * Sets the remoteCacheAttributes attribute of the RemoteCacheNoWaitFacade object.
-     * <p>
-     * @param rca The new remoteCacheAttributes value
-     */
-    public void setRemoteCacheAttributes( RemoteCacheAttributes rca )
-    {
-        this.remoteCacheAttributes = rca;
-    }
-
     /**
      * Constructs with the given remote cache, and fires events to any listeners.
      * <p>
@@ -108,284 +56,7 @@
                                     ICompositeCacheManager cacheMgr, ICacheEventLogger cacheEventLogger,
                                     IElementSerializer elementSerializer )
     {
-        if ( log.isDebugEnabled() )
-        {
-            log.debug( "CONSTRUCTING NO WAIT FACADE" );
-        }
-        this.noWaits = noWaits;
-        this.remoteCacheAttributes = rca;
-        this.cacheName = rca.getCacheName();
-        this.cacheMgr = cacheMgr;
-        this.cacheEventLogger = cacheEventLogger;
-        this.elementSerializer = elementSerializer;
-    }
-
-    /**
-     * Put an element in the cache.
-     * <p>
-     * @param ce
-     * @throws IOException
-     */
-    public void update( ICacheElement ce )
-        throws IOException
-    {
-        if ( log.isDebugEnabled() )
-        {
-            log.debug( "updating through cache facade, noWaits.length = " + noWaits.length );
-        }
-        int i = 0;
-        try
-        {
-            for ( ; i < noWaits.length; i++ )
-            {
-                noWaits[i].update( ce );
-                // an initial move into a zombie will lock this to primary
-                // recovery. will not discover other servers until primary
-                // reconnect
-                // and subsequent error
-            }
-        }
-        catch ( Exception ex )
-        {
-            String message = "Problem updating no wait.  Will initiate failover if the noWait is in error.";
-            log.error( message, ex );
-
-            if ( getCacheEventLogger() != null )
-            {
-                getCacheEventLogger().logError(
-                                                "RemoteCacheNoWaitFacade",
-                                                ICacheEventLogger.UPDATE_EVENT,
-                                                message + ":" + ex.getMessage() + " REGION: " + ce.getCacheName()
-                                                    + " ELEMENT: " + ce );
-            }
-
-            // can handle failover here? Is it safe to try the others?
-            // check to see it the noWait is now a zombie
-            // if it is a zombie, then move to the next in the failover list
-            // will need to keep them in order or a count
-            failover( i );
-            // should start a failover thread
-            // should probably only failover if there is only one in the noWait
-            // list
-            // Should start a background thread to restore the original primary if we are in failover state.
-        }
-    }
-
-    /**
-     * Synchronously reads from the remote cache.
-     * <p>
-     * @param key
-     * @return Either an ICacheElement or null if it is not found.
-     */
-    public ICacheElement get( Serializable key )
-    {
-        for ( int i = 0; i < noWaits.length; i++ )
-        {
-            try
-            {
-                Object obj = noWaits[i].get( key );
-                if ( obj != null )
-                {
-                    return (ICacheElement) obj;
-                }
-            }
-            catch ( Exception ex )
-            {
-                log.debug( "Failed to get." );
-                return null;
-            }
-        }
-        return null;
-    }
-
-    /**
-     * Synchronously read from the remote cache.
-     * <p>
-     * @param pattern
-     * @return map
-     * @throws IOException
-     */
-    public Map getMatching( String pattern )
-        throws IOException
-    {
-        for ( int i = 0; i < noWaits.length; i++ )
-        {
-            try
-            {
-                return noWaits[i].getMatching( pattern );
-            }
-            catch ( Exception ex )
-            {
-                log.debug( "Failed to getMatching." );
-                return Collections.EMPTY_MAP;
-            }
-        }
-        return Collections.EMPTY_MAP;
-    }
-
-    /**
-     * Gets multiple items from the cache based on the given set of keys.
-     * <p>
-     * @param keys
-     * @return a map of Serializable key to ICacheElement element, or an empty map if there is no
-     *         data in cache for any of these keys
-     */
-    public Map getMultiple( Set keys )
-    {
-        if ( keys != null && !keys.isEmpty() )
-        {
-            for ( int i = 0; i < noWaits.length; i++ )
-            {
-                try
-                {
-                    return noWaits[i].getMultiple( keys );
-                }
-                catch ( Exception ex )
-                {
-                    log.debug( "Failed to get." );
-                    return Collections.EMPTY_MAP;
-                }
-            }
-        }
-        return new HashMap();
-    }
-
-    /**
-     * Gets the set of keys of objects currently in the group.
-     * <p>
-     * @param group
-     * @return the set of keys of objects currently in the group
-     * @throws IOException
-     */
-    public Set getGroupKeys( String group )
-        throws IOException
-    {
-        HashSet allKeys = new HashSet();
-        for ( int i = 0; i < noWaits.length; i++ )
-        {
-            AuxiliaryCache aux = noWaits[i];
-            if ( aux != null )
-            {
-                allKeys.addAll( aux.getGroupKeys( group ) );
-            }
-        }
-        return allKeys;
-    }
-
-    /**
-     * Adds a remove request to the remote cache.
-     * <p>
-     * @param key
-     * @return whether or not it was removed, right now it return false.
-     */
-    public boolean remove( Serializable key )
-    {
-        try
-        {
-            for ( int i = 0; i < noWaits.length; i++ )
-            {
-                noWaits[i].remove( key );
-            }
-        }
-        catch ( Exception ex )
-        {
-            log.error( ex );
-        }
-        return false;
-    }
-
-    /**
-     * Adds a removeAll request to the remote cache.
-     */
-    public void removeAll()
-    {
-        try
-        {
-            for ( int i = 0; i < noWaits.length; i++ )
-            {
-                noWaits[i].removeAll();
-            }
-        }
-        catch ( Exception ex )
-        {
-            log.error( ex );
-        }
-    }
-
-    /** Adds a dispose request to the remote cache. */
-    public void dispose()
-    {
-        try
-        {
-            for ( int i = 0; i < noWaits.length; i++ )
-            {
-                noWaits[i].dispose();
-            }
-        }
-        catch ( Exception ex )
-        {
-            log.error( "Problem in dispose.", ex );
-        }
-    }
-
-    /**
-     * No remote invocation.
-     * <p>
-     * @return The size value
-     */
-    public int getSize()
-    {
-        return 0;
-        // cache.getSize();
-    }
-
-    /**
-     * Gets the cacheType attribute of the RemoteCacheNoWaitFacade object.
-     * <p>
-     * @return The cacheType value
-     */
-    public int getCacheType()
-    {
-        return ICacheType.REMOTE_CACHE;
-    }
-
-    /**
-     * Gets the cacheName attribute of the RemoteCacheNoWaitFacade object.
-     * <p>
-     * @return The cacheName value
-     */
-    public String getCacheName()
-    {
-        return remoteCacheAttributes.getCacheName();
-    }
-
-    /**
-     * Gets the status attribute of the RemoteCacheNoWaitFacade object
-     * <p>
-     * Return ALIVE if any are alive.
-     * <p>
-     * @return The status value
-     */
-    public int getStatus()
-    {
-        for ( int i = 0; i < noWaits.length; i++ )
-        {
-            if ( noWaits[i].getStatus() == CacheConstants.STATUS_ALIVE )
-            {
-                return CacheConstants.STATUS_ALIVE;
-            }
-        }
-        return 0;
-    }
-
-    /**
-     * String form of some of the configuration information for the remote cache.
-     * <p>
-     * @return Some info for logging.
-     */
-    public String toString()
-    {
-        return "RemoteCacheNoWaitFacade: " + cacheName + ", rca = " + remoteCacheAttributes;
+        super( noWaits, rca, cacheMgr, cacheEventLogger, elementSerializer );
     }
 
     /**
@@ -405,8 +76,8 @@
             if ( noWaits[i].getStatus() == CacheConstants.STATUS_ERROR )
             {
                 // start failover, primary recovery process
-                RemoteCacheFailoverRunner runner = new RemoteCacheFailoverRunner( this, cacheMgr, cacheEventLogger,
-                                                                                  elementSerializer );
+                RemoteCacheFailoverRunner runner = new RemoteCacheFailoverRunner( this, getCompositeCacheManager(),
+                                                                                  cacheEventLogger, elementSerializer );
                 // If the returned monitor is null, it means it's already
                 // started elsewhere.
                 if ( runner != null )
@@ -433,67 +104,4 @@
         }
     }
 
-    /**
-     * @return Returns the AuxiliaryCacheAttributes.
-     */
-    public AuxiliaryCacheAttributes getAuxiliaryCacheAttributes()
-    {
-        return this.remoteCacheAttributes;
-    }
-
-    /**
-     * getStats
-     * @return String
-     */
-    public String getStats()
-    {
-        return getStatistics().toString();
-    }
-
-    /**
-     * @return statistics about the cache region
-     */
-    public IStats getStatistics()
-    {
-        IStats stats = new Stats();
-        stats.setTypeName( "Remote Cache No Wait Facade" );
-
-        ArrayList elems = new ArrayList();
-
-        IStatElement se = null;
-
-        if ( noWaits != null )
-        {
-            se = new StatElement();
-            se.setName( "Number of No Waits" );
-            se.setData( "" + noWaits.length );
-            elems.add( se );
-
-            for ( int i = 0; i < noWaits.length; i++ )
-            {
-                // get the stats from the super too
-                // get as array, convert to list, add list to our outer list
-                IStats sStats = noWaits[i].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;
-    }
-
-    /**
-     * This typically returns end point info .
-     * <p>
-     * @return the name
-     */
-    public String getEventLoggingExtraInfo()
-    {
-        return "Remote Cache No Wait Facade";
-    }
 }

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheDispatcher.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheDispatcher.java?rev=726149&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheDispatcher.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheDispatcher.java Fri Dec 12 15:37:22 2008
@@ -0,0 +1,26 @@
+package org.apache.jcs.auxiliary.remote.behavior;
+
+import java.io.IOException;
+
+import org.apache.jcs.auxiliary.remote.value.RemoteCacheRequest;
+import org.apache.jcs.auxiliary.remote.value.RemoteCacheResponse;
+
+/**
+ * In the future, this can be used as a generic dispatcher abstraction.
+ * <p>
+ * At the time of creation, only the http remote cache uses it. The RMI remote could be converted to
+ * use it as well.
+ */
+public interface IRemoteCacheDispatcher
+{
+    /**
+     * All requests will go through this method. The dispatcher implementation will send the request
+     * remotely.
+     * <p>
+     * @param remoteCacheRequest
+     * @return RemoteCacheResponse
+     * @throws IOException
+     */
+    RemoteCacheResponse dispatchRequest( RemoteCacheRequest remoteCacheRequest )
+        throws IOException;
+}

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheListener.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheListener.java?rev=726149&r1=726148&r2=726149&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheListener.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheListener.java Fri Dec 12 15:37:22 2008
@@ -21,20 +21,18 @@
 
 import java.io.IOException;
 
-import java.rmi.Remote;
-
 import org.apache.jcs.engine.behavior.ICacheListener;
 
 /**
  * Listens for remote cache event notification ( rmi callback ).
  */
 public interface IRemoteCacheListener
-    extends ICacheListener, Remote
+    extends ICacheListener
 {
     /** SERVER_LISTENER -- for the cluster */
     public final static int SERVER_LISTENER = 0;
 
-    /** CLIENT_LISTENER -- these aren't used any longer.  remove*/
+    /** CLIENT_LISTENER -- these aren't used any longer. remove */
     public final static int CLIENT_LISTENER = 1;
 
     /**
@@ -67,7 +65,7 @@
 
     /**
      * This is for debugging. It allows the remote cache server to log the address of any listeners
-     * that regiser.
+     * that register.
      * <p>
      * @return the local host address.
      * @throws IOException
@@ -82,5 +80,4 @@
      */
     public void dispose()
         throws IOException;
-
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheService.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheService.java?rev=726149&r1=726148&r2=726149&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheService.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheService.java Fri Dec 12 15:37:22 2008
@@ -112,7 +112,8 @@
      * @param groupName
      * @return A Set of keys
      * @throws java.rmi.RemoteException
+     * @throws IOException 
      */
     Set getGroupKeys( String cacheName, String groupName )
-        throws java.rmi.RemoteException;
+        throws java.rmi.RemoteException, IOException;
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/behavior/IRemoteHttpCacheConstants.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/behavior/IRemoteHttpCacheConstants.java?rev=726149&r1=726148&r2=726149&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/behavior/IRemoteHttpCacheConstants.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/behavior/IRemoteHttpCacheConstants.java Fri Dec 12 15:37:22 2008
@@ -3,24 +3,6 @@
 /** Constants used throughout the HTTP remote cache. */
 public interface IRemoteHttpCacheConstants
 {
-    /** Get request type. */
-    public static final byte REQUEST_TYPE_GET = 0;
-
-    /** Get Multiple request type. */
-    public static final byte REQUEST_TYPE_GET_MULTIPLE = 1;
-
-    /** Get Matching request type. */
-    public static final byte REQUEST_TYPE_GET_MATCHING = 2;
-
-    /** Update request type. */
-    public static final byte REQUEST_TYPE_UPDATE = 3;
-
-    /** Remove request type. */
-    public static final byte REQUEST_TYPE_REMOVE = 4;
-
-    /** Remove All request type. */
-    public static final byte REQUEST_TYPE_REMOVE_ALL = 5;
-
     /** The prefix for cache server config. */
     public final static String HTTP_CACHE_SERVER_PREFIX = "jcs.remotehttpcache";
 

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/AbstractHttpClient.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/AbstractHttpClient.java?rev=726149&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/AbstractHttpClient.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/AbstractHttpClient.java Fri Dec 12 15:37:22 2008
@@ -0,0 +1,139 @@
+package org.apache.jcs.auxiliary.remote.http.client;
+
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.HttpVersion;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.commons.httpclient.cookie.CookiePolicy;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * This class simply configures the http multithreaded connection manager.
+ * <p>
+ * This class does common functions required by basic WSRequestDispatchers such as loading the
+ * properties files.
+ * <p>
+ * This is abstract because it can't do anything. It used to require the reload properties method to
+ * be implemented. I was able to get a default implementation of that method here as well. Child
+ * classes can overwrite whatever they want.
+ */
+public abstract class AbstractHttpClient
+{
+    /** The connection manager. */
+    private MultiThreadedHttpConnectionManager connectionManager;
+
+    /** The client */
+    private HttpClient httpClient;
+
+    /** Configuration settings. */
+    private RemoteHttpCacheAttributes remoteHttpCacheAttributes;
+
+    /** The Logger. */
+    private final static Log log = LogFactory.getLog( AbstractHttpClient.class );
+
+    /**
+     * Sets the default Properties File and Heading, and creates the HttpClient and connection
+     * manager.
+     * <p>
+     * @param remoteHttpCacheAttributes
+     */
+    public AbstractHttpClient( RemoteHttpCacheAttributes remoteHttpCacheAttributes )
+    {
+        setRemoteHttpCacheAttributes( remoteHttpCacheAttributes );
+        setConnectionManager( new MultiThreadedHttpConnectionManager() );
+
+        // THIS IS NOT THREAD SAFE:
+        // setHttpClient( new HttpClient() );
+        // THIS IS:
+        setHttpClient( new HttpClient( getConnectionManager() ) );
+
+        configureClient();
+    }
+
+    /**
+     * Configures the http client.
+     */
+    public void configureClient()
+    {
+        if ( getRemoteHttpCacheAttributes().getMaxConnectionsPerHost() > 0 )
+        {
+            getConnectionManager().getParams().setMaxTotalConnections(
+                                                                       getRemoteHttpCacheAttributes()
+                                                                           .getMaxConnectionsPerHost() );
+        }
+
+        getConnectionManager().getParams().setSoTimeout( getRemoteHttpCacheAttributes().getSocketTimeoutMillis() );
+
+        String httpVersion = getRemoteHttpCacheAttributes().getHttpVersion();
+        if ( httpVersion != null )
+        {
+            if ( "1.1".equals( httpVersion ) )
+            {
+                getHttpClient().getParams().setParameter( "http.protocol.version", HttpVersion.HTTP_1_1 );
+            }
+            else if ( "1.0".equals( httpVersion ) )
+            {
+                getHttpClient().getParams().setParameter( "http.protocol.version", HttpVersion.HTTP_1_0 );
+            }
+            else
+            {
+                log.warn( "Unrecognized value for 'httpVersion': [" + httpVersion + "]" );
+            }
+        }
+
+        getConnectionManager().getParams()
+            .setConnectionTimeout( getRemoteHttpCacheAttributes().getConnectionTimeoutMillis() );
+
+        // By default we instruct HttpClient to ignore cookies.
+        String cookiePolicy = CookiePolicy.IGNORE_COOKIES;
+        getHttpClient().getParams().setCookiePolicy( cookiePolicy );
+    }
+
+    /**
+     * @return Returns the httpClient.
+     */
+    public HttpClient getHttpClient()
+    {
+        return httpClient;
+    }
+
+    /**
+     * @param httpClient The httpClient to set.
+     */
+    public void setHttpClient( HttpClient httpClient )
+    {
+        this.httpClient = httpClient;
+    }
+
+    /**
+     * @return Returns the connectionManager.
+     */
+    public MultiThreadedHttpConnectionManager getConnectionManager()
+    {
+        return connectionManager;
+    }
+
+    /**
+     * @param connectionManager The connectionManager to set.
+     */
+    public void setConnectionManager( MultiThreadedHttpConnectionManager connectionManager )
+    {
+        this.connectionManager = connectionManager;
+    }
+
+    /**
+     * @param remoteHttpCacheAttributes the remoteHttpCacheAttributes to set
+     */
+    public void setRemoteHttpCacheAttributes( RemoteHttpCacheAttributes remoteHttpCacheAttributes )
+    {
+        this.remoteHttpCacheAttributes = remoteHttpCacheAttributes;
+    }
+
+    /**
+     * @return the remoteHttpCacheAttributes
+     */
+    public RemoteHttpCacheAttributes getRemoteHttpCacheAttributes()
+    {
+        return remoteHttpCacheAttributes;
+    }
+}

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCache.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCache.java?rev=726149&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCache.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCache.java Fri Dec 12 15:37:22 2008
@@ -0,0 +1,98 @@
+package org.apache.jcs.auxiliary.remote.http.client;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.jcs.auxiliary.remote.AbstractRemoteAuxiliaryCache;
+import org.apache.jcs.auxiliary.remote.ZombieRemoteCacheService;
+import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheListener;
+import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheService;
+
+/**
+ * This uses an http client as the service.
+ */
+public class RemoteHttpCache
+    extends AbstractRemoteAuxiliaryCache
+{
+    /** Don't change. */
+    private static final long serialVersionUID = -5329231850422826461L;
+
+    /** The logger. */
+    private final static Log log = LogFactory.getLog( RemoteHttpCache.class );
+
+    /** Keep the child copy here for the restore process. */
+    private RemoteHttpCacheAttributes remoteHttpCacheAttributes;
+
+    /**
+     * Constructor for the RemoteCache object. This object communicates with a remote cache server.
+     * One of these exists for each region. This also holds a reference to a listener. The same
+     * listener is used for all regions for one remote server. Holding a reference to the listener
+     * allows this object to know the listener id assigned by the remote cache.
+     * <p>
+     * @param remoteHttpCacheAttributes
+     * @param remote
+     * @param listener
+     */
+    public RemoteHttpCache( RemoteHttpCacheAttributes remoteHttpCacheAttributes, IRemoteCacheService remote,
+                            IRemoteCacheListener listener )
+    {
+        super( remoteHttpCacheAttributes, remote, listener );
+
+        setRemoteHttpCacheAttributes( remoteHttpCacheAttributes );
+    }
+
+    /**
+     * Nothing right now. This should setup a zombie and initiate recovery.
+     * <p>
+     * @param ex
+     * @param msg
+     * @param eventName
+     * @throws IOException
+     */
+    protected void handleException( Exception ex, String msg, String eventName )
+        throws IOException
+    {
+        // we should not switch if the existing is a zombie.
+        if ( !( getRemoteCacheService() instanceof ZombieRemoteCacheService ) )
+        {
+            String message = "Disabling remote cache due to error: " + msg;
+            logError( cacheName, "", message );
+            log.error( message, ex );
+
+            setRemoteCacheService( new ZombieRemoteCacheService( getRemoteCacheAttributes().getZombieQueueMaxSize() ) );
+
+            RemoteHttpCacheMonitor.getInstance().notifyError( this );
+        }
+
+        if ( ex instanceof IOException )
+        {
+            throw (IOException) ex;
+        }
+        throw new IOException( ex.getMessage() );
+    }
+
+    /**
+     * @return url of service
+     */
+    public String getEventLoggingExtraInfo()
+    {
+        return null;
+    }
+
+    /**
+     * @param remoteHttpCacheAttributes the remoteHttpCacheAttributes to set
+     */
+    public void setRemoteHttpCacheAttributes( RemoteHttpCacheAttributes remoteHttpCacheAttributes )
+    {
+        this.remoteHttpCacheAttributes = remoteHttpCacheAttributes;
+    }
+
+    /**
+     * @return the remoteHttpCacheAttributes
+     */
+    public RemoteHttpCacheAttributes getRemoteHttpCacheAttributes()
+    {
+        return remoteHttpCacheAttributes;
+    }
+}

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheAttributes.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheAttributes.java?rev=726149&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheAttributes.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheAttributes.java Fri Dec 12 15:37:22 2008
@@ -0,0 +1,185 @@
+package org.apache.jcs.auxiliary.remote.http.client;
+
+import org.apache.jcs.auxiliary.remote.RemoteCacheAttributes;
+
+/** Http client specific settings. */
+public class RemoteHttpCacheAttributes
+    extends RemoteCacheAttributes
+{
+    /** Don't change. */
+    private static final long serialVersionUID = -5944327125140505212L;
+
+    /** http verison to use. */
+    private static final String DEFAULT_HTTP_VERSION = "1.0";
+
+    /** The max connections allowed per host */
+    private int maxConnectionsPerHost = 100;
+
+    /** The socket timeout. */
+    private int socketTimeoutMillis = 3000;
+
+    /** The socket connections timeout */
+    private int connectionTimeoutMillis = 5000;
+
+    /** http verison to use. */
+    private String httpVersion = DEFAULT_HTTP_VERSION;
+
+    /** The cache name will be included on the parameters */
+    private boolean includeCacheNameAsParameter = true;
+
+    /** keys and patterns will be included in the parameters */
+    private boolean includeKeysAndPatternsAsParameter = true;
+
+    /** keys and patterns will be included in the parameters */
+    private boolean includeRequestTypeasAsParameter = true;
+    
+    /** The complete URL to the service. */
+    private String url;
+
+    /**
+     * @param maxConnectionsPerHost the maxConnectionsPerHost to set
+     */
+    public void setMaxConnectionsPerHost( int maxConnectionsPerHost )
+    {
+        this.maxConnectionsPerHost = maxConnectionsPerHost;
+    }
+
+    /**
+     * @return the maxConnectionsPerHost
+     */
+    public int getMaxConnectionsPerHost()
+    {
+        return maxConnectionsPerHost;
+    }
+
+    /**
+     * @param socketTimeoutMillis the socketTimeoutMillis to set
+     */
+    public void setSocketTimeoutMillis( int socketTimeoutMillis )
+    {
+        this.socketTimeoutMillis = socketTimeoutMillis;
+    }
+
+    /**
+     * @return the socketTimeoutMillis
+     */
+    public int getSocketTimeoutMillis()
+    {
+        return socketTimeoutMillis;
+    }
+
+    /**
+     * @param httpVersion the httpVersion to set
+     */
+    public void setHttpVersion( String httpVersion )
+    {
+        this.httpVersion = httpVersion;
+    }
+
+    /**
+     * @return the httpVersion
+     */
+    public String getHttpVersion()
+    {
+        return httpVersion;
+    }
+
+    /**
+     * @param connectionTimeoutMillis the connectionTimeoutMillis to set
+     */
+    public void setConnectionTimeoutMillis( int connectionTimeoutMillis )
+    {
+        this.connectionTimeoutMillis = connectionTimeoutMillis;
+    }
+
+    /**
+     * @return the connectionTimeoutMillis
+     */
+    public int getConnectionTimeoutMillis()
+    {
+        return connectionTimeoutMillis;
+    }
+
+    /**
+     * @param includeCacheNameInURL the includeCacheNameInURL to set
+     */
+    public void setIncludeCacheNameAsParameter( boolean includeCacheNameInURL )
+    {
+        this.includeCacheNameAsParameter = includeCacheNameInURL;
+    }
+
+    /**
+     * @return the includeCacheNameInURL
+     */
+    public boolean isIncludeCacheNameAsParameter()
+    {
+        return includeCacheNameAsParameter;
+    }
+
+    /**
+     * @param includeKeysAndPatternsInURL the includeKeysAndPatternsInURL to set
+     */
+    public void setIncludeKeysAndPatternsAsParameter( boolean includeKeysAndPatternsInURL )
+    {
+        this.includeKeysAndPatternsAsParameter = includeKeysAndPatternsInURL;
+    }
+
+    /**
+     * @return the includeKeysAndPatternsInURL
+     */
+    public boolean isIncludeKeysAndPatternsAsParameter()
+    {
+        return includeKeysAndPatternsAsParameter;
+    }
+
+    /**
+     * @param includeRequestTypeasAsParameter the includeRequestTypeasAsParameter to set
+     */
+    public void setIncludeRequestTypeasAsParameter( boolean includeRequestTypeasAsParameter )
+    {
+        this.includeRequestTypeasAsParameter = includeRequestTypeasAsParameter;
+    }
+
+    /**
+     * @return the includeRequestTypeasAsParameter
+     */
+    public boolean isIncludeRequestTypeasAsParameter()
+    {
+        return includeRequestTypeasAsParameter;
+    }
+
+    /**
+     * @param url the url to set
+     */
+    public void setUrl( String url )
+    {
+        this.url = url;
+    }
+
+    /**
+     * @return the url
+     */
+    public String getUrl()
+    {
+        return url;
+    }
+
+    /**
+     * @return String details
+     */
+    public String toString()
+    {
+        StringBuffer buf = new StringBuffer();
+        buf.append( "\n RemoteHttpCacheAttributes" );
+        buf.append( "\n maxConnectionsPerHost = [" + getMaxConnectionsPerHost() + "]" );
+        buf.append( "\n socketTimeoutMillis = [" + getSocketTimeoutMillis() + "]" );
+        buf.append( "\n httpVersion = [" + getHttpVersion() + "]" );
+        buf.append( "\n connectionTimeoutMillis = [" + getConnectionTimeoutMillis() + "]" );
+        buf.append( "\n includeCacheNameAsParameter = [" + isIncludeCacheNameAsParameter() + "]" );
+        buf.append( "\n includeKeysAndPatternsAsParameter = [" + isIncludeKeysAndPatternsAsParameter() + "]" );
+        buf.append( "\n includeRequestTypeasAsParameter = [" + isIncludeRequestTypeasAsParameter() + "]" );
+        buf.append( "\n url = [" + getUrl() + "]" );
+        buf.append( super.toString() );
+        return buf.toString();
+    }
+}

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheClient.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheClient.java?rev=726149&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheClient.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheClient.java Fri Dec 12 15:37:22 2008
@@ -0,0 +1,337 @@
+package org.apache.jcs.auxiliary.remote.http.client;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheDispatcher;
+import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheService;
+import org.apache.jcs.auxiliary.remote.http.server.RemoteCacheServiceAdaptor;
+import org.apache.jcs.auxiliary.remote.util.RemoteCacheRequestFactory;
+import org.apache.jcs.auxiliary.remote.value.RemoteCacheRequest;
+import org.apache.jcs.auxiliary.remote.value.RemoteCacheResponse;
+import org.apache.jcs.engine.behavior.ICacheElement;
+
+/** This is the service used by the remote http auxiliary cache. */
+public class RemoteHttpCacheClient
+    implements IRemoteCacheService
+{
+    /** The Logger. */
+    private final static Log log = LogFactory.getLog( RemoteCacheServiceAdaptor.class );
+
+    /** The internal client. */
+    private IRemoteCacheDispatcher remoteDispatcher;
+
+    /**
+     * Constructs a client.
+     * <p>
+     * @param attributes
+     */
+    public RemoteHttpCacheClient( RemoteHttpCacheAttributes attributes )
+    {
+        initialize( attributes );
+    }
+
+    /**
+     * The provides an extension point. If you want to extend this and use a special dispatcher,
+     * here is the place to do it.
+     * <p>
+     * @param attributes
+     */
+    protected void initialize( RemoteHttpCacheAttributes attributes )
+    {
+        setRemoteDispatcher( new RemoteHttpCacheDispatcher( attributes ) );
+
+        if ( log.isInfoEnabled() )
+        {
+            log.info( "Created remote Dispatcher." + getRemoteDispatcher() );
+        }
+    }
+
+    /**
+     * Create a request, process, extract the payload.
+     * <p>
+     * @param cacheName
+     * @param key
+     * @return ICacheElement
+     * @throws IOException
+     */
+    public ICacheElement get( String cacheName, Serializable key )
+        throws IOException
+    {
+        return get( cacheName, key, 0 );
+    }
+
+    /**
+     * Create a request, process, extract the payload.
+     * <p>
+     * @param cacheName
+     * @param key
+     * @param requesterId
+     * @return ICacheElement
+     * @throws IOException
+     */
+    public ICacheElement get( String cacheName, Serializable key, long requesterId )
+        throws IOException
+    {
+        RemoteCacheRequest remoteHttpCacheRequest = RemoteCacheRequestFactory.createGetRequest( cacheName, key,
+                                                                                                requesterId );
+
+        RemoteCacheResponse remoteHttpCacheResponse = getRemoteDispatcher().dispatchRequest( remoteHttpCacheRequest );
+
+        if ( log.isDebugEnabled() )
+        {
+            log.debug( "Get [" + key + "] = " + remoteHttpCacheResponse );
+        }
+        
+        ICacheElement retval = null;
+        if ( remoteHttpCacheResponse != null && remoteHttpCacheResponse.getPayload() != null )
+        {
+            retval = (ICacheElement) remoteHttpCacheResponse.getPayload().get( key );
+        }
+        return retval;
+    }
+
+    /**
+     * Gets multiple items from the cache matching the pattern.
+     * <p>
+     * @param cacheName
+     * @param pattern
+     * @return a map of Serializable key to ICacheElement element, or an empty map if there is no
+     *         data in cache matching the pattern.
+     * @throws IOException
+     */
+    public Map getMatching( String cacheName, String pattern )
+        throws IOException
+    {
+        return getMatching( cacheName, pattern, 0 );
+    }
+
+    /**
+     * Gets multiple items from the cache matching the pattern.
+     * <p>
+     * @param cacheName
+     * @param pattern
+     * @param requesterId
+     * @return a map of Serializable key to ICacheElement element, or an empty map if there is no
+     *         data in cache matching the pattern.
+     * @throws IOException
+     */
+    public Map getMatching( String cacheName, String pattern, long requesterId )
+        throws IOException
+    {
+        RemoteCacheRequest remoteHttpCacheRequest = RemoteCacheRequestFactory.createGetMatchingRequest( cacheName,
+                                                                                                        pattern,
+                                                                                                        requesterId );
+
+        RemoteCacheResponse remoteHttpCacheResponse = getRemoteDispatcher().dispatchRequest( remoteHttpCacheRequest );
+
+        return remoteHttpCacheResponse.getPayload();
+    }
+
+    /**
+     * Gets multiple items from the cache based on the given set of keys.
+     * <p>
+     * @param cacheName
+     * @param keys
+     * @return a map of Serializable key to ICacheElement element, or an empty map if there is no
+     *         data in cache for any of these keys
+     * @throws IOException
+     */
+    public Map getMultiple( String cacheName, Set keys )
+        throws IOException
+    {
+        return getMultiple( cacheName, keys, 0 );
+    }
+
+    /**
+     * Gets multiple items from the cache based on the given set of keys.
+     * <p>
+     * @param cacheName
+     * @param keys
+     * @param requesterId
+     * @return a map of Serializable key to ICacheElement element, or an empty map if there is no
+     *         data in cache for any of these keys
+     * @throws IOException
+     */
+    public Map getMultiple( String cacheName, Set keys, long requesterId )
+        throws IOException
+    {
+        RemoteCacheRequest remoteHttpCacheRequest = RemoteCacheRequestFactory.createGetMultipleRequest( cacheName,
+                                                                                                        keys,
+                                                                                                        requesterId );
+
+        RemoteCacheResponse remoteHttpCacheResponse = getRemoteDispatcher().dispatchRequest( remoteHttpCacheRequest );
+
+        return remoteHttpCacheResponse.getPayload();
+    }
+
+    /**
+     * Removes the given key from the specified cache.
+     * <p>
+     * @param cacheName
+     * @param key
+     * @throws IOException
+     */
+    public void remove( String cacheName, Serializable key )
+        throws IOException
+    {
+        remove( cacheName, key, 0 );
+    }
+
+    /**
+     * Removes the given key from the specified cache.
+     * <p>
+     * @param cacheName
+     * @param key
+     * @param requesterId
+     * @throws IOException
+     */
+    public void remove( String cacheName, Serializable key, long requesterId )
+        throws IOException
+    {
+        RemoteCacheRequest remoteHttpCacheRequest = RemoteCacheRequestFactory.createRemoveRequest( cacheName, key,
+                                                                                                   requesterId );
+
+        getRemoteDispatcher().dispatchRequest( remoteHttpCacheRequest );
+    }
+
+    /**
+     * Remove all keys from the sepcified cache.
+     * <p>
+     * @param cacheName
+     * @throws IOException
+     */
+    public void removeAll( String cacheName )
+        throws IOException
+    {
+        removeAll( cacheName, 0 );
+    }
+
+    /**
+     * Remove all keys from the sepcified cache.
+     * <p>
+     * @param cacheName
+     * @param requesterId
+     * @throws IOException
+     */
+    public void removeAll( String cacheName, long requesterId )
+        throws IOException
+    {
+        RemoteCacheRequest remoteHttpCacheRequest = RemoteCacheRequestFactory.createRemoveAllRequest( cacheName,
+                                                                                                      requesterId );
+
+        getRemoteDispatcher().dispatchRequest( remoteHttpCacheRequest );
+    }
+
+    /**
+     * Puts a cache item to the cache.
+     * <p>
+     * @param item
+     * @throws IOException
+     */
+    public void update( ICacheElement item )
+        throws IOException
+    {
+        update( item, 0 );
+    }
+
+    /**
+     * Puts a cache item to the cache.
+     * <p>
+     * @param cacheElement
+     * @param requesterId
+     * @throws IOException
+     */
+    public void update( ICacheElement cacheElement, long requesterId )
+        throws IOException
+    {
+        RemoteCacheRequest remoteHttpCacheRequest = RemoteCacheRequestFactory.createUpdateRequest( cacheElement,
+                                                                                                   requesterId );
+
+        getRemoteDispatcher().dispatchRequest( remoteHttpCacheRequest );
+    }
+
+    /**
+     * Frees the specified cache.
+     * <p>
+     * @param cacheName
+     * @throws IOException
+     */
+    public void dispose( String cacheName )
+        throws IOException
+    {
+        RemoteCacheRequest remoteHttpCacheRequest = RemoteCacheRequestFactory.createDisposeRequest( cacheName, 0 );
+
+        getRemoteDispatcher().dispatchRequest( remoteHttpCacheRequest );
+    }
+
+    /**
+     * Frees the specified cache.
+     * <p>
+     * @throws IOException
+     */
+    public void release()
+        throws IOException
+    {
+        // noop
+    }
+
+    /**
+     * @param cacheName
+     * @param groupName
+     * @return A Set of keys
+     * @throws IOException
+     */
+    public Set getGroupKeys( String cacheName, String groupName )
+        throws IOException
+    {
+        RemoteCacheRequest remoteHttpCacheRequest = RemoteCacheRequestFactory.createGetGroupKeysRequest( cacheName,
+                                                                                                         groupName, 0 );
+
+        getRemoteDispatcher().dispatchRequest( remoteHttpCacheRequest );
+
+        // FIX ME
+        return Collections.EMPTY_SET;
+    }
+
+    /**
+     * Make and alive request.
+     * <p>
+     * @return true if we make a successful alive request.
+     * @throws IOException 
+     */
+    protected boolean isAlive()
+        throws IOException
+    {
+        RemoteCacheRequest remoteHttpCacheRequest = RemoteCacheRequestFactory.createAliveCheckRequest( 0 );
+        RemoteCacheResponse remoteHttpCacheResponse = getRemoteDispatcher().dispatchRequest( remoteHttpCacheRequest );
+
+        if ( remoteHttpCacheResponse != null )
+        {
+            return remoteHttpCacheResponse.isSuccess();
+        }
+
+        return false;
+    }
+
+    /**
+     * @param remoteDispatcher the remoteDispatcher to set
+     */
+    public void setRemoteDispatcher( IRemoteCacheDispatcher remoteDispatcher )
+    {
+        this.remoteDispatcher = remoteDispatcher;
+    }
+
+    /**
+     * @return the remoteDispatcher
+     */
+    public IRemoteCacheDispatcher getRemoteDispatcher()
+    {
+        return remoteDispatcher;
+    }
+}

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheDispatcher.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheDispatcher.java?rev=726149&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheDispatcher.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheDispatcher.java Fri Dec 12 15:37:22 2008
@@ -0,0 +1,221 @@
+package org.apache.jcs.auxiliary.remote.http.client;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+
+import org.apache.commons.httpclient.HttpException;
+import org.apache.commons.httpclient.HttpMethod;
+import org.apache.commons.httpclient.HttpState;
+import org.apache.commons.httpclient.methods.ByteArrayRequestEntity;
+import org.apache.commons.httpclient.methods.PostMethod;
+import org.apache.commons.httpclient.methods.RequestEntity;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheDispatcher;
+import org.apache.jcs.auxiliary.remote.util.RemoteCacheRequestUtil;
+import org.apache.jcs.auxiliary.remote.value.RemoteCacheRequest;
+import org.apache.jcs.auxiliary.remote.value.RemoteCacheResponse;
+import org.apache.jcs.utils.serialization.StandardSerializer;
+
+/** Calls the service. */
+public class RemoteHttpCacheDispatcher
+    extends AbstractHttpClient
+    implements IRemoteCacheDispatcher
+{
+    /** Named of the parameter */
+    private static final String PARAMETER_REQUEST_TYPE = "RequestType";
+
+    /** Named of the parameter */
+    private static final String PARAMETER_KEY = "Key";
+
+    /** Named of the parameter */
+    private static final String PARAMETER_CACHE_NAME = "CacheName";
+
+    /** The Logger. */
+    private final static Log log = LogFactory.getLog( RemoteHttpCacheDispatcher.class );
+
+    /** This needs to be standard, since the other side is standard */
+    private StandardSerializer serializer = new StandardSerializer();
+
+    /**
+     * @param remoteHttpCacheAttributes
+     */
+    public RemoteHttpCacheDispatcher( RemoteHttpCacheAttributes remoteHttpCacheAttributes )
+    {
+        super( remoteHttpCacheAttributes );
+    }
+
+    /**
+     * All requests will go through this method.
+     * <p>
+     * TODO consider taking in a URL instead of using the one in the configuration.
+     * <p>
+     * @param remoteCacheRequest
+     * @return RemoteCacheResponse
+     * @throws IOException
+     */
+    public RemoteCacheResponse dispatchRequest( RemoteCacheRequest remoteCacheRequest )
+        throws IOException
+    {
+        try
+        {
+            byte[] requestAsByteArray = serializer.serialize( remoteCacheRequest );
+
+            String url = addParameters( remoteCacheRequest, getRemoteHttpCacheAttributes().getUrl() );
+
+            byte[] responseAsByteArray = processRequest( requestAsByteArray, url );
+
+            RemoteCacheResponse remoteCacheResponse = null;
+            try
+            {
+                remoteCacheResponse = (RemoteCacheResponse) serializer.deSerialize( responseAsByteArray );
+            }
+            catch ( ClassNotFoundException e )
+            {
+                log.error( "Couldn't deserialize the response.", e );
+            }
+            return remoteCacheResponse;
+        }
+        catch ( Exception e )
+        {
+            log.error( "Problem dispatching request.", e );
+            throw new IOException( e.getMessage() );
+        }
+    }
+
+    /**
+     * @param requestAsByteArray
+     * @param url
+     * @return byte[] - the response
+     * @throws IOException
+     * @throws HttpException
+     */
+    protected byte[] processRequest( byte[] requestAsByteArray, String url )
+        throws IOException, HttpException
+    {
+        PostMethod post = new PostMethod( url );
+        RequestEntity requestEntity = new ByteArrayRequestEntity( requestAsByteArray );
+        post.setRequestEntity( requestEntity );
+        getHttpClient().executeMethod( post );
+        byte[] response = post.getResponseBody();
+        return response;
+    }
+
+    /**
+     * @param remoteCacheRequest
+     * @param baseUrl
+     * @return String
+     */
+    protected String addParameters( RemoteCacheRequest remoteCacheRequest, String baseUrl )
+    {
+        StringBuffer url = new StringBuffer( baseUrl );
+
+        try
+        {
+            if ( baseUrl != null && ( baseUrl.indexOf( "?" ) == -1 ) )
+            {
+                url.append( "?" );
+            }
+
+            if ( getRemoteHttpCacheAttributes().isIncludeCacheNameAsParameter() )
+            {
+                if ( remoteCacheRequest.getCacheName() != null )
+                {
+                    url.append( PARAMETER_CACHE_NAME + "="
+                        + URLEncoder.encode( remoteCacheRequest.getCacheName(), "UTF-8" ) );
+                }
+            }
+            if ( getRemoteHttpCacheAttributes().isIncludeKeysAndPatternsAsParameter() )
+            {
+                String keyValue = "";
+                switch ( remoteCacheRequest.getRequestType() )
+                {
+                    case RemoteCacheRequest.REQUEST_TYPE_GET:
+                        keyValue = remoteCacheRequest.getKey() + "";
+                        break;
+                    case RemoteCacheRequest.REQUEST_TYPE_REMOVE:
+                        keyValue = remoteCacheRequest.getKey() + "";
+                        break;
+                    case RemoteCacheRequest.REQUEST_TYPE_GET_MATCHING:
+                        keyValue = remoteCacheRequest.getPattern();
+                        break;
+                    case RemoteCacheRequest.REQUEST_TYPE_GET_MULTIPLE:
+                        keyValue = remoteCacheRequest.getKeySet() + "";
+                        break;
+                    case RemoteCacheRequest.REQUEST_TYPE_GET_GROUP_KEYS:
+                        keyValue = remoteCacheRequest.getKey() + "";
+                        break;
+                    case RemoteCacheRequest.REQUEST_TYPE_UPDATE:
+                        keyValue = remoteCacheRequest.getCacheElement().getKey() + "";
+                        break;
+                    default:
+                        break;
+                }
+                String encodedKeyValue = URLEncoder.encode( keyValue, "UTF-8" );
+                url.append( "&" + PARAMETER_KEY + "=" + encodedKeyValue );
+            }
+            if ( getRemoteHttpCacheAttributes().isIncludeRequestTypeasAsParameter() )
+            {
+                url.append( "&"
+                    + PARAMETER_REQUEST_TYPE
+                    + "="
+                    + URLEncoder.encode( RemoteCacheRequestUtil
+                        .getRequestTypeName( remoteCacheRequest.getRequestType() ), "UTF-8" ) );
+            }
+        }
+        catch ( UnsupportedEncodingException e )
+        {
+            log.error( "Couldn't encode URL.", e );
+        }
+
+        if ( log.isDebugEnabled() )
+        {
+            log.debug( "Url: " + url.toString() );
+        }
+
+        return url.toString();
+    }
+
+    /**
+     * Extracted method that can be overwritten to do additional things to the post before the call
+     * is made.
+     * <p>
+     * @param post the post that is about to get executed.
+     * @throws IOException on i/o error
+     */
+    protected final void doWebserviceCall( HttpMethod post )
+        throws IOException
+    {
+        HttpState httpState = preProcessWebserviceCall( post );
+        getHttpClient().executeMethod( null, post, httpState );
+        postProcessWebserviceCall( post, httpState );
+    }
+
+    /**
+     * Called before the executeMethod on the client.
+     * <p>
+     * @param post http method
+     * @return HttpState
+     * @throws IOException
+     */
+    public HttpState preProcessWebserviceCall( HttpMethod post )
+        throws IOException
+    {
+        // do nothing. Child can override.
+        return null;
+    }
+
+    /**
+     * Called after the executeMethod on the client.
+     * <p>
+     * @param post http method
+     * @param httpState state
+     * @throws IOException
+     */
+    public void postProcessWebserviceCall( HttpMethod post, HttpState httpState )
+        throws IOException
+    {
+        // do nothing. Child can override.
+    }
+}

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheFactory.java?rev=726149&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheFactory.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheFactory.java Fri Dec 12 15:37:22 2008
@@ -0,0 +1,124 @@
+package org.apache.jcs.auxiliary.remote.http.client;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.jcs.auxiliary.AuxiliaryCache;
+import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
+import org.apache.jcs.auxiliary.AuxiliaryCacheFactory;
+import org.apache.jcs.auxiliary.remote.RemoteCacheAttributes;
+import org.apache.jcs.auxiliary.remote.RemoteCacheNoWait;
+import org.apache.jcs.auxiliary.remote.RemoteCacheNoWaitFacade;
+import org.apache.jcs.engine.behavior.ICache;
+import org.apache.jcs.engine.behavior.ICompositeCacheManager;
+import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
+
+/**
+ * The RemoteCacheFactory creates remote caches for the cache hub. It returns a no wait facade which
+ * is a wrapper around a no wait. The no wait object is either an active connection to a remote
+ * cache or a balking zombie if the remote cache is not accessible. It should be transparent to the
+ * clients.
+ */
+public class RemoteHttpCacheFactory
+    implements AuxiliaryCacheFactory
+{
+    /** The logger. */
+    private final static Log log = LogFactory.getLog( RemoteHttpCacheFactory.class );
+
+    /** The name of this auxiliary */
+    private String name;
+
+    /** store reference of facades to initiate failover */
+    private final static HashMap facades = new HashMap();
+
+    /**
+     * For LOCAL clients we get a handle to all the failovers, but we do not register a listener
+     * with them. We create the RemoteCacheManager, but we do not get a cache.
+     * <p>
+     * The failover runner will get a cache from the manager. When the primary is restored it will
+     * tell the manager for the failover to deregister the listener.
+     * <p>
+     * @param iaca
+     * @param cacheMgr
+     * @param cacheEventLogger
+     * @param elementSerializer
+     * @return AuxiliaryCache
+     */
+    public AuxiliaryCache createCache( AuxiliaryCacheAttributes iaca, ICompositeCacheManager cacheMgr,
+                                       ICacheEventLogger cacheEventLogger, IElementSerializer elementSerializer )
+    {
+        RemoteHttpCacheAttributes rca = (RemoteHttpCacheAttributes) iaca;
+
+        ArrayList noWaits = new ArrayList();
+
+        RemoteHttpCacheManager rcm = RemoteHttpCacheManager.getInstance( cacheMgr, cacheEventLogger, elementSerializer );
+        rca.setRemoteType( RemoteCacheAttributes.CLUSTER );
+        ICache ic = rcm.getCache( rca );
+        if ( ic != null )
+        {
+            noWaits.add( ic );
+        }
+        else
+        {
+            log.info( "noWait is null" );
+        }
+
+        RemoteCacheNoWaitFacade rcnwf = new RemoteCacheNoWaitFacade( (RemoteCacheNoWait[]) noWaits
+            .toArray( new RemoteCacheNoWait[0] ), rca, cacheMgr, cacheEventLogger, elementSerializer );
+
+        getFacades().put( rca.getCacheName(), rcnwf );
+
+        return rcnwf;
+    }
+
+    /**
+     * Gets the name attribute of the RemoteCacheFactory object
+     * <p>
+     * @return The name value
+     */
+    public String getName()
+    {
+        return this.name;
+    }
+
+    /**
+     * Sets the name attribute of the RemoteCacheFactory object
+     * <p>
+     * @param name The new name value
+     */
+    public void setName( String name )
+    {
+        this.name = name;
+    }
+
+    /**
+     * The facades are what the cache hub talks to.
+     * @return Returns the facades.
+     */
+    public static HashMap getFacades()
+    {
+        return facades;
+    }
+}

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheManager.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheManager.java?rev=726149&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheManager.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/http/client/RemoteHttpCacheManager.java Fri Dec 12 15:37:22 2008
@@ -0,0 +1,291 @@
+package org.apache.jcs.auxiliary.remote.http.client;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.jcs.auxiliary.AuxiliaryCache;
+import org.apache.jcs.auxiliary.AuxiliaryCacheManager;
+import org.apache.jcs.auxiliary.remote.RemoteCacheNoWait;
+import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheAttributes;
+import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheClient;
+import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheService;
+import org.apache.jcs.engine.behavior.ICache;
+import org.apache.jcs.engine.behavior.ICompositeCacheManager;
+import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.behavior.IShutdownObserver;
+import org.apache.jcs.engine.control.CompositeCacheManager;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
+
+/**
+ * This is a very crude copy of the RMI remote manager. It needs a lot of work!
+ */
+public class RemoteHttpCacheManager
+    implements AuxiliaryCacheManager, IShutdownObserver
+{
+    /** Don't change */
+    private static final long serialVersionUID = 798077557166389498L;
+
+    /** The logger */
+    private final static Log log = LogFactory.getLog( RemoteHttpCacheManager.class );
+
+    /** Contains mappings of Location instance to RemoteCacheManager instance. */
+    private static RemoteHttpCacheManager instance;
+
+    /** Contains instances of RemoteCacheNoWait managed by a RemoteCacheManager instance. */
+    static final Map caches = new HashMap();
+
+    /** The configuration attributes. */
+    private IRemoteCacheAttributes remoteCacheAttributes;
+
+    /** The event logger. */
+    private ICacheEventLogger cacheEventLogger;
+
+    /** The serializer. */
+    private IElementSerializer elementSerializer;
+
+    /** The cache manager listeners will need to use to get a cache. */
+    private ICompositeCacheManager cacheMgr;
+
+    /** Remote cache monitor. */
+    private static RemoteHttpCacheMonitor monitor;
+
+    /**
+     * Constructs an instance to with the given remote connection parameters. If the connection
+     * cannot be made, "zombie" services will be temporarily used until a successful re-connection
+     * is made by the monitoring daemon.
+     * <p>
+     * @param cacheMgr
+     * @param cacheEventLogger
+     * @param elementSerializer
+     */
+    private RemoteHttpCacheManager( ICompositeCacheManager cacheMgr, ICacheEventLogger cacheEventLogger,
+                                    IElementSerializer elementSerializer )
+    {
+        this.cacheMgr = cacheMgr;
+        this.cacheEventLogger = cacheEventLogger;
+        this.elementSerializer = elementSerializer;
+
+        // register shutdown observer
+        // TODO add the shutdown observable methods to the interface
+        if ( this.cacheMgr instanceof CompositeCacheManager )
+        {
+            ( (CompositeCacheManager) this.cacheMgr ).registerShutdownObserver( this );
+        }
+    }
+
+    /**
+     * Gets the defaultCattr attribute of the RemoteCacheManager object.
+     * <p>
+     * @return The defaultCattr value
+     */
+    public IRemoteCacheAttributes getDefaultCattr()
+    {
+        return this.remoteCacheAttributes;
+    }
+
+    /** @return Returns an instance if it exists. else null. */
+    public synchronized static RemoteHttpCacheManager getInstance()
+    {
+        return instance;
+    }
+
+    /**
+     * Get the singleton instance.
+     * <p>
+     * @param cacheMgr
+     * @param cacheEventLogger
+     * @param elementSerializer
+     * @return The instance value
+     */
+    public synchronized static RemoteHttpCacheManager getInstance( ICompositeCacheManager cacheMgr,
+                                                                   ICacheEventLogger cacheEventLogger,
+                                                                   IElementSerializer elementSerializer )
+    {
+        if ( instance == null )
+        {
+            instance = new RemoteHttpCacheManager( cacheMgr, cacheEventLogger, elementSerializer );
+        }
+
+        // Fires up the monitoring daemon.
+        if ( monitor == null )
+        {
+            monitor = RemoteHttpCacheMonitor.getInstance();
+            // If the returned monitor is null, it means it's already started
+            // elsewhere.
+            if ( monitor != null )
+            {
+                Thread t = new Thread( monitor );
+                t.setDaemon( true );
+                t.start();
+            }
+        }
+
+        return instance;
+    }
+
+    /**
+     * Returns a remote cache for the given cache name.
+     * <p>
+     * @param cacheName
+     * @return The cache value
+     */
+    public AuxiliaryCache getCache( String cacheName )
+    {
+        // TODO get some defaults!
+        // Perhaps we will need a manager per URL????
+        RemoteHttpCacheAttributes ca = new RemoteHttpCacheAttributes();
+        ca.setCacheName( cacheName );
+        return getCache( ca );
+    }
+
+    /**
+     * Gets a RemoteCacheNoWait from the RemoteCacheManager. The RemoteCacheNoWait objects are
+     * identified by the cache name value of the RemoteCacheAttributes object.
+     * <p>
+     * If the client is configured to register a listener, this call results on a listener being
+     * created if one isn't already registered with the remote cache for this region.
+     * <p>
+     * @param cattr
+     * @return The cache value
+     */
+    public AuxiliaryCache getCache( RemoteHttpCacheAttributes cattr )
+    {
+        RemoteCacheNoWait remoteCacheNoWait = null;
+
+        synchronized ( caches )
+        {
+            remoteCacheNoWait = (RemoteCacheNoWait) caches.get( cattr.getCacheName() + cattr.getUrl() );
+            if ( remoteCacheNoWait == null )
+            {
+                RemoteHttpClientListener listener = new RemoteHttpClientListener( cattr, cacheMgr );
+
+                RemoteHttpCacheClient remoteService = createRemoteHttpCacheClientForAttributes( cattr );
+
+                IRemoteCacheClient remoteCacheClient = new RemoteHttpCache( cattr, remoteService, listener );
+                remoteCacheClient.setCacheEventLogger( cacheEventLogger );
+                remoteCacheClient.setElementSerializer( elementSerializer );
+
+                remoteCacheNoWait = new RemoteCacheNoWait( remoteCacheClient );
+                remoteCacheNoWait.setCacheEventLogger( cacheEventLogger );
+                remoteCacheNoWait.setElementSerializer( elementSerializer );
+
+                caches.put( cattr.getCacheName() + cattr.getUrl(), remoteCacheNoWait );
+            }
+            // might want to do some listener sanity checking here.
+        }
+
+        return remoteCacheNoWait;
+    }
+
+    /**
+     * This is an extension point. The manager and other classes will only create
+     * RemoteHttpCacheClient through this method.
+     * <p>
+     * @param cattr
+     * @return RemoteHttpCacheClient
+     */
+    protected RemoteHttpCacheClient createRemoteHttpCacheClientForAttributes( RemoteHttpCacheAttributes cattr )
+    {
+        RemoteHttpCacheClient remoteService = new RemoteHttpCacheClient( cattr );
+        return remoteService;
+    }
+
+    /**
+     * Gets the stats attribute of the RemoteCacheManager object
+     * <p>
+     * @return The stats value
+     */
+    public String getStats()
+    {
+        StringBuffer stats = new StringBuffer();
+        Iterator allCaches = caches.values().iterator();
+        while ( allCaches.hasNext() )
+        {
+            ICache c = (ICache) allCaches.next();
+            if ( c != null )
+            {
+                stats.append( c.getCacheName() );
+            }
+        }
+        return stats.toString();
+    }
+
+    /**
+     * Fixes up all the caches managed by this cache manager.
+     * <p>
+     * @param remoteService
+     */
+    public void fixCaches( IRemoteCacheService remoteService )
+    {
+        synchronized ( caches )
+        {
+            //            this.remoteService = remoteService;
+            //            // this.remoteWatch.setCacheWatch( remoteWatch );
+            //            for ( Iterator en = caches.values().iterator(); en.hasNext(); )
+            //            {
+            //                RemoteCacheNoWait cache = (RemoteCacheNoWait) en.next();
+            //                cache.fixCache( this.remoteService );
+            //            }
+        }
+    }
+
+    /**
+     * Gets the cacheType attribute of the RemoteCacheManager object
+     * @return The cacheType value
+     */
+    public int getCacheType()
+    {
+        return REMOTE_CACHE;
+    }
+
+    /**
+     * Shutdown callback from composite cache manager.
+     * <p>
+     * @see org.apache.jcs.engine.behavior.IShutdownObserver#shutdown()
+     */
+    public void shutdown()
+    {
+        if ( log.isInfoEnabled() )
+        {
+            log.info( "Observed shutdown request." );
+        }
+        //release();
+    }
+
+    /**
+     * Logs an event if an event logger is configured.
+     * <p>
+     * @param source
+     * @param eventName
+     * @param optionalDetails
+     */
+    protected void logApplicationEvent( String source, String eventName, String optionalDetails )
+    {
+        if ( cacheEventLogger != null )
+        {
+            cacheEventLogger.logApplicationEvent( source, eventName, optionalDetails );
+        }
+    }
+}



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