You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jackrabbit.apache.org by ma...@apache.org on 2009/09/24 14:48:22 UTC

svn commit: r818466 - in /jackrabbit/sandbox/JCR-1456/jackrabbit-core/src: main/java/org/apache/jackrabbit/core/ main/java/org/apache/jackrabbit/core/config/ main/java/org/apache/jackrabbit/core/data/db/ main/java/org/apache/jackrabbit/core/fs/db/ main...

Author: martijnh
Date: Thu Sep 24 12:48:21 2009
New Revision: 818466

URL: http://svn.apache.org/viewvc?rev=818466&view=rev
Log:
JCR-1456 Database connection pooling

* Use commons-dbcp testWhileIdle validation strategy instead of testOnBorrow
* Made ConnectionFactory repository-local, which required some changes in the configuration mechanismn

Added:
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/DatabaseAware.java   (with props)
Modified:
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/RepositoryImpl.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/BeanConfig.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfig.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfigurationParser.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/data/db/DbDataStore.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/fs/db/DbFileSystem.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/journal/DatabaseJournal.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/BundleDbPersistenceManager.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/db/SimpleDbPersistenceManager.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/ConnectionFactory.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/SecurityConfigTest.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/WorkspaceConfigTest.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/fs/db/DerbyFileSystemTest.java
    jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/util/db/ConnectionFactoryTest.java

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/RepositoryImpl.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/RepositoryImpl.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/RepositoryImpl.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/RepositoryImpl.java Thu Sep 24 12:48:21 2009
@@ -34,8 +34,6 @@
 import java.util.Properties;
 import java.util.Set;
 import java.util.Iterator;
-import java.util.concurrent.Executors;
-import java.util.concurrent.RejectedExecutionHandler;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -105,7 +103,6 @@
 import org.apache.jackrabbit.core.state.ManagedMLRUItemStateCacheFactory;
 import org.apache.jackrabbit.core.state.SharedItemStateManager;
 import org.apache.jackrabbit.core.util.RepositoryLockMechanism;
-import org.apache.jackrabbit.core.util.db.ConnectionFactory;
 import org.apache.jackrabbit.core.version.InternalVersionManager;
 import org.apache.jackrabbit.core.version.InternalVersionManagerImpl;
 import org.apache.jackrabbit.core.xml.ClonedInputSource;
@@ -287,9 +284,6 @@
         boolean succeeded = false;
         try {
             this.repConfig = repConfig;
-
-            // register the configured datasources
-            ConnectionFactory.registerDataSources(repConfig.getDataSourceConfig());
             
             // setup file systems
             repStore = repConfig.getFileSystem();
@@ -1197,6 +1191,8 @@
             log.warn("Interrupted while waiting for background threads", e);
         }
 
+        repConfig.getConnectionFactory().close();
+
         // finally release repository lock
         if (repLock != null) {
             try {

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/BeanConfig.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/BeanConfig.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/BeanConfig.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/BeanConfig.java Thu Sep 24 12:48:21 2009
@@ -17,6 +17,8 @@
 package org.apache.jackrabbit.core.config;
 
 import org.apache.commons.collections.BeanMap;
+import org.apache.jackrabbit.core.util.db.ConnectionFactory;
+import org.apache.jackrabbit.core.util.db.DatabaseAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -79,6 +81,11 @@
     private final Properties properties;
 
     /**
+     * The repositories {@link ConnectionFactory}.
+     */
+    private ConnectionFactory connectionFactory = null;
+
+    /**
      * Flag to validate the configured bean property names against
      * the configured bean class. By default this is <code>true</code>
      * to prevent incorrect property names. However, in some cases this
@@ -116,6 +123,7 @@
      */
     public BeanConfig(BeanConfig config) {
         this(config.getClassName(), config.getParameters());
+        setConnectionFactory(config.connectionFactory);
     }
 
     /**
@@ -129,6 +137,14 @@
     }
 
     /**
+     * @param connectionFactory the {@link ConnectionFactory} to inject (if possible) in the
+     *            {@link #newInstance()} method
+     */
+    public void setConnectionFactory(ConnectionFactory connectionFactory) {
+        this.connectionFactory = connectionFactory;
+    }
+
+    /**
      * Returns the class name of the configured bean.
      *
      * @return class name of the bean
@@ -170,6 +186,10 @@
                 }
             }
 
+            if (object instanceof DatabaseAware) {
+                ((DatabaseAware) object).setConnectionFactory(connectionFactory);
+            }
+
             if (validate) {
                 // Check that no invalid property names were configured
                 Iterator it = properties.keySet().iterator();

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfig.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfig.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfig.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfig.java Thu Sep 24 12:48:21 2009
@@ -26,6 +26,7 @@
 import org.apache.jackrabbit.core.fs.FileSystemPathUtil;
 import org.apache.jackrabbit.core.util.RepositoryLockMechanism;
 import org.apache.jackrabbit.core.util.RepositoryLockMechanismFactory;
+import org.apache.jackrabbit.core.util.db.ConnectionFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Element;
@@ -253,7 +254,7 @@
         variables.setProperty(
                 RepositoryConfigurationParser.REPOSITORY_HOME_VARIABLE, home);
         RepositoryConfigurationParser parser =
-            new RepositoryConfigurationParser(variables);
+            new RepositoryConfigurationParser(variables, new ConnectionFactory());
 
         RepositoryConfig config = parser.parseRepositoryConfig(xml);
         config.init();
@@ -350,6 +351,11 @@
     private final DataSourceConfig dsc;
 
     /**
+     * The {@link ConnectionFactory}
+     */
+    private final ConnectionFactory cf;
+
+    /**
      * Creates a repository configuration object.
      *
      * @param home repository home directory
@@ -366,6 +372,7 @@
      * @param dsf data store factory
      * @param rlf the RepositoryLockMechanismFactory
      * @param dsc the DataSource configuration
+     * @param cf the ConnectionFactory for all DatabasAware beans
      * @param parser configuration parser
      */
     public RepositoryConfig(
@@ -376,6 +383,7 @@
             ClusterConfig cc, DataStoreFactory dsf,
             RepositoryLockMechanismFactory rlf,
             DataSourceConfig dsc,
+            ConnectionFactory cf,
             RepositoryConfigurationParser parser) {
         workspaces = new HashMap<String, WorkspaceConfig>();
         this.home = home;
@@ -392,6 +400,7 @@
         this.dsf = dsf;
         this.rlf = rlf;
         this.dsc = dsc;
+        this.cf = cf;
         this.parser = parser;
     }
 
@@ -404,6 +413,15 @@
      *                               been initialized
      */
     public void init() throws ConfigurationException, IllegalStateException {
+        
+        // This needs to be done here and not by clients (e.g., RepositoryImpl ctor) because
+        // fsf is used below and this might be a DatabaseAware FileSystem
+        try {
+            cf.registerDataSources(dsc);
+        } catch (RepositoryException e) {
+            throw new ConfigurationException("failed to register data sources", e);
+        }
+
         if (!workspaces.isEmpty()) {
             throw new IllegalStateException(
                     "Repository configuration has already been initialized.");
@@ -905,10 +923,11 @@
     }
 
     /**
-     * Returns the datasource configuration.
+     * Returns the {@link ConnectionFactory} for this repository.
+     * Please note that it must be closed explicitly.
      */
-    public DataSourceConfig getDataSourceConfig() {
-        return dsc;
+    public ConnectionFactory getConnectionFactory() {
+        return cf;
     }
 
     /**

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfigurationParser.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfigurationParser.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfigurationParser.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/config/RepositoryConfigurationParser.java Thu Sep 24 12:48:21 2009
@@ -27,6 +27,7 @@
 import org.apache.jackrabbit.core.util.RepositoryLock;
 import org.apache.jackrabbit.core.util.RepositoryLockMechanism;
 import org.apache.jackrabbit.core.util.RepositoryLockMechanismFactory;
+import org.apache.jackrabbit.core.util.db.ConnectionFactory;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
@@ -175,12 +176,18 @@
     private static final String AC_PROVIDER_ELEMENT = "AccessControlProvider";
 
     /**
+     * The repositories {@link ConnectionFactory}. 
+     */
+    private final ConnectionFactory connectionFactory;
+
+    /**
      * Creates a new configuration parser with the given parser variables.
      *
      * @param variables parser variables
      */
-    public RepositoryConfigurationParser(Properties variables) {
+    public RepositoryConfigurationParser(Properties variables, ConnectionFactory connectionFactory) {
         super(variables);
+        this.connectionFactory = connectionFactory;
     }
 
     /**
@@ -278,12 +285,33 @@
 
         RepositoryLockMechanismFactory rlf = getRepositoryLockMechanismFactory(root);
 
-        // Optional journal configuration
+        // Optional data source configuration
         DataSourceConfig dsc = parseDataSourceConfig(root);
 
         return new RepositoryConfig(home, securityConfig, fsf,
                 workspaceDirectory, workspaceConfigDirectory, defaultWorkspace,
-                maxIdleTime, template, vc, sc, cc, dsf, rlf, dsc, this);
+                maxIdleTime, template, vc, sc, cc, dsf, rlf, dsc, connectionFactory, this);
+    }
+
+    
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    protected BeanConfig parseBeanConfig(Element parent, String name) throws ConfigurationException {
+        BeanConfig cfg = super.parseBeanConfig(parent, name);
+        cfg.setConnectionFactory(connectionFactory);
+        return cfg;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    protected BeanConfig parseBeanConfig(Element element) throws ConfigurationException {
+        BeanConfig cfg = super.parseBeanConfig(element);
+        cfg.setConnectionFactory(connectionFactory);
+        return cfg;
     }
 
     /**
@@ -853,7 +881,7 @@
 
     /**
      * Creates a new instance of a configuration parser but with overlayed
-     * variables.
+     * variables and the same connection factory as this parser.
      *
      * @param variables the variables overlay
      * @return a new configuration parser instance
@@ -862,7 +890,7 @@
         // overlay the properties
         Properties props = new Properties(getVariables());
         props.putAll(variables);
-        return new RepositoryConfigurationParser(props);
+        return new RepositoryConfigurationParser(props, connectionFactory);
     }
 
     /**

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/data/db/DbDataStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/data/db/DbDataStore.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/data/db/DbDataStore.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/data/db/DbDataStore.java Thu Sep 24 12:48:21 2009
@@ -24,6 +24,7 @@
 import org.apache.jackrabbit.core.util.db.CheckSchemaOperation;
 import org.apache.jackrabbit.core.util.db.ConnectionFactory;
 import org.apache.jackrabbit.core.util.db.ConnectionHelper;
+import org.apache.jackrabbit.core.util.db.DatabaseAware;
 import org.apache.jackrabbit.core.util.db.DbUtility;
 import org.apache.jackrabbit.core.util.db.StreamWrapper;
 import org.apache.jackrabbit.util.Text;
@@ -95,7 +96,7 @@
  * The tablePrefix can be used to specify a schema and / or catalog name:
  * &lt;param name="tablePrefix" value="ds.">
  */
-public class DbDataStore implements DataStore {
+public class DbDataStore implements DataStore, DatabaseAware {
 
     /**
      * The default value for the minimum object size.
@@ -294,6 +295,18 @@
     protected ConnectionHelper conHelper;
 
     /**
+     * The repositories {@link ConnectionFactory}.
+     */
+    private ConnectionFactory connectionFactory;
+
+    /**
+     * {@inheritDoc}
+     */
+    public void setConnectionFactory(ConnectionFactory connnectionFactory) {
+        this.connectionFactory = connnectionFactory;
+    }
+
+    /**
      * {@inheritDoc}
      */
     public DataRecord addRecord(InputStream stream) throws DataStoreException {
@@ -568,9 +581,9 @@
 
     private DataSource getDataSource() throws Exception {
         if (getDataSourceName() == null || "".equals(getDataSourceName())) {
-            return ConnectionFactory.getDataSource(getDriver(), getUrl(), getUser(), getPassword());
+            return connectionFactory.getDataSource(getDriver(), getUrl(), getUser(), getPassword());
         } else {
-            return ConnectionFactory.getDataSource(dataSourceName);
+            return connectionFactory.getDataSource(dataSourceName);
         }
     }
 
@@ -603,7 +616,7 @@
         if (databaseType == null) {
             if (dataSourceName != null) {
                 try {
-                    databaseType = ConnectionFactory.getDataBaseType(dataSourceName);
+                    databaseType = connectionFactory.getDataBaseType(dataSourceName);
                 } catch (RepositoryException e) {
                     throw new DataStoreException(e);
                 }

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/fs/db/DbFileSystem.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/fs/db/DbFileSystem.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/fs/db/DbFileSystem.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/fs/db/DbFileSystem.java Thu Sep 24 12:48:21 2009
@@ -17,6 +17,7 @@
 package org.apache.jackrabbit.core.fs.db;
 
 import org.apache.jackrabbit.core.util.db.ConnectionFactory;
+import org.apache.jackrabbit.core.util.db.DatabaseAware;
 
 import javax.sql.DataSource;
 
@@ -105,7 +106,7 @@
  * </pre>
  * See also {@link DerbyFileSystem}, {@link DB2FileSystem}, {@link OracleFileSystem}.
  */
-public class DbFileSystem extends DatabaseFileSystem {
+public class DbFileSystem extends DatabaseFileSystem implements DatabaseAware {
 
     /**
      * the full qualified JDBC driver name
@@ -129,6 +130,18 @@
 
     protected String dataSourceName;
 
+    /**
+     * The repositories {@link ConnectionFactory}.
+     */
+    private ConnectionFactory connectionFactory;
+
+    /**
+     * {@inheritDoc}
+     */
+    public void setConnectionFactory(ConnectionFactory connnectionFactory) {
+        this.connectionFactory = connnectionFactory;
+    }
+
     //----------------------------------------------------< setters & getters >
     public String getUrl() {
         return url;
@@ -210,13 +223,13 @@
     @Override
     protected final DataSource getDataSource() throws Exception {
         if (getDataSourceName() == null || "".equals(getDataSourceName())) {
-            return ConnectionFactory.getDataSource(getDriver(), getUrl(), getUser(), getPassword());
+            return connectionFactory.getDataSource(getDriver(), getUrl(), getUser(), getPassword());
         } else {
-            String dbType = ConnectionFactory.getDataBaseType(dataSourceName);
+            String dbType = connectionFactory.getDataBaseType(dataSourceName);
             if (DatabaseFileSystem.class.getResourceAsStream(dbType + ".ddl") != null) {
                 setSchema(dbType);
             }
-            return ConnectionFactory.getDataSource(dataSourceName);
+            return connectionFactory.getDataSource(dataSourceName);
         }
     }
 }

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/journal/DatabaseJournal.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/journal/DatabaseJournal.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/journal/DatabaseJournal.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/journal/DatabaseJournal.java Thu Sep 24 12:48:21 2009
@@ -20,6 +20,7 @@
 import org.apache.jackrabbit.core.util.db.CheckSchemaOperation;
 import org.apache.jackrabbit.core.util.db.ConnectionFactory;
 import org.apache.jackrabbit.core.util.db.ConnectionHelper;
+import org.apache.jackrabbit.core.util.db.DatabaseAware;
 import org.apache.jackrabbit.core.util.db.DbUtility;
 import org.apache.jackrabbit.core.util.db.StreamWrapper;
 import org.apache.jackrabbit.spi.commons.namespace.NamespaceResolver;
@@ -77,7 +78,7 @@
  * </pre> *
  * </ul>
  */
-public class DatabaseJournal extends AbstractJournal {
+public class DatabaseJournal extends AbstractJournal implements DatabaseAware {
 
     /**
      * Default journal table name, used to check schema completeness.
@@ -226,6 +227,11 @@
      */
     protected String schemaObjectPrefix;
 
+    /**
+     * The repositories {@link ConnectionFactory}.
+     */
+    private ConnectionFactory connectionFactory;
+
     public DatabaseJournal() {
         databaseType = "default";
         schemaObjectPrefix = "";
@@ -234,6 +240,13 @@
     /**
      * {@inheritDoc}
      */
+    public void setConnectionFactory(ConnectionFactory connnectionFactory) {
+        this.connectionFactory = connnectionFactory;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
     public void init(String id, NamespaceResolver resolver)
             throws JournalException {
 
@@ -268,9 +281,9 @@
 
     private DataSource getDataSource() throws Exception {
         if (getDataSourceName() == null || "".equals(getDataSourceName())) {
-            return ConnectionFactory.getDataSource(getDriver(), getUrl(), getUser(), getPassword());
+            return connectionFactory.getDataSource(getDriver(), getUrl(), getUser(), getPassword());
         } else {
-            return ConnectionFactory.getDataSource(dataSourceName);
+            return connectionFactory.getDataSource(dataSourceName);
         }
     }
 
@@ -321,7 +334,7 @@
         }
         if (dataSourceName != null) {
             try {
-                String configuredDatabaseType = ConnectionFactory.getDataBaseType(dataSourceName);
+                String configuredDatabaseType = connectionFactory.getDataBaseType(dataSourceName);
                 if (DatabaseJournal.class.getResourceAsStream(configuredDatabaseType + ".ddl") != null) {
                     setDatabaseType(configuredDatabaseType);
                 }

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/BundleDbPersistenceManager.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/BundleDbPersistenceManager.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/BundleDbPersistenceManager.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/BundleDbPersistenceManager.java Thu Sep 24 12:48:21 2009
@@ -56,6 +56,7 @@
 import org.apache.jackrabbit.core.util.db.CheckSchemaOperation;
 import org.apache.jackrabbit.core.util.db.ConnectionFactory;
 import org.apache.jackrabbit.core.util.db.ConnectionHelper;
+import org.apache.jackrabbit.core.util.db.DatabaseAware;
 import org.apache.jackrabbit.core.util.db.DbUtility;
 import org.apache.jackrabbit.core.util.db.StreamWrapper;
 import org.slf4j.Logger;
@@ -82,7 +83,7 @@
  * <li>&lt;param name="{@link #setSchemaCheckEnabled(String) schemaCheckEnabled}" value="true"/>
  * </ul>
  */
-public class BundleDbPersistenceManager extends AbstractBundlePersistenceManager {
+public class BundleDbPersistenceManager extends AbstractBundlePersistenceManager implements DatabaseAware {
 
     /** the default logger */
     private static Logger log = LoggerFactory.getLogger(BundleDbPersistenceManager.class);
@@ -182,6 +183,18 @@
     private boolean schemaCheckEnabled = true;
 
     /**
+     * The repositories {@link ConnectionFactory}.
+     */
+    private ConnectionFactory connectionFactory;
+
+    /**
+     * {@inheritDoc}
+     */
+    public void setConnectionFactory(ConnectionFactory connnectionFactory) {
+        this.connectionFactory = connnectionFactory;
+    }
+
+    /**
      * Returns the configured JDBC connection url.
      * @return the configured JDBC connection url.
      */
@@ -523,13 +536,13 @@
 
     private DataSource getDataSource() throws Exception {
         if (getDataSourceName() == null || "".equals(getDataSourceName())) {
-            return ConnectionFactory.getDataSource(getDriver(), getUrl(), getUser(), getPassword());
+            return connectionFactory.getDataSource(getDriver(), getUrl(), getUser(), getPassword());
         } else {
-            String dbType = ConnectionFactory.getDataBaseType(dataSourceName);
+            String dbType = connectionFactory.getDataBaseType(dataSourceName);
             if (BundleDbPersistenceManager.class.getResourceAsStream(dbType + ".ddl") != null) {
                 setDatabaseType(dbType);
             }
-            return ConnectionFactory.getDataSource(dataSourceName);
+            return connectionFactory.getDataSource(dataSourceName);
         }
     }
 

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/db/SimpleDbPersistenceManager.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/db/SimpleDbPersistenceManager.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/db/SimpleDbPersistenceManager.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/db/SimpleDbPersistenceManager.java Thu Sep 24 12:48:21 2009
@@ -17,6 +17,7 @@
 package org.apache.jackrabbit.core.persistence.db;
 
 import org.apache.jackrabbit.core.util.db.ConnectionFactory;
+import org.apache.jackrabbit.core.util.db.DatabaseAware;
 
 import java.sql.Connection;
 import java.sql.SQLException;
@@ -129,13 +130,25 @@
  * </pre>
  * See also {@link DerbyPersistenceManager}, {@link OraclePersistenceManager}.
  */
-public class SimpleDbPersistenceManager extends DatabasePersistenceManager {
+public class SimpleDbPersistenceManager extends DatabasePersistenceManager implements DatabaseAware {
 
     protected String driver;
     protected String url;
     protected String user;
     protected String password;
 
+    /**
+     * The repositories {@link ConnectionFactory}.
+     */
+    private ConnectionFactory connectionFactory;
+
+    /**
+     * {@inheritDoc}
+     */
+    public void setConnectionFactory(ConnectionFactory connnectionFactory) {
+        this.connectionFactory = connnectionFactory;
+    }
+
     //----------------------------------------------------< setters & getters >
     public String getUrl() {
         return url;
@@ -169,8 +182,6 @@
         this.driver = driver;
     }
 
-    //------------------------------------------< DatabasePersistenceManager >
-
     /**
      * Returns a JDBC connection acquired using the JDBC {@link DriverManager}.
      * @throws SQLException
@@ -180,7 +191,7 @@
      * @see DatabasePersistenceManager#getConnection()
      */
     protected Connection getConnection() throws RepositoryException, SQLException {
-        return ConnectionFactory.getDataSource(driver, url, user, password).getConnection();
+        return connectionFactory.getDataSource(driver, url, user, password).getConnection();
     }
 
 }

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/ConnectionFactory.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/ConnectionFactory.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/ConnectionFactory.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/ConnectionFactory.java Thu Sep 24 12:48:21 2009
@@ -18,7 +18,9 @@
 
 import java.sql.Connection;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import javax.jcr.RepositoryException;
@@ -41,38 +43,46 @@
  * FIXME: the registry currently is ClassLoader wide. I.e., if you start two repositories
  * then you share the registered datasources...
  */
-public class ConnectionFactory {
+public final class ConnectionFactory {
 
     private static final Logger log = LoggerFactory.getLogger(ConnectionFactory.class);
 
     /**
-     * The lock to protect the registries. 
+     * The lock to protect the fields of this class. 
      */
-    private static final Object lock = new Object();
+    private final Object lock = new Object();
 
     /**
      * The data sources without logical name. The keys in the map are based on driver-url-user combination.
      */
-    private static final Map<String, DataSource> keyToDataSource = new HashMap<String, DataSource>();
+    private final Map<String, DataSource> keyToDataSource = new HashMap<String, DataSource>();
 
     /**
      * The configured data sources with logical name. The keys in the map are the logical name.
      */
-    private static final Map<String, DataSource> nameToDataSource = new HashMap<String, DataSource>();
+    private final Map<String, DataSource> nameToDataSource = new HashMap<String, DataSource>();
 
     /**
      * The configured data source defs. The keys in the map are the logical name.
      */
-    private static final Map<String, DataSourceDefinition> nameToDataSourceDef = new HashMap<String, DataSourceDefinition>();
+    private final Map<String, DataSourceDefinition> nameToDataSourceDef = new HashMap<String, DataSourceDefinition>();
+
+    /**
+     * The list of data sources created by this factory.
+     */
+    private final List<BasicDataSource> created = new ArrayList<BasicDataSource>();
+
+    private boolean closed = false;
 
     /**
      * Registers a number of data sources.
      * 
      * @param dsc the {@link DataSourceConfig} which contains the configuration
      */
-    public static void registerDataSources(DataSourceConfig dsc) throws RepositoryException {
-        for (DataSourceDefinition def : dsc.getDefinitions()) {
-            synchronized (lock) {
+    public void registerDataSources(DataSourceConfig dsc) throws RepositoryException {
+        synchronized (lock) {
+            sanityCheck();
+            for (DataSourceDefinition def : dsc.getDefinitions()) {
                 Class<?> driverClass = getDriverClass(def.getDriver());
                 if (driverClass != null
                         && Context.class.isAssignableFrom(driverClass)) {
@@ -102,8 +112,9 @@
      * @return a {@code DataSource}
      * @throws RepositoryException if there is no {@code DataSource} with the given name
      */
-    public static DataSource getDataSource(String logicalName) throws RepositoryException {
+    public DataSource getDataSource(String logicalName) throws RepositoryException {
         synchronized (lock) {
+            sanityCheck();
             DataSource ds = nameToDataSource.get(logicalName);
             if (ds == null) {
                 throw new RepositoryException("DataSource with logicalName " + logicalName
@@ -118,8 +129,9 @@
      * @return the configured database type
      * @throws RepositoryException if there is no {@code DataSource} with the given name
      */
-    public static String getDataBaseType(String logicalName) throws RepositoryException {
+    public String getDataBaseType(String logicalName) throws RepositoryException {
         synchronized (lock) {
+            sanityCheck();
             DataSourceDefinition def = nameToDataSourceDef.get(logicalName);
             if (def == null) {
                 throw new RepositoryException("DataSource with logicalName " + logicalName
@@ -144,10 +156,11 @@
      * @throws RepositoryException if the driver could not be loaded
      * @throws SQLException if the connection could not be established
      */
-    public static DataSource getDataSource(String driver, String url, String user, String password)
+    public DataSource getDataSource(String driver, String url, String user, String password)
             throws RepositoryException, SQLException    {
         final String key = driver + url + user;
         synchronized(lock) {
+            sanityCheck();
             DataSource ds = keyToDataSource.get(key);
             if (ds == null) {
                 ds = createDataSource(driver, url, user, password);
@@ -158,6 +171,27 @@
     }
 
     /**
+     * 
+     */
+    public void close() {
+        synchronized(lock) {
+            sanityCheck();
+            for (BasicDataSource ds : created) {
+                try {
+                    ds.close();
+                } catch (SQLException e) {
+                    log.error("failed to close " + ds, e);
+                }
+            }
+            keyToDataSource.clear();
+            nameToDataSource.clear();
+            nameToDataSourceDef.clear();
+            created.clear();
+            closed = true;
+        }
+    }
+
+    /**
      * Needed for pre-10R2 Oracle blob support....:(
      * 
      * This method actually assumes that we are using commons DBCP 1.2.2.
@@ -174,6 +208,12 @@
         }
     }
 
+    private void sanityCheck() {
+        if (closed) {
+            throw new IllegalStateException("this factory has already been closed");
+        }
+    }
+
     /**
      * Create a new pooling data source or finds an existing JNDI data source (depends on driver).
      * 
@@ -184,7 +224,7 @@
      * @return
      * @throws RepositoryException
      */
-    private static DataSource createDataSource(String driver, String url, String user, String password)
+    private DataSource createDataSource(String driver, String url, String user, String password)
             throws RepositoryException {
         Class<?> driverClass = getDriverClass(driver);
         if (driverClass != null
@@ -209,7 +249,7 @@
      * @return driver class, or <code>null</code>
      * @throws RepositoryException if the class can not be loaded
      */
-    private static Class<?> getDriverClass(String driver)
+    private Class<?> getDriverClass(String driver)
             throws RepositoryException {
         try {
             if (driver != null && driver.length() > 0) {
@@ -233,7 +273,7 @@
      * @throws RepositoryException if the JNDI context can not be accessed,
      *                             or if the named DataSource is not found
      */
-    private static DataSource getJndiDataSource(
+    private DataSource getJndiDataSource(
             Class<Context> contextClass, String name)
             throws RepositoryException {
         try {
@@ -267,9 +307,10 @@
      * @param url the JDBC connection URL
      * @return pooling DataSource for accessing the specified database
      */
-    private static BasicDataSource getDriverDataSource(
+    private BasicDataSource getDriverDataSource(
             Class<?> driverClass, String url, String user, String password) {
         BasicDataSource ds = new BasicDataSource();
+        created.add(ds);
 
         if (driverClass != null) {
             try {
@@ -290,7 +331,9 @@
         ds.setUsername(user);
         ds.setPassword(password);
         ds.setDefaultAutoCommit(true);
-        ds.setTestOnBorrow(true);
+        ds.setTestOnBorrow(false);
+        ds.setTestWhileIdle(true);
+        ds.setTimeBetweenEvictionRunsMillis(1000);
         ds.setMaxActive(-1); // unlimited
         ds.setValidationQuery(guessValidationQuery(url));   
         ds.setAccessToUnderlyingConnectionAllowed(true);
@@ -299,7 +342,7 @@
         return ds;
     }
 
-    private static String guessValidationQuery(String url) {
+    private String guessValidationQuery(String url) {
         if (url.contains("derby")) {
             return "values(1)";
         } else if (url.contains("mysql")) {

Added: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/DatabaseAware.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/DatabaseAware.java?rev=818466&view=auto
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/DatabaseAware.java (added)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/DatabaseAware.java Thu Sep 24 12:48:21 2009
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jackrabbit.core.util.db;
+
+/**
+ * Bean components (i.e., classes that appear in the repository descriptor) that implement this interface will
+ * get the repositories {@link ConnectionFactory} instance injected just after construction and before
+ * initialization.
+ */
+public interface DatabaseAware {
+
+    /**
+     * @param connnectionFactory
+     */
+    void setConnectionFactory(ConnectionFactory connnectionFactory);
+}

Propchange: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/util/db/DatabaseAware.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/SecurityConfigTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/SecurityConfigTest.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/SecurityConfigTest.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/SecurityConfigTest.java Thu Sep 24 12:48:21 2009
@@ -50,7 +50,7 @@
 
     protected void setUp() throws Exception {
         super.setUp();
-        parser = new RepositoryConfigurationParser(new Properties());
+        parser = new RepositoryConfigurationParser(new Properties(), null);
     }
 
     protected void tearDown() throws Exception {

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/WorkspaceConfigTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/WorkspaceConfigTest.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/WorkspaceConfigTest.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/config/WorkspaceConfigTest.java Thu Sep 24 12:48:21 2009
@@ -34,7 +34,7 @@
     protected void setUp() {
         Properties variables = new Properties();
         variables.setProperty("wsp.home", "target");
-        parser = new RepositoryConfigurationParser(variables);
+        parser = new RepositoryConfigurationParser(variables, null);
     }
 
     /**

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/fs/db/DerbyFileSystemTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/fs/db/DerbyFileSystemTest.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/fs/db/DerbyFileSystemTest.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/fs/db/DerbyFileSystemTest.java Thu Sep 24 12:48:21 2009
@@ -21,16 +21,20 @@
 import org.apache.commons.io.FileUtils;
 import org.apache.jackrabbit.core.fs.AbstractFileSystemTest;
 import org.apache.jackrabbit.core.fs.FileSystem;
+import org.apache.jackrabbit.core.util.db.ConnectionFactory;
 
 /**
  * Tests the Apache Derby file system.
  */
 public class DerbyFileSystemTest extends AbstractFileSystemTest {
 
+    private ConnectionFactory conFac;
+
     private File file;
 
     protected FileSystem getFileSystem() {
         DerbyFileSystem filesystem = new DerbyFileSystem();
+        filesystem.setConnectionFactory(conFac);
         filesystem.setUrl("jdbc:derby:" + file.getPath() + ";create=true");
         return filesystem;
     }
@@ -38,11 +42,13 @@
     protected void setUp() throws Exception {
         file = File.createTempFile("jackrabbit", "derbyfs");
         file.delete();
+        conFac = new ConnectionFactory();
         super.setUp();
     }
 
     protected void tearDown() throws Exception {
         super.tearDown();
         FileUtils.deleteDirectory(file);
+        conFac.close();
     }
 }

Modified: jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/util/db/ConnectionFactoryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/util/db/ConnectionFactoryTest.java?rev=818466&r1=818465&r2=818466&view=diff
==============================================================================
--- jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/util/db/ConnectionFactoryTest.java (original)
+++ jackrabbit/sandbox/JCR-1456/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/util/db/ConnectionFactoryTest.java Thu Sep 24 12:48:21 2009
@@ -34,6 +34,8 @@
  */
 public class ConnectionFactoryTest extends TestCase {
 
+    private ConnectionFactory connectionFactory;
+
     private static final String DRIVER = "org.apache.derby.jdbc.EmbeddedDriver";
 
     private static final String DERBY_URL = "jdbc:derby:target/connection-factory-test/db;create=true";
@@ -51,96 +53,117 @@
      @Override
      public void setUp() {
          System.setProperty("derby.stream.error.file", "target/derby-connectionfactorytest.log");
+         connectionFactory = new ConnectionFactory();
      }
 
     public void testGetDataSource_defaults_Derby() throws Exception {
-        DataSource ds1 = ConnectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
+        DataSource ds1 = connectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
         assertTrue(ds1 instanceof BasicDataSource);
         BasicDataSource ds = (BasicDataSource) ds1;
-        assertEquals(-1, ds.getMaxActive());
-        assertEquals("values(1)", ds.getValidationQuery());
-        assertTrue(ds.getDefaultAutoCommit());
-        assertTrue(ds.getTestOnBorrow());
-        assertTrue(ds.isPoolPreparedStatements());
-        assertEquals(-1, ds.getMaxOpenPreparedStatements());
+        assertPoolDefaults(ds, "values(1)", -1);
     }
 
     public void testGuessValidationQuery_MYSQL() throws Exception {
-        DataSource ds1 = ConnectionFactory.getDataSource(DRIVER, MYSQL_URL, "user", "password");
+        DataSource ds1 = connectionFactory.getDataSource(DRIVER, MYSQL_URL, "user", "password");
         assertEquals("select 1", ((BasicDataSource) ds1).getValidationQuery());
     }
 
     public void testGuessValidationQuery_MSSQL() throws Exception {
-        DataSource ds1 = ConnectionFactory.getDataSource(DRIVER, MSSQL_URL_1, "user", "password");
+        DataSource ds1 = connectionFactory.getDataSource(DRIVER, MSSQL_URL_1, "user", "password");
         assertEquals("select 1", ((BasicDataSource) ds1).getValidationQuery());
-        DataSource ds2 = ConnectionFactory.getDataSource(DRIVER, MSSQL_URL_2, "user", "password");
+        DataSource ds2 = connectionFactory.getDataSource(DRIVER, MSSQL_URL_2, "user", "password");
         assertEquals("select 1", ((BasicDataSource) ds2).getValidationQuery());
     }
 
     public void testGuessValidationQuery_ORACLE() throws Exception {
-        DataSource ds1 = ConnectionFactory.getDataSource(DRIVER, ORACLE_URL, "user", "password");
+        DataSource ds1 = connectionFactory.getDataSource(DRIVER, ORACLE_URL, "user", "password");
         assertEquals("select 'validationQuery' from dual", ((BasicDataSource) ds1).getValidationQuery());
     }
 
     public void testGuessValidationQuery_H2() throws Exception {
-        DataSource ds1 = ConnectionFactory.getDataSource(DRIVER, H2_URL, "user", "password");
+        DataSource ds1 = connectionFactory.getDataSource(DRIVER, H2_URL, "user", "password");
         assertEquals("select 1", ((BasicDataSource) ds1).getValidationQuery());
     }
 
     public void testRegisterDataSources_defaultValues() throws Exception {
-        BasicDataSource ds = registerAndGet(DERBY_URL, "overwrite");
-        assertEquals(10, ds.getMaxActive());
-        assertEquals("overwrite", ds.getValidationQuery());
-        assertTrue(ds.getDefaultAutoCommit());
-        assertTrue(ds.getTestOnBorrow());
-        assertTrue(ds.isPoolPreparedStatements());
-        assertEquals(-1, ds.getMaxOpenPreparedStatements());
+        BasicDataSource ds = registerAndGet(DERBY_URL, "overwrite", -1);
+        assertPoolDefaults(ds, "overwrite", -1);
     }
 
     public void testRegisterDataSources_noValidationQuery() throws Exception {
-        BasicDataSource ds = registerAndGet(MYSQL_URL, "");
+        BasicDataSource ds = registerAndGet(MYSQL_URL, "", -1);
         assertEquals("select 1", ds.getValidationQuery());
     }
 
     public void testGetDatabaseType() throws Exception {
-        String name = register(MYSQL_URL, "");
-        assertEquals("dbType", ConnectionFactory.getDataBaseType(name));
+        String name = register(MYSQL_URL, "", -1);
+        assertEquals("dbType", connectionFactory.getDataBaseType(name));
     }
 
     public void testGetDataSource_identity() throws Exception {
-        DataSource ds1 = ConnectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
-        DataSource ds2 = ConnectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
+        DataSource ds1 = connectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
+        DataSource ds2 = connectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
         assertSame(ds1, ds2);
     }
 
     public void testGetDataSource_identity_differentPasswords() throws Exception {
-        DataSource ds1 = ConnectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
-        DataSource ds2 = ConnectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password2");
+        DataSource ds1 = connectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
+        DataSource ds2 = connectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password2");
         assertSame(ds1, ds2);
     }
 
     public void testGetDataSource_noIdentity() throws Exception {
-        DataSource ds1 = ConnectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
-        DataSource ds2 = ConnectionFactory.getDataSource(DRIVER, DERBY_URL, "user2", "password");
+        DataSource ds1 = connectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
+        DataSource ds2 = connectionFactory.getDataSource(DRIVER, DERBY_URL, "user2", "password");
         assertNotSame(ds1, ds2);
     }
 
     public void testUnwrap() throws Exception {
-        DataSource ds = ConnectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
+        DataSource ds = connectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
         Connection wrappedCon = ds.getConnection();
         assertNotNull(wrappedCon);
         Connection con = ConnectionFactory.unwrap(wrappedCon);
         assertTrue(con instanceof EngineConnection);
     }
 
-    private BasicDataSource registerAndGet(String url, String validationQuery) throws Exception {
-        final String name = register(url, validationQuery);
-        DataSource ds = ConnectionFactory.getDataSource(name);
+    public void testClose() throws Exception {
+        connectionFactory.close();
+        try {
+            connectionFactory.getDataBaseType("logicalName");
+            fail("could retrieve after close");
+        } catch (IllegalStateException expected) {
+        }
+        try {
+            connectionFactory.getDataSource("logicalName");
+            fail("could retrieve after close");
+        } catch (IllegalStateException expected) {
+        }
+        try {
+            connectionFactory.getDataSource(DRIVER, DERBY_URL, "user", "password");
+            fail("could retrieve after close");
+        } catch (IllegalStateException expected) {
+        }
+    }
+
+    private void assertPoolDefaults(BasicDataSource ds, String validationQuery, int maxCons) {
+        assertEquals(maxCons, ds.getMaxActive());
+        assertEquals(validationQuery, ds.getValidationQuery());
+        assertTrue(ds.getDefaultAutoCommit());
+        assertFalse(ds.getTestOnBorrow());
+        assertTrue(ds.getTestWhileIdle());
+        assertEquals(1000, ds.getTimeBetweenEvictionRunsMillis());
+        assertTrue(ds.isPoolPreparedStatements());
+        assertEquals(-1, ds.getMaxOpenPreparedStatements());
+    }
+
+    private BasicDataSource registerAndGet(String url, String validationQuery, int maxCons) throws Exception {
+        final String name = register(url, validationQuery, maxCons);
+        DataSource ds = connectionFactory.getDataSource(name);
         assertTrue(ds instanceof BasicDataSource);
         return (BasicDataSource) ds;
     }
 
-    private String register(String url, String validationQuery) throws ConfigurationException,
+    private String register(String url, String validationQuery, int maxCons) throws ConfigurationException,
             RepositoryException {
         final String name = "some random name to not interfere with integration tests...";
         DataSourceConfig dsc = new DataSourceConfig();
@@ -149,10 +172,10 @@
         props.put(DataSourceConfig.DRIVER, DRIVER);
         props.put(DataSourceConfig.URL, url);
         props.put(DataSourceConfig.DB_TYPE, "dbType");
-        props.put(DataSourceConfig.MAX_POOL_SIZE, "10");
+        props.put(DataSourceConfig.MAX_POOL_SIZE, Integer.toString(maxCons));
         props.put(DataSourceConfig.VALIDATION_QUERY, validationQuery);
         dsc.addDataSourceDefinition(props);
-        ConnectionFactory.registerDataSources(dsc);
+        connectionFactory.registerDataSources(dsc);
         return name;
     }
 }