You are viewing a plain text version of this content. The canonical link for it is here.
Posted to ojb-dev@db.apache.org by mk...@apache.org on 2005/04/30 22:55:15 UTC

cvs commit: db-ojb/src/test/org/apache/ojb repository.dtd repository_database.xml

mkalen      2005/04/30 13:55:15

  Modified:    src/java/org/apache/ojb/broker/accesslayer Tag:
                        OJB_1_0_RELEASE ConnectionFactoryAbstractImpl.java
                        ConnectionFactoryDBCPImpl.java
                        ConnectionFactoryPooledImpl.java
               src/java/org/apache/ojb/broker/metadata Tag: OJB_1_0_RELEASE
                        ConnectionPoolDescriptor.java
               src/java/org/apache/ojb/broker/util/pooling Tag:
                        OJB_1_0_RELEASE PoolConfiguration.java
               src/test/org/apache/ojb Tag: OJB_1_0_RELEASE repository.dtd
                        repository_database.xml
  Added:       src/java/org/apache/ojb/broker/util/logging Tag:
                        OJB_1_0_RELEASE LoggerWrapperPrintWriter.java
  Log:
  ConnectionFactory adjustments: make DBCP AbandonedConfig work (!), options for DBCP Statement cache and JIRA OJB-37 connection unwrapping. Make the new Commons Pool-parameter "minActive" configurable in OJB. Improve pooling docs in DTD. Remove "homebrewed" validation from CFAbstractImpl, let Commons Pool API handle everything.
  
  Revision  Changes    Path
  No                   revision
  No                   revision
  1.10.2.5  +3 -2      db-ojb/src/java/org/apache/ojb/broker/accesslayer/ConnectionFactoryAbstractImpl.java
  
  Index: ConnectionFactoryAbstractImpl.java
  ===================================================================
  RCS file: /home/cvs/db-ojb/src/java/org/apache/ojb/broker/accesslayer/ConnectionFactoryAbstractImpl.java,v
  retrieving revision 1.10.2.4
  retrieving revision 1.10.2.5
  diff -u -r1.10.2.4 -r1.10.2.5
  --- ConnectionFactoryAbstractImpl.java	26 Apr 2005 03:41:35 -0000	1.10.2.4
  +++ ConnectionFactoryAbstractImpl.java	30 Apr 2005 20:55:15 -0000	1.10.2.5
  @@ -84,7 +84,8 @@
        * @param con the connection to release.
        * Callers <em>must</em> guarantee that the passed connection was obtained by calling
        * {@link #checkOutJdbcConnection(org.apache.ojb.broker.metadata.JdbcConnectionDescriptor)}.
  -     * @throws LookupException //TODO: mkalen: document when. Rename from "LookupException"?
  +     * @throws LookupException if errors occured during release of object. Typically happens
  +     * if return of object to pool fails in a pooled implementation.
        */
       public abstract void releaseJdbcConnection(JdbcConnectionDescriptor jcd, Connection con)
               throws LookupException;
  
  
  
  1.10.2.4  +174 -61   db-ojb/src/java/org/apache/ojb/broker/accesslayer/ConnectionFactoryDBCPImpl.java
  
  Index: ConnectionFactoryDBCPImpl.java
  ===================================================================
  RCS file: /home/cvs/db-ojb/src/java/org/apache/ojb/broker/accesslayer/ConnectionFactoryDBCPImpl.java,v
  retrieving revision 1.10.2.3
  retrieving revision 1.10.2.4
  diff -u -r1.10.2.3 -r1.10.2.4
  --- ConnectionFactoryDBCPImpl.java	26 Apr 2005 03:41:35 -0000	1.10.2.3
  +++ ConnectionFactoryDBCPImpl.java	30 Apr 2005 20:55:15 -0000	1.10.2.4
  @@ -16,43 +16,50 @@
    */
   
   import org.apache.commons.dbcp.AbandonedConfig;
  +import org.apache.commons.dbcp.AbandonedObjectPool;
   import org.apache.commons.dbcp.DriverManagerConnectionFactory;
   import org.apache.commons.dbcp.PoolableConnectionFactory;
   import org.apache.commons.dbcp.PoolingDataSource;
   import org.apache.commons.pool.KeyedObjectPoolFactory;
   import org.apache.commons.pool.ObjectPool;
  -import org.apache.commons.pool.KeyedPoolableObjectFactory;
  +import org.apache.commons.pool.impl.GenericKeyedObjectPool;
   import org.apache.commons.pool.impl.GenericKeyedObjectPoolFactory;
   import org.apache.commons.pool.impl.GenericObjectPool;
  -import org.apache.commons.pool.impl.GenericKeyedObjectPool;
  +import org.apache.ojb.broker.PBKey;
   import org.apache.ojb.broker.metadata.JdbcConnectionDescriptor;
   import org.apache.ojb.broker.util.ClassHelper;
   import org.apache.ojb.broker.util.logging.Logger;
   import org.apache.ojb.broker.util.logging.LoggerFactory;
  -import org.apache.ojb.broker.PBKey;
  +import org.apache.ojb.broker.util.logging.LoggerWrapperPrintWriter;
   
   import javax.sql.DataSource;
   import java.sql.Connection;
   import java.sql.SQLException;
  -import java.util.HashMap;
  -import java.util.Map;
   import java.util.Collection;
  -import java.util.Iterator;
   import java.util.Collections;
  +import java.util.HashMap;
  +import java.util.Iterator;
  +import java.util.Map;
   import java.util.Properties;
   
   /**
  - * ConnectionFactory implementation using Jakarta DBCP and Commons Pool
  - * to pool driver based connections.
  + * ConnectionFactory implementation using Commons DBCP and Commons Pool API
  + * to pool connections.
    *
    * Based on a proposal of Dirk Verbeek - Thanks.
    *
    * @author <a href="mailto:armin@codeAuLait.de">Armin Waibel</a>
    * @version $Id$
  + * @see <a href="http://jakarta.apache.org/commons/pool/">Commons Pool Website</a>
  + * @see <a href="http://jakarta.apache.org/commons/dbcp/">Commons DBCP Website</a>
    */
   public class ConnectionFactoryDBCPImpl extends ConnectionFactoryAbstractImpl
   {
   
  +    public static final String PARAM_NAME_UNWRAP_ALLOWED = "accessToUnderlyingConnectionAllowed";
  +    public static final String PARAM_NAME_POOL_STATEMENTS = "poolPreparedStatements";
  +    public static final String PARAM_NAME_STATEMENT_POOL_MAX_TOTAL = "maxOpenPreparedStatements";
  +
       private Logger log = LoggerFactory.getLogger(ConnectionFactoryDBCPImpl.class);
   
       /** Key=PBKey, value=ObjectPool. */
  @@ -149,8 +156,8 @@
                       // Setup new object pool
                       ObjectPool pool = setupPool(jcd);
                       poolMap.put(key, pool);
  -                    // Create a DBCP PoolingDataSource from the object pool
  -                    ds = createPoolingDataSource(pool);
  +                    // Wrap the underlying object pool as DataSource
  +                    ds = wrapAsDataSource(jcd, pool);
                       dsMap.put(key, ds);
                   }
               }
  @@ -190,90 +197,169 @@
               log.fatal("Could not find the driver class : " + jcd.getDriver() + " in ConnectionFactoryDBCImpl!" , e);
           }
   
  -
  -        // get the configuration for the connection pool
  +        // Get the configuration for the connection pool
           GenericObjectPool.Config conf = jcd.getConnectionPoolDescriptor().getObjectPoolConfig();
   
  -        // First, we'll need a ObjectPool that serves as the
  -        // actual pool of connections.
  -        ObjectPool connectionPool = createObjectPool(conf);
  -
  -        // Next, we'll create a ConnectionFactory that the
  -        // pool will use to create Connections.
  -        // We'll use the DriverManagerConnectionFactory,
  -        //
  -        org.apache.commons.dbcp.ConnectionFactory connectionFactory = createConnectionFactory(jcd);
  +        // Get the additional abandoned configuration
  +        AbandonedConfig ac = jcd.getConnectionPoolDescriptor().getAbandonedConfig();
  +
  +        // Create the ObjectPool that serves as the actual pool of connections.
  +        final ObjectPool connectionPool = createConnectionPool(conf, ac);
   
  +        // Create a DriverManager-based ConnectionFactory that
  +        // the connectionPool will use to create Connection instances
  +        final org.apache.commons.dbcp.ConnectionFactory connectionFactory;
  +        connectionFactory = createConnectionFactory(jcd);
  +
  +        // Create PreparedStatement object pool (if any)
           KeyedObjectPoolFactory statementPoolFactory = createStatementPoolFactory(jcd);
  -        // set the validation query
  -        String validationQuery = jcd.getConnectionPoolDescriptor().getValidationQuery();
  -        boolean defaultReadOnly = false;
  -        // set autocommit mode
  -        boolean defaultAutoCommit = (jcd.getUseAutoCommit() == JdbcConnectionDescriptor.AUTO_COMMIT_SET_FALSE) ?
  -                false : true;
   
  -        // Abandoned configuration
  -        AbandonedConfig ac = jcd.getConnectionPoolDescriptor().getAbandonedConfig();
  +        // Set validation query and auto-commit mode
  +        final String validationQuery;
  +        final boolean defaultAutoCommit;
  +        final boolean defaultReadOnly = false;
  +        validationQuery = jcd.getConnectionPoolDescriptor().getValidationQuery();
  +        defaultAutoCommit = (jcd.getUseAutoCommit() == JdbcConnectionDescriptor.AUTO_COMMIT_SET_FALSE) ?
  +                false : true;
   
           //
           // Now we'll create the PoolableConnectionFactory, which wraps
           // the "real" Connections created by the ConnectionFactory with
           // the classes that implement the pooling functionality.
           //
  -        PoolableConnectionFactory poolableConnectionFactory = new PoolableConnectionFactory(
  -                connectionFactory,
  +        final PoolableConnectionFactory poolableConnectionFactory;
  +        poolableConnectionFactory = new PoolableConnectionFactory(connectionFactory,
                   connectionPool,
                   statementPoolFactory,
                   validationQuery,
                   defaultReadOnly,
                   defaultAutoCommit,
                   ac);
  -
           return poolableConnectionFactory.getPool();
       }
   
  -    protected ObjectPool createObjectPool(GenericObjectPool.Config config)
  +    protected ObjectPool createConnectionPool(GenericObjectPool.Config config,
  +                                              AbandonedConfig ac)
       {
  -        // We'll use a GenericObjectPool instance, although
  -        // any ObjectPool implementation will suffice.
  -        //
  -        // TODO: make objectPool configurable at runtime?
  -        return new GenericObjectPool(null, config);
  +        final GenericObjectPool connectionPool;
  +        final boolean doRemoveAbandoned = ac != null && ac.getRemoveAbandoned() == true;
  +
  +        if (doRemoveAbandoned) {
  +            connectionPool = new AbandonedObjectPool(null, ac);
  +        } else {
  +            connectionPool = new GenericObjectPool();
  +        }
  +        connectionPool.setMaxActive(config.maxActive);
  +        connectionPool.setMaxIdle(config.maxIdle);
  +        connectionPool.setMinIdle(config.minIdle);
  +        connectionPool.setMaxWait(config.maxWait);
  +        connectionPool.setTestOnBorrow(config.testOnBorrow);
  +        connectionPool.setTestOnReturn(config.testOnReturn);
  +        connectionPool.setTimeBetweenEvictionRunsMillis(config.timeBetweenEvictionRunsMillis);
  +        connectionPool.setNumTestsPerEvictionRun(config.numTestsPerEvictionRun);
  +        connectionPool.setMinEvictableIdleTimeMillis(config.minEvictableIdleTimeMillis);
  +        connectionPool.setTestWhileIdle(config.testWhileIdle);
  +        return connectionPool;
       }
   
       protected KeyedObjectPoolFactory createStatementPoolFactory(JdbcConnectionDescriptor jcd)
       {
           final String platform = jcd.getDbms();
  -        if (platform.equals("Oracle9i"))
  +        if (platform.startsWith("Oracle9i"))
           {
               // mkalen: let the platform set Oracle-specific statement pooling
               return null;
           }
   
  -        final KeyedObjectPoolFactory stmtPoolFactory;
  -        final KeyedPoolableObjectFactory objectFactory = null;
  -        final GenericKeyedObjectPool.Config factoryConfig = new GenericKeyedObjectPool.Config();
  -        /*
  -        // TODO: mkalen: allow to configure PreparedStatement pool
  -        final int maxTotalStmts = 100;
  -        factoryConfig.maxActive = (int) (maxTotalStmts * 0.7);
  -        factoryConfig.maxIdle = (int) (maxTotalStmts * 0.3);
  -        factoryConfig.maxTotal = maxTotalStmts;
  -        factoryConfig.testOnBorrow = true;
  -        factoryConfig.testWhileIdle = true;
  -        factoryConfig.testOnReturn = true;
  -        factoryConfig.numTestsPerEvictionRun = factoryConfig.maxTotal;
  -        factoryConfig.minEvictableIdleTimeMillis = 30 * 1000;
  -        */
  -        stmtPoolFactory = new GenericKeyedObjectPoolFactory(objectFactory, factoryConfig);
  -        return stmtPoolFactory;
  +        // Set up statement pool, if desired
  +        GenericKeyedObjectPoolFactory statementPoolFactory = null;
  +        final Properties properties = jcd.getConnectionPoolDescriptor().getDbcpProperties();
  +        final String poolStmtParam = properties.getProperty(PARAM_NAME_POOL_STATEMENTS);
  +        if (poolStmtParam != null && Boolean.valueOf(poolStmtParam).booleanValue())
  +        {
  +            final String maxOpenPrepStmtString;
  +
  +            int maxOpenPreparedStatements = GenericKeyedObjectPool.DEFAULT_MAX_TOTAL;
  +            maxOpenPrepStmtString = properties.getProperty(PARAM_NAME_STATEMENT_POOL_MAX_TOTAL);
  +            if (maxOpenPrepStmtString != null)
  +            {
  +                maxOpenPreparedStatements = Integer.parseInt(maxOpenPrepStmtString);
  +            }
  +            // Use the same values as Commons DBCP BasicDataSource
  +            statementPoolFactory = new GenericKeyedObjectPoolFactory(null,
  +                        -1, // unlimited maxActive (per key)
  +                        GenericKeyedObjectPool.WHEN_EXHAUSTED_FAIL,
  +                        0, // maxWait
  +                        1, // maxIdle (per key)
  +                        maxOpenPreparedStatements);
  +        }
  +        return statementPoolFactory;
       }
   
  -    protected PoolingDataSource createPoolingDataSource(ObjectPool pool)
  +    /**
  +     * Wraps the specified object pool for connections as a DataSource.
  +     *
  +     * @param jcd the OJB connection descriptor for the pool to be wrapped
  +     * @param connectionPool the connection pool to be wrapped
  +     * @return a DataSource attached to the connection pool.
  +     * Connections will be wrapped using DBCP PoolGuard, that will not allow
  +     * unwrapping unless the "accessToUnderlyingConnectionAllowed=true" configuration
  +     * is specified.
  +     */
  +    protected DataSource wrapAsDataSource(JdbcConnectionDescriptor jcd,
  +                                          ObjectPool connectionPool)
       {
  -        return new PoolingDataSource(pool);
  +        final boolean allowConnectionUnwrap;
  +        if (jcd == null)
  +        {
  +            allowConnectionUnwrap = false;
  +        }
  +        else
  +        {
  +            final Properties properties = jcd.getConnectionPoolDescriptor().getDbcpProperties();
  +            final String allowConnectionUnwrapParam;
  +            allowConnectionUnwrapParam = properties.getProperty(PARAM_NAME_UNWRAP_ALLOWED);
  +            if (allowConnectionUnwrapParam != null &&
  +                Boolean.valueOf(allowConnectionUnwrapParam).booleanValue())
  +            {
  +                allowConnectionUnwrap = true;
  +            }
  +            else
  +            {
  +                allowConnectionUnwrap = false;
  +            }
  +        }
  +        final PoolingDataSource dataSource;
  +        dataSource = new PoolingDataSource(connectionPool);
  +        dataSource.setAccessToUnderlyingConnectionAllowed(allowConnectionUnwrap);
  +
  +        final AbandonedConfig ac = jcd.getConnectionPoolDescriptor().getAbandonedConfig();
  +        if (ac.getRemoveAbandoned() && ac.getLogAbandoned()) {
  +            final LoggerWrapperPrintWriter loggerPiggyBack;
  +            loggerPiggyBack = new LoggerWrapperPrintWriter(log, Logger.ERROR);
  +            dataSource.setLogWriter(loggerPiggyBack);
  +        }
  +        return dataSource;
       }
   
  +    /**
  +     * Creates a DriverManager-based ConnectionFactory for creating the Connection
  +     * instances to feed into the object pool of the specified jcd-alias.
  +     * <p>
  +     * <b>NB!</b> If you override this method to specify your own ConnectionFactory
  +     * you <em>must</em> make sure that you follow OJB's lifecycle contract defined in the
  +     * {@link org.apache.ojb.broker.platforms.Platform} API - ie that you call
  +     * initializeJdbcConnection when a new Connection is created. For convenience, use
  +     * {@link ConnectionFactoryAbstractImpl#initializeJdbcConnection} instead of Platform call.
  +     * <p>
  +     * The above is automatically true if you re-use the inner class {@link ConPoolFactory}
  +     * below and just override this method for additional user-defined "tweaks".
  +     *
  +     * @param jcd the jdbc-connection-alias for which we are creating a ConnectionFactory
  +     * @return a DriverManager-based ConnectionFactory that creates Connection instances
  +     * using DriverManager, and that follows the lifecycle contract defined in OJB
  +     * {@link org.apache.ojb.broker.platforms.Platform} API.
  +     */
       protected org.apache.commons.dbcp.ConnectionFactory createConnectionFactory(JdbcConnectionDescriptor jcd)
       {
           final ConPoolFactory result;
  @@ -282,15 +368,42 @@
           return result;
       }
   
  +    // ----- deprecated methods, to be removed -----
  +
  +    /**
  +     * mkalen: Left for binary API-compatibility with OJB 1.0.3 (don't break users' factories)
  +     * @deprecated since OJB 1.0.4,
  +     * please use {@link #createConnectionPool(org.apache.commons.pool.impl.GenericObjectPool.Config, org.apache.commons.dbcp.AbandonedConfig)}
  +     */
  +    protected ObjectPool createObjectPool(GenericObjectPool.Config config)
  +    {
  +        return createConnectionPool(config, null);
  +    }
  +
  +    /**
  +     * mkalen: Left for binary API-compatibility with OJB 1.0.3 (don't break users' factories)
  +     * @deprecated since OJB 1.0.4,
  +     * please use {@link #wrapAsDataSource(org.apache.ojb.broker.metadata.JdbcConnectionDescriptor, org.apache.commons.pool.ObjectPool)}
  +     */
  +    protected PoolingDataSource createPoolingDataSource(ObjectPool connectionPool)
  +    {
  +        // mkalen: not a nice cast but we do not want to break signature and it is safe
  +        // since any new implementations will not be based on this method and the wrapper-
  +        // call here goes to code we control (where we know it's PoolingDataSource)
  +        return (PoolingDataSource) wrapAsDataSource(null, connectionPool);
  +    }
  +
  +    // ----- end deprecated methods -----
  +
       //**************************************************************************************
       // Inner classes
       //************************************************************************************
   
       /**
  -     * Inner class used as factory for connection pooling.
  +     * Inner class used as factory for DBCP connection pooling.
        * Adhers to OJB platform specification by calling platform-specific init methods
        * on newly created connections.
  -     * @see DriverManagerConnectionFactory
  +     * @see org.apache.ojb.broker.platforms.Platform#initializeJdbcConnection
        */
       class ConPoolFactory extends DriverManagerConnectionFactory
       {
  
  
  
  1.15.2.6  +25 -17    db-ojb/src/java/org/apache/ojb/broker/accesslayer/ConnectionFactoryPooledImpl.java
  
  Index: ConnectionFactoryPooledImpl.java
  ===================================================================
  RCS file: /home/cvs/db-ojb/src/java/org/apache/ojb/broker/accesslayer/ConnectionFactoryPooledImpl.java,v
  retrieving revision 1.15.2.5
  retrieving revision 1.15.2.6
  diff -u -r1.15.2.5 -r1.15.2.6
  --- ConnectionFactoryPooledImpl.java	26 Apr 2005 03:41:35 -0000	1.15.2.5
  +++ ConnectionFactoryPooledImpl.java	30 Apr 2005 20:55:15 -0000	1.15.2.6
  @@ -32,14 +32,16 @@
   import java.util.HashMap;
   import java.util.Iterator;
   import java.util.Map;
  +import java.util.NoSuchElementException;
   
   /**
    * Connection factory which pools the requested
    * connections for different JdbcConnectionDescriptors
  - * using jakarta-commons-pool api.
  + * using Commons Pool API.
    *
    * @author <a href="mailto:armin@codeAuLait.de">Armin Waibel</a>.
    * @version $Id$
  + * @see <a href="http://jakarta.apache.org/commons/pool/">Commons Pool Website</a>
    */
   public class ConnectionFactoryPooledImpl extends ConnectionFactoryAbstractImpl
   {
  @@ -50,21 +52,20 @@
       public void releaseJdbcConnection(JdbcConnectionDescriptor jcd, Connection con)
               throws LookupException
       {
  +        final ObjectPool op = (ObjectPool) connectionPools.get(jcd.getPBKey());
           try
           {
  -            /**
  -             * Patch by Ilkka Priha
  -             * To avoid problems with closed connections we only return unclosed
  -             * connections to pool. We do this test independent from the from the
  -             * commons-pool settings, which also supports validation on return of
  -             * a connecion.
  -             * TODO: mkalen: this breaks the ObjectPool API contract and should only be
  -             * TODO: mkalen: handled when using testOnReturn=true, awaiting user feedback to remove
  +            /* mkalen: NB - according to the Commons Pool API we should _not_ perform
  +             * any additional checks here since we will then break testOnX semantics
  +             *
  +             * To enable Connection validation on releaseJdbcConnection,
  +             * set a validation query and specify testOnRelease=true
  +             *
  +             * Destruction of pooled objects is performed by the actual Commons Pool
  +             * ObjectPool implementation when the object factory's validateObject method
  +             * returns false. See ConPoolFactory#validateObject.
                */
  -            if (!con.isClosed())
  -            {
  -                ((ObjectPool) this.connectionPools.get(jcd.getPBKey())).returnObject(con);
  -            }
  +            op.returnObject(con);
           }
           catch (Exception e)
           {
  @@ -89,6 +90,13 @@
           {
               conn = (Connection) op.borrowObject();
           }
  +        catch (NoSuchElementException e)
  +        {
  +            throw new LookupException("Could not borrow connection from pool, " +
  +                                      "ObjectPool exhausted for jcd-alias " + jcd.getJcdAlias() +
  +                                      " Active/Idle instances=" + op.getNumActive() + "/" +
  +                                      op.getNumIdle());
  +        }
           catch (Exception e)
           {
               throw new LookupException("Could not borrow connection from pool - " +
  @@ -146,9 +154,9 @@
        */
       class ConPoolFactory extends BasePoolableObjectFactory
       {
  -        int failedValidationQuery;
  -        JdbcConnectionDescriptor jcd;
  -        ConnectionFactoryPooledImpl cf;
  +        final private JdbcConnectionDescriptor jcd;
  +        final private ConnectionFactoryPooledImpl cf;
  +        private int failedValidationQuery;
   
           public ConPoolFactory(ConnectionFactoryPooledImpl cf, JdbcConnectionDescriptor jcd)
           {
  
  
  
  No                   revision
  No                   revision
  1.14.2.2  +31 -3     db-ojb/src/java/org/apache/ojb/broker/metadata/Attic/ConnectionPoolDescriptor.java
  
  Index: ConnectionPoolDescriptor.java
  ===================================================================
  RCS file: /home/cvs/db-ojb/src/java/org/apache/ojb/broker/metadata/Attic/ConnectionPoolDescriptor.java,v
  retrieving revision 1.14.2.1
  retrieving revision 1.14.2.2
  diff -u -r1.14.2.1 -r1.14.2.2
  --- ConnectionPoolDescriptor.java	26 Apr 2005 03:41:36 -0000	1.14.2.1
  +++ ConnectionPoolDescriptor.java	30 Apr 2005 20:55:15 -0000	1.14.2.2
  @@ -39,9 +39,20 @@
       /** String prefix for JDBC properties passed to DriverManager. */
       public static final String JDBC_PROPERTY_NAME_PREFIX = "jdbc.";
       private static final int JDBC_PROPERTY_NAME_LENGTH = JDBC_PROPERTY_NAME_PREFIX.length();
  +    /**
  +     * String prefix for DBCP properties.
  +     * Currently OJB only uses this for setting DBCP parameters for pooling of Statement,
  +     * not the max/test-parameters etc for the DBCP Connection pool
  +     * (since there is only a JDBC2.0+ version of the Basic-classes ie BasicDataSource
  +     *  and no DriverManager-based one).
  +     */
  +    public static final String DBCP_PROPERTY_NAME_PREFIX = "dbcp.";
  +    private static final int DBCP_PROPERTY_NAME_LENGTH = DBCP_PROPERTY_NAME_PREFIX.length();
   
       /** JDBC properties configured in OJB (not used for DataSource connections). */
       protected Properties jdbcProperties;
  +    /** DBCP Statement cache properties configured in OJB (not used for DataSource connections). */
  +    protected Properties dbcpProperties;
   
       /** Configuration attribute name for JDBC fetchSize hint. */
       public static final String FETCH_SIZE = "fetchSize";
  @@ -61,6 +72,7 @@
       private void init()
       {
           jdbcProperties = new Properties();
  +        dbcpProperties = new Properties();
           setFetchSize(0);
           this.setTestOnBorrow(true);
           this.setTestOnReturn(false);
  @@ -110,11 +122,22 @@
       }
   
       /**
  +     * Returns the DBCP properties to be used for Statement caching
  +     * when creating DBCP connection pool in OJB ConnectionFactory.
  +     * @return DBCP properties (might be empty, never null)
  +     */
  +    public Properties getDbcpProperties()
  +    {
  +        return dbcpProperties;
  +    }
  +
  +    /**
        * Sets a custom configuration attribute.
        * @param attributeName the attribute name. Names starting with
  -     * {@link @JDBC_PROPERTY_NAME_PREFIX} will be used (without the prefix) by the
  +     * {@link #JDBC_PROPERTY_NAME_PREFIX} will be used (without the prefix) by the
        * ConnectionFactory when creating connections from DriverManager
  -     * (not used for DataSource connections).
  +     * (not used for external DataSource connections). Names starting with
  +     * {@link #DBCP_PROPERTY_NAME_PREFIX} to Commons DBCP (if used, also without prefix).
        * @param attributeValue the attribute value
        */
       public void addAttribute(String attributeName, String attributeValue)
  @@ -124,6 +147,11 @@
               final String jdbcPropertyName = attributeName.substring(JDBC_PROPERTY_NAME_LENGTH);
               jdbcProperties.setProperty(jdbcPropertyName, attributeValue);
           }
  +        else if (attributeName != null && attributeName.startsWith(DBCP_PROPERTY_NAME_PREFIX))
  +        {
  +            final String dbcpPropertyName = attributeName.substring(DBCP_PROPERTY_NAME_LENGTH);
  +            dbcpProperties.setProperty(dbcpPropertyName, attributeValue);
  +        }
           else
           {
               super.addAttribute(attributeName, attributeValue);
  
  
  
  No                   revision
  No                   revision
  1.1.2.1   +273 -0    db-ojb/src/java/org/apache/ojb/broker/util/logging/Attic/LoggerWrapperPrintWriter.java
  
  
  
  
  No                   revision
  No                   revision
  1.11.2.3  +15 -2     db-ojb/src/java/org/apache/ojb/broker/util/pooling/PoolConfiguration.java
  
  Index: PoolConfiguration.java
  ===================================================================
  RCS file: /home/cvs/db-ojb/src/java/org/apache/ojb/broker/util/pooling/PoolConfiguration.java,v
  retrieving revision 1.11.2.2
  retrieving revision 1.11.2.3
  diff -u -r1.11.2.2 -r1.11.2.3
  --- PoolConfiguration.java	26 Apr 2005 03:41:36 -0000	1.11.2.2
  +++ PoolConfiguration.java	30 Apr 2005 20:55:15 -0000	1.11.2.3
  @@ -41,6 +41,7 @@
       //*****************************************************
       public static final String MAX_ACTIVE = "maxActive";
       public static final String MAX_IDLE = "maxIdle";
  +    public static final String MIN_IDLE = "minIdle";
       public static final String MAX_WAIT = "maxWait";
       public static final String WHEN_EXHAUSTED_ACTION = "whenExhaustedAction";
       public static final String TEST_ON_BORROW = "testOnBorrow";
  @@ -59,6 +60,7 @@
       //*****************************************************
       public static final int DEFAULT_MAX_ACTIVE = 21;
       public static final int DEFAULT_MAX_IDLE = -1;
  +    public static final int DEFAULT_MIN_IDLE = 0;
       public static final long DEFAULT_MAX_WAIT = 5000;
       public static final byte DEFAULT_WHEN_EXHAUSTED_ACTION = GenericObjectPool.WHEN_EXHAUSTED_FAIL;
       public static final boolean DEFAULT_TEST_ON_BORROW = true;
  @@ -75,6 +77,7 @@
       {
           this.setMaxActive(DEFAULT_MAX_ACTIVE);
           this.setMaxIdle(DEFAULT_MAX_IDLE);
  +        this.setMinIdle(DEFAULT_MIN_IDLE);
           this.setMaxWait(DEFAULT_MAX_WAIT);
           this.setWhenExhaustedAction(DEFAULT_WHEN_EXHAUSTED_ACTION);
           this.setTestOnBorrow(DEFAULT_TEST_ON_BORROW);
  @@ -105,6 +108,7 @@
           GenericObjectPool.Config conf = new GenericObjectPool.Config();
           conf.maxActive = getMaxActive();
           conf.maxIdle = getMaxIdle();
  +        conf.minIdle = getMinIdle();
           conf.maxWait = getMaxWait();
           conf.minEvictableIdleTimeMillis = getMinEvictableIdleTimeMillis();
           conf.numTestsPerEvictionRun = getNumTestsPerEvictionRun();
  @@ -141,9 +145,9 @@
       public AbandonedConfig getAbandonedConfig()
       {
           AbandonedConfig conf = new AbandonedConfig();
  -        conf.setLogAbandoned(isLogAbandoned());
           conf.setRemoveAbandoned(isRemoveAbandoned());
           conf.setRemoveAbandonedTimeout(getRemoveAbandonedTimeout());
  +        conf.setLogAbandoned(isLogAbandoned());
           return conf;
       }
   
  @@ -224,6 +228,15 @@
           this.setProperty(MAX_IDLE, Integer.toString(maxIdle));
       }
   
  +    public int getMinIdle()
  +    {
  +        return Integer.parseInt(getProperty(MIN_IDLE));
  +    }
  +
  +    public void setMinIdle(int minIdle)
  +    {
  +        this.setProperty(MIN_IDLE, Integer.toString(minIdle));
  +    }
   
       public long getMaxWait()
       {
  
  
  
  No                   revision
  No                   revision
  1.56.2.6  +142 -89   db-ojb/src/test/org/apache/ojb/repository.dtd
  
  Index: repository.dtd
  ===================================================================
  RCS file: /home/cvs/db-ojb/src/test/org/apache/ojb/repository.dtd,v
  retrieving revision 1.56.2.5
  retrieving revision 1.56.2.6
  diff -u -r1.56.2.5 -r1.56.2.6
  --- repository.dtd	26 Apr 2005 03:41:37 -0000	1.56.2.5
  +++ repository.dtd	30 Apr 2005 20:55:15 -0000	1.56.2.6
  @@ -1,4 +1,3 @@
  -<!-- Version: $Id$ -->
   <!--
   #/* Copyright 2004-2005 Apache Software Foundation
   # *
  @@ -16,8 +15,12 @@
   # */
   -->
   <!--
  -	ObJectRelationalBridge - Bridging Java objects and relational dabatases
  +	Apache OJB - ObJectRelationalBridge
  +
  +        Bridging Java objects and relational dabatases
  +
   	This DTD describes the grammar of the Descriptor repository
  +
   	Initial author: Thomas Mahler
     -->
   
  @@ -139,27 +142,27 @@
       using PBKey.java to obtain a PersistenceBroker
     -->
   <!ATTLIST jdbc-connection-descriptor
  -	jcd-alias CDATA #REQUIRED
  +    jcd-alias CDATA #REQUIRED
       default-connection (true | false) "false"
  -    platform (Db2 | Hsqldb | Informix | MsAccess | MsSQLServer |
  -	          MySQL | Oracle | PostgreSQL | Sybase | SybaseASE |
  -              SybaseASA | Sapdb | Firebird | Axion | NonstopSql |
  -              Oracle9i | MaxDB ) "Hsqldb"
  -	jdbc-level (1.0 | 2.0 | 3.0) "1.0"
  -	eager-release (true | false) "false"
  +    platform (  Db2 | Hsqldb | Informix | MsAccess | MsSQLServer |
  +                MySQL | Oracle | PostgreSQL | Sybase | SybaseASE |
  +                SybaseASA | Sapdb | Firebird | Axion | NonstopSql |
  +                Oracle9i | MaxDB ) "Hsqldb"
  +    jdbc-level (1.0 | 2.0 | 3.0) "1.0"
  +    eager-release (true | false) "false"
       batch-mode (true | false) "false"
       useAutoCommit (0 | 1 | 2) "1"
       ignoreAutoCommitExceptions (true | false) "false"
   
  -	jndi-datasource-name CDATA #IMPLIED
  +    jndi-datasource-name CDATA #IMPLIED
   
  -	driver CDATA #IMPLIED
  -	protocol CDATA #IMPLIED
  -	subprotocol CDATA #IMPLIED
  -	dbalias CDATA #IMPLIED
  +    driver CDATA #IMPLIED
  +    protocol CDATA #IMPLIED
  +    subprotocol CDATA #IMPLIED
  +    dbalias CDATA #IMPLIED
   
  -	username CDATA #IMPLIED
  -	password CDATA #IMPLIED
  +    username CDATA #IMPLIED
  +    password CDATA #IMPLIED
   >
   
   
  @@ -201,73 +204,94 @@
   
   
   <!--
  -	maxActive
  -	maximum number of connections that can be borrowed from the
  -	pool at one time. When non-positive, there is no limit.
  -
  -	maxIdle
  -	controls the maximum number of connections that can sit idle in the
  -	pool at any time. When non-positive, there is no limit
  -
  -	maxWait
  -	max time block to get connection instance from pool, after that exception is thrown.
  -	When non-positive, block till last judgement
  -
  -	whenExhaustedAction
  -	0 - fail when pool is exhausted
  -	1 - block when pool is exhausted
  -	2 - grow when pool is exhausted
  -
  -	testOnBorrow
  -	The pool will attempt to validate each object before it is returned from the pool
  -
  -	testOnReturn
  -	The pool will attempt to validate each object before it is returned to the pool
  -
  -	testWhileIdle
  -	Indicates whether or not idle objects should be validated.
  -	Objects that fail to validate will be dropped from the pool
  -
  -	timeBetweenEvictionRunsMillis
  -	indicates how long the eviction thread should sleep before "runs" of examining
  -	idle objects. When non-positive, no eviction thread will be launched.
  -
  -	minEvictableIdleTimeMillis
  -	specifies the minimum amount of time that a connection may sit idle
  -	in the pool before it is eligable for eviction due to idle time.
  -	When non-positive, no connection will be dropped from the pool due
  -	to idle time alone (depends on timeBetweenEvictionRunsMillis > 0)
  -
  -	numTestsPerEvictionRun
  -	The number of connections to examine during each run of the
  -	idle object evictor thread (if any)
  -
  -    validationQuery
  -    Here you could specify a validation query used by pool to test a
  -    obtained connection (e.g. "select 1 from dual"), else a default query was
  -    used - if defined in the platform class for your database.
  -
  -    logAbandoned
  -    Only supported when using
  -    org.apache.ojb.broker.accesslayer.ConnectionFactoryDBCPImpl
  -    ConnectionFactory implementation.
  -    Flag to log stack traces for application code which abandoned
  -    a Statement or Connection. Defaults to false. Logging of
  -    abandoned Statements and Connections adds overhead for
  -    every Connection open or new Statement because a
  -    stack trace has to be generated.
  -
  -    removeAbandoned/removeAbandonedTimeout
  -    Only supported when using
  -    org.apache.ojb.broker.accesslayer.ConnectionFactoryDBCPImpl
  -    ConnectionFactory implementation.
  +    maxActive (default=21)
  +    The maximum number of active connections that can be allocated from this
  +    pool at the same time, or zero for no limit.
  +
  +    maxIdle (default=-1)
  +    The maximum number of active connections that can remain idle in the pool,
  +    without extra ones being released, or zero for no limit.
  +
  +    minIdle (default=0)
  +    The minimum number of active connections that can remain idle in the pool,
  +    without extra ones being created, or zero to create none.
  +    Since: 1.0.4
  +
  +    maxWait (default=5000)
  +    The maximum number of milliseconds that the pool will wait (when there
  +    are no available connections) for a connection to be returned before
  +    throwing an exception, or -1 to wait indefinitely.
  +    Must be > 0 for timeout to actually happen in DBCP PoolingDataSource.
  +
  +    whenExhaustedAction (default=0/WHEN_EXHAUSTED_FAIL)
  +    0 - fail when pool is exhausted
  +    1 - block when pool is exhausted
  +    2 - grow when pool is exhausted
  +
  +    validationQuery (default=not specified)
  +    The SQL query that will be used to validate connections from this pool
  +    before returning them to the caller. If specified, this query MUST be
  +    an SQL SELECT statement that returns at least one row.
  +    Eg:
  +        Oracle      SELECT 1 FROM DUAL
  +        PostgreSQL  SELECT 1
  +        MySQL       SELECT 1
  +
  +    testOnBorrow (default=true)
  +    The indication of whether connections will be validated before being
  +    borrowed from the pool. If the connection fails to validate, it will
  +    be dropped from the pool, and OJB will attempt to borrow another.
  +
  +    testOnReturn (default=false)
  +    The indication of whether connections will be validated before being
  +    returned to the pool.
  +
  +    testWhileIdle (default=false)
  +    The indication of whether connections will be validated by the idle object
  +    evictor (if any). If a connection fails to validate, it will be dropped
  +    from the pool.
  +
  +    timeBetweenEvictionRunsMillis (default=-1)
  +    The number of milliseconds to sleep between runs of the idle
  +    object evictor thread. When non-positive, no idle object evictor thread
  +    will be run.
  +
  +    numTestsPerEvictionRun (default=10)
  +    The number of objects to examine during each run of the idle object
  +    evictor thread (if any).
  +
  +    minEvictableIdleTimeMillis (default=1800000)
  +    The minimum amount of time a connection may sit idle in the pool before
  +    it is eligable for eviction by the idle object evictor (if any).
  +
  +    removeAbandoned [only for ConnectionFactoryDBCPImpl] (default=false)
       Flag to remove abandoned connections if they exceed the
  -    removeAbandonedTimeout. Set to true or false, default false.
  -    If set to true a connection is considered abandoned and
  -    eligible for removal if it has been idle longer than the
  -    removeAbandonedTimeout. Setting this to true can recover
  -    db connections from poorly written applications which
  -    fail to close a connection.
  +    removeAbandonedTimout.
  +    If set to true a connection is considered abandoned and eligible for
  +    removal if it has been idle longer than the removeAbandonedTimeout.
  +    Setting this to true can recover db connections from poorly written
  +    applications which fail to close a connection.
  +    If you have enabled "removeAbandoned" then it is possible that a
  +    connection is reclaimed by the pool because it is considered to be
  +    abandoned. This mechanism is triggered on borrowObject (ie in OJB when
  +    a PersistenceBroker gets a Connection) and:
  +     (numIdle < 2) and (numActive > maxActive - 3).
  +    The abandoned object eviction takes place before normal borrowObject
  +    logic (there is no asynch evictor thread like for testWhileIdle).
  +    For example maxActive=20 and 18 active connections and 1 idle connection
  +    would trigger the "removeAbandoned". But only the active connections that
  +    aren't used for more then "removeAbandonedTimeout" seconds are removed.
  +    Traversing a resultset doesn't count as being used.
  +
  +    removeAbandonedTimeout [only for ConnectionFactoryDBCPImpl] (default=300)
  +    Timeout in seconds before an abandoned connection can be removed.
  +
  +    logAbandoned [only for ConnectionFactoryDBCPImpl] (default=false)
  +    Flag to log stack traces for application code which abandoned a
  +    Statement or Connection.
  +    Logging of abandoned Statements and Connections adds overhead for every
  +    Connection open or new Statement because a stack trace has to be
  +    generated.
   
       Custom attributes:
       <attribute attribute-name="fetchSize" attribute-value="(int)"/>
  @@ -290,23 +314,53 @@
           when creating new Connection objects.
           (Ignored when acquiring Connection objects through
            JNDI DataSource lookup or in managed environments.)
  +
  +     <attribute attribute-name="dbcp.(Commons DBCP-options)"
  +         attribute-value="(value)"/>
  +
  +         If using the ConnectionFactoryDBCPImpl (set in OJB.properties),
  +         all parameters starting with "dbcp." will be made available
  +         (without dbcp. prefix) to the ConnectionFactory for
  +         configuring Commons DBCP accordingly.
  +
  +         Currently the following DBCP attributes are used:
  +
  +            dbcp.poolPreparedStatements (default=false)
  +                Enable prepared statement pooling for this pool.
  +
  +            dbcp.maxOpenPreparedStatements (default=0/unlimited)
  +                The maximum number of open statements that can be
  +                allocated from the statement pool at the same time,
  +                or zero for no limit.
  +
  +             dbcp.accessToUnderlyingConnectionAllowed (default=false)
  +                Controls if the DBCP PoolGuard allows access to the underlying
  +                connection from the JDBC-driver.
  +
  +                Only use when you need direct access to driver specific
  +                extentions.
  +
  +                NOTE: Do not close the underlying connection, only the
  +                original one.
  +
   -->
   <!ATTLIST connection-pool
       maxActive                       CDATA #IMPLIED
  +    minIdle                         CDATA #IMPLIED
       maxIdle                         CDATA #IMPLIED
       maxWait                         CDATA #IMPLIED
       minEvictableIdleTimeMillis      CDATA #IMPLIED
       numTestsPerEvictionRun          CDATA #IMPLIED
  -    testOnBorrow                    (true|false) #IMPLIED
  -    testOnReturn                    (true|false) #IMPLIED
  -    testWhileIdle                   (true|false) #IMPLIED
  +    testOnBorrow                    ( true | false ) #IMPLIED
  +    testOnReturn                    ( true | false ) #IMPLIED
  +    testWhileIdle                   ( true | false ) #IMPLIED
       timeBetweenEvictionRunsMillis   CDATA #IMPLIED
  -    whenExhaustedAction             (0|1|2) #IMPLIED
  +    whenExhaustedAction             ( 0 | 1 | 2 ) #IMPLIED
       validationQuery                 CDATA #IMPLIED
   
  -    logAbandoned                    (true|false) #IMPLIED
  -    removeAbandoned                 (true|false) #IMPLIED
  +    removeAbandoned                 ( true | false ) #IMPLIED
       removeAbandonedTimeout          CDATA #IMPLIED
  +    logAbandoned                    ( true | false ) #IMPLIED
   >
   
   
  @@ -969,4 +1023,3 @@
   <!ATTLIST constant-argument
       value CDATA #REQUIRED
   >
  -
  
  
  
  1.22.2.10 +13 -1     db-ojb/src/test/org/apache/ojb/repository_database.xml
  
  Index: repository_database.xml
  ===================================================================
  RCS file: /home/cvs/db-ojb/src/test/org/apache/ojb/repository_database.xml,v
  retrieving revision 1.22.2.9
  retrieving revision 1.22.2.10
  diff -u -r1.22.2.9 -r1.22.2.10
  --- repository_database.xml	27 Apr 2005 00:23:52 -0000	1.22.2.9
  +++ repository_database.xml	30 Apr 2005 20:55:15 -0000	1.22.2.10
  @@ -89,6 +89,18 @@
               <!-- Attributes with name prefix "jdbc." are passed directly to the JDBC driver. -->
               <!-- Example setting (used by Oracle driver when Statement batching is enabled) -->
               <attribute attribute-name="jdbc.defaultBatchValue" attribute-value="5"/>
  +
  +            <!-- Attributes determining if ConnectionFactoryDBCPImpl
  +                 should also pool PreparedStatement. This is programmatically disabled
  +                 when using platform=Oracle9i since Oracle statement caching will conflict
  +                 with DBCP ObjectPool-based PreparepdStatement caching (ie setting true
  +                 here has no effect for Oracle9i platform). -->
  +            <attribute attribute-name="dbcp.poolPreparedStatements" attribute-value="false"/>
  +            <attribute attribute-name="dbcp.maxOpenPreparedStatements" attribute-value="10"/>
  +            <!-- Attribute determining if the Commons DBCP connection wrapper will allow
  +                 access to the underlying concrete Connection instance from the JDBC-driver
  +                 (normally this is not allowed, like in J2EE-containers using wrappers). -->
  +            <attribute attribute-name="dbcp.accessToUnderlyingConnectionAllowed" attribute-value="false"/>
           </connection-pool>
   
           <!-- alternative sequence manager implementations, see docs section "Sequence Manager" -->
  
  
  

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