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/08/19 00:11:16 UTC

svn commit: r686881 [1/2] - in /jakarta/jcs/trunk/src: conf/ java/org/apache/jcs/auxiliary/ java/org/apache/jcs/auxiliary/disk/ java/org/apache/jcs/auxiliary/disk/block/ java/org/apache/jcs/auxiliary/disk/indexed/ java/org/apache/jcs/auxiliary/disk/jdb...

Author: asmuts
Date: Mon Aug 18 15:11:14 2008
New Revision: 686881

URL: http://svn.apache.org/viewvc?rev=686881&view=rev
Log:
changed event logging interface, added abstract base, cleaned some javadocs.

Added:
    jakarta/jcs/trunk/src/conf/cacheRC_CEL.ccf
    jakarta/jcs/trunk/src/conf/remote.cacheCEL.ccf
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AbstractAuxiliaryCache.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheConfigurator.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICacheElementSerialized.java   (with props)
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/logging/
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/logging/CacheEvent.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/logging/CacheEventLoggerDebugLoggerImpl.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/logging/behavior/
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/logging/behavior/ICacheEvent.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/logging/behavior/ICacheEventLogger.java   (contents, props changed)
      - copied, changed from r685702, jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICacheEventLogger.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/AuxiliaryCacheConfiguratorUnitTest.java
Removed:
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICacheEventLogger.java
Modified:
    jakarta/jcs/trunk/src/conf/log4j.properties
    jakarta/jcs/trunk/src/conf/remote.cache.ccf
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCache.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCache.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCacheAttributes.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDisk.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCache.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCacheFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDisk.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCacheFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskElementDescriptor.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCache.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheManagerAbstractTemplate.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/mysql/MySQLDiskCacheFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheAbstractFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWait.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWaitFacade.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/LateralTCPCacheFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCache.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFailoverRunner.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheManager.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWait.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWaitFacade.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/behavior/IRemoteCacheConstants.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/RemoteCacheServerFactory.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/ZombieCacheService.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/CompositeCache.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/CompositeCacheConfigurator.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/CompositeCacheManager.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/event/behavior/IElementEvent.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/event/behavior/IElementEventConstants.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/group/GroupAttrName.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/control/group/GroupId.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/memory/lru/LRUMemoryCache.java
    jakarta/jcs/trunk/src/scripts/startRemoteCache.bat
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/MockAuxiliaryCache.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/MockCacheEventLogger.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/MockRemoteCacheClient.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/MockRemoteCacheListener.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/remote/RemoteCacheClientTester.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/engine/control/CompositeCacheConfiguratorUnitTest.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/engine/control/CompositeCacheDiskUsageUnitTest.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/engine/control/MockCacheEventLogger.java

Added: jakarta/jcs/trunk/src/conf/cacheRC_CEL.ccf
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/conf/cacheRC_CEL.ccf?rev=686881&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/conf/cacheRC_CEL.ccf (added)
+++ jakarta/jcs/trunk/src/conf/cacheRC_CEL.ccf Mon Aug 18 15:11:14 2008
@@ -0,0 +1,248 @@
+# 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.
+# #############################################################
+# ################# DEFAULT CACHE REGION  #####################
+# sets the default aux value for any non configured caches
+jcs.default=DC,RC
+jcs.default.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.default.cacheattributes.MaxObjects=200001
+jcs.default.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+jcs.default.cacheattributes.UseMemoryShrinker=true
+jcs.default.cacheattributes.MaxMemoryIdleTimeSeconds=3600
+jcs.default.cacheattributes.ShrinkerIntervalSeconds=60
+jcs.default.elementattributes=org.apache.jcs.engine.ElementAttributes
+jcs.default.elementattributes.IsEternal=false
+jcs.default.elementattributes.MaxLifeSeconds=700
+jcs.default.elementattributes.IdleTime=1800
+jcs.default.elementattributes.IsSpool=true
+jcs.default.elementattributes.IsRemote=true
+jcs.default.elementattributes.IsLateral=true
+
+
+# #############################################################
+# ################# CACHE REGIONS AVAILABLE ###################
+# Regions preconfirgured for caching
+jcs.region.testCache1=DC,RC
+jcs.region.testCache1.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.region.testCache1.cacheattributes.MaxObjects=1000000
+jcs.region.testCache1.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+jcs.region.testCache1.cacheattributes.UseMemoryShrinker=true
+jcs.region.testCache1.cacheattributes.ShrinkerIntervalSeconds=30
+jcs.region.testCache1.cacheattributes.MaxMemoryIdleTimeSeconds=300
+jcs.region.testCache1.cacheattributes.MaxSpoolPerRun=100
+jcs.region.testCache1.elementattributes=org.apache.jcs.engine.ElementAttributes
+jcs.region.testCache1.elementattributes.IsEternal=false
+jcs.region.testCache1.elementattributes.MaxLifeSeconds=60000
+jcs.region.testCache1.elementattributes.IsLateral=true
+jcs.region.testCache1.elementattributes.IsRemote=true
+
+jcs.region.testCache2=DC,RC
+jcs.region.testCache2.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.region.testCache2.cacheattributes.MaxObjects=100
+jcs.region.testCache2.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+jcs.region.testCache2.cacheattributes.UseMemoryShrinker=true
+jcs.region.testCache2.cacheattributes.MaxMemoryIdleTimeSeconds=1000
+jcs.region.testCache2.cacheattributes.ShrinkerIntervalSeconds=40
+jcs.region.testCache2.elementattributes=org.apache.jcs.engine.ElementAttributes
+jcs.region.testCache2.elementattributes.IsEternal=false
+jcs.region.testCache2.elementattributes.MaxLifeSeconds=600
+jcs.region.testCache2.elementattributes.IsSpool=true
+jcs.region.testCache2.elementattributes.IsRemote=true
+jcs.region.testCache2.elementattributes.IsLateral=true
+
+jcs.region.testCache3=
+jcs.region.testCache3.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.region.testCache3.cacheattributes.MaxObjects=100000
+jcs.region.testCache3.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+jcs.region.testCache3.cacheattributes.UseMemoryShrinker=false
+jcs.region.testCache3.cacheattributes.MaxMemoryIdleTimeSeconds=10
+jcs.region.testCache3.cacheattributes.ShrinkerIntervalSeconds=60
+jcs.region.testCache3.elementattributes=org.apache.jcs.engine.ElementAttributes
+jcs.region.testCache3.elementattributes.IsEternal=false
+jcs.region.testCache3.elementattributes.MaxLifeSeconds=3600
+jcs.region.testCache3.elementattributes.IsSpool=true
+jcs.region.testCache3.elementattributes.IsRemote=true
+jcs.region.testCache3.elementattributes.IsLateral=true
+
+
+# #############################################################
+# ################# AUXILIARY CACHES AVAILABLE ################
+
+# Remote RMI cache without failover
+jcs.auxiliary.RGroup=org.apache.jcs.auxiliary.remote.RemoteCacheFactory
+jcs.auxiliary.RGroup.attributes=org.apache.jcs.auxiliary.remote.RemoteCacheAttributes
+jcs.auxiliary.RGroup.attributes.RemoteTypeName=LOCAL
+jcs.auxiliary.RGroup.attributes.RemoteHost=localhost
+jcs.auxiliary.RGroup.attributes.RemotePort=1102
+jcs.auxiliary.RGroup.attributes.GetOnly=true
+
+# Remote RMI Cache set up to failover
+jcs.auxiliary.RFailover=org.apache.jcs.auxiliary.remote.RemoteCacheFactory
+jcs.auxiliary.RFailover.attributes=org.apache.jcs.auxiliary.remote.RemoteCacheAttributes
+jcs.auxiliary.RFailover.attributes.RemoteTypeName=LOCAL
+jcs.auxiliary.RFailover.attributes.FailoverServers=localhost:1102
+jcs.auxiliary.RFailover.attributes.GetOnly=false
+
+# Primary Disk Cache-- faster than the rest because of memory key storage
+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=target/test-sandbox/raf
+jcs.auxiliary.DC.attributes.MaxPurgatorySize=10000000
+jcs.auxiliary.DC.attributes.MaxKeySize=1000000
+jcs.auxiliary.DC.attributes.MaxRecycleBinSize=5000
+jcs.auxiliary.DC.attributes.OptimizeAtRemoveCount=300000
+jcs.auxiliary.DC.attributes.ShutdownSpoolTimeLimit=60
+
+# Disk Cache Using a Pooled Event Queue -- this allows you
+# to control the maximum number of threads it will use.
+# Each region uses 1 thread by default in the SINGLE model.
+# adding more threads does not help.
+# If you want to use a separate pool for each disk cache, either use
+# the single model or define a different auxiliary for each region and use the Pooled.
+# SINGLE is best unless you ahve a huge # of regions.
+jcs.auxiliary.DC2=org.apache.jcs.auxiliary.disk.indexed.IndexedDiskCacheFactory
+jcs.auxiliary.DC2.attributes=org.apache.jcs.auxiliary.disk.indexed.IndexedDiskCacheAttributes
+jcs.auxiliary.DC2.attributes.DiskPath=target/test-sandbox/raf
+jcs.auxiliary.DC2.attributes.MaxPurgatorySize=10000
+jcs.auxiliary.DC2.attributes.MaxKeySize=10000
+jcs.auxiliary.DC2.attributes.MaxRecycleBinSize=5000
+jcs.auxiliary.DC2.attributes.OptimizeAtRemoveCount=300000
+jcs.auxiliary.DC2.attributes.EventQueueType=POOLED
+jcs.auxiliary.DC2.attributes.EventQueuePoolName=disk_cache_event_queue
+
+# Berkeley DB JE
+jcs.auxiliary.JE=org.apache.jcs.auxiliary.disk.bdbje.BDBJECacheFactory
+jcs.auxiliary.JE.attributes=org.apache.jcs.auxiliary.disk.bdbje.BDBJECacheAttributes
+jcs.auxiliary.JE.attributes.DiskPath=target/test-sandbox/bdbje-disk-cache-conc
+# the minimum cache size is 1024
+jcs.auxiliary.indexedDiskCache.attributes.CacheSize=1024
+# jcs.auxiliary.indexedDiskCache.attributes.CachePercent=0
+
+# HSQL Disk Cache -- too slow as is
+jcs.auxiliary.HDC=org.apache.jcs.auxiliary.disk.hsql.HSQLCacheFactory
+jcs.auxiliary.HDC.attributes=org.apache.jcs.auxiliary.disk.hsql.HSQLCacheAttributes
+jcs.auxiliary.HDC.attributes.DiskPath=@project_home_f@hsql
+
+# JISP Disk Cache -- save memory with disk key storage
+jcs.auxiliary.JDC=org.apache.jcs.auxiliary.disk.jisp.JISPCacheFactory
+jcs.auxiliary.JDC.attributes=org.apache.jcs.auxiliary.disk.jisp.JISPCacheAttributes
+jcs.auxiliary.JDC.attributes.DiskPath=@project_home_f@raf
+jcs.auxiliary.JDC.attributes.ClearOnStart=false
+
+# 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,localhost:1102
+jcs.auxiliary.RC.attributes.LocalPort=1201
+jcs.auxiliary.RC.attributes.RemoveUponRemotePut=false
+# jcs.auxiliary.RC.attributes.RemoteServiceName=RemoteCache
+# -1 means no timeout, this is the default
+# if the timeout is -1, no threadpool will be used.
+jcs.auxiliary.RC.attributes.GetTimeoutMillis=500
+jcs.auxiliary.RC.attributes.ThreadPoolName=remote_cache_client
+jcs.auxiliary.RC.attributes.GetOnly=false
+jcs.auxiliary.RC.cacheeventlogger=org.apache.jcs.engine.logging.CacheEventLoggerDebugLoggerImpl
+jcs.auxiliary.RC.cacheeventlogger.attributes.logCategoryName=TestEventLogCategory
+
+# unreliable
+jcs.auxiliary.LUDP=org.apache.jcs.auxiliary.lateral.LateralCacheFactory
+jcs.auxiliary.LUDP.attributes=org.apache.jcs.auxiliary.lateral.LateralCacheAttributes
+jcs.auxiliary.LUDP.attributes.TransmissionTypeName=UDP
+jcs.auxiliary.LUDP.attributes.UdpMulticastAddr=228.5.6.7
+jcs.auxiliary.LUDP.attributes.UdpMulticastPort=6789
+
+jcs.auxiliary.LJG=org.apache.jcs.auxiliary.lateral.LateralCacheFactory
+jcs.auxiliary.LJG.attributes=org.apache.jcs.auxiliary.lateral.LateralCacheAttributes
+jcs.auxiliary.LJG.attributes.TransmissionTypeName=JAVAGROUPS
+jcs.auxiliary.LJG.attributes.PutOnlyMode=true
+jcs.auxiliary.LJG.attributes.JGChannelProperties = UDP(mcast_addr=224.0.0.100;mcast_port=751):PING(timeout=3000):FD:STABLE:NAKACK:UNICAST:FRAG:FLUSH:GMS:VIEW_ENFORCER:QUEUE
+
+
+jcs.auxiliary.JG = org.apache.jcs.auxiliary.javagroups.JavaGroupsCacheFactory
+jcs.auxiliary.JG.attributes = org.apache.jcs.auxiliary.javagroups.JavaGroupsCacheAttributes
+jcs.auxiliary.JG.attributes.ChannelFactoryClassName = org.javagroups.JChannelFactory
+jcs.auxiliary.JG.attributes.ChannelProperties = UDP(mcast_addr=224.0.0.100;mcast_port=7501):PING:FD:STABLE:NAKACK:UNICAST:FRAG:FLUSH:GMS:VIEW_ENFORCER:QUEUE
+
+
+# almost complete
+jcs.auxiliary.LTCP=org.apache.jcs.auxiliary.lateral.LateralCacheFactory
+jcs.auxiliary.LTCP.attributes=org.apache.jcs.auxiliary.lateral.LateralCacheAttributes
+jcs.auxiliary.LTCP.attributes.TransmissionTypeName=TCP
+jcs.auxiliary.LTCP.attributes.TcpServers=localhost:1112
+jcs.auxiliary.LTCP.attributes.TcpListenerPort=1111
+jcs.auxiliary.LTCP.attributes.PutOnlyMode=true
+
+jcs.auxiliary.LTCP2=org.apache.jcs.auxiliary.lateral.LateralCacheFactory
+jcs.auxiliary.LTCP2.attributes=org.apache.jcs.auxiliary.lateral.LateralCacheAttributes
+jcs.auxiliary.LTCP2.attributes.TransmissionTypeName=TCP
+jcs.auxiliary.LTCP2.attributes.TcpServers=localhost:1112
+jcs.auxiliary.LTCP2.attributes.TcpListenerPort=1111
+jcs.auxiliary.LTCP2.attributes.PutOnlyMode=true
+
+jcs.auxiliary.XMLRPC=org.apache.jcs.auxiliary.lateral.LateralCacheFactory
+jcs.auxiliary.XMLRPC.attributes=org.apache.jcs.auxiliary.lateral.LateralCacheAttributes
+jcs.auxiliary.XMLRPC.attributes.TransmissionTypeName=XMLRPC
+jcs.auxiliary.XMLRPC.attributes.HttpServers=localhost:8182
+jcs.auxiliary.XMLRPC.attributes.HttpListenerPort=8181
+jcs.auxiliary.XMLRPC.attributes.PutOnlyMode=false
+
+
+# example of how to configure the http version of the lateral cache
+# not converteed to new cache
+jcs.auxiliary.LCHTTP=org.apache.jcs.auxiliary.lateral.LateralCacheFactory
+jcs.auxiliary.LCHTTP.attributes=org.apache.jcs.auxiliary.lateral.LateralCacheAttributes
+jcs.auxiliary.LCHTTP.attributes.TransmissionType=HTTP
+jcs.auxiliary.LCHTTP.attributes.httpServers=localhost:8080,localhost:7001,localhost:80
+jcs.auxiliary.LCHTTP.attributes.httpReceiveServlet=/cache/LateralCacheReceiverServlet
+jcs.auxiliary.LCHTTP.attributes.httpDeleteServlet=/cache/DeleteCacheServlet
+
+
+# #############################################################
+# ################# OPTIONAL 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
+# since it doesn't use a boundary, some of the options are unnecessary
+thread_pool.cache_event_queue.useBoundary=false
+thread_pool.cache_event_queue.minimumPoolSize=5
+thread_pool.cache_event_queue.keepAliveTime=3500
+thread_pool.cache_event_queue.startUpSize=5
+
+# Disk Cache pool
+thread_pool.disk_cache_event_queue.useBoundary=false
+thread_pool.remote_cache_client.maximumPoolSize=15
+thread_pool.disk_cache_event_queue.minimumPoolSize=1
+thread_pool.disk_cache_event_queue.keepAliveTime=3500
+thread_pool.disk_cache_event_queue.startUpSize=1
+
+# 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/viewvc/jakarta/jcs/trunk/src/conf/log4j.properties?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/conf/log4j.properties (original)
+++ jakarta/jcs/trunk/src/conf/log4j.properties Mon Aug 18 15:11:14 2008
@@ -37,6 +37,8 @@
 
 # ################################################################################
 
+log4j.category.TestEventLogCategory= DEBUG
+
 log4j.category.org.jgroups=WARN
 
 log4j.category.org.apache.jcs=INFO

Modified: jakarta/jcs/trunk/src/conf/remote.cache.ccf
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/conf/remote.cache.ccf?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/conf/remote.cache.ccf (original)
+++ jakarta/jcs/trunk/src/conf/remote.cache.ccf Mon Aug 18 15:11:14 2008
@@ -19,11 +19,10 @@
 # Registry used to register and provide the IRmiCacheService service.
 registry.host=localhost
 registry.port=1101
-# call back port to local caches.
 remote.cache.service.port=1101
-# cluster setting
 remote.cluster.LocalClusterConsistency=true
-
+#remote.cacheeventlogger=org.apache.jcs.engine.logging.CacheEventLoggerDebugLoggerImpl
+#remote.cacheeventlogger.attributes.logCategoryName=TestEventLogCategory
 
 # #############################################################
 # ################# DEFAULT CACHE REGION  #####################

Added: jakarta/jcs/trunk/src/conf/remote.cacheCEL.ccf
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/conf/remote.cacheCEL.ccf?rev=686881&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/conf/remote.cacheCEL.ccf (added)
+++ jakarta/jcs/trunk/src/conf/remote.cacheCEL.ccf Mon Aug 18 15:11:14 2008
@@ -0,0 +1,68 @@
+# 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.
+# #############################################################
+# ################# REMOTE SERVER CONFIG  #####################
+# Registry used to register and provide the IRmiCacheService service.
+registry.host=localhost
+registry.port=1101
+remote.cache.service.port=1101
+remote.cluster.LocalClusterConsistency=true
+remote.cacheeventlogger=org.apache.jcs.engine.logging.CacheEventLoggerDebugLoggerImpl
+remote.cacheeventlogger.attributes.logCategoryName=TestEventLogCategory
+
+# #############################################################
+# ################# DEFAULT CACHE REGION  #####################
+# sets the default aux value for any non configured caches
+jcs.default=DC,RCluster
+jcs.default.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.default.cacheattributes.MaxObjects=200000
+jcs.default.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+jcs.default.cacheattributes.UseMemoryShrinker=true
+jcs.default.cacheattributes.MaxMemoryIdleTimeSeconds=3600
+jcs.default.cacheattributes.ShrinkerIntervalSeconds=60
+jcs.default.elementattributes=org.apache.jcs.engine.ElementAttributes
+jcs.default.elementattributes.IsEternal=false
+jcs.default.elementattributes.MaxLifeSeconds=7000
+jcs.default.elementattributes.IdleTime=1800
+jcs.default.elementattributes.IsSpool=true
+jcs.default.elementattributes.IsRemote=true
+jcs.default.elementattributes.IsLateral=true
+
+
+# #############################################################
+# ################# CACHE REGIONS AVAILABLE ###################
+jcs.region.testCache1=DC,RCluster
+jcs.region.testCache1.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.region.testCache1.cacheattributes.MaxObjects=200002
+
+
+# #############################################################
+# ################# AUXILIARY CACHES AVAILABLE ################
+# server to update for clustering -- remote.cache2.ccf(1102)  and remote.cache3.ccf(1103)
+jcs.auxiliary.RCluster=org.apache.jcs.auxiliary.remote.RemoteCacheFactory
+jcs.auxiliary.RCluster.attributes=org.apache.jcs.auxiliary.remote.RemoteCacheAttributes
+jcs.auxiliary.RCluster.attributes.RemoteTypeName=CLUSTER
+jcs.auxiliary.RCluster.attributes.RemoveUponRemotePut=false
+jcs.auxiliary.RCluster.attributes.ClusterServers=localhost:1102
+jcs.auxiliary.RCluster.attributes.GetOnly=false
+jcs.auxiliary.RCluster.attributes.LocalClusterConsistency=true
+
+
+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=@project_home@/raf/remote
+

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AbstractAuxiliaryCache.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AbstractAuxiliaryCache.java?rev=686881&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AbstractAuxiliaryCache.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AbstractAuxiliaryCache.java Mon Aug 18 15:11:14 2008
@@ -0,0 +1,140 @@
+package org.apache.jcs.auxiliary;
+
+import java.io.Serializable;
+
+import org.apache.jcs.engine.behavior.ICacheElement;
+import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.CacheEvent;
+import org.apache.jcs.engine.logging.behavior.ICacheEvent;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
+import org.apache.jcs.utils.serialization.StandardSerializer;
+
+/** This holds convenience methods used by most auxiliary caches. */
+public abstract class AbstractAuxiliaryCache
+    implements AuxiliaryCache
+{
+    /** Don't change. */
+    private static final long serialVersionUID = -1285708398502576617L;
+
+    /** An optional event logger */
+    protected ICacheEventLogger cacheEventLogger;
+
+    /** The serializer. Uses a standard serializer by default. */
+    protected IElementSerializer elementSerializer = new StandardSerializer();
+
+    /** If there is no event logger, we will return this event for all create calls. */
+    private static final ICacheEvent EMPTY_ICACHE_EVENT = new CacheEvent();
+
+    /**
+     * Logs an event if an event logger is configured.
+     * <p>
+     * @param item
+     * @param requesterId
+     */
+    protected ICacheEvent createICacheEvent( ICacheElement item, String eventName )
+    {
+        if ( cacheEventLogger == null )
+        {
+            return EMPTY_ICACHE_EVENT;
+        }
+        String diskLocation = getEventLoggingExtraInfo();
+        String regionName = null;
+        if ( item != null )
+        {
+            regionName = item.getCacheName();
+        }
+        return cacheEventLogger.createICacheEvent( getAuxiliaryCacheAttributes().getName(), regionName, eventName,
+                                                   diskLocation, item );
+    }
+
+    /**
+     * Logs an event if an event logger is configured.
+     * <p>
+     * @param cacheName
+     * @param key
+     * @param requesterId
+     */
+    protected ICacheEvent createICacheEvent( String regionName, Serializable key, String eventName )
+    {
+        if ( cacheEventLogger == null )
+        {
+            return EMPTY_ICACHE_EVENT;
+        }
+        String diskLocation = getEventLoggingExtraInfo();
+        return cacheEventLogger.createICacheEvent( getAuxiliaryCacheAttributes().getName(), regionName, eventName,
+                                                   diskLocation, key );
+
+    }
+
+    /**
+     * Logs an event if an event logger is configured.
+     * <p>
+     * @param item
+     * @param requesterId
+     */
+    protected void logICacheEvent( ICacheEvent cacheEvent )
+    {
+        if ( cacheEventLogger != null )
+        {
+            cacheEventLogger.logICacheEvent( cacheEvent );
+        }
+    }
+
+    /**
+     * 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 );
+        }
+    }
+    
+    /**
+     * Logs an event if an event logger is configured.
+     * <p>
+     * @param source
+     * @param eventName
+     * @param errorMessage
+     */
+    protected void logError( String source, String eventName, String errorMessage )
+    {
+        if ( cacheEventLogger != null )
+        {
+            cacheEventLogger.logError( source, eventName, errorMessage );
+        }
+    }    
+    
+    /**
+     * Gets the extra info for the event log.
+     * <p>
+     * @return IP, or disk location, etc.
+     */
+    public abstract String getEventLoggingExtraInfo();
+
+    /**
+     * Allows it to be injected.
+     * <p>
+     * @param cacheEventLogger
+     */
+    public void setCacheEventLogger( ICacheEventLogger cacheEventLogger )
+    {
+        this.cacheEventLogger = cacheEventLogger;
+    }
+
+    /**
+     * Allows you to inject a custom serializer. A good example would be a compressing standard
+     * serializer.
+     * <p>
+     * @param elementSerializer
+     */
+    public void setElementSerializer( IElementSerializer elementSerializer )
+    {
+        this.elementSerializer = elementSerializer;
+    }
+}

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCache.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCache.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCache.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCache.java Mon Aug 18 15:11:14 2008
@@ -23,8 +23,8 @@
 import java.util.Set;
 
 import org.apache.jcs.engine.behavior.ICache;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.stats.behavior.IStats;
 
 /**
@@ -32,7 +32,6 @@
  * ICache, but I anticipate that will change. For example, there will be a mechanism for determining
  * the type (disk/lateral/remote) of the auxiliary here -- and the existing getCacheType will be
  * removed from ICache.
- * @version $Id$
  */
 public interface AuxiliaryCache
     extends ICache

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheConfigurator.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheConfigurator.java?rev=686881&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheConfigurator.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheConfigurator.java Mon Aug 18 15:11:14 2008
@@ -0,0 +1,82 @@
+package org.apache.jcs.auxiliary;
+
+import java.util.Properties;
+
+import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
+import org.apache.jcs.utils.config.OptionConverter;
+import org.apache.jcs.utils.config.PropertySetter;
+
+/**
+ * Configuration util for auxiliary caches. I plan to move the auxiliary configuration from the
+ * composite cache configurator here.
+ */
+public class AuxiliaryCacheConfigurator
+{
+    /** .attributes */
+    final static String ATTRIBUTE_PREFIX = ".attributes";
+
+    /**
+     * jcs.auxiliary.NAME.cacheeventlogger=CLASSNAME
+     * <p>
+     * jcs.auxiliary.NAME.cacheeventlogger.attributes.CUSTOMPROPERTY=VALUE
+     */
+    final static String CACHE_EVENT_LOGGER_PREFIX = ".cacheeventlogger";
+
+    /**
+     * jcs.auxiliary.NAME.serializer=CLASSNAME
+     * <p>
+     * jcs.auxiliary.NAME.serializer.attributes.CUSTOMPROPERTY=VALUE
+     */
+    final static String SERIALIZER_PREFIX = ".serializer";
+
+    /**
+     * Parses the event logger config, if there is any for the auxiliary.
+     * <p>
+     * @param props
+     * @param auxPrefix - ex. AUXILIARY_PREFIX + auxName
+     * @return cacheEventLogger
+     */
+    public static ICacheEventLogger parseCacheEventLogger( Properties props, String auxPrefix )
+    {
+        ICacheEventLogger cacheEventLogger = null;
+
+        // auxFactory was not previously initialized.
+        String eventLoggerClassName = auxPrefix + CACHE_EVENT_LOGGER_PREFIX;
+        cacheEventLogger = (ICacheEventLogger) OptionConverter
+            .instantiateByKey( props, eventLoggerClassName, org.apache.jcs.engine.logging.behavior.ICacheEventLogger.class,
+                               null );
+        if ( cacheEventLogger != null )
+        {
+            String cacheEventLoggerAttributePrefix = auxPrefix + CACHE_EVENT_LOGGER_PREFIX + ATTRIBUTE_PREFIX;
+            PropertySetter.setProperties( cacheEventLogger, props, cacheEventLoggerAttributePrefix + "." );
+        }
+        return cacheEventLogger;
+    }
+
+    /**
+     * Parses the element config, if there is any for the auxiliary.
+     * <p>
+     * @param props
+     * @param auxPrefix - ex. AUXILIARY_PREFIX + auxName
+     * @return cacheEventLogger
+     */
+    public static IElementSerializer parseElementSerializer( Properties props, String auxPrefix )
+    {
+        // TODO take in the entire prop key
+
+        IElementSerializer elementSerializer = null;
+
+        // auxFactory was not previously initialized.
+        String elementSerializerClassName = auxPrefix + SERIALIZER_PREFIX;
+        elementSerializer = (IElementSerializer) OptionConverter
+            .instantiateByKey( props, elementSerializerClassName,
+                               org.apache.jcs.engine.behavior.IElementSerializer.class, null );
+        if ( elementSerializer != null )
+        {
+            String cacheEventLoggerAttributePrefix = auxPrefix + SERIALIZER_PREFIX + ATTRIBUTE_PREFIX;
+            PropertySetter.setProperties( elementSerializer, props, cacheEventLoggerAttributePrefix + "." );
+        }
+        return elementSerializer;
+    }
+}

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheFactory.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheFactory.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/AuxiliaryCacheFactory.java Mon Aug 18 15:11:14 2008
@@ -15,9 +15,9 @@
  * the License.
  */
 
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 
 /**
  * All auxiliary caches must have a factory that the cache configurator can use to create instances.

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCache.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCache.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCache.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCache.java Mon Aug 18 15:11:14 2008
@@ -31,6 +31,7 @@
 
 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.disk.behavior.IDiskCacheAttributes;
 import org.apache.jcs.engine.CacheConstants;
@@ -38,10 +39,8 @@
 import org.apache.jcs.engine.CacheInfo;
 import org.apache.jcs.engine.behavior.ICache;
 import org.apache.jcs.engine.behavior.ICacheElement;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICacheEventQueue;
 import org.apache.jcs.engine.behavior.ICacheListener;
-import org.apache.jcs.engine.behavior.IElementSerializer;
 import org.apache.jcs.engine.stats.StatElement;
 import org.apache.jcs.engine.stats.Stats;
 import org.apache.jcs.engine.stats.behavior.IStatElement;
@@ -62,6 +61,7 @@
  * Should it dispose itself?
  */
 public abstract class AbstractDiskCache
+    extends AbstractAuxiliaryCache
     implements AuxiliaryCache, Serializable
 {
     /** Don't change. */
@@ -89,20 +89,13 @@
      */
     protected ICacheEventQueue cacheEventQueue;
 
-    /**
-     * Indicates whether the cache is 'alive', defined as having been initialized, but not yet
-     * disposed.
-     */
+    /** Indicates whether the cache is 'alive': initialized, but not yet disposed. */
     protected boolean alive = false;
 
-    /**
-     * Every cache will have a name, subclasses must set this when they are initialized.
-     */
+    /** Every cache will have a name, subclasses must set this when they are initialized. */
     protected String cacheName;
 
-    /**
-     * DEBUG: Keeps a count of the number of purgatory hits for debug messages
-     */
+    /** DEBUG: Keeps a count of the number of purgatory hits for debug messages */
     protected int purgHits = 0;
 
     /**
@@ -111,12 +104,6 @@
      */
     private WriterPreferenceReadWriteLock removeAllLock = new WriterPreferenceReadWriteLock();
 
-    /** An optional event logger */
-    protected ICacheEventLogger cacheEventLogger;
-    
-    /** The serializer. */
-    protected IElementSerializer elementSerializer;    
-
     // ----------------------------------------------------------- constructors
 
     /**
@@ -218,7 +205,7 @@
             // Wrap the CacheElement in a PurgatoryElement
             PurgatoryElement pe = new PurgatoryElement( cacheElement );
 
-            // Indicates the the element is eligable to be spooled to disk,
+            // Indicates the the element is eligible to be spooled to disk,
             // this will remain true unless the item is pulled back into
             // memory.
             pe.setSpoolable( true );
@@ -564,27 +551,6 @@
     }
 
     /**
-     * Allows it to be injected.
-     * <p>
-     * @param cacheEventLogger
-     */
-    public void setCacheEventLogger( ICacheEventLogger cacheEventLogger )
-    {
-        this.cacheEventLogger = cacheEventLogger;
-    }
-
-    /**
-     * Allows you to inject a custom serializer. A good example would be a compressing standard
-     * serializer.
-     * <p>
-     * @param elementSerializer
-     */
-    public void setElementSerializer( IElementSerializer elementSerializer )
-    {
-        this.elementSerializer = elementSerializer;
-    }
-    
-    /**
      * Cache that implements the CacheListener interface, and calls appropriate methods in its
      * parent class.
      */
@@ -786,4 +752,20 @@
      */
     protected abstract void doDispose();
 
+    /**
+     * Gets the extra info for the event log.
+     * <p>
+     * @return disk location
+     */
+    public String getEventLoggingExtraInfo()
+    {
+        return getDiskLocation();
+    }
+
+    /**
+     * This is used by the event logging.
+     * <p>
+     * @return the location of the disk, either path or ip.
+     */
+    protected abstract String getDiskLocation();
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCacheAttributes.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCacheAttributes.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCacheAttributes.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCacheAttributes.java Mon Aug 18 15:11:14 2008
@@ -25,13 +25,14 @@
 
 /**
  * This has common attributes that any conceivable disk cache would need.
- * <p>
- * @author aaronsm
  */
 public abstract class AbstractDiskCacheAttributes
     extends AbstractAuxiliaryCacheAttributes
     implements IDiskCacheAttributes
 {
+    /** Don't change. */
+    private static final long serialVersionUID = 8306631920391711229L;
+
     /** path to disk */
     protected String diskPath;
 
@@ -41,7 +42,7 @@
     /** default to 5000 */
     protected int maxPurgatorySize = MAX_PURGATORY_SIZE_DEFUALT;
 
-    /** Default amount of time to allow for keypersistence on shutdown */
+    /** Default amount of time to allow for key persistence on shutdown */
     private static final int DEFAULT_shutdownSpoolTimeLimit = 60;
 
     /**

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDisk.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDisk.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDisk.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDisk.java Mon Aug 18 15:11:14 2008
@@ -497,4 +497,14 @@
         }
         return buf.toString();
     }
+    
+    /**
+     * This is used for debugging.
+     * <p>
+     * @return the file path.
+     */
+    protected String getFilePath()
+    {
+        return filepath;
+    }    
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCache.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCache.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCache.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCache.java Mon Aug 18 15:11:14 2008
@@ -474,7 +474,7 @@
      * Dispose of the disk cache in a background thread. Joins against this thread to put a cap on
      * the disposal time.
      * <p>
-     * @todo make dispose window configurable.
+     * TODO make dispose window configurable.
      */
     public void doDispose()
     {
@@ -726,4 +726,14 @@
 
         return stats;
     }
+    
+    /**
+     * This is used by the event logging.
+     * <p>
+     * @return the location of the disk, either path or ip.
+     */
+    protected String getDiskLocation()
+    {
+        return dataFile.getFilePath();
+    }    
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCacheFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCacheFactory.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCacheFactory.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/block/BlockDiskCacheFactory.java Mon Aug 18 15:11:14 2008
@@ -24,9 +24,9 @@
 import org.apache.jcs.auxiliary.AuxiliaryCache;
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.auxiliary.AuxiliaryCacheFactory;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 
 /**
  * Creates disk cache instances.

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDisk.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDisk.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDisk.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDisk.java Mon Aug 18 15:11:14 2008
@@ -308,4 +308,14 @@
         }
         raf.setLength( length );
     }
+    
+    /**
+     * This is used for debugging.
+     * <p>
+     * @return the file path.
+     */
+    protected String getFilePath()
+    {
+        return filepath;
+    }
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.java Mon Aug 18 15:11:14 2008
@@ -43,6 +43,8 @@
 import org.apache.jcs.engine.behavior.ICacheElement;
 import org.apache.jcs.engine.control.group.GroupAttrName;
 import org.apache.jcs.engine.control.group.GroupId;
+import org.apache.jcs.engine.logging.behavior.ICacheEvent;
+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;
@@ -86,7 +88,7 @@
 
     boolean isShutdownOptimizationEnabled = true;
 
-    /** are we currenlty optimizing the files */
+    /** are we currently optimizing the files */
     boolean isOptimizing = false;
 
     private int timesOptimized = 0;
@@ -114,7 +116,6 @@
     private long bytesFree = 0;
 
     private int hitCount = 0;
-       
 
     /**
      * Use this lock to synchronize reads and writes to the underlying storage mechansism.
@@ -410,6 +411,25 @@
      */
     public void doUpdate( ICacheElement ce )
     {
+        ICacheEvent cacheEvent = createICacheEvent( ce, ICacheEventLogger.UPDATE_EVENT );
+        try
+        {
+            processUpdate( ce );
+        }
+        finally
+        {
+            logICacheEvent( cacheEvent );
+        }
+    }
+
+    /**
+     * Update the disk cache. Called from the Queue. Makes sure the Item has not been retireved from
+     * purgatory while in queue for disk. Remove items from purgatory when they go to disk.
+     * <p>
+     * @param ce The ICacheElement to put to disk.
+     */
+    private void processUpdate( ICacheElement ce )
+    {
         if ( !alive )
         {
             log.error( logCacheName + "No longer alive; aborting put of key = " + ce.getKey() );
@@ -517,10 +537,29 @@
      */
     protected ICacheElement doGet( Serializable key )
     {
+        ICacheElement object = null;
+        ICacheEvent cacheEvent = createICacheEvent( cacheName, key, ICacheEventLogger.GET_EVENT );
+        try
+        {
+            object = processGet( key, object );
+        }
+        finally
+        {
+            logICacheEvent( cacheEvent );
+        }
+        return object;
+    }
+
+    /**
+     * @param key
+     * @return ICacheElement or null
+     * @see AbstractDiskCache#doGet
+     */
+    private ICacheElement processGet( Serializable key, ICacheElement object )
+    {
         if ( !alive )
         {
             log.error( logCacheName + "No longer alive so returning null for key = " + key );
-
             return null;
         }
 
@@ -529,7 +568,6 @@
             log.debug( logCacheName + "Trying to get from disk: " + key );
         }
 
-        ICacheElement object = null;
         try
         {
             storageLock.readLock().acquire();
@@ -556,7 +594,6 @@
         {
             log.error( logCacheName + "Failure getting from disk, key = " + key, e );
         }
-
         return object;
     }
 
@@ -636,7 +673,7 @@
     }
 
     /**
-     * Returns true if the removal was succesful; or false if there is nothing to remove. Current
+     * Returns true if the removal was successful; or false if there is nothing to remove. Current
      * implementation always result in a disk orphan.
      * <p>
      * @return true if at least one item was removed.
@@ -644,6 +681,26 @@
      */
     public boolean doRemove( Serializable key )
     {
+        ICacheEvent cacheEvent = createICacheEvent( cacheName, key, ICacheEventLogger.REMOVE_EVENT );
+        try
+        {
+            return processRemove( key );
+        }
+        finally
+        {
+            logICacheEvent( cacheEvent );
+        }
+    }
+
+    /**
+     * Returns true if the removal was successful; or false if there is nothing to remove. Current
+     * implementation always result in a disk orphan.
+     * <p>
+     * @return true if at least one item was removed.
+     * @param key
+     */
+    private boolean processRemove( Serializable key )
+    {
         if ( !alive )
         {
             log.error( logCacheName + "No longer alive so returning false for key = " + key );
@@ -689,7 +746,7 @@
             reset();
         }
 
-        // this increments the removecount.
+        // this increments the remove count.
         // there is no reason to call this if an item was not removed.
         if ( removed )
         {
@@ -811,6 +868,7 @@
      */
     public void doRemoveAll()
     {
+        ICacheEvent cacheEvent = createICacheEvent( cacheName, "all", ICacheEventLogger.REMOVEALL_EVENT );
         try
         {
             reset();
@@ -820,6 +878,10 @@
             log.error( logCacheName + "Problem removing all.", e );
             reset();
         }
+        finally
+        {
+            logICacheEvent( cacheEvent );
+        }
     }
 
     /**
@@ -914,27 +976,35 @@
      * Dispose of the disk cache in a background thread. Joins against this thread to put a cap on
      * the disposal time.
      * <p>
-     * @todo make dispose window configurable.
+     * TODO make dispose window configurable.
      */
     public void doDispose()
     {
-        Runnable disR = new Runnable()
+        ICacheEvent cacheEvent = createICacheEvent( cacheName, "none", ICacheEventLogger.DISPOSE_EVENT );
+        try
         {
-            public void run()
+            Runnable disR = new Runnable()
             {
-                disposeInternal();
+                public void run()
+                {
+                    disposeInternal();
+                }
+            };
+            Thread t = new Thread( disR, "IndexedDiskCache-DisposalThread" );
+            t.start();
+            // wait up to 60 seconds for dispose and then quit if not done.
+            try
+            {
+                t.join( 60 * 1000 );
+            }
+            catch ( InterruptedException ex )
+            {
+                log.error( logCacheName + "Interrupted while waiting for disposal thread to finish.", ex );
             }
-        };
-        Thread t = new Thread( disR, "IndexedDiskCache-DisposalThread" );
-        t.start();
-        // wait up to 60 seconds for dispose and then quit if not done.
-        try
-        {
-            t.join( 60 * 1000 );
         }
-        catch ( InterruptedException ex )
+        finally
         {
-            log.error( logCacheName + "Interrupted while waiting for disposal thread to finish.", ex );
+            logICacheEvent( cacheEvent );
         }
     }
 
@@ -1077,21 +1147,17 @@
     /**
      * File optimization is handled by this method. It works as follows:
      * <ol>
-     * <li>Shutdown recycling and turn on queuing of puts. </li>
-     * <li>Take a snapshot of the current descriptors. If there are any removes, ignore them, as
-     * they will be compacted during the next optimization.</li>
-     * <li>Optimize the snapshot. For each descriptor:
+     * <li>Shutdown recycling and turn on queuing of puts. </li> <li>Take a snapshot of the current
+     * descriptors. If there are any removes, ignore them, as they will be compacted during the next
+     * optimization.</li> <li>Optimize the snapshot. For each descriptor:
      * <ol>
-     * <li>Obtain the write-lock.</li>
-     * <li>Shift the element on the disk, in order to compact out the free space. </li>
-     * <li>Release the write-lock. This allows elements to still be accessible during optimization.</li>
+     * <li>Obtain the write-lock.</li> <li>Shift the element on the disk, in order to compact out
+     * the free space. </li> <li>Release the write-lock. This allows elements to still be accessible
+     * during optimization.</li>
      * </ol>
-     * <li>Obtain the write-lock.</li>
-     * <li>All queued puts are made at the end of the file. Optimize these under a single
-     * write-lock.</li>
-     * <li>Truncate the file.</li>
-     * <li>Release the write-lock.</li>
-     * <li>Restore system to standard operation.</li>
+     * <li>Obtain the write-lock.</li> <li>All queued puts are made at the end of the file. Optimize
+     * these under a single write-lock.</li> <li>Truncate the file.</li> <li>Release the write-lock.
+     * </li> <li>Restore system to standard operation.</li>
      * </ol>
      */
     protected void optimizeFile()
@@ -1519,6 +1585,16 @@
     }
 
     /**
+     * This is used by the event logging.
+     * <p>
+     * @return the location of the disk, either path or ip.
+     */
+    protected String getDiskLocation()
+    {
+        return dataFile.getFilePath();
+    }
+
+    /**
      * Compares IndexedDiskElementDescriptor based on their position.
      * <p>
      */
@@ -1551,8 +1627,8 @@
     }
 
     /**
-     * Class for recylcing and lru. This implments the LRU overflow callback, so we can add items to
-     * the recycle bin.
+     * Class for recylcing and lru. This implements the LRU overflow callback, so we can add items
+     * to the recycle bin.
      */
     public class LRUMap
         extends LRUMapJCS
@@ -1582,7 +1658,7 @@
         }
 
         /**
-         * This is called when the may key size is reaced. The least recently used item will be
+         * This is called when the may key size is reached. The least recently used item will be
          * passed here. We will store the position and size of the spot on disk in the recycle bin.
          * <p>
          * @param key

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCacheFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCacheFactory.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCacheFactory.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCacheFactory.java Mon Aug 18 15:11:14 2008
@@ -24,9 +24,9 @@
 import org.apache.jcs.auxiliary.AuxiliaryCache;
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.auxiliary.AuxiliaryCacheFactory;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 
 /**
  * Creates disk cache instances.

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskElementDescriptor.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskElementDescriptor.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskElementDescriptor.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskElementDescriptor.java Mon Aug 18 15:11:14 2008
@@ -66,6 +66,9 @@
         this.len = len;
     }
 
+    /**
+     * @return debug string
+     */
     public String toString()
     {
         StringBuffer buf = new StringBuffer();

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCache.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCache.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCache.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCache.java Mon Aug 18 15:11:14 2008
@@ -1023,4 +1023,14 @@
     {
         return this.getStats();
     }
+    
+    /**
+     * This is used by the event logging.
+     * <p>
+     * @return the location of the disk, either path or ip.
+     */
+    protected String getDiskLocation()
+    {
+        return this.jdbcDiskCacheAttributes.getUrl();
+    }    
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheFactory.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheFactory.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheFactory.java Mon Aug 18 15:11:14 2008
@@ -22,9 +22,9 @@
 import org.apache.jcs.auxiliary.AuxiliaryCache;
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.auxiliary.AuxiliaryCacheFactory;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 
 /**
  * This factory should create mysql disk caches.

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheManagerAbstractTemplate.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheManagerAbstractTemplate.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheManagerAbstractTemplate.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheManagerAbstractTemplate.java Mon Aug 18 15:11:14 2008
@@ -33,35 +33,32 @@
 
 /**
  * This class serves as an abstract template for JDBCDiskCache Manager. The MySQL JDBC Disk Cache
- * needs many of the same features as the generic maanger.
+ * needs many of the same features as the generic manager.
  * <p>
  * @author Aaron Smuts
  */
 public abstract class JDBCDiskCacheManagerAbstractTemplate
     implements AuxiliaryCacheManager
 {
+    /** Don't change. */
+    private static final long serialVersionUID = 218557927622128905L;
+
     /** The logger. */
     private static final Log log = LogFactory.getLog( JDBCDiskCacheManagerAbstractTemplate.class );
 
-    /**
-     * Incremented on getIntance, decremented on release.
-     */
+    /** Incremented on getIntance, decremented on release.  */
     protected static int clients;
 
-    /**
-     * A map of JDBCDiskCache objects to region names.
-     */
+    /** A map of JDBCDiskCache objects to region names.  */
     protected static Hashtable caches = new Hashtable();
 
     /**
      * A map of TableState objects to table names. Each cache has a table state object, which is
-     * used to determin if any long processes such as deletes or optimizations are running.
+     * used to determine if any long processes such as deletes or optimizations are running.
      */
     protected static Hashtable tableStates = new Hashtable();
 
-    /**
-     * The background disk shrinker, one for all regions.
-     */
+    /** The background disk shrinker, one for all regions.  */
     private ClockDaemon shrinkerDaemon;
 
     /**
@@ -203,9 +200,7 @@
     }
 
     /**
-     * Allows us to set the daemon status on the clockdaemon
-     * <p>
-     * @author aaronsm
+     * Allows us to set the daemon status on the clock-daemon
      */
     class MyThreadFactory
         implements ThreadFactory

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java Mon Aug 18 15:11:14 2008
@@ -34,9 +34,9 @@
 import org.apache.jcs.auxiliary.AuxiliaryCacheFactory;
 import org.apache.jcs.auxiliary.disk.jdbc.JDBCDiskCacheAttributes;
 import org.apache.jcs.auxiliary.disk.jdbc.JDBCDiskCacheManager;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 
 /**
  * This factory should create mysql disk caches.

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/mysql/MySQLDiskCacheFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/mysql/MySQLDiskCacheFactory.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/mysql/MySQLDiskCacheFactory.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/mysql/MySQLDiskCacheFactory.java Mon Aug 18 15:11:14 2008
@@ -22,9 +22,9 @@
 import org.apache.jcs.auxiliary.AuxiliaryCache;
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.auxiliary.AuxiliaryCacheFactory;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 
 /**
  * This factory should create mysql disk caches.

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheAbstractFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheAbstractFactory.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheAbstractFactory.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheAbstractFactory.java Mon Aug 18 15:11:14 2008
@@ -23,9 +23,9 @@
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.auxiliary.AuxiliaryCacheFactory;
 import org.apache.jcs.auxiliary.lateral.behavior.ILateralCacheAttributes;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 
 
 /**

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWait.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWait.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWait.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWait.java Mon Aug 18 15:11:14 2008
@@ -39,9 +39,9 @@
 import org.apache.jcs.engine.CacheConstants;
 import org.apache.jcs.engine.CacheEventQueueFactory;
 import org.apache.jcs.engine.behavior.ICacheElement;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICacheEventQueue;
 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;

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWaitFacade.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWaitFacade.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWaitFacade.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/LateralCacheNoWaitFacade.java Mon Aug 18 15:11:14 2008
@@ -36,9 +36,9 @@
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.auxiliary.lateral.behavior.ILateralCacheAttributes;
 import org.apache.jcs.engine.behavior.ICacheElement;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICacheType;
 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;

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/LateralTCPCacheFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/LateralTCPCacheFactory.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/LateralTCPCacheFactory.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/lateral/socket/tcp/LateralTCPCacheFactory.java Mon Aug 18 15:11:14 2008
@@ -34,9 +34,9 @@
 import org.apache.jcs.auxiliary.lateral.socket.tcp.discovery.UDPDiscoveryManager;
 import org.apache.jcs.auxiliary.lateral.socket.tcp.discovery.UDPDiscoveryService;
 import org.apache.jcs.engine.behavior.ICache;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 
 /**
  * Constructs a LateralCacheNoWaitFacade for the given configuration. Each lateral service / local

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCache.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCache.java?rev=686881&r1=686880&r2=686881&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 Aug 18 15:11:14 2008
@@ -33,6 +33,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.jcs.auxiliary.AbstractAuxiliaryCache;
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheAttributes;
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheClient;
@@ -41,16 +42,16 @@
 import org.apache.jcs.engine.CacheConstants;
 import org.apache.jcs.engine.behavior.ICacheElement;
 import org.apache.jcs.engine.behavior.ICacheElementSerialized;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 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.logging.behavior.ICacheEvent;
+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;
 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;
 
@@ -63,14 +64,19 @@
  * failover recovery when an error is encountered.
  */
 public class RemoteCache
+    extends AbstractAuxiliaryCache
     implements IRemoteCacheClient
 {
+    /** Don't change. */
     private static final long serialVersionUID = -5329231850422826460L;
 
+    /** The logger. */
     private final static Log log = LogFactory.getLog( RemoteCache.class );
 
+    /** The cacheName */
     final String cacheName;
 
+    /** The configuration values. */
     private IRemoteCacheAttributes irca;
 
     /** This is a handle on the remote server. In zombie mode it is replaced with a balking facade. */
@@ -84,12 +90,6 @@
 
     private boolean usePoolForGet = false;
 
-    /** The object serializer */
-    private IElementSerializer elementSerializer = new StandardSerializer();
-
-    /** An optional event logger */
-    private ICacheEventLogger cacheEventLogger;
-
     /**
      * 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
@@ -134,12 +134,23 @@
             }
         }
 
+        configureCustomSocketFactory();
+    }
+
+    /** Configure a custom socket factory to set the timeout value. */
+    private void configureCustomSocketFactory()
+    {
         try
         {
             // Don't set a socket factory if the setting is -1
             if ( irca.getRmiSocketFactoryTimeoutMillis() > 0 )
             {
-                // TODO make configurable.
+                if ( log.isInfoEnabled() )
+                {
+                    log.info( "RmiSocketFactoryTimeoutMillis [" + irca.getRmiSocketFactoryTimeoutMillis() + "]. "
+                        + " Configuring a custom socket factory." );
+                }
+
                 // use this socket factory to add a timeout.
                 RMISocketFactory.setSocketFactory( new RMISocketFactory()
                 {
@@ -164,7 +175,8 @@
         {
             // TODO change this so that we only try to do it once. Otherwise we
             // Generate errors for each region on construction.
-            log.info( e.getMessage() );
+            log.info( "Could not create new custom socket factory. " + e.getMessage() + " Factory in use = "
+                + RMISocketFactory.getSocketFactory() );
         }
     }
 
@@ -199,14 +211,14 @@
     public void update( ICacheElement ce )
         throws IOException
     {
-        logEventStart( ce, ICacheEventLogger.UPDATE_EVENT );
+        ICacheEvent cacheEvent = createICacheEvent( ce, ICacheEventLogger.UPDATE_EVENT );
         try
         {
             processUpdate( ce );
         }
         finally
         {
-            logEventFinish( ce, ICacheEventLogger.UPDATE_EVENT );
+            logICacheEvent( cacheEvent );
         }
     }
 
@@ -244,7 +256,7 @@
             catch ( Exception ex )
             {
                 // event queue will wait and retry
-                handleException( ex, "Failed to put [" + ce.getKey() + "] to " + ce.getCacheName() );
+                handleException( ex, "Failed to put [" + ce.getKey() + "] to " + ce.getCacheName(), ICacheEventLogger.UPDATE_EVENT );
             }
         }
         else
@@ -273,14 +285,14 @@
         throws IOException
     {
         ICacheElement retVal = null;
-        logEventStart( cacheName, key, ICacheEventLogger.GET_EVENT );
+        ICacheEvent cacheEvent = createICacheEvent( cacheName, key, ICacheEventLogger.GET_EVENT );
         try
         {
             retVal = processGet( key, retVal );
         }
         finally
         {
-            logEventFinish( retVal, ICacheEventLogger.GET_EVENT );
+            logICacheEvent( cacheEvent );
         }
         return retVal;
     }
@@ -327,7 +339,7 @@
         }
         catch ( Exception ex )
         {
-            handleException( ex, "Failed to get [" + key + "] from [" + cacheName + "]" );
+            handleException( ex, "Failed to get [" + key + "] from [" + cacheName + "]", ICacheEventLogger.GET_EVENT );
         }
         return retVal;
     }
@@ -343,14 +355,14 @@
     public Map getMultiple( Set keys )
         throws IOException
     {
-        logEventStart( cacheName, (Serializable) keys, ICacheEventLogger.GETMULTIPLE_EVENT );
+        ICacheEvent cacheEvent = createICacheEvent( cacheName, (Serializable) keys, ICacheEventLogger.GETMULTIPLE_EVENT );
         try
         {
             return processGetMultiple( keys );
         }
         finally
         {
-            logEventEnd( cacheName, (Serializable) keys, ICacheEventLogger.GETMULTIPLE_EVENT );
+            logICacheEvent( cacheEvent );
         }
     }
 
@@ -468,14 +480,14 @@
     public boolean remove( Serializable key )
         throws IOException
     {
-        logEventStart( cacheName, key, ICacheEventLogger.REMOVE_EVENT );
+        ICacheEvent cacheEvent = createICacheEvent( cacheName, key, ICacheEventLogger.REMOVE_EVENT );
         try
         {
             return processRemove( key );
         }
         finally
         {
-            logEventEnd( cacheName, key, ICacheEventLogger.REMOVE_EVENT );
+            logICacheEvent( cacheEvent );
         }
     }
 
@@ -502,7 +514,7 @@
             }
             catch ( Exception ex )
             {
-                handleException( ex, "Failed to remove " + key + " from " + cacheName );
+                handleException( ex, "Failed to remove " + key + " from " + cacheName, ICacheEventLogger.REMOVE_EVENT );
             }
             return true;
         }
@@ -518,7 +530,7 @@
     public void removeAll()
         throws IOException
     {
-        logEventStart( cacheName, "all", ICacheEventLogger.REMOVEALL_EVENT );
+        ICacheEvent cacheEvent = createICacheEvent( cacheName, "all", ICacheEventLogger.REMOVEALL_EVENT );
         try
         {
             if ( !this.irca.getGetOnly() )
@@ -529,13 +541,13 @@
                 }
                 catch ( Exception ex )
                 {
-                    handleException( ex, "Failed to remove all from " + cacheName );
+                    handleException( ex, "Failed to remove all from " + cacheName, ICacheEventLogger.REMOVEALL_EVENT );
                 }
             }
         }
         finally
         {
-            logEventEnd( cacheName, "all", ICacheEventLogger.REMOVEALL_EVENT );
+            logICacheEvent( cacheEvent );
         }
     }
 
@@ -547,7 +559,7 @@
     public void dispose()
         throws IOException
     {
-        logEventStart( cacheName, "none", ICacheEventLogger.DISPOSE_EVENT );
+        ICacheEvent cacheEvent = createICacheEvent( cacheName, "none", ICacheEventLogger.DISPOSE_EVENT );
         try
         {
             if ( log.isInfoEnabled() )
@@ -561,12 +573,12 @@
             catch ( Exception ex )
             {
                 log.error( "Couldn't dispose", ex );
-                handleException( ex, "Failed to dispose [" + cacheName + "]" );
+                handleException( ex, "Failed to dispose [" + cacheName + "]", ICacheEventLogger.DISPOSE_EVENT );
             }
         }
         finally
         {
-            logEventEnd( cacheName, "none", ICacheEventLogger.DISPOSE_EVENT );
+            logICacheEvent( cacheEvent );
         }
     }
 
@@ -700,7 +712,7 @@
             {
                 try
                 {
-                    handleException( e, "Problem propagating events from Zombie Queue to new Remote Service." );
+                    handleException( e, "Problem propagating events from Zombie Queue to new Remote Service.", "fixCache" );
                 }
                 catch ( IOException e1 )
                 {
@@ -721,12 +733,15 @@
      * <p>
      * @param ex
      * @param msg
+     * @param eventName
      * @throws IOException
      */
-    private void handleException( Exception ex, String msg )
+    private void handleException( Exception ex, String msg, String eventName )
         throws IOException
     {
         String message = "Disabling remote cache due to error: " + msg;
+        
+        logError( cacheName, "", message );
         log.error( message, ex );
 
         // we should not switch if the existing is a zombie.
@@ -873,74 +888,17 @@
     }
 
     /**
-     * Logs an event if an event logger is configured.
-     * <p>
-     * @param item
-     * @param requesterId
-     */
-    private void logEventStart( ICacheElement item, String eventName )
-    {
-        if ( cacheEventLogger != null )
-        {
-            String ipAddress = getIPAddressForService();
-            cacheEventLogger.logStartICacheEvent( "RemoteCacheClient", item.getCacheName(), eventName, ipAddress, item );
-        }
-    }
-
-    /**
-     * Logs an event if an event logger is configured.
-     * <p>
-     * @param cacheName
-     * @param key
-     * @param requesterId
-     */
-    private void logEventStart( String cacheName, Serializable key, String eventName )
-    {
-        if ( cacheEventLogger != null )
-        {
-            String ipAddress = getIPAddressForService();
-            cacheEventLogger.logStartICacheEvent( "RemoteCacheClient", cacheName, eventName, ipAddress, key );
-        }
-    }
-
-    /**
-     * Logs an event if an event logger is configured.
+     * Gets the extra info for the event log.
      * <p>
-     * @param item
-     * @param requesterId
+     * @return disk location
      */
-    private void logEventFinish( ICacheElement item, String eventName )
+    public String getEventLoggingExtraInfo()
     {
-        if ( cacheEventLogger != null )
-        {
-            String ipAddress = getIPAddressForService();
-            String cacheName = null;
-            if ( item != null )
-            {
-                cacheName = item.getCacheName();
-            }
-            cacheEventLogger.logEndICacheEvent( "RemoteCacheClient", cacheName, eventName, ipAddress, item );
-        }
-    }
-
-    /**
-     * Logs an event if an event logger is configured.
-     * <p>
-     * @param cacheName
-     * @param key
-     * @param requesterId
-     */
-    private void logEventEnd( String cacheName, Serializable key, String eventName )
-    {
-        if ( cacheEventLogger != null )
-        {
-            String ipAddress = getIPAddressForService();
-            cacheEventLogger.logEndICacheEvent( "RemoteCacheClient", cacheName, eventName, ipAddress, key );
-        }
+        return getIPAddressForService();
     }
 
     /**
-     * Ip address for the service, if one is stored.
+     * IP address for the service, if one is stored.
      * <p>
      * Protected for testing.
      * <p>

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFactory.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFactory.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFactory.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFactory.java Mon Aug 18 15:11:14 2008
@@ -29,9 +29,9 @@
 import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
 import org.apache.jcs.auxiliary.AuxiliaryCacheFactory;
 import org.apache.jcs.engine.behavior.ICache;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 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

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFailoverRunner.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFailoverRunner.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFailoverRunner.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheFailoverRunner.java Mon Aug 18 15:11:14 2008
@@ -23,9 +23,9 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.jcs.engine.CacheConstants;
 import org.apache.jcs.engine.behavior.ICache;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICompositeCacheManager;
 import org.apache.jcs.engine.behavior.IElementSerializer;
+import org.apache.jcs.engine.logging.behavior.ICacheEventLogger;
 
 /**
  * The RemoteCacheFailoverRunner tries to establish a connection with a failover

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheManager.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheManager.java?rev=686881&r1=686880&r2=686881&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 Aug 18 15:11:14 2008
@@ -36,11 +36,11 @@
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheObserver;
 import org.apache.jcs.auxiliary.remote.behavior.IRemoteCacheService;
 import org.apache.jcs.engine.behavior.ICache;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 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;
 
 /**
  * An instance of RemoteCacheManager corresponds to one remote connection of a specific host and

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWait.java
URL: http://svn.apache.org/viewvc/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWait.java?rev=686881&r1=686880&r2=686881&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWait.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/remote/RemoteCacheNoWait.java Mon Aug 18 15:11:14 2008
@@ -31,6 +31,7 @@
 
 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.auxiliary.remote.behavior.IRemoteCacheClient;
@@ -39,17 +40,16 @@
 import org.apache.jcs.engine.CacheConstants;
 import org.apache.jcs.engine.CacheEventQueueFactory;
 import org.apache.jcs.engine.behavior.ICacheElement;
-import org.apache.jcs.engine.behavior.ICacheEventLogger;
 import org.apache.jcs.engine.behavior.ICacheEventQueue;
 import org.apache.jcs.engine.behavior.ICacheType;
-import org.apache.jcs.engine.behavior.IElementSerializer;
 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;
 
 /**
- * The RemoteCacheNoWait wraps the RemoteCacheClient.  The client holds a handle on the RemoteCacheService.
+ * The RemoteCacheNoWait wraps the RemoteCacheClient. The client holds a handle on the
+ * RemoteCacheService.
  * <p>
  * Used to queue up update requests to the underlying cache. These requests will be processed in
  * their order of arrival via the cache event queue processor.
@@ -68,6 +68,7 @@
  * complicated worker queues.
  */
 public class RemoteCacheNoWait
+    extends AbstractAuxiliaryCache
     implements AuxiliaryCache
 {
     /** For serialization. Don't change. */
@@ -94,12 +95,6 @@
     /** how many times put has been called. */
     private int putCount = 0;
 
-    /** An optional event logger */
-    private ICacheEventLogger cacheEventLogger;
-
-    /** The serializer. */
-    private IElementSerializer elementSerializer;
-
     /**
      * Constructs with the given remote cache, and fires up an event queue for asynchronous
      * processing.
@@ -414,27 +409,6 @@
     }
 
     /**
-     * Allows it to be injected.
-     * <p>
-     * @param cacheEventLogger
-     */
-    public void setCacheEventLogger( ICacheEventLogger cacheEventLogger )
-    {
-        this.cacheEventLogger = cacheEventLogger;
-    }
-
-    /**
-     * Allows you to inject a custom serializer. A good example would be a compressing standard
-     * serializer.
-     * <p>
-     * @param elementSerializer
-     */
-    public void setElementSerializer( IElementSerializer elementSerializer )
-    {
-        this.elementSerializer = elementSerializer;
-    }
-
-    /**
      * Returns the stats and the cache.toString().
      * <p>
      * (non-Javadoc)
@@ -533,4 +507,14 @@
 
         return stats;
     }
+
+    /**
+     * this won't be called since we don't do ICache logging here.
+     * <p>
+     * @return String
+     */
+    public String getEventLoggingExtraInfo()
+    {
+        return "Remote Cache No Wait";
+    }
 }



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