ARTEMIS-4401 improving JDBC Performance with Paging by a significant factor

This commit is contained in:
Clebert Suconic 2023-08-24 14:25:26 -04:00 committed by clebertsuconic
parent d830f04de8
commit 6a8cd175dc
93 changed files with 4121 additions and 2435 deletions

View File

@ -116,13 +116,13 @@ public class PrintData extends DBOption {
printBanner(out, BINDINGS_BANNER); printBanner(out, BINDINGS_BANNER);
DescribeJournal.printSurvivingRecords(storageManager.getBindingsJournal(), out, safe); DescribeJournal bindings = DescribeJournal.printSurvivingRecords(storageManager.getBindingsJournal(), out, safe);
printBanner(out, MESSAGES_BANNER); printBanner(out, MESSAGES_BANNER);
DescribeJournal describeJournal = DescribeJournal.printSurvivingRecords(storageManager.getMessageJournal(), out, safe); DescribeJournal describeJournal = DescribeJournal.printSurvivingRecords(storageManager.getMessageJournal(), out, safe);
printPages(describeJournal, storageManager, pagingmanager, out, safe, maxPages, null); printPages(describeJournal, storageManager, pagingmanager, out, safe, maxPages, bindings);
cleanup(); cleanup();

View File

@ -14,5 +14,6 @@
<jdbc-lock-expiration>${jdbcLockExpiration}</jdbc-lock-expiration> <jdbc-lock-expiration>${jdbcLockExpiration}</jdbc-lock-expiration>
<jdbc-lock-renew-period>${jdbcLockRenewPeriod}</jdbc-lock-renew-period> <jdbc-lock-renew-period>${jdbcLockRenewPeriod}</jdbc-lock-renew-period>
<jdbc-network-timeout>${jdbcNetworkTimeout}</jdbc-network-timeout> <jdbc-network-timeout>${jdbcNetworkTimeout}</jdbc-network-timeout>
<jdbc-max-page-size-bytes>100K</jdbc-max-page-size-bytes>
</database-store> </database-store>
</store> </store>

View File

@ -496,6 +496,8 @@ public final class ActiveMQDefaultConfiguration {
private static final long DEFAULT_JDBC_ALLOWED_TIME_DIFF_MILLIS = 250; private static final long DEFAULT_JDBC_ALLOWED_TIME_DIFF_MILLIS = 250;
private static final int DEFAULT_JDBC_MAX_PAGE_SIZE_BYTES = 100 * 1024;
// Default period to wait between connection TTL checks // Default period to wait between connection TTL checks
public static final long DEFAULT_CONNECTION_TTL_CHECK_INTERVAL = 2000; public static final long DEFAULT_CONNECTION_TTL_CHECK_INTERVAL = 2000;
@ -1512,6 +1514,10 @@ public final class ActiveMQDefaultConfiguration {
return DEFAULT_JDBC_ALLOWED_TIME_DIFF_MILLIS; return DEFAULT_JDBC_ALLOWED_TIME_DIFF_MILLIS;
} }
public static int getDefaultJdbcMaxPageSizeBytes() {
return DEFAULT_JDBC_MAX_PAGE_SIZE_BYTES;
}
public static long getDefaultConnectionTtlCheckInterval() { public static long getDefaultConnectionTtlCheckInterval() {
return DEFAULT_CONNECTION_TTL_CHECK_INTERVAL; return DEFAULT_CONNECTION_TTL_CHECK_INTERVAL;
} }

View File

@ -184,6 +184,10 @@ public abstract class AbstractJDBCDriver {
} }
} }
public SQLProvider getSqlProvider() {
return sqlProvider;
}
public void setSqlProvider(SQLProvider sqlProvider) { public void setSqlProvider(SQLProvider sqlProvider) {
this.sqlProvider = sqlProvider; this.sqlProvider = sqlProvider;
} }

View File

@ -117,6 +117,10 @@ public class JDBCConnectionProvider {
} }
} }
public int getNetworkTimeoutMillis() {
return networkTimeoutMillis;
}
public void setNetworkTimeout(Executor executor, int milliseconds) { public void setNetworkTimeout(Executor executor, int milliseconds) {
this.networkTimeoutExecutor = executor; this.networkTimeoutExecutor = executor;
this.networkTimeoutMillis = milliseconds; this.networkTimeoutMillis = milliseconds;

View File

@ -32,7 +32,7 @@ public class JDBCDataSourceUtils {
.map(key -> key + "=" + (key.equalsIgnoreCase("password") ? "****" : dataSourceProperties.get(key))) .map(key -> key + "=" + (key.equalsIgnoreCase("password") ? "****" : dataSourceProperties.get(key)))
.collect(Collectors.joining(", ", "{", "}"))); .collect(Collectors.joining(", ", "{", "}")));
try { try {
DataSource dataSource = (DataSource) Class.forName(dataSourceClassName).newInstance(); DataSource dataSource = (DataSource) Class.forName(dataSourceClassName).getDeclaredConstructor().newInstance();
for (Map.Entry<String, Object> entry : dataSourceProperties.entrySet()) { for (Map.Entry<String, Object> entry : dataSourceProperties.entrySet()) {
PropertyUtils.setProperty(dataSource, entry.getKey(), entry.getValue()); PropertyUtils.setProperty(dataSource, entry.getKey(), entry.getValue());
} }

View File

@ -18,25 +18,32 @@ package org.apache.activemq.artemis.jdbc.store.file;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.sql.SQLException; import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executor; import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer; import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQBuffers; import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
import org.apache.activemq.artemis.api.core.ActiveMQException; import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.ActiveMQExceptionType; import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
import org.apache.activemq.artemis.api.core.ActiveMQIOErrorException;
import org.apache.activemq.artemis.core.io.IOCallback; import org.apache.activemq.artemis.core.io.IOCallback;
import org.apache.activemq.artemis.core.io.SequentialFile; import org.apache.activemq.artemis.core.io.SequentialFile;
import org.apache.activemq.artemis.core.io.buffer.TimedBuffer; import org.apache.activemq.artemis.core.io.buffer.TimedBuffer;
import org.apache.activemq.artemis.core.journal.EncodingSupport; import org.apache.activemq.artemis.core.journal.EncodingSupport;
import org.apache.activemq.artemis.core.journal.impl.SimpleWaitIOCallback; import org.apache.activemq.artemis.core.journal.impl.SimpleWaitIOCallback;
import org.apache.activemq.artemis.core.server.ActiveMQScheduledComponent;
import org.apache.activemq.artemis.utils.ReusableLatch;
import org.jctools.queues.MpscUnboundedArrayQueue;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.lang.invoke.MethodHandles;
public class JDBCSequentialFile implements SequentialFile { public class JDBCSequentialFile implements SequentialFile {
@ -64,12 +71,24 @@ public class JDBCSequentialFile implements SequentialFile {
private final JDBCSequentialFileFactoryDriver dbDriver; private final JDBCSequentialFileFactoryDriver dbDriver;
MpscUnboundedArrayQueue<ScheduledWrite> writeQueue = new MpscUnboundedArrayQueue<>(8192);
// Allows DB Drivers to cache meta data. // Allows DB Drivers to cache meta data.
private final Map<Object, Object> metaData = new ConcurrentHashMap<>(); private final Map<Object, Object> metaData = new ConcurrentHashMap<>();
final JDBCPageWriteScheduler pageWriteScheduler;
final ScheduledExecutorService scheduledExecutorService;
private final ReusableLatch pendingWrites = new ReusableLatch();
final long syncDelay;
JDBCSequentialFile(final JDBCSequentialFileFactory fileFactory, JDBCSequentialFile(final JDBCSequentialFileFactory fileFactory,
final String filename, final String filename,
final Executor executor, final Executor executor,
final ScheduledExecutorService scheduledExecutorService,
final long syncDelay,
final JDBCSequentialFileFactoryDriver driver, final JDBCSequentialFileFactoryDriver driver,
final Object writeLock) throws SQLException { final Object writeLock) throws SQLException {
this.fileFactory = fileFactory; this.fileFactory = fileFactory;
@ -78,6 +97,9 @@ public class JDBCSequentialFile implements SequentialFile {
this.executor = executor; this.executor = executor;
this.writeLock = writeLock; this.writeLock = writeLock;
this.dbDriver = driver; this.dbDriver = driver;
this.scheduledExecutorService = scheduledExecutorService;
this.syncDelay = syncDelay;
this.pageWriteScheduler = new JDBCPageWriteScheduler(scheduledExecutorService, executor, syncDelay);
} }
void setWritePosition(long writePosition) { void setWritePosition(long writePosition) {
@ -168,9 +190,9 @@ public class JDBCSequentialFile implements SequentialFile {
} }
} }
private synchronized int internalWrite(byte[] data, IOCallback callback, boolean append) { private synchronized int jdbcWrite(byte[] data, IOCallback callback, boolean append) {
try { try {
open(); logger.debug("Writing {} bytes into {}", data.length, filename);
synchronized (writeLock) { synchronized (writeLock) {
int noBytes = dbDriver.writeToFile(this, data, append); int noBytes = dbDriver.writeToFile(this, data, append);
seek(append ? writePosition + noBytes : noBytes); seek(append ? writePosition + noBytes : noBytes);
@ -189,52 +211,104 @@ public class JDBCSequentialFile implements SequentialFile {
return 0; return 0;
} }
public synchronized int internalWrite(ActiveMQBuffer buffer, IOCallback callback) { public synchronized int jdbcWrite(ActiveMQBuffer buffer, IOCallback callback) {
return internalWrite(buffer, callback, true); return jdbcWrite(buffer, callback, true);
} }
public synchronized int internalWrite(ActiveMQBuffer buffer, IOCallback callback, boolean append) { public synchronized int jdbcWrite(ActiveMQBuffer buffer, IOCallback callback, boolean append) {
byte[] data = new byte[buffer.readableBytes()]; byte[] data = new byte[buffer.readableBytes()];
buffer.readBytes(data); buffer.readBytes(data);
return internalWrite(data, callback, append); return jdbcWrite(data, callback, append);
} }
private synchronized int internalWrite(ByteBuffer buffer, IOCallback callback) {
final byte[] data;
if (buffer.hasArray() && buffer.arrayOffset() == 0 && buffer.position() == 0 && buffer.limit() == buffer.array().length) { private void pollWrites() {
data = buffer.array(); if (writeQueue.isEmpty()) {
} else { return;
byte[] copy = new byte[buffer.remaining()];
buffer.get(copy);
data = copy;
} }
return internalWrite(data, callback, true);
logger.debug("polling {} elements on {}", writeQueue.size(), this.filename);
ArrayList<ScheduledWrite> writeList = new ArrayList<>(writeQueue.size()); // the size here is just an estimate
byte[] bytes = extractBytes(writeList);
jdbcWrite(bytes, null, true);
writeList.forEach(this::doCallback);
}
/* Even though I would love to have a reusable byte[] for the following buffer
PreparedStatement.setData takes a byte[] without any sizing on the interface.
Blob interface would support setBytes with an offset and size, but some of the databases we are using
(DB2 specifically) is not allowing us to use Blob (at least during our dev time).
for that reason I'm using this byte[] with the very specific size that needs to be written
Also Notice that our PagingManager will make sure that this size wouldn't go beyond our page-size limit
which we also limit at the JDBC storage, which should be 100K. */
private byte[] extractBytes(ArrayList<ScheduledWrite> writeList) {
int totalSize = 0;
ScheduledWrite write;
while ((write = writeQueue.poll()) != null) {
writeList.add(write);
totalSize += write.readable();
}
byte[] bytes = new byte[totalSize];
int writePosition = 0;
for (ScheduledWrite el : writeList) {
writePosition += el.readAt(bytes, writePosition);
el.releaseBuffer();
}
return bytes;
}
private void doCallback(ScheduledWrite write) {
if (write != null && write.callback != null) {
write.callback.done();
}
pendingWrites.countDown();
} }
private void scheduleWrite(final ActiveMQBuffer bytes, final IOCallback callback, boolean append) { private void scheduleWrite(final ActiveMQBuffer bytes, final IOCallback callback, boolean append) {
executor.execute(() -> { scheduleWrite(new ScheduledWrite(bytes, callback, append));
internalWrite(bytes, callback, append); }
});
private void scheduleWrite(ScheduledWrite scheduledWrite) {
logger.debug("offering {} bytes into {}", scheduledWrite.readable(), filename);
pendingWrites.countUp();
writeQueue.offer(scheduledWrite);
this.pageWriteScheduler.delay();
} }
private void scheduleWrite(final ByteBuffer bytes, final IOCallback callback) { private void scheduleWrite(final ByteBuffer bytes, final IOCallback callback) {
executor.execute(() -> { scheduleWrite(new ScheduledWrite(bytes, callback, true));
internalWrite(bytes, callback);
});
} }
synchronized void seek(long noBytes) { synchronized void seek(long noBytes) {
writePosition = noBytes; writePosition = noBytes;
} }
public void write(ActiveMQBuffer bytes, boolean sync, IOCallback callback, boolean append) throws Exception { public void sendToDB(ActiveMQBuffer bytes, IOCallback callback, boolean append) throws Exception {
// We ignore sync since we schedule writes straight away. SimpleWaitIOCallback waitIOCallback = null;
if (callback == null) {
waitIOCallback = new SimpleWaitIOCallback();
callback = waitIOCallback;
}
scheduleWrite(bytes, callback, append); scheduleWrite(bytes, callback, append);
if (callback != null) {
waitIOCallback.waitCompletion();
}
} }
@Override @Override
public void write(ActiveMQBuffer bytes, boolean sync, IOCallback callback) throws Exception { public void write(ActiveMQBuffer bytes, boolean sync, IOCallback callback) throws Exception {
write(bytes, sync, callback, true); sendToDB(bytes, callback, true);
} }
@Override @Override
@ -246,7 +320,7 @@ public class JDBCSequentialFile implements SequentialFile {
public void write(EncodingSupport bytes, boolean sync, IOCallback callback) throws Exception { public void write(EncodingSupport bytes, boolean sync, IOCallback callback) throws Exception {
ActiveMQBuffer data = ActiveMQBuffers.fixedBuffer(bytes.getEncodeSize()); ActiveMQBuffer data = ActiveMQBuffers.fixedBuffer(bytes.getEncodeSize());
bytes.encode(data); bytes.encode(data);
write(data, sync, callback, true); sendToDB(data, callback, true);
} }
@Override @Override
@ -330,15 +404,27 @@ public class JDBCSequentialFile implements SequentialFile {
fileFactory.sequentialFileClosed(this); fileFactory.sequentialFileClosed(this);
} }
public int getNetworkTimeoutMillis() {
return dbDriver.getJdbcConnectionProvider().getNetworkTimeoutMillis();
}
@Override @Override
public void sync() throws IOException { public void sync() throws IOException {
final SimpleWaitIOCallback callback = new SimpleWaitIOCallback();
executor.execute(callback::done);
try { try {
callback.waitCompletion(); int syncTimeout = getNetworkTimeoutMillis();
if (syncTimeout >= 0) {
if (!pendingWrites.await(syncTimeout, TimeUnit.MILLISECONDS)) {
fileFactory.onIOError(new ActiveMQIOErrorException("Database not responding to syncs before timeout"), "Error during JDBC file sync.", this);
}
} else {
// waiting forever however logger.debug while doing so
while (!pendingWrites.await(1, TimeUnit.SECONDS)) {
logger.debug("Awaiting syncs from database for page file {}", this.filename);
}
}
} catch (Exception e) { } catch (Exception e) {
callback.onError(ActiveMQExceptionType.IO_ERROR.getCode(), "Error during JDBC file sync.");
fileFactory.onIOError(e, "Error during JDBC file sync.", this); fileFactory.onIOError(e, "Error during JDBC file sync.", this);
} }
} }
@ -363,7 +449,7 @@ public class JDBCSequentialFile implements SequentialFile {
@Override @Override
public SequentialFile cloneFile() { public SequentialFile cloneFile() {
try { try {
JDBCSequentialFile clone = new JDBCSequentialFile(fileFactory, filename, executor, dbDriver, writeLock); JDBCSequentialFile clone = new JDBCSequentialFile(fileFactory, filename, executor, scheduledExecutorService, syncDelay, dbDriver, writeLock);
clone.setWritePosition(this.writePosition); clone.setWritePosition(this.writePosition);
return clone; return clone;
} catch (Exception e) { } catch (Exception e) {
@ -421,4 +507,20 @@ public class JDBCSequentialFile implements SequentialFile {
public Object getMetaData(Object key) { public Object getMetaData(Object key) {
return metaData.get(key); return metaData.get(key);
} }
private class JDBCPageWriteScheduler extends ActiveMQScheduledComponent {
JDBCPageWriteScheduler(ScheduledExecutorService scheduledExecutorService,
Executor executor,
long checkPeriod) {
super(scheduledExecutorService, executor, checkPeriod, TimeUnit.MILLISECONDS, true);
}
@Override
public void run() {
pollWrites();
}
}
} }

View File

@ -36,6 +36,7 @@ import org.apache.activemq.artemis.utils.collections.ConcurrentHashSet;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.util.concurrent.ScheduledExecutorService;
public class JDBCSequentialFileFactory implements SequentialFileFactory, ActiveMQComponent { public class JDBCSequentialFileFactory implements SequentialFileFactory, ActiveMQComponent {
@ -53,13 +54,22 @@ public class JDBCSequentialFileFactory implements SequentialFileFactory, ActiveM
private final IOCriticalErrorListener criticalErrorListener; private final IOCriticalErrorListener criticalErrorListener;
private final long syncDelay;
private final ScheduledExecutorService scheduledExecutorService;
public JDBCSequentialFileFactory(final JDBCConnectionProvider connectionProvider, public JDBCSequentialFileFactory(final JDBCConnectionProvider connectionProvider,
final SQLProvider sqlProvider, final SQLProvider sqlProvider,
Executor executor, Executor executor,
ScheduledExecutorService scheduledExecutorService,
long syncDelay,
IOCriticalErrorListener criticalErrorListener) throws Exception { IOCriticalErrorListener criticalErrorListener) throws Exception {
this.executor = executor; this.executor = executor;
this.criticalErrorListener = criticalErrorListener; this.criticalErrorListener = criticalErrorListener;
this.scheduledExecutorService = scheduledExecutorService;
this.syncDelay = syncDelay;
try { try {
this.dbDriver = JDBCFileUtils.getDBFileDriver(connectionProvider, sqlProvider); this.dbDriver = JDBCFileUtils.getDBFileDriver(connectionProvider, sqlProvider);
@ -68,6 +78,11 @@ public class JDBCSequentialFileFactory implements SequentialFileFactory, ActiveM
} }
} }
@Override
public boolean supportsIndividualContext() {
return true;
}
public JDBCSequentialFileFactoryDriver getDbDriver() { public JDBCSequentialFileFactoryDriver getDbDriver() {
return dbDriver; return dbDriver;
} }
@ -114,7 +129,7 @@ public class JDBCSequentialFileFactory implements SequentialFileFactory, ActiveM
public SequentialFile createSequentialFile(String fileName) { public SequentialFile createSequentialFile(String fileName) {
try { try {
fileLocks.putIfAbsent(fileName, new Object()); fileLocks.putIfAbsent(fileName, new Object());
JDBCSequentialFile file = new JDBCSequentialFile(this, fileName, executor, dbDriver, fileLocks.get(fileName)); JDBCSequentialFile file = new JDBCSequentialFile(this, fileName, executor, scheduledExecutorService, syncDelay, dbDriver, fileLocks.get(fileName));
files.add(file); files.add(file);
return file; return file;
} catch (Exception e) { } catch (Exception e) {
@ -223,6 +238,13 @@ public class JDBCSequentialFileFactory implements SequentialFileFactory, ActiveM
return null; return null;
} }
@Override
public String getDirectoryName() {
return dbDriver.getSqlProvider().getTableName();
}
@Override @Override
public boolean isStarted() { public boolean isStarted() {
return started; return started;

View File

@ -251,10 +251,6 @@ public class JDBCSequentialFileFactoryDriver extends AbstractJDBCDriver {
} }
} }
public int writeToFile(JDBCSequentialFile file, byte[] data) throws SQLException {
return writeToFile(file, data, true);
}
/** /**
* Persists data to this files associated database mapping. * Persists data to this files associated database mapping.
* *

View File

@ -0,0 +1,82 @@
/*
* 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.jdbc.store.file;
import java.nio.ByteBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.core.io.IOCallback;
public class ScheduledWrite {
ActiveMQBuffer amqBuffer;
ByteBuffer ioBuffer;
final IOCallback callback;
final boolean append;
public ScheduledWrite(ActiveMQBuffer amqBuffer, IOCallback callback, boolean append) {
this.amqBuffer = amqBuffer;
this.ioBuffer = null;
this.callback = callback;
this.append = append;
}
public ScheduledWrite(ByteBuffer ioBuffer, IOCallback callback, boolean append) {
this.ioBuffer = ioBuffer;
this.amqBuffer = null;
this.callback = callback;
this.append = append;
}
// for a scheduled Callback without a write
public ScheduledWrite(IOCallback callback) {
this.ioBuffer = null;
this.amqBuffer = null;
this.callback = callback;
this.append = false;
}
public int readable() {
if (ioBuffer != null) {
return ioBuffer.remaining();
} else if (amqBuffer != null) {
return amqBuffer.readableBytes();
} else {
return 0;
}
}
public int readAt(byte[] dst, int offset) {
if (ioBuffer != null) {
int size = ioBuffer.remaining();
ioBuffer.get(dst, offset, size);
return size;
} else if (amqBuffer != null) {
int size = amqBuffer.readableBytes();
amqBuffer.getBytes(0, dst, offset, size);
return size;
} else {
return 0;
}
}
/** Remove references letting buffer to be ready for GC */
public void releaseBuffer() {
amqBuffer = null;
ioBuffer = null;
}
}

View File

@ -31,6 +31,7 @@ import java.util.UUID;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration; import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
@ -71,6 +72,8 @@ public class JDBCSequentialFileFactoryTest {
private ExecutorService executor; private ExecutorService executor;
private ScheduledExecutorService scheduledExecutorService;
@Parameterized.Parameter @Parameterized.Parameter
public boolean useAuthentication; public boolean useAuthentication;
private String user = null; private String user = null;
@ -84,6 +87,7 @@ public class JDBCSequentialFileFactoryTest {
@Before @Before
public void setup() throws Exception { public void setup() throws Exception {
executor = Executors.newSingleThreadExecutor(ActiveMQThreadFactory.defaultThreadFactory(getClass().getName())); executor = Executors.newSingleThreadExecutor(ActiveMQThreadFactory.defaultThreadFactory(getClass().getName()));
scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(ActiveMQThreadFactory.defaultThreadFactory(getClass().getName()));
Map<String, Object> dataSourceProperties = new HashMap<>(); Map<String, Object> dataSourceProperties = new HashMap<>();
if (useAuthentication) { if (useAuthentication) {
user = "testuser"; user = "testuser";
@ -97,7 +101,7 @@ public class JDBCSequentialFileFactoryTest {
dataSourceProperties.put("driverClassName", className); dataSourceProperties.put("driverClassName", className);
String tableName = "FILES"; String tableName = "FILES";
String jdbcDatasourceClass = ActiveMQDefaultConfiguration.getDefaultDataSourceClassName(); String jdbcDatasourceClass = ActiveMQDefaultConfiguration.getDefaultDataSourceClassName();
factory = new JDBCSequentialFileFactory(new JDBCConnectionProvider(JDBCDataSourceUtils.getDataSource(jdbcDatasourceClass, dataSourceProperties)), JDBCUtils.getSQLProvider(dataSourceProperties, tableName, SQLProvider.DatabaseStoreType.PAGE), executor, new IOCriticalErrorListener() { factory = new JDBCSequentialFileFactory(new JDBCConnectionProvider(JDBCDataSourceUtils.getDataSource(jdbcDatasourceClass, dataSourceProperties)), JDBCUtils.getSQLProvider(dataSourceProperties, tableName, SQLProvider.DatabaseStoreType.PAGE), executor, scheduledExecutorService, 100, new IOCriticalErrorListener() {
@Override @Override
public void onIOException(Throwable code, String message, String file) { public void onIOException(Throwable code, String message, String file) {
} }
@ -108,6 +112,7 @@ public class JDBCSequentialFileFactoryTest {
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
executor.shutdown(); executor.shutdown();
scheduledExecutorService.shutdown();
factory.destroy(); factory.destroy();
} }
@ -231,7 +236,7 @@ public class JDBCSequentialFileFactoryTest {
} }
IOCallbackCountdown callback = new IOCallbackCountdown(1); IOCallbackCountdown callback = new IOCallbackCountdown(1);
file.internalWrite(src, callback); file.jdbcWrite(src, callback);
callback.assertEmpty(5); callback.assertEmpty(5);
checkData(file, src); checkData(file, src);
@ -245,7 +250,7 @@ public class JDBCSequentialFileFactoryTest {
ActiveMQBuffer src = ActiveMQBuffers.fixedBuffer(1); ActiveMQBuffer src = ActiveMQBuffers.fixedBuffer(1);
src.writeByte((byte)7); src.writeByte((byte)7);
file.internalWrite(src, null); file.jdbcWrite(src, null);
checkData(file, src); checkData(file, src);
assertEquals(1, file.size()); assertEquals(1, file.size());
file.close(); file.close();
@ -258,7 +263,7 @@ public class JDBCSequentialFileFactoryTest {
for (int i = 0; i < bufferSize; i++) { for (int i = 0; i < bufferSize; i++) {
src.writeByte((byte)i); src.writeByte((byte)i);
} }
file.internalWrite(src, null, false); file.jdbcWrite(src, null, false);
checkData(file, src); checkData(file, src);
assertEquals(bufferSize, file.size()); assertEquals(bufferSize, file.size());
} }
@ -276,7 +281,7 @@ public class JDBCSequentialFileFactoryTest {
} }
IOCallbackCountdown callback = new IOCallbackCountdown(1); IOCallbackCountdown callback = new IOCallbackCountdown(1);
file.internalWrite(src, callback); file.jdbcWrite(src, callback);
JDBCSequentialFile copy = (JDBCSequentialFile) factory.createSequentialFile("copy.txt"); JDBCSequentialFile copy = (JDBCSequentialFile) factory.createSequentialFile("copy.txt");
file.copyTo(copy); file.copyTo(copy);
@ -301,7 +306,7 @@ public class JDBCSequentialFileFactoryTest {
} }
IOCallbackCountdown callback = new IOCallbackCountdown(1); IOCallbackCountdown callback = new IOCallbackCountdown(1);
file.internalWrite(src, callback); file.jdbcWrite(src, callback);
assertEquals(bufferSize, file.size()); assertEquals(bufferSize, file.size());
JDBCSequentialFile copy = (JDBCSequentialFile) file.cloneFile(); JDBCSequentialFile copy = (JDBCSequentialFile) file.cloneFile();

View File

@ -122,6 +122,10 @@ public interface SequentialFileFactory {
File getDirectory(); File getDirectory();
default String getDirectoryName() {
return getDirectory().getName();
}
void clearBuffer(ByteBuffer buffer); void clearBuffer(ByteBuffer buffer);
void start(); void start();
@ -140,4 +144,11 @@ public interface SequentialFileFactory {
boolean isDatasync(); boolean isDatasync();
long getBufferSize(); long getBufferSize();
/** Only JDBC supports individual context.
* Meaning for Files we need to use the Sync scheduler.
* for JDBC we need to use a callback from the JDBC completion thread to complete the IOContexts. */
default boolean supportsIndividualContext() {
return false;
}
} }

View File

@ -22,6 +22,7 @@ import java.io.StringWriter;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.LinkOption; import java.nio.file.LinkOption;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
@ -64,6 +65,11 @@ public abstract class ArtemisAbstractPlugin extends AbstractMojo {
@Parameter(defaultValue = "${localRepository}") @Parameter(defaultValue = "${localRepository}")
protected ArtifactRepository localRepository; protected ArtifactRepository localRepository;
@Parameter
protected String[] extraRepositories;
List<RemoteRepository> inUseRepositories;
@Override @Override
public void execute() throws MojoExecutionException, MojoFailureException { public void execute() throws MojoExecutionException, MojoFailureException {
if (isIgnore()) { if (isIgnore()) {
@ -90,8 +96,6 @@ public abstract class ArtemisAbstractPlugin extends AbstractMojo {
protected abstract boolean isIgnore(); protected abstract boolean isIgnore();
protected abstract void doExecute() throws MojoExecutionException, MojoFailureException;
protected Artifact newArtifact(String artifactID) throws MojoFailureException { protected Artifact newArtifact(String artifactID) throws MojoFailureException {
Artifact artifact; Artifact artifact;
try { try {
@ -102,6 +106,26 @@ public abstract class ArtemisAbstractPlugin extends AbstractMojo {
return artifact; return artifact;
} }
protected void doExecute() throws MojoExecutionException, MojoFailureException {
int repositories = 0;
if (extraRepositories != null) {
inUseRepositories = new ArrayList<>();
for (String strRepo: extraRepositories) {
RemoteRepository repo = new RemoteRepository.Builder("repo" + (repositories++), "default", strRepo).build();
inUseRepositories.add(repo);
remoteRepos.add(repo);
}
}
}
protected void done() {
if (inUseRepositories != null) {
inUseRepositories.forEach(r -> remoteRepos.remove(r));
inUseRepositories.clear(); // give a help to GC
}
inUseRepositories = null;
}
protected File resolveArtifactFile(Artifact artifact) throws MojoExecutionException, DependencyCollectionException { protected File resolveArtifactFile(Artifact artifact) throws MojoExecutionException, DependencyCollectionException {
ArtifactResult result = resolveArtifact(artifact); ArtifactResult result = resolveArtifact(artifact);

View File

@ -141,6 +141,10 @@ public class ArtemisCreatePlugin extends ArtemisAbstractPlugin {
@Parameter @Parameter
private String[] webListWithDeps; private String[] webListWithDeps;
/** Folders with libs to be copied into target */
@Parameter()
private String[] libFolders;
@Parameter(defaultValue = "${localRepository}") @Parameter(defaultValue = "${localRepository}")
private org.apache.maven.artifact.repository.ArtifactRepository localRepository; private org.apache.maven.artifact.repository.ArtifactRepository localRepository;
@ -301,6 +305,17 @@ public class ArtemisCreatePlugin extends ArtemisAbstractPlugin {
FileUtil.makeExec(commandLine); FileUtil.makeExec(commandLine);
if (libFolders != null) {
for (String libFolder : libFolders) {
File folder = new File(libFolder);
for (File file : folder.listFiles()) {
if (!file.isDirectory()) {
copyToDir("lib", file, commandLineStream);
}
}
}
}
if (getLog().isDebugEnabled()) { if (getLog().isDebugEnabled()) {
getLog().debug("###################################################################################################"); getLog().debug("###################################################################################################");
getLog().debug(commandLine.getName() + " created with commands to reproduce " + instance.getName()); getLog().debug(commandLine.getName() + " created with commands to reproduce " + instance.getName());

View File

@ -22,8 +22,6 @@ import java.io.FileOutputStream;
import java.io.PrintStream; import java.io.PrintStream;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.StandardCopyOption; import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -34,7 +32,6 @@ import org.apache.maven.plugins.annotations.LifecyclePhase;
import org.apache.maven.plugins.annotations.Mojo; import org.apache.maven.plugins.annotations.Mojo;
import org.apache.maven.plugins.annotations.Parameter; import org.apache.maven.plugins.annotations.Parameter;
import org.apache.maven.project.MavenProject; import org.apache.maven.project.MavenProject;
import org.eclipse.aether.repository.RemoteRepository;
@Mojo(name = "dependency-scan", defaultPhase = LifecyclePhase.VERIFY) @Mojo(name = "dependency-scan", defaultPhase = LifecyclePhase.VERIFY)
public class ArtemisDependencyScanPlugin extends ArtemisAbstractPlugin { public class ArtemisDependencyScanPlugin extends ArtemisAbstractPlugin {
@ -53,9 +50,6 @@ public class ArtemisDependencyScanPlugin extends ArtemisAbstractPlugin {
@Parameter @Parameter
private String[] libList; private String[] libList;
@Parameter
private String[] extraRepositories;
@Parameter @Parameter
private String variableName; private String variableName;
@ -82,15 +76,8 @@ public class ArtemisDependencyScanPlugin extends ArtemisAbstractPlugin {
@Override @Override
protected void doExecute() throws MojoExecutionException, MojoFailureException { protected void doExecute() throws MojoExecutionException, MojoFailureException {
int repositories = 0; super.doExecute();
List<RemoteRepository> listRepo = new ArrayList<>();
if (extraRepositories != null) {
for (String strRepo: extraRepositories) {
RemoteRepository repo = new RemoteRepository.Builder("repo" + (repositories++), "default", strRepo).build();
listRepo.add(repo);
remoteRepos.add(repo);
}
}
getLog().debug("Local " + localRepository); getLog().debug("Local " + localRepository);
project = (MavenProject) getPluginContext().get("project"); project = (MavenProject) getPluginContext().get("project");
@ -160,9 +147,7 @@ public class ArtemisDependencyScanPlugin extends ArtemisAbstractPlugin {
throw new MojoFailureException(e.getMessage()); throw new MojoFailureException(e.getMessage());
} }
} finally { } finally {
for (RemoteRepository repository : listRepo) { done();
remoteRepos.remove(repository);
}
} }

View File

@ -26,4 +26,8 @@ public interface StoreConfiguration extends Serializable {
} }
StoreType getStoreType(); StoreType getStoreType();
default int getAllowedPageSize(int pageSize) {
return pageSize;
}
} }

View File

@ -69,6 +69,8 @@ public class DatabaseStorageConfiguration implements StoreConfiguration {
private long jdbcAllowedTimeDiff = ActiveMQDefaultConfiguration.getDefaultJdbcAllowedTimeDiffMillis(); private long jdbcAllowedTimeDiff = ActiveMQDefaultConfiguration.getDefaultJdbcAllowedTimeDiffMillis();
private int maxPageSizeBytes = ActiveMQDefaultConfiguration.getDefaultJdbcMaxPageSizeBytes();
@Override @Override
public StoreType getStoreType() { public StoreType getStoreType() {
return StoreType.DATABASE; return StoreType.DATABASE;
@ -150,6 +152,15 @@ public class DatabaseStorageConfiguration implements StoreConfiguration {
return jdbcAllowedTimeDiff; return jdbcAllowedTimeDiff;
} }
public int getMaxPageSizeBytes() {
return maxPageSizeBytes;
}
public DatabaseStorageConfiguration setMaxPageSizeBytes(int maxPageSizeBytes) {
this.maxPageSizeBytes = maxPageSizeBytes;
return this;
}
/** /**
* The DataSource to use to store Artemis data in the data store (can be {@code null} if {@code jdbcConnectionUrl} and {@code jdbcDriverClassName} are used instead). * The DataSource to use to store Artemis data in the data store (can be {@code null} if {@code jdbcConnectionUrl} and {@code jdbcDriverClassName} are used instead).
* *
@ -306,4 +317,10 @@ public class DatabaseStorageConfiguration implements StoreConfiguration {
public void setJdbcAllowedTimeDiff(long jdbcAllowedTimeDiff) { public void setJdbcAllowedTimeDiff(long jdbcAllowedTimeDiff) {
this.jdbcAllowedTimeDiff = jdbcAllowedTimeDiff; this.jdbcAllowedTimeDiff = jdbcAllowedTimeDiff;
} }
@Override
public int getAllowedPageSize(int pageSize) {
return Math.min(pageSize, maxPageSizeBytes);
}
} }

View File

@ -2003,6 +2003,7 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
conf.setJdbcLockExpirationMillis(getLong(storeNode, "jdbc-lock-expiration", conf.getJdbcLockExpirationMillis(), Validators.NO_CHECK)); conf.setJdbcLockExpirationMillis(getLong(storeNode, "jdbc-lock-expiration", conf.getJdbcLockExpirationMillis(), Validators.NO_CHECK));
conf.setJdbcJournalSyncPeriodMillis(getLong(storeNode, "jdbc-journal-sync-period", conf.getJdbcJournalSyncPeriodMillis(), Validators.NO_CHECK)); conf.setJdbcJournalSyncPeriodMillis(getLong(storeNode, "jdbc-journal-sync-period", conf.getJdbcJournalSyncPeriodMillis(), Validators.NO_CHECK));
conf.setJdbcAllowedTimeDiff(getLong(storeNode, "jdbc-allowed-time-diff", conf.getJdbcAllowedTimeDiff(), Validators.NO_CHECK)); conf.setJdbcAllowedTimeDiff(getLong(storeNode, "jdbc-allowed-time-diff", conf.getJdbcAllowedTimeDiff(), Validators.NO_CHECK));
conf.setMaxPageSizeBytes(getTextBytesAsIntBytes(storeNode, "jdbc-max-page-size-bytes", conf.getMaxPageSizeBytes(), Validators.NO_CHECK));
String jdbcUser = getString(storeNode, "jdbc-user", conf.getJdbcUser(), Validators.NO_CHECK); String jdbcUser = getString(storeNode, "jdbc-user", conf.getJdbcUser(), Validators.NO_CHECK);
if (jdbcUser != null) { if (jdbcUser != null) {
jdbcUser = PasswordMaskingUtil.resolveMask(mainConfig.isMaskPassword(), jdbcUser, mainConfig.getPasswordCodec()); jdbcUser = PasswordMaskingUtil.resolveMask(mainConfig.isMaskPassword(), jdbcUser, mainConfig.getPasswordCodec());

View File

@ -25,6 +25,7 @@ import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider; import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription; import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.paging.impl.Page; import org.apache.activemq.artemis.core.paging.impl.Page;
import org.apache.activemq.artemis.core.persistence.OperationContext;
import org.apache.activemq.artemis.core.persistence.StorageManager; import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.replication.ReplicationManager; import org.apache.activemq.artemis.core.replication.ReplicationManager;
import org.apache.activemq.artemis.core.server.ActiveMQComponent; import org.apache.activemq.artemis.core.server.ActiveMQComponent;
@ -60,6 +61,10 @@ public interface PagingStore extends ActiveMQComponent, RefCountMessageListener
File getFolder(); File getFolder();
default String getFolderName() {
return getFolder().getName();
}
AddressFullMessagePolicy getAddressFullMessagePolicy(); AddressFullMessagePolicy getAddressFullMessagePolicy();
PageFullMessagePolicy getPageFullMessagePolicy(); PageFullMessagePolicy getPageFullMessagePolicy();
@ -99,7 +104,7 @@ public interface PagingStore extends ActiveMQComponent, RefCountMessageListener
/** /**
* Schedules sync to the file storage. * Schedules sync to the file storage.
*/ */
void sync() throws Exception; void addSyncPoint(OperationContext context) throws Exception;
/** /**
* Performs a real sync on the current IO file. * Performs a real sync on the current IO file.

View File

@ -75,7 +75,7 @@ public class PageCounterRebuildManager implements Runnable {
try { try {
paging = store.isPaging(); paging = store.isPaging();
if (!paging) { if (!paging) {
logger.debug("Destination {} was not paging, no need to rebuild counters"); logger.trace("Destination {} was not paging, no need to rebuild counters");
store.getCursorProvider().forEachSubscription(subscription -> { store.getCursorProvider().forEachSubscription(subscription -> {
subscription.getCounter().markRebuilding(); subscription.getCounter().markRebuilding();
subscription.getCounter().finishRebuild(); subscription.getCounter().finishRebuild();
@ -89,7 +89,7 @@ public class PageCounterRebuildManager implements Runnable {
limitPageId = store.getCurrentWritingPage(); limitPageId = store.getCurrentWritingPage();
limitMessageNr = currentPage.getNumberOfMessages(); limitMessageNr = currentPage.getNumberOfMessages();
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("PageCounterRebuild for {}, Current writing page {} and limit will be {} with lastMessage on last page={}", store.getStoreName(), store.getCurrentWritingPage(), limitPageId, limitMessageNr); logger.trace("PageCounterRebuild for {}, Current writing page {} and limit will be {} with lastMessage on last page={}", store.getStoreName(), store.getCurrentWritingPage(), limitPageId, limitMessageNr);
} }
} catch (Exception e) { } catch (Exception e) {
logger.warn(e.getMessage(), e); logger.warn(e.getMessage(), e);
@ -196,25 +196,26 @@ public class PageCounterRebuildManager implements Runnable {
public void rebuild() throws Exception { public void rebuild() throws Exception {
if (pgStore == null) { if (pgStore == null) {
logger.debug("Page store is null during rebuildCounters"); logger.trace("Page store is null during rebuildCounters");
return; return;
} }
if (!paging) { if (!paging) {
logger.debug("Ignoring call to rebuild pgStore {}", pgStore.getAddress()); logger.trace("Ignoring call to rebuild pgStore {}", pgStore.getAddress());
} }
logger.debug("Rebuilding counter for store {}", pgStore.getAddress()); logger.debug("Rebuilding page counter for address {}", pgStore.getAddress());
for (long pgid = pgStore.getFirstPage(); pgid <= limitPageId; pgid++) { for (long pgid = pgStore.getFirstPage(); pgid <= limitPageId; pgid++) {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("Rebuilding counter on messages from page {} on rebuildCounters for address {}", pgid, pgStore.getAddress()); logger.trace("Rebuilding counter on messages from page {} on rebuildCounters for address {}", pgid, pgStore.getAddress());
} }
logger.debug("{} reading paging {} of {}", pgStore.getAddress(), pgid, limitPageId);
Page page = pgStore.newPageObject(pgid); Page page = pgStore.newPageObject(pgid);
if (!page.getFile().exists()) { if (!page.getFile().exists()) {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("Skipping page {} on store {}", pgid, pgStore.getAddress()); logger.trace("Skipping page {} on store {}", pgid, pgStore.getAddress());
} }
continue; continue;
} }
@ -227,14 +228,14 @@ public class PageCounterRebuildManager implements Runnable {
PagedMessage msg = iter.next(); PagedMessage msg = iter.next();
if (storedLargeMessages != null && msg.getMessage().isLargeMessage()) { if (storedLargeMessages != null && msg.getMessage().isLargeMessage()) {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("removing storedLargeMessage {}", msg.getMessage().getMessageID()); logger.trace("removing storedLargeMessage {}", msg.getMessage().getMessageID());
} }
storedLargeMessages.remove(msg.getMessage().getMessageID()); storedLargeMessages.remove(msg.getMessage().getMessageID());
} }
if (limitPageId == pgid) { if (limitPageId == pgid) {
if (msg.getMessageNumber() >= limitMessageNr) { if (msg.getMessageNumber() >= limitMessageNr) {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("Rebuild counting on {} reached the last message at {}-{}", pgStore.getAddress(), limitPageId, limitMessageNr); logger.trace("Rebuild counting on {} reached the last message at {}-{}", pgStore.getAddress(), limitPageId, limitMessageNr);
} }
// this is the limit where we should count.. // this is the limit where we should count..
// anything beyond this will be new data // anything beyond this will be new data
@ -286,7 +287,7 @@ public class PageCounterRebuildManager implements Runnable {
boolean txIncluded = msg.getTransactionID() <= 0 || transactions == null || txInfo != null; boolean txIncluded = msg.getTransactionID() <= 0 || transactions == null || txInfo != null;
if (!txIncluded) { if (!txIncluded) {
logger.debug("TX is not included for {}", msg); logger.trace("TX is not included for {}", msg);
} }
if (ok && txIncluded) { // not acked and TX is ok if (ok && txIncluded) { // not acked and TX is ok

View File

@ -27,7 +27,9 @@ import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
import org.apache.activemq.artemis.core.io.SequentialFile; import org.apache.activemq.artemis.core.io.SequentialFile;
import org.apache.activemq.artemis.core.io.SequentialFileFactory; import org.apache.activemq.artemis.core.io.SequentialFileFactory;
import org.apache.activemq.artemis.core.paging.PagedMessage; import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.persistence.OperationContext;
import org.apache.activemq.artemis.core.persistence.StorageManager; import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.persistence.impl.journal.OperationContextImpl;
import org.apache.activemq.artemis.core.server.LargeServerMessage; import org.apache.activemq.artemis.core.server.LargeServerMessage;
import org.apache.activemq.artemis.utils.DataConstants; import org.apache.activemq.artemis.utils.DataConstants;
import org.apache.activemq.artemis.utils.Env; import org.apache.activemq.artemis.utils.Env;
@ -83,7 +85,15 @@ public class PageReadWriter {
assert (activeMQBuffer.readableBytes() == bufferSize) : "messageEncodedSize is different from expected"; assert (activeMQBuffer.readableBytes() == bufferSize) : "messageEncodedSize is different from expected";
//buffer limit and position are the same //buffer limit and position are the same
assert (buffer.remaining() == bufferSize) : "buffer position or limit are changed"; assert (buffer.remaining() == bufferSize) : "buffer position or limit are changed";
if (fileFactory.supportsIndividualContext()) {
OperationContext context = OperationContextImpl.getContext();
if (context != null) {
context.storeLineUp();
}
file.writeDirect(buffer, false, context);
} else {
file.writeDirect(buffer, false); file.writeDirect(buffer, false);
}
return bufferSize; return bufferSize;
} }

View File

@ -124,7 +124,7 @@ public class PagingStoreFactoryDatabase implements PagingStoreFactory {
if (sqlProviderFactory == null) { if (sqlProviderFactory == null) {
sqlProviderFactory = new PropertySQLProvider.Factory(dbConf.getConnectionProvider()); sqlProviderFactory = new PropertySQLProvider.Factory(dbConf.getConnectionProvider());
} }
pagingFactoryFileFactory = new JDBCSequentialFileFactory(dbConf.getConnectionProvider(), sqlProviderFactory.create(pageStoreTableNamePrefix, SQLProvider.DatabaseStoreType.PAGE), executorFactory.getExecutor(), criticalErrorListener); pagingFactoryFileFactory = new JDBCSequentialFileFactory(dbConf.getConnectionProvider(), sqlProviderFactory.create(pageStoreTableNamePrefix, SQLProvider.DatabaseStoreType.PAGE), executorFactory.getExecutor(), scheduledExecutor, dbConf.getJdbcJournalSyncPeriodMillis(), criticalErrorListener);
pagingFactoryFileFactory.start(); pagingFactoryFileFactory.start();
started = true; started = true;
} }
@ -193,7 +193,7 @@ public class PagingStoreFactoryDatabase implements PagingStoreFactory {
} }
} }
directoryList.write(writeBuffer, true, null, false); directoryList.sendToDB(writeBuffer, null, false);
directoryList.close(); directoryList.close();
} }
} }
@ -258,7 +258,7 @@ public class PagingStoreFactoryDatabase implements PagingStoreFactory {
sqlProviderFactory = new PropertySQLProvider.Factory(dbConf.getConnectionProvider()); sqlProviderFactory = new PropertySQLProvider.Factory(dbConf.getConnectionProvider());
} }
sqlProvider = sqlProviderFactory.create(getTableNameForGUID(directoryName), SQLProvider.DatabaseStoreType.PAGE); sqlProvider = sqlProviderFactory.create(getTableNameForGUID(directoryName), SQLProvider.DatabaseStoreType.PAGE);
final JDBCSequentialFileFactory fileFactory = new JDBCSequentialFileFactory(dbConf.getConnectionProvider(), sqlProvider, executorFactory.getExecutor(), criticalErrorListener); final JDBCSequentialFileFactory fileFactory = new JDBCSequentialFileFactory(dbConf.getConnectionProvider(), sqlProvider, executorFactory.getExecutor(), scheduledExecutor, dbConf.getJdbcJournalSyncPeriodMillis(), criticalErrorListener);
factoryToTableName.put(fileFactory, directoryName); factoryToTableName.put(fileFactory, directoryName);
return fileFactory; return fileFactory;
} }

View File

@ -43,6 +43,7 @@ import org.apache.activemq.artemis.core.paging.PagingStore;
import org.apache.activemq.artemis.core.paging.PagingStoreFactory; import org.apache.activemq.artemis.core.paging.PagingStoreFactory;
import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider; import org.apache.activemq.artemis.core.paging.cursor.PageCursorProvider;
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription; import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
import org.apache.activemq.artemis.core.persistence.OperationContext;
import org.apache.activemq.artemis.core.persistence.StorageManager; import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.replication.ReplicationManager; import org.apache.activemq.artemis.core.replication.ReplicationManager;
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle; import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
@ -232,7 +233,9 @@ public class PagingStoreImpl implements PagingStore {
configureSizeMetric(); configureSizeMetric();
pageSize = addressSettings.getPageSizeBytes(); // JDBC has a maximum page size of 100K by default.
// it can be reconfigured through jdbc-max-page-size-bytes in the JDBC configuration section
pageSize = storageManager.getAllowedPageSize(addressSettings.getPageSizeBytes());
addressFullMessagePolicy = addressSettings.getAddressFullMessagePolicy(); addressFullMessagePolicy = addressSettings.getAddressFullMessagePolicy();
@ -430,6 +433,11 @@ public class PagingStoreImpl implements PagingStore {
} }
} }
@Override
public String getFolderName() {
return fileFactory.getDirectoryName();
}
@Override @Override
public boolean isPaging() { public boolean isPaging() {
AddressFullMessagePolicy policy = this.addressFullMessagePolicy; AddressFullMessagePolicy policy = this.addressFullMessagePolicy;
@ -461,17 +469,19 @@ public class PagingStoreImpl implements PagingStore {
} }
@Override @Override
public void sync() throws Exception { public void addSyncPoint(OperationContext context) throws Exception {
if (fileFactory == null || !fileFactory.supportsIndividualContext()) {
if (syncTimer != null) { if (syncTimer != null) {
syncTimer.addSync(storageManager.getContext()); syncTimer.addSync(context);
} else { } else {
ioSync(); ioSync();
} }
}
} }
@Override @Override
public void ioSync() throws Exception { public void ioSync() throws Exception {
if (!fileFactory.supportsIndividualContext()) {
lock.readLock().lock(); lock.readLock().lock();
try { try {
@ -483,6 +493,7 @@ public class PagingStoreImpl implements PagingStore {
lock.readLock().unlock(); lock.readLock().unlock();
} }
} }
}
@Override @Override
public void processReload() throws Exception { public void processReload() throws Exception {
@ -1214,7 +1225,7 @@ public class PagingStoreImpl implements PagingStore {
page.write(pagedMessage); page.write(pagedMessage);
if (tx == null && syncNonTransactional && message.isDurable()) { if (tx == null && syncNonTransactional && message.isDurable()) {
sync(); addSyncPoint(storageManager.getContext());
} }
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
@ -1384,7 +1395,7 @@ public class PagingStoreImpl implements PagingStore {
*/ */
private void syncStore() throws Exception { private void syncStore() throws Exception {
for (PagingStore store : usedStores) { for (PagingStore store : usedStores) {
store.sync(); store.addSyncPoint(storageManager.getContext());
} }
} }

View File

@ -499,4 +499,8 @@ public interface StorageManager extends IDGenerator, ActiveMQComponent {
void persistIdGenerator(); void persistIdGenerator();
void injectMonitor(FileStoreMonitor monitor) throws Exception; void injectMonitor(FileStoreMonitor monitor) throws Exception;
default int getAllowedPageSize(int pageSize) {
return pageSize;
}
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.activemq.artemis.core.persistence.impl.journal; package org.apache.activemq.artemis.core.persistence.impl.journal;
import java.lang.invoke.MethodHandles;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
@ -32,9 +33,13 @@ import org.apache.activemq.artemis.jdbc.store.sql.PropertySQLProvider;
import org.apache.activemq.artemis.jdbc.store.sql.SQLProvider; import org.apache.activemq.artemis.jdbc.store.sql.SQLProvider;
import org.apache.activemq.artemis.utils.ExecutorFactory; import org.apache.activemq.artemis.utils.ExecutorFactory;
import org.apache.activemq.artemis.utils.critical.CriticalAnalyzer; import org.apache.activemq.artemis.utils.critical.CriticalAnalyzer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class JDBCJournalStorageManager extends JournalStorageManager { public class JDBCJournalStorageManager extends JournalStorageManager {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public JDBCJournalStorageManager(Configuration config, public JDBCJournalStorageManager(Configuration config,
CriticalAnalyzer analyzer, CriticalAnalyzer analyzer,
ExecutorFactory executorFactory, ExecutorFactory executorFactory,
@ -80,12 +85,15 @@ public class JDBCJournalStorageManager extends JournalStorageManager {
connectionProvider, connectionProvider,
sqlProviderFactory.create(dbConf.getLargeMessageTableName(), SQLProvider.DatabaseStoreType.LARGE_MESSAGE), sqlProviderFactory.create(dbConf.getLargeMessageTableName(), SQLProvider.DatabaseStoreType.LARGE_MESSAGE),
executorFactory.getExecutor(), executorFactory.getExecutor(),
scheduledExecutorService,
dbConf.getJdbcJournalSyncPeriodMillis(),
criticalErrorListener); criticalErrorListener);
this.bindingsJournal = bindingsJournal; this.bindingsJournal = bindingsJournal;
this.messageJournal = messageJournal; this.messageJournal = messageJournal;
this.largeMessagesFactory = largeMessagesFactory; this.largeMessagesFactory = largeMessagesFactory;
largeMessagesFactory.start(); largeMessagesFactory.start();
} catch (Exception e) { } catch (Exception e) {
logger.warn(e.getMessage(), e);
criticalErrorListener.onIOException(e, e.getMessage(), null); criticalErrorListener.onIOException(e, e.getMessage(), null);
} }
} }

View File

@ -895,4 +895,13 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
monitor.addStore(bindingsFF.getDirectory()); monitor.addStore(bindingsFF.getDirectory());
} }
} }
@Override
public int getAllowedPageSize(int pageSize) {
if (config.getStoreConfiguration() == null) {
return pageSize;
}
return config.getStoreConfiguration().getAllowedPageSize(pageSize);
}
} }

View File

@ -554,6 +554,7 @@ public class ActiveMQServerImpl implements ActiveMQServer {
private void configureJdbcNetworkTimeout() { private void configureJdbcNetworkTimeout() {
if (configuration.isPersistenceEnabled()) { if (configuration.isPersistenceEnabled()) {
if (configuration.getStoreConfiguration() != null && configuration.getStoreConfiguration().getStoreType() == StoreConfiguration.StoreType.DATABASE) { if (configuration.getStoreConfiguration() != null && configuration.getStoreConfiguration().getStoreType() == StoreConfiguration.StoreType.DATABASE) {
configuration.setMaxDiskUsage(-1); // it does not make sense with JDBC
DatabaseStorageConfiguration databaseStorageConfiguration = (DatabaseStorageConfiguration) configuration.getStoreConfiguration(); DatabaseStorageConfiguration databaseStorageConfiguration = (DatabaseStorageConfiguration) configuration.getStoreConfiguration();
databaseStorageConfiguration.setConnectionProviderNetworkTimeout(threadPool, databaseStorageConfiguration.getJdbcNetworkTimeout()); databaseStorageConfiguration.setConnectionProviderNetworkTimeout(threadPool, databaseStorageConfiguration.getJdbcNetworkTimeout());
} }

View File

@ -2786,6 +2786,14 @@
</xsd:documentation> </xsd:documentation>
</xsd:annotation> </xsd:annotation>
</xsd:element> </xsd:element>
<xsd:element name="jdbc-max-page-size-bytes" type="xsd:string" default="102400" maxOccurs="1" minOccurs="0">
<xsd:annotation>
<xsd:documentation>
The max page size (in bytes) to use for all addresses when using JDBC.
Supports byte notation like "K", "Mb", "MiB", "GB", etc.
</xsd:documentation>
</xsd:annotation>
</xsd:element>
</xsd:all> </xsd:all>
<xsd:attributeGroup ref="xml:specialAttrs"/> <xsd:attributeGroup ref="xml:specialAttrs"/>
</xsd:complexType> </xsd:complexType>

View File

@ -332,7 +332,7 @@ public abstract class ActiveMQTestBase extends Assert {
private static final String EXPECTED_DERBY_SHUTDOWN_STATE = "XJ015"; private static final String EXPECTED_DERBY_SHUTDOWN_STATE = "XJ015";
/** This method will be passed as a lambda into runAfter from createDefaultDatabaseStorageConfiguration */ /** This method will be passed as a lambda into runAfter from createDefaultDatabaseStorageConfiguration */
protected final void dropDerby() throws Exception { protected void dropDerby() throws Exception {
String user = getJDBCUser(); String user = getJDBCUser();
String password = getJDBCPassword(); String password = getJDBCPassword();
try { try {
@ -926,11 +926,11 @@ public abstract class ActiveMQTestBase extends Assert {
return "memory:" + getTestDir(); return "memory:" + getTestDir();
} }
protected final String getTestJDBCConnectionUrl() { protected String getTestJDBCConnectionUrl() {
return System.getProperty("jdbc.connection.url", "jdbc:derby:" + getEmbeddedDataBaseName() + ";create=true"); return System.getProperty("jdbc.connection.url", "jdbc:derby:" + getEmbeddedDataBaseName() + ";create=true");
} }
protected final String getJDBCClassName() { protected String getJDBCClassName() {
return System.getProperty("jdbc.driver.class", "org.apache.derby.jdbc.EmbeddedDriver"); return System.getProperty("jdbc.driver.class", "org.apache.derby.jdbc.EmbeddedDriver");
} }
@ -1213,15 +1213,11 @@ public abstract class ActiveMQTestBase extends Assert {
* @throws InterruptedException * @throws InterruptedException
*/ */
protected void waitForNotPaging(Queue queue) throws InterruptedException { protected void waitForNotPaging(Queue queue) throws InterruptedException {
waitForNotPaging(queue.getPageSubscription().getPagingStore()); waitForNotPaging(queue.getPagingStore());
} }
protected void waitForNotPaging(PagingStore store) throws InterruptedException { protected void waitForNotPaging(PagingStore store) throws InterruptedException {
long timeout = System.currentTimeMillis() + 20000; Wait.assertFalse("Store is still paging", store::isPaging, 20_000);
while (timeout > System.currentTimeMillis() && store.isPaging()) {
Thread.sleep(100);
}
assertFalse(store.isPaging());
} }
protected static Topology waitForTopology(final ActiveMQServer server, final int nodes) throws Exception { protected static Topology waitForTopology(final ActiveMQServer server, final int nodes) throws Exception {

View File

@ -123,6 +123,11 @@ This tells what to do if the system is reaching `page-limit-bytes` or `page-limi
| |
|=== |===
[NOTE]
When using the JDBC storage, the effective page-size-bytes used is limited to jdbc-max-page-size-bytes, configured in the JDBC storage section.
=== max-size-bytes and max-size-messages simultaneous usage === max-size-bytes and max-size-messages simultaneous usage
It is possible to define max-size-messages (as the maximum number of messages) and max-messages-size (as the max number of estimated memory used by the address) concurrently. It is possible to define max-size-messages (as the maximum number of messages) and max-messages-size (as the max number of estimated memory used by the address) concurrently.

View File

@ -343,11 +343,14 @@ Performance for both paging and large messages is especially diminished with JDB
The JDBC persistence layer is targeted to those users who _must_ use a database e.g. due to internal company policy. The JDBC persistence layer is targeted to those users who _must_ use a database e.g. due to internal company policy.
==== ====
ActiveMQ Artemis currently has support for a limited number of database vendors (older versions may work but mileage may vary): ActiveMQ Artemis currently has support for a limited number of database vendors:
. PostgreSQL 9.4.x . PostgreSQL
. MySQL 5.7.x . MySQL
. Apache Derby 10.11.1.1 . Microsoft SQL Server
. Oracle
. DB2
. Apache Derby
The JDBC store uses a JDBC connection to store messages and bindings data in records in database tables. The JDBC store uses a JDBC connection to store messages and bindings data in records in database tables.
The data stored in the database tables is encoded using Apache ActiveMQ Artemis internal encodings. The data stored in the database tables is encoded using Apache ActiveMQ Artemis internal encodings.
@ -428,6 +431,10 @@ The maximal time offset between the broker and the database in milliseconds when
Currently this value only affects the logging and will show a warning if the detected difference exceeds the limit. Currently this value only affects the logging and will show a warning if the detected difference exceeds the limit.
The default value is 250 milliseconds. The default value is 250 milliseconds.
jdbc-max-page-size-bytes::
The maximal size a page can use. The default and recommended maximum value is 100K bytes.
Using larger sizes will result in downloading large blobs that would affect performance when using paged messages.
NOTE: Some DBMS (e.g. Oracle, 30 chars) have restrictions on the size of table names, this should be taken into consideration when configuring table names for the Artemis database store, pay particular attention to the page store table name, which can be appended with a unique ID of up to 20 characters. NOTE: Some DBMS (e.g. Oracle, 30 chars) have restrictions on the size of table names, this should be taken into consideration when configuring table names for the Artemis database store, pay particular attention to the page store table name, which can be appended with a unique ID of up to 20 characters.
(for Oracle this would mean configuring a page-store-table-name of max size of 10 chars). (for Oracle this would mean configuring a page-store-table-name of max size of 10 chars).
@ -446,6 +453,7 @@ It is also possible to explicitly add the user and password rather than in the J
<page-store-table-name>MESSAGE_TABLE</page-store-table-name> <page-store-table-name>MESSAGE_TABLE</page-store-table-name>
<large-message-table-name>LARGE_MESSAGES_TABLE</large-message-table-name> <large-message-table-name>LARGE_MESSAGES_TABLE</large-message-table-name>
<node-manager-store-table-name>NODE_MANAGER_TABLE</node-manager-store-table-name> <node-manager-store-table-name>NODE_MANAGER_TABLE</node-manager-store-table-name>
<jdbc-page-max-size-bytes>100K</jdbc-page-max-size-bytes>
</database-store> </database-store>
</store> </store>
---- ----

View File

@ -1816,6 +1816,9 @@
<exclude>examples/protocols/amqp/dotnet/**/bin/**/*</exclude> <exclude>examples/protocols/amqp/dotnet/**/bin/**/*</exclude>
<exclude>examples/protocols/amqp/dotnet/**/readme.md</exclude> <exclude>examples/protocols/amqp/dotnet/**/readme.md</exclude>
<exclude>examples/protocols/amqp/**/readme.md</exclude> <exclude>examples/protocols/amqp/**/readme.md</exclude>
<!-- DB Test config files -->
<exclude>tests/db-tests/jdbc-drivers/**/*</exclude>
</excludes> </excludes>
</configuration> </configuration>
</plugin> </plugin>

View File

@ -43,6 +43,12 @@
<version>${project.version}</version> <version>${project.version}</version>
<scope>provided</scope> <scope>provided</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-cli</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency> <dependency>
<groupId>org.apache.activemq</groupId> <groupId>org.apache.activemq</groupId>
<artifactId>artemis-jms-client</artifactId> <artifactId>artemis-jms-client</artifactId>

View File

@ -0,0 +1,272 @@
/*
* 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.utils;
import javax.management.MBeanServerInvocationHandler;
import javax.management.ObjectName;
import javax.management.remote.JMXConnector;
import javax.management.remote.JMXConnectorFactory;
import javax.management.remote.JMXServiceURL;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.MalformedURLException;
import java.util.HashSet;
import java.util.Set;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.management.ActiveMQServerControl;
import org.apache.activemq.artemis.api.core.management.ObjectNameBuilder;
import org.apache.activemq.artemis.api.core.management.QueueControl;
import org.apache.activemq.artemis.api.jms.ActiveMQJMSClient;
import org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.util.ServerUtil;
import org.junit.After;
import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
// Base class for tests that will start a real server
public class RealServerTestBase extends ActiveMQTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String STOP_FILE_NAME = "STOP_ME";
Set<Process> processes = new HashSet<>();
private static final String JMX_SERVER_HOSTNAME = "localhost";
private static final int JMX_SERVER_PORT = 10099;
public static final String basedir = System.getProperty("basedir");
@After
public void after() throws Exception {
// close ServerLocators before killing the server otherwise they'll hang and delay test termination
closeAllServerLocatorsFactories();
for (Process process : processes) {
try {
ServerUtil.killServer(process, true);
} catch (Throwable e) {
e.printStackTrace();
}
}
processes.clear();
}
public void killServer(Process process) {
processes.remove(process);
try {
ServerUtil.killServer(process);
} catch (Throwable e) {
e.printStackTrace();
}
}
protected static void stopServerWithFile(String serverLocation) throws IOException {
File serverPlace = new File(serverLocation);
File etcPlace = new File(serverPlace, "etc");
File stopMe = new File(etcPlace, STOP_FILE_NAME);
Assert.assertTrue(stopMe.createNewFile());
}
public static String getServerLocation(String serverName) {
return basedir + "/target/" + serverName;
}
public static boolean cleanupData(String serverName) {
String location = getServerLocation(serverName);
boolean result = deleteDirectory(new File(location, "data"));
deleteDirectory(new File(location, "log"));
return result;
}
public void addProcess(Process process) {
processes.add(process);
}
public void removeProcess(Process process) {
processes.remove(process);
}
public Process startServer(String serverName, int portID, int timeout) throws Exception {
Process process = ServerUtil.startServer(getServerLocation(serverName), serverName, portID, timeout);
addProcess(process);
return process;
}
public Process startServer(String serverName, String uri, int timeout) throws Exception {
Process process = ServerUtil.startServer(getServerLocation(serverName), serverName, uri, timeout);
addProcess(process);
return process;
}
protected JMXConnector getJmxConnector() throws MalformedURLException {
return getJmxConnector(JMX_SERVER_HOSTNAME, JMX_SERVER_PORT);
}
protected static JMXConnector newJMXFactory(String uri) throws Throwable {
return JMXConnectorFactory.connect(new JMXServiceURL(uri));
}
protected static ActiveMQServerControl getServerControl(String uri,
ObjectNameBuilder builder,
long timeout) throws Throwable {
long expireLoop = System.currentTimeMillis() + timeout;
Throwable lastException = null;
do {
try {
JMXConnector connector = newJMXFactory(uri);
ActiveMQServerControl serverControl = MBeanServerInvocationHandler.newProxyInstance(connector.getMBeanServerConnection(), builder.getActiveMQServerObjectName(), ActiveMQServerControl.class, false);
serverControl.isActive(); // making one call to make sure it's working
return serverControl;
} catch (Throwable e) {
lastException = e;
Thread.sleep(500);
}
}
while (expireLoop > System.currentTimeMillis());
throw lastException;
}
protected static JMXConnector getJmxConnector(String hostname, int port) throws MalformedURLException {
// Without this, the RMI server would bind to the default interface IP (the user's local IP mostly)
System.setProperty("java.rmi.server.hostname", hostname);
// I don't specify both ports here manually on purpose. See actual RMI registry connection port extraction below.
String urlString = "service:jmx:rmi:///jndi/rmi://" + hostname + ":" + port + "/jmxrmi";
JMXServiceURL url = new JMXServiceURL(urlString);
JMXConnector jmxConnector = null;
try {
jmxConnector = JMXConnectorFactory.connect(url);
System.out.println("Successfully connected to: " + urlString);
} catch (Exception e) {
jmxConnector = null;
e.printStackTrace();
Assert.fail(e.getMessage());
}
return jmxConnector;
}
protected static final void recreateBrokerDirectory(final String homeInstance) {
recreateDirectory(homeInstance + "/data");
recreateDirectory(homeInstance + "/log");
}
protected void checkLogRecord(File logFile, boolean exist, String... values) throws Exception {
Assert.assertTrue(logFile.exists());
boolean hasRecord = false;
try (BufferedReader reader = new BufferedReader(new FileReader(logFile))) {
String line = reader.readLine();
while (line != null) {
if (line.contains(values[0])) {
boolean hasAll = true;
for (int i = 1; i < values.length; i++) {
if (!line.contains(values[i])) {
hasAll = false;
break;
}
}
if (hasAll) {
hasRecord = true;
System.out.println("audit has it: " + line);
break;
}
}
line = reader.readLine();
}
if (exist) {
Assert.assertTrue(hasRecord);
} else {
Assert.assertFalse(hasRecord);
}
}
}
protected static QueueControl getQueueControl(String uri,
ObjectNameBuilder builder,
String address,
String queueName,
RoutingType routingType,
long timeout) throws Throwable {
long expireLoop = System.currentTimeMillis() + timeout;
Throwable lastException = null;
do {
try {
JMXConnector connector = newJMXFactory(uri);
ObjectName objectQueueName = builder.getQueueObjectName(SimpleString.toSimpleString(address), SimpleString.toSimpleString(queueName), routingType);
QueueControl queueControl = MBeanServerInvocationHandler.newProxyInstance(connector.getMBeanServerConnection(), objectQueueName, QueueControl.class, false);
queueControl.getMessagesAcknowledged(); // making one call
return queueControl;
} catch (Throwable e) {
logger.warn(e.getMessage(), e);
lastException = e;
Thread.sleep(500);
}
}
while (expireLoop > System.currentTimeMillis());
throw lastException;
}
protected static void unzip(File zipFile, File serverFolder) throws IOException, ClassNotFoundException, InterruptedException {
ProcessBuilder zipBuilder = new ProcessBuilder("unzip", zipFile.getAbsolutePath()).directory(serverFolder);
Process process = zipBuilder.start();
SpawnedVMSupport.startLogger("zip", process);
logger.info("Zip finished with {}", process.waitFor());
}
protected static void zip(File zipFile, File serverFolder) throws IOException, ClassNotFoundException, InterruptedException {
logger.info("Zipping data folder for {}", zipFile);
ProcessBuilder zipBuilder = new ProcessBuilder("zip", "-r", zipFile.getAbsolutePath(), "data").directory(serverFolder);
Process process = zipBuilder.start();
SpawnedVMSupport.startLogger("zip", process);
logger.info("Zip finished with {}", process.waitFor());
}
public boolean waitForServerToStart(String uri, String username, String password, long timeout) throws InterruptedException {
long realTimeout = System.currentTimeMillis() + timeout;
while (System.currentTimeMillis() < realTimeout) {
try (ActiveMQConnectionFactory cf = ActiveMQJMSClient.createConnectionFactory(uri, null)) {
cf.createConnection(username, password).close();
System.out.println("server " + uri + " started");
} catch (Exception e) {
System.out.println("awaiting server " + uri + " start at ");
Thread.sleep(500);
continue;
}
return true;
}
return false;
}
}

View File

@ -15,12 +15,13 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.activemq.artemis.tests.soak; package org.apache.activemq.artemis.utils;
import java.lang.invoke.MethodHandles;
import org.junit.Assert; import org.junit.Assert;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.lang.invoke.MethodHandles;
/** Encapsulates System properties that could be passed on to the test. */ /** Encapsulates System properties that could be passed on to the test. */
public class TestParameters { public class TestParameters {

61
tests/db-tests/README.md Normal file
View File

@ -0,0 +1,61 @@
# Database Tests
This module runs tests against selected Databases.
There is one profile for each supported Database:
- DB-derby-tests
- DB-postgres-tests
- DB-mysql-tests
- DB-mssql-tests
- DB-db2-tests
- DB-oracle-tests
To enable the testsuite to run against any of these databases, simply enable the corresponding profiles.
## Providing Databases
When enabling a Database profile, you must download and provide the running database for that profile. You can also configure the JDBC URI.
You can refer to the examples provided under `./scripts`. Please note that you are responsible for complying with the licensing requirements of each database you provide.
## Configuring the JDBC URI
You can pass the JDBC URI as a parameter using the following supported parameters:
- `derby.uri`
- `postgres.uri`
- `mysql.uri`
- `mssql.uri`
- `oracle.uri`
- `db2.uri`
Example:
```shell
mvn -Pdb2.uri='jdbc:db2://MyDB2Server:50000/artemis:user=db2inst1;password=artemis;'
```
#Security Authorization
Tests on this module will perform several `drop table` and `create table` operations. So the user used to connecto these databases must have such `security grants`.
Also It is recommended to the database schema allocated exclusively to this testsuite.
# Servers
One Artemis server is created for each supported database. After building, they will be available under ./target/${DATABASE}:
- `./target/derby`
- `./target/postgres`
- `./target/mysql`
- `./target/mssql`
- `./target/db2`
Some of the tests on this module are connecting to the database directly, and these tests will use the JDBC jar directly from the `lib folder` from each of these servers.
# Oracle JDBC Driver
All the JDBC drivers using in this module are available as maven central repository and are being downloaded by the artemis-maven-plugin during the compilation phase of the tests.
The exception to this rule is [Oracle database](https://www.oracle.com/database/technologies/appdev/jdbc-downloads.html), for which you must provide the JAR under this location:
- jdbc-drivers/oracle

View File

@ -0,0 +1 @@
*.jar

View File

@ -0,0 +1,3 @@
Add the oracle jdbc driver that you downloaded from oracle.com and own a license on this folder. so the create process will place it in the correct place.
At the time of the creation of this file, the JDBC driver for Oracle could be found at https://www.oracle.com/database/technologies/appdev/jdbc-downloads.html

477
tests/db-tests/pom.xml Normal file
View File

@ -0,0 +1,477 @@
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.activemq.tests</groupId>
<artifactId>artemis-tests-pom</artifactId>
<version>2.31.0-SNAPSHOT</version>
</parent>
<artifactId>db-tests</artifactId>
<packaging>jar</packaging>
<name>Database Tests</name>
<properties>
<activemq.basedir>${project.basedir}/../../</activemq.basedir>
<!-- note: I am passing these properties through command line as opposed to syste-properties on surefire as IntelijIDEA wouldn't pick up these properties automatically. It was the simplest way to make it work -->
<db-extra-args>-Dderby.load=${derby.load} -Dpostgres.load=${postgres.load} -Dpostgres.uri=${postgres.uri}
-Dpostgres.class=${postgres.class} -Dmssql.load=${mssql.load} -Dmssql.uri=${mssql.uri}
-Dmssql.class=${mssql.class} -Dmysql.load=${mysql.load} -Dmysql.uri=${mysql.uri} -Dmysql.class=${mysql.class}
-Ddb2.load=${db2.load} -Ddb2.class=${db2.class} -Ddb2.uri=${db2.uri} -Doracle.load=${oracle.load}
-Doracle.uri=${oracle.uri} -Doracle.class=${oracle.class}
</db-extra-args>
<artemis-distribution-lib-dir>
-Ddistribution.lib="${activemq.basedir}/artemis-distribution/target/apache-artemis-${project.version}-bin/apache-artemis-${project.version}/lib"
</artemis-distribution-lib-dir>
<derby.load>false</derby.load>
<db2.load>false</db2.load>
<db2.uri>jdbc:db2://localhost:50000/artemis:user=db2inst1;password=artemis;</db2.uri>
<db2.class>com.ibm.db2.jcc.DB2Driver</db2.class>
<mysql.load>false</mysql.load>
<mysql.uri>jdbc:mysql://localhost/ARTEMIS-TEST?user=root&amp;#38;password=artemis</mysql.uri>
<mysql.class>com.mysql.cj.jdbc.Driver</mysql.class>
<postgres.load>false</postgres.load>
<postgres.uri>jdbc:postgresql://localhost:5432/artemis?user=artemis&amp;#38;password=artemis</postgres.uri>
<postgres.class>org.postgresql.Driver</postgres.class>
<oracle.load>false</oracle.load>
<oracle.uri>jdbc:oracle:thin:system/artemis@localhost:1521:FREE</oracle.uri>
<oracle.class>oracle.jdbc.driver.OracleDriver</oracle.class>
<mssql.load>false</mssql.load>
<mssql.uri>jdbc:sqlserver://localhost:1433;user=sa;password=ActiveMQ*Artemis</mssql.uri>
<mssql.class>com.microsoft.sqlserver.jdbc.SQLServerDriver</mssql.class>
<skipDBTests>true</skipDBTests>
</properties>
<dependencies>
<dependency>
<!-- this dependency is here to make sure this module is only executed
after the distribution is created.
Otherwise it will get here before the build eventually.
e.g if you use mvn install -T 20 -->
<groupId>org.apache.activemq</groupId>
<artifactId>apache-artemis</artifactId>
<version>${project.version}</version>
<scope>compile</scope>
<type>pom</type>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-server</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.activemq.tests</groupId>
<artifactId>artemis-test-support</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-core-client</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-jms-client</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-commons</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-cli</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-journal</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-common</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.jctools</groupId>
<artifactId>jctools-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.qpid</groupId>
<artifactId>qpid-jms-client</artifactId>
<scope>test</scope>
</dependency>
<!-- logging -->
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-slf4j-impl</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>jakarta.jms</groupId>
<artifactId>jakarta.jms-api</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>jakarta.management.j2ee</groupId>
<artifactId>jakarta.management.j2ee-api</artifactId>
<scope>test</scope>
</dependency>
<!-- The johnzon-core and json-api contents are repackaged in -commons,
However maven can still need them during tests, which run against
the original -commons classes when built+run in the same reactor,
and not the jar containing the shaded bits. -->
<dependency>
<groupId>org.apache.johnzon</groupId>
<artifactId>johnzon-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>jakarta.json</groupId>
<artifactId>jakarta.json-api</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-unit-test-support</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<!-- DB Test Deps -->
<dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derby</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.activemq</groupId>
<artifactId>artemis-maven-plugin</artifactId>
<executions>
<execution>
<phase>test-compile</phase>
<id>create-jdbc-bad-driver</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<role>amq</role>
<user>admin</user>
<password>admin</password>
<allowAnonymous>false</allowAnonymous>
<noWeb>true</noWeb>
<instance>${basedir}/target/jdbc-bad-driver</instance>
<args>
<arg>--shared-store</arg>
<arg>--jdbc</arg>
<arg>--jdbc-connection-url</arg>
<arg>tcp://noexist</arg>
<arg>--jdbc-driver-class-name</arg>
<arg>badDriver</arg>
</args>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-derby</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
<instance>${basedir}/target/derby</instance>
<configuration>${basedir}/target/classes/servers/derby</configuration>
<noWeb>true</noWeb>
<libList>
<arg>org.apache.derby:derby:${apache.derby.version}</arg>
</libList>
<args>
<arg>--jdbc</arg>
<arg>--global-max-messages</arg>
<arg>100</arg>
<arg>--java-options</arg>
<arg>-ea</arg>
</args>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-mysql</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
<instance>${basedir}/target/mysql</instance>
<configuration>${basedir}/target/classes/servers/mysql</configuration>
<noWeb>true</noWeb>
<libListWithDeps>
<arg>com.mysql:mysql-connector-j:8.0.33</arg>
</libListWithDeps>
<args>
<arg>--jdbc</arg>
<arg>--jdbc-connection-url</arg>
<arg>${mysql.uri}</arg>
<arg>--jdbc-driver-class-name</arg>
<arg>${mysql.class}</arg>
<arg>--global-max-messages</arg>
<arg>100</arg>
<arg>--java-options</arg>
<arg>-ea</arg>
</args>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-postgres</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
<instance>${basedir}/target/postgres</instance>
<configuration>${basedir}/target/classes/servers/postgres</configuration>
<noWeb>true</noWeb>
<libListWithDeps>
<arg>org.postgresql:postgresql:42.6.0</arg>
</libListWithDeps>
<args>
<arg>--jdbc</arg>
<arg>--jdbc-connection-url</arg>
<arg>${postgres.uri}</arg>
<arg>--jdbc-driver-class-name</arg>
<arg>${postgres.class}</arg>
<arg>--global-max-messages</arg>
<arg>100</arg>
<arg>--java-options</arg>
<arg>-ea</arg>
</args>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-oracle</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
<instance>${basedir}/target/oracle</instance>
<noWeb>true</noWeb>
<libFolders>
<!-- you must download the driver here manually before you can use it -->
<arg>${project.basedir}/jdbc-drivers/oracle</arg>
</libFolders>
<args>
<arg>--jdbc</arg>
<arg>--jdbc-connection-url</arg>
<arg>${oracle.uri}</arg>
<arg>--jdbc-driver-class-name</arg>
<arg>${oracle.class}</arg>
<arg>--global-max-messages</arg>
<arg>100</arg>
<arg>--java-options</arg>
<arg>-ea</arg>
</args>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-mssql</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<libList>
<arg>com.microsoft.sqlserver:mssql-jdbc:8.4.1.jre11</arg>
</libList>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
<instance>${basedir}/target/mssql</instance>
<noWeb>true</noWeb>
<args>
<arg>--jdbc</arg>
<arg>--jdbc-connection-url</arg>
<arg>${mssql.uri}</arg>
<arg>--jdbc-driver-class-name</arg>
<arg>${mssql.class}</arg>
<arg>--global-max-messages</arg>
<arg>100</arg>
<arg>--java-options</arg>
<arg>-ea</arg>
</args>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-DB2</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<libList>
<arg>com.ibm.db2:jcc:11.5.8.0</arg>
</libList>
<!-- <libFolders>
<arg>${project.basedir}/jdbc-drivers/db2</arg>
</libFolders> -->
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
<instance>${basedir}/target/db2</instance>
<noWeb>true</noWeb>
<args>
<arg>--jdbc</arg>
<arg>--jdbc-connection-url</arg>
<arg>${db2.uri}</arg>
<arg>--jdbc-driver-class-name</arg>
<arg>${db2.class}</arg>
<arg>--global-max-messages</arg>
<arg>100</arg>
<arg>--java-options</arg>
<arg>-ea</arg>
</args>
</configuration>
</execution>
</executions>
<dependencies>
<dependency>
<groupId>org.apache.activemq.tests</groupId>
<artifactId>smoke-tests</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<forkCount>1</forkCount>
<reuseForks>false</reuseForks>
<skipTests>${skipDBTests}</skipTests>
<argLine>${db-extra-args} ${activemq-surefire-argline} ${artemis-distribution-lib-dir}</argLine>
<systemProperties>
<postgres.uri>${postgres.uri}</postgres.uri>
</systemProperties>
</configuration>
</plugin>
</plugins>
</build>
<profiles>
<profile>
<id>DB-all-tests</id>
<properties>
<skipDBTests>false</skipDBTests>
<oracle.load>true</oracle.load>
<mysql.load>true</mysql.load>
<mssql.load>true</mssql.load>
<db2.load>true</db2.load>
<postgres.load>true</postgres.load>
</properties>
</profile>
<profile>
<id>DB-oracle-tests</id>
<properties>
<skipDBTests>false</skipDBTests>
<oracle.load>true</oracle.load>
</properties>
</profile>
<profile>
<id>DB-mysql-tests</id>
<properties>
<mysql.load>true</mysql.load>
<skipDBTests>false</skipDBTests>
</properties>
</profile>
<profile>
<id>DB-mssql-tests</id>
<properties>
<mssql.load>true</mssql.load>
<skipDBTests>false</skipDBTests>
</properties>
</profile>
<profile>
<id>DB-db2-tests</id>
<properties>
<db2.load>true</db2.load>
<skipDBTests>false</skipDBTests>
</properties>
</profile>
<profile>
<id>DB-postgres-tests</id>
<properties>
<postgres.load>true</postgres.load>
<skipDBTests>false</skipDBTests>
</properties>
</profile>
<profile>
<id>DB-derby-tests</id>
<properties>
<derby.load>true</derby.load>
<skipDBTests>false</skipDBTests>
</properties>
</profile>
</profiles>
</project>

2
tests/db-tests/scripts/.gitignore vendored Normal file
View File

@ -0,0 +1,2 @@
../../soak-tests/src/test/scripts/oradb
db2db

View File

@ -16,7 +16,6 @@
# specific language governing permissions and limitations # specific language governing permissions and limitations
# under the License. # under the License.
# This script shows a simple way to stop a mysql with podman source ./container-define.sh
podman kill mysql-artemis-test $CONTAINER_COMMAND exec -it db2-artemis-test bash
podman rm mysql-artemis-test

View File

@ -16,5 +16,6 @@
# specific language governing permissions and limitations # specific language governing permissions and limitations
# under the License. # under the License.
# Start a command line mysql for the Database started with ./start-mysql-podman.sh source ./container-define.sh
podman exec -it mysql-artemis-test mysql ARTEMIS-TEST -u root --password=artemis
$CONTAINER_COMMAND exec -it mssql-artemis-test /opt/mssql-tools/bin/sqlcmd -S localhost -U sa -P ActiveMQ*Artemis

View File

@ -16,7 +16,6 @@
# specific language governing permissions and limitations # specific language governing permissions and limitations
# under the License. # under the License.
# This script shows a simple way to stop a mysql with podman source ./container-define.sh
podman kill postgres-artemis-test $CONTAINER_COMMAND exec -it mysql-artemis-test mysql ARTEMIS-TEST -u root --password=artemis
podman rm postgres-artemis-test

View File

@ -16,6 +16,7 @@
# specific language governing permissions and limitations # specific language governing permissions and limitations
# under the License. # under the License.
# Start a command line mysql for the Database started with ./start-postgres-podman.sh source ./container-define.sh
podman exec -it postgres-artemis-test psql -U artemis artemis
#podman exec -it mysql-artemis-test mysql ARTEMIS-TEST -u root --password=artemis $CONTAINER_COMMAND exec -it oracle-artemis-test sqlplus system/artemis@FREE

View File

@ -0,0 +1,21 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND exec -it postgres-artemis-test psql -U artemis artemis

View File

@ -0,0 +1,22 @@
#!/bin/sh
# 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.
#CONTAINER_COMMAND=$CONTAINER_COMMAND
# you may use 'podman' or 'docker'
CONTAINER_COMMAND=podman

View File

@ -1,4 +1,3 @@
#!/bin/sh
# Licensed to the Apache Software Foundation (ASF) under one # Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file # or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information # distributed with this work for additional information
@ -16,7 +15,18 @@
# specific language governing permissions and limitations # specific language governing permissions and limitations
# under the License. # under the License.
# This script shows a simple way to start a mysql with podman DB2INSTANCE=db2inst1
DB2INST1_PASSWORD=artemis
./stop-mysql-podman.sh DBNAME=artemis
podman run -d -p 3306:3306 --name mysql-artemis-test --rm -e MYSQL_ROOT_PASSWORD=artemis -e MYSQL_USER=artemis -e MYSQL_PASSWORD=artemis -e MYSQL_DATABASE=ARTEMIS-TEST mysql:8 BLU=false
ENABLE_ORACLE_COMPATIBILITY=false
UPDATEAVAIL=NO
TO_CREATE_SAMPLEDB=false
REPODB=false
IS_OSXFS=false
PERSISTENT_HOME=false
HADR_ENABLED=false
ETCD_ENDPOINT=
ETCD_USERNAME=
ETCD_PASSWORD=
AUTOCONFIG=false

View File

@ -0,0 +1,21 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND logs -f db2-artemis-test

View File

@ -0,0 +1,21 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND logs -f mssql-artemis-test

View File

@ -0,0 +1,21 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND logs -f mysql-artemis-test

View File

@ -0,0 +1,22 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND logs -f oracle-artemis-test

View File

@ -0,0 +1,21 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND logs -f postgres-artemis-test

View File

@ -16,7 +16,13 @@
# specific language governing permissions and limitations # specific language governing permissions and limitations
# under the License. # under the License.
# This script shows a simple way to start a mysql with podman source ./container-define.sh
./stop-postgres-podman.sh echo ""
podman run --name postgres-artemis-test --rm -d -e POSTGRES_USER=artemis -e POSTGRES_PASSWORD=artemis -e POSTGRES_DB=artemis -p 5432:5432 -p 9876:80 postgres echo "*******************************************************************************************************************************"
echo " Notice:"
echo " This script is provided solely to assist you in running a $1 Free Database in a $CONTAINER_COMMAND environment,"
echo " to facilitate development and testing, using an image provided by $2."
echo " By running this script, you agree to abide by all licensing terms issued by $2 for the $1 image being downloaded here."
echo "*******************************************************************************************************************************"
echo ""

View File

@ -0,0 +1,60 @@
#!/bin/sh
# 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.
source ./container-define.sh
# NOTE: at the time this script was written podman had an issue starting DB2 without a folder specified. Docker ran it without any problems.
# If you must use podman you could specify a data folder and it should work fine
# As documented on https://www.ibm.com/docs/en/db2/11.5?topic=system-linux
export LICENSE=reject
if [ $# -ge 1 ]; then
# Check if the first argument is --accept-license
if [ "$1" == "--accept-license" ]; then
./print-license.sh "DB2" "IBM"
export LICENSE=accept
else
echo "Warning: you must accept the DB2 license. Run ./logs-db2.sh to check the log output."
echo "Usage: $0 --accept-license <folder_data>"
fi
else
echo "Warning: you must accept the DB2 license. Run ./logs-db2.sh to check the log output."
echo "Usage: $0 --accept-license <folder_data>"
fi
if [ $# -ne 2 ]; then
echo "NO_DATA has been specified. not using a data folder"
data_argument=""
folder_data="NO_DATA"
else
folder_data=$2
data_argument="-v $folder_data:/database:Z"
fi
./stop-db2.sh
if [ ! -d "$folder_data" ] && [ "$folder_data" != "NO_DATA" ]; then
mkdir "$folder_data"
chmod 777 $folder_data
echo "Folder '$folder_data' created."
fi
$CONTAINER_COMMAND run -d -h db2-artemis-test --name db2-artemis-test --privileged=true -p 50000:50000 -eLICENSE=$LICENSE --env-file db2.env $data_argument icr.io/db2_community/db2

View File

@ -0,0 +1,37 @@
#!/bin/sh
# 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.
source ./container-define.sh
export LICENSE="ACCEPT_EULA=N"
if [ $# -ge 1 ]; then
# Check if the first argument is --accept-license
if [ "$1" == "--accept-license" ]; then
./print-license.sh "SQL Server" "Microsoft"
export LICENSE="ACCEPT_EULA=Y"
else
echo "Warning: you must accept the Microsoft license. Run ./logs-mssql.sh to check the log output."
fi
else
echo "Warning: you must accept the Microsoft license. Run ./logs-mssql.sh to check the log output."
fi
./stop-mssql.sh
$CONTAINER_COMMAND run -d --name mssql-artemis-test -e "$LICENSE" -e "MSSQL_SA_PASSWORD=ActiveMQ*Artemis" -p 1433:1433 mcr.microsoft.com/mssql/server:2019-latest

View File

@ -0,0 +1,22 @@
#!/bin/sh
# 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.
source ./container-define.sh
./stop-mysql.sh
$CONTAINER_COMMAND run -d -p 3306:3306 --name mysql-artemis-test --rm -e MYSQL_ROOT_PASSWORD=artemis -e MYSQL_USER=artemis -e MYSQL_PASSWORD=artemis -e MYSQL_DATABASE=ARTEMIS-TEST mysql:8

View File

@ -0,0 +1,46 @@
#!/bin/sh
# 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.
source ./container-define.sh
if [ $# -ne 1 ]; then
echo "Usage: $0 <folder_data>"
echo " setting folder_data as NO_DATA by default"
folder_data="NO_DATA"
else
folder_data="$1"
fi
./stop-oracle.sh
./print-license.sh Oracle Oracle
if [ "$folder_data" = "NO_DATA" ]; then
echo "NO_DATA has been specified. not using a data folder"
data_argument=""
else
data_argument="-v $folder_data:/opt/oracle/oradata:Z"
fi
if [ ! -d "$folder_data" ] && [ "$folder_data" != "NO_DATA" ]; then
mkdir "$folder_data"
chmod 777 $folder_data
echo "Folder '$folder_data' created."
fi
$CONTAINER_COMMAND run -d --name oracle-artemis-test -p 1521:1521 $data_argument -e ORACLE_PWD=artemis container-registry.oracle.com/database/free:latest

View File

@ -0,0 +1,22 @@
#!/bin/sh
# 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.
source ./container-define.sh
./stop-postgres.sh
$CONTAINER_COMMAND run --name postgres-artemis-test --rm -d -e POSTGRES_USER=artemis -e POSTGRES_PASSWORD=artemis -e POSTGRES_DB=artemis -p 5432:5432 -p 9876:80 postgres

View File

@ -0,0 +1,23 @@
#!/bin/sh
# 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.
./stop-db2.sh
./stop-mssql.sh
./stop-mysql.sh
./stop-oracle.sh
./stop-postgres.sh

View File

@ -0,0 +1,22 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND kill db2-artemis-test
$CONTAINER_COMMAND rm -f db2-artemis-test

View File

@ -0,0 +1,22 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND kill mssql-artemis-test
$CONTAINER_COMMAND rm mssql-artemis-test

View File

@ -0,0 +1,22 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND kill mysql-artemis-test
$CONTAINER_COMMAND rm mysql-artemis-test

View File

@ -0,0 +1,22 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND kill oracle-artemis-test
$CONTAINER_COMMAND rm oracle-artemis-test

View File

@ -0,0 +1,22 @@
#!/bin/sh
# 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.
source ./container-define.sh
$CONTAINER_COMMAND kill postgres-artemis-test
$CONTAINER_COMMAND rm postgres-artemis-test

View File

@ -0,0 +1,68 @@
/*
* 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.db;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.List;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.tests.db.common.Database;
import org.apache.activemq.artemis.tests.db.common.ParameterDBTestBase;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Test;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DropDBTest extends ParameterDBTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Parameterized.Parameters(name = "db={0}")
public static Collection<Object[]> parameters() {
List<Database> dbList = Database.selectedList();
dbList.remove(Database.DERBY); // no derby on this test
return convertParameters(dbList);
}
@Override
public void setUp() throws Exception {
super.setUp();
Assume.assumeTrue(database != Database.DERBY);
dropDatabase();
}
@Override
protected final String getJDBCClassName() {
return database.getDriverClass();
}
@Test
public void testSimpleDrop() throws Exception {
ActiveMQServer server = createServer(createDefaultConfig(0, true));
server.start();
server.stop();
int tablesDroppped = dropDatabase();
if (tablesDroppped < 4) {
logger.warn("At least 4 tables should be removed, while only {} tables were dropped", tablesDroppped);
Assert.fail("Only " + tablesDroppped + " tables were dropped");
}
}
}

View File

@ -0,0 +1,28 @@
/*
* 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.db.common;
import java.lang.invoke.MethodHandles;
import org.apache.activemq.artemis.utils.RealServerTestBase;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DBTestBase extends RealServerTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
}

View File

@ -0,0 +1,181 @@
/*
* 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.db.common;
import java.io.File;
import java.net.URL;
import java.net.URLClassLoader;
import java.sql.Connection;
import java.sql.Driver;
import java.sql.DriverManager;
import java.util.ArrayList;
import java.util.List;
import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
import org.apache.activemq.artemis.utils.RandomUtil;
public enum Database {
MYSQL("mysql"), POSTGRES("postgres"), ORACLE("oracle"), MSSQL("mssql"), DB2("db2"), JOURNAL("journal"), DERBY("derby");
private String dbname;
private boolean load;
private String jdbcURI;
private String driverClass;
private Driver driver;
private ClassLoader dbClassLoader;
Database(String dbname) {
this.dbname = dbname;
load = Boolean.parseBoolean(System.getProperty(dbname + ".load", "false"));
}
public static ClassLoader defineClassLoader(File location, ClassLoader parentClassLoader) throws Exception {
File[] jars = location.listFiles((dir, name) -> name.toLowerCase().endsWith(".jar"));
URL[] url = new URL[jars.length];
for (int i = 0; i < jars.length; i++) {
url[i] = jars[i].toURI().toURL();
}
return new URLClassLoader(url, parentClassLoader);
}
public Connection getConnection() throws Exception {
switch (this) {
case DERBY:
return DriverManager.getConnection(getJdbcURI());
case JOURNAL:
return null;
default:
return getDriver().connect(getJdbcURI(), null);
}
}
// There is one artemis server for each database we provide on the tests
public ClassLoader getDBClassLoader() throws Exception {
if (this != JOURNAL && this != DERBY && dbClassLoader == null) {
String serverLocation = ParameterDBTestBase.getServerLocation(getName());
File lib = new File(serverLocation + "/lib");
dbClassLoader = defineClassLoader(lib, getClass().getClassLoader());
}
return dbClassLoader;
}
public String getName() {
return dbname;
}
public String getJdbcURI() {
if (jdbcURI == null) {
switch (this) {
case DERBY:
String derbyData = ParameterDBTestBase.getServerLocation("derby") + "/data/derby/db";
jdbcURI = "jdbc:derby:" + derbyData + ";create=true";
break;
case JOURNAL:
jdbcURI = null;
break;
default:
jdbcURI = System.getProperty(dbname + ".uri");
if (jdbcURI != null) {
jdbcURI = jdbcURI.replaceAll("&#38;", "&");
}
}
}
return jdbcURI;
}
public String getDriverClass() {
if (driverClass != null) {
return driverClass;
}
switch (this) {
case DERBY:
this.driverClass = ActiveMQDefaultConfiguration.getDefaultDriverClassName();
break;
case JOURNAL:
this.driverClass = null;
break;
default:
driverClass = System.getProperty(dbname + ".class");
}
return driverClass;
}
public Driver getDriver() throws Exception {
if (driver == null) {
String className = getDriverClass();
ClassLoader loader = getDBClassLoader();
Class clazz = loader.loadClass(className);
driver = (Driver) clazz.getDeclaredConstructor().newInstance();
}
return driver;
}
// this must be called within the test classLoader (Thread Context Class Loader)
public void registerDriver() throws Exception {
if (driver != null) {
DriverManager.registerDriver(driver);
}
}
public boolean isLoad() {
return load;
}
@Override
public String toString() {
return dbname;
}
// it will return a list of Databases selected to be tested
public static List<Database> selectedList() {
ArrayList<Database> dbList = new ArrayList<>();
for (Database db : Database.values()) {
if (db.isLoad()) {
dbList.add(db);
}
}
return dbList;
}
public static Database random() {
List<Database> selectedDatabases = selectedList();
if (selectedDatabases.isEmpty()) {
return null;
} else {
return selectedDatabases.get(RandomUtil.randomInterval(0, selectedDatabases.size()));
}
}
public static List<Database> randomList() {
List<Database> selectedDatabases = selectedList();
ArrayList<Database> list = new ArrayList<>();
Database randomDB = random();
if (randomDB != null) {
list.add(randomDB);
}
return list;
}
}

View File

@ -0,0 +1,194 @@
/*
* 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.db.common;
import java.lang.invoke.MethodHandles;
import java.sql.Connection;
import java.sql.ResultSet;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import org.apache.activemq.artemis.core.config.Configuration;
import org.apache.activemq.artemis.core.config.storage.DatabaseStorageConfiguration;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@RunWith(Parameterized.class)
public abstract class ParameterDBTestBase extends DBTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Override
protected void dropDerby() throws Exception {
cleanupData(database.getName());
}
@Override
protected final String getTestJDBCConnectionUrl() {
return database.getJdbcURI();
}
@Parameterized.Parameter
public Database database;
@Override
public void setUp() throws Exception {
super.setUp();
disableCheckThread();
if (database == Database.DERBY) {
runAfter(this::shutdownDerby);
}
registerDB();
dropDatabase();
}
// Register the database on driver and prepares the classLoader to be used
private void registerDB() throws Exception {
ClassLoader dbClassLoader = database.getDBClassLoader();
if (dbClassLoader != null) {
ClassLoader tccl = Thread.currentThread().getContextClassLoader();
Thread.currentThread().setContextClassLoader(dbClassLoader);
final Thread currentThread = Thread.currentThread();
runAfter((() -> {
currentThread.setContextClassLoader(tccl);
}));
database.registerDriver();
}
}
protected static ArrayList<Object[]> convertParameters(List<Database> dbList) {
ArrayList<Object[]> parameters = new ArrayList<>();
dbList.forEach(s -> {
logger.info("Adding {} to the list for the test", s);
parameters.add(new Object[]{s});
});
return parameters;
}
protected ClassLoader getDBClassLoader() throws Exception {
return database.getDBClassLoader();
}
public Connection getConnection() throws Exception {
return database.getConnection();
}
public int dropDatabase() {
switch (database) {
case JOURNAL:
return 0;
case DERBY:
try {
logger.info("Drop derby");
dropDerby();
} catch (Exception e) {
logger.debug("Error dropping derby db: {}", e.getMessage());
}
return 1;
default:
return dropTables("BINDING", "MESSAGE", "LARGE_MESSAGE", "PAGE_STORE", "NODE_MANAGER");
}
}
private int dropTables(String...tables) {
int dropped = 0;
try (Connection connection = getConnection()) {
ResultSet data = connection.getMetaData().getTables(null, "%", "%", new String[]{"TABLE"});
while (data.next()) {
String table = data.getString("TABLE_NAME");
logger.debug("Checking table {}", table);
boolean drop = false;
for (String str : tables) {
logger.debug("Checking pattern {}", str);
if (table.toUpperCase(Locale.ROOT).contains(str)) {
logger.debug("Table {} is part of the list as it is matching", table, str);
drop = true;
break;
}
}
if (drop) {
if (dropTable(connection, table)) {
dropped++;
}
}
}
} catch (Exception e) {
logger.warn(e.getMessage(), e);
return -1;
}
return dropped;
}
private boolean dropTable(Connection connection, String table) {
try {
connection.createStatement().execute("DROP TABLE " + table);
logger.debug("Dropped {}", table);
return true;
} catch (Exception e) {
logger.warn("Error dropping {} -> {}", table, e.getMessage());
return false;
}
}
@Override
protected Configuration createDefaultConfig(final int serverID, final boolean netty) throws Exception {
Configuration configuration = super.createDefaultConfig(serverID, netty);
if (database != Database.JOURNAL) {
setDBStoreType(configuration);
}
configuration.getAddressSettings().clear();
return configuration;
}
@Override
protected DatabaseStorageConfiguration createDefaultDatabaseStorageConfiguration() {
DatabaseStorageConfiguration dbStorageConfiguration = new DatabaseStorageConfiguration();
String connectionURI = getTestJDBCConnectionUrl();
dbStorageConfiguration.setJdbcDriverClassName(database.getDriverClass());
dbStorageConfiguration.setJdbcConnectionUrl(connectionURI);
dbStorageConfiguration.setBindingsTableName("BINDINGS");
dbStorageConfiguration.setMessageTableName("MESSAGES");
dbStorageConfiguration.setLargeMessageTableName("LARGE_MESSAGES");
dbStorageConfiguration.setPageStoreTableName("PAGE_STORE");
dbStorageConfiguration.setJdbcPassword(getJDBCPassword());
dbStorageConfiguration.setJdbcUser(getJDBCUser());
dbStorageConfiguration.setJdbcLockAcquisitionTimeoutMillis(getJdbcLockAcquisitionTimeoutMillis());
dbStorageConfiguration.setJdbcLockExpirationMillis(getJdbcLockExpirationMillis());
dbStorageConfiguration.setJdbcLockRenewPeriodMillis(getJdbcLockRenewPeriodMillis());
dbStorageConfiguration.setJdbcNetworkTimeout(-1);
dbStorageConfiguration.setJdbcAllowedTimeDiff(250L);
return dbStorageConfiguration;
}
}

View File

@ -0,0 +1,46 @@
/*
* 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.db.invalid;
import java.lang.invoke.MethodHandles;
import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.tests.db.common.DBTestBase;
import org.junit.Assert;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class JdbcStartupInvalidTest extends DBTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected static final String SERVER_NAME = "jdbc-bad-driver";
@Test
public void startupBadJdbcConnectionTest() throws Exception {
Process p = startServer(SERVER_NAME, 0, 0);
try {
p.waitFor(20, TimeUnit.SECONDS);
Assert.assertFalse(p.isAlive());
} catch (Exception e) {
logger.warn(e.getMessage(), e);
Assert.fail(e.getMessage());
}
}
}

View File

@ -0,0 +1,217 @@
/*
* 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.db.paging;
import javax.jms.BytesMessage;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.Message;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Session;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.tests.db.common.Database;
import org.apache.activemq.artemis.tests.db.common.ParameterDBTestBase;
import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.activemq.artemis.utils.RandomUtil;
import org.apache.activemq.artemis.utils.Wait;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class GlobalPagingTest extends ParameterDBTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Parameterized.Parameters(name = "db={0}")
public static Collection<Object[]> parameters() {
return convertParameters(Database.randomList());
}
@Override
public void setUp() throws Exception {
super.setUp();
dropDatabase();
}
@Override
protected final String getJDBCClassName() {
return database.getDriverClass();
}
@Test
public void testMaxMessages() throws Exception {
ActiveMQServer server = createServer(createDefaultConfig(0, true));
server.getConfiguration().getAddressSettings().clear();
server.getConfiguration().getAddressSettings().put("#", new AddressSettings().setMaxSizeMessages(10));
server.getConfiguration().setGlobalMaxMessages(5);
server.start();
String addressName = "Q" + RandomUtil.randomString();
server.addAddressInfo(new AddressInfo(addressName).addRoutingType(RoutingType.ANYCAST));
server.createQueue(new QueueConfiguration(addressName).setRoutingType(RoutingType.ANYCAST).setDurable(true));
ConnectionFactory cf = CFUtil.createConnectionFactory("core", "tcp://localhost:61616");
try (Connection connection = cf.createConnection()) {
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
MessageProducer producer = session.createProducer(session.createQueue(addressName));
for (int i = 0; i < 6; i++) {
producer.send(session.createTextMessage());
}
session.commit();
Queue queue = server.locateQueue(addressName);
Wait.assertTrue(queue.getPagingStore()::isPaging, 1000, 100);
connection.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(addressName));
for (int i = 0; i < 6; i++) {
Message message = consumer.receive(5000);
Assert.assertNotNull(message);
}
session.commit();
Wait.assertFalse(queue.getPagingStore()::isPaging, 1000, 100);
}
}
@Test
public void testMaxMessagesOpposite() throws Exception {
ActiveMQServer server = createServer(createDefaultConfig(0, true));
server.getConfiguration().getAddressSettings().clear();
server.getConfiguration().getAddressSettings().put("#", new AddressSettings().setMaxSizeMessages(5));
server.getConfiguration().setGlobalMaxMessages(500);
server.start();
String addressName = "Q" + RandomUtil.randomString();
server.addAddressInfo(new AddressInfo(addressName).addRoutingType(RoutingType.ANYCAST));
server.createQueue(new QueueConfiguration(addressName).setRoutingType(RoutingType.ANYCAST).setDurable(true));
ConnectionFactory cf = CFUtil.createConnectionFactory("core", "tcp://localhost:61616");
try (Connection connection = cf.createConnection()) {
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
MessageProducer producer = session.createProducer(session.createQueue(addressName));
for (int i = 0; i < 6; i++) {
producer.send(session.createTextMessage());
}
session.commit();
Queue queue = server.locateQueue(addressName);
Wait.assertTrue(queue.getPagingStore()::isPaging, 1000, 100);
connection.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(addressName));
for (int i = 0; i < 6; i++) {
Message message = consumer.receive(5000);
Assert.assertNotNull(message);
}
session.commit();
Wait.assertFalse(queue.getPagingStore()::isPaging, 1000, 100);
}
}
@Test
public void testMaxMessagesBytes() throws Exception {
ActiveMQServer server = createServer(createDefaultConfig(0, true));
server.getConfiguration().getAddressSettings().clear();
server.getConfiguration().getAddressSettings().put("#", new AddressSettings().setMaxSizeMessages(10000).setMaxSizeBytes(100 * 1024 * 1024));
server.getConfiguration().setGlobalMaxSize(50 * 1024);
server.start();
String addressName = "Q" + RandomUtil.randomString();
server.addAddressInfo(new AddressInfo(addressName).addRoutingType(RoutingType.ANYCAST));
server.createQueue(new QueueConfiguration(addressName).setRoutingType(RoutingType.ANYCAST).setDurable(true));
ConnectionFactory cf = CFUtil.createConnectionFactory("core", "tcp://localhost:61616");
try (Connection connection = cf.createConnection()) {
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
MessageProducer producer = session.createProducer(session.createQueue(addressName));
for (int i = 0; i < 6; i++) {
BytesMessage message = session.createBytesMessage();
message.writeBytes(new byte[20 * 1024]);
producer.send(message);
}
session.commit();
Queue queue = server.locateQueue(addressName);
Wait.assertTrue(queue.getPagingStore()::isPaging, 1000, 100);
connection.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(addressName));
for (int i = 0; i < 6; i++) {
Message message = consumer.receive(5000);
Assert.assertNotNull(message);
}
session.commit();
Wait.assertFalse(queue.getPagingStore()::isPaging, 1000, 100);
}
}
@Test
public void testMaxMessagesBytesOpposite() throws Exception {
ActiveMQServer server = createServer(createDefaultConfig(0, true));
server.getConfiguration().getAddressSettings().clear();
server.getConfiguration().getAddressSettings().put("#", new AddressSettings().setMaxSizeMessages(10000).setMaxSizeBytes(50 * 1024));
server.getConfiguration().setGlobalMaxSize(10 * 1024 * 1024);
server.start();
String addressName = "Q" + RandomUtil.randomString();
server.addAddressInfo(new AddressInfo(addressName).addRoutingType(RoutingType.ANYCAST));
server.createQueue(new QueueConfiguration(addressName).setRoutingType(RoutingType.ANYCAST).setDurable(true));
ConnectionFactory cf = CFUtil.createConnectionFactory("core", "tcp://localhost:61616");
try (Connection connection = cf.createConnection()) {
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
MessageProducer producer = session.createProducer(session.createQueue(addressName));
for (int i = 0; i < 6; i++) {
BytesMessage message = session.createBytesMessage();
message.writeBytes(new byte[20 * 1024]);
producer.send(message);
}
session.commit();
Queue queue = server.locateQueue(addressName);
Wait.assertTrue(queue.getPagingStore()::isPaging, 1000, 100);
connection.start();
MessageConsumer consumer = session.createConsumer(session.createQueue(addressName));
for (int i = 0; i < 6; i++) {
Message message = consumer.receive(5000);
Assert.assertNotNull(message);
}
session.commit();
Wait.assertFalse(queue.getPagingStore()::isPaging, 1000, 100);
}
}
}

View File

@ -0,0 +1,86 @@
/*
* 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.db.paging;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.core.config.storage.DatabaseStorageConfiguration;
import org.apache.activemq.artemis.core.paging.impl.Page;
import org.apache.activemq.artemis.core.paging.impl.PagingStoreImpl;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.jdbc.store.file.JDBCSequentialFile;
import org.apache.activemq.artemis.tests.db.common.Database;
import org.apache.activemq.artemis.tests.db.common.ParameterDBTestBase;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class NetworkTimeoutCheckTest extends ParameterDBTestBase {
private static final int TIMEOUT = 33_333;
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Parameterized.Parameters(name = "db={0}")
public static Collection<Object[]> parameters() {
return convertParameters(Database.selectedList());
}
@Override
public void setUp() throws Exception {
super.setUp();
dropDatabase();
}
@Override
protected final String getJDBCClassName() {
return database.getDriverClass();
}
// check if the sync timeout is propated to the page file
@Test
public void testDBTimeoutPropagated() throws Exception {
ActiveMQServer server = createServer(createDefaultConfig(0, true));
server.start();
server.addAddressInfo(new AddressInfo(getName()).addRoutingType(RoutingType.ANYCAST));
Queue queue = server.createQueue(new QueueConfiguration(getName()).setRoutingType(RoutingType.ANYCAST).setDurable(true));
queue.getPagingStore().startPaging();
PagingStoreImpl store = (PagingStoreImpl) queue.getPagingStore();
Page page = store.getCurrentPage();
JDBCSequentialFile file = (JDBCSequentialFile) page.getFile();
Assert.assertEquals(TIMEOUT, file.getNetworkTimeoutMillis());
server.stop();
}
@Override
protected DatabaseStorageConfiguration createDefaultDatabaseStorageConfiguration() {
DatabaseStorageConfiguration dbStorageConfiguration = super.createDefaultDatabaseStorageConfiguration();
dbStorageConfiguration.setJdbcNetworkTimeout(TIMEOUT);
return dbStorageConfiguration;
}
}

View File

@ -0,0 +1,139 @@
/*
* 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.db.paging;
import javax.jms.BytesMessage;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.MessageProducer;
import javax.jms.Session;
import java.lang.invoke.MethodHandles;
import java.sql.ResultSet;
import java.util.Collection;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.core.config.storage.DatabaseStorageConfiguration;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.tests.db.common.Database;
import org.apache.activemq.artemis.tests.db.common.ParameterDBTestBase;
import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.activemq.artemis.utils.RandomUtil;
import org.apache.activemq.artemis.utils.Wait;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class PageSizeTest extends ParameterDBTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Parameterized.Parameters(name = "db={0}")
public static Collection<Object[]> parameters() {
return convertParameters(Database.selectedList());
}
@Override
public void setUp() throws Exception {
super.setUp();
dropDatabase();
}
@Override
protected final String getJDBCClassName() {
return database.getDriverClass();
}
@Test
public void testMaxSizePage() throws Exception {
ActiveMQServer server = createServer(createDefaultConfig(0, true));
server.getConfiguration().getAddressSettings().clear();
server.getConfiguration().getAddressSettings().put("#", new AddressSettings().setMaxSizeMessages(1));
DatabaseStorageConfiguration dbstoreConfig = (DatabaseStorageConfiguration) server.getConfiguration().getStoreConfiguration();
dbstoreConfig.setMaxPageSizeBytes(30 * 1024);
server.start();
String addressName = "Q" + RandomUtil.randomString();
server.addAddressInfo(new AddressInfo(addressName).addRoutingType(RoutingType.ANYCAST));
server.createQueue(new QueueConfiguration(addressName).setRoutingType(RoutingType.ANYCAST).setDurable(true));
ConnectionFactory cf = CFUtil.createConnectionFactory("core", "tcp://localhost:61616");
try (Connection connection = cf.createConnection()) {
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
MessageProducer producer = session.createProducer(session.createQueue(addressName));
for (int i = 0; i < 100; i++) {
BytesMessage message = session.createBytesMessage();
message.writeBytes(new byte[1024]);
producer.send(message);
}
session.commit();
Queue queue = server.locateQueue(addressName);
Wait.assertTrue(queue.getPagingStore()::isPaging, 1000, 100);
long size = getMaxSizeBytesStored(queue);
// organically all the pages should have less than 30K
Assert.assertTrue("size is " + size, size <= dbstoreConfig.getMaxPageSizeBytes());
// will send one very large message, but bellow the streaming size. We should have one record > page_size
BytesMessage message = session.createBytesMessage();
message.writeBytes(new byte[50 * 1024]);
producer.send(message);
session.commit();
// Since we sent a large message (I mean, not streaming large, but larger than page-size,
// a page with more than 30K should been created to allow the "large" message to be stored.
size = getMaxSizeBytesStored(queue);
Assert.assertTrue("size is " + size, size >= 50 * 1024);
}
}
protected long getMaxSizeBytesStored(Queue queue) throws Exception {
String tableName = queue.getPagingStore().getFolderName();
try (java.sql.Connection sqlConn = database.getConnection()) {
String sql = null;
switch (database) {
case MSSQL:
sql = "SELECT MAX(LEN(DATA)) FROM " + tableName;
break;
case ORACLE:
case DB2:
case DERBY:
case MYSQL:
sql = "SELECT MAX(LENGTH(DATA)) FROM " + tableName;
break;
case POSTGRES:
sql = "SELECT MAX(OCTET_LENGTH(lo_get(DATA))) FROM " + tableName;
break;
}
logger.debug("query: {}", sql);
if (sql != null) {
ResultSet resultSet = null;
resultSet = sqlConn.createStatement().executeQuery(sql);
Assert.assertTrue(resultSet.next());
return resultSet.getLong(1);
} else {
return -1;
}
}
}
}

View File

@ -0,0 +1,103 @@
/*
* 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.db.paging;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.MessageProducer;
import javax.jms.Session;
import javax.jms.TextMessage;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.cli.commands.tools.PrintData;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.tests.db.common.Database;
import org.apache.activemq.artemis.tests.db.common.ParameterDBTestBase;
import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.activemq.artemis.utils.RandomUtil;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runners.Parameterized;
public class PrintDataTest extends ParameterDBTestBase {
ActiveMQServer server;
@Parameterized.Parameters(name = "db={0}")
public static Collection<Object[]> parameters() {
return convertParameters(Database.selectedList());
}
@Override
public void setUp() throws Exception {
super.setUp();
server = createServer(createDefaultConfig(0, true));
server.start();
}
@Test
public void testData() throws Exception {
String queueName = RandomUtil.randomString();
server.addAddressInfo(new AddressInfo(queueName).addRoutingType(RoutingType.ANYCAST));
Queue queue = server.createQueue(new QueueConfiguration().setAddress(queueName).setName(queueName).setDurable(true).setRoutingType(RoutingType.ANYCAST));
queue.getPagingStore().startPaging();
int numberOfMessages = 10;
ConnectionFactory cf = CFUtil.createConnectionFactory("core", "tcp://localhost:61616");
try (Connection connection = cf.createConnection()) {
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
MessageProducer producer = session.createProducer(session.createQueue(queueName));
for (int i = 0; i < numberOfMessages; i++) {
TextMessage message = session.createTextMessage("message " + i);
message.setStringProperty("i", "message " + i);
producer.send(message);
}
session.commit();
}
server.stop();
PrintData printData = new PrintData();
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
PrintStream printStream = new PrintStream(byteArrayOutputStream, true, StandardCharsets.UTF_8.name());
printData.printDataJDBC(server.getConfiguration(), printStream);
String printDataOutput = byteArrayOutputStream.toString();
for (int i = 0; i < numberOfMessages; i++) {
Assert.assertTrue(printDataOutput.lastIndexOf("message " + i) >= 0);
}
// I know this is a bit fragile, but the queues routed portion of the report was not working.
// if the report ever changes, so the test will need to be changed.
Assert.assertTrue(printDataOutput.lastIndexOf("queues routed") >= 0);
}
}

View File

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.activemq.artemis.tests.soak.paging; package org.apache.activemq.artemis.tests.db.paging;
import javax.jms.BytesMessage; import javax.jms.BytesMessage;
import javax.jms.Connection; import javax.jms.Connection;
@ -25,70 +25,45 @@ import javax.jms.MessageProducer;
import javax.jms.Queue; import javax.jms.Queue;
import javax.jms.Session; import javax.jms.Session;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.tests.soak.SoakTestBase; import org.apache.activemq.artemis.tests.db.common.Database;
import org.apache.activemq.artemis.tests.db.common.ParameterDBTestBase;
import org.apache.activemq.artemis.tests.util.CFUtil; import org.apache.activemq.artemis.tests.util.CFUtil;
import org.apache.activemq.artemis.tests.util.RandomUtil; import org.apache.activemq.artemis.utils.RandomUtil;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import static org.apache.activemq.artemis.tests.soak.TestParameters.testProperty; import static org.apache.activemq.artemis.utils.TestParameters.testProperty;
@RunWith(Parameterized.class) public class RealServerDatabasePagingTest extends ParameterDBTestBase {
public class DatabasePagingTest extends SoakTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final String TEST_NAME = "PGDB"; private static final String TEST_NAME = "PGDB";
// you can use ./start-${database}-podman.sh scripts from ./src/test/scripts to start the databases.
// support values are derby, mysql and postgres
private static final String DB_LIST = testProperty(TEST_NAME, "DB_LIST", "derby");
private static final int MAX_MESSAGES = Integer.parseInt(testProperty(TEST_NAME, "MAX_MESSAGES", "200")); private static final int MAX_MESSAGES = Integer.parseInt(testProperty(TEST_NAME, "MAX_MESSAGES", "200"));
private static final int MESSAGE_SIZE = Integer.parseInt(testProperty(TEST_NAME, "MESSAGE_SIZE", "200")); private static final int MESSAGE_SIZE = Integer.parseInt(testProperty(TEST_NAME, "MESSAGE_SIZE", "1000"));
private static final int COMMIT_INTERVAL = Integer.parseInt(testProperty(TEST_NAME, "COMMIT_INTERVAL", "100")); private static final int COMMIT_INTERVAL = Integer.parseInt(testProperty(TEST_NAME, "COMMIT_INTERVAL", "100"));
Process serverProcess; Process serverProcess;
final String database; @Parameterized.Parameters(name = "db={0}")
final String serverName;
@Parameterized.Parameters(name = "protocol={0}")
public static Collection<Object[]> parameters() { public static Collection<Object[]> parameters() {
String[] protocols = DB_LIST.split(","); return convertParameters(Database.selectedList());
ArrayList<Object[]> parameters = new ArrayList<>();
for (String str : protocols) {
logger.info("Adding {} to the list for the test", str);
parameters.add(new Object[]{str});
}
return parameters;
}
public DatabasePagingTest(String database) {
this.database = database;
serverName = "database-paging/" + database;
} }
@Before @Before
public void before() throws Exception { public void before() throws Exception {
cleanupData(serverName); serverProcess = startServer(database.getName(), 0, 60_000);
serverProcess = startServer(serverName, 0, 60_000);
} }
@ -106,7 +81,6 @@ public class DatabasePagingTest extends SoakTestBase {
ConnectionFactory connectionFactory = CFUtil.createConnectionFactory(protocol, "tcp://localhost:61616"); ConnectionFactory connectionFactory = CFUtil.createConnectionFactory(protocol, "tcp://localhost:61616");
try (Connection connection = connectionFactory.createConnection()) { try (Connection connection = connectionFactory.createConnection()) {
byte[] messageLoad = new byte[MESSAGE_SIZE]; byte[] messageLoad = new byte[MESSAGE_SIZE];
Session session = connection.createSession(true, Session.SESSION_TRANSACTED); Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
@ -129,8 +103,7 @@ public class DatabasePagingTest extends SoakTestBase {
serverProcess.waitFor(1, TimeUnit.MINUTES); serverProcess.waitFor(1, TimeUnit.MINUTES);
Assert.assertFalse(serverProcess.isAlive()); Assert.assertFalse(serverProcess.isAlive());
serverProcess = startServer(serverName, 0, 60_000); serverProcess = startServer(database.getName(), 0, 60_000);
try (Connection connection = connectionFactory.createConnection()) { try (Connection connection = connectionFactory.createConnection()) {
connection.start(); connection.start();

View File

@ -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.activemq.artemis.tests.db.paging;
import java.util.Collection;
import org.apache.activemq.artemis.core.config.Configuration;
import org.apache.activemq.artemis.core.config.storage.DatabaseStorageConfiguration;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.logs.AssertionLoggerHandler;
import org.apache.activemq.artemis.tests.db.common.Database;
import org.apache.activemq.artemis.tests.db.common.ParameterDBTestBase;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runners.Parameterized;
public class SchemaValidationTest extends ParameterDBTestBase {
@Parameterized.Parameters(name = "db={0}")
public static Collection<Object[]> parameters() {
return convertParameters(Database.selectedList());
}
@Test
public void testTableNameTooLong() throws Exception {
try (AssertionLoggerHandler loggerHandler = new AssertionLoggerHandler(true)) {
final Configuration config = createDefaultInVMConfig();
final DatabaseStorageConfiguration storageConfiguration = (DatabaseStorageConfiguration) config.getStoreConfiguration();
//set the page store table to be longer than 10 chars -> the paging manager initialization will fail
storageConfiguration.setPageStoreTableName("PAGE_STORE_");
final int PAGE_MAX = 20 * 1024;
final int PAGE_SIZE = 10 * 1024;
final ActiveMQServer server = createServer(true, config, PAGE_SIZE, PAGE_MAX);
server.start();
//due to a failed initialisation of the paging manager, it must be null
Assert.assertNull(server.getPagingManager());
server.stop();
Assert.assertTrue(loggerHandler.findText("AMQ224000")); // Failure in initialization
}
}
}

View File

@ -40,7 +40,6 @@ import org.apache.activemq.artemis.api.core.client.ServerLocator;
import org.apache.activemq.artemis.api.core.management.ManagementHelper; import org.apache.activemq.artemis.api.core.management.ManagementHelper;
import org.apache.activemq.artemis.core.client.impl.ClientMessageImpl; import org.apache.activemq.artemis.core.client.impl.ClientMessageImpl;
import org.apache.activemq.artemis.core.config.Configuration; import org.apache.activemq.artemis.core.config.Configuration;
import org.apache.activemq.artemis.core.config.StoreConfiguration;
import org.apache.activemq.artemis.core.paging.PagingManager; import org.apache.activemq.artemis.core.paging.PagingManager;
import org.apache.activemq.artemis.core.server.ActiveMQServer; import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.MessageReference; import org.apache.activemq.artemis.core.server.MessageReference;
@ -50,19 +49,10 @@ 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.AddressSettings;
import org.apache.activemq.artemis.tests.util.Wait; import org.apache.activemq.artemis.tests.util.Wait;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Assume;
import org.junit.Before; import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@RunWith(Parameterized.class) public class GlobalJournalPagingTest extends JournalPagingTest {
public class GlobalPagingTest extends PagingTest {
public GlobalPagingTest(StoreConfiguration.StoreType storeType) {
super(storeType);
}
@Override @Override
@Before @Before
@ -95,20 +85,13 @@ public class GlobalPagingTest extends PagingTest {
server.getAddressSettingsRepository().addMatch("#", defaultSetting); server.getAddressSettingsRepository().addMatch("#", defaultSetting);
} }
// test doesn't make sense on GlobalPaging due to configuration issues
@Test @Ignore @Override
public void testPurge() throws Exception {
}
@Test @Test
public void testPagingOverFullDisk() throws Exception { public void testPagingOverFullDisk() throws Exception {
Assume.assumeTrue(storeType != StoreConfiguration.StoreType.DATABASE);
clearDataRecreateServerDirs(); clearDataRecreateServerDirs();
Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false); Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false);
server = createServer(true, config, PagingTest.PAGE_SIZE, PagingTest.PAGE_MAX, -1, -1, new HashMap<>()); server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX, -1, -1, new HashMap<>());
Assert.assertTrue(customServerCreated); Assert.assertTrue(customServerCreated);
server.getConfiguration().setGlobalMaxSize(-1); server.getConfiguration().setGlobalMaxSize(-1);
server.getConfiguration().setAddressQueueScanPeriod(100); server.getConfiguration().setAddressQueueScanPeriod(100);
@ -127,9 +110,9 @@ public class GlobalPagingTest extends PagingTest {
final ClientSession session = sf.createSession(false, false, false); final ClientSession session = sf.createSession(false, false, false);
session.createQueue(new QueueConfiguration(PagingTest.ADDRESS)); session.createQueue(new QueueConfiguration(JournalPagingTest.ADDRESS));
final ClientProducer producer = session.createProducer(PagingTest.ADDRESS); final ClientProducer producer = session.createProducer(JournalPagingTest.ADDRESS);
ClientMessage message = null; ClientMessage message = null;
@ -182,7 +165,7 @@ public class GlobalPagingTest extends PagingTest {
// The consumer has to be created after the getMessageCount(queue) assertion // The consumer has to be created after the getMessageCount(queue) assertion
// otherwise delivery could alter the messagecount and give us a false failure // otherwise delivery could alter the messagecount and give us a false failure
ClientConsumer consumer = session.createConsumer(PagingTest.ADDRESS); ClientConsumer consumer = session.createConsumer(JournalPagingTest.ADDRESS);
ClientMessage msg = null; ClientMessage msg = null;
for (int i = 0; i < numberOfMessages * 2; i++) { for (int i = 0; i < numberOfMessages * 2; i++) {
@ -224,7 +207,7 @@ public class GlobalPagingTest extends PagingTest {
Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false); Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false);
final ActiveMQServer server = createServer(true, config, PagingTest.PAGE_SIZE, -1); final ActiveMQServer server = createServer(true, config, JournalPagingTest.PAGE_SIZE, -1);
try { try {
final SimpleString managementAddress = server.getConfiguration().getManagementAddress(); final SimpleString managementAddress = server.getConfiguration().getManagementAddress();
@ -311,7 +294,7 @@ public class GlobalPagingTest extends PagingTest {
Configuration config = createDefaultNettyConfig().setJournalSyncNonTransactional(false); Configuration config = createDefaultNettyConfig().setJournalSyncNonTransactional(false);
final ActiveMQServer server = createServer(true, config, PagingTest.PAGE_SIZE, -1); final ActiveMQServer server = createServer(true, config, JournalPagingTest.PAGE_SIZE, -1);
try { try {
final SimpleString managementAddress = server.getConfiguration().getManagementAddress(); final SimpleString managementAddress = server.getConfiguration().getManagementAddress();

View File

@ -0,0 +1,762 @@
/*
* 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.paging;
import javax.jms.Connection;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Session;
import javax.transaction.xa.XAResource;
import javax.transaction.xa.Xid;
import java.io.File;
import java.io.FileOutputStream;
import java.io.PrintWriter;
import java.lang.invoke.MethodHandles;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.client.ClientConsumer;
import org.apache.activemq.artemis.api.core.client.ClientMessage;
import org.apache.activemq.artemis.api.core.client.ClientProducer;
import org.apache.activemq.artemis.api.core.client.ClientSession;
import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
import org.apache.activemq.artemis.api.core.client.ServerLocator;
import org.apache.activemq.artemis.core.config.Configuration;
import org.apache.activemq.artemis.core.paging.PagedMessage;
import org.apache.activemq.artemis.core.paging.impl.Page;
import org.apache.activemq.artemis.core.paging.impl.PageTransactionInfoImpl;
import org.apache.activemq.artemis.core.paging.impl.PagingStoreFactoryNIO;
import org.apache.activemq.artemis.core.paging.impl.PagingStoreImpl;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
import org.apache.activemq.artemis.core.server.impl.QueueImpl;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
import org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory;
import org.apache.activemq.artemis.logs.AssertionLoggerHandler;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.tests.util.RandomUtil;
import org.apache.activemq.artemis.tests.util.Wait;
import org.apache.activemq.artemis.utils.RetryRule;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class JournalPagingTest extends ActiveMQTestBase {
protected static final int PAGE_MAX = 100 * 1024;
protected static final int PAGE_SIZE = 10 * 1024;
static final int MESSAGE_SIZE = 1024; // 1k
static final SimpleString ADDRESS = new SimpleString("SimpleAddress");
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Rule
public RetryRule retryMethod = new RetryRule(1);
protected ServerLocator locator;
protected ActiveMQServer server;
protected ClientSessionFactory sf;
private AssertionLoggerHandler loggerHandler;
@Before
public void checkLoggerStart() throws Exception {
loggerHandler = new AssertionLoggerHandler();
}
@After
public void checkLoggerEnd() throws Exception {
try {
// These are the message errors for the negative size address size
Assert.assertFalse(loggerHandler.findText("222214"));
Assert.assertFalse(loggerHandler.findText("222215"));
} finally {
loggerHandler.close();
}
}
@Override
@Before
public void setUp() throws Exception {
super.setUp();
locator = createInVMNonHALocator();
}
@Test
public void testPageCleanupWithInvalidDataTruncated() throws Exception {
testPageCleanupWithInvalidData(true);
}
@Test
public void testPageCleanupWithInvalidData() throws Exception {
testPageCleanupWithInvalidData(false);
}
public void testPageCleanupWithInvalidData(boolean truncated) throws Exception {
clearDataRecreateServerDirs();
Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false);
server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX);
server.start();
final int numberOfMessages = 100;
locator = createInVMNonHALocator().setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true);
sf = createSessionFactory(locator);
ClientSession session = sf.createSession(false, true, true);
session.createQueue(new QueueConfiguration(JournalPagingTest.ADDRESS));
ClientProducer producer = session.createProducer(JournalPagingTest.ADDRESS);
ClientMessage message = null;
byte[] body = new byte[10];
ByteBuffer bb = ByteBuffer.wrap(body);
for (int j = 1; j <= 10; j++) {
bb.put(getSamplebyte(j));
}
Queue queue = server.locateQueue(ADDRESS);
queue.getPagingStore().startPaging();
queue.getPagingStore().forceAnotherPage(); // forcing an empty file, just to make it more challenging
int page = 1;
for (int i = 0; i < numberOfMessages; i++) {
if (i % 10 == 0 && i > 0) {
queue.getPagingStore().forceAnotherPage();
page++;
}
message = session.createMessage(true);
ActiveMQBuffer bodyLocal = message.getBodyBuffer();
bodyLocal.writeBytes(body);
message.putIntProperty("i", i);
message.putIntProperty("page", page);
producer.send(message);
}
queue.getPagingStore().getCursorProvider().disableCleanup();
ClientConsumer consumer = session.createConsumer(ADDRESS);
session.start();
for (int i = 0; i < 11; i++) {
ClientMessage msgRec = consumer.receive(1000);
Assert.assertNotNull(msgRec);
msgRec.acknowledge();
}
session.commit();
consumer.close();
consumer = session.createConsumer(ADDRESS, SimpleString.toSimpleString("i=29"));
message = consumer.receive(5000);
Assert.assertNotNull(message);
message.acknowledge();
session.commit();
File folder = queue.getPagingStore().getFolder();
// We will truncate two files
for (int f = 2; f <= 3; f++) {
String fileName = ((PagingStoreImpl) queue.getPagingStore()).createFileName(f);
File file = new File(folder, fileName);
file.delete();
file.createNewFile();
if (!truncated) {
FileOutputStream fileOutputStream = new FileOutputStream(file);
fileOutputStream.write(new byte[10]);
fileOutputStream.close();
}
}
sf.close();
server.getStorageManager().getMessageJournal().scheduleCompactAndBlock(5000);
Page page4 = queue.getPagingStore().newPageObject(4);
page4.open(true);
org.apache.activemq.artemis.utils.collections.LinkedList<PagedMessage> messagesRead = page4.read(server.getStorageManager());
Assert.assertEquals(10, messagesRead.size());
page4.close(false);
page4.delete(null);
page4.open(true);
for (int i = 0; i < 9; i++) {
page4.write(messagesRead.get(i)); // this will make message 29 disappear
}
page4.close(false);
server.stop();
server.start();
queue = server.locateQueue(ADDRESS);
Assert.assertTrue(queue.getPagingStore().isPaging());
queue.getPageSubscription().enableAutoCleanup(); // this should been true already as the server was restarted, just braces and belts
sf = createSessionFactory(locator);
session = sf.createSession(false, true, true);
logger.info("*******************************************************************************************************************************");
logger.info("Creating consumer");
consumer = session.createConsumer(ADDRESS);
session.start();
for (int i = 20; i < numberOfMessages; i++) { // I made one message disappear on page 4
if (i != 29) { // I made message 29 disappear
ClientMessage msgClient = consumer.receive(1000);
Assert.assertNotNull(msgClient);
Assert.assertEquals(i, msgClient.getIntProperty("i").intValue());
msgClient.acknowledge();
}
}
ClientMessage msgClient = consumer.receiveImmediate();
Assert.assertNull(msgClient);
session.commit();
Wait.assertFalse(queue.getPagingStore()::isPaging, 5000, 100);
}
@Test
public void testEmptyAddress() throws Exception {
clearDataRecreateServerDirs();
Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false);
server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX);
server.start();
final int numberOfMessages = 500;
locator = createInVMNonHALocator().setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true);
sf = createSessionFactory(locator);
ClientSession session = sf.createSession(false, false, false);
session.createQueue(new QueueConfiguration(JournalPagingTest.ADDRESS).setRoutingType(RoutingType.ANYCAST));
ClientProducer producer = session.createProducer(JournalPagingTest.ADDRESS);
byte[] body = new byte[MESSAGE_SIZE];
ByteBuffer bb = ByteBuffer.wrap(body);
for (int j = 1; j <= MESSAGE_SIZE; j++) {
bb.put(getSamplebyte(j));
}
for (int i = 0; i < numberOfMessages; i++) {
ClientMessage message = session.createMessage(true);
message.getBodyBuffer().writeBytes(body);
producer.send(message);
if (i % 1000 == 0) {
session.commit();
}
}
session.commit();
producer.close();
session.close();
String addressTxt = server.getPagingManager().getPageStore(JournalPagingTest.ADDRESS).getFolder().getAbsolutePath() + File.separator + PagingStoreFactoryNIO.ADDRESS_FILE;
server.stop();
new PrintWriter(addressTxt).close();
Assert.assertTrue(new File(addressTxt).exists());
final AtomicBoolean activationFailures = new AtomicBoolean();
server.registerActivationFailureListener(exception -> activationFailures.set(true));
server.start();
server.stop();
assertFalse(activationFailures.get());
}
@Test
public void testPurge() throws Exception {
clearDataRecreateServerDirs();
Configuration config = createDefaultNettyConfig().setJournalSyncNonTransactional(false);
server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX);
server.start();
SimpleString queue = new SimpleString("testPurge:" + RandomUtil.randomString());
server.addAddressInfo(new AddressInfo(queue, RoutingType.ANYCAST));
QueueImpl purgeQueue = (QueueImpl) server.createQueue(new QueueConfiguration(queue).setRoutingType(RoutingType.ANYCAST).setMaxConsumers(1).setPurgeOnNoConsumers(true).setAutoCreateAddress(false));
ActiveMQConnectionFactory cf = new ActiveMQConnectionFactory();
Connection connection = cf.createConnection();
Session session = connection.createSession(true, Session.SESSION_TRANSACTED);
javax.jms.Queue jmsQueue = session.createQueue(queue.toString());
MessageProducer producer = session.createProducer(jmsQueue);
for (int i = 0; i < 100; i++) {
producer.send(session.createTextMessage("hello" + i));
}
session.commit();
Wait.assertEquals(0, purgeQueue::getMessageCount);
Wait.assertEquals(0, purgeQueue.getPageSubscription().getPagingStore()::getAddressSize);
MessageConsumer consumer = session.createConsumer(jmsQueue);
for (int i = 0; i < 100; i++) {
producer.send(session.createTextMessage("hello" + i));
if (i == 10) {
purgeQueue.getPageSubscription().getPagingStore().startPaging();
}
}
session.commit();
consumer.close();
Wait.assertEquals(0, purgeQueue::getMessageCount);
Wait.assertFalse(purgeQueue.getPageSubscription()::isPaging);
Wait.assertEquals(0, purgeQueue.getPageSubscription().getPagingStore()::getAddressSize);
consumer = session.createConsumer(jmsQueue);
for (int i = 0; i < 100; i++) {
purgeQueue.getPageSubscription().getPagingStore().startPaging();
Assert.assertTrue(purgeQueue.getPageSubscription().isPaging());
producer.send(session.createTextMessage("hello" + i));
if (i % 2 == 0) {
session.commit();
}
}
session.commit();
Wait.assertTrue(purgeQueue.getPageSubscription()::isPaging);
connection.start();
server.getStorageManager().getMessageJournal().scheduleCompactAndBlock(50000);
Assert.assertNotNull(consumer.receive(5000));
session.commit();
consumer.close();
Wait.assertEquals(0, purgeQueue::getMessageCount);
Wait.assertEquals(0, purgeQueue.getPageSubscription().getPagingStore()::getAddressSize);
Wait.assertFalse(purgeQueue.getPageSubscription()::isPaging, 5000, 100);
StorageManager sm = server.getStorageManager();
for (int i = 0; i < 1000; i++) {
long tx = sm.generateID();
PageTransactionInfoImpl txinfo = new PageTransactionInfoImpl(tx);
sm.storePageTransaction(tx, txinfo);
sm.commit(tx);
tx = sm.generateID();
sm.updatePageTransaction(tx, txinfo, 1);
sm.commit(tx);
}
server.stop();
server.start();
Wait.assertEquals(0, () -> server.getPagingManager().getTransactions().size());
}
// First page is complete but it wasn't deleted
@Test
public void testPreparedACKRemoveAndRestart() throws Exception {
clearDataRecreateServerDirs();
Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false);
server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX);
server.start();
final int numberOfMessages = 10;
locator = createInVMNonHALocator().setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true).setAckBatchSize(0);
sf = createSessionFactory(locator);
ClientSession session = sf.createSession(false, true, true);
session.createQueue(new QueueConfiguration(JournalPagingTest.ADDRESS));
Queue queue = server.locateQueue(JournalPagingTest.ADDRESS);
ClientProducer producer = session.createProducer(JournalPagingTest.ADDRESS);
byte[] body = new byte[MESSAGE_SIZE];
ByteBuffer bb = ByteBuffer.wrap(body);
for (int j = 1; j <= MESSAGE_SIZE; j++) {
bb.put(getSamplebyte(j));
}
queue.getPageSubscription().getPagingStore().startPaging();
forcePage(queue);
for (int i = 0; i < numberOfMessages; i++) {
ClientMessage message = session.createMessage(true);
message.putIntProperty("count", i);
ActiveMQBuffer bodyLocal = message.getBodyBuffer();
bodyLocal.writeBytes(body);
producer.send(message);
if (i == 4) {
session.commit();
queue.getPageSubscription().getPagingStore().forceAnotherPage();
}
}
session.commit();
session.close();
session = sf.createSession(true, false, false);
ClientConsumer cons = session.createConsumer(ADDRESS);
session.start();
for (int i = 0; i <= 4; i++) {
Xid xidConsumeNoCommit = newXID();
session.start(xidConsumeNoCommit, XAResource.TMNOFLAGS);
// First message is consumed, prepared, will be rolled back later
ClientMessage firstMessageConsumed = cons.receive(5000);
assertNotNull(firstMessageConsumed);
firstMessageConsumed.acknowledge();
session.end(xidConsumeNoCommit, XAResource.TMSUCCESS);
session.prepare(xidConsumeNoCommit);
}
File pagingFolder = queue.getPageSubscription().getPagingStore().getFolder();
server.stop();
// remove the very first page. a restart should not fail
File fileToRemove = new File(pagingFolder, "000000001.page");
Assert.assertTrue(fileToRemove.delete());
server.start();
sf = createSessionFactory(locator);
session = sf.createSession(false, true, true);
cons = session.createConsumer(ADDRESS);
session.start();
for (int i = 5; i < numberOfMessages; i++) {
ClientMessage message = cons.receive(1000);
assertNotNull(message);
assertEquals(i, message.getIntProperty("count").intValue());
message.acknowledge();
}
assertNull(cons.receiveImmediate());
session.commit();
}
/**
* @param queue
* @throws InterruptedException
*/
private void forcePage(Queue queue) throws InterruptedException {
for (long timeout = System.currentTimeMillis() + 5000; timeout > System.currentTimeMillis() && !queue.getPageSubscription().getPagingStore().isPaging(); ) {
Thread.sleep(10);
}
assertTrue(queue.getPageSubscription().getPagingStore().isPaging());
}
@Test
public void testInabilityToCreateDirectoryDuringPaging() throws Exception {
try (AssertionLoggerHandler loggerHandler = new AssertionLoggerHandler()) {
clearDataRecreateServerDirs();
Configuration config = createDefaultInVMConfig().setJournalSyncNonTransactional(false).setPagingDirectory("/" + UUID.randomUUID().toString());
server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX);
server.start();
final int numberOfMessages = 100;
locator = createInVMNonHALocator().setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true);
sf = createSessionFactory(locator);
ClientSession session = sf.createSession(false, true, true);
session.createQueue(new QueueConfiguration(JournalPagingTest.ADDRESS));
ClientProducer producer = session.createProducer(JournalPagingTest.ADDRESS);
ClientMessage message = null;
byte[] body = new byte[MESSAGE_SIZE];
ByteBuffer bb = ByteBuffer.wrap(body);
for (int j = 1; j <= MESSAGE_SIZE; j++) {
bb.put(getSamplebyte(j));
}
for (int i = 0; i < numberOfMessages; i++) {
message = session.createMessage(true);
ActiveMQBuffer bodyLocal = message.getBodyBuffer();
bodyLocal.writeBytes(body);
message.putIntProperty(new SimpleString("id"), i);
try {
producer.send(message);
} catch (Exception e) {
// ignore
}
}
assertTrue(Wait.waitFor(() -> server.getState() == ActiveMQServer.SERVER_STATE.STOPPED, 5000, 200));
session.close();
sf.close();
locator.close();
} finally {
Assert.assertTrue(loggerHandler.findText("AMQ144010"));
}
}
/**
* This test will remove all the page directories during a restart, simulating a crash scenario. The server should still start after this
*/
@Test
public void testDeletePhysicalPages() throws Exception {
clearDataRecreateServerDirs();
Configuration config = createDefaultInVMConfig().setPersistDeliveryCountBeforeDelivery(true);
config.setJournalSyncNonTransactional(false);
server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX);
server.start();
final int numberOfMessages = 300;
locator = createInVMNonHALocator().setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true);
sf = createSessionFactory(locator);
ClientSession session = sf.createSession(false, false, false);
session.createQueue(new QueueConfiguration(JournalPagingTest.ADDRESS));
ClientProducer producer = session.createProducer(JournalPagingTest.ADDRESS);
ClientMessage message = null;
byte[] body = new byte[MESSAGE_SIZE];
ByteBuffer bb = ByteBuffer.wrap(body);
for (int j = 1; j <= MESSAGE_SIZE; j++) {
bb.put(getSamplebyte(j));
}
for (int i = 0; i < numberOfMessages; i++) {
message = session.createMessage(true);
ActiveMQBuffer bodyLocal = message.getBodyBuffer();
bodyLocal.writeBytes(body);
message.putIntProperty(new SimpleString("id"), i);
producer.send(message);
if (i % 1000 == 0) {
session.commit();
}
}
session.commit();
session.close();
session = null;
sf.close();
locator.close();
server.stop();
server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX);
server.start();
locator = createInVMNonHALocator();
sf = createSessionFactory(locator);
Queue queue = server.locateQueue(ADDRESS);
Wait.assertEquals(numberOfMessages, queue::getMessageCount);
ClientSession sessionConsumer = sf.createSession(false, false, false);
sessionConsumer.start();
ClientConsumer consumer = sessionConsumer.createConsumer(JournalPagingTest.ADDRESS);
for (int msgCount = 0; msgCount < numberOfMessages; msgCount++) {
logger.debug("Received {}", msgCount);
ClientMessage msg = consumer.receive(100);
if (msg == null) {
logger.debug("It's null. leaving now");
sessionConsumer.commit();
fail("Didn't receive a message");
}
msg.acknowledge();
if (msgCount % 5 == 0) {
logger.debug("commit");
sessionConsumer.commit();
}
}
sessionConsumer.commit();
sessionConsumer.close();
sf.close();
locator.close();
Wait.assertEquals(0, queue::getMessageCount);
Wait.assertFalse(queue.getPagingStore()::isPaging, 1000, 100);
server.stop();
// Deleting the paging data. Simulating a failure
// a dumb user, or anything that will remove the data
deleteDirectory(new File(getPageDir()));
server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX);
server.start();
locator = createInVMNonHALocator().setBlockOnNonDurableSend(true).setBlockOnDurableSend(true).setBlockOnAcknowledge(true);
sf = createSessionFactory(locator);
queue = server.locateQueue(ADDRESS);
sf = createSessionFactory(locator);
session = sf.createSession(false, false, false);
producer = session.createProducer(JournalPagingTest.ADDRESS);
for (int i = 0; i < numberOfMessages * 2; i++) {
message = session.createMessage(true);
ActiveMQBuffer bodyLocal = message.getBodyBuffer();
bodyLocal.writeBytes(body);
message.putIntProperty(new SimpleString("theid"), i);
producer.send(message);
if (i % 1000 == 0) {
session.commit();
}
}
session.commit();
server.stop();
server = createServer(true, config, JournalPagingTest.PAGE_SIZE, JournalPagingTest.PAGE_MAX);
server.start();
locator = createInVMNonHALocator();
sf = createSessionFactory(locator);
sessionConsumer = sf.createSession(false, false, false);
sessionConsumer.start();
consumer = sessionConsumer.createConsumer(JournalPagingTest.ADDRESS);
for (int msgCount = 0; msgCount < numberOfMessages; msgCount++) {
logger.debug("Received {}", msgCount);
ClientMessage msg = consumer.receive(100);
if (msg == null) {
logger.debug("It's null. leaving now");
sessionConsumer.commit();
fail("Didn't receive a message");
}
msg.acknowledge();
if (msgCount % 5 == 0) {
logger.debug("commit");
sessionConsumer.commit();
}
}
sessionConsumer.commit();
sessionConsumer.close();
}
@Override
protected void applySettings(ActiveMQServer server,
final Configuration configuration,
final int pageSize,
final long maxAddressSize,
final Integer maxReadPageMessages,
final Integer maxReadPageBytes,
final Map<String, AddressSettings> settings) {
server.getConfiguration().setAddressQueueScanPeriod(100);
}
}

View File

@ -56,6 +56,7 @@ import org.apache.activemq.artemis.jms.client.ActiveMQJMSConnectionFactory;
import org.apache.activemq.artemis.jms.server.impl.JMSServerManagerImpl; import org.apache.activemq.artemis.jms.server.impl.JMSServerManagerImpl;
import org.apache.activemq.artemis.tests.unit.util.InVMNamingContext; import org.apache.activemq.artemis.tests.unit.util.InVMNamingContext;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase; import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.tests.util.Wait;
import org.junit.Test; import org.junit.Test;
/** /**
@ -69,7 +70,7 @@ public class PagingOrderTest extends ActiveMQTestBase {
private static final int PAGE_SIZE = 10 * 1024; private static final int PAGE_SIZE = 10 * 1024;
static final SimpleString ADDRESS = new SimpleString("SimpleAddress"); static final SimpleString ADDRESS = new SimpleString("TestQueue");
private Connection conn; private Connection conn;
@ -96,7 +97,7 @@ public class PagingOrderTest extends ActiveMQTestBase {
server.addAddressInfo(new AddressInfo(ADDRESS, RoutingType.ANYCAST)); server.addAddressInfo(new AddressInfo(ADDRESS, RoutingType.ANYCAST));
server.createQueue(new QueueConfiguration(ADDRESS).setRoutingType(RoutingType.ANYCAST)); server.createQueue(new QueueConfiguration(ADDRESS).setRoutingType(RoutingType.ANYCAST));
ClientProducer producer = session.createProducer(PagingTest.ADDRESS); ClientProducer producer = session.createProducer(ADDRESS);
byte[] body = new byte[messageSize]; byte[] body = new byte[messageSize];
@ -188,7 +189,7 @@ public class PagingOrderTest extends ActiveMQTestBase {
Queue q2 = server.createQueue(new QueueConfiguration(new SimpleString("inactive")).setAddress(ADDRESS).setRoutingType(RoutingType.MULTICAST)); Queue q2 = server.createQueue(new QueueConfiguration(new SimpleString("inactive")).setAddress(ADDRESS).setRoutingType(RoutingType.MULTICAST));
ClientProducer producer = session.createProducer(PagingTest.ADDRESS); ClientProducer producer = session.createProducer(ADDRESS);
byte[] body = new byte[messageSize]; byte[] body = new byte[messageSize];
@ -285,11 +286,15 @@ public class PagingOrderTest extends ActiveMQTestBase {
assertNotNull(q2); assertNotNull(q2);
assertEquals("q2 msg count", numberOfMessages, getMessageCount(q2)); {
assertEquals("q2 msgs added", numberOfMessages, getMessagesAdded(q2)); Queue finalQ2 = q2;
assertEquals("q1 msg count", 0, getMessageCount(q1)); Queue finalQ1 = q1;
Wait.assertEquals(numberOfMessages, () -> getMessageCount(finalQ2), 5000);
Wait.assertEquals(numberOfMessages, () -> getMessagesAdded(finalQ2), 5000);
Wait.assertEquals(0, () -> getMessageCount(finalQ1));
// 0, since nothing was sent to the queue after the server was restarted // 0, since nothing was sent to the queue after the server was restarted
assertEquals("q1 msgs added", 0, getMessagesAdded(q1)); Wait.assertEquals(0, () -> getMessagesAdded(finalQ1));
}
} }
@ -318,7 +323,7 @@ public class PagingOrderTest extends ActiveMQTestBase {
Queue q2 = server.createQueue(new QueueConfiguration(new SimpleString("inactive")).setAddress(ADDRESS).setRoutingType(RoutingType.MULTICAST)); Queue q2 = server.createQueue(new QueueConfiguration(new SimpleString("inactive")).setAddress(ADDRESS).setRoutingType(RoutingType.MULTICAST));
ClientProducer producer = session.createProducer(PagingTest.ADDRESS); ClientProducer producer = session.createProducer(ADDRESS);
byte[] body = new byte[messageSize]; byte[] body = new byte[messageSize];
@ -410,7 +415,7 @@ public class PagingOrderTest extends ActiveMQTestBase {
server.addAddressInfo(new AddressInfo(ADDRESS, RoutingType.ANYCAST)); server.addAddressInfo(new AddressInfo(ADDRESS, RoutingType.ANYCAST));
server.createQueue(new QueueConfiguration(ADDRESS).setRoutingType(RoutingType.ANYCAST)); server.createQueue(new QueueConfiguration(ADDRESS).setRoutingType(RoutingType.ANYCAST));
ClientProducer producer = session.createProducer(PagingTest.ADDRESS); ClientProducer producer = session.createProducer(ADDRESS);
byte[] body = new byte[messageSize]; byte[] body = new byte[messageSize];
@ -495,7 +500,7 @@ public class PagingOrderTest extends ActiveMQTestBase {
server.addAddressInfo(new AddressInfo(ADDRESS, RoutingType.ANYCAST)); server.addAddressInfo(new AddressInfo(ADDRESS, RoutingType.ANYCAST));
QueueImpl queue = (QueueImpl) server.createQueue(new QueueConfiguration(ADDRESS).setRoutingType(RoutingType.ANYCAST)); QueueImpl queue = (QueueImpl) server.createQueue(new QueueConfiguration(ADDRESS).setRoutingType(RoutingType.ANYCAST));
ClientProducer producer = session.createProducer(PagingTest.ADDRESS); ClientProducer producer = session.createProducer(ADDRESS);
byte[] body = new byte[messageSize]; byte[] body = new byte[messageSize];

View File

@ -48,7 +48,7 @@ public class PagingSyncTest extends ActiveMQTestBase {
static final SimpleString ADDRESS = new SimpleString("SimpleAddress"); static final SimpleString ADDRESS = new SimpleString("TestQueue");
@Test @Test
public void testOrder1() throws Throwable { public void testOrder1() throws Throwable {
@ -73,7 +73,7 @@ public class PagingSyncTest extends ActiveMQTestBase {
server.addAddressInfo(new AddressInfo(ADDRESS, RoutingType.ANYCAST)); server.addAddressInfo(new AddressInfo(ADDRESS, RoutingType.ANYCAST));
server.createQueue(new QueueConfiguration(ADDRESS).setRoutingType(RoutingType.ANYCAST)); server.createQueue(new QueueConfiguration(ADDRESS).setRoutingType(RoutingType.ANYCAST));
ClientProducer producer = session.createProducer(PagingTest.ADDRESS); ClientProducer producer = session.createProducer(ADDRESS);
byte[] body = new byte[messageSize]; byte[] body = new byte[messageSize];

View File

@ -387,7 +387,7 @@ public class PersistMultiThreadTest extends ActiveMQTestBase {
} }
@Override @Override
public void sync() throws Exception { public void addSyncPoint(OperationContext context) throws Exception {
} }

View File

@ -141,5 +141,6 @@
<module>smoke-tests</module> <module>smoke-tests</module>
<module>e2e-tests</module> <module>e2e-tests</module>
<module>leak-tests</module> <module>leak-tests</module>
<module>db-tests</module>
</modules> </modules>
</project> </project>

View File

@ -29,7 +29,7 @@
<properties> <properties>
<activemq.basedir>${project.basedir}/../../</activemq.basedir> <activemq.basedir>${project.basedir}/../../</activemq.basedir>
<sts-surefire-extra-args /> <sts-surefire-extra-args />
<artemis-distribuiton-lib-dir>-Ddistribution.lib="${activemq.basedir}/artemis-distribution/target/apache-artemis-${project.version}-bin/apache-artemis-${project.version}/lib"</artemis-distribuiton-lib-dir> <artemis-distribution-lib-dir>-Ddistribution.lib="${activemq.basedir}/artemis-distribution/target/apache-artemis-${project.version}-bin/apache-artemis-${project.version}/lib"</artemis-distribution-lib-dir>
<sts-http-host>localhost</sts-http-host> <sts-http-host>localhost</sts-http-host>
</properties> </properties>
@ -1553,7 +1553,7 @@
<forkCount>1</forkCount> <forkCount>1</forkCount>
<reuseForks>false</reuseForks> <reuseForks>false</reuseForks>
<skipTests>${skipSmokeTests}</skipTests> <skipTests>${skipSmokeTests}</skipTests>
<argLine>${sts-surefire-extra-args} ${activemq-surefire-argline} ${artemis-distribuiton-lib-dir}</argLine> <argLine>${sts-surefire-extra-args} ${activemq-surefire-argline} ${artemis-distribution-lib-dir}</argLine>
</configuration> </configuration>
</plugin> </plugin>
</plugins> </plugins>

View File

@ -16,178 +16,7 @@
*/ */
package org.apache.activemq.artemis.tests.smoke.common; package org.apache.activemq.artemis.tests.smoke.common;
import javax.management.MBeanServerInvocationHandler; import org.apache.activemq.artemis.utils.RealServerTestBase;
import javax.management.remote.JMXConnector;
import javax.management.remote.JMXConnectorFactory;
import javax.management.remote.JMXServiceURL;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.net.MalformedURLException;
import java.util.HashSet;
import java.util.Set;
import org.apache.activemq.artemis.api.core.management.ActiveMQServerControl;
import org.apache.activemq.artemis.api.core.management.ObjectNameBuilder;
import org.apache.activemq.artemis.cli.commands.Stop;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.util.ServerUtil;
import org.junit.After;
import org.junit.Assert;
public class SmokeTestBase extends ActiveMQTestBase {
Set<Process> processes = new HashSet<>();
private static final String JMX_SERVER_HOSTNAME = "localhost";
private static final int JMX_SERVER_PORT = 10099;
public static final String basedir = System.getProperty("basedir");
@After
public void after() throws Exception {
// close ServerLocators before killing the server otherwise they'll hang and delay test termination
closeAllServerLocatorsFactories();
for (Process process : processes) {
try {
ServerUtil.killServer(process, true);
} catch (Throwable e) {
e.printStackTrace();
}
}
processes.clear();
}
public void killServer(Process process) {
processes.remove(process);
try {
ServerUtil.killServer(process);
} catch (Throwable e) {
e.printStackTrace();
}
}
protected static void stopServerWithFile(String serverLocation) throws IOException {
File serverPlace = new File(serverLocation);
File etcPlace = new File(serverPlace, "etc");
File stopMe = new File(etcPlace, Stop.STOP_FILE_NAME);
Assert.assertTrue(stopMe.createNewFile());
}
public static String getServerLocation(String serverName) {
return basedir + "/target/" + serverName;
}
public static void cleanupData(String serverName) {
String location = getServerLocation(serverName);
deleteDirectory(new File(location, "data"));
deleteDirectory(new File(location, "log"));
}
public void addProcess(Process process) {
processes.add(process);
}
public void removeProcess(Process process) {
processes.remove(process);
}
public Process startServer(String serverName, int portID, int timeout) throws Exception {
Process process = ServerUtil.startServer(getServerLocation(serverName), serverName, portID, timeout);
addProcess(process);
return process;
}
public Process startServer(String serverName, String uri, int timeout) throws Exception {
Process process = ServerUtil.startServer(getServerLocation(serverName), serverName, uri, timeout);
addProcess(process);
return process;
}
protected JMXConnector getJmxConnector() throws MalformedURLException {
return getJmxConnector(JMX_SERVER_HOSTNAME, JMX_SERVER_PORT);
}
protected static JMXConnector newJMXFactory(String uri) throws Throwable {
return JMXConnectorFactory.connect(new JMXServiceURL(uri));
}
protected static ActiveMQServerControl getServerControl(String uri,
ObjectNameBuilder builder,
long timeout) throws Throwable {
long expireLoop = System.currentTimeMillis() + timeout;
Throwable lastException = null;
do {
try {
JMXConnector connector = newJMXFactory(uri);
ActiveMQServerControl serverControl = MBeanServerInvocationHandler.newProxyInstance(connector.getMBeanServerConnection(), builder.getActiveMQServerObjectName(), ActiveMQServerControl.class, false);
serverControl.isActive(); // making one call to make sure it's working
return serverControl;
} catch (Throwable e) {
lastException = e;
Thread.sleep(500);
}
}
while (expireLoop > System.currentTimeMillis());
throw lastException;
}
protected static JMXConnector getJmxConnector(String hostname, int port) throws MalformedURLException {
// Without this, the RMI server would bind to the default interface IP (the user's local IP mostly)
System.setProperty("java.rmi.server.hostname", hostname);
// I don't specify both ports here manually on purpose. See actual RMI registry connection port extraction below.
String urlString = "service:jmx:rmi:///jndi/rmi://" + hostname + ":" + port + "/jmxrmi";
JMXServiceURL url = new JMXServiceURL(urlString);
JMXConnector jmxConnector = null;
try {
jmxConnector = JMXConnectorFactory.connect(url);
System.out.println("Successfully connected to: " + urlString);
} catch (Exception e) {
jmxConnector = null;
e.printStackTrace();
Assert.fail(e.getMessage());
}
return jmxConnector;
}
protected static final void recreateBrokerDirectory(final String homeInstance) {
recreateDirectory(homeInstance + "/data");
recreateDirectory(homeInstance + "/log");
}
protected void checkLogRecord(File logFile, boolean exist, String... values) throws Exception {
Assert.assertTrue(logFile.exists());
boolean hasRecord = false;
try (BufferedReader reader = new BufferedReader(new FileReader(logFile))) {
String line = reader.readLine();
while (line != null) {
if (line.contains(values[0])) {
boolean hasAll = true;
for (int i = 1; i < values.length; i++) {
if (!line.contains(values[i])) {
hasAll = false;
break;
}
}
if (hasAll) {
hasRecord = true;
System.out.println("audit has it: " + line);
break;
}
}
line = reader.readLine();
}
if (exist) {
Assert.assertTrue(hasRecord);
} else {
Assert.assertFalse(hasRecord);
}
}
}
public class SmokeTestBase extends RealServerTestBase {
} }

View File

@ -581,83 +581,6 @@
</args> </args>
</configuration> </configuration>
</execution> </execution>
<execution>
<phase>test-compile</phase>
<id>create-database-paging</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
<instance>${basedir}/target/database-paging/derby</instance>
<configuration>${basedir}/target/classes/servers/database-paging/derby</configuration>
<libList>
<arg>org.apache.derby:derby:${apache.derby.version}</arg>
</libList>
<args>
<arg>--jdbc</arg>
<arg>--global-max-messages</arg>
<arg>100</arg>
<arg>--java-options</arg>
<arg>-ea</arg>
</args>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-database-paging-mysql</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
<instance>${basedir}/target/database-paging/mysql</instance>
<configuration>${basedir}/target/classes/servers/database-paging/mysql</configuration>
<libListWithDeps>
<arg>com.mysql:mysql-connector-j:8.0.33</arg>
</libListWithDeps>
<args>
<arg>--jdbc</arg>
<arg>--jdbc-connection-url</arg>
<arg>jdbc:mysql://localhost/ARTEMIS-TEST?user=root&amp;#38;password=artemis</arg>
<arg>--jdbc-driver-class-name</arg>
<arg>com.mysql.cj.jdbc.Driver</arg>
<arg>--global-max-messages</arg>
<arg>100</arg>
<arg>--java-options</arg>
<arg>-ea</arg>
</args>
</configuration>
</execution>
<execution>
<phase>test-compile</phase>
<id>create-database-paging-postgres</id>
<goals>
<goal>create</goal>
</goals>
<configuration>
<!-- this makes it easier in certain envs -->
<javaOptions>-Djava.net.preferIPv4Stack=true</javaOptions>
<instance>${basedir}/target/database-paging/postgres</instance>
<configuration>${basedir}/target/classes/servers/database-paging/postgres</configuration>
<libListWithDeps>
<arg>org.postgresql:postgresql:42.6.0</arg>
</libListWithDeps>
<args>
<arg>--jdbc</arg>
<arg>--jdbc-connection-url</arg>
<arg>jdbc:postgresql:artemis?user=artemis&amp;#38;password=artemis</arg>
<arg>--jdbc-driver-class-name</arg>
<arg>org.postgresql.Driver</arg>
<arg>--global-max-messages</arg>
<arg>100</arg>
<arg>--java-options</arg>
<arg>-ea</arg>
</args>
</configuration>
</execution>
<execution> <execution>
<phase>test-compile</phase> <phase>test-compile</phase>
<id>create-paging-export</id> <id>create-paging-export</id>

View File

@ -16,246 +16,14 @@
*/ */
package org.apache.activemq.artemis.tests.soak; package org.apache.activemq.artemis.tests.soak;
import javax.management.MBeanServerInvocationHandler;
import javax.management.ObjectName;
import javax.management.remote.JMXConnector;
import javax.management.remote.JMXConnectorFactory;
import javax.management.remote.JMXServiceURL;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.net.MalformedURLException;
import java.util.HashSet;
import java.util.Set;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.management.ActiveMQServerControl;
import org.apache.activemq.artemis.api.core.management.ObjectNameBuilder;
import org.apache.activemq.artemis.api.core.management.QueueControl;
import org.apache.activemq.artemis.api.jms.ActiveMQJMSClient;
import org.apache.activemq.artemis.cli.commands.Stop;
import org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory;
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
import org.apache.activemq.artemis.util.ServerUtil;
import org.apache.activemq.artemis.utils.SpawnedVMSupport;
import org.junit.After;
import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
public class SoakTestBase extends ActiveMQTestBase { import org.apache.activemq.artemis.utils.RealServerTestBase;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class SoakTestBase extends RealServerTestBase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
Set<Process> processes = new HashSet<>();
private static final String JMX_SERVER_HOSTNAME = "localhost";
private static final int JMX_SERVER_PORT = 10099;
public static final String basedir = System.getProperty("basedir");
protected static void unzip(File zipFile, File serverFolder) throws IOException, ClassNotFoundException, InterruptedException {
ProcessBuilder zipBuilder = new ProcessBuilder("unzip", zipFile.getAbsolutePath()).directory(serverFolder);
Process process = zipBuilder.start();
SpawnedVMSupport.startLogger("zip", process);
logger.info("Zip finished with {}", process.waitFor());
}
protected static void zip(File zipFile, File serverFolder) throws IOException, ClassNotFoundException, InterruptedException {
logger.info("Zipping data folder for {}", zipFile);
ProcessBuilder zipBuilder = new ProcessBuilder("zip", "-r", zipFile.getAbsolutePath(), "data").directory(serverFolder);
Process process = zipBuilder.start();
SpawnedVMSupport.startLogger("zip", process);
logger.info("Zip finished with {}", process.waitFor());
}
@After
public void after() throws Exception {
for (Process process : processes) {
try {
ServerUtil.killServer(process, true);
} catch (Throwable e) {
e.printStackTrace();
}
}
processes.clear();
}
public void killServer(Process process) {
processes.remove(process);
try {
ServerUtil.killServer(process);
} catch (Throwable e) {
e.printStackTrace();
}
}
protected static void stopServerWithFile(String serverLocation) throws IOException {
File serverPlace = new File(serverLocation);
File etcPlace = new File(serverPlace, "etc");
File stopMe = new File(etcPlace, Stop.STOP_FILE_NAME);
Assert.assertTrue(stopMe.createNewFile());
}
public static String getServerLocation(String serverName) {
return basedir + "/target/" + serverName;
}
public static void cleanupData(String serverName) {
String location = getServerLocation(serverName);
deleteDirectory(new File(location, "data"));
deleteDirectory(new File(location, "log"));
}
public void addProcess(Process process) {
processes.add(process);
}
public Process startServer(String serverName, int portID, int timeout) throws Exception {
Process process = ServerUtil.startServer(getServerLocation(serverName), serverName, portID, timeout);
addProcess(process);
return process;
}
public Process startServer(String serverName, String uri, int timeout) throws Exception {
Process process = ServerUtil.startServer(getServerLocation(serverName), serverName, uri, timeout);
addProcess(process);
return process;
}
protected JMXConnector getJmxConnector() throws MalformedURLException {
return getJmxConnector(JMX_SERVER_HOSTNAME, JMX_SERVER_PORT);
}
protected static JMXConnector newJMXFactory(String uri) throws Throwable {
return JMXConnectorFactory.connect(new JMXServiceURL(uri));
}
protected static ActiveMQServerControl getServerControl(String uri,
ObjectNameBuilder builder,
long timeout) throws Throwable {
long expireLoop = System.currentTimeMillis() + timeout;
Throwable lastException = null;
do {
try {
JMXConnector connector = newJMXFactory(uri);
ActiveMQServerControl serverControl = MBeanServerInvocationHandler.newProxyInstance(connector.getMBeanServerConnection(), builder.getActiveMQServerObjectName(), ActiveMQServerControl.class, false);
serverControl.isActive(); // making one call to make sure it's working
return serverControl;
} catch (Throwable e) {
lastException = e;
Thread.sleep(500);
}
}
while (expireLoop > System.currentTimeMillis());
throw lastException;
}
protected static QueueControl getQueueControl(String uri,
ObjectNameBuilder builder,
String address,
String queueName,
RoutingType routingType,
long timeout) throws Throwable {
long expireLoop = System.currentTimeMillis() + timeout;
Throwable lastException = null;
do {
try {
JMXConnector connector = newJMXFactory(uri);
ObjectName objectQueueName = builder.getQueueObjectName(SimpleString.toSimpleString(address), SimpleString.toSimpleString(queueName), routingType);
QueueControl queueControl = MBeanServerInvocationHandler.newProxyInstance(connector.getMBeanServerConnection(), objectQueueName, QueueControl.class, false);
queueControl.getMessagesAcknowledged(); // making one call
return queueControl;
} catch (Throwable e) {
logger.warn(e.getMessage(), e);
lastException = e;
Thread.sleep(500);
}
}
while (expireLoop > System.currentTimeMillis());
throw lastException;
}
protected static JMXConnector getJmxConnector(String hostname, int port) throws MalformedURLException {
// Without this, the RMI server would bind to the default interface IP (the user's local IP mostly)
System.setProperty("java.rmi.server.hostname", hostname);
// I don't specify both ports here manually on purpose. See actual RMI registry connection port extraction below.
String urlString = "service:jmx:rmi:///jndi/rmi://" + hostname + ":" + port + "/jmxrmi";
JMXServiceURL url = new JMXServiceURL(urlString);
JMXConnector jmxConnector = null;
try {
jmxConnector = JMXConnectorFactory.connect(url);
System.out.println("Successfully connected to: " + urlString);
} catch (Exception e) {
jmxConnector = null;
e.printStackTrace();
Assert.fail(e.getMessage());
}
return jmxConnector;
}
protected static final void recreateBrokerDirectory(final String homeInstance) {
recreateDirectory(homeInstance + "/data");
recreateDirectory(homeInstance + "/log");
}
public boolean waitForServerToStart(String uri, String username, String password, long timeout) throws InterruptedException {
long realTimeout = System.currentTimeMillis() + timeout;
while (System.currentTimeMillis() < realTimeout) {
try (ActiveMQConnectionFactory cf = ActiveMQJMSClient.createConnectionFactory(uri, null)) {
cf.createConnection(username, password).close();
System.out.println("server " + uri + " started");
} catch (Exception e) {
System.out.println("awaiting server " + uri + " start at ");
Thread.sleep(500);
continue;
}
return true;
}
return false;
}
protected void checkLogRecord(File logFile, boolean exist, String... values) throws Exception {
Assert.assertTrue(logFile.exists());
boolean hasRecord = false;
try (BufferedReader reader = new BufferedReader(new FileReader(logFile))) {
String line = reader.readLine();
while (line != null) {
if (line.contains(values[0])) {
boolean hasAll = true;
for (int i = 1; i < values.length; i++) {
if (!line.contains(values[i])) {
hasAll = false;
break;
}
}
if (hasAll) {
hasRecord = true;
System.out.println("audit has it: " + line);
break;
}
}
line = reader.readLine();
}
if (exist) {
Assert.assertTrue(hasRecord);
} else {
Assert.assertFalse(hasRecord);
}
}
}
} }

View File

@ -17,7 +17,7 @@
package org.apache.activemq.artemis.tests.soak.client; package org.apache.activemq.artemis.tests.soak.client;
import static org.apache.activemq.artemis.tests.soak.TestParameters.testProperty; import static org.apache.activemq.artemis.utils.TestParameters.testProperty;
public class ClientParameters { public class ClientParameters {
private static final String TEST_NAME = "CLIENT"; private static final String TEST_NAME = "CLIENT";

View File

@ -18,7 +18,7 @@ package org.apache.activemq.artemis.tests.soak.failover;
import org.apache.activemq.artemis.tests.integration.cluster.reattach.RandomReattachTest; import org.apache.activemq.artemis.tests.integration.cluster.reattach.RandomReattachTest;
import static org.apache.activemq.artemis.tests.soak.TestParameters.testProperty; import static org.apache.activemq.artemis.utils.TestParameters.testProperty;
public class RandomFailoverSoakTest extends RandomReattachTest { public class RandomFailoverSoakTest extends RandomReattachTest {

View File

@ -47,8 +47,8 @@ import org.junit.runners.Parameterized;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import static org.apache.activemq.artemis.tests.soak.TestParameters.intMandatoryProperty; import static org.apache.activemq.artemis.utils.TestParameters.intMandatoryProperty;
import static org.apache.activemq.artemis.tests.soak.TestParameters.testProperty; import static org.apache.activemq.artemis.utils.TestParameters.testProperty;
/** /**
* Refer to ./scripts/parameters.sh for suggested parameters * Refer to ./scripts/parameters.sh for suggested parameters

View File

@ -44,8 +44,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import static org.apache.activemq.artemis.tests.soak.TestParameters.intMandatoryProperty; import static org.apache.activemq.artemis.utils.TestParameters.intMandatoryProperty;
import static org.apache.activemq.artemis.tests.soak.TestParameters.testProperty; import static org.apache.activemq.artemis.utils.TestParameters.testProperty;
/** /**
* Refer to ./scripts/parameters.sh for suggested parameters * Refer to ./scripts/parameters.sh for suggested parameters

View File

@ -45,8 +45,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import static org.apache.activemq.artemis.tests.soak.TestParameters.intMandatoryProperty; import static org.apache.activemq.artemis.utils.TestParameters.intMandatoryProperty;
import static org.apache.activemq.artemis.tests.soak.TestParameters.testProperty; import static org.apache.activemq.artemis.utils.TestParameters.testProperty;
/** /**
* Refer to ./scripts/parameters.sh for suggested parameters * Refer to ./scripts/parameters.sh for suggested parameters

View File

@ -47,8 +47,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import static org.apache.activemq.artemis.tests.soak.TestParameters.intMandatoryProperty; import static org.apache.activemq.artemis.utils.TestParameters.intMandatoryProperty;
import static org.apache.activemq.artemis.tests.soak.TestParameters.testProperty; import static org.apache.activemq.artemis.utils.TestParameters.testProperty;
/** /**
* Refer to ./scripts/parameters.sh for suggested parameters * Refer to ./scripts/parameters.sh for suggested parameters

View File

@ -56,6 +56,7 @@ import org.apache.activemq.artemis.core.paging.impl.PageTransactionInfoImpl;
import org.apache.activemq.artemis.core.paging.impl.PagingStoreImpl; import org.apache.activemq.artemis.core.paging.impl.PagingStoreImpl;
import org.apache.activemq.artemis.core.paging.impl.PagingStoreTestAccessor; import org.apache.activemq.artemis.core.paging.impl.PagingStoreTestAccessor;
import org.apache.activemq.artemis.core.persistence.StorageManager; import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.persistence.impl.journal.OperationContextImpl;
import org.apache.activemq.artemis.core.persistence.impl.nullpm.NullStorageManager; import org.apache.activemq.artemis.core.persistence.impl.nullpm.NullStorageManager;
import org.apache.activemq.artemis.core.server.files.FileStoreMonitor; import org.apache.activemq.artemis.core.server.files.FileStoreMonitor;
import org.apache.activemq.artemis.core.server.impl.RoutingContextImpl; import org.apache.activemq.artemis.core.server.impl.RoutingContextImpl;
@ -185,7 +186,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
Assert.assertEquals(1, storeImpl.getNumberOfPages()); Assert.assertEquals(1, storeImpl.getNumberOfPages());
storeImpl.sync(); storeImpl.addSyncPoint(OperationContextImpl.getContext());
storeImpl = new PagingStoreImpl(PagingStoreImplTest.destinationTestName, null, 100, createMockManager(), createStorageManagerMock(), factory, storeFactory, PagingStoreImplTest.destinationTestName, addressSettings, getExecutorFactory().getExecutor(), getExecutorFactory().getExecutor(), true); storeImpl = new PagingStoreImpl(PagingStoreImplTest.destinationTestName, null, 100, createMockManager(), createStorageManagerMock(), factory, storeFactory, PagingStoreImplTest.destinationTestName, addressSettings, getExecutorFactory().getExecutor(), getExecutorFactory().getExecutor(), true);
@ -230,7 +231,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
Assert.assertEquals(1, storeImpl.getNumberOfPages()); Assert.assertEquals(1, storeImpl.getNumberOfPages());
storeImpl.sync(); storeImpl.addSyncPoint(OperationContextImpl.getContext());
Page page = storeImpl.depage(); Page page = storeImpl.depage();
@ -567,7 +568,7 @@ public class PagingStoreImplTest extends ActiveMQTestBase {
Assert.assertEquals(2, store.getNumberOfPages()); Assert.assertEquals(2, store.getNumberOfPages());
store.sync(); store.addSyncPoint(OperationContextImpl.getContext());
int sequence = 0; int sequence = 0;