You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aries.apache.org by ti...@apache.org on 2017/02/23 15:47:42 UTC

svn commit: r1784160 - in /aries/trunk/tx-control: tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/ tx-control-provider-jdbc-common/src/main/java/org/apache/aries/tx/control/jdbc/common/impl/ tx-control-provider-jdbc-local/src/main/j...

Author: timothyjward
Date: Thu Feb 23 15:47:41 2017
New Revision: 1784160

URL: http://svn.apache.org/viewvc?rev=1784160&view=rev
Log:
[tx-control] Support Connection Test Queries when pooling

Fixes ARIES-1692, includes patches from mit_jones with minor tidy-up applied

Added:
    aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/ConnectionRefreshTest.java
    aries/trunk/tx-control/tx-control-provider-jdbc-common/src/main/java/org/apache/aries/tx/control/jdbc/common/impl/AbstractInternalJDBCConnectionProviderFactory.java
    aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/Config.java
      - copied, changed from r1783901, aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java
    aries/trunk/tx-control/tx-control-provider-jpa-xa/src/main/java/org/apache/aries/tx/control/jpa/xa/impl/Config.java
      - copied, changed from r1783901, aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java
Modified:
    aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/AbstractTransactionTest.java
    aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java
    aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/JDBCConnectionProviderFactoryImpl.java
    aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/Config.java
    aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/JDBCConnectionProviderFactoryImpl.java
    aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/ManagedServiceFactoryImpl.java
    aries/trunk/tx-control/tx-control-provider-jpa-common/src/main/java/org/apache/aries/tx/control/jpa/common/impl/AbstractManagedJPADataSourceSetup.java
    aries/trunk/tx-control/tx-control-provider-jpa-local/pom.xml
    aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/LocalJPAManagedServiceFactory.java
    aries/trunk/tx-control/tx-control-provider-jpa-xa/pom.xml

Modified: aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/AbstractTransactionTest.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/AbstractTransactionTest.java?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/AbstractTransactionTest.java (original)
+++ aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/AbstractTransactionTest.java Thu Feb 23 15:47:41 2017
@@ -39,6 +39,7 @@ import java.util.Properties;
 
 import javax.inject.Inject;
 
+import org.h2.Driver;
 import org.h2.tools.Server;
 import org.junit.After;
 import org.junit.Before;
@@ -80,7 +81,7 @@ public abstract class AbstractTransactio
 	
 	protected Connection connection;
 
-	private Server server;
+	protected Server server;
 	
 	protected final List<ServiceTracker<?,?>> trackers = new ArrayList<>();
 
@@ -104,20 +105,25 @@ public abstract class AbstractTransactio
 			jdbcUrl = "jdbc:h2:mem:test;DB_CLOSE_DELAY=-1";
 		}
 		
+		initTestTable(jdbcUrl);
+		
 		jdbc.setProperty(DataSourceFactory.JDBC_URL, jdbcUrl);
 		
 		boolean configuredProvider = isConfigured();
 		
 		connection = configuredProvider ? configuredConnection(jdbc) : programaticConnection(jdbc);
-		
-		txControl.required(() -> {
-				Statement s = connection.createStatement();
-				try {
-					s.execute("DROP TABLE TEST_TABLE");
-				} catch (SQLException sqle) {}
-				s.execute("CREATE TABLE TEST_TABLE ( message varchar(255) )");
-				return null;
-			});
+	}
+
+	protected void initTestTable(String jdbcUrl) throws SQLException {
+		Driver d = new Driver();
+		try (Connection c = d.connect(jdbcUrl, null)) {
+			Statement s = c.createStatement();
+			try {
+				s.execute("DROP TABLE TEST_TABLE");
+			} catch (SQLException sqle) {}
+			s.execute("CREATE TABLE TEST_TABLE ( message varchar(255) )");
+			c.commit();
+		}
 	}
 
 	protected Map<String, Object> resourceProviderConfig() {

Added: aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/ConnectionRefreshTest.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/ConnectionRefreshTest.java?rev=1784160&view=auto
==============================================================================
--- aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/ConnectionRefreshTest.java (added)
+++ aries/trunk/tx-control/tx-control-itests/src/test/java/org/apache/aries/tx/control/itests/ConnectionRefreshTest.java Thu Feb 23 15:47:41 2017
@@ -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 WARRANTIESOR 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.aries.tx.control.itests;
+
+
+import static java.util.stream.Collectors.toList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.ResultSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.ops4j.pax.exam.junit.PaxExam;
+import org.ops4j.pax.exam.spi.reactors.ExamReactorStrategy;
+import org.ops4j.pax.exam.spi.reactors.PerClass;
+import org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory;
+
+
+@RunWith(PaxExam.class)
+@ExamReactorStrategy(PerClass.class)
+public class ConnectionRefreshTest extends AbstractTransactionTest {
+
+	private static final String COUNT_CONNECTION_CHECKS = "Select EXECUTION_COUNT from INFORMATION_SCHEMA.QUERY_STATISTICS WHERE SQL_STATEMENT='Select COUNT(*) from TEST_TABLE'";
+
+	private static final String COUNT_SESSIONS = "Select COUNT(*) from INFORMATION_SCHEMA.SESSIONS";
+
+	private static final int CONNECTIONS = 5;	
+	
+	@Override
+	protected Map<String, Object> resourceProviderConfig() {
+		Map<String, Object> config = new HashMap<>();
+		
+		// Add a test query for the aries.connection.test.query property so that the test query will be
+		// used by the Hikari pooling library to test if a connection is okay as opposed to using the default 
+		// Connection.isValid() method.
+		config.put("aries.connection.test.query", "Select COUNT(*) from TEST_TABLE");
+		
+		config.put(JDBCConnectionProviderFactory.MAX_CONNECTIONS, CONNECTIONS);
+		config.put(JDBCConnectionProviderFactory.MIN_CONNECTIONS, CONNECTIONS);		
+		return config;
+	}	
+	
+	/**
+	 * Test that setting the aries.connection.test.query property will indeed be used by the Hikari pooling 
+	 * library to test if a connection is okay as opposed to using the default Connection.isValid() method.
+	 * 
+	 * @throws Exception
+	 */
+	@Test
+	public void testConnectionTestQuery() throws Exception {
+
+		System.out.println("-------------- start testConnectionTestQuery -------------------------");			
+
+		// Enable statistics gathering so that all connection checks will be logged
+		
+		txControl.required(() -> 
+			connection.createStatement().executeUpdate("SET QUERY_STATISTICS TRUE"));
+		
+		// Wait a second so that the pool will re-check connection liveness the next time 
+		// that we use one
+		
+		Thread.sleep(1000);
+
+		// Spin up some threads to execute some queries to ensure all connections in the pool
+		// are checked for liveness
+		List<Thread> threads = Stream.generate(() -> new Thread(new ExecuteQuery()))
+					.limit(CONNECTIONS * 2)
+					.collect(toList());
+		
+		threads.stream().forEach(Thread::start);
+		
+		System.out.println("testConnectionTestQuery() - Waiting while queries are run");
+		threads.stream().forEach(t -> { 
+				try {
+					t.join(5000);
+				} catch (InterruptedException ie) {}
+				assertFalse("The query did not complete in time", t.isAlive());
+			});
+		
+		// Run a query to check that there are the expected number of connections to the Db server 		
+		txControl.notSupported(() -> {
+
+			// First check we saturated the pool
+			System.out.println("testConnectionTestQuery() - Execute query to get number of active connections");				
+			ResultSet rs = connection.createStatement().executeQuery(COUNT_SESSIONS);
+
+			assertTrue(rs.next()); 
+
+			int numberOfConnections = rs.getInt(1);
+			assertNotNull(numberOfConnections);				
+			assertEquals("Number of connections " + numberOfConnections, CONNECTIONS, numberOfConnections);
+				
+			// Now check that each connection was tested for liveness
+			rs = connection.createStatement().executeQuery(COUNT_CONNECTION_CHECKS);
+			
+			assertTrue(rs.next()); 
+
+			assertEquals("There should be a check for every connection", CONNECTIONS, rs.getInt(1));
+			
+			return null;
+
+		});			
+		
+
+		System.out.println("-------------- end testConnectionTestQuery -------------------------");			
+
+	}	
+
+	public class ExecuteQuery implements Runnable {
+		
+		public void run() {		
+			
+			txControl.required(() -> {
+				
+				System.out.println("   ExecuteQuery - query to get number of active connections after Db server has restarted");				
+				ResultSet rs = connection.createStatement().executeQuery(COUNT_SESSIONS);
+	
+				if(rs.next()) {
+	
+					Integer numberOfConnections = rs.getInt(1);
+					System.out.println("   ExecuteQuery - numberOfConnections after starting Db server =<" + numberOfConnections + ">");
+	
+				}
+				// A short sleep to ensure contention for connections
+				Thread.sleep(100);
+	
+				return null;
+	
+			});					
+		}		
+	}
+}
\ No newline at end of file

Added: aries/trunk/tx-control/tx-control-provider-jdbc-common/src/main/java/org/apache/aries/tx/control/jdbc/common/impl/AbstractInternalJDBCConnectionProviderFactory.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jdbc-common/src/main/java/org/apache/aries/tx/control/jdbc/common/impl/AbstractInternalJDBCConnectionProviderFactory.java?rev=1784160&view=auto
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jdbc-common/src/main/java/org/apache/aries/tx/control/jdbc/common/impl/AbstractInternalJDBCConnectionProviderFactory.java (added)
+++ aries/trunk/tx-control/tx-control-provider-jdbc-common/src/main/java/org/apache/aries/tx/control/jdbc/common/impl/AbstractInternalJDBCConnectionProviderFactory.java Thu Feb 23 15:47:41 2017
@@ -0,0 +1,124 @@
+/*
+ * 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 WARRANTIESOR 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.aries.tx.control.jdbc.common.impl;
+
+import static java.util.Optional.ofNullable;
+import static java.util.concurrent.TimeUnit.HOURS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.CONNECTION_LIFETIME;
+import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.CONNECTION_POOLING_ENABLED;
+import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.CONNECTION_TIMEOUT;
+import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.IDLE_TIMEOUT;
+import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.MAX_CONNECTIONS;
+import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.MIN_CONNECTIONS;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import javax.sql.DataSource;
+
+import com.zaxxer.hikari.HikariConfig;
+import com.zaxxer.hikari.HikariDataSource;
+
+public abstract class AbstractInternalJDBCConnectionProviderFactory implements InternalJDBCConnectionProviderFactory {
+
+	public static final String CONNECTION_TEST_QUERY = "aries.connection.test.query";	
+
+	protected DataSource poolIfNecessary(Map<String, Object> resourceProviderProperties, DataSource unpooled) {
+		DataSource toUse;
+
+		if (toBoolean(resourceProviderProperties, CONNECTION_POOLING_ENABLED, true)) {
+			HikariConfig hcfg = new HikariConfig();
+			hcfg.setDataSource(unpooled);
+
+			// Sizes
+			hcfg.setMaximumPoolSize(toInt(resourceProviderProperties, MAX_CONNECTIONS, 10));
+			hcfg.setMinimumIdle(toInt(resourceProviderProperties, MIN_CONNECTIONS, 10));
+
+			// Timeouts
+			hcfg.setConnectionTimeout(toLong(resourceProviderProperties, CONNECTION_TIMEOUT, SECONDS.toMillis(30)));
+			hcfg.setIdleTimeout(toLong(resourceProviderProperties, IDLE_TIMEOUT, TimeUnit.MINUTES.toMillis(3)));
+			hcfg.setMaxLifetime(toLong(resourceProviderProperties, CONNECTION_LIFETIME, HOURS.toMillis(3)));
+
+			hcfg.setConnectionTestQuery(toString(resourceProviderProperties, CONNECTION_TEST_QUERY, null));					
+			
+			toUse = new HikariDataSource(hcfg);
+
+		} else {
+			toUse = unpooled;
+		}
+		return toUse;
+	}
+
+	public static boolean toBoolean(Map<String, Object> props, String key, boolean defaultValue) {
+		Object o =  ofNullable(props)
+			.map(m -> m.get(key))
+			.orElse(defaultValue);
+		
+		if (o instanceof Boolean) {
+			return ((Boolean) o).booleanValue();
+		} else if(o instanceof String) {
+			return Boolean.parseBoolean((String) o);
+		} else {
+			throw new IllegalArgumentException("The property " + key + " cannot be converted to a boolean");
+		}
+	}
+
+	public static int toInt(Map<String, Object> props, String key, int defaultValue) {
+		
+		Object o =  ofNullable(props)
+				.map(m -> m.get(key))
+				.orElse(defaultValue);
+		
+		if (o instanceof Number) {
+			return ((Number) o).intValue();
+		} else if(o instanceof String) {
+			return Integer.parseInt((String) o);
+		} else {
+			throw new IllegalArgumentException("The property " + key + " cannot be converted to an int");
+		}
+	}
+
+	public static long toLong(Map<String, Object> props, String key, long defaultValue) {
+		
+		Object o =  ofNullable(props)
+				.map(m -> m.get(key))
+				.orElse(defaultValue);
+		
+		if (o instanceof Number) {
+			return ((Number) o).longValue();
+		} else if(o instanceof String) {
+			return Long.parseLong((String) o);
+		} else {
+			throw new IllegalArgumentException("The property " + key + " cannot be converted to a long");
+		}
+	}
+
+	public static String toString(Map<String, Object> props, String key, String defaultValue) {
+		
+		Object o =  ofNullable(props)
+				.map(m -> m.get(key))
+				.orElse(defaultValue);
+		if(o == null) {
+			return null;
+		} else {
+			return String.valueOf(o);
+		}
+	}
+}

Modified: aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java (original)
+++ aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java Thu Feb 23 15:47:41 2017
@@ -72,6 +72,9 @@ public @interface Config {
 	@AttributeDefinition(required=false, description="The maximum time (in ms) that a connection will stay in the pool before being discarded")
 	long osgi_connection_lifetime() default 10800000;
 	
+	@AttributeDefinition(required=false, description="The query that will be executed just before a connection is given to you from the pool to validate that the connection to the database is still alive. If your driver supports JDBC4 we strongly recommend not setting this property. This is for 'legacy' databases that do not support the JDBC Connection.isValid() API")
+	String aries_connection_test_query();		
+	
 	// Detailed Configuration
 	
 	@AttributeDefinition(required=false, description="The filter to use when finding the DataSourceFactory service. This property need not be defined if osgi.jdbc.driver.class is defined.")

Modified: aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/JDBCConnectionProviderFactoryImpl.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/JDBCConnectionProviderFactoryImpl.java?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/JDBCConnectionProviderFactoryImpl.java (original)
+++ aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/JDBCConnectionProviderFactoryImpl.java Thu Feb 23 15:47:41 2017
@@ -18,17 +18,8 @@
  */
 package org.apache.aries.tx.control.jdbc.local.impl;
 
-import static java.util.Optional.ofNullable;
-import static java.util.concurrent.TimeUnit.HOURS;
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.osgi.service.jdbc.DataSourceFactory.JDBC_URL;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.CONNECTION_LIFETIME;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.CONNECTION_POOLING_ENABLED;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.CONNECTION_TIMEOUT;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.IDLE_TIMEOUT;
 import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.LOCAL_ENLISTMENT_ENABLED;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.MAX_CONNECTIONS;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.MIN_CONNECTIONS;
 import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.USE_DRIVER;
 import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.XA_ENLISTMENT_ENABLED;
 
@@ -36,21 +27,17 @@ import java.sql.Driver;
 import java.sql.SQLException;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
 import javax.sql.DataSource;
 import javax.sql.XADataSource;
 
+import org.apache.aries.tx.control.jdbc.common.impl.AbstractInternalJDBCConnectionProviderFactory;
 import org.apache.aries.tx.control.jdbc.common.impl.AbstractJDBCConnectionProvider;
 import org.apache.aries.tx.control.jdbc.common.impl.DriverDataSource;
-import org.apache.aries.tx.control.jdbc.common.impl.InternalJDBCConnectionProviderFactory;
 import org.osgi.service.jdbc.DataSourceFactory;
 import org.osgi.service.transaction.control.TransactionException;
 
-import com.zaxxer.hikari.HikariConfig;
-import com.zaxxer.hikari.HikariDataSource;
-
-public class JDBCConnectionProviderFactoryImpl implements InternalJDBCConnectionProviderFactory {
+public class JDBCConnectionProviderFactoryImpl extends AbstractInternalJDBCConnectionProviderFactory {
 
 	@Override
 	public AbstractJDBCConnectionProvider getProviderFor(DataSourceFactory dsf, Properties jdbcProperties,
@@ -116,73 +103,4 @@ public class JDBCConnectionProviderFacto
 					"This Resource Provider always enlists in local transactions as it does not support XA");
 		}
 	}
-
-	private DataSource poolIfNecessary(Map<String, Object> resourceProviderProperties, DataSource unpooled) {
-		DataSource toUse;
-
-		if (toBoolean(resourceProviderProperties, CONNECTION_POOLING_ENABLED, true)) {
-			HikariConfig hcfg = new HikariConfig();
-			hcfg.setDataSource(unpooled);
-
-			// Sizes
-			hcfg.setMaximumPoolSize(toInt(resourceProviderProperties, MAX_CONNECTIONS, 10));
-			hcfg.setMinimumIdle(toInt(resourceProviderProperties, MIN_CONNECTIONS, 10));
-
-			// Timeouts
-			hcfg.setConnectionTimeout(toLong(resourceProviderProperties, CONNECTION_TIMEOUT, SECONDS.toMillis(30)));
-			hcfg.setIdleTimeout(toLong(resourceProviderProperties, IDLE_TIMEOUT, TimeUnit.MINUTES.toMillis(3)));
-			hcfg.setMaxLifetime(toLong(resourceProviderProperties, CONNECTION_LIFETIME, HOURS.toMillis(3)));
-
-			toUse = new HikariDataSource(hcfg);
-
-		} else {
-			toUse = unpooled;
-		}
-		return toUse;
-	}
-
-	private boolean toBoolean(Map<String, Object> props, String key, boolean defaultValue) {
-		Object o =  ofNullable(props)
-			.map(m -> m.get(key))
-			.orElse(defaultValue);
-		
-		if (o instanceof Boolean) {
-			return ((Boolean) o).booleanValue();
-		} else if(o instanceof String) {
-			return Boolean.parseBoolean((String) o);
-		} else {
-			throw new IllegalArgumentException("The property " + key + " cannot be converted to a boolean");
-		}
-	}
-
-	private int toInt(Map<String, Object> props, String key, int defaultValue) {
-		
-		Object o =  ofNullable(props)
-				.map(m -> m.get(key))
-				.orElse(defaultValue);
-		
-		if (o instanceof Number) {
-			return ((Number) o).intValue();
-		} else if(o instanceof String) {
-			return Integer.parseInt((String) o);
-		} else {
-			throw new IllegalArgumentException("The property " + key + " cannot be converted to an int");
-		}
-	}
-
-	private long toLong(Map<String, Object> props, String key, long defaultValue) {
-		
-		Object o =  ofNullable(props)
-				.map(m -> m.get(key))
-				.orElse(defaultValue);
-		
-		if (o instanceof Number) {
-			return ((Number) o).longValue();
-		} else if(o instanceof String) {
-			return Long.parseLong((String) o);
-		} else {
-			throw new IllegalArgumentException("The property " + key + " cannot be converted to a long");
-		}
-	}
-
 }

Modified: aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/Config.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/Config.java?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/Config.java (original)
+++ aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/Config.java Thu Feb 23 15:47:41 2017
@@ -76,6 +76,9 @@ public @interface Config {
 	@AttributeDefinition(required=false, description="The maximum time (in ms) that a connection will stay in the pool before being discarded")
 	long osgi_connection_lifetime() default 10800000;
 	
+	@AttributeDefinition(required=false, description="The query that will be executed just before a connection is given to you from the pool to validate that the connection to the database is still alive. If your driver supports JDBC4 we strongly recommend not setting this property. This is for 'legacy' databases that do not support the JDBC Connection.isValid() API")
+	String aries_connection_test_query();	
+		
 	// Recovery credential configuration
 	
 	@AttributeDefinition(required=false, description="The user that should be used for recovery. If not specified then recovery will use the same user credentials as normal operation")

Modified: aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/JDBCConnectionProviderFactoryImpl.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/JDBCConnectionProviderFactoryImpl.java?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/JDBCConnectionProviderFactoryImpl.java (original)
+++ aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/JDBCConnectionProviderFactoryImpl.java Thu Feb 23 15:47:41 2017
@@ -19,16 +19,8 @@
 package org.apache.aries.tx.control.jdbc.xa.impl;
 
 import static java.util.Optional.ofNullable;
-import static java.util.concurrent.TimeUnit.HOURS;
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.osgi.service.jdbc.DataSourceFactory.JDBC_URL;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.CONNECTION_LIFETIME;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.CONNECTION_POOLING_ENABLED;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.CONNECTION_TIMEOUT;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.IDLE_TIMEOUT;
 import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.LOCAL_ENLISTMENT_ENABLED;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.MAX_CONNECTIONS;
-import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.MIN_CONNECTIONS;
 import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.OSGI_RECOVERY_IDENTIFIER;
 import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.USE_DRIVER;
 import static org.osgi.service.transaction.control.jdbc.JDBCConnectionProviderFactory.XA_ENLISTMENT_ENABLED;
@@ -37,24 +29,20 @@ import java.sql.Driver;
 import java.sql.SQLException;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
 import javax.sql.DataSource;
 import javax.sql.XADataSource;
 
+import org.apache.aries.tx.control.jdbc.common.impl.AbstractInternalJDBCConnectionProviderFactory;
 import org.apache.aries.tx.control.jdbc.common.impl.AbstractJDBCConnectionProvider;
 import org.apache.aries.tx.control.jdbc.common.impl.DriverDataSource;
-import org.apache.aries.tx.control.jdbc.common.impl.InternalJDBCConnectionProviderFactory;
 import org.apache.aries.tx.control.jdbc.xa.connection.impl.XADataSourceMapper;
 import org.osgi.service.jdbc.DataSourceFactory;
 import org.osgi.service.transaction.control.TransactionException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.zaxxer.hikari.HikariConfig;
-import com.zaxxer.hikari.HikariDataSource;
-
-public class JDBCConnectionProviderFactoryImpl implements InternalJDBCConnectionProviderFactory {
+public class JDBCConnectionProviderFactoryImpl extends AbstractInternalJDBCConnectionProviderFactory {
 
 	private static final Logger LOG = LoggerFactory.getLogger(ManagedServiceFactoryImpl.class);
 	
@@ -157,73 +145,4 @@ public class JDBCConnectionProviderFacto
 			throw new TransactionException("The configuration is XA enabled but the resource is not suitable for XA enlistment");
 		}
 	}
-
-	private DataSource poolIfNecessary(Map<String, Object> resourceProviderProperties, DataSource unpooled) {
-		DataSource toUse;
-
-		if (toBoolean(resourceProviderProperties, CONNECTION_POOLING_ENABLED, true)) {
-			HikariConfig hcfg = new HikariConfig();
-			hcfg.setDataSource(unpooled);
-
-			// Sizes
-			hcfg.setMaximumPoolSize(toInt(resourceProviderProperties, MAX_CONNECTIONS, 10));
-			hcfg.setMinimumIdle(toInt(resourceProviderProperties, MIN_CONNECTIONS, 10));
-
-			// Timeouts
-			hcfg.setConnectionTimeout(toLong(resourceProviderProperties, CONNECTION_TIMEOUT, SECONDS.toMillis(30)));
-			hcfg.setIdleTimeout(toLong(resourceProviderProperties, IDLE_TIMEOUT, TimeUnit.MINUTES.toMillis(3)));
-			hcfg.setMaxLifetime(toLong(resourceProviderProperties, CONNECTION_LIFETIME, HOURS.toMillis(3)));
-
-			toUse = new HikariDataSource(hcfg);
-
-		} else {
-			toUse = unpooled;
-		}
-		return toUse;
-	}
-
-	static boolean toBoolean(Map<String, Object> props, String key, boolean defaultValue) {
-		Object o =  ofNullable(props)
-			.map(m -> m.get(key))
-			.orElse(defaultValue);
-		
-		if (o instanceof Boolean) {
-			return ((Boolean) o).booleanValue();
-		} else if(o instanceof String) {
-			return Boolean.parseBoolean((String) o);
-		} else {
-			throw new IllegalArgumentException("The property " + key + " cannot be converted to a boolean");
-		}
-	}
-
-	private int toInt(Map<String, Object> props, String key, int defaultValue) {
-		
-		Object o =  ofNullable(props)
-				.map(m -> m.get(key))
-				.orElse(defaultValue);
-		
-		if (o instanceof Number) {
-			return ((Number) o).intValue();
-		} else if(o instanceof String) {
-			return Integer.parseInt((String) o);
-		} else {
-			throw new IllegalArgumentException("The property " + key + " cannot be converted to an int");
-		}
-	}
-
-	private long toLong(Map<String, Object> props, String key, long defaultValue) {
-		
-		Object o =  ofNullable(props)
-				.map(m -> m.get(key))
-				.orElse(defaultValue);
-		
-		if (o instanceof Number) {
-			return ((Number) o).longValue();
-		} else if(o instanceof String) {
-			return Long.parseLong((String) o);
-		} else {
-			throw new IllegalArgumentException("The property " + key + " cannot be converted to a long");
-		}
-	}
-
 }

Modified: aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/ManagedServiceFactoryImpl.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/ManagedServiceFactoryImpl.java?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/ManagedServiceFactoryImpl.java (original)
+++ aries/trunk/tx-control/tx-control-provider-jdbc-xa/src/main/java/org/apache/aries/tx/control/jdbc/xa/impl/ManagedServiceFactoryImpl.java Thu Feb 23 15:47:41 2017
@@ -19,7 +19,7 @@
 package org.apache.aries.tx.control.jdbc.xa.impl;
 
 import static java.util.Arrays.asList;
-import static org.apache.aries.tx.control.jdbc.xa.impl.JDBCConnectionProviderFactoryImpl.toBoolean;
+import static org.apache.aries.tx.control.jdbc.common.impl.AbstractInternalJDBCConnectionProviderFactory.toBoolean;
 import static org.osgi.framework.Constants.OBJECTCLASS;
 import static org.osgi.service.jdbc.DataSourceFactory.JDBC_DATABASE_NAME;
 import static org.osgi.service.jdbc.DataSourceFactory.JDBC_DATASOURCE_NAME;

Modified: aries/trunk/tx-control/tx-control-provider-jpa-common/src/main/java/org/apache/aries/tx/control/jpa/common/impl/AbstractManagedJPADataSourceSetup.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jpa-common/src/main/java/org/apache/aries/tx/control/jpa/common/impl/AbstractManagedJPADataSourceSetup.java?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jpa-common/src/main/java/org/apache/aries/tx/control/jpa/common/impl/AbstractManagedJPADataSourceSetup.java (original)
+++ aries/trunk/tx-control/tx-control-provider-jpa-common/src/main/java/org/apache/aries/tx/control/jpa/common/impl/AbstractManagedJPADataSourceSetup.java Thu Feb 23 15:47:41 2017
@@ -59,6 +59,9 @@ public abstract class AbstractManagedJPA
 
 	private static final Logger LOG = LoggerFactory.getLogger(AbstractManagedJPADataSourceSetup.class);
 	
+	// TODO - where should this go?
+	private static final String CONNECTION_TEST_QUERY = "aries.connection.test.query";
+	
 	private final BundleContext context;
 	private final String pid;
 	private final Properties jdbcProperties;
@@ -176,7 +179,9 @@ public abstract class AbstractManagedJPA
 			hcfg.setConnectionTimeout(toLong(resourceProviderProperties, CONNECTION_TIMEOUT, SECONDS.toMillis(30)));
 			hcfg.setIdleTimeout(toLong(resourceProviderProperties, IDLE_TIMEOUT, TimeUnit.MINUTES.toMillis(3)));
 			hcfg.setMaxLifetime(toLong(resourceProviderProperties, CONNECTION_LIFETIME, HOURS.toMillis(3)));
-
+	
+			hcfg.setConnectionTestQuery((String)resourceProviderProperties.get(CONNECTION_TEST_QUERY));
+			
 			toUse = new HikariDataSource(hcfg);
 
 		} else {

Modified: aries/trunk/tx-control/tx-control-provider-jpa-local/pom.xml
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jpa-local/pom.xml?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jpa-local/pom.xml (original)
+++ aries/trunk/tx-control/tx-control-provider-jpa-local/pom.xml Thu Feb 23 15:47:41 2017
@@ -88,6 +88,10 @@
             <groupId>org.osgi</groupId>
             <artifactId>org.osgi.service.jpa</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.osgi</groupId>
+            <artifactId>org.osgi.service.metatype.annotations</artifactId>
+        </dependency>
         
         <!-- JPA API dependency -->
         <dependency>

Copied: aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/Config.java (from r1783901, aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java)
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/Config.java?p2=aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/Config.java&p1=aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java&r1=1783901&r2=1784160&rev=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java (original)
+++ aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/Config.java Thu Feb 23 15:47:41 2017
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.aries.tx.control.jdbc.local.impl;
+package org.apache.aries.tx.control.jpa.local.impl;
 
 import static org.osgi.service.jdbc.DataSourceFactory.JDBC_DATABASE_NAME;
 import static org.osgi.service.jdbc.DataSourceFactory.JDBC_DATASOURCE_NAME;
@@ -33,12 +33,16 @@ import org.osgi.service.metatype.annotat
 import org.osgi.service.metatype.annotations.AttributeType;
 import org.osgi.service.metatype.annotations.ObjectClassDefinition;
 
-@ObjectClassDefinition(factoryPid="org.apache.aries.tx.control.jdbc.local", description="Aries Transaction Control Factory for Local JDBCResourceProvider Services")
+@ObjectClassDefinition(factoryPid="org.apache.aries.tx.control.jpa.local", description="Aries Transaction Control Factory for Local JPAEntityManagerProvider Services")
 public @interface Config {
 
 	// Most commonly used properties declared first so that they go into the metatype first
 	
 	@AttributeDefinition(required=false, 
+			description="The name of the persistence unit that this configuration should create a resource for.")
+	String osgi_unit_name();
+
+	@AttributeDefinition(required=false, 
 			description="The name of the driver class for the DataSourceFactory service. This property need not be defined if aries.dsf.target.filter is defined.")
 	String osgi_jdbc_driver_class();
 
@@ -51,7 +55,7 @@ public @interface Config {
 	@AttributeDefinition(type=AttributeType.PASSWORD, required=false, 
 			description="The password to pass to the DataSourceFactory (not visible as a service property)")
 	String password();
-	
+
 	// Pool configuration properties
 	
 	@AttributeDefinition(required=false, description="Is connection pooling enabled for this JDBCResourceProvider")
@@ -72,8 +76,14 @@ public @interface Config {
 	@AttributeDefinition(required=false, description="The maximum time (in ms) that a connection will stay in the pool before being discarded")
 	long osgi_connection_lifetime() default 10800000;
 	
+	@AttributeDefinition(required=false, description="The query that will be executed just before a connection is given to you from the pool to validate that the connection to the database is still alive. If your driver supports JDBC4 we strongly recommend not setting this property. This is for 'legacy' databases that do not support the JDBC Connection.isValid() API")
+	String aries_connection_test_query();	
+	
 	// Detailed Configuration
 	
+	@AttributeDefinition(required=false, description="The filter to use when searching for an EntityManagerFactoryBuilder. This property need not be defined if osgi.unit.name is defined.")
+	String aries_emf_builder_target_filter();
+	
 	@AttributeDefinition(required=false, description="The filter to use when finding the DataSourceFactory service. This property need not be defined if osgi.jdbc.driver.class is defined.")
 	String aries_dsf_target_filter();
 	

Modified: aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/LocalJPAManagedServiceFactory.java
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/LocalJPAManagedServiceFactory.java?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/LocalJPAManagedServiceFactory.java (original)
+++ aries/trunk/tx-control/tx-control-provider-jpa-local/src/main/java/org/apache/aries/tx/control/jpa/local/impl/LocalJPAManagedServiceFactory.java Thu Feb 23 15:47:41 2017
@@ -26,7 +26,6 @@ import javax.persistence.spi.Persistence
 import org.apache.aries.tx.control.jpa.common.impl.AbstractJPAManagedServiceFactory;
 import org.apache.aries.tx.control.jpa.common.impl.AbstractManagedJPADataSourceSetup;
 import org.apache.aries.tx.control.jpa.common.impl.AbstractManagedJPAEMFLocator;
-import org.apache.aries.tx.control.resource.common.impl.LifecycleAware;
 import org.osgi.framework.BundleContext;
 import org.osgi.framework.InvalidSyntaxException;
 import org.osgi.service.cm.ConfigurationException;

Modified: aries/trunk/tx-control/tx-control-provider-jpa-xa/pom.xml
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jpa-xa/pom.xml?rev=1784160&r1=1784159&r2=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jpa-xa/pom.xml (original)
+++ aries/trunk/tx-control/tx-control-provider-jpa-xa/pom.xml Thu Feb 23 15:47:41 2017
@@ -100,6 +100,10 @@
 			<groupId>org.osgi</groupId>
 			<artifactId>org.osgi.service.jpa</artifactId>
 		</dependency>
+		<dependency>
+            <groupId>org.osgi</groupId>
+            <artifactId>org.osgi.service.metatype.annotations</artifactId>
+        </dependency>
 
 		<!-- JPA API dependency -->
 		<dependency>

Copied: aries/trunk/tx-control/tx-control-provider-jpa-xa/src/main/java/org/apache/aries/tx/control/jpa/xa/impl/Config.java (from r1783901, aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java)
URL: http://svn.apache.org/viewvc/aries/trunk/tx-control/tx-control-provider-jpa-xa/src/main/java/org/apache/aries/tx/control/jpa/xa/impl/Config.java?p2=aries/trunk/tx-control/tx-control-provider-jpa-xa/src/main/java/org/apache/aries/tx/control/jpa/xa/impl/Config.java&p1=aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java&r1=1783901&r2=1784160&rev=1784160&view=diff
==============================================================================
--- aries/trunk/tx-control/tx-control-provider-jdbc-local/src/main/java/org/apache/aries/tx/control/jdbc/local/impl/Config.java (original)
+++ aries/trunk/tx-control/tx-control-provider-jpa-xa/src/main/java/org/apache/aries/tx/control/jpa/xa/impl/Config.java Thu Feb 23 15:47:41 2017
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.aries.tx.control.jdbc.local.impl;
+package org.apache.aries.tx.control.jpa.xa.impl;
 
 import static org.osgi.service.jdbc.DataSourceFactory.JDBC_DATABASE_NAME;
 import static org.osgi.service.jdbc.DataSourceFactory.JDBC_DATASOURCE_NAME;
@@ -33,12 +33,16 @@ import org.osgi.service.metatype.annotat
 import org.osgi.service.metatype.annotations.AttributeType;
 import org.osgi.service.metatype.annotations.ObjectClassDefinition;
 
-@ObjectClassDefinition(factoryPid="org.apache.aries.tx.control.jdbc.local", description="Aries Transaction Control Factory for Local JDBCResourceProvider Services")
+@ObjectClassDefinition(factoryPid="org.apache.aries.tx.control.jpa.xa", description="Aries Transaction Control Factory for XA enabled JPAEntityManagerProvider Services")
 public @interface Config {
 
 	// Most commonly used properties declared first so that they go into the metatype first
 	
 	@AttributeDefinition(required=false, 
+			description="The name of the persistence unit that this configuration should create a resource for.")
+	String osgi_unit_name();
+
+	@AttributeDefinition(required=false, 
 			description="The name of the driver class for the DataSourceFactory service. This property need not be defined if aries.dsf.target.filter is defined.")
 	String osgi_jdbc_driver_class();
 
@@ -51,7 +55,7 @@ public @interface Config {
 	@AttributeDefinition(type=AttributeType.PASSWORD, required=false, 
 			description="The password to pass to the DataSourceFactory (not visible as a service property)")
 	String password();
-	
+
 	// Pool configuration properties
 	
 	@AttributeDefinition(required=false, description="Is connection pooling enabled for this JDBCResourceProvider")
@@ -72,8 +76,14 @@ public @interface Config {
 	@AttributeDefinition(required=false, description="The maximum time (in ms) that a connection will stay in the pool before being discarded")
 	long osgi_connection_lifetime() default 10800000;
 	
+	@AttributeDefinition(required=false, description="The query that will be executed just before a connection is given to you from the pool to validate that the connection to the database is still alive. If your driver supports JDBC4 we strongly recommend not setting this property. This is for 'legacy' databases that do not support the JDBC Connection.isValid() API")
+	String aries_connection_test_query();	
+	
 	// Detailed Configuration
 	
+	@AttributeDefinition(required=false, description="The filter to use when searching for an EntityManagerFactoryBuilder. This property need not be defined if osgi.unit.name is defined.")
+	String aries_emf_builder_target_filter();
+	
 	@AttributeDefinition(required=false, description="The filter to use when finding the DataSourceFactory service. This property need not be defined if osgi.jdbc.driver.class is defined.")
 	String aries_dsf_target_filter();