mirror of https://github.com/apache/activemq.git
AMQ-7086 - make kahadb gc/cleanup on shutdown optional to trade availability over disk usage for fast failover
This commit is contained in:
parent
e093a8c1d8
commit
bf8eb08aca
|
@ -183,11 +183,11 @@ public interface PersistenceAdapter extends Service {
|
|||
/**
|
||||
* checkpoint any
|
||||
*
|
||||
* @param sync
|
||||
* @param cleanup
|
||||
* @throws IOException
|
||||
*
|
||||
*/
|
||||
void checkpoint(boolean sync) throws IOException;
|
||||
void checkpoint(boolean cleanup) throws IOException;
|
||||
|
||||
/**
|
||||
* A hint to return the size of the store on disk
|
||||
|
|
|
@ -55,6 +55,7 @@ public abstract class AbstractKahaDBStore extends LockableServiceSupport {
|
|||
protected boolean failIfDatabaseIsLocked;
|
||||
protected long checkpointInterval = 5*1000;
|
||||
protected long cleanupInterval = 30*1000;
|
||||
private boolean cleanupOnStop = true;
|
||||
protected boolean checkForCorruptJournalFiles = false;
|
||||
protected boolean checksumJournalFiles = true;
|
||||
protected boolean forceRecoverIndex = false;
|
||||
|
@ -202,6 +203,14 @@ public abstract class AbstractKahaDBStore extends LockableServiceSupport {
|
|||
this.cleanupInterval = cleanupInterval;
|
||||
}
|
||||
|
||||
public void setCleanupOnStop(boolean cleanupOnStop) {
|
||||
this.cleanupOnStop = cleanupOnStop;
|
||||
}
|
||||
|
||||
public boolean getCleanupOnStop() {
|
||||
return this.cleanupOnStop;
|
||||
}
|
||||
|
||||
public boolean isChecksumJournalFiles() {
|
||||
return checksumJournalFiles;
|
||||
}
|
||||
|
@ -666,7 +675,7 @@ public abstract class AbstractKahaDBStore extends LockableServiceSupport {
|
|||
*/
|
||||
protected void startCheckpoint() {
|
||||
if (checkpointInterval == 0 && cleanupInterval == 0) {
|
||||
LOG.info("periodic checkpoint/cleanup disabled, will ocurr on clean shutdown/restart");
|
||||
LOG.info("periodic checkpoint/cleanup disabled, will occur on clean " + (getCleanupOnStop() ? "shutdown/" : "") + "restart");
|
||||
return;
|
||||
}
|
||||
synchronized (checkpointThreadLock) {
|
||||
|
|
|
@ -79,13 +79,13 @@ public class KahaDBPersistenceAdapter extends LockableServiceSupport implements
|
|||
}
|
||||
|
||||
/**
|
||||
* @param sync
|
||||
* @param cleanup
|
||||
* @throws IOException
|
||||
* @see org.apache.activemq.store.PersistenceAdapter#checkpoint(boolean)
|
||||
*/
|
||||
@Override
|
||||
public void checkpoint(boolean sync) throws IOException {
|
||||
this.letter.checkpoint(sync);
|
||||
public void checkpoint(boolean cleanup) throws IOException {
|
||||
this.letter.checkpoint(cleanup);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -817,4 +817,15 @@ public class KahaDBPersistenceAdapter extends LockableServiceSupport implements
|
|||
public boolean isPersistNoLocal() {
|
||||
return this.letter.isPersistNoLocal();
|
||||
}
|
||||
|
||||
/*
|
||||
* When set, ensure that the cleanup/gc operation is executed during the stop procedure
|
||||
*/
|
||||
public void setCleanupOnStop(boolean cleanupOnStop) {
|
||||
this.letter.setCleanupOnStop(cleanupOnStop);
|
||||
}
|
||||
|
||||
public boolean getCleanupOnStop() {
|
||||
return this.letter.getCleanupOnStop();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -266,6 +266,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
long journalDiskSyncInterval = 1000;
|
||||
long checkpointInterval = 5*1000;
|
||||
long cleanupInterval = 30*1000;
|
||||
boolean cleanupOnStop = true;
|
||||
int journalMaxFileLength = Journal.DEFAULT_MAX_FILE_LENGTH;
|
||||
int journalMaxWriteBatchSize = Journal.DEFAULT_MAX_WRITE_BATCH_SIZE;
|
||||
boolean enableIndexWriteAsync = false;
|
||||
|
@ -375,7 +376,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
|
||||
private void startCheckpoint() {
|
||||
if (checkpointInterval == 0 && cleanupInterval == 0) {
|
||||
LOG.info("periodic checkpoint/cleanup disabled, will ocurr on clean shutdown/restart");
|
||||
LOG.info("periodic checkpoint/cleanup disabled, will occur on clean " + (getCleanupOnStop() ? "shutdown/" : "") + "restart");
|
||||
return;
|
||||
}
|
||||
synchronized (schedulerLock) {
|
||||
|
@ -508,7 +509,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
checkpointLock.writeLock().lock();
|
||||
try {
|
||||
if (metadata.page != null) {
|
||||
checkpointUpdate(true);
|
||||
checkpointUpdate(getCleanupOnStop());
|
||||
}
|
||||
pageFile.unload();
|
||||
metadata = createMetadata();
|
||||
|
@ -1147,9 +1148,6 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
after.run();
|
||||
}
|
||||
|
||||
if (scheduler == null && opened.get()) {
|
||||
startCheckpoint();
|
||||
}
|
||||
return location;
|
||||
} catch (IOException ioe) {
|
||||
LOG.error("KahaDB failed to store to Journal, command of type: " + data.type(), ioe);
|
||||
|
@ -3311,6 +3309,14 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
this.cleanupInterval = cleanupInterval;
|
||||
}
|
||||
|
||||
public boolean getCleanupOnStop() {
|
||||
return cleanupOnStop;
|
||||
}
|
||||
|
||||
public void setCleanupOnStop(boolean cleanupOnStop) {
|
||||
this.cleanupOnStop = cleanupOnStop;
|
||||
}
|
||||
|
||||
public void setJournalMaxFileLength(int journalMaxFileLength) {
|
||||
this.journalMaxFileLength = journalMaxFileLength;
|
||||
}
|
||||
|
|
|
@ -173,9 +173,9 @@ public class MultiKahaDBPersistenceAdapter extends LockableServiceSupport implem
|
|||
}
|
||||
|
||||
@Override
|
||||
public void checkpoint(final boolean sync) throws IOException {
|
||||
public void checkpoint(final boolean cleanup) throws IOException {
|
||||
for (PersistenceAdapter persistenceAdapter : adapters) {
|
||||
persistenceAdapter.checkpoint(sync);
|
||||
persistenceAdapter.checkpoint(cleanup);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -59,6 +59,10 @@ import org.apache.activemq.util.IOHelper;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/*
|
||||
* @org.apache.xbean.XBean element="kahaDBJobScheduler"
|
||||
*/
|
||||
|
||||
public class JobSchedulerStoreImpl extends AbstractKahaDBStore implements JobSchedulerStore {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(JobSchedulerStoreImpl.class);
|
||||
|
@ -230,7 +234,7 @@ public class JobSchedulerStoreImpl extends AbstractKahaDBStore implements JobSch
|
|||
checkpointLock.writeLock().lock();
|
||||
try {
|
||||
if (metaData.getPage() != null) {
|
||||
checkpointUpdate(true);
|
||||
checkpointUpdate(getCleanupOnStop());
|
||||
}
|
||||
} finally {
|
||||
checkpointLock.writeLock().unlock();
|
||||
|
|
|
@ -0,0 +1,184 @@
|
|||
/**
|
||||
* 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.store.kahadb.scheduler;
|
||||
|
||||
import org.apache.activemq.ActiveMQConnectionFactory;
|
||||
import org.apache.activemq.ScheduledMessage;
|
||||
import org.apache.activemq.broker.BrokerService;
|
||||
import org.apache.activemq.command.ActiveMQQueue;
|
||||
import org.apache.activemq.store.kahadb.KahaDBPersistenceAdapter;
|
||||
import org.apache.activemq.store.kahadb.disk.journal.Journal;
|
||||
import org.apache.activemq.util.IOHelper;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.jms.*;
|
||||
import java.io.File;
|
||||
import java.io.FilenameFilter;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
||||
public class AMQ7086Test {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(AMQ7086Test.class);
|
||||
|
||||
BrokerService brokerService;
|
||||
JobSchedulerStoreImpl jobSchedulerStore;
|
||||
KahaDBPersistenceAdapter kahaDBPersistenceAdapter;
|
||||
|
||||
@Test
|
||||
public void testGcDoneAtStop() throws Exception {
|
||||
|
||||
brokerService = createBroker(true);
|
||||
brokerService.start();
|
||||
|
||||
produceWithScheduledDelayAndConsume();
|
||||
|
||||
LOG.info("job store: " + jobSchedulerStore);
|
||||
int numSchedulerFiles = jobSchedulerStore.getJournal().getFileMap().size();
|
||||
LOG.info("kahadb store: " + kahaDBPersistenceAdapter);
|
||||
int numKahadbFiles = kahaDBPersistenceAdapter.getStore().getJournal().getFileMap().size();
|
||||
|
||||
LOG.info("Num files, job store: {}, messge store: {}", numKahadbFiles, numKahadbFiles);
|
||||
|
||||
// pull the dirs before we stop
|
||||
File jobDir = jobSchedulerStore.getJournal().getDirectory();
|
||||
File kahaDir = kahaDBPersistenceAdapter.getStore().getJournal().getDirectory();
|
||||
|
||||
brokerService.stop();
|
||||
|
||||
|
||||
assertEquals("Expected job store data files", 1, verifyFilesOnDisk(jobDir));
|
||||
assertEquals("Expected kahadb data files", 1, verifyFilesOnDisk(kahaDir));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoGcAtStop() throws Exception {
|
||||
|
||||
brokerService = createBroker(false);
|
||||
brokerService.start();
|
||||
|
||||
produceWithScheduledDelayAndConsume();
|
||||
|
||||
LOG.info("job store: " + jobSchedulerStore);
|
||||
int numSchedulerFiles = jobSchedulerStore.getJournal().getFileMap().size();
|
||||
LOG.info("kahadb store: " + kahaDBPersistenceAdapter);
|
||||
int numKahadbFiles = kahaDBPersistenceAdapter.getStore().getJournal().getFileMap().size();
|
||||
|
||||
LOG.info("Num files, job store: {}, messge store: {}", numKahadbFiles, numKahadbFiles);
|
||||
|
||||
// pull the dirs before we stop
|
||||
File jobDir = jobSchedulerStore.getJournal().getDirectory();
|
||||
File kahaDir = kahaDBPersistenceAdapter.getStore().getJournal().getDirectory();
|
||||
|
||||
brokerService.stop();
|
||||
|
||||
assertEquals("Expected job store data files", numSchedulerFiles, verifyFilesOnDisk(jobDir));
|
||||
assertEquals("Expected kahadb data files", numKahadbFiles, verifyFilesOnDisk(kahaDir));
|
||||
}
|
||||
|
||||
private int verifyFilesOnDisk(File directory) {
|
||||
|
||||
LOG.info("Broker: " + brokerService);
|
||||
LOG.info("dir: " + directory);
|
||||
int result = 0;
|
||||
|
||||
File[] files = directory.listFiles(new FilenameFilter() {
|
||||
@Override
|
||||
public boolean accept(File dir, String n) {
|
||||
return dir.equals(directory) && n.startsWith(Journal.DEFAULT_FILE_PREFIX) && n.endsWith(Journal.DEFAULT_FILE_SUFFIX);
|
||||
}
|
||||
});
|
||||
|
||||
LOG.info("File count: " + (files != null ? files.length : " empty!"));
|
||||
|
||||
if (files != null) {
|
||||
result = files.length;
|
||||
}
|
||||
for (File file : files) {
|
||||
LOG.info("name :" + file.getAbsolutePath());
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
protected BrokerService createBroker(boolean doCleanupOnStop) throws Exception {
|
||||
File schedulerDirectory = new File("target/scheduler");
|
||||
File kahadbDir = new File("target/kahadb");
|
||||
|
||||
for (File directory: new File[]{schedulerDirectory, kahadbDir}) {
|
||||
IOHelper.mkdirs(directory);
|
||||
IOHelper.deleteChildren(directory);
|
||||
}
|
||||
|
||||
BrokerService broker = new BrokerService();
|
||||
broker.setUseJmx(false);
|
||||
broker.setSchedulerSupport(true);
|
||||
|
||||
|
||||
jobSchedulerStore = new JobSchedulerStoreImpl();
|
||||
jobSchedulerStore.setDirectory(schedulerDirectory);
|
||||
jobSchedulerStore.setJournalMaxFileLength(16*1024);
|
||||
|
||||
jobSchedulerStore.setCheckpointInterval(0);
|
||||
jobSchedulerStore.setCleanupOnStop(doCleanupOnStop);
|
||||
|
||||
broker.setJobSchedulerStore(jobSchedulerStore);
|
||||
|
||||
|
||||
kahaDBPersistenceAdapter = new KahaDBPersistenceAdapter();
|
||||
kahaDBPersistenceAdapter.setDirectory(kahadbDir);
|
||||
kahaDBPersistenceAdapter.setJournalMaxFileLength(16*1024);
|
||||
|
||||
kahaDBPersistenceAdapter.setCleanupInterval(0);
|
||||
kahaDBPersistenceAdapter.setCleanupOnStop(doCleanupOnStop);
|
||||
|
||||
broker.setPersistenceAdapter(kahaDBPersistenceAdapter);
|
||||
|
||||
return broker;
|
||||
}
|
||||
|
||||
public void produceWithScheduledDelayAndConsume() throws Exception {
|
||||
Connection connection = new ActiveMQConnectionFactory("vm://localhost").createConnection();
|
||||
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
|
||||
|
||||
connection.start();
|
||||
final ActiveMQQueue destination = new ActiveMQQueue("QQ");
|
||||
final int numMessages = 50;
|
||||
final long time = 1000l;
|
||||
final byte[] payload = new byte[1024];
|
||||
MessageProducer producer = session.createProducer(destination);
|
||||
for (int i = 0; i < numMessages; i++) {
|
||||
BytesMessage bytesMessage = session.createBytesMessage();
|
||||
bytesMessage.writeBytes(payload);
|
||||
bytesMessage.setLongProperty(ScheduledMessage.AMQ_SCHEDULED_DELAY, time);
|
||||
producer.send(bytesMessage);
|
||||
}
|
||||
|
||||
MessageConsumer messageConsumer = session.createConsumer(destination);
|
||||
for (int i = 0; i < numMessages; i++) {
|
||||
assertNotNull(messageConsumer.receive(5000));
|
||||
}
|
||||
connection.close();
|
||||
|
||||
// let last ack settle
|
||||
TimeUnit.SECONDS.sleep(1);
|
||||
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue