NO-JIRA Testsuite speedup: proper JDBC drop from derby

This commit is contained in:
Clebert Suconic 2023-03-16 20:36:09 -04:00 committed by clebertsuconic
parent aaa8627795
commit d730d1a684
24 changed files with 309 additions and 366 deletions

View File

@ -53,12 +53,21 @@ import org.apache.activemq.artemis.quorum.UnavailableStateException;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.hamcrest.MatcherAssert;
import org.hamcrest.core.IsInstanceOf;
import org.junit.After;
import org.junit.Test;
import static org.hamcrest.CoreMatchers.instanceOf;
public class HAPolicyConfigurationTest extends ActiveMQTestBase {
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
shutdownDerby();
}
@Test
public void shouldNotUseJdbcNodeManagerWithoutHAPolicy() throws Exception {
Configuration configuration = createConfiguration("database-store-no-hapolicy-config.xml");

View File

@ -16,26 +16,26 @@
*/
package org.apache.activemq.artemis.core.server.impl.jdbc;
import java.sql.DriverManager;
import java.lang.invoke.MethodHandles;
import java.util.Arrays;
import java.util.Collection;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.core.config.storage.DatabaseStorageConfiguration;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@RunWith(Parameterized.class)
public class JdbcNodeManagerTest extends ActiveMQTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Parameterized.Parameter
public boolean useAuthentication;
private DatabaseStorageConfiguration dbConf;
@ -50,43 +50,18 @@ public class JdbcNodeManagerTest extends ActiveMQTestBase {
public void configure() {
if (useAuthentication) {
System.setProperty("derby.connection.requireAuthentication", "true");
System.setProperty("derby.user." + getJdbcUser(), getJdbcPassword());
System.setProperty("derby.user." + getJDBCUser(), getJDBCPassword());
}
dbConf = createDefaultDatabaseStorageConfiguration();
dbConf.setJdbcUser(getJdbcUser());
dbConf.setJdbcPassword(getJdbcPassword());
dbConf.setJdbcUser(getJDBCUser());
dbConf.setJdbcPassword(getJDBCPassword());
leaseLockExecutor = Executors.newSingleThreadScheduledExecutor();
runAfter(leaseLockExecutor::shutdownNow);
}
@After
public void shutdownExecutors() throws InterruptedException {
try {
final CountDownLatch latch = new CountDownLatch(1);
leaseLockExecutor.execute(latch::countDown);
Assert.assertTrue("the scheduler of the lease lock has some pending task in ", latch.await(10, TimeUnit.SECONDS));
} finally {
leaseLockExecutor.shutdownNow();
}
}
@After
@Override
public void shutdownDerby() {
try {
if (useAuthentication) {
DriverManager.getConnection("jdbc:derby:;shutdown=true", getJdbcUser(), getJdbcPassword());
} else {
DriverManager.getConnection("jdbc:derby:;shutdown=true");
}
} catch (Exception ignored) {
}
if (useAuthentication) {
System.clearProperty("derby.connection.requireAuthentication");
System.clearProperty("derby.user." + getJdbcUser());
}
}
protected String getJdbcUser() {
protected String getJDBCUser() {
if (useAuthentication) {
return System.getProperty("jdbc.user", "testuser");
} else {
@ -94,7 +69,8 @@ public class JdbcNodeManagerTest extends ActiveMQTestBase {
}
}
protected String getJdbcPassword() {
@Override
protected String getJDBCPassword() {
if (useAuthentication) {
return System.getProperty("jdbc.password", "testpassword");
} else {

View File

@ -38,12 +38,8 @@ import java.io.ObjectOutputStream;
import java.io.OutputStream;
import java.lang.management.ManagementFactory;
import java.lang.ref.WeakReference;
import java.sql.Connection;
import java.sql.Driver;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@ -138,8 +134,6 @@ import org.apache.activemq.artemis.core.settings.impl.AddressFullMessagePolicy;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.core.settings.impl.PageFullMessagePolicy;
import org.apache.activemq.artemis.core.transaction.impl.XidImpl;
import org.apache.activemq.artemis.jdbc.store.drivers.JDBCUtils;
import org.apache.activemq.artemis.jdbc.store.sql.SQLProvider;
import org.apache.activemq.artemis.json.JsonObject;
import org.apache.activemq.artemis.nativo.jlibaio.LibaioContext;
import org.apache.activemq.artemis.spi.core.security.ActiveMQJAASSecurityManager;
@ -160,6 +154,7 @@ import org.apache.activemq.artemis.utils.actors.OrderedExecutorFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.lang.invoke.MethodHandles;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@ -178,12 +173,6 @@ import org.junit.runner.Description;
*/
public abstract class ActiveMQTestBase extends Assert {
public MBeanServer getMBeanServer() {
MBeanServer mBeanServer = MBeanServerFactory.createMBeanServer();
runAfter(() -> MBeanServerFactory.releaseMBeanServer(mBeanServer));
return mBeanServer;
}
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
static {
@ -235,29 +224,64 @@ public abstract class ActiveMQTestBase extends Assert {
// There is a verification about thread leakages. We only fail a single thread when this happens
private static Set<Thread> alreadyFailedThread = new HashSet<>();
private LinkedList<RunnableEx> runAfter;
private List<RunnableEx> runAfter;
protected synchronized void runAfter(RunnableEx run) {
Assert.assertNotNull(run);
/**
* Use this method to cleanup your resources by passing lambdas.
* Exceptions thrown from your lambdas will just be logged and not passed as failures.
* @param lambda A RunnableEX instance that will be passed possibly from a lambda method
*/
protected void runAfter(RunnableEx lambda) {
runAfterEx(() -> {
try {
lambda.run();
} catch (Throwable e) {
logger.warn("Lambda {} is throwing an exception", lambda.toString(), e);
}
});
}
/**
* Use this method to cleanup your resources and validating exceptional results by passing lambdas.
* Exceptions thrown from your lambdas will be sent straight to JUNIT.
* If more than one lambda threw an exception they will all be executed, however only the exception of the first one will
* sent to the junit runner.
* @param lambda A RunnableEX instance that will be passed possibly from a lambda method
*/
protected synchronized void runAfterEx(RunnableEx lambda) {
Assert.assertNotNull(lambda);
if (runAfter == null) {
runAfter = new LinkedList();
runAfter = new ArrayList<>();
}
runAfter.add(run);
runAfter.add(lambda);
}
@After
public void runAfter() {
if (runAfter != null) {
runAfter.forEach((r) -> {
public synchronized void runAfter() throws Throwable {
ArrayList<Throwable> throwables = new ArrayList<>();
List<RunnableEx> localRunAfter = runAfter;
runAfter = null;
if (localRunAfter != null) {
localRunAfter.forEach((r) -> {
try {
r.run();
} catch (Exception e) {
} catch (Throwable e) {
logger.warn(e.getMessage(), e);
throwables.add(e);
}
});
}
if (!throwables.isEmpty()) {
throw throwables.get(0);
}
}
public MBeanServer createMBeanServer() {
MBeanServer mBeanServer = MBeanServerFactory.createMBeanServer();
runAfter(() -> MBeanServerFactory.releaseMBeanServer(mBeanServer));
return mBeanServer;
}
protected void clearServers() {
servers.clear();
@ -300,16 +324,50 @@ public abstract class ActiveMQTestBase extends Assert {
}
};
@After
public void shutdownDerby() {
// Static variable used by dropDerby
private static final String EXPECTED_DERBY_DROP_STATE = "08006";
// Static variable used by dropDerby
private static final String EXPECTED_DERBY_SHUTDOWN_STATE = "XJ015";
/** This method will be passed as a lambda into runAfter from createDefaultDatabaseStorageConfiguration */
protected final void dropDerby() throws Exception {
String user = getJDBCUser();
String password = getJDBCPassword();
try {
DriverManager.getConnection("jdbc:derby:" + getEmbeddedDataBaseName() + ";destroy=true");
} catch (Exception ignored) {
if (user == null) {
DriverManager.getConnection("jdbc:derby:" + getEmbeddedDataBaseName() + ";drop=true");
} else {
DriverManager.getConnection("jdbc:derby:" + getEmbeddedDataBaseName() + ";drop=true", user, password);
}
} catch (SQLException sqlE) {
if (!sqlE.getSQLState().equals(EXPECTED_DERBY_DROP_STATE)) {
logger.warn("{} / {}", sqlE.getMessage(), sqlE.getSQLState());
throw sqlE;
} else {
logger.info("{} / {}", sqlE.getMessage(), sqlE.getSQLState());
}
}
}
/** Some tests may be using file database as they share the database with a process.
* these tests will call shutdown Derby only */
protected void shutdownDerby() throws SQLException {
String user = getJDBCUser();
String password = getJDBCPassword();
try {
DriverManager.getConnection("jdbc:derby:;shutdown=true");
} catch (Exception ignored) {
// it always throws an exception on shutdown
if (user == null) {
DriverManager.getConnection("jdbc:derby:;shutdown=true;deregister=false");
} else {
DriverManager.getConnection("jdbc:derby:;shutdown=true;deregister=false", user, password);
}
} catch (SQLException sqlE) {
Assert.assertEquals("XJ015", sqlE.getSQLState());
logger.debug("{} / {}", sqlE.getMessage(), sqlE.getSQLState());
if (!sqlE.getSQLState().equals(EXPECTED_DERBY_SHUTDOWN_STATE)) {
throw sqlE;
}
}
}
@ -560,9 +618,22 @@ public abstract class ActiveMQTestBase extends Assert {
configuration.setStoreConfiguration(createDefaultDatabaseStorageConfiguration());
}
private boolean derbyDropped = false;
protected DatabaseStorageConfiguration createDefaultDatabaseStorageConfiguration() {
DatabaseStorageConfiguration dbStorageConfiguration = new DatabaseStorageConfiguration();
dbStorageConfiguration.setJdbcConnectionUrl(getTestJDBCConnectionUrl());
String connectionURI = getTestJDBCConnectionUrl();
/** The connectionURI could be passed into the testsuite as a system property (say you are testing against Oracle).
* So, we only schedule the drop on Derby if we are using a derby memory database */
if (connectionURI.contains("derby") && connectionURI.contains("memory") && !derbyDropped) {
// some tests will reinitialize the server and call this method more than one time
// and we should only schedule one task
derbyDropped = true;
runAfterEx(this::dropDerby);
runAfterEx(this::shutdownDerby);
}
dbStorageConfiguration.setJdbcConnectionUrl(connectionURI);
dbStorageConfiguration.setBindingsTableName("BINDINGS");
dbStorageConfiguration.setMessageTableName("MESSAGE");
dbStorageConfiguration.setLargeMessageTableName("LARGE_MESSAGE");
@ -590,55 +661,6 @@ public abstract class ActiveMQTestBase extends Assert {
return Long.getLong("jdbc.lock.renew", 200);
}
public void destroyTables(List<String> tableNames) throws Exception {
Driver driver = getDriver(getJDBCClassName());
Connection connection = driver.connect(getTestJDBCConnectionUrl(), null);
Statement statement = connection.createStatement();
try {
for (String tableName : tableNames) {
connection.setAutoCommit(false);
SQLProvider sqlProvider = JDBCUtils.getSQLProvider(getJDBCClassName(), tableName, SQLProvider.DatabaseStoreType.LARGE_MESSAGE);
try (ResultSet rs = connection.getMetaData().getTables(null, null, sqlProvider.getTableName(), null)) {
if (rs.next()) {
statement.execute("DROP TABLE " + sqlProvider.getTableName());
}
connection.commit();
} catch (SQLException e) {
connection.rollback();
}
}
connection.setAutoCommit(true);
} catch (Throwable e) {
e.printStackTrace();
} finally {
connection.close();
}
}
private Driver getDriver(String className) throws Exception {
try {
Driver driver = (Driver) Class.forName(className).newInstance();
// Shutdown the derby if using the derby embedded driver.
if (className.equals("org.apache.derby.jdbc.EmbeddedDriver")) {
Runtime.getRuntime().addShutdownHook(new Thread() {
@Override
public void run() {
try {
DriverManager.getConnection("jdbc:derby:;shutdown=true");
} catch (Exception e) {
}
}
});
}
return driver;
} catch (ClassNotFoundException cnfe) {
throw new RuntimeException("Could not find class: " + className);
} catch (Exception e) {
throw new RuntimeException("Unable to instantiate driver class: ", e);
}
}
protected Map<String, Object> generateInVMParams(final int node) {
Map<String, Object> params = new HashMap<>();
@ -672,7 +694,7 @@ public abstract class ActiveMQTestBase extends Assert {
protected final OrderedExecutorFactory getOrderedExecutor() {
final ExecutorService executor = Executors.newCachedThreadPool(ActiveMQThreadFactory.defaultThreadFactory(getClass().getName()));
executorSet.add(executor);
runAfter(executor::shutdownNow);
return new OrderedExecutorFactory(executor);
}
@ -897,7 +919,7 @@ public abstract class ActiveMQTestBase extends Assert {
return "memory:" + getTestDir();
}
protected final String getTestJDBCConnectionUrl() {
private String getTestJDBCConnectionUrl() {
return System.getProperty("jdbc.connection.url", "jdbc:derby:" + getEmbeddedDataBaseName() + ";create=true");
}

View File

@ -95,7 +95,7 @@ public class AmqpClientTestSupport extends AmqpTestSupport {
protected ActiveMQServer server;
protected MBeanServer mBeanServer = getMBeanServer();
protected MBeanServer mBeanServer = createMBeanServer();
@Before
@Override

View File

@ -412,7 +412,7 @@ public class HeuristicXATest extends ActiveMQTestBase {
@Before
public void setUp() throws Exception {
super.setUp();
mbeanServer = getMBeanServer();
mbeanServer = createMBeanServer();
locator = createInVMNonHALocator();
}

View File

@ -42,7 +42,6 @@ public class JdbcNodeManagerTest extends NodeManagerTest {
return th;
};
Thread[] threads = new Thread[actions.length];
List<ExecutorService> executors = new ArrayList<>(actions.length);
List<NodeManager> nodeManagers = new ArrayList<>(actions.length * 2);
AtomicBoolean failedRenew = new AtomicBoolean(false);
for (NodeManagerAction action : actions) {
@ -55,8 +54,8 @@ public class JdbcNodeManagerTest extends NodeManagerTest {
NodeRunner nodeRunner = new NodeRunner(nodeManager, action);
nodeRunners.add(nodeRunner);
nodeManagers.add(nodeManager);
executors.add(scheduledExecutorService);
executors.add(executor);
runAfter(scheduledExecutorService::shutdownNow);
runAfter(executor::shutdownNow);
}
for (int i = 0, nodeRunnersSize = nodeRunners.size(); i < nodeRunnersSize; i++) {
NodeRunner nodeRunner = nodeRunners.get(i);
@ -89,8 +88,6 @@ public class JdbcNodeManagerTest extends NodeManagerTest {
}
});
// stop executors
executors.forEach(ExecutorService::shutdownNow);
for (NodeRunner nodeRunner : nodeRunners) {
if (nodeRunner.e != null) {

View File

@ -52,7 +52,7 @@ public class FederatedTestBase extends ActiveMQTestBase {
public void setUp() throws Exception {
super.setUp();
for (int i = 0; i < numberOfServers(); i++) {
MBeanServer mBeanServer = getMBeanServer();
MBeanServer mBeanServer = createMBeanServer();
mBeanServers.add(mBeanServer);
Configuration config = createDefaultConfig(i, false).setSecurityEnabled(false);
for (int j = 0; j < numberOfServers(); j++) {

View File

@ -16,7 +16,7 @@
*/
package org.apache.activemq.artemis.tests.integration.jdbc.store.journal;
import java.sql.DriverManager;
import java.lang.invoke.MethodHandles;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Arrays;
@ -46,10 +46,14 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@RunWith(Parameterized.class)
public class JDBCJournalTest extends ActiveMQTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Rule
public ThreadLeakCheckRule threadLeakCheckRule = new ThreadLeakCheckRule();
@ -74,27 +78,16 @@ public class JDBCJournalTest extends ActiveMQTestBase {
@After
@Override
public void tearDown() throws Exception {
super.tearDown();
journal.destroy();
try {
if (useAuthentication) {
DriverManager.getConnection("jdbc:derby:;shutdown=true", getJdbcUser(), getJdbcPassword());
} else {
DriverManager.getConnection("jdbc:derby:;shutdown=true");
}
} catch (Exception ignored) {
}
if (useAuthentication) {
System.clearProperty("derby.connection.requireAuthentication");
System.clearProperty("derby.user." + getJdbcUser());
}
scheduledExecutorService.shutdownNow();
scheduledExecutorService.awaitTermination(5, TimeUnit.SECONDS);
scheduledExecutorService = null;
executorService.shutdown();
executorService = null;
}
protected String getJdbcUser() {
@Override
protected String getJDBCUser() {
if (useAuthentication) {
return System.getProperty("jdbc.user", "testuser");
} else {
@ -102,7 +95,8 @@ public class JDBCJournalTest extends ActiveMQTestBase {
}
}
protected String getJdbcPassword() {
@Override
protected String getJDBCPassword() {
if (useAuthentication) {
return System.getProperty("jdbc.password", "testpassword");
} else {
@ -115,9 +109,9 @@ public class JDBCJournalTest extends ActiveMQTestBase {
dbConf = createDefaultDatabaseStorageConfiguration();
if (useAuthentication) {
System.setProperty("derby.connection.requireAuthentication", "true");
System.setProperty("derby.user." + getJdbcUser(), getJdbcPassword());
dbConf.setJdbcUser(getJdbcUser());
dbConf.setJdbcPassword(getJdbcPassword());
System.setProperty("derby.user." + getJDBCUser(), getJDBCPassword());
dbConf.setJdbcUser(getJDBCUser());
dbConf.setJdbcPassword(getJDBCPassword());
}
sqlProvider = JDBCUtils.getSQLProvider(
dbConf.getJdbcDriverClassName(),

View File

@ -70,7 +70,7 @@ public abstract class MultiprotocolJMSClientTestSupport extends ActiveMQTestBase
protected ActiveMQServer server;
protected MBeanServer mBeanServer = getMBeanServer();
protected MBeanServer mBeanServer = createMBeanServer();
protected ConnectionSupplier AMQPConnection = () -> createConnection();
protected ConnectionSupplier CoreConnection = () -> createCoreConnection();

View File

@ -78,9 +78,6 @@ public abstract class LargeMessageTestBase extends ActiveMQTestBase {
@Override
public void tearDown() throws Exception {
super.tearDown();
if (storeType == StoreConfiguration.StoreType.DATABASE) {
destroyTables(Arrays.asList("BINDINGS", "LARGE_MESSAGE", "MESSAGE", "NODE_MANAGER_STORE"));
}
}
@Parameterized.Parameters(name = "storeType={0}")

View File

@ -138,7 +138,7 @@ public class BridgeControlTest extends ManagementTestBase {
Configuration conf_0 = createBasicConfig().addAcceptorConfiguration(new TransportConfiguration(InVMAcceptorFactory.class.getName())).addConnectorConfiguration(connectorConfig.getName(), connectorConfig).addQueueConfiguration(sourceQueueConfig).addBridgeConfiguration(bridgeConfig);
server_1 = addServer(ActiveMQServers.newActiveMQServer(conf_1, getMBeanServer(), false));
server_1 = addServer(ActiveMQServers.newActiveMQServer(conf_1, createMBeanServer(), false));
addServer(server_1);
server_1.start();

View File

@ -115,7 +115,7 @@ public class BridgeControlUsingCoreTest extends ManagementTestBase {
Configuration conf_0 = createBasicConfig().addAcceptorConfiguration(new TransportConfiguration(INVM_ACCEPTOR_FACTORY)).addConnectorConfiguration(connectorConfig.getName(), connectorConfig).addQueueConfiguration(sourceQueueConfig).addBridgeConfiguration(bridgeConfig);
server_1 = addServer(ActiveMQServers.newActiveMQServer(conf_1, getMBeanServer(), false));
server_1 = addServer(ActiveMQServers.newActiveMQServer(conf_1, createMBeanServer(), false));
server_1.start();
server_0 = addServer(ActiveMQServers.newActiveMQServer(conf_0, mbeanServer, false));

View File

@ -117,7 +117,7 @@ public class ClusterConnectionControl2Test extends ManagementTestBase {
Configuration conf_0 = createBasicConfig(1).addClusterConfiguration(clusterConnectionConfig_0).addAcceptorConfiguration(acceptorConfig_0).addConnectorConfiguration("netty", connectorConfig_0).addDiscoveryGroupConfiguration(discoveryName, discoveryGroupConfig).addBroadcastGroupConfiguration(broadcastGroupConfig);
mbeanServer_1 = getMBeanServer();
mbeanServer_1 = createMBeanServer();
server1 = addServer(ActiveMQServers.newActiveMQServer(conf_1, mbeanServer_1, false));
server0 = addServer(ActiveMQServers.newActiveMQServer(conf_0, mbeanServer, false));

View File

@ -205,7 +205,7 @@ public class ClusterConnectionControlTest extends ManagementTestBase {
Configuration conf_0 = createBasicConfig().addAcceptorConfiguration(new TransportConfiguration(InVMAcceptorFactory.class.getName())).addConnectorConfiguration(connectorConfig.getName(), connectorConfig).addClusterConfiguration(clusterConnectionConfig1).addClusterConfiguration(clusterConnectionConfig2).addDiscoveryGroupConfiguration(discoveryGroupName, discoveryGroupConfig);
mbeanServer_1 = getMBeanServer();
mbeanServer_1 = createMBeanServer();
server_1 = addServer(ActiveMQServers.newActiveMQServer(conf_1, mbeanServer_1, false));
server_1.start();

View File

@ -43,7 +43,7 @@ public class ConnectionRouterControlTest extends RoutingTestBase {
public void setUp() throws Exception {
super.setUp();
mbeanServer = getMBeanServer();
mbeanServer = createMBeanServer();
}
@Override

View File

@ -34,7 +34,7 @@ import org.junit.Before;
public abstract class ManagementTestBase extends ActiveMQTestBase {
protected MBeanServer mbeanServer = getMBeanServer();
protected MBeanServer mbeanServer = createMBeanServer();
@ -70,7 +70,7 @@ public abstract class ManagementTestBase extends ActiveMQTestBase {
public void setUp() throws Exception {
super.setUp();
getMBeanServer();
createMBeanServer();
}
@Override

View File

@ -113,7 +113,7 @@ public class OpenWireTestBase extends ActiveMQTestBase {
securityManager.getConfiguration().addRole("openwireDestinationManager", "advisoryReceiver");
}
mbeanServer = getMBeanServer();
mbeanServer = createMBeanServer();
server.setMBeanServer(mbeanServer);
addServer(server);
server.start();

View File

@ -1,161 +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.activemq.artemis.tests.integration.persistence;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
import org.apache.activemq.artemis.api.core.Pair;
import org.apache.activemq.artemis.core.config.StoreConfiguration;
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
import org.apache.activemq.artemis.jms.persistence.config.PersistedConnectionFactory;
import org.apache.activemq.artemis.jms.server.config.ConnectionFactoryConfiguration;
import org.apache.activemq.artemis.jms.server.config.impl.ConnectionFactoryConfigurationImpl;
import org.apache.activemq.artemis.utils.RandomUtil;
import org.junit.Before;
import org.junit.Test;
public class JMSConnectionFactoryConfigurationStorageTest extends StorageManagerTestBase {
private Map<String, PersistedConnectionFactory> mapExpectedCFs;
public JMSConnectionFactoryConfigurationStorageTest(StoreConfiguration.StoreType storeType) {
super(storeType);
}
@Override
@Before
public void setUp() throws Exception {
super.setUp();
mapExpectedCFs = new HashMap<>();
}
protected void addSetting(PersistedConnectionFactory setting) throws Exception {
mapExpectedCFs.put(setting.getName(), setting);
jmsJournal.storeConnectionFactory(setting);
}
@Test
public void testSettings() throws Exception {
createJMSStorage();
List<String> transportConfigs = new ArrayList<>();
for (int i = 0; i < 5; i++) {
transportConfigs.add("c1-" + i);
transportConfigs.add("c2-" + i);
}
ConnectionFactoryConfiguration config = new ConnectionFactoryConfigurationImpl().setName("some-name").setConnectorNames(transportConfigs);
addSetting(new PersistedConnectionFactory(config));
jmsJournal.stop();
createJMSStorage();
List<PersistedConnectionFactory> cfs = jmsJournal.recoverConnectionFactories();
assertEquals(1, cfs.size());
assertEquals("some-name", cfs.get(0).getName());
PersistedConnectionFactory cf1 = cfs.get(0);
assertEquals(10, cf1.getConfig().getConnectorNames().size());
List<String> configs = cf1.getConfig().getConnectorNames();
for (int i = 0, j = 0; i < 10; i += 2, j++) {
assertEquals(configs.get(i), "c1-" + j);
assertEquals(configs.get(i + 1), "c2-" + j);
}
}
@Test
public void testSizeOfCF() throws Exception {
String[] str = new String[5];
for (int i = 0; i < 5; i++) {
str[i] = "str" + i;
}
ConnectionFactoryConfiguration config = new ConnectionFactoryConfigurationImpl().setName("some-name").setConnectorNames(new ArrayList<String>()).setBindings("");
int size = config.getEncodeSize();
ActiveMQBuffer buffer = ActiveMQBuffers.fixedBuffer(size);
config.encode(buffer);
assertEquals(size, buffer.writerIndex());
PersistedConnectionFactory persistedCF = new PersistedConnectionFactory(config);
size = persistedCF.getEncodeSize();
buffer = ActiveMQBuffers.fixedBuffer(size);
persistedCF.encode(buffer);
assertEquals(size, buffer.writerIndex());
}
@Test
public void testSettingsWithConnectorConfigs() throws Exception {
createJMSStorage();
String[] str = new String[5];
for (int i = 0; i < 5; i++) {
str[i] = "str" + i;
}
List<String> connectorConfigs = new ArrayList<>();
Map<String, Object> liveParams = new HashMap<>();
liveParams.put(TransportConstants.PORT_PROP_NAME, 5665);
Map<String, Object> backupParams = new HashMap<>();
backupParams.put(TransportConstants.PORT_PROP_NAME, 5775);
Map<String, Object> liveParams2 = new HashMap<>();
liveParams2.put(TransportConstants.PORT_PROP_NAME, 6665);
ConnectionFactoryConfiguration config = new ConnectionFactoryConfigurationImpl().setName("some-name").setConnectorNames(connectorConfigs).setBindings(str).setCallTimeout(RandomUtil.randomPositiveLong());
List<Pair<String, String>> connectors = new ArrayList<>();
connectors.add(new Pair<String, String>(RandomUtil.randomString(), null));
//config.setConnectorNames(connectors);
addSetting(new PersistedConnectionFactory(config));
jmsJournal.stop();
createJMSStorage();
List<PersistedConnectionFactory> cfs = jmsJournal.recoverConnectionFactories();
assertEquals(1, cfs.size());
assertEquals("some-name", cfs.get(0).getName());
assertEquals(config.getCallTimeout(), cfs.get(0).getConfig().getCallTimeout());
}
}

View File

@ -16,21 +16,159 @@
*/
package org.apache.activemq.artemis.tests.integration.persistence;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.activemq.artemis.core.config.StoreConfiguration;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
import org.apache.activemq.artemis.api.core.Pair;
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
import org.apache.activemq.artemis.jms.persistence.JMSStorageManager;
import org.apache.activemq.artemis.jms.persistence.config.PersistedBindings;
import org.apache.activemq.artemis.jms.persistence.config.PersistedConnectionFactory;
import org.apache.activemq.artemis.jms.persistence.config.PersistedDestination;
import org.apache.activemq.artemis.jms.persistence.config.PersistedType;
import org.apache.activemq.artemis.jms.persistence.impl.journal.JMSJournalStorageManagerImpl;
import org.apache.activemq.artemis.jms.server.config.ConnectionFactoryConfiguration;
import org.apache.activemq.artemis.jms.server.config.impl.ConnectionFactoryConfigurationImpl;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.utils.RandomUtil;
import org.apache.activemq.artemis.utils.TimeAndCounterIDGenerator;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class JMSStorageManagerTest extends StorageManagerTestBase {
public class JMSStorageManagerTest extends ActiveMQTestBase {
public JMSStorageManagerTest(StoreConfiguration.StoreType storeType) {
super(storeType);
private Map<String, PersistedConnectionFactory> mapExpectedCFs;
protected JMSStorageManager jmsJournal;
public JMSStorageManagerTest() {
super();
}
@Override
@Before
public void setUp() throws Exception {
super.setUp();
mapExpectedCFs = new HashMap<>();
}
protected void addSetting(PersistedConnectionFactory setting) throws Exception {
mapExpectedCFs.put(setting.getName(), setting);
jmsJournal.storeConnectionFactory(setting);
}
@Test
public void testSettings() throws Exception {
createJMSStorage();
List<String> transportConfigs = new ArrayList<>();
for (int i = 0; i < 5; i++) {
transportConfigs.add("c1-" + i);
transportConfigs.add("c2-" + i);
}
ConnectionFactoryConfiguration config = new ConnectionFactoryConfigurationImpl().setName("some-name").setConnectorNames(transportConfigs);
addSetting(new PersistedConnectionFactory(config));
jmsJournal.stop();
createJMSStorage();
List<PersistedConnectionFactory> cfs = jmsJournal.recoverConnectionFactories();
assertEquals(1, cfs.size());
assertEquals("some-name", cfs.get(0).getName());
PersistedConnectionFactory cf1 = cfs.get(0);
assertEquals(10, cf1.getConfig().getConnectorNames().size());
List<String> configs = cf1.getConfig().getConnectorNames();
for (int i = 0, j = 0; i < 10; i += 2, j++) {
assertEquals(configs.get(i), "c1-" + j);
assertEquals(configs.get(i + 1), "c2-" + j);
}
}
@Test
public void testSizeOfCF() throws Exception {
String[] str = new String[5];
for (int i = 0; i < 5; i++) {
str[i] = "str" + i;
}
ConnectionFactoryConfiguration config = new ConnectionFactoryConfigurationImpl().setName("some-name").setConnectorNames(new ArrayList<String>()).setBindings("");
int size = config.getEncodeSize();
ActiveMQBuffer buffer = ActiveMQBuffers.fixedBuffer(size);
config.encode(buffer);
assertEquals(size, buffer.writerIndex());
PersistedConnectionFactory persistedCF = new PersistedConnectionFactory(config);
size = persistedCF.getEncodeSize();
buffer = ActiveMQBuffers.fixedBuffer(size);
persistedCF.encode(buffer);
assertEquals(size, buffer.writerIndex());
}
@Test
public void testSettingsWithConnectorConfigs() throws Exception {
createJMSStorage();
String[] str = new String[5];
for (int i = 0; i < 5; i++) {
str[i] = "str" + i;
}
List<String> connectorConfigs = new ArrayList<>();
Map<String, Object> liveParams = new HashMap<>();
liveParams.put(TransportConstants.PORT_PROP_NAME, 5665);
Map<String, Object> backupParams = new HashMap<>();
backupParams.put(TransportConstants.PORT_PROP_NAME, 5775);
Map<String, Object> liveParams2 = new HashMap<>();
liveParams2.put(TransportConstants.PORT_PROP_NAME, 6665);
ConnectionFactoryConfiguration config = new ConnectionFactoryConfigurationImpl().setName("some-name").setConnectorNames(connectorConfigs).setBindings(str).setCallTimeout(RandomUtil.randomPositiveLong());
List<Pair<String, String>> connectors = new ArrayList<>();
connectors.add(new Pair<String, String>(RandomUtil.randomString(), null));
//config.setConnectorNames(connectors);
addSetting(new PersistedConnectionFactory(config));
jmsJournal.stop();
createJMSStorage();
List<PersistedConnectionFactory> cfs = jmsJournal.recoverConnectionFactories();
assertEquals(1, cfs.size());
assertEquals("some-name", cfs.get(0).getName());
assertEquals(config.getCallTimeout(), cfs.get(0).getConfig().getCallTimeout());
}
//https://issues.jboss.org/browse/HORNETQ-812
@Test
public void testJNDIPersistence() throws Exception {
@ -76,4 +214,16 @@ public class JMSStorageManagerTest extends StorageManagerTestBase {
}
/**
* @throws Exception
*/
protected void createJMSStorage() throws Exception {
jmsJournal = new JMSJournalStorageManagerImpl(null, new TimeAndCounterIDGenerator(), createDefaultInVMConfig(), null);
runAfter(jmsJournal::stop);
jmsJournal.start();
jmsJournal.load();
}
}

View File

@ -31,13 +31,10 @@ import org.apache.activemq.artemis.core.persistence.QueueBindingInfo;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.persistence.impl.journal.JDBCJournalStorageManager;
import org.apache.activemq.artemis.core.persistence.impl.journal.JournalStorageManager;
import org.apache.activemq.artemis.jms.persistence.JMSStorageManager;
import org.apache.activemq.artemis.jms.persistence.impl.journal.JMSJournalStorageManagerImpl;
import org.apache.activemq.artemis.tests.unit.core.server.impl.fakes.FakeJournalLoader;
import org.apache.activemq.artemis.tests.unit.core.server.impl.fakes.FakePostOffice;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.utils.ExecutorFactory;
import org.apache.activemq.artemis.utils.TimeAndCounterIDGenerator;
import org.apache.activemq.artemis.utils.critical.EmptyCriticalAnalyzer;
import org.junit.After;
import org.junit.Before;
@ -55,8 +52,6 @@ public abstract class StorageManagerTestBase extends ActiveMQTestBase {
protected StorageManager journal;
protected JMSStorageManager jmsJournal;
protected StoreConfiguration.StoreType storeType;
public StorageManagerTestBase(StoreConfiguration.StoreType storeType) {
@ -73,10 +68,6 @@ public abstract class StorageManagerTestBase extends ActiveMQTestBase {
@Before
@SuppressWarnings("unused")
public void setUp() throws Exception {
if (storeType == StoreConfiguration.StoreType.DATABASE) {
Object unused = Class.forName("org.apache.derby.jdbc.EmbeddedDriver").newInstance();
}
super.setUp();
execFactory = getOrderedExecutor();
@ -99,20 +90,8 @@ public abstract class StorageManagerTestBase extends ActiveMQTestBase {
journal = null;
}
if (jmsJournal != null) {
try {
jmsJournal.stop();
} catch (Exception e) {
if (exception != null)
exception = e;
}
jmsJournal = null;
}
scheduledExecutorService.shutdown();
destroyTables(Arrays.asList(new String[]{"MESSAGE", "BINDINGS", "LARGE_MESSAGE", "NODE_MANAGER_STORE"}));
super.tearDown();
if (exception != null)
throw exception;
@ -154,14 +133,4 @@ public abstract class StorageManagerTestBase extends ActiveMQTestBase {
return jsm;
}
/**
* @throws Exception
*/
protected void createJMSStorage() throws Exception {
jmsJournal = new JMSJournalStorageManagerImpl(null, new TimeAndCounterIDGenerator(), createDefaultInVMConfig(), null);
addActiveMQComponent(jmsJournal);
jmsJournal.start();
jmsJournal.load();
}
}

View File

@ -365,7 +365,7 @@ public class ElasticQueueTest extends ActiveMQTestBase {
}
}
MBeanServer mBeanServer = getMBeanServer();
MBeanServer mBeanServer = createMBeanServer();
// hardwire authenticaton to map USER to EQ_USER etc
final ActiveMQSecurityManager5 customSecurityManager = new ActiveMQSecurityManager5() {

View File

@ -43,7 +43,6 @@ import org.apache.activemq.artemis.jms.client.ActiveMQTextMessage;
import org.apache.activemq.artemis.tests.integration.management.ManagementControlHelper;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.utils.UUIDGenerator;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -108,7 +107,7 @@ public class BasicXaRecoveryTest extends ActiveMQTestBase {
configuration = createDefaultInVMConfig().setJMXManagementEnabled(true);
}
mbeanServer = getMBeanServer();
mbeanServer = createMBeanServer();
server = createServer(true, configuration, -1, -1, addressSettings);
server.setMBeanServer(mbeanServer);
@ -120,15 +119,6 @@ public class BasicXaRecoveryTest extends ActiveMQTestBase {
createClients(true, false);
}
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
if (storeType == StoreConfiguration.StoreType.DATABASE) {
destroyTables(Arrays.asList("BINDINGS", "LARGE_MESSAGE", "MESSAGE", "NODE_MANAGER_STORE"));
}
}
@Test
public void testBasicSendWithCommit() throws Exception {

View File

@ -118,7 +118,7 @@ public class JMSClusteredTestBase extends ActiveMQTestBase {
JMSConfigurationImpl jmsconfig = new JMSConfigurationImpl();
mBeanServer2 = getMBeanServer();
mBeanServer2 = createMBeanServer();
server2 = addServer(ActiveMQServers.newActiveMQServer(configuration, mBeanServer2, enablePersistence()));
jmsServer2 = new JMSServerManagerImpl(server2, jmsconfig);
context2 = new InVMNamingContext();
@ -133,7 +133,7 @@ public class JMSClusteredTestBase extends ActiveMQTestBase {
JMSConfigurationImpl jmsconfig = new JMSConfigurationImpl();
mBeanServer1 = getMBeanServer();
mBeanServer1 = createMBeanServer();
server1 = addServer(ActiveMQServers.newActiveMQServer(configuration, mBeanServer1, enablePersistence()));
jmsServer1 = new JMSServerManagerImpl(server1, jmsconfig);
context1 = new InVMNamingContext();

View File

@ -126,7 +126,7 @@ public class JMSTestBase extends ActiveMQTestBase {
public void setUp() throws Exception {
super.setUp();
mbeanServer = getMBeanServer();
mbeanServer = createMBeanServer();
Configuration config = createDefaultConfig(true).setSecurityEnabled(useSecurity()).
addConnectorConfiguration("invm", new TransportConfiguration(INVM_CONNECTOR_FACTORY)).