You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jcs-dev@jakarta.apache.org by as...@apache.org on 2006/03/17 01:16:42 UTC

svn commit: r386493 - in /jakarta/jcs/trunk/src: java/org/apache/jcs/auxiliary/disk/ java/org/apache/jcs/auxiliary/disk/behavior/ java/org/apache/jcs/auxiliary/disk/indexed/ java/org/apache/jcs/auxiliary/disk/jdbc/ java/org/apache/jcs/auxiliary/disk/jd...

Author: asmuts
Date: Thu Mar 16 16:16:40 2006
New Revision: 386493

URL: http://svn.apache.org/viewcvs?rev=386493&view=rev
Log:
--Removed old hsql disk cache.

--Added new hsql using new jdbc disk cache base.

--Tested the jdbc with mysql for several hours and found the need for a removeall limiting setting.  I added the ability to prohibit removeall commands for the disk cache.

--Added new tests for hsql, found race condition with removeall resulting from my changes earlier.  added new locking for update and remove all to solve the issue.  Unit tests prove the fix.

--Added runtime shutdown hook for indexed disk cache.

--Added startup size the indexed disk cache stats.

Added:
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheConcurrentUnitTest.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheUnitTest.java
Modified:
    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/behavior/IDiskCacheAttributes.java
    jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.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/JDBCDiskCacheAttributes.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/JDBCDiskCacheManager.java
    jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/indexed/IndexDiskCacheUnitTest.java

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCache.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCache.java?rev=386493&r1=386492&r2=386493&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 Thu Mar 16 16:16:40 2006
@@ -41,6 +41,8 @@
 import org.apache.jcs.engine.stats.behavior.IStatElement;
 import org.apache.jcs.engine.stats.behavior.IStats;
 
+import EDU.oswego.cs.dl.util.concurrent.WriterPreferenceReadWriteLock;
+
 /**
  * Abstract class providing a base implementation of a disk cache, which can be
  * easily extended to implement a disk cache for a specific perstistence
@@ -98,6 +100,10 @@
      */
     protected int purgHits = 0;
 
+    // we lock here, so that we cannot get an update after a remove all
+    // an individual removeal locks the item.
+    private WriterPreferenceReadWriteLock removeAllLock = new WriterPreferenceReadWriteLock();
+
     // ----------------------------------------------------------- constructors
 
     /**
@@ -134,15 +140,45 @@
      */
     private void initPurgatory()
     {
-        purgatory = null;
+        try
+        {
+            // we need this so we can stop the updates from happening after a
+            // removeall
+            removeAllLock.writeLock().acquire();
 
-        if ( dcattr.getMaxPurgatorySize() >= 0 )
+            if ( purgatory != null )
+            {
+                synchronized ( purgatory )
+                {
+                    if ( dcattr.getMaxPurgatorySize() >= 0 )
+                    {
+                        purgatory = new LRUMapJCS( dcattr.getMaxPurgatorySize() );
+                    }
+                    else
+                    {
+                        purgatory = new HashMap();
+                    }
+                }
+            }
+            else
+            {
+                if ( dcattr.getMaxPurgatorySize() >= 0 )
+                {
+                    purgatory = new LRUMapJCS( dcattr.getMaxPurgatorySize() );
+                }
+                else
+                {
+                    purgatory = new HashMap();
+                }
+            }
+        }
+        catch ( InterruptedException e )
         {
-            purgatory = new LRUMapJCS( dcattr.getMaxPurgatorySize() );
+            log.error( "problem encountered resseting purgatory.", e );
         }
-        else
+        finally
         {
-            purgatory = new HashMap();
+            removeAllLock.writeLock().release();
         }
     }
 
@@ -300,7 +336,7 @@
                 {
                     purgatory.remove( key );
                 }
-                
+
                 // no way to remove from queue, just make sure it doesn't get on
                 // disk and then removed right afterwards
                 pe.setSpoolable( false );
@@ -323,11 +359,21 @@
      */
     public final void removeAll()
     {
-        // Replace purgatory with a new empty hashtable
-        initPurgatory();
+        if ( this.dcattr.isAllowRemoveAll() )
+        {
+            // Replace purgatory with a new empty hashtable
+            initPurgatory();
 
-        // Remove all from persistent store immediately
-        doRemoveAll();
+            // Remove all from persistent store immediately
+            doRemoveAll();
+        }
+        else
+        {
+            if ( log.isInfoEnabled() )
+            {
+                log.info( "RemoveAll was requested but the request was not fulfilled: allowRemoveAll is set to false." );
+            }
+        }
     }
 
     /**
@@ -531,24 +577,43 @@
 
                     synchronized ( pe.getCacheElement() )
                     {
-                        // String keyAsString = element.getKey().toString();
-                        synchronized ( purgatory )
+                        try
                         {
-                            // If the element has already been removed from
-                            // purgatory do nothing
-                            if ( !purgatory.containsKey( pe.getKey() ) )
+                            // TODO consider a timeout.
+                            // we need this so that we can have multiple update
+                            // threads
+                            // and still have removeAll request come in that
+                            // always win
+                            removeAllLock.readLock().acquire();
+
+                            // TODO consider changing purgatory sync
+                            // String keyAsString = element.getKey().toString();
+                            synchronized ( purgatory )
                             {
-                                return;
+                                // If the element has already been removed from
+                                // purgatory do nothing
+                                if ( !purgatory.containsKey( pe.getKey() ) )
+                                {
+                                    return;
+                                }
+
+                                element = pe.getCacheElement();
                             }
 
-                            element = pe.getCacheElement();
+                            // I took this out of the purgatory sync block.
+                            // If the element is still eligable, spool it.
+                            if ( pe.isSpoolable() )
+                            {
+                                doUpdate( element );
+                            }
                         }
-
-                        // I took this out of the sync block. 
-                        // If the element is still eligable, spool it.
-                        if ( pe.isSpoolable() )
+                        catch ( InterruptedException e )
                         {
-                            doUpdate( element );
+                            log.error( e );
+                        }
+                        finally
+                        {
+                            removeAllLock.readLock().release();
                         }
 
                         synchronized ( purgatory )
@@ -577,7 +642,6 @@
                 // during normal opertations.
 
                 // String keyAsString = element.getKey().toString();
-
                 synchronized ( purgatory )
                 {
                     purgatory.remove( element.getKey() );

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCacheAttributes.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/AbstractDiskCacheAttributes.java?rev=386493&r1=386492&r2=386493&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 Thu Mar 16 16:16:40 2006
@@ -34,6 +34,9 @@
     /** path to disk */
     protected String diskPath;
 
+    // if this is false, we will not execute remove all
+    private boolean allowRemoveAll = true;
+    
     /** default to 5000 */
     protected int maxPurgatorySize = MAX_PURGATORY_SIZE_DEFUALT;
 
@@ -121,6 +124,22 @@
     }
 
     /**
+     * @param allowRemoveAll The allowRemoveAll to set.
+     */
+    public void setAllowRemoveAll( boolean allowRemoveAll )
+    {
+        this.allowRemoveAll = allowRemoveAll;
+    }
+
+    /**
+     * @return Returns the allowRemoveAll.
+     */
+    public boolean isAllowRemoveAll()
+    {
+        return allowRemoveAll;
+    }
+    
+    /**
      * Description of the Method
      * 
      * @return String
@@ -131,6 +150,7 @@
         str.append( "AbstractDiskCacheAttributes " );
         str.append( "\n diskPath = " + diskPath );
         str.append( "\n maxPurgatorySize   = " + maxPurgatorySize );
+        str.append( "\n allowRemoveAll   = " + allowRemoveAll );
         return str.toString();
     }
 

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/behavior/IDiskCacheAttributes.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/behavior/IDiskCacheAttributes.java?rev=386493&r1=386492&r2=386493&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/behavior/IDiskCacheAttributes.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/behavior/IDiskCacheAttributes.java Thu Mar 16 16:16:40 2006
@@ -84,5 +84,19 @@
      */
     public void setShutdownSpoolTimeLimit( int shutdownSpoolTimeLimit );
 
+    /**
+     * If this is true then remove all is not prohibited.
+     * 
+     * @return
+     */
+    public boolean isAllowRemoveAll();
+    
+    /**
+     * If this is false, then remove all requests will not be honored.
+     * <p>
+     * This provides a safety mechanism for the persistent store.
+     * @param allowRemoveAll
+     */
+    public void setAllowRemoveAll( boolean allowRemoveAll );
 }
 //   end interface

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/indexed/IndexedDiskCache.java?rev=386493&r1=386492&r2=386493&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 Thu Mar 16 16:16:40 2006
@@ -94,6 +94,8 @@
 
     private int recycleCnt = 0;
 
+    private int startupSize = 0;
+    
     /**
      * use this lock to synchronize reads and writes to the underlying storage
      * mechansism.
@@ -149,6 +151,10 @@
                         dataFile.reset();
                         log.warn( "Corruption detected.  Reset data and keys files." );
                     }
+                    else
+                    {
+                        startupSize = keyHash.size();
+                    }
                 }
             }
 
@@ -175,6 +181,9 @@
         {
             log.error( "Failure initializing for fileName: " + fileName + " and root directory: " + rootDirName, e );
         }
+        
+        ShutdownHook shutdownHook = new ShutdownHook();
+        Runtime.getRuntime().addShutdownHook( shutdownHook );
     }
 
     /**
@@ -1368,6 +1377,11 @@
         se.setData( "" + this.recycleCnt );
         elems.add( se );
 
+        se = new StatElement();
+        se.setName( "Startup Size" );
+        se.setData( "" + this.startupSize );
+        elems.add( se );
+
         // get the stats from the super too
         // get as array, convert to list, add list to our outer list
         IStats sStats = super.getStatistics();
@@ -1439,5 +1453,25 @@
 
         }
     }
-
+    
+    /**
+     * Called on shutdown
+     *
+     * @author Aaron Smuts
+     *
+     */
+    class ShutdownHook extends Thread
+    {
+        
+        public void run()
+        {
+            if ( alive )
+            {
+                log.info( "Disk cache was not shutdown properly.  Will try to dispose." );
+                
+                doDispose();
+            }            
+        }
+        
+    }
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCache.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCache.java?rev=386493&r1=386492&r2=386493&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 Thu Mar 16 16:16:40 2006
@@ -55,23 +55,24 @@
  * configurable.
  * 
  * <pre>
- *         drop TABLE JCS_STORE;
- *        
- *         CREATE TABLE JCS_STORE
- *         (
- *         CACHE_KEY             VARCHAR(250)          NOT NULL,
- *         REGION                VARCHAR(250)          NOT NULL,
- *         ELEMENT               BLOB,
- *         CREATE_TIME           DATE,
- *         CREATE_TIME_SECONDS   BIGINT,
- *         MAX_LIFE_SECONDS      BIGINT,
- *         IS_ETERNAL            CHAR(1),
- *         PRIMARY KEY (CACHE_KEY, REGION)
- *         );
+ *          drop TABLE JCS_STORE;
+ *         
+ *          CREATE TABLE JCS_STORE
+ *          (
+ *          CACHE_KEY             VARCHAR(250)          NOT NULL,
+ *          REGION                VARCHAR(250)          NOT NULL,
+ *          ELEMENT               BLOB,
+ *          CREATE_TIME           DATE,
+ *          CREATE_TIME_SECONDS   BIGINT,
+ *          MAX_LIFE_SECONDS      BIGINT,
+ *          IS_ETERNAL            CHAR(1),
+ *          PRIMARY KEY (CACHE_KEY, REGION)
+ *          );
  * </pre>
  * 
  * 
- * The cleanup thread will delete non eternal items where (now - create time) > max life seconds * 1000
+ * The cleanup thread will delete non eternal items where (now - create time) >
+ * max life seconds * 1000
  * 
  * @author Aaron Smuts
  * 
@@ -131,7 +132,8 @@
                 log.error( "Couldn't find class for driver [" + cattr.getDriverClassName() + "]", e );
             }
 
-            setupDriver( cattr.getUrl(), cattr.getUserName(), cattr.getPassword(), cattr.getMaxActive() );
+            setupDriver( cattr.getUrl() + cattr.getDatabase(), cattr.getUserName(), cattr.getPassword(), cattr
+                .getMaxActive() );
 
             logDriverStats();
         }
@@ -236,7 +238,8 @@
 
                 try
                 {
-                    String sqlI = "insert into " + getJdbcDiskCacheAttributes().getTableName()
+                    String sqlI = "insert into "
+                        + getJdbcDiskCacheAttributes().getTableName()
                         + " (CACHE_KEY, REGION, ELEMENT, MAX_LIFE_SECONDS, IS_ETERNAL, CREATE_TIME, CREATE_TIME_SECONDS) values (?, ?, ?, ?, ?, ?, ?)";
 
                     PreparedStatement psInsert = con.prepareStatement( sqlI );
@@ -246,11 +249,11 @@
                     psInsert.setLong( 4, ce.getElementAttributes().getMaxLifeSeconds() );
                     if ( ce.getElementAttributes().getIsEternal() )
                     {
-                        psInsert.setString( 5, "T" );                        
+                        psInsert.setString( 5, "T" );
                     }
                     else
                     {
-                        psInsert.setString( 5, "F" );                        
+                        psInsert.setString( 5, "F" );
                     }
                     Date createTime = new Date( ce.getElementAttributes().getCreateTime() );
                     psInsert.setDate( 6, createTime );
@@ -574,47 +577,58 @@
         return false;
     }
 
-    /** This should remove all elements. For now this is not implemented. */
+    /** This should remove all elements. */
     public void doRemoveAll()
     {
-        try
+        // it should never get here formt he abstract dis cache.
+        if ( this.jdbcDiskCacheAttributes.isAllowRemoveAll() )
         {
-            String sql = "delete from " + getJdbcDiskCacheAttributes().getTableName() + " where REGION = '"
-                + this.getCacheName() + "'";
-            Connection con = DriverManager.getConnection( getPoolUrl() );
-            Statement sStatement = null;
             try
             {
-                sStatement = con.createStatement();
-                alive = true;
-
-                sStatement.executeUpdate( sql );
-            }
-            catch ( SQLException e )
-            {
-                log.error( "Problem creating statement.", e );
-                alive = false;
-            }
-            finally
-            {
+                String sql = "delete from " + getJdbcDiskCacheAttributes().getTableName() + " where REGION = '"
+                    + this.getCacheName() + "'";
+                Connection con = DriverManager.getConnection( getPoolUrl() );
+                Statement sStatement = null;
                 try
                 {
-                    if ( sStatement != null )
-                    {
-                        sStatement.close();
-                    }
-                    con.close();
+                    sStatement = con.createStatement();
+                    alive = true;
+
+                    sStatement.executeUpdate( sql );
                 }
-                catch ( SQLException e1 )
+                catch ( SQLException e )
                 {
-                    log.error( "Problem closing statement.", e1 );
+                    log.error( "Problem creating statement.", e );
+                    alive = false;
+                }
+                finally
+                {
+                    try
+                    {
+                        if ( sStatement != null )
+                        {
+                            sStatement.close();
+                        }
+                        con.close();
+                    }
+                    catch ( SQLException e1 )
+                    {
+                        log.error( "Problem closing statement.", e1 );
+                    }
                 }
             }
+            catch ( Exception e )
+            {
+                log.error( "Problem removing all.", e );
+                reset();
+            }
         }
-        catch ( Exception e )
+        else
         {
-            log.error( "Problem removing all.", e );
-            reset();
+            if ( log.isInfoEnabled() )
+            {
+                log.info( "RemoveAll was requested but the request was not fulfilled: allowRemoveAll is set to false." );
+            }
         }
     }
 
@@ -622,7 +636,7 @@
      * Removed the expired.
      * 
      * (now - create time) > max life seconds * 1000
-     *
+     * 
      */
     protected void deleteExpired()
     {
@@ -665,7 +679,7 @@
         {
             log.error( "Problem removing all.", e );
             reset();
-        }        
+        }
     }
 
     /**
@@ -700,7 +714,8 @@
         int size = 0;
 
         // region, key
-        String selectString = "select count(*) from " + getJdbcDiskCacheAttributes().getTableName() + " where REGION = ?";
+        String selectString = "select count(*) from " + getJdbcDiskCacheAttributes().getTableName()
+            + " where REGION = ?";
 
         Connection con;
         try
@@ -814,7 +829,8 @@
      * @param connectURI
      * @param userName
      * @param password
-     * @param maxActive max connetions
+     * @param maxActive
+     *            max connetions
      * @throws Exception
      */
     public void setupDriver( String connectURI, String userName, String password, int maxActive )
@@ -883,7 +899,6 @@
         }
     }
 
-    
     /**
      * How many are idle in the pool.
      * 
@@ -908,8 +923,8 @@
             log.error( e );
         }
         return numIdle;
-    }   
-    
+    }
+
     /**
      * How many are active in the pool.
      * 
@@ -985,7 +1000,8 @@
     }
 
     /**
-     * @param jdbcDiskCacheAttributes The jdbcDiskCacheAttributes to set.
+     * @param jdbcDiskCacheAttributes
+     *            The jdbcDiskCacheAttributes to set.
      */
     protected void setJdbcDiskCacheAttributes( JDBCDiskCacheAttributes jdbcDiskCacheAttributes )
     {

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheAttributes.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheAttributes.java?rev=386493&r1=386492&r2=386493&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheAttributes.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheAttributes.java Thu Mar 16 16:16:40 2006
@@ -38,6 +38,8 @@
 
     private String url;
 
+    private String database = "";
+    
     private String driverClassName;
 
     private String tableName = DEFAULT_TABLE_NAME;
@@ -46,7 +48,7 @@
 
     private static final int DEFAULT_MAX_ACTIVE = 10;
     
-    private int maxActive = DEFAULT_MAX_ACTIVE;
+    private int maxActive = DEFAULT_MAX_ACTIVE;   
     
     /**
      * @param userName
@@ -100,6 +102,23 @@
     }
 
     /**
+     * This is appended to the url.
+     * @param database The database to set.
+     */
+    public void setDatabase( String database )
+    {
+        this.database = database;
+    }
+
+    /**
+     * @return Returns the database.
+     */
+    public String getDatabase()
+    {
+        return database;
+    }
+
+    /**
      * @param driverClassName
      *            The driverClassName to set.
      */
@@ -178,10 +197,12 @@
         buf.append( "\nJDBCCacheAttributes" );
         buf.append( "\nUserName [" + getUserName() + "]" );
         buf.append( "\nUrl [" + getUrl() + "]" );
+        buf.append( "\nDatabase [" + getDatabase() + "]" );
         buf.append( "\nDriverClassName [" + getDriverClassName() + "]" );
         buf.append( "\nTableName [" + getTableName() + "]" );
         buf.append( "\nTestBeforeInsert [" + isTestBeforeInsert() + "]" );
         buf.append( "\nMaxActive [" + getMaxActive() + "]" );
+        buf.append( "\nAllowRemoveAll [" + isAllowRemoveAll() + "]" );
         return buf.toString();
     }
 

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheFactory.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheFactory.java?rev=386493&r1=386492&r2=386493&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 Thu Mar 16 16:16:40 2006
@@ -30,8 +30,7 @@
 public class JDBCDiskCacheFactory
     implements AuxiliaryCacheFactory
 {
-
-    private String name = "MysqlDiskCacheFactory";
+    private String name = "JDBCDiskCacheFactory";
 
     /**
      * This factory method should create an instance of the mysqlcache.
@@ -57,5 +56,4 @@
     {
         return name;
     }
-
 }

Modified: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheManager.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheManager.java?rev=386493&r1=386492&r2=386493&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheManager.java (original)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/JDBCDiskCacheManager.java Thu Mar 16 16:16:40 2006
@@ -33,7 +33,6 @@
 public class JDBCDiskCacheManager
     implements AuxiliaryCacheManager
 {
-
     private static final long serialVersionUID = -8258856770927857896L;
 
     private static final Log log = LogFactory.getLog( JDBCDiskCacheManager.class );

Added: jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java?rev=386493&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java (added)
+++ jakarta/jcs/trunk/src/java/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheFactory.java Thu Mar 16 16:16:40 2006
@@ -0,0 +1,216 @@
+package org.apache.jcs.auxiliary.disk.jdbc.hsql;
+
+/*
+ * 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.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.jcs.auxiliary.AuxiliaryCache;
+import org.apache.jcs.auxiliary.AuxiliaryCacheAttributes;
+import org.apache.jcs.auxiliary.AuxiliaryCacheFactory;
+import org.apache.jcs.auxiliary.disk.jdbc.JDBCDiskCacheAttributes;
+import org.apache.jcs.auxiliary.disk.jdbc.JDBCDiskCacheManager;
+import org.apache.jcs.engine.behavior.ICompositeCacheManager;
+
+/**
+ * This factory should create mysql disk caches.
+ * 
+ * @author Aaron Smuts
+ * 
+ */
+public class HSQLDiskCacheFactory
+    implements AuxiliaryCacheFactory
+{
+    private final static Log log = LogFactory.getLog( HSQLDiskCacheFactory.class );
+
+    private String name = "HSQLDiskCacheFactory";
+
+    private Set databases = Collections.synchronizedSet( new HashSet() );
+
+    /**
+     * This factory method should create an instance of the mysqlcache.
+     */
+    public AuxiliaryCache createCache( AuxiliaryCacheAttributes rawAttr, ICompositeCacheManager arg1 )
+    {
+
+        JDBCDiskCacheManager mgr = JDBCDiskCacheManager.getInstance( (JDBCDiskCacheAttributes) rawAttr );
+
+        try
+        {
+            setupDatabase( (JDBCDiskCacheAttributes) rawAttr );
+        }
+        catch ( Exception e )
+        {
+            // TODO we may not want to try and get the cache at this point.
+            log.error( "Problem setting up database.", e );
+        }
+
+        return mgr.getCache( (JDBCDiskCacheAttributes) rawAttr );
+    }
+
+    /**
+     * The name of the factory.
+     */
+    public void setName( String nameArg )
+    {
+        name = nameArg;
+    }
+
+    /**
+     * Returns the display name
+     */
+    public String getName()
+    {
+        return name;
+    }
+
+    /**
+     * Creates the database if it doesn't exist, registers the driver class,
+     * etc.
+     * 
+     * @param attributes
+     * @throws Exception
+     */
+    protected void setupDatabase( JDBCDiskCacheAttributes attributes )
+        throws Exception
+    {
+
+        if ( attributes == null )
+        {
+            throw new Exception( "The attributes are null." );
+        }
+
+        // url should start with "jdbc:hsqldb:"
+        String database = attributes.getUrl() + attributes.getDatabase();
+
+        if ( databases.contains( database ) )
+        {
+            if ( log.isInfoEnabled() )
+            {
+                log.info( "We already setup database [" + database + "]" );
+            }
+            return;
+        }
+
+        // TODO get this from the attributes.
+        System.setProperty( "hsqldb.cache_scale", "8" );
+
+        // "org.hsqldb.jdbcDriver"
+        String driver = attributes.getDriverClassName();
+        // "sa"
+        String user = attributes.getUserName();
+        // ""
+        String password = attributes.getPassword();
+
+        new org.hsqldb.jdbcDriver();
+        try
+        {
+            Class.forName( driver ).newInstance();
+
+            Connection cConn = DriverManager.getConnection( database, user, password );
+
+            setupTABLE( cConn, attributes.getTableName() );
+
+            if ( log.isInfoEnabled() )
+            {
+                log.info( "Finished setting up database [" + database + "]" );
+            }
+
+            databases.add( database );
+        }
+        catch ( Exception e )
+        {
+            log.error( "Fatal problem setting up the database.", e );
+        }
+    }
+
+    /**
+     * SETUP TABLE FOR CACHE
+     * 
+     * @param cConn
+     * @param tableName
+     */
+    private void setupTABLE( Connection cConn, String tableName )
+    {
+        boolean newT = true;
+
+        // TODO make the cached nature of the table configurable
+        StringBuffer createSql = new StringBuffer();
+        createSql.append( "CREATE CACHED TABLE " + tableName );
+        createSql.append( "( " );
+        createSql.append( "CACHE_KEY             VARCHAR(250)          NOT NULL, " );
+        createSql.append( "REGION                VARCHAR(250)          NOT NULL, " );
+        createSql.append( "ELEMENT               BINARY, " );
+        createSql.append( "CREATE_TIME           DATE, " );
+        createSql.append( "CREATE_TIME_SECONDS   BIGINT, " );
+        createSql.append( "MAX_LIFE_SECONDS      BIGINT, " );
+        createSql.append( "IS_ETERNAL            CHAR(1), " );
+        createSql.append( "PRIMARY KEY (CACHE_KEY, REGION) " );
+        createSql.append( ");" );
+
+        Statement sStatement = null;
+        try
+        {
+            sStatement = cConn.createStatement();
+        }
+        catch ( SQLException e )
+        {
+            log.error( "problem creating a statement.", e );
+        }
+
+        try
+        {
+            sStatement.executeQuery( createSql.toString() );
+            sStatement.close();
+        }
+        catch ( SQLException e )
+        {
+            if ( e.toString().indexOf( "already exists" ) != -1 )
+            {
+                newT = false;
+            }
+            else
+            {
+                log.error( "Problem creating table.", e );
+            }
+        }
+
+        String setupData[] = { "create index iKEY on " + tableName + " (CACHE_KEY, REGION)" };
+
+        if ( newT )
+        {
+            for ( int i = 1; i < setupData.length; i++ )
+            {
+                try
+                {
+                    sStatement.executeQuery( setupData[i] );
+                }
+                catch ( SQLException e )
+                {
+                    log.error( "Exception caught when creating index." + e );
+                }
+            }
+        } // end ifnew
+    }
+}

Modified: jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/indexed/IndexDiskCacheUnitTest.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/indexed/IndexDiskCacheUnitTest.java?rev=386493&r1=386492&r2=386493&view=diff
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/indexed/IndexDiskCacheUnitTest.java (original)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/indexed/IndexDiskCacheUnitTest.java Thu Mar 16 16:16:40 2006
@@ -25,13 +25,13 @@
     {
         IndexedDiskCacheAttributes cattr = new IndexedDiskCacheAttributes();
         cattr.setCacheName( "testSimplePutAndGet" );
-        cattr.setMaxKeySize( 100 );
+        cattr.setMaxKeySize( 1000 );
         cattr.setDiskPath( "target/test-sandbox/IndexDiskCacheUnitTest" );
         IndexedDiskCache disk = new IndexedDiskCache( cattr );
 
         disk.doRemoveAll();
 
-        int cnt = 25;
+        int cnt = 999;
         for ( int i = 0; i < cnt; i++ )
         {
             IElementAttributes eAttr = new ElementAttributes();
@@ -47,6 +47,8 @@
             assertNotNull( "Should have recevied an element.", element );
             assertEquals( "Element is wrong.", "data:" + i, element.getVal() );
         }
+        
+        System.out.println( disk.getStats() );
     }
 
     /**

Added: jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheConcurrentUnitTest.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheConcurrentUnitTest.java?rev=386493&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheConcurrentUnitTest.java (added)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheConcurrentUnitTest.java Thu Mar 16 16:16:40 2006
@@ -0,0 +1,159 @@
+package org.apache.jcs.auxiliary.disk.jdbc.hsql;
+
+/*
+ * 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;
+
+/**
+ * Test which exercises the indexed disk cache. This one uses three different
+ * regions for thre threads.
+ * 
+ * @version $Id: TestDiskCache.java 224346 2005-06-04 02:01:59Z asmuts $
+ */
+public class HSQLDiskCacheConcurrentUnitTest
+    extends TestCase
+{
+    /**
+     * Number of items to cache, twice the configured maxObjects for the memory
+     * cache regions.
+     */
+    private static int items = 100;
+
+    /**
+     * Constructor for the TestDiskCache object.
+     * 
+     * @param testName
+     */
+    public HSQLDiskCacheConcurrentUnitTest( String testName )
+    {
+        super( testName );
+    }
+
+    /**
+     * Main method passes this test to the text test runner.
+     * 
+     * @param args
+     */
+    public static void main( String args[] )
+    {
+        String[] testCaseName = { HSQLDiskCacheConcurrentUnitTest.class.getName() };
+        junit.textui.TestRunner.main( testCaseName );
+    }
+
+    /**
+     * A unit test suite for JUnit
+     * 
+     * @return The test suite
+     */
+    public static Test suite()
+    {
+        ActiveTestSuite suite = new ActiveTestSuite();
+
+        suite.addTest( new HSQLDiskCacheConcurrentUnitTest( "testHSQLDiskCache1" )
+        {
+            public void runTest()
+                throws Exception
+            {
+                this.runTestForRegion( "indexedRegion1" );
+            }
+        } );
+
+        suite.addTest( new HSQLDiskCacheConcurrentUnitTest( "testHSQLDiskCache2" )
+        {
+            public void runTest()
+                throws Exception
+            {
+                this.runTestForRegion( "indexedRegion2" );
+            }
+        } );
+
+        suite.addTest( new HSQLDiskCacheConcurrentUnitTest( "testHSQLDiskCache3" )
+        {
+            public void runTest()
+                throws Exception
+            {
+                this.runTestForRegion( "indexedRegion3" );
+            }
+        } );
+
+        return suite;
+    }
+
+    /**
+     * Test setup
+     */
+    public void setUp()
+    {
+        JCS.setConfigFilename( "/TestHSQLDiskCache.ccf" );
+    }
+
+    /**
+     * Adds items to cache, gets them, and removes them. The item count is more
+     * than the size of the memory cache, so items should spool to disk.
+     * 
+     * @param region
+     *            Name of the region to access
+     * 
+     * @exception Exception
+     *                If an error occurs
+     */
+    public void runTestForRegion( String region )
+        throws Exception
+    {
+        JCS jcs = JCS.getInstance( region );
+
+        // Add items to cache
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            jcs.put( i + ":key", region + " data " + i );
+        }
+
+        System.out.println( jcs.getStats() );
+
+        // Thread.sleep( 1000 );
+
+        // System.out.println( jcs.getStats() );
+
+        // Test that all items are in cache
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            String value = (String) jcs.get( i + ":key" );
+
+            assertEquals( "key = [" + i + ":key] value = [" + value + "]", region + " data " + i, value );
+        }
+
+        // Remove all the items
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            jcs.remove( i + ":key" );
+        }
+
+        // Verify removal
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            assertNull( "Removed key should be null: " + i + ":key", jcs.get( i + ":key" ) );
+        }
+    }
+}

Added: jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheUnitTest.java
URL: http://svn.apache.org/viewcvs/jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheUnitTest.java?rev=386493&view=auto
==============================================================================
--- jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheUnitTest.java (added)
+++ jakarta/jcs/trunk/src/test/org/apache/jcs/auxiliary/disk/jdbc/hsql/HSQLDiskCacheUnitTest.java Thu Mar 16 16:16:40 2006
@@ -0,0 +1,173 @@
+package org.apache.jcs.auxiliary.disk.jdbc.hsql;
+
+/*
+ * 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.framework.TestCase;
+
+import org.apache.jcs.JCS;
+import org.apache.jcs.access.exception.CacheException;
+
+/**
+ * Test which exercises the indexed disk cache. This one uses three different
+ * regions for thre threads.
+ * 
+ * @version $Id: TestDiskCache.java 224346 2005-06-04 02:01:59Z asmuts $
+ */
+public class HSQLDiskCacheUnitTest
+    extends TestCase
+{
+    /**
+     * Test setup
+     */
+    public void setUp()
+    {
+        JCS.setConfigFilename( "/TestHSQLDiskCache.ccf" );
+    }
+
+    /**
+     * Adds items to cache, gets them, and removes them. The item count is more
+     * than the size of the memory cache, so items should spool to disk.
+     * 
+     * @param region
+     *            Name of the region to access
+     * 
+     * @exception Exception
+     *                If an error occurs
+     */
+    public void testBasicPutRemove()
+        throws Exception
+    {
+        int items = 200;
+
+        String region = "testCache";
+
+        JCS jcs = JCS.getInstance( region );
+
+        // Add items to cache
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            jcs.put( i + ":key", region + " data " + i );
+        }
+
+        System.out.println( jcs.getStats() );
+
+        // Thread.sleep( 1000 );
+
+        // System.out.println( jcs.getStats() );
+
+        // Test that all items are in cache
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            String value = (String) jcs.get( i + ":key" );
+
+            assertEquals( "key = [" + i + ":key] value = [" + value + "]", region + " data " + i, value );
+        }
+
+        // Remove all the items
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            jcs.remove( i + ":key" );
+        }
+
+        // Verify removal
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            assertNull( "Removed key should be null: " + i + ":key", jcs.get( i + ":key" ) );
+        }
+    }
+
+    /**
+     * Verify that remove all work son a region where it is not prohibited.
+     * 
+     * @throws CacheException
+     * @throws InterruptedException 
+     * 
+     */
+    public void testRemoveAll()
+        throws CacheException, InterruptedException
+    {
+        String region = "removeAllAllowed";
+        JCS jcs = JCS.getInstance( region );
+
+        int items = 200;
+
+        // Add items to cache
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            jcs.put( i + ":key", region + " data " + i );
+        }
+
+        // a db thread could be updating when we call remove all?
+        // there was a race on remove all, an element may be put to disk after it is called even though the put
+        // was called before clear.  
+        // I discovered it and removed it.
+        //Thread.sleep( 500 );
+        
+        System.out.println( jcs.getStats() );
+        
+        jcs.clear();
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            String value = (String) jcs.get( i + ":key" );
+
+            assertNull( "value should be null key = [" + i + ":key] value = [" + value + "]",  value );
+        }
+    }
+
+    /**
+     * Verify that remove all does not work on a region where it is prohibited.
+     * 
+     * @throws CacheException
+     * @throws InterruptedException 
+     * 
+     */
+    public void testRemoveAllProhibition()
+        throws CacheException, InterruptedException
+    {
+        String region = "noRemoveAll";
+        JCS jcs = JCS.getInstance( region );
+
+        int items = 200;
+
+        // Add items to cache
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            jcs.put( i + ":key", region + " data " + i );
+        }
+
+        // a db thread could be updating the disk when 
+        //Thread.sleep( 500 );
+        
+        System.out.println( jcs.getStats() );
+        
+        jcs.clear();
+
+        for ( int i = 0; i <= items; i++ )
+        {
+            String value = (String) jcs.get( i + ":key" );
+
+            assertEquals( "key = [" + i + ":key] value = [" + value + "]", region + " data " + i, value );
+        }
+    }
+}



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