You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cayenne.apache.org by aa...@apache.org on 2015/04/30 17:59:12 UTC

[2/2] cayenne git commit: CAY-2008 Connection pool refactoring and validation query support in Cayenne DataSource

CAY-2008 Connection pool refactoring and validation query support in Cayenne DataSource

* Refactoring and cleanup of DataSource implementation
* DataSourceBuilder
* splitting pool management thread into a separate DS wrapper
* implementing java.sql.Wrapper methods
* pool specific parameters are passed as a special object
* stop supporting getting connections with expclicit username/password. We do not have any
  notion of per-user connection pools, so it is only misleading and can lead
  to unpredictable pool state
* validate query


Project: http://git-wip-us.apache.org/repos/asf/cayenne/repo
Commit: http://git-wip-us.apache.org/repos/asf/cayenne/commit/8b54c052
Tree: http://git-wip-us.apache.org/repos/asf/cayenne/tree/8b54c052
Diff: http://git-wip-us.apache.org/repos/asf/cayenne/diff/8b54c052

Branch: refs/heads/master
Commit: 8b54c052d6e54a5218c049819e35d718b4049af0
Parents: 5e037fc
Author: aadamchik <aa...@apache.org>
Authored: Thu Apr 30 13:34:21 2015 +0300
Committer: aadamchik <aa...@apache.org>
Committed: Thu Apr 30 18:46:25 2015 +0300

----------------------------------------------------------------------
 .../apache/cayenne/di/ScopeEventListener.java   |   19 +-
 .../server/PropertyDataSourceFactory.java       |   86 +-
 .../server/XMLPoolingDataSourceFactory.java     |   59 +-
 .../apache/cayenne/conn/DataSourceBuilder.java  |  135 ++
 .../org/apache/cayenne/conn/DataSourceInfo.java |   74 +-
 .../cayenne/conn/ManagedPoolingDataSource.java  |  155 ++
 .../org/apache/cayenne/conn/PoolDataSource.java |   90 -
 .../org/apache/cayenne/conn/PoolManager.java    |  649 --------
 .../cayenne/conn/PooledConnectionFactory.java   |   90 +
 .../cayenne/conn/PooledConnectionImpl.java      |  320 ++--
 .../apache/cayenne/conn/PoolingDataSource.java  |  471 ++++++
 .../conn/PoolingDataSourceParameters.java       |   64 +
 .../apache/cayenne/access/DataContextIT.java    | 1571 +++++++++---------
 .../cayenne/conn/BasePoolingDataSourceIT.java   |   74 +
 .../org/apache/cayenne/conn/PoolManagerIT.java  |  128 --
 .../cayenne/conn/PooledConnectionImplTest.java  |   68 +-
 .../cayenne/conn/PoolingDataSourceIT.java       |   72 +
 ...lingDataSource_FailingValidationQueryIT.java |   43 +
 .../PoolingDataSource_ValidationQueryIT.java    |   55 +
 .../di/server/ServerCaseDataSourceFactory.java  |   89 +-
 docs/doc/src/main/resources/RELEASE-NOTES.txt   |    1 +
 21 files changed, 2296 insertions(+), 2017 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-di/src/main/java/org/apache/cayenne/di/ScopeEventListener.java
----------------------------------------------------------------------
diff --git a/cayenne-di/src/main/java/org/apache/cayenne/di/ScopeEventListener.java b/cayenne-di/src/main/java/org/apache/cayenne/di/ScopeEventListener.java
index 171bd50..b6959ac 100644
--- a/cayenne-di/src/main/java/org/apache/cayenne/di/ScopeEventListener.java
+++ b/cayenne-di/src/main/java/org/apache/cayenne/di/ScopeEventListener.java
@@ -18,18 +18,19 @@
  ****************************************************************/
 package org.apache.cayenne.di;
 
-
 /**
- * This interface duplicates default reflection based mechanism for receiving DI events.
- * It is not fully supported and its usage are reserved for cases when for some reason
- * it is not possible to use reflection. It is used for example in
- * {@link javax.sql.DataSource} managing layer to provide compatibility with java version 5.
+ * This interface duplicates default reflection based mechanism for receiving DI
+ * events. It is not fully supported and its usage are reserved for cases when
+ * for some reason it is not possible to use reflection. It is used for example
+ * in {@link javax.sql.DataSource} management layer to provide compatibility
+ * with java version 5.
  *
  * @since 3.1
  */
 public interface ScopeEventListener {
-    /**
-     * Similar to {@link BeforeScopeEnd}
-     */
-    void beforeScopeEnd();
+	
+	/**
+	 * Similar to {@link BeforeScopeEnd}
+	 */
+	void beforeScopeEnd();
 }

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/PropertyDataSourceFactory.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/PropertyDataSourceFactory.java b/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/PropertyDataSourceFactory.java
index 237f4e4..87d18de 100644
--- a/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/PropertyDataSourceFactory.java
+++ b/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/PropertyDataSourceFactory.java
@@ -18,17 +18,14 @@
  ****************************************************************/
 package org.apache.cayenne.configuration.server;
 
-import java.sql.Driver;
-
 import javax.sql.DataSource;
 
 import org.apache.cayenne.ConfigurationException;
 import org.apache.cayenne.configuration.Constants;
 import org.apache.cayenne.configuration.DataNodeDescriptor;
 import org.apache.cayenne.configuration.RuntimeProperties;
-import org.apache.cayenne.conn.DriverDataSource;
-import org.apache.cayenne.conn.PoolDataSource;
-import org.apache.cayenne.conn.PoolManager;
+import org.apache.cayenne.conn.DataSourceBuilder;
+import org.apache.cayenne.conn.PoolingDataSource;
 import org.apache.cayenne.di.AdhocObjectFactory;
 import org.apache.cayenne.di.Inject;
 import org.apache.cayenne.log.JdbcEventLogger;
@@ -52,57 +49,50 @@ import org.apache.cayenne.log.JdbcEventLogger;
  */
 public class PropertyDataSourceFactory implements DataSourceFactory {
 
-    @Inject
-    protected RuntimeProperties properties;
-
-    @Inject
-    protected JdbcEventLogger jdbcEventLogger;
+	@Inject
+	protected RuntimeProperties properties;
 
-    @Inject
-    private AdhocObjectFactory objectFactory;
+	@Inject
+	protected JdbcEventLogger jdbcEventLogger;
 
-    @Override
-    public DataSource getDataSource(DataNodeDescriptor nodeDescriptor) throws Exception {
+	@Inject
+	private AdhocObjectFactory objectFactory;
 
-        String suffix = "." + nodeDescriptor.getDataChannelDescriptor().getName() + "." + nodeDescriptor.getName();
+	@Override
+	public DataSource getDataSource(DataNodeDescriptor nodeDescriptor) throws Exception {
 
-        String driverClass = getProperty(Constants.JDBC_DRIVER_PROPERTY, suffix);
-        String url = getProperty(Constants.JDBC_URL_PROPERTY, suffix);
-        String username = getProperty(Constants.JDBC_USERNAME_PROPERTY, suffix);
-        String password = getProperty(Constants.JDBC_PASSWORD_PROPERTY, suffix);
-        int minConnections = getIntProperty(Constants.JDBC_MIN_CONNECTIONS_PROPERTY, suffix, 1);
-        int maxConnections = getIntProperty(Constants.JDBC_MAX_CONNECTIONS_PROPERTY, suffix, 1);
+		String suffix = "." + nodeDescriptor.getDataChannelDescriptor().getName() + "." + nodeDescriptor.getName();
 
-        Driver driver = objectFactory.newInstance(Driver.class, driverClass);
-        DriverDataSource driverDS = new DriverDataSource(driver, url, username, password);
-        driverDS.setLogger(jdbcEventLogger);
-        PoolDataSource poolDS = new PoolDataSource(driverDS);
+		String driverClass = getProperty(Constants.JDBC_DRIVER_PROPERTY, suffix);
+		String url = getProperty(Constants.JDBC_URL_PROPERTY, suffix);
+		String username = getProperty(Constants.JDBC_USERNAME_PROPERTY, suffix);
+		String password = getProperty(Constants.JDBC_PASSWORD_PROPERTY, suffix);
+		int minConnections = getIntProperty(Constants.JDBC_MIN_CONNECTIONS_PROPERTY, suffix, 1);
+		int maxConnections = getIntProperty(Constants.JDBC_MAX_CONNECTIONS_PROPERTY, suffix, 1);
+		long maxQueueWaitTime = properties.getLong(Constants.SERVER_MAX_QUEUE_WAIT_TIME,
+				PoolingDataSource.MAX_QUEUE_WAIT_DEFAULT);
 
-        try {
-            return new PoolManager(poolDS, minConnections, maxConnections, username, password, properties.getLong(
-                    Constants.SERVER_MAX_QUEUE_WAIT_TIME, PoolManager.MAX_QUEUE_WAIT_DEFAULT));
-        } catch (Exception e) {
-            jdbcEventLogger.logConnectFailure(e);
-            throw e;
-        }
-    }
+		return DataSourceBuilder.builder(objectFactory, jdbcEventLogger).driver(driverClass).url(url)
+				.userName(username).password(password).minConnections(minConnections).maxConnections(maxConnections)
+				.maxQueueWaitTime(maxQueueWaitTime).build();
+	}
 
-    protected int getIntProperty(String propertyName, String suffix, int defaultValue) {
-        String string = getProperty(propertyName, suffix);
+	protected int getIntProperty(String propertyName, String suffix, int defaultValue) {
+		String string = getProperty(propertyName, suffix);
 
-        if (string == null) {
-            return defaultValue;
-        }
+		if (string == null) {
+			return defaultValue;
+		}
 
-        try {
-            return Integer.parseInt(string);
-        } catch (NumberFormatException e) {
-            throw new ConfigurationException("Invalid int property '%s': '%s'", propertyName, string);
-        }
-    }
+		try {
+			return Integer.parseInt(string);
+		} catch (NumberFormatException e) {
+			throw new ConfigurationException("Invalid int property '%s': '%s'", propertyName, string);
+		}
+	}
 
-    protected String getProperty(String propertyName, String suffix) {
-        String value = properties.get(propertyName + suffix);
-        return value != null ? value : properties.get(propertyName);
-    }
+	protected String getProperty(String propertyName, String suffix) {
+		String value = properties.get(propertyName + suffix);
+		return value != null ? value : properties.get(propertyName);
+	}
 }

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/XMLPoolingDataSourceFactory.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/XMLPoolingDataSourceFactory.java b/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/XMLPoolingDataSourceFactory.java
index b3bad8d..a62a2d9 100644
--- a/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/XMLPoolingDataSourceFactory.java
+++ b/cayenne-server/src/main/java/org/apache/cayenne/configuration/server/XMLPoolingDataSourceFactory.java
@@ -18,18 +18,15 @@
  ****************************************************************/
 package org.apache.cayenne.configuration.server;
 
-import java.sql.Driver;
-
 import javax.sql.DataSource;
 
 import org.apache.cayenne.ConfigurationException;
 import org.apache.cayenne.configuration.Constants;
 import org.apache.cayenne.configuration.DataNodeDescriptor;
 import org.apache.cayenne.configuration.RuntimeProperties;
+import org.apache.cayenne.conn.DataSourceBuilder;
 import org.apache.cayenne.conn.DataSourceInfo;
-import org.apache.cayenne.conn.DriverDataSource;
-import org.apache.cayenne.conn.PoolDataSource;
-import org.apache.cayenne.conn.PoolManager;
+import org.apache.cayenne.conn.PoolingDataSource;
 import org.apache.cayenne.di.AdhocObjectFactory;
 import org.apache.cayenne.di.Inject;
 import org.apache.cayenne.log.JdbcEventLogger;
@@ -47,43 +44,35 @@ import org.apache.commons.logging.LogFactory;
 // something else?
 public class XMLPoolingDataSourceFactory implements DataSourceFactory {
 
-    private static final Log logger = LogFactory.getLog(XMLPoolingDataSourceFactory.class);
+	private static final Log logger = LogFactory.getLog(XMLPoolingDataSourceFactory.class);
 
-    @Inject
-    protected JdbcEventLogger jdbcEventLogger;
+	@Inject
+	protected JdbcEventLogger jdbcEventLogger;
 
-    @Inject
-    private RuntimeProperties properties;
+	@Inject
+	private RuntimeProperties properties;
 
-    @Inject
-    private AdhocObjectFactory objectFactory;
+	@Inject
+	private AdhocObjectFactory objectFactory;
 
-    @Override
-    public DataSource getDataSource(DataNodeDescriptor nodeDescriptor) throws Exception {
+	@Override
+	public DataSource getDataSource(DataNodeDescriptor nodeDescriptor) throws Exception {
 
-        DataSourceInfo dataSourceDescriptor = nodeDescriptor.getDataSourceDescriptor();
+		DataSourceInfo descriptor = nodeDescriptor.getDataSourceDescriptor();
 
-        if (dataSourceDescriptor == null) {
-            String message = "Null dataSourceDescriptor for nodeDescriptor '" + nodeDescriptor.getName() + "'";
-            logger.info(message);
-            throw new ConfigurationException(message);
-        }
+		if (descriptor == null) {
+			String message = "Null dataSourceDescriptor for nodeDescriptor '" + nodeDescriptor.getName() + "'";
+			logger.info(message);
+			throw new ConfigurationException(message);
+		}
 
-        Driver driver = objectFactory.newInstance(Driver.class, dataSourceDescriptor.getJdbcDriver());
-        DriverDataSource driverDS = new DriverDataSource(driver, dataSourceDescriptor.getDataSourceUrl(),
-                dataSourceDescriptor.getUserName(), dataSourceDescriptor.getPassword());
-        driverDS.setLogger(jdbcEventLogger);
-        PoolDataSource poolDS = new PoolDataSource(driverDS);
+		long maxQueueWaitTime = properties.getLong(Constants.SERVER_MAX_QUEUE_WAIT_TIME,
+				PoolingDataSource.MAX_QUEUE_WAIT_DEFAULT);
 
-        try {
-            return new PoolManager(poolDS, dataSourceDescriptor.getMinConnections(),
-                    dataSourceDescriptor.getMaxConnections(), dataSourceDescriptor.getUserName(),
-                    dataSourceDescriptor.getPassword(), properties.getLong(Constants.SERVER_MAX_QUEUE_WAIT_TIME,
-                            PoolManager.MAX_QUEUE_WAIT_DEFAULT));
-        } catch (Exception e) {
-            jdbcEventLogger.logConnectFailure(e);
-            throw e;
-        }
-    }
+		return DataSourceBuilder.builder(objectFactory, jdbcEventLogger).driver(descriptor.getJdbcDriver())
+				.url(descriptor.getDataSourceUrl()).userName(descriptor.getUserName())
+				.password(descriptor.getPassword()).minConnections(descriptor.getMinConnections())
+				.maxConnections(descriptor.getMaxConnections()).maxQueueWaitTime(maxQueueWaitTime).build();
+	}
 
 }

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/conn/DataSourceBuilder.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/conn/DataSourceBuilder.java b/cayenne-server/src/main/java/org/apache/cayenne/conn/DataSourceBuilder.java
new file mode 100644
index 0000000..c850d64
--- /dev/null
+++ b/cayenne-server/src/main/java/org/apache/cayenne/conn/DataSourceBuilder.java
@@ -0,0 +1,135 @@
+/*****************************************************************
+ *   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.cayenne.conn;
+
+import java.sql.Driver;
+
+import javax.sql.DataSource;
+
+import org.apache.cayenne.CayenneRuntimeException;
+import org.apache.cayenne.di.AdhocObjectFactory;
+import org.apache.cayenne.log.JdbcEventLogger;
+import org.apache.cayenne.log.NoopJdbcEventLogger;
+
+/**
+ * A builder class that creates a default Cayenne implementation of a pooling
+ * {@link DataSource}.
+ * 
+ * @since 4.0
+ */
+public class DataSourceBuilder {
+
+	private AdhocObjectFactory objectFactory;
+	private JdbcEventLogger logger;
+	private String userName;
+	private String password;
+	private String driver;
+	private String url;
+	private PoolingDataSourceParameters poolParameters;
+
+	public static DataSourceBuilder builder(AdhocObjectFactory objectFactory, JdbcEventLogger logger) {
+		return new DataSourceBuilder(objectFactory, logger);
+	}
+
+	private DataSourceBuilder(AdhocObjectFactory objectFactory, JdbcEventLogger logger) {
+		this.objectFactory = objectFactory;
+		this.logger = logger;
+		this.logger = NoopJdbcEventLogger.getInstance();
+		this.poolParameters = new PoolingDataSourceParameters();
+
+		poolParameters.setMinConnections(1);
+		poolParameters.setMaxConnections(1);
+		poolParameters.setMaxQueueWaitTime(PoolingDataSource.MAX_QUEUE_WAIT_DEFAULT);
+	}
+
+	public DataSourceBuilder userName(String userName) {
+		this.userName = userName;
+		return this;
+	}
+
+	public DataSourceBuilder password(String password) {
+		this.password = password;
+		return this;
+	}
+
+	public DataSourceBuilder driver(String driver) {
+		this.driver = driver;
+		return this;
+	}
+
+	public DataSourceBuilder url(String url) {
+		this.url = url;
+		return this;
+	}
+
+	public DataSourceBuilder minConnections(int minConnections) {
+		poolParameters.setMinConnections(minConnections);
+		return this;
+	}
+
+	public DataSourceBuilder maxConnections(int maxConnections) {
+		poolParameters.setMaxConnections(maxConnections);
+		return this;
+	}
+
+	public DataSourceBuilder maxQueueWaitTime(long maxQueueWaitTime) {
+		poolParameters.setMaxQueueWaitTime(maxQueueWaitTime);
+		return this;
+	}
+
+	public DataSource build() {
+
+		// sanity checks...
+		if (poolParameters.getMaxConnections() < 0) {
+			throw new CayenneRuntimeException("Maximum number of connections can not be negative ("
+					+ poolParameters.getMaxConnections() + ").");
+		}
+
+		if (poolParameters.getMinConnections() < 0) {
+			throw new CayenneRuntimeException("Minimum number of connections can not be negative ("
+					+ poolParameters.getMinConnections() + ").");
+		}
+
+		if (poolParameters.getMinConnections() > poolParameters.getMaxConnections()) {
+			throw new CayenneRuntimeException("Minimum number of connections can not be bigger then maximum.");
+		}
+
+		DataSource nonPooling = buildNonPoolingDataSource();
+		return buildPoolingDataSource(new PooledConnectionFactory(nonPooling));
+	}
+
+	private DataSource buildNonPoolingDataSource() {
+		Driver driver = objectFactory.newInstance(Driver.class, this.driver);
+		DriverDataSource dataSource = new DriverDataSource(driver, url, userName, password);
+		dataSource.setLogger(logger);
+		return dataSource;
+	}
+
+	private DataSource buildPoolingDataSource(PooledConnectionFactory connectionFactory) {
+		PoolingDataSource poolDS;
+		try {
+			poolDS = new PoolingDataSource(connectionFactory, poolParameters);
+		} catch (Exception e) {
+			logger.logConnectFailure(e);
+			throw new CayenneRuntimeException("Error creating DataSource", e);
+		}
+
+		return new ManagedPoolingDataSource(poolDS);
+	}
+}

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/conn/DataSourceInfo.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/conn/DataSourceInfo.java b/cayenne-server/src/main/java/org/apache/cayenne/conn/DataSourceInfo.java
index 44c95c1..2bb2664 100644
--- a/cayenne-server/src/main/java/org/apache/cayenne/conn/DataSourceInfo.java
+++ b/cayenne-server/src/main/java/org/apache/cayenne/conn/DataSourceInfo.java
@@ -39,6 +39,11 @@ public class DataSourceInfo implements Cloneable, Serializable, XMLSerializable
 
 	private static Log logger = LogFactory.getLog(DataSourceInfo.class);
 
+	public static final String PASSWORD_LOCATION_CLASSPATH = "classpath";
+	public static final String PASSWORD_LOCATION_EXECUTABLE = "executable";
+	public static final String PASSWORD_LOCATION_MODEL = "model";
+	public static final String PASSWORD_LOCATION_URL = "url";
+
 	protected String userName;
 	protected String password;
 	protected String jdbcDriver;
@@ -46,14 +51,6 @@ public class DataSourceInfo implements Cloneable, Serializable, XMLSerializable
 	protected String adapterClassName;
 	protected int minConnections = 1;
 	protected int maxConnections = 1;
-
-	// Constants for passwordLocation
-	public static final String PASSWORD_LOCATION_CLASSPATH = "classpath";
-	public static final String PASSWORD_LOCATION_EXECUTABLE = "executable";
-	public static final String PASSWORD_LOCATION_MODEL = "model";
-	public static final String PASSWORD_LOCATION_URL = "url";
-
-	// Extended parameters
 	protected String passwordEncoderClass = PlainTextPasswordEncoder.class.getName();
 	protected String passwordEncoderKey = "";
 	protected String passwordLocation = PASSWORD_LOCATION_MODEL;
@@ -64,43 +61,72 @@ public class DataSourceInfo implements Cloneable, Serializable, XMLSerializable
 
 	@Override
 	public boolean equals(Object obj) {
-		if (obj == this)
+		
+		if (obj == this) {
 			return true;
+		}
 
-		if (obj == null)
+		if (obj == null) {
 			return false;
+		}
 
-		if (obj.getClass() != this.getClass())
+		if (obj.getClass() != this.getClass()) {
 			return false;
+		}
 
 		DataSourceInfo dsi = (DataSourceInfo) obj;
 
-		if (!Util.nullSafeEquals(this.userName, dsi.userName))
+		if (!Util.nullSafeEquals(this.userName, dsi.userName)) {
 			return false;
-		if (!Util.nullSafeEquals(this.password, dsi.password))
+		}
+
+		if (!Util.nullSafeEquals(this.password, dsi.password)) {
 			return false;
-		if (!Util.nullSafeEquals(this.jdbcDriver, dsi.jdbcDriver))
+		}
+
+		if (!Util.nullSafeEquals(this.jdbcDriver, dsi.jdbcDriver)) {
 			return false;
-		if (!Util.nullSafeEquals(this.dataSourceUrl, dsi.dataSourceUrl))
+		}
+
+		if (!Util.nullSafeEquals(this.dataSourceUrl, dsi.dataSourceUrl)) {
 			return false;
-		if (!Util.nullSafeEquals(this.adapterClassName, dsi.adapterClassName))
+		}
+
+		if (!Util.nullSafeEquals(this.adapterClassName, dsi.adapterClassName)) {
 			return false;
-		if (this.minConnections != dsi.minConnections)
+		}
+
+		if (this.minConnections != dsi.minConnections) {
 			return false;
-		if (this.maxConnections != dsi.maxConnections)
+		}
+
+		if (this.maxConnections != dsi.maxConnections) {
 			return false;
-		if (!Util.nullSafeEquals(this.passwordEncoderClass, dsi.passwordEncoderClass))
+		}
+
+		if (!Util.nullSafeEquals(this.passwordEncoderClass, dsi.passwordEncoderClass)) {
 			return false;
-		if (!Util.nullSafeEquals(this.passwordEncoderKey, dsi.passwordEncoderKey))
+		}
+
+		if (!Util.nullSafeEquals(this.passwordEncoderKey, dsi.passwordEncoderKey)) {
 			return false;
-		if (!Util.nullSafeEquals(this.passwordSourceFilename, dsi.passwordSourceFilename))
+		}
+
+		if (!Util.nullSafeEquals(this.passwordSourceFilename, dsi.passwordSourceFilename)) {
 			return false;
-		if (!Util.nullSafeEquals(this.passwordSourceModel, dsi.passwordSourceModel))
+		}
+
+		if (!Util.nullSafeEquals(this.passwordSourceModel, dsi.passwordSourceModel)) {
 			return false;
-		if (!Util.nullSafeEquals(this.passwordSourceUrl, dsi.passwordSourceUrl))
+		}
+
+		if (!Util.nullSafeEquals(this.passwordSourceUrl, dsi.passwordSourceUrl)) {
 			return false;
-		if (!Util.nullSafeEquals(this.passwordLocation, dsi.passwordLocation))
+		}
+
+		if (!Util.nullSafeEquals(this.passwordLocation, dsi.passwordLocation)) {
 			return false;
+		}
 
 		return true;
 	}

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/conn/ManagedPoolingDataSource.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/conn/ManagedPoolingDataSource.java b/cayenne-server/src/main/java/org/apache/cayenne/conn/ManagedPoolingDataSource.java
new file mode 100644
index 0000000..357c3e0
--- /dev/null
+++ b/cayenne-server/src/main/java/org/apache/cayenne/conn/ManagedPoolingDataSource.java
@@ -0,0 +1,155 @@
+/*****************************************************************
+ *   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.cayenne.conn;
+
+import java.io.PrintWriter;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.logging.Logger;
+
+import javax.sql.DataSource;
+
+import org.apache.cayenne.di.ScopeEventListener;
+
+/**
+ * A wrapper for {@link PoolingDataSourceManager} that manages the underlying
+ * connection pool size, shrinking it if needed.
+ * 
+ * @since 4.0
+ */
+public class ManagedPoolingDataSource implements DataSource, ScopeEventListener {
+
+	private PoolingDataSourceManager dataSourceManager;
+	private PoolingDataSource dataSource;
+
+	public ManagedPoolingDataSource(PoolingDataSource dataSource) {
+
+		this.dataSource = dataSource;
+		this.dataSourceManager = new PoolingDataSourceManager();
+
+		dataSourceManager.start();
+	}
+
+	@Override
+	public void beforeScopeEnd() {
+		dataSourceManager.shouldStop();
+	}
+
+	@Override
+	public Connection getConnection() throws SQLException {
+		return dataSource.getConnection();
+	}
+
+	@Override
+	public Connection getConnection(String username, String password) throws SQLException {
+		return dataSource.getConnection(username, password);
+	}
+
+	@Override
+	public PrintWriter getLogWriter() throws SQLException {
+		return dataSource.getLogWriter();
+	}
+
+	@Override
+	public int getLoginTimeout() throws SQLException {
+		return dataSource.getLoginTimeout();
+	}
+
+	@Override
+	public boolean isWrapperFor(Class<?> iface) throws SQLException {
+		return (ManagedPoolingDataSource.class.equals(iface)) ? true : dataSource.isWrapperFor(iface);
+	}
+
+	@Override
+	public void setLogWriter(PrintWriter arg0) throws SQLException {
+		dataSource.setLogWriter(arg0);
+	}
+
+	@Override
+	public void setLoginTimeout(int arg0) throws SQLException {
+		dataSource.setLoginTimeout(arg0);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public <T> T unwrap(Class<T> iface) throws SQLException {
+		return ManagedPoolingDataSource.class.equals(iface) ? (T) this : dataSource.unwrap(iface);
+	}
+
+	// JDBC 4.1 compatibility under Java 1.6 and newer
+	public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+		throw new UnsupportedOperationException();
+	}
+
+	boolean shouldShrinkPool() {
+		int unused = dataSource.getCurrentlyUnused();
+		int used = dataSource.getCurrentlyInUse();
+		int total = unused + used;
+		int median = dataSource.getMinConnections() + 1
+				+ (dataSource.getMaxConnections() - dataSource.getMinConnections()) / 2;
+
+		return unused > 0 && total > median;
+	}
+
+	class PoolingDataSourceManager extends Thread {
+
+		private volatile boolean shouldStop;
+
+		PoolingDataSourceManager() {
+			setName("PoolManagerCleanup-" + dataSource.hashCode());
+			setDaemon(true);
+			this.shouldStop = false;
+		}
+
+		public void shouldStop() {
+			shouldStop = true;
+			interrupt();
+		}
+
+		@Override
+		public void run() {
+			while (true) {
+
+				try {
+					// don't do it too often
+					Thread.sleep(600000);
+				} catch (InterruptedException iex) {
+					// ignore...
+				}
+
+				synchronized (dataSource) {
+
+					// simple pool management - close one connection if the
+					// count is
+					// above median and there are any idle connections.
+
+					if (shouldStop) {
+						break;
+					}
+
+					if (shouldShrinkPool()) {
+						dataSource.shrinkPool(1);
+					}
+				}
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolDataSource.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolDataSource.java b/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolDataSource.java
deleted file mode 100644
index ad0b33e..0000000
--- a/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolDataSource.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*****************************************************************
- *   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.cayenne.conn;
-
-import java.io.PrintWriter;
-import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
-import java.util.logging.Logger;
-
-import javax.sql.ConnectionPoolDataSource;
-import javax.sql.DataSource;
-import javax.sql.PooledConnection;
-
-/**
- * PoolDataSource allows to generate pooled connections.
- *
- * <p>
- * It is implemented as a wrapper around a non-pooled data source object.
- * Delegates all method calls except for "getPooledConnection" to the underlying
- * DataSource.
- * 
- */
-public class PoolDataSource implements ConnectionPoolDataSource {
-
-	private DataSource nonPooledDatasource;
-
-	public PoolDataSource(DataSource nonPooledDatasource) {
-		this.nonPooledDatasource = nonPooledDatasource;
-	}
-
-	public PoolDataSource(String jdbcDriver, String connectionUrl) throws SQLException {
-		nonPooledDatasource = new DriverDataSource(jdbcDriver, connectionUrl);
-	}
-
-	@Override
-	public int getLoginTimeout() throws SQLException {
-		return nonPooledDatasource.getLoginTimeout();
-	}
-
-	@Override
-	public void setLoginTimeout(int seconds) throws SQLException {
-		nonPooledDatasource.setLoginTimeout(seconds);
-	}
-
-	@Override
-	public PrintWriter getLogWriter() throws SQLException {
-		return nonPooledDatasource.getLogWriter();
-	}
-
-	@Override
-	public void setLogWriter(PrintWriter out) throws SQLException {
-		nonPooledDatasource.setLogWriter(out);
-	}
-
-	@Override
-	public PooledConnection getPooledConnection() throws SQLException {
-		return new PooledConnectionImpl(nonPooledDatasource, null, null);
-	}
-
-	@Override
-	public PooledConnection getPooledConnection(String user, String password) throws SQLException {
-		return new PooledConnectionImpl(nonPooledDatasource, user, password);
-	}
-
-	/**
-	 * @since 3.1
-	 *
-	 *        JDBC 4.1 compatibility under Java 1.7
-	 */
-	public Logger getParentLogger() throws SQLFeatureNotSupportedException {
-		throw new UnsupportedOperationException();
-	}
-}

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolManager.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolManager.java b/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolManager.java
deleted file mode 100644
index 1a71024..0000000
--- a/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolManager.java
+++ /dev/null
@@ -1,649 +0,0 @@
-/*****************************************************************
- *   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.cayenne.conn;
-
-import java.io.PrintWriter;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.logging.Logger;
-
-import javax.sql.ConnectionEvent;
-import javax.sql.ConnectionEventListener;
-import javax.sql.ConnectionPoolDataSource;
-import javax.sql.DataSource;
-import javax.sql.PooledConnection;
-
-import org.apache.cayenne.CayenneRuntimeException;
-import org.apache.cayenne.di.ScopeEventListener;
-import org.apache.cayenne.log.JdbcEventLogger;
-
-/**
- * PoolManager is a Cayenne implementation of a pooling DataSource.
- */
-public class PoolManager implements ScopeEventListener, DataSource,
-        ConnectionEventListener {
-
-    /**
-     * Defines a maximum time in milliseconds that a connection request could wait in the
-     * connection queue. After this period expires, an exception will be thrown in the
-     * calling method. 
-     */
-    public static final int MAX_QUEUE_WAIT_DEFAULT = 20000;
-
-    /**
-     * An exception indicating that a connection request waiting in the queue
-     * timed out and was unable to obtain a connection.
-     */
-    public static class ConnectionUnavailableException extends SQLException {
-        private static final long serialVersionUID = 1063973806941023165L;
-
-        public ConnectionUnavailableException(String message) {
-    		super(message);
-    	}
-    }
-    
-    protected ConnectionPoolDataSource poolDataSource;
-    protected int minConnections;
-    protected int maxConnections;
-    protected String dataSourceUrl;
-    protected String jdbcDriver;
-    protected String password;
-    protected String userName;
-
-    protected List<PooledConnection> unusedPool;
-    protected List<PooledConnection> usedPool;
-
-    private PoolMaintenanceThread poolMaintenanceThread;
-
-    private boolean shuttingDown;
-    private long maxQueueWaitTime;
-    
-    /**
-     * Creates new PoolManager using org.apache.cayenne.conn.PoolDataSource for an
-     * underlying ConnectionPoolDataSource.
-     * 
-     * @deprecated since 4.0 This constructor causes implicit class loading that should avoided.
-     */
-    @Deprecated
-    public PoolManager(String jdbcDriver, String dataSourceUrl, int minCons, int maxCons,
-            String userName, String password) throws SQLException {
-
-        this(jdbcDriver, dataSourceUrl, minCons, maxCons, userName, password, null, MAX_QUEUE_WAIT_DEFAULT);
-    }
-
-    /**
-     * @deprecated since 4.0 This constructor causes implicit class loading that should avoided.
-     */
-    @Deprecated
-    public PoolManager(String jdbcDriver, String dataSourceUrl, int minCons, int maxCons,
-            String userName, String password, JdbcEventLogger logger, long maxQueueWaitTime) throws SQLException {
-
-        if (logger != null) {
-            DataSourceInfo info = new DataSourceInfo();
-            info.setJdbcDriver(jdbcDriver);
-            info.setDataSourceUrl(dataSourceUrl);
-            info.setMinConnections(minCons);
-            info.setMaxConnections(maxCons);
-            info.setUserName(userName);
-            info.setPassword(password);
-            logger.logPoolCreated(info);
-        }
-
-        this.jdbcDriver = jdbcDriver;
-        this.dataSourceUrl = dataSourceUrl;
-        DriverDataSource driverDS = new DriverDataSource(jdbcDriver, dataSourceUrl);
-        driverDS.setLogger(logger);
-        PoolDataSource poolDS = new PoolDataSource(driverDS);
-        init(poolDS, minCons, maxCons, userName, password, maxQueueWaitTime);
-    }
-    
-    /**
-     * Creates new PoolManager with the specified policy for connection pooling and a
-     * ConnectionPoolDataSource object.
-     * 
-     * @param poolDataSource data source for pooled connections
-     * @param minCons Non-negative integer that specifies a minimum number of open
-     *            connections to keep in the pool at all times
-     * @param maxCons Non-negative integer that specifies maximum number of simultaneuosly
-     *            open connections
-     * @throws SQLException if pool manager can not be created.
-     * @deprecated since 4.0 use {@link #PoolManager(ConnectionPoolDataSource, int, int, String, String, long)}
-     */
-    public PoolManager(ConnectionPoolDataSource poolDataSource, int minCons, int maxCons,
-            String userName, String password) throws SQLException {
-        this(poolDataSource, minCons, maxCons, userName, password, PoolManager.MAX_QUEUE_WAIT_DEFAULT);
-    }
-
-    /**
-     * Creates new PoolManager with the specified policy for connection pooling and a
-     * ConnectionPoolDataSource object.
-     * 
-     * @param poolDataSource data source for pooled connections
-     * @param minCons Non-negative integer that specifies a minimum number of open
-     *            connections to keep in the pool at all times
-     * @param maxCons Non-negative integer that specifies maximum number of simultaneuosly
-     *            open connections
-     * @throws SQLException if pool manager can not be created.
-     * @since 4.0
-     */
-    public PoolManager(ConnectionPoolDataSource poolDataSource, int minCons, int maxCons,
-            String userName, String password, long maxQueueWaitTime) throws SQLException {
-        init(poolDataSource, minCons, maxCons, userName, password, maxQueueWaitTime);
-    }
-
-    /** Initializes pool. Normally called from constructor. */
-    protected void init(
-            ConnectionPoolDataSource poolDataSource,
-            int minCons,
-            int maxCons,
-            String userName,
-            String password,
-            long maxQueueWaitTime) throws SQLException {
-
-        // do sanity checks...
-        if (maxConnections < 0) {
-            throw new SQLException("Maximum number of connections can not be negative ("
-                    + maxCons
-                    + ").");
-        }
-
-        if (minConnections < 0) {
-            throw new SQLException("Minimum number of connections can not be negative ("
-                    + minCons
-                    + ").");
-        }
-
-        if (minConnections > maxConnections) {
-            throw new SQLException(
-                    "Minimum number of connections can not be bigger then maximum.");
-        }
-
-        // init properties
-        this.userName = userName;
-        this.password = password;
-        this.minConnections = minCons;
-        this.maxConnections = maxCons;
-        this.poolDataSource = poolDataSource;
-        this.maxQueueWaitTime = maxQueueWaitTime;
-        
-        // init pool... use linked lists to use the queue in the FIFO manner
-        usedPool = new LinkedList<PooledConnection>();
-        unusedPool = new LinkedList<PooledConnection>();
-        growPool(minConnections, userName, password);
-
-        startMaintenanceThread();
-    }
-
-    protected synchronized void startMaintenanceThread() {
-        disposeOfMaintenanceThread();
-        this.poolMaintenanceThread = new PoolMaintenanceThread(this);
-        this.poolMaintenanceThread.start();
-    }
-
-    /**
-     * Creates and returns new PooledConnection object, adding itself as a listener for
-     * connection events.
-     */
-    protected PooledConnection newPooledConnection(String userName, String password)
-            throws SQLException {
-        PooledConnection connection = (userName != null) ? poolDataSource
-                .getPooledConnection(userName, password) : poolDataSource
-                .getPooledConnection();
-        connection.addConnectionEventListener(this);
-        return connection;
-    }
-
-    /**
-     * Closes all existing connections, drains the pool and stops the maintenance thread.
-     * 
-     * @since 3.1
-     */
-    public synchronized void shutdown() throws SQLException {
-
-        // disposing maintenance thread first to avoid any changes to pools
-        // during shutdown
-        disposeOfMaintenanceThread();
-
-        // using boolean variable instead of locking PoolManager instance due to
-        // possible deadlock during shutdown when one of connections locks its
-        // event listeners list trying to invoke locked PoolManager's listener methods
-        shuttingDown = true;
-
-        ListIterator<PooledConnection> unusedIterator = unusedPool.listIterator();
-        while (unusedIterator.hasNext()) {
-            PooledConnection con = unusedIterator.next();
-            // close connection
-            con.close();
-            // remove connection from the list
-            unusedIterator.remove();
-        }
-
-        // clean used connections
-        ListIterator<PooledConnection> usedIterator = usedPool.listIterator();
-        while (usedIterator.hasNext()) {
-            PooledConnection con = usedIterator.next();
-            // stop listening for connection events
-            con.removeConnectionEventListener(this);
-            // close connection
-            con.close();
-            // remove connection from the list
-            usedIterator.remove();
-        }
-    }
-
-    /**
-     * An implementation of {@link ScopeEventListener} that simply calls
-     * {@link #shutdown()}.
-     * 
-     * @since 3.1
-     */
-    public void beforeScopeEnd() {
-        try {
-            shutdown();
-        }
-        catch (SQLException e) {
-            throw new CayenneRuntimeException("Error while shutting down");
-        }
-    }
-
-    protected void disposeOfMaintenanceThread() {
-        if (poolMaintenanceThread != null) {
-            poolMaintenanceThread.shutdown();
-            poolMaintenanceThread = null;
-        }
-    }
-
-    /**
-     * @return true if at least one more connection can be added to the pool.
-     */
-    protected synchronized boolean canGrowPool() {
-        return getPoolSize() < maxConnections;
-    }
-
-    /**
-     * Increases connection pool by the specified number of connections.
-     * 
-     * @return the actual number of created connections.
-     * @throws SQLException if an error happens when creating a new connection.
-     */
-    protected synchronized int growPool(
-            int addConnections,
-            String userName,
-            String password) throws SQLException {
-
-        int i = 0;
-        int startPoolSize = getPoolSize();
-        for (; i < addConnections && startPoolSize + i < maxConnections; i++) {
-            PooledConnection newConnection = newPooledConnection(userName, password);
-            unusedPool.add(newConnection);
-        }
-
-        return i;
-    }
-
-    protected synchronized void shrinkPool(int closeConnections) {
-        int idleSize = unusedPool.size();
-        for (int i = 0; i < closeConnections && i < idleSize; i++) {
-            PooledConnection con = unusedPool.remove(i);
-
-            try {
-                con.close();
-            }
-            catch (SQLException ex) {
-                // ignore
-            }
-        }
-    }
-
-    /**
-     * Returns maximum number of connections this pool can keep. This parameter when
-     * configured allows to limit the number of simultaneously open connections.
-     */
-    public int getMaxConnections() {
-        return maxConnections;
-    }
-
-    public void setMaxConnections(int maxConnections) {
-        this.maxConnections = maxConnections;
-    }
-
-    /**
-     * Returns the absolute minimum number of connections allowed in this pool at any
-     * moment in time.
-     */
-    public int getMinConnections() {
-        return minConnections;
-    }
-
-    public void setMinConnections(int minConnections) {
-        this.minConnections = minConnections;
-    }
-
-    /**
-     * Returns a database URL used to initialize this pool. Will return null if the pool
-     * was initialized with ConnectionPoolDataSource.
-     */
-    public String getDataSourceUrl() {
-        return dataSourceUrl;
-    }
-
-    /**
-     * Returns a name of a JDBC driver used to initialize this pool. Will return null if
-     * the pool was initialized with ConnectionPoolDataSource.
-     */
-    public String getJdbcDriver() {
-        return jdbcDriver;
-    }
-
-    /** Returns a data source password used to initialize this pool. */
-    public String getPassword() {
-        return password;
-    }
-
-    /** Returns a data source user name used to initialize this pool. */
-    public String getUserName() {
-        return userName;
-    }
-
-    /**
-     * Returns current number of connections.
-     */
-    public synchronized int getPoolSize() {
-        return usedPool.size() + unusedPool.size();
-    }
-
-    /**
-     * Returns the number of connections obtained via this DataSource that are currently
-     * in use by the DataSource clients.
-     */
-    public synchronized int getCurrentlyInUse() {
-        return usedPool.size();
-    }
-
-    /**
-     * Returns the number of connections maintained in the pool that are currently not
-     * used by any clients and are available immediately via <code>getConnection</code>
-     * method.
-     */
-    public synchronized int getCurrentlyUnused() {
-        return unusedPool.size();
-    }
-
-    /**
-     * Returns connection from the pool using internal values of user name and password.
-     * Equivalent to calling:
-     * <p>
-     * <code>ds.getConnection(ds.getUserName(), ds.getPassword())</code>
-     * </p>
-     */
-    public Connection getConnection() throws SQLException {
-        return getConnection(userName, password);
-    }
-
-    /** Returns connection from the pool. */
-    public synchronized Connection getConnection(String userName, String password)
-            throws SQLException {
-
-        if (shuttingDown) {
-            throw new SQLException("Pool manager is shutting down.");
-        }
-
-        PooledConnection pooledConnection = uncheckPooledConnection(userName, password);
-
-        try {
-            return uncheckConnection(pooledConnection);
-        }
-        catch (SQLException ex) {
-
-            try {
-                pooledConnection.close();
-            }
-            catch (SQLException ignored) {
-            }
-
-            // do one reconnect attempt...
-            pooledConnection = uncheckPooledConnection(userName, password);
-            try {
-                return uncheckConnection(pooledConnection);
-            }
-            catch (SQLException reconnectEx) {
-                try {
-                    pooledConnection.close();
-                }
-                catch (SQLException ignored) {
-                }
-
-                throw reconnectEx;
-            }
-        }
-    }
-
-    private Connection uncheckConnection(PooledConnection pooledConnection)
-            throws SQLException {
-        Connection c = pooledConnection.getConnection();
-
-        // only do that on successfully unchecked connection...
-        usedPool.add(pooledConnection);
-        return c;
-    }
-
-    private PooledConnection uncheckPooledConnection(String userName, String password)
-            throws SQLException {
-        // wait for returned connections or the maintenance thread
-        // to bump the pool size...
-
-        if (unusedPool.size() == 0) {
-
-            // first try to open a new connection
-            if (canGrowPool()) {
-                return newPooledConnection(userName, password);
-            }
-
-            // can't open no more... will have to wait for others to return a connection
-
-            // note that if we were woken up
-            // before the full wait period expired, and no connections are
-            // available yet, go back to sleep. Otherwise we don't give a maintenance
-            // thread a chance to increase pool size
-            long waitTill = System.currentTimeMillis() + maxQueueWaitTime;
-        	
-            do {
-                try {
-                    wait(maxQueueWaitTime);
-                }
-                catch (InterruptedException iex) {
-                    // ignoring
-                }
-
-            } while (unusedPool.size() == 0 && (maxQueueWaitTime == 0 || waitTill > System.currentTimeMillis()));
-
-            if (unusedPool.size() == 0) {
-                throw new ConnectionUnavailableException(
-                        "Can't obtain connection. Request timed out. Total used connections: "
-                                + usedPool.size());
-            }
-        }
-
-        // get first connection... lets cycle them in FIFO manner
-        return unusedPool.remove(0);
-    }
-
-    public int getLoginTimeout() throws java.sql.SQLException {
-        return poolDataSource.getLoginTimeout();
-    }
-
-    public void setLoginTimeout(int seconds) throws java.sql.SQLException {
-        poolDataSource.setLoginTimeout(seconds);
-    }
-
-    public PrintWriter getLogWriter() throws java.sql.SQLException {
-        return poolDataSource.getLogWriter();
-    }
-
-    public void setLogWriter(PrintWriter out) throws java.sql.SQLException {
-        poolDataSource.setLogWriter(out);
-    }
-
-    /**
-     * Returns closed connection to the pool.
-     */
-    public synchronized void connectionClosed(ConnectionEvent event) {
-
-        if (shuttingDown) {
-            return;
-        }
-
-        // return connection to the pool
-        PooledConnection closedConn = (PooledConnection) event.getSource();
-
-        // remove this connection from the list of connections
-        // managed by this pool...
-        int usedInd = usedPool.indexOf(closedConn);
-        if (usedInd >= 0) {
-            usedPool.remove(usedInd);
-            unusedPool.add(closedConn);
-
-            // notify threads waiting for connections
-            notifyAll();
-        }
-        // else ....
-        // other possibility is that this is a bad connection, so just ignore its closing
-        // event,
-        // since it was unregistered in "connectionErrorOccurred"
-    }
-
-    /**
-     * Removes connection with an error from the pool. This method is called by
-     * PoolManager connections on connection errors to notify PoolManager that connection
-     * is in invalid state.
-     */
-    public synchronized void connectionErrorOccurred(ConnectionEvent event) {
-
-        if (shuttingDown) {
-            return;
-        }
-
-        // later on we should analyze the error to see if this
-        // is fatal... right now just kill this PooledConnection
-
-        PooledConnection errorSrc = (PooledConnection) event.getSource();
-
-        // remove this connection from the list of connections
-        // managed by this pool...
-
-        int usedInd = usedPool.indexOf(errorSrc);
-        if (usedInd >= 0) {
-            usedPool.remove(usedInd);
-        }
-        else {
-            int unusedInd = unusedPool.indexOf(errorSrc);
-            if (unusedInd >= 0)
-                unusedPool.remove(unusedInd);
-        }
-
-        // do not close connection,
-        // let the code that catches the exception handle it
-        // ....
-    }
-
-    static class PoolMaintenanceThread extends Thread {
-
-        private boolean shouldDie;
-        private PoolManager pool;
-
-        PoolMaintenanceThread(PoolManager pool) {
-            super.setName("PoolManagerCleanup-" + pool.hashCode());
-            super.setDaemon(true);
-            this.pool = pool;
-        }
-
-        @Override
-        public void run() {
-            // periodically wakes up to check if the pool should grow or shrink
-            while (true) {
-
-                try {
-                    // don't do it too often
-                    sleep(600000);
-                }
-                catch (InterruptedException iex) {
-                    // ignore...
-                }
-
-                synchronized (pool) {
-                    // TODO: implement a smarter algorithm for pool management...
-                    // right now it will simply close one connection if the count is
-                    // above median and there are any idle connections.
-
-                    if (shouldDie) {
-                        break;
-                    }
-
-                    int unused = pool.getCurrentlyUnused();
-                    int used = pool.getCurrentlyInUse();
-                    int total = unused + used;
-                    int median = pool.minConnections
-                            + 1
-                            + (pool.maxConnections - pool.minConnections)
-                            / 2;
-
-                    if (unused > 0 && total > median) {
-                        pool.shrinkPool(1);
-                    }
-                }
-            }
-        }
-
-        /**
-         * Stops the maintenance thread.
-         */
-        void shutdown() {
-            shouldDie = true;
-            interrupt();
-        }
-    }
-
-    /**
-     * @since 3.0
-     */
-    // JDBC 4 compatibility under Java 1.5
-    public boolean isWrapperFor(Class<?> iface) throws SQLException {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @since 3.0
-     */
-    // JDBC 4 compatibility under Java 1.5
-    public <T> T unwrap(Class<T> iface) throws SQLException {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @since 3.1 JDBC 4.1 compatibility under Java 1.5
-     */
-    public Logger getParentLogger() throws SQLFeatureNotSupportedException {
-        throw new UnsupportedOperationException();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/conn/PooledConnectionFactory.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/conn/PooledConnectionFactory.java b/cayenne-server/src/main/java/org/apache/cayenne/conn/PooledConnectionFactory.java
new file mode 100644
index 0000000..2ebfeab
--- /dev/null
+++ b/cayenne-server/src/main/java/org/apache/cayenne/conn/PooledConnectionFactory.java
@@ -0,0 +1,90 @@
+/*****************************************************************
+ *   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.cayenne.conn;
+
+import java.io.PrintWriter;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.logging.Logger;
+
+import javax.sql.ConnectionPoolDataSource;
+import javax.sql.DataSource;
+import javax.sql.PooledConnection;
+
+/**
+ * A {@link ConnectionPoolDataSource} implementation.
+ * <p>
+ * It is implemented as a wrapper around a non-pooled data source object.
+ * Delegates all method calls except for "getPooledConnection" to the underlying
+ * DataSource.
+ * 
+ * @since 4.0
+ */
+public class PooledConnectionFactory implements ConnectionPoolDataSource {
+
+	private DataSource nonPooledDatasource;
+
+	public PooledConnectionFactory(DataSource nonPooledDatasource) {
+		this.nonPooledDatasource = nonPooledDatasource;
+	}
+
+	public PooledConnectionFactory(String jdbcDriver, String connectionUrl) throws SQLException {
+		nonPooledDatasource = new DriverDataSource(jdbcDriver, connectionUrl);
+	}
+
+	@Override
+	public int getLoginTimeout() throws SQLException {
+		return nonPooledDatasource.getLoginTimeout();
+	}
+
+	@Override
+	public void setLoginTimeout(int seconds) throws SQLException {
+		nonPooledDatasource.setLoginTimeout(seconds);
+	}
+
+	@Override
+	public PrintWriter getLogWriter() throws SQLException {
+		return nonPooledDatasource.getLogWriter();
+	}
+
+	@Override
+	public void setLogWriter(PrintWriter out) throws SQLException {
+		nonPooledDatasource.setLogWriter(out);
+	}
+
+	@Override
+	public PooledConnection getPooledConnection() throws SQLException {
+		return new PooledConnectionImpl(nonPooledDatasource, null, null);
+	}
+
+	@Override
+	public PooledConnection getPooledConnection(String user, String password) throws SQLException {
+		return new PooledConnectionImpl(nonPooledDatasource, user, password);
+	}
+
+	/**
+	 * @since 3.1
+	 *
+	 *        JDBC 4.1 compatibility under Java 1.7
+	 */
+	public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+		throw new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/conn/PooledConnectionImpl.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/conn/PooledConnectionImpl.java b/cayenne-server/src/main/java/org/apache/cayenne/conn/PooledConnectionImpl.java
index f63dcb8..eb327ff 100644
--- a/cayenne-server/src/main/java/org/apache/cayenne/conn/PooledConnectionImpl.java
+++ b/cayenne-server/src/main/java/org/apache/cayenne/conn/PooledConnectionImpl.java
@@ -32,170 +32,164 @@ import javax.sql.PooledConnection;
 import javax.sql.StatementEventListener;
 
 /**
- * PooledConnectionImpl is an implementation of a pooling wrapper for the database
- * connection as per JDBC3 spec. Most of the modern JDBC drivers should have its own
- * implementation that may be used instead of this class.
- * 
+ * PooledConnectionImpl is an implementation of a pooling wrapper for the
+ * database connection as per JDBC3 spec. Most of the modern JDBC drivers should
+ * have its own implementation that may be used instead of this class.
  */
 public class PooledConnectionImpl implements PooledConnection {
 
-    private Connection connectionObj;
-    private List<ConnectionEventListener> connectionEventListeners;
-    private boolean hadErrors;
-    private DataSource connectionSource;
-    private String userName;
-    private String password;
-
-    protected PooledConnectionImpl() {
-        // TODO: maybe remove synchronization and use
-        // FastArrayList from commons-collections? After
-        // all the only listener is usually pool manager.
-        this.connectionEventListeners = Collections
-                .synchronizedList(new ArrayList<ConnectionEventListener>(10));
-    }
-
-    /** Creates new PooledConnection */
-    public PooledConnectionImpl(DataSource connectionSource, String userName,
-            String password) {
-
-        this();
-
-        this.connectionSource = connectionSource;
-        this.userName = userName;
-        this.password = password;
-
-    }
-
-    public void reconnect() throws SQLException {
-        if (connectionObj != null) {
-            try {
-                connectionObj.close();
-            }
-            catch (SQLException ex) {
-                // ignore exception, since connection is expected
-                // to be in a bad state
-            }
-            finally {
-                connectionObj = null;
-            }
-        }
-
-        connectionObj = (userName != null) ? connectionSource.getConnection(
-                userName,
-                password) : connectionSource.getConnection();
-    }
-
-    public void addConnectionEventListener(ConnectionEventListener listener) {
-        synchronized (connectionEventListeners) {
-            if (!connectionEventListeners.contains(listener))
-                connectionEventListeners.add(listener);
-        }
-    }
-
-    public void removeConnectionEventListener(ConnectionEventListener listener) {
-        synchronized (connectionEventListeners) {
-            connectionEventListeners.remove(listener);
-        }
-    }
-
-    public void close() throws SQLException {
-
-        synchronized (connectionEventListeners) {
-            // remove all listeners
-            connectionEventListeners.clear();
-        }
-
-        if (connectionObj != null) {
-            try {
-                connectionObj.close();
-            }
-            finally {
-                connectionObj = null;
-            }
-        }
-    }
-
-    public Connection getConnection() throws SQLException {
-        if (connectionObj == null) {
-            reconnect();
-        }
-
-        // set autocommit to false to return connection
-        // always in consistent state
-        if (!connectionObj.getAutoCommit()) {
-
-            try {
-                connectionObj.setAutoCommit(true);
-            }
-            catch (SQLException sqlEx) {
-                // try applying Sybase patch
-                ConnectionWrapper.sybaseAutoCommitPatch(connectionObj, sqlEx, true);
-            }
-        }
-
-        connectionObj.clearWarnings();
-        return new ConnectionWrapper(connectionObj, this);
-    }
-
-    protected void returnConnectionToThePool() throws SQLException {
-        // do not return to pool bad connections
-        if (hadErrors)
-            close();
-        else
-            // notify the listeners that connection is no longer used by application...
-            this.connectionClosedNotification();
-    }
-
-    /**
-     * This method creates and sents an event to listeners when an error occurs in the
-     * underlying connection. Listeners can have special logic to analyze the error and do
-     * things like closing this PooledConnection (if the error is fatal), etc...
-     */
-    public void connectionErrorNotification(SQLException exception) {
-        // hint for later to avoid returning bad connections to the pool
-        hadErrors = true;
-
-        synchronized (connectionEventListeners) {
-            if (connectionEventListeners.size() == 0)
-                return;
-
-            ConnectionEvent closedEvent = new ConnectionEvent(this, exception);
-            for (final ConnectionEventListener nextListener : connectionEventListeners) {
-                nextListener.connectionErrorOccurred(closedEvent);
-            }
-        }
-    }
-
-    /**
-     * Creates and sends an event to listeners when a user closes java.sql.Connection
-     * object belonging to this PooledConnection.
-     */
-    protected void connectionClosedNotification() {
-        synchronized (connectionEventListeners) {
-            if (connectionEventListeners.size() == 0)
-                return;
-
-            ConnectionEvent closedEvent = new ConnectionEvent(this);
-
-            for (final ConnectionEventListener nextListener : connectionEventListeners) {
-                nextListener.connectionClosed(closedEvent);
-            }
-        }
-    }
-
-    /**
-     * @since 3.0
-     */
-    // JDBC 4 compatibility under Java 1.5
-    public void addStatementEventListener(StatementEventListener listener) {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @since 3.0
-     */
-    // JDBC 4 compatibility under Java 1.5
-    public void removeStatementEventListener(StatementEventListener listener) {
-        throw new UnsupportedOperationException();
-    }
+	private Connection connectionObj;
+	private List<ConnectionEventListener> connectionEventListeners;
+	private boolean hadErrors;
+	private DataSource dataSource;
+	private String userName;
+	private String password;
+
+	public PooledConnectionImpl(DataSource dataSource, String userName, String password) {
+
+		// TODO: maybe remove synchronization and use
+		// FastArrayList from commons-collections? After
+		// all the only listener is usually pool manager.
+		this.connectionEventListeners = Collections.synchronizedList(new ArrayList<ConnectionEventListener>(10));
+
+		this.dataSource = dataSource;
+		this.userName = userName;
+		this.password = password;
+	}
+
+	public void reconnect() throws SQLException {
+		if (connectionObj != null) {
+			try {
+				connectionObj.close();
+			} catch (SQLException ex) {
+				// ignore exception, since connection is expected
+				// to be in a bad state
+			} finally {
+				connectionObj = null;
+			}
+		}
+
+		connectionObj = (userName != null) ? dataSource.getConnection(userName, password) : dataSource.getConnection();
+	}
+
+	@Override
+	public void addConnectionEventListener(ConnectionEventListener listener) {
+		synchronized (connectionEventListeners) {
+			if (!connectionEventListeners.contains(listener)) {
+				connectionEventListeners.add(listener);
+			}
+		}
+	}
+
+	@Override
+	public void removeConnectionEventListener(ConnectionEventListener listener) {
+		synchronized (connectionEventListeners) {
+			connectionEventListeners.remove(listener);
+		}
+	}
+
+	@Override
+	public void close() throws SQLException {
+
+		synchronized (connectionEventListeners) {
+			// remove all listeners
+			connectionEventListeners.clear();
+		}
+
+		if (connectionObj != null) {
+			try {
+				connectionObj.close();
+			} finally {
+				connectionObj = null;
+			}
+		}
+	}
+
+	@Override
+	public Connection getConnection() throws SQLException {
+		if (connectionObj == null) {
+			reconnect();
+		}
+
+		// set autocommit to false to return connection
+		// always in consistent state
+		if (!connectionObj.getAutoCommit()) {
+
+			try {
+				connectionObj.setAutoCommit(true);
+			} catch (SQLException sqlEx) {
+				// try applying Sybase patch
+				ConnectionWrapper.sybaseAutoCommitPatch(connectionObj, sqlEx, true);
+			}
+		}
+
+		connectionObj.clearWarnings();
+		return new ConnectionWrapper(connectionObj, this);
+	}
+
+	protected void returnConnectionToThePool() throws SQLException {
+		// do not return to pool bad connections
+		if (hadErrors)
+			close();
+		else
+			// notify the listeners that connection is no longer used by
+			// application...
+			this.connectionClosedNotification();
+	}
+
+	/**
+	 * This method creates and sents an event to listeners when an error occurs
+	 * in the underlying connection. Listeners can have special logic to analyze
+	 * the error and do things like closing this PooledConnection (if the error
+	 * is fatal), etc...
+	 */
+	public void connectionErrorNotification(SQLException exception) {
+		// hint for later to avoid returning bad connections to the pool
+		hadErrors = true;
+
+		synchronized (connectionEventListeners) {
+			if (connectionEventListeners.isEmpty()) {
+				return;
+			}
+
+			ConnectionEvent closedEvent = new ConnectionEvent(this, exception);
+			for (ConnectionEventListener nextListener : connectionEventListeners) {
+				nextListener.connectionErrorOccurred(closedEvent);
+			}
+		}
+	}
+
+	/**
+	 * Creates and sends an event to listeners when a user closes
+	 * java.sql.Connection object belonging to this PooledConnection.
+	 */
+	protected void connectionClosedNotification() {
+		synchronized (connectionEventListeners) {
+			if (connectionEventListeners.size() == 0) {
+				return;
+			}
+
+			ConnectionEvent closedEvent = new ConnectionEvent(this);
+
+			for (ConnectionEventListener nextListener : connectionEventListeners) {
+				nextListener.connectionClosed(closedEvent);
+			}
+		}
+	}
+
+	/**
+	 * @since 3.0
+	 */
+	@Override
+	public void addStatementEventListener(StatementEventListener listener) {
+		throw new UnsupportedOperationException("Statement events are unsupported");
+	}
+
+	/**
+	 * @since 3.0
+	 */
+	@Override
+	public void removeStatementEventListener(StatementEventListener listener) {
+		throw new UnsupportedOperationException("Statement events are unsupported");
+	}
 }

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolingDataSource.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolingDataSource.java b/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolingDataSource.java
new file mode 100644
index 0000000..cccf905
--- /dev/null
+++ b/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolingDataSource.java
@@ -0,0 +1,471 @@
+/*****************************************************************
+ *   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.cayenne.conn;
+
+import java.io.PrintWriter;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.Statement;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.logging.Logger;
+
+import javax.sql.ConnectionEvent;
+import javax.sql.ConnectionEventListener;
+import javax.sql.ConnectionPoolDataSource;
+import javax.sql.DataSource;
+import javax.sql.PooledConnection;
+
+import org.apache.cayenne.CayenneRuntimeException;
+import org.apache.cayenne.di.ScopeEventListener;
+
+/**
+ * A {@link DataSource} with a pool of connections, that can automatically grow
+ * to the max size as more connections are requested.
+ * 
+ * @since 4.0
+ */
+public class PoolingDataSource implements ScopeEventListener, DataSource, ConnectionEventListener {
+
+	/**
+	 * Defines a maximum time in milliseconds that a connection request could
+	 * wait in the connection queue. After this period expires, an exception
+	 * will be thrown in the calling method.
+	 */
+	public static final int MAX_QUEUE_WAIT_DEFAULT = 20000;
+
+	/**
+	 * An exception indicating that a connection request waiting in the queue
+	 * timed out and was unable to obtain a connection.
+	 */
+	public static class ConnectionUnavailableException extends SQLException {
+		private static final long serialVersionUID = 1063973806941023165L;
+
+		public ConnectionUnavailableException(String message) {
+			super(message);
+		}
+	}
+
+	protected ConnectionPoolDataSource pooledConnectionFactory;
+
+	private int minConnections;
+	private int maxConnections;
+	private long maxQueueWaitTime;
+	private String validationQuery;
+
+	private List<PooledConnection> unusedPool;
+	private List<PooledConnection> usedPool;
+	private boolean shuttingDown;
+
+	/**
+	 * Creates new PoolManager with the specified policy for connection pooling
+	 * and a ConnectionPoolDataSource object.
+	 * 
+	 * @param poolDataSource
+	 *            data source for pooled connections
+	 * @param minCons
+	 *            Non-negative integer that specifies a minimum number of open
+	 *            connections to keep in the pool at all times
+	 * @param maxCons
+	 *            Non-negative integer that specifies maximum number of
+	 *            simultaneously open connections
+	 * @throws SQLException
+	 *             if pool manager can not be created.
+	 * @since 4.0
+	 */
+	public PoolingDataSource(ConnectionPoolDataSource poolDataSource, PoolingDataSourceParameters parameters)
+			throws SQLException {
+
+		this.pooledConnectionFactory = poolDataSource;
+
+		// clone parameters to keep DataSource immutable
+		this.minConnections = parameters.getMinConnections();
+		this.maxConnections = parameters.getMaxConnections();
+		this.maxQueueWaitTime = parameters.getMaxQueueWaitTime();
+		this.validationQuery = parameters.getValidationQuery();
+
+		// init pool... use linked lists to use the queue in the FIFO manner
+		this.usedPool = new LinkedList<PooledConnection>();
+		this.unusedPool = new LinkedList<PooledConnection>();
+		growPool(minConnections);
+	}
+
+	/**
+	 * Creates and returns new PooledConnection object, adding itself as a
+	 * listener for connection events.
+	 */
+	protected PooledConnection newPooledConnection() throws SQLException {
+		PooledConnection connection = pooledConnectionFactory.getPooledConnection();
+		connection.addConnectionEventListener(this);
+		return connection;
+	}
+
+	/**
+	 * Shuts down the pool, closing all open connections. This is an
+	 * implementation of {@link ScopeEventListener}.
+	 * 
+	 * @since 3.1
+	 */
+	@Override
+	public synchronized void beforeScopeEnd() {
+		try {
+
+			// using boolean variable instead of locking PoolManager instance
+			// due to possible deadlock during shutdown when one of connections
+			// locks its event listeners list trying to invoke locked
+			// PoolManager's listener methods
+			shuttingDown = true;
+
+			ListIterator<PooledConnection> unusedIterator = unusedPool.listIterator();
+			while (unusedIterator.hasNext()) {
+				PooledConnection con = unusedIterator.next();
+				// close connection
+				con.close();
+				// remove connection from the list
+				unusedIterator.remove();
+			}
+
+			// clean used connections
+			ListIterator<PooledConnection> usedIterator = usedPool.listIterator();
+			while (usedIterator.hasNext()) {
+				PooledConnection con = usedIterator.next();
+				// stop listening for connection events
+				con.removeConnectionEventListener(this);
+				// close connection
+				con.close();
+				// remove connection from the list
+				usedIterator.remove();
+			}
+		} catch (SQLException e) {
+			throw new CayenneRuntimeException("Error while shutting down");
+		}
+	}
+
+	/**
+	 * @return true if at least one more connection can be added to the pool.
+	 */
+	protected synchronized boolean canGrowPool() {
+		return getPoolSize() < maxConnections;
+	}
+
+	/**
+	 * Increases connection pool by the specified number of connections.
+	 * 
+	 * @return the actual number of created connections.
+	 * @throws SQLException
+	 *             if an error happens when creating a new connection.
+	 */
+	protected synchronized int growPool(int addConnections) throws SQLException {
+
+		int i = 0;
+		int startPoolSize = getPoolSize();
+		for (; i < addConnections && startPoolSize + i < maxConnections; i++) {
+			PooledConnection newConnection = newPooledConnection();
+			unusedPool.add(newConnection);
+		}
+
+		return i;
+	}
+
+	public synchronized void shrinkPool(int closeConnections) {
+		int idleSize = unusedPool.size();
+		for (int i = 0; i < closeConnections && i < idleSize; i++) {
+			PooledConnection con = unusedPool.remove(i);
+
+			try {
+				con.close();
+			} catch (SQLException ex) {
+				// ignore
+			}
+		}
+	}
+
+	public String getValidationQuery() {
+		return validationQuery;
+	}
+
+	/**
+	 * Returns maximum number of connections this pool can keep. This parameter
+	 * when configured allows to limit the number of simultaneously open
+	 * connections.
+	 */
+	public int getMaxConnections() {
+		return maxConnections;
+	}
+
+	/**
+	 * Returns the absolute minimum number of connections allowed in this pool
+	 * at any moment in time.
+	 */
+	public int getMinConnections() {
+		return minConnections;
+	}
+
+	/**
+	 * Returns current number of connections.
+	 */
+	public synchronized int getPoolSize() {
+		return usedPool.size() + unusedPool.size();
+	}
+
+	/**
+	 * Returns the number of connections obtained via this DataSource that are
+	 * currently in use by the DataSource clients.
+	 */
+	public synchronized int getCurrentlyInUse() {
+		return usedPool.size();
+	}
+
+	/**
+	 * Returns the number of connections maintained in the pool that are
+	 * currently not used by any clients and are available immediately via
+	 * <code>getConnection</code> method.
+	 */
+	public synchronized int getCurrentlyUnused() {
+		return unusedPool.size();
+	}
+
+	/**
+	 * Returns connection from the pool using internal values of user name and
+	 * password.
+	 */
+	@Override
+	public synchronized Connection getConnection() throws SQLException {
+		if (shuttingDown) {
+			throw new SQLException("Pool manager is shutting down.");
+		}
+
+		PooledConnection pooledConnection = uncheckPooledConnection();
+
+		try {
+			return uncheckAndValidateConnection(pooledConnection);
+		} catch (SQLException ex) {
+
+			try {
+				pooledConnection.close();
+			} catch (SQLException ignored) {
+			}
+
+			// do one reconnect attempt...
+			pooledConnection = uncheckPooledConnection();
+			try {
+				return uncheckAndValidateConnection(pooledConnection);
+			} catch (SQLException reconnectEx) {
+				try {
+					pooledConnection.close();
+				} catch (SQLException ignored) {
+				}
+
+				throw reconnectEx;
+			}
+		}
+	}
+
+	/**
+	 * Returns connection from the pool.
+	 */
+	@Override
+	public synchronized Connection getConnection(String userName, String password) throws SQLException {
+		throw new UnsupportedOperationException(
+				"Connections for a specific user are not supported by the pooled DataSource");
+	}
+
+	private Connection uncheckConnection(PooledConnection pooledConnection) throws SQLException {
+		Connection c = pooledConnection.getConnection();
+
+		// only do that on successfully unchecked connection...
+		usedPool.add(pooledConnection);
+		return c;
+	}
+
+	private Connection uncheckAndValidateConnection(PooledConnection pooledConnection) throws SQLException {
+		Connection c = uncheckConnection(pooledConnection);
+
+		if (validationQuery != null) {
+
+			Statement statement = c.createStatement();
+			try {
+				ResultSet rs = statement.executeQuery(validationQuery);
+				try {
+
+					if (!rs.next()) {
+						throw new SQLException("Connection validation failed, no result for query: " + validationQuery);
+					}
+
+				} finally {
+					rs.close();
+				}
+			} finally {
+				statement.close();
+			}
+		}
+
+		return c;
+	}
+
+	private PooledConnection uncheckPooledConnection() throws SQLException {
+		// wait for returned connections or the maintenance thread
+		// to bump the pool size...
+
+		if (unusedPool.size() == 0) {
+
+			// first try to open a new connection
+			if (canGrowPool()) {
+				return newPooledConnection();
+			}
+
+			// can't open no more... will have to wait for others to return a
+			// connection
+
+			// note that if we were woken up
+			// before the full wait period expired, and no connections are
+			// available yet, go back to sleep. Otherwise we don't give a
+			// maintenance
+			// thread a chance to increase pool size
+			long waitTill = System.currentTimeMillis() + maxQueueWaitTime;
+
+			do {
+				try {
+					wait(maxQueueWaitTime);
+				} catch (InterruptedException iex) {
+					// ignoring
+				}
+
+			} while (unusedPool.size() == 0 && (maxQueueWaitTime == 0 || waitTill > System.currentTimeMillis()));
+
+			if (unusedPool.size() == 0) {
+				throw new ConnectionUnavailableException(
+						"Can't obtain connection. Request timed out. Total used connections: " + usedPool.size());
+			}
+		}
+
+		// get first connection... lets cycle them in FIFO manner
+		return unusedPool.remove(0);
+	}
+
+	@Override
+	public int getLoginTimeout() throws java.sql.SQLException {
+		return pooledConnectionFactory.getLoginTimeout();
+	}
+
+	@Override
+	public void setLoginTimeout(int seconds) throws java.sql.SQLException {
+		pooledConnectionFactory.setLoginTimeout(seconds);
+	}
+
+	@Override
+	public PrintWriter getLogWriter() throws java.sql.SQLException {
+		return pooledConnectionFactory.getLogWriter();
+	}
+
+	@Override
+	public void setLogWriter(PrintWriter out) throws java.sql.SQLException {
+		pooledConnectionFactory.setLogWriter(out);
+	}
+
+	/**
+	 * Returns closed connection to the pool.
+	 */
+	@Override
+	public synchronized void connectionClosed(ConnectionEvent event) {
+
+		if (shuttingDown) {
+			return;
+		}
+
+		// return connection to the pool
+		PooledConnection closedConn = (PooledConnection) event.getSource();
+
+		// remove this connection from the list of connections
+		// managed by this pool...
+		int usedInd = usedPool.indexOf(closedConn);
+		if (usedInd >= 0) {
+			usedPool.remove(usedInd);
+			unusedPool.add(closedConn);
+
+			// notify threads waiting for connections
+			notifyAll();
+		}
+		// else ....
+		// other possibility is that this is a bad connection, so just ignore
+		// its closing
+		// event,
+		// since it was unregistered in "connectionErrorOccurred"
+	}
+
+	/**
+	 * Removes connection with an error from the pool. This method is called by
+	 * PoolManager connections on connection errors to notify PoolManager that
+	 * connection is in invalid state.
+	 */
+	@Override
+	public synchronized void connectionErrorOccurred(ConnectionEvent event) {
+
+		if (shuttingDown) {
+			return;
+		}
+
+		// later on we should analyze the error to see if this
+		// is fatal... right now just kill this PooledConnection
+
+		PooledConnection errorSrc = (PooledConnection) event.getSource();
+
+		// remove this connection from the list of connections
+		// managed by this pool...
+
+		int usedInd = usedPool.indexOf(errorSrc);
+		if (usedInd >= 0) {
+			usedPool.remove(usedInd);
+		} else {
+			int unusedInd = unusedPool.indexOf(errorSrc);
+			if (unusedInd >= 0) {
+				unusedPool.remove(unusedInd);
+			}
+		}
+
+		// do not close connection,
+		// let the code that catches the exception handle it
+		// ....
+	}
+
+	@Override
+	public boolean isWrapperFor(Class<?> iface) throws SQLException {
+		return PoolingDataSource.class.equals(iface);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public <T> T unwrap(Class<T> iface) throws SQLException {
+		if (PoolingDataSource.class.equals(iface)) {
+			return (T) this;
+		}
+
+		throw new SQLException("Not a wrapper for " + iface);
+	}
+
+	// JDBC 4.1 compatibility under Java <= 1.6
+	public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+		throw new UnsupportedOperationException();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/cayenne/blob/8b54c052/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolingDataSourceParameters.java
----------------------------------------------------------------------
diff --git a/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolingDataSourceParameters.java b/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolingDataSourceParameters.java
new file mode 100644
index 0000000..a03fd39
--- /dev/null
+++ b/cayenne-server/src/main/java/org/apache/cayenne/conn/PoolingDataSourceParameters.java
@@ -0,0 +1,64 @@
+/*****************************************************************
+ *   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.cayenne.conn;
+
+/**
+ * A connection of pooling parameters used by {@link PoolingDataSource}.
+ * 
+ * @since 4.0
+ */
+public class PoolingDataSourceParameters {
+
+	private String validationQuery;
+	private int minConnections;
+	private int maxConnections;
+	private long maxQueueWaitTime;
+
+	public int getMinConnections() {
+		return minConnections;
+	}
+
+	public void setMinConnections(int minConnections) {
+		this.minConnections = minConnections;
+	}
+
+	public int getMaxConnections() {
+		return maxConnections;
+	}
+
+	public void setMaxConnections(int maxConnections) {
+		this.maxConnections = maxConnections;
+	}
+
+	public long getMaxQueueWaitTime() {
+		return maxQueueWaitTime;
+	}
+
+	public void setMaxQueueWaitTime(long maxQueueWaitTime) {
+		this.maxQueueWaitTime = maxQueueWaitTime;
+	}
+
+	public String getValidationQuery() {
+		return validationQuery;
+	}
+
+	public void setValidationQuery(String validationQuery) {
+		this.validationQuery = validationQuery;
+	}
+}