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

svn commit: r234395 - in /jakarta/jcs/trunk: auxiliary-builds/jdk14/src/test-conf/ src/java/org/apache/jcs/engine/behavior/ src/java/org/apache/jcs/utils/props/ src/test-conf/ src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/ src/test/org/apache/jc...

Author: asmuts
Date: Sun Aug 21 17:20:09 2005
New Revision: 234395

URL: http://svn.apache.org/viewcvs?rev=234395&view=rev
Log:
Properties are loaded using the current thread's classloader by default.
When a config file is not found a descriptive error message is printed to the logs.
Made an interface for the cache manager to improve testability.
Auxiliaries are assigned a cache manager and the lateral TCP has a settable source id, so it can be tested.
Increased JGroups version to 2.2.8.  It works again.  The old JGroups didn't work for unknown reasons.
Cleaned up some javadocs.
Improved info level logging for TCP lateral.
Added configuration properties xdocs for TCP lateral, indexed disk, and memory cache.
Created concurrent tests for TCP lateral to verify correct region behavior.
Made BDBJE work with new auxiliary changes.
UDP discovery no longer activates if the lateral type isn't TCP.
Reduced lateral get timeout to 1 second, rather than 10.  We still need to disable gets when a timeout occurs.



Added:
    jakarta/jcs/trunk/auxiliary-builds/jdk14/src/test-conf/log4j.properties
    jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICompositeCacheManager.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/utils/props/
    jakarta/jcs/trunk/src/java/org/apache/jcs/utils/props/PropertyLoader.java
    jakarta/jcs/trunk/src/test-conf/TestSystemPropertyUsage.ccf
    jakarta/jcs/trunk/src/test-conf/TestTCPLateralCacheConcurrent.ccf
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentForDeadLock.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentRandom.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/engine/TestSystemPropertyUsage.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/engine/control/CompositeCacheManagerMockImpl.java
    jakarta/jcs/trunk/tempbuild/jcs-1.2.6.6.jar   (with props)
    jakarta/jcs/trunk/tempbuild/jcs-jdk14-ext-1.2.6.6.jar   (with props)
    jakarta/jcs/trunk/xdocs/IndexedDiskCacheProperties.xml
    jakarta/jcs/trunk/xdocs/LateralTCPProperties.xml
    jakarta/jcs/trunk/xdocs/RegionProperties.xml
    jakarta/jcs/trunk/xdocs/changes.xml

Added: jakarta/jcs/trunk/auxiliary-builds/jdk14/src/test-conf/log4j.properties
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/auxiliary-builds/jdk14/src/test-conf/log4j.properties?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/auxiliary-builds/jdk14/src/test-conf/log4j.properties (added)
+++ jakarta/jcs/trunk/auxiliary-builds/jdk14/src/test-conf/log4j.properties Sun Aug 21 17:20:09 2005
@@ -0,0 +1,24 @@
+log4j.rootCategory=INFO, stdout, logfile
+
+log4j.category.org.apache.jcs=INFO
+log4j.category.org.apache.jcs.config=INFO
+log4j.category.org.apache.jcs.engine.CacheEventQueueFactory=DEBUG
+log4j.category.org.apache.jcs.auxiliary.disk=INFO
+log4j.category.org.apache.jcs.auxiliary.disk.bdbje=DEBUG
+log4j.category.org.apache.jcs.auxiliary.remote=INFO
+log4j.category.org.apache.jcs.auxiliary.lateral=INFO
+log4j.category.org.apache.jcs.utils.threadpool=INFO
+
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d %p [%c] - <%m>%n
+
+log4j.appender.logfile=org.apache.log4j.RollingFileAppender
+log4j.appender.logfile.File=target/test-sandbox/logs/jcs.log
+log4j.appender.logfile.MaxFileSize=2MB
+# Keep three backup files
+log4j.appender.logfile.MaxBackupIndex=3
+log4j.appender.logfile.layout=org.apache.log4j.PatternLayout
+#Pattern to output : date priority [category] - <message>line_separator
+log4j.appender.logfile.layout.ConversionPattern=%d %p [%c] - <%m>%n
+

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICompositeCacheManager.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICompositeCacheManager.java?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICompositeCacheManager.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/engine/behavior/ICompositeCacheManager.java Sun Aug 21 17:20:09 2005
@@ -0,0 +1,30 @@
+package org.apache.jcs.engine.behavior;
+
+/*
+ * Copyright 2001-2004 The Apache Software Foundation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License")
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.jcs.engine.control.CompositeCache;
+
+/**
+ * I need the interface so I can plug in mock managers for testing.
+ * 
+ * @author Aaron Smuts
+ */
+public interface ICompositeCacheManager
+{
+    /** Gets the cache attribute of the CacheHub object */
+    public abstract CompositeCache getCache( String cacheName );
+}
\ No newline at end of file

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/utils/props/PropertyLoader.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/utils/props/PropertyLoader.java?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/utils/props/PropertyLoader.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/utils/props/PropertyLoader.java Sun Aug 21 17:20:09 2005
@@ -0,0 +1,161 @@
+package org.apache.jcs.utils.props;
+
+/*
+ * Copyright 2001-2004 The Apache Software Foundation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License")
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.InputStream;
+import java.util.Properties;
+
+/**
+ * I modified this class to work with .ccf files in particular. I also removed
+ * the resource bundle functionality.
+ * 
+ * A simple class for loading java.util.Properties backed by .ccf files deployed
+ * as classpath resources. See individual methods for details.
+ * <p>
+ * The original source is from:
+ * 
+ * @author (C) <a
+ *         href="http://www.javaworld.com/columns/jw-qna-index.shtml">Vlad
+ *         Roubtsov </a>, 2003
+ *  
+ */
+public abstract class PropertyLoader
+{
+
+    private static final boolean THROW_ON_LOAD_FAILURE = true;
+
+    private static final String SUFFIX = ".ccf";
+
+    /**
+     * Looks up a resource named 'name' in the classpath. The resource must map
+     * to a file with .ccf extention. The name is assumed to be absolute and can
+     * use either "/" or "." for package segment separation with an optional
+     * leading "/" and optional ".ccf" suffix.
+     * <p>
+     * The suffix ".ccf" will be appended if it is not set.
+     * 
+     * Thus, the following names refer to the same resource:
+     * 
+     * <pre>
+     * 
+     *  
+     *   
+     *    
+     *     some.pkg.Resource
+     *     some.pkg.Resource.ccf
+     *     some/pkg/Resource
+     *     some/pkg/Resource.ccf
+     *     /some/pkg/Resource
+     *     /some/pkg/Resource.ccf
+     *     
+     *    
+     *   
+     *  
+     * </pre>
+     * 
+     * @param name
+     *            classpath resource name [may not be null]
+     * @param loader
+     *            classloader through which to load the resource [null is
+     *            equivalent to the application loader]
+     * 
+     * @return resource converted to java.util.properties [may be null if the
+     *         resource was not found and THROW_ON_LOAD_FAILURE is false]
+     * @throws IllegalArgumentException
+     *             if the resource was not found and THROW_ON_LOAD_FAILURE is
+     *             true
+     */
+    public static Properties loadProperties( String name, ClassLoader loader )
+    {
+        if ( name == null )
+            throw new IllegalArgumentException( "null input: name" );
+
+        if ( name.startsWith( "/" ) )
+        {
+            name = name.substring( 1 );
+        }
+
+        if (name.endsWith (SUFFIX))
+        {            
+            name = name.substring (0, name.length () - SUFFIX.length ());
+        }
+        
+        Properties result = null;
+
+        InputStream in = null;
+        try
+        {
+            if ( loader == null )
+            {                
+                loader = ClassLoader.getSystemClassLoader();
+            }
+
+            name = name.replace( '.', '/' );
+
+            if ( !name.endsWith( SUFFIX ) )
+            {
+                name = name.concat( SUFFIX );
+            }
+
+            // returns null on lookup failures:
+            in = loader.getResourceAsStream( name );
+            if ( in != null )
+            {
+                result = new Properties();
+                result.load( in ); // can throw IOException
+            }
+        }
+        catch ( Exception e )
+        {
+            result = null;
+        }
+        finally
+        {
+            if ( in != null )
+                try
+                {
+                    in.close();
+                }
+                catch ( Throwable ignore )
+                {
+                }
+        }
+
+        if ( THROW_ON_LOAD_FAILURE && ( result == null ) )
+        {
+            throw new IllegalArgumentException( "could not load [" + name + "]" + " as " + "a classloader resource" );
+        }
+
+        return result;
+    }
+
+    /**
+     * A convenience overload of {@link #loadProperties(String, ClassLoader)}
+     * that uses the current thread's context classloader. A better strategy
+     * would be to use techniques shown in
+     * http://www.javaworld.com/javaworld/javaqa/2003-06/01-qa-0606-load.html
+     */
+    public static Properties loadProperties( final String name )
+    {
+        return loadProperties( name, Thread.currentThread().getContextClassLoader() );
+    }
+
+    private PropertyLoader()
+    {
+    } // this class is not extentible
+
+}

Added: jakarta/jcs/trunk/src/test-conf/TestSystemPropertyUsage.ccf
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test-conf/TestSystemPropertyUsage.ccf?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test-conf/TestSystemPropertyUsage.ccf (added)
+++ jakarta/jcs/trunk/src/test-conf/TestSystemPropertyUsage.ccf Sun Aug 21 17:20:09 2005
@@ -0,0 +1,4 @@
+jcs.default=
+jcs.default.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.default.cacheattributes.MaxObjects=10
+jcs.default.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache

Added: jakarta/jcs/trunk/src/test-conf/TestTCPLateralCacheConcurrent.ccf
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test-conf/TestTCPLateralCacheConcurrent.ccf?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test-conf/TestTCPLateralCacheConcurrent.ccf (added)
+++ jakarta/jcs/trunk/src/test-conf/TestTCPLateralCacheConcurrent.ccf Sun Aug 21 17:20:09 2005
@@ -0,0 +1,15 @@
+jcs.default=LTCP
+jcs.default.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.default.cacheattributes.MaxObjects=10000
+jcs.default.cacheattributes.MemoryCacheName=org.apache.jcs.engine.memory.lru.LRUMemoryCache
+
+##### AUXILIARY CACHES
+# simple Lateral TCP auxiliary
+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=
+jcs.auxiliary.LTCP.attributes.TcpListenerPort=1102
+jcs.auxiliary.LTCP.attributes.PutOnlyMode=true
+
+

Added: jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentForDeadLock.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentForDeadLock.java?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentForDeadLock.java (added)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentForDeadLock.java Sun Aug 21 17:20:09 2005
@@ -0,0 +1,139 @@
+package org.apache.jcs.auxiliary.lateral.socket.tcp;
+
+/*
+ * Copyright 2001-2004 The Apache Software Foundation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License")
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import junit.extensions.ActiveTestSuite;
+import junit.framework.Test;
+import junit.framework.TestCase;
+
+import org.apache.jcs.JCS;
+import org.apache.jcs.engine.control.CompositeCacheManager;
+
+/**
+ * Test which exercises the tcp lateral cache. Runs two threads against the
+ * same region and two against other regions.
+ * 
+ */
+public class TestLateralTCPConcurrentForDeadLock
+    extends TestCase
+{
+    /**
+     * Constructor for the TestDiskCache object.
+     * 
+     * @param testName
+     */
+    public TestLateralTCPConcurrentForDeadLock( String testName )
+    {
+        super( testName );
+    }
+
+    /**
+     * Main method passes this test to the text test runner.
+     * 
+     * @param args
+     */
+    public static void main( String args[] )
+    {
+        String[] testCaseName = { TestLateralTCPConcurrentForDeadLock.class.getName() };
+        junit.textui.TestRunner.main( testCaseName );
+    }
+
+    /**
+     * A unit test suite for JUnit
+     * 
+     * @return The test suite
+     */
+    public static Test suite()
+    {
+        
+        System.setProperty( "jcs.auxiliary.LTCP.attributes.PutOnlyMode", "false" );
+        
+        ActiveTestSuite suite = new ActiveTestSuite();
+
+        suite.addTest( new TestLateralTCPConcurrentRandom( "testLateralTCPCache1" )
+        {
+            public void runTest()
+                throws Exception
+            {
+                this.runTestForRegion( "region1", 1, 200, 1 );
+            }
+        } );
+
+        suite.addTest( new TestLateralTCPConcurrentRandom( "testLateralTCPCache2" )
+        {
+            public void runTest()
+                throws Exception
+            {
+                this.runTestForRegion( "region2", 10000, 12000, 2 );
+            }
+        } );
+
+        suite.addTest( new TestLateralTCPConcurrentRandom( "testLateralTCPCache3" )
+        {
+            public void runTest()
+                throws Exception
+            {
+                this.runTestForRegion( "region3", 10000, 12000, 3 );
+            }
+        } );
+
+        suite.addTest( new TestLateralTCPConcurrentRandom( "testLateralTCPCache4" )
+        {
+            public void runTest()
+                throws Exception
+            {
+                this.runTestForRegion( "region3", 10000, 13000, 4 );
+            }
+        } );
+
+        suite.addTest( new TestLateralTCPConcurrentRandom( "testLateralTCPCache5" )
+        {
+            public void runTest()
+                throws Exception
+            {
+                this.runTestForRegion( "region4", 10000, 11000, 5 );
+            }
+        } );
+
+        return suite;
+    }
+
+    /**
+     * Test setup
+     */
+    public void setUp()
+    {
+        JCS.setConfigFilename( "/TestTCPLateralCacheConcurrent.ccf" );
+    }
+
+    /**
+     * Test tearDown. Dispose of the cache.
+     */
+    public void tearDown()
+    {
+        try
+        {
+            CompositeCacheManager cacheMgr = CompositeCacheManager.getInstance();
+            cacheMgr.shutDown();
+        }
+        catch ( Exception e )
+        {
+            e.printStackTrace();
+        }
+    }
+
+}

Added: jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentRandom.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentRandom.java?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentRandom.java (added)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/lateral/socket/tcp/TestLateralTCPConcurrentRandom.java Sun Aug 21 17:20:09 2005
@@ -0,0 +1,179 @@
+package org.apache.jcs.auxiliary.lateral.socket.tcp;
+
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+import org.apache.jcs.JCS;
+import org.apache.jcs.auxiliary.lateral.LateralCacheAttributes;
+import org.apache.jcs.auxiliary.lateral.behavior.ILateralCacheAttributes;
+import org.apache.jcs.engine.CacheElement;
+import org.apache.jcs.engine.behavior.ICacheElement;
+
+/**
+ * @author asmuts
+ */
+public class TestLateralTCPConcurrentRandom
+    extends TestCase
+{
+
+    private static boolean isSysOut = false;
+    //private static boolean isSysOut = true;
+
+    /**
+     * Constructor for the TestDiskCache object.
+     * 
+     * @param testName
+     */
+    public TestLateralTCPConcurrentRandom( String testName )
+    {
+        super( testName );
+    }
+
+    /**
+     * Test setup
+     */
+    public void setUp()
+    {
+        JCS.setConfigFilename( "/TestTCPLateralCacheConcurrent.ccf" );
+    }
+
+    /**
+     * Randomly adds items to cache, gets them, and removes them. The range
+     * count is more than the size of the memory cache, so items should spool to
+     * disk.
+     * 
+     * @param region
+     *            Name of the region to access
+     * @param range
+     * @param numOps
+     * @param testNum
+     * 
+     * @exception Exception
+     *                If an error occurs
+     */
+    public void runTestForRegion( String region, int range, int numOps, int testNum )
+        throws Exception
+    {
+
+        boolean show = true;//false;
+
+        JCS cache = JCS.getInstance( region );
+        
+        
+        ILateralCacheAttributes lattr2 = new LateralCacheAttributes();
+        lattr2.setTcpListenerPort( 1102 );
+        lattr2.setTransmissionTypeName( "TCP" );
+        lattr2.setTcpServer( "localhost:1102" );
+
+        LateralTCPService service = new LateralTCPService( lattr2 );
+        service.setListenerId( 123456 );
+
+        try
+        {
+            for ( int i = 1; i < numOps; i++ )
+            {
+                Random ran = new Random( i );
+                int n = ran.nextInt( 4 );
+                int kn = ran.nextInt( range );
+                String key = "key" + kn;
+                if ( n == 1 )
+                {
+                    ICacheElement element = new CacheElement( region, key, region + ":data" + i + " junk asdfffffffadfasdfasf "
+                        + kn + ":" + n );
+                    service.update( element );
+                    if ( show )
+                    {
+                        p( "put " + key );
+                    }
+                }
+                /**/
+                else if ( n == 2 )
+                {
+                   service.remove( region, key );
+                   if ( show )
+                    {
+                        p( "removed " + key );
+                    }
+                }
+                /**/
+                else
+                {
+                    // slightly greater chance of get
+                    try 
+                    {
+                        Object obj = service.get( region, key );
+                        if ( show && obj != null )
+                        {
+                            p( obj.toString() );
+                        }                        
+                    }
+                    catch( Exception e )
+                    {
+                        // consider failing, some timeouts are expected
+                        e.printStackTrace();
+                    }
+                }
+
+                if ( i % 100 == 0 )
+                {
+                    System.out.println( cache.getStats() );
+                }
+
+            }
+            p( "Finished random cycle of " + numOps );
+        }
+        catch ( Exception e )
+        {
+            p( e.toString() );
+            e.printStackTrace( System.out );
+            throw e;
+        }
+        
+        JCS jcs = JCS.getInstance( region );
+        String key = "testKey" + testNum;
+        String data = "testData" + testNum;
+        jcs.put( key, data );
+        String value = (String) jcs.get( key );
+        assertEquals( "Couldn't put normally.", data, value );
+        
+        // make sure the items we can find are in the correct region.
+        for ( int i = 1; i < numOps; i++ )
+        {
+            String keyL = "key" + i;
+            String dataL = (String)jcs.get( keyL );
+            if ( dataL != null )
+            {
+                assertTrue( "Incorrect region detected.", dataL.startsWith( region ) );
+            }
+            
+        }
+
+        //Thread.sleep( 1000 );
+
+        //ICacheElement element = new CacheElement( region, "abc", "testdata");
+        //service.update( element );
+        
+        //Thread.sleep( 2500 );
+        // could be too mcuh going on right now to get ti through, sot he test might fail.
+        //String value2 = (String) jcs.get( "abc" );
+        //assertEquals( "Couldn't put laterally, could be too much traffic in queue.", "testdata", value2 );
+                
+
+    }
+
+    public static void p( String s )
+    {
+        if ( isSysOut )
+        {
+            System.out.println( s );
+        }
+        else
+        {
+            //if ( log.isInfoEnabled() )
+            //{
+            //    log.info( s );
+            //}
+        }
+    }
+}

Added: jakarta/jcs/trunk/src/test/org/apache/jcs/engine/TestSystemPropertyUsage.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/engine/TestSystemPropertyUsage.java?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/engine/TestSystemPropertyUsage.java (added)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/engine/TestSystemPropertyUsage.java Sun Aug 21 17:20:09 2005
@@ -0,0 +1,53 @@
+package org.apache.jcs.engine;
+
+import java.util.Properties;
+
+import org.apache.jcs.JCS;
+import org.apache.jcs.engine.control.CompositeCacheManager;
+import org.apache.jcs.utils.props.PropertyLoader;
+
+import junit.framework.TestCase;
+
+/**
+ * Verify that system properties can override.
+ */
+public class TestSystemPropertyUsage
+    extends TestCase
+{
+
+    /**
+     * Verify that the system properties are used.
+     *
+     */
+    public void testSystemPropertyUsage() throws Exception
+    {
+        System.getProperties().setProperty( "jcs.default.cacheattributes.MaxObjects", "6789" );
+        
+        JCS.setConfigFilename( "/TestSystemPropertyUsage.ccf" );
+        
+        JCS jcs = JCS.getInstance( "someCacheNotInFile" );
+        
+        assertEquals( "System property value is not reflected",  jcs.getCacheAttributes().getMaxObjects(), Integer.parseInt( "6789" ) );
+
+    }
+    
+    /**
+     * Verify that the system properties are nto used is specified.
+     *
+     */
+    public void testSystemPropertyUsage_inactive() throws Exception
+    {
+        System.getProperties().setProperty( "jcs.default.cacheattributes.MaxObjects", "6789" );
+        
+        CompositeCacheManager mgr = CompositeCacheManager.getUnconfiguredInstance();
+        
+        Properties props = PropertyLoader.loadProperties( "TestSystemPropertyUsage.ccf" );
+        
+        mgr.configure( props, false );
+        
+        JCS jcs = JCS.getInstance( "someCacheNotInFile" );
+                     
+        assertFalse( "System property value should not be reflected",  jcs.getCacheAttributes().getMaxObjects() == Integer.parseInt( props.getProperty("jcs.default.cacheattributes.MaxObjects") ) );
+
+    }
+}

Added: jakarta/jcs/trunk/src/test/org/apache/jcs/engine/control/CompositeCacheManagerMockImpl.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/engine/control/CompositeCacheManagerMockImpl.java?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/engine/control/CompositeCacheManagerMockImpl.java (added)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/engine/control/CompositeCacheManagerMockImpl.java Sun Aug 21 17:20:09 2005
@@ -0,0 +1,45 @@
+package org.apache.jcs.engine.control;
+
+import org.apache.jcs.engine.CompositeCacheAttributes;
+import org.apache.jcs.engine.ElementAttributes;
+import org.apache.jcs.engine.behavior.ICompositeCacheManager;
+
+/**
+ */
+public class CompositeCacheManagerMockImpl
+    implements ICompositeCacheManager
+{
+
+    private CompositeCache cache;
+    
+    /* (non-Javadoc)
+     * @see org.apache.jcs.engine.behavior.ICompositeCacheManager#getCache(java.lang.String)
+     */
+    public CompositeCache getCache( String cacheName )
+    {
+        if ( cache == null )
+        {
+            System.out.println( "Creating mock cache" );
+            CompositeCache newCache = new CompositeCache( cacheName, new CompositeCacheAttributes(), new ElementAttributes() );
+            this.setCache( newCache );
+        }
+        return cache;
+    }
+
+    /**
+     * @param cache The cache to set.
+     */
+    public void setCache( CompositeCache cache )
+    {
+        this.cache = cache;
+    }
+
+    /**
+     * @return Returns the cache.
+     */
+    public CompositeCache getCache()
+    {
+        return cache;
+    }
+
+}

Added: jakarta/jcs/trunk/tempbuild/jcs-1.2.6.6.jar
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/tempbuild/jcs-1.2.6.6.jar?rev=234395&view=auto
==============================================================================
Binary file - no diff available.

Propchange: jakarta/jcs/trunk/tempbuild/jcs-1.2.6.6.jar
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: jakarta/jcs/trunk/tempbuild/jcs-jdk14-ext-1.2.6.6.jar
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/tempbuild/jcs-jdk14-ext-1.2.6.6.jar?rev=234395&view=auto
==============================================================================
Binary file - no diff available.

Propchange: jakarta/jcs/trunk/tempbuild/jcs-jdk14-ext-1.2.6.6.jar
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: jakarta/jcs/trunk/xdocs/IndexedDiskCacheProperties.xml
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/xdocs/IndexedDiskCacheProperties.xml?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/xdocs/IndexedDiskCacheProperties.xml (added)
+++ jakarta/jcs/trunk/xdocs/IndexedDiskCacheProperties.xml Sun Aug 21 17:20:09 2005
@@ -0,0 +1,150 @@
+<?xml version="1.0"?>
+
+<document>
+  <properties>
+    <title>Indexed Disk Cache Configuration</title>
+    <author email="ASmuts@apache.org">Aaron Smuts</author>
+  </properties>
+
+  <body>
+    <section name="Indexed Disk Auxiliary Cache Configuration">
+
+      <p>
+        The following properties apply to the Indexed Disk Cache plugin.
+      </p>
+
+	  <subsection name="Indexed Disk Configuration Properties">
+        <table>
+          <tr>
+            <th>Property</th>
+            <th>Description</th>
+            <th>Required</th>
+            <th>Default Value</th>
+          </tr>
+          <tr>
+            <td>DiskPath</td>
+            <td>
+              The directory where the disk cache should write its files.
+            </td>
+            <td>Y</td>
+            <td>n/a</td>
+          </tr>
+          <tr>
+            <td>MaxPurgatorySize</td>
+            <td>
+              The maximum number of items allowed in the queue of items to 
+              be written to disk.
+            </td>
+            <td>N</td>
+            <td>5000</td>
+          </tr>
+          <tr>
+            <td>MaxKeySize</td>
+            <td>
+              The maximum number of keys that the indexed disk cache can have.  
+              Since the keys are stored in memory, you may want to limit
+              this number to something reasonable.  The default is a bit small.
+            </td>
+            <td>N</td>
+            <td>5000</td>
+          </tr>
+          <tr>
+            <td>OptimizeAtRemoveCount</td>
+            <td>
+              At how many removes should the cache try to defragment the
+              data file.  Since we recycle empty spots, defragmentation
+              is usually not needed.  To prevent the cache from defragmenting
+              the data file, you can set this to -1.  This is the default value.
+            </td>
+            <td>N</td>
+            <td>-1</td>
+          </tr>
+          <tr>
+            <td>MaxRecycleBinSize</td>
+            <td>
+              The maximum number of empty spots the cache will keep
+              track of.  The smallest are removed when the maximum size
+              is reached.  Keeping track of empty spots on disk allows us 
+              to reuse spots, thereby keeping the file from growing unncessarily.
+            </td>
+            <td>N</td>
+            <td>5000</td>
+          </tr>                 
+        </table>
+      </subsection>
+      
+      <subsection name="Example Configuration">
+              <source><![CDATA[
+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/indexed-disk-cache
+jcs.auxiliary.DC.attributes.MaxPurgatorySize=10000
+jcs.auxiliary.DC.attributes.MaxKeySize=10000
+jcs.auxiliary.DC.attributes.OptimizeAtRemoveCount=300000
+jcs.auxiliary.DC.attributes.MaxRecycleBinSize=7500
+        ]]></source>
+      </subsection>
+      
+	  <subsection name="Indexed Disk Event Queue Configuration">
+      
+        <table>
+          <tr>
+            <th>Property</th>
+            <th>Description</th>
+            <th>Required</th>
+            <th>Default Value</th>
+          </tr>
+          <tr>
+            <td>EventQueueType</td>
+            <td>
+			  This should be either SINGLE or POOLED.  By default the single
+			  style pool is used.  The single style pool uses a single thread
+			  per event queue.  That thread is killed whenever the queue is inactive
+			  for 30 seconds.  Since the disk cache uses an event queue for every region,
+			  if you have many regions and they are all active, you will be using many threads.
+			  To limit the number of threads, you can configure the disk cache to 
+			  use the pooled event queue.  Using more threads than regions will not
+			  add any benefit for the indexed disk cache, since only one thread can read or
+			  write at a time for a single region.                
+            </td>
+            <td>N</td>
+            <td>SINGLE</td>
+          </tr>   
+          <tr>
+            <td>EventQueuePoolName</td>
+            <td>
+			  This is the name of the pool to use.  It is required if you
+			  choose the POOLED event queue type, otherwise it is ignored.
+            </td>
+            <td>Y</td>
+            <td>n/a</td>
+          </tr>                         
+        </table>
+      </subsection>
+      
+      <subsection name="Example Configuration Using Thread Pool">
+              <source><![CDATA[      
+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/indexed-disk-cache
+jcs.auxiliary.DC.attributes.MaxPurgatorySize=10000
+jcs.auxiliary.DC.attributes.MaxKeySize=10000
+jcs.auxiliary.DC.attributes.OptimizeAtRemoveCount=300000
+jcs.auxiliary.DC.attributes.MaxRecycleBinSize=7500      
+jcs.auxiliary.DC.attributes.EventQueueType=POOLED
+jcs.auxiliary.DC.attributes.EventQueuePoolName=disk_cache_event_queue
+
+# Disk Cache pool
+thread_pool.disk_cache_event_queue.boundarySize=50
+thread_pool.disk_cache_event_queue.useBoundary=true
+thread_pool.disk_cache_event_queue.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   
+        ]]></source>
+      </subsection>
+
+   
+    </section>
+  </body>
+</document>      
\ No newline at end of file

Added: jakarta/jcs/trunk/xdocs/LateralTCPProperties.xml
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/xdocs/LateralTCPProperties.xml?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/xdocs/LateralTCPProperties.xml (added)
+++ jakarta/jcs/trunk/xdocs/LateralTCPProperties.xml Sun Aug 21 17:20:09 2005
@@ -0,0 +1,117 @@
+<?xml version="1.0"?>
+
+<document>
+  <properties>
+    <title>Lateral TCP Auxiliary Cache Configuration</title>
+    <author email="ASmuts@apache.org">Aaron Smuts</author>
+  </properties>
+
+  <body>
+    <section name="Lateral TCP Auxiliary Cache Configuration">
+
+      <p>
+        The following properties apply to the TCP Lateral Cache plugin.
+      </p>
+
+	  <subsection name="TCP Configuration Properties">
+        <table>
+          <tr>
+            <th>Property</th>
+            <th>Description</th>
+            <th>Required</th>
+            <th>Default Value</th>
+          </tr>
+          <tr>
+            <td>TransmissionTypeName</td>
+            <td>
+              This tells the lateral factory what kind of lateral
+              to create.
+            </td>
+            <td>Y</td>
+            <td>n/a</td>
+          </tr>
+          <tr>
+            <td>TcpServers</td>
+            <td>
+              This is the list of servers this cache should try to connect to.  
+              With UDP discovery this is not necessary.
+            </td>
+            <td>N</td>
+            <td>none</td>
+          </tr>
+          <tr>
+            <td>TcpListenerPort</td>
+            <td>
+              This is the port this cache should listen on.
+            </td>
+            <td>Y</td>
+            <td>n/a</td>
+          </tr>
+          <tr>
+            <td>PutOnlyMode</td>
+            <td>
+              Should this cache be allowed to get from other laterals.  True means
+              that it can only put, i.e. send updates and remove requests to other laterals.
+              Lateral gets are not recommended for performance reasons.
+            </td>
+            <td>N</td>
+            <td>true</td>
+          </tr>
+          <tr>
+            <td>Receive</td>
+            <td>
+              Should this cache receive or only send to other laterals.  You may want to
+              set receive to false if you just need to broadcast to other caches.  If you
+              have a feed data parser, that doesn't need to receive updates, but you
+              do want it to send invalidation messages, then you would set receive to false.
+              If receive is false, teh discovery service, if enabled, will only listen.
+            </td>
+            <td>N</td>
+            <td>true</td>
+          </tr>    
+          <tr>
+            <td>UdpDiscoveryAddr</td>
+            <td>
+              The address the UDP discovery process should broadcast messages to.
+            </td>
+            <td>N</td>
+            <td>228.5.6.7</td>
+          </tr>       
+          <tr>
+            <td>UdpDiscoveryPort</td>
+            <td>
+              The port the UDP discovery process should send messages to.
+            </td>
+            <td>N</td>
+            <td>6789</td>
+          </tr>   
+          <tr>
+            <td>UdpDiscoveryEnabled</td>
+            <td>
+              Whether or not the UDP discovery service should be used to
+              locate other lateral caches.
+            </td>
+            <td>N</td>
+            <td>true</td>
+          </tr>                               
+        </table>
+      </subsection>
+      
+      <subsection name="Example Configuration">
+              <source><![CDATA[
+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=
+jcs.auxiliary.LTCP.attributes.TcpListenerPort=1118
+jcs.auxiliary.LTCP.attributes.PutOnlyMode=true
+jcs.auxiliary.LTCP.attributes.Receive=true
+jcs.auxiliary.LTCP.attributes.UdpDiscoveryAddr=228.5.6.8
+jcs.auxiliary.LTCP.attributes.UdpDiscoveryPort=6780
+jcs.auxiliary.LTCP.attributes.UdpDiscoveryEnabled=true
+        ]]></source>
+      </subsection>
+      
+    </section>
+  </body>
+</document>      
\ No newline at end of file

Added: jakarta/jcs/trunk/xdocs/RegionProperties.xml
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/xdocs/RegionProperties.xml?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/xdocs/RegionProperties.xml (added)
+++ jakarta/jcs/trunk/xdocs/RegionProperties.xml Sun Aug 21 17:20:09 2005
@@ -0,0 +1,198 @@
+<?xml version="1.0"?>
+
+<document>
+  <properties>
+    <title>Cache Region Configuration</title>
+    <author email="ASmuts@apache.org">Aaron Smuts</author>
+  </properties>
+
+  <body>
+    <section name="Cache Region Configuration">
+
+      <p>
+        The following properties apply to any cache region.  They can be specified as
+        default values and specified on a region by region basis.  There are three types
+        of settings: auxiliary, cache, and element.  The cache settings define the memory management for the region.
+        The element settings define default element behavior within the region.
+      </p>
+
+	  <subsection name="Region (Auxiliary) Properties">
+        <table>
+          <tr>
+            <th>Property</th>
+            <th>Description</th>
+            <th>Required</th>
+            <th>Default Value</th>
+          </tr>
+          <tr>
+            <td></td>
+            <td>
+              You can specify the list of auxiliaries that regions can use.  This has no attribute name.
+              The list can be empty, otherwise it should be comma delimited. 
+            </td>
+            <td>Y</td>
+            <td>n/a</td>
+          </tr>
+        </table>
+      </subsection>
+                
+	  <subsection name="Region (Cache) Properties">
+        <table>
+          <tr>
+            <th>Property</th>
+            <th>Description</th>
+            <th>Required</th>
+            <th>Default Value</th>
+          </tr>
+          <tr>
+            <td>MaxObjects</td>
+            <td>
+			  The maximum number of items allowed in memory. Eviction of elements
+			  in excess of this number is determined by the memory cache.  By default
+			  JCS uses the LRU memory cache.	
+            </td>
+            <td>Y</td>
+            <td>n/a</td>
+          </tr>
+          <tr>
+            <td>MemoryCacheName</td>
+            <td>
+			  This property allows you to specify what memory manager you would like to use.
+			  You can create your own memory manager by implementing the org.apache.jcs.engine.memory.MemoryCache interface.
+			  Alternatively, you can extend the org.apache.jcs.engine.memory.AbstractMemoryCache class.  Several different
+			  memory caches are available: two LRU implementations, an LFU, and an adaptive replacement algorithm.
+            </td>
+            <td>N</td>
+            <td>org.apache.jcs.engine.memory.lru.LRUMemoryCache</td>
+          </tr>
+          <tr>
+            <td>UseMemoryShrinker</td>
+            <td>
+			  By default, the memory shrinker is shared by all regions thta use the
+			  LRU memory cache.  The memory shrinker iterates through the items in
+			  memory, looking for items that have expired or that have exceeded their max
+			  memory idle time.
+            </td>
+            <td>N</td>
+            <td>false</td>
+          </tr>
+          <tr>
+            <td>MaxMemoryIdleTimeSeconds</td>
+            <td>
+			  This is only used if you are using the memory shrinker.  If
+			  this value is set above -1, then if an item has not been accessed
+			  in this number of seconds, it will be spooled to disk if
+			  the disk is available.  You can register an event handler on this
+			  event.	
+            </td>
+            <td>N</td>
+            <td>-1</td>
+          </tr>    
+          <tr>
+            <td>ShrinkerIntervalSeconds</td>
+            <td>
+ 	          This specifies how often the shrinker should run, if it has
+ 	          been activated.  If you set UseMemoryShrinker to false, then this
+ 	          setting has no effect.
+            </td>
+            <td>N</td>
+            <td>60</td>
+          </tr>                 
+        </table>
+      </subsection>
+      
+	  <subsection name="Region (Element) Properties">
+        <table>
+          <tr>
+            <th>Property</th>
+            <th>Description</th>
+            <th>Required</th>
+            <th>Default Value</th>
+          </tr>
+          <tr>
+            <td>IsEternal</td>
+            <td>
+			  If an element is specified as eternal, then it will never be subject to
+			  removal for exceeding its max life.  
+            </td>
+            <td>N</td>
+            <td>true</td>
+          </tr>
+          <tr>
+            <td>MaxLifeSeconds</td>
+            <td>
+			  If you specify that elements within a region are not eternal, then
+			  you can set the max life seconds.  If this is exceeded the elmenets will
+			  be removed passively when a client tries to retrieve them.  If you
+			  are using a memory shrinker, then the items can be removed actively.			  
+            </td>
+            <td>N</td>
+            <td>-1</td>
+          </tr>
+          <tr>
+            <td>IsSpool</td>
+            <td>
+              By default, can elements in this region be sent to a disk cache
+              if one is available.
+            </td>
+            <td>N</td>
+            <td>true</td>
+          </tr>
+          <tr>
+            <td>IsRemote</td>
+            <td>
+              By default, can elements in this region be sent to a lateral cache
+              if one is available.            
+            </td>
+            <td>N</td>
+            <td>true</td>
+          </tr>
+          <tr>
+            <td>IsLateral</td>
+            <td>
+              By default, can elements in this region be sent to a remote cache
+              if one is available.            
+            </td>
+            <td>N</td>
+            <td>true</td>
+          </tr>
+        </table>
+      </subsection>      
+      
+      <subsection name="Example Configuration">
+              <source><![CDATA[
+jcs.default=
+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.IsSpool=true
+jcs.default.elementattributes.IsRemote=true
+jcs.default.elementattributes.IsLateral=true
+
+# optional region "testCache1" specific configuration settings
+jcs.region.testCache1=
+jcs.region.testCache1.cacheattributes=org.apache.jcs.engine.CompositeCacheAttributes
+jcs.region.testCache1.cacheattributes.MaxObjects=123456
+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.IsSpool=true
+jcs.region.testCache1.elementattributes.IsLateral=true
+jcs.region.testCache1.elementattributes.IsRemote=true
+        ]]></source>
+      </subsection>
+      
+    </section>
+  </body>
+</document>      
\ No newline at end of file

Added: jakarta/jcs/trunk/xdocs/changes.xml
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/xdocs/changes.xml?rev=234395&view=auto
==============================================================================
--- jakarta/jcs/trunk/xdocs/changes.xml (added)
+++ jakarta/jcs/trunk/xdocs/changes.xml Sun Aug 21 17:20:09 2005
@@ -0,0 +1,51 @@
+<document>
+  <properties>
+    <title>JCS Change Log</title>
+    <author email="asmuts@apache.org">Aaron Smuts</author>
+  </properties>
+  <body>
+
+    <release version="1.2.6.6" date="in CVS">
+      <action dev="asmuts" type="fix">
+        Properties are loaded using the current thread's classloader by default.
+      </action>   
+      <action dev="asmuts" type="fix">
+        When a config file is not found a descriptive error message is printed to the logs.
+      </action>               
+      <action dev="asmuts" type="update">
+        Made an interface for the cache manager to improve testability.
+      </action>              
+      <action dev="asmuts" type="update">
+        Auxiliaries are assigned a cache manager and the lateral TCP has
+        a settable source id, so it can be tested.
+      </action>              
+      <action dev="asmuts" type="update">
+		Increased JGroups version to 2.2.8.  It works again.  The old JGroups
+		didn't work for unknown reasons.
+      </action>              
+      <action dev="asmuts" type="update">
+		Cleaned up some javadocs.
+      </action>     
+      <action dev="asmuts" type="update">
+		Improved info level logging for TCP lateral.
+      </action>                 
+      <action dev="asmuts" type="update">
+		Added configuration properties xdocs for TCP lateral, indexed disk, and memory cache.
+      </action>              
+      <action dev="asmuts" type="update">
+		Created concurrent tests for TCP lateral to verify correct region behavior.
+      </action>              
+      <action dev="asmuts" type="update">
+		Made BDBJE work with new auxiliary changes.
+      </action>              
+      <action dev="asmuts" type="fix">
+		UDP discovery no longer activates if the lateral type isn't TCP.
+      </action>              
+      <action dev="asmuts" type="fix">
+		Reduced lateral get timeout to 1 second, rather than 10.  We still
+		need to disable gets when a timeout occurs.
+      </action>              
+    </release>
+
+  </body>
+</document>
\ No newline at end of file



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