ARTEMIS-822 Add executor service to JournalImpl for append operations and remove synchronization
https://issues.apache.org/jira/browse/ARTEMIS-822
This commit is contained in:
parent
bfb9bedb2d
commit
4b47461f03
|
@ -33,7 +33,6 @@ import org.apache.activemq.artemis.cli.commands.ActionContext;
|
||||||
import org.apache.activemq.artemis.core.io.nio.NIOSequentialFileFactory;
|
import org.apache.activemq.artemis.core.io.nio.NIOSequentialFileFactory;
|
||||||
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
||||||
import org.apache.activemq.artemis.core.journal.impl.JournalImpl;
|
import org.apache.activemq.artemis.core.journal.impl.JournalImpl;
|
||||||
import org.apache.activemq.artemis.core.journal.impl.JournalRecord;
|
|
||||||
import org.apache.activemq.artemis.utils.Base64;
|
import org.apache.activemq.artemis.utils.Base64;
|
||||||
|
|
||||||
@Command(name = "decode", description = "Decode a journal's internal format into a new journal set of files")
|
@Command(name = "decode", description = "Decode a journal's internal format into a new journal set of files")
|
||||||
|
@ -125,8 +124,6 @@ public class DecodeJournal extends LockAbstract {
|
||||||
|
|
||||||
long lineNumber = 0;
|
long lineNumber = 0;
|
||||||
|
|
||||||
Map<Long, JournalRecord> journalRecords = journal.getRecords();
|
|
||||||
|
|
||||||
while ((line = buffReader.readLine()) != null) {
|
while ((line = buffReader.readLine()) != null) {
|
||||||
lineNumber++;
|
lineNumber++;
|
||||||
String[] splitLine = line.split(",");
|
String[] splitLine = line.split(",");
|
||||||
|
@ -150,12 +147,6 @@ public class DecodeJournal extends LockAbstract {
|
||||||
counter.incrementAndGet();
|
counter.incrementAndGet();
|
||||||
RecordInfo info = parseRecord(lineProperties);
|
RecordInfo info = parseRecord(lineProperties);
|
||||||
journal.appendAddRecordTransactional(txID, info.id, info.userRecordType, info.data);
|
journal.appendAddRecordTransactional(txID, info.id, info.userRecordType, info.data);
|
||||||
} else if (operation.equals("AddRecordTX")) {
|
|
||||||
long txID = parseLong("txID", lineProperties);
|
|
||||||
AtomicInteger counter = getCounter(txID, txCounters);
|
|
||||||
counter.incrementAndGet();
|
|
||||||
RecordInfo info = parseRecord(lineProperties);
|
|
||||||
journal.appendAddRecordTransactional(txID, info.id, info.userRecordType, info.data);
|
|
||||||
} else if (operation.equals("UpdateTX")) {
|
} else if (operation.equals("UpdateTX")) {
|
||||||
long txID = parseLong("txID", lineProperties);
|
long txID = parseLong("txID", lineProperties);
|
||||||
AtomicInteger counter = getCounter(txID, txCounters);
|
AtomicInteger counter = getCounter(txID, txCounters);
|
||||||
|
@ -168,20 +159,17 @@ public class DecodeJournal extends LockAbstract {
|
||||||
} else if (operation.equals("DeleteRecord")) {
|
} else if (operation.equals("DeleteRecord")) {
|
||||||
long id = parseLong("id", lineProperties);
|
long id = parseLong("id", lineProperties);
|
||||||
|
|
||||||
// If not found it means the append/update records were reclaimed already
|
try {
|
||||||
if (journalRecords.get(id) != null) {
|
|
||||||
journal.appendDeleteRecord(id, false);
|
journal.appendDeleteRecord(id, false);
|
||||||
|
} catch (IllegalStateException ignored) {
|
||||||
|
// If not found it means the append/update records were reclaimed already
|
||||||
}
|
}
|
||||||
} else if (operation.equals("DeleteRecordTX")) {
|
} else if (operation.equals("DeleteRecordTX")) {
|
||||||
long txID = parseLong("txID", lineProperties);
|
long txID = parseLong("txID", lineProperties);
|
||||||
long id = parseLong("id", lineProperties);
|
long id = parseLong("id", lineProperties);
|
||||||
AtomicInteger counter = getCounter(txID, txCounters);
|
AtomicInteger counter = getCounter(txID, txCounters);
|
||||||
counter.incrementAndGet();
|
counter.incrementAndGet();
|
||||||
|
|
||||||
// If not found it means the append/update records were reclaimed already
|
|
||||||
if (journalRecords.get(id) != null) {
|
|
||||||
journal.appendDeleteRecordTransactional(txID, id);
|
journal.appendDeleteRecordTransactional(txID, id);
|
||||||
}
|
|
||||||
} else if (operation.equals("Prepare")) {
|
} else if (operation.equals("Prepare")) {
|
||||||
long txID = parseLong("txID", lineProperties);
|
long txID = parseLong("txID", lineProperties);
|
||||||
int numberOfRecords = parseInt("numberOfRecords", lineProperties);
|
int numberOfRecords = parseInt("numberOfRecords", lineProperties);
|
||||||
|
|
|
@ -22,7 +22,6 @@ import java.util.concurrent.Executor;
|
||||||
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
|
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
|
||||||
|
|
||||||
import org.apache.activemq.artemis.api.core.ActiveMQInterruptedException;
|
import org.apache.activemq.artemis.api.core.ActiveMQInterruptedException;
|
||||||
import org.apache.activemq.artemis.core.client.ActiveMQClientLogger;
|
|
||||||
import org.jboss.logging.Logger;
|
import org.jboss.logging.Logger;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -104,7 +103,7 @@ public final class OrderedExecutorFactory implements ExecutorFactory {
|
||||||
// This could happen during shutdowns. Nothing to be concerned about here
|
// This could happen during shutdowns. Nothing to be concerned about here
|
||||||
logger.debug("Interrupted Thread", e);
|
logger.debug("Interrupted Thread", e);
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
ActiveMQClientLogger.LOGGER.caughtunexpectedThrowable(t);
|
logger.warn(t.getMessage(), t);
|
||||||
}
|
}
|
||||||
task = tasks.poll();
|
task = tasks.poll();
|
||||||
}
|
}
|
|
@ -0,0 +1,79 @@
|
||||||
|
/**
|
||||||
|
* 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 java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
|
public class SimpleFuture<V> implements Future<V> {
|
||||||
|
|
||||||
|
public SimpleFuture() {
|
||||||
|
}
|
||||||
|
|
||||||
|
V value;
|
||||||
|
Exception exception;
|
||||||
|
|
||||||
|
private final CountDownLatch latch = new CountDownLatch(1);
|
||||||
|
|
||||||
|
boolean canceled = false;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean cancel(boolean mayInterruptIfRunning) {
|
||||||
|
canceled = true;
|
||||||
|
latch.countDown();
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isCancelled() {
|
||||||
|
return canceled;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isDone() {
|
||||||
|
return latch.getCount() <= 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void fail(Exception e) {
|
||||||
|
this.exception = e;
|
||||||
|
latch.countDown();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public V get() throws InterruptedException, ExecutionException {
|
||||||
|
latch.await();
|
||||||
|
if (this.exception != null) {
|
||||||
|
throw new ExecutionException(this.exception);
|
||||||
|
}
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void set(V v) {
|
||||||
|
this.value = v;
|
||||||
|
latch.countDown();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
|
||||||
|
latch.await(timeout, unit);
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,69 @@
|
||||||
|
/**
|
||||||
|
* 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 org.junit.Assert;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class SimpleFutureTest {
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ThreadLeakCheckRule threadLeakCheckRule = new ThreadLeakCheckRule();
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFuture() throws Exception {
|
||||||
|
final long randomStart = System.currentTimeMillis();
|
||||||
|
final SimpleFuture<Long> simpleFuture = new SimpleFuture<>();
|
||||||
|
Thread t = new Thread() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
simpleFuture.set(randomStart);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
t.start();
|
||||||
|
|
||||||
|
Assert.assertEquals(randomStart, simpleFuture.get().longValue());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testException() throws Exception {
|
||||||
|
final SimpleFuture<Long> simpleFuture = new SimpleFuture<>();
|
||||||
|
Thread t = new Thread() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
simpleFuture.fail(new Exception("hello"));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
t.start();
|
||||||
|
|
||||||
|
boolean failed = false;
|
||||||
|
try {
|
||||||
|
simpleFuture.get();
|
||||||
|
} catch (Exception e) {
|
||||||
|
failed = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
Assert.assertTrue(failed);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -29,11 +29,13 @@ import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
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.Future;
|
||||||
import java.util.concurrent.ThreadFactory;
|
import java.util.concurrent.ThreadFactory;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
@ -45,6 +47,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
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.Pair;
|
import org.apache.activemq.artemis.api.core.Pair;
|
||||||
|
import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
|
||||||
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.SequentialFileFactory;
|
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
|
||||||
|
@ -160,6 +163,8 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
// Compacting may replace this structure
|
// Compacting may replace this structure
|
||||||
private final ConcurrentMap<Long, JournalRecord> records = new ConcurrentHashMap<>();
|
private final ConcurrentMap<Long, JournalRecord> records = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
|
private final Set<Long> pendingRecords = Collections.newSetFromMap(new ConcurrentHashMap<Long, Boolean>());
|
||||||
|
|
||||||
// Compacting may replace this structure
|
// Compacting may replace this structure
|
||||||
private final ConcurrentMap<Long, JournalTransaction> transactions = new ConcurrentHashMap<>();
|
private final ConcurrentMap<Long, JournalTransaction> transactions = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
|
@ -172,12 +177,9 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
|
|
||||||
private ExecutorService compactorExecutor = null;
|
private ExecutorService compactorExecutor = null;
|
||||||
|
|
||||||
private ConcurrentHashSet<CountDownLatch> latches = new ConcurrentHashSet<>();
|
private ExecutorService appendExecutor = null;
|
||||||
|
|
||||||
// Lock used during the append of records
|
private ConcurrentHashSet<CountDownLatch> latches = new ConcurrentHashSet<>();
|
||||||
// This lock doesn't represent a global lock.
|
|
||||||
// After a record is appended, the usedFile can't be changed until the positives and negatives are updated
|
|
||||||
private final Object lockAppend = new Object();
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* We don't lock the journal during the whole compacting operation. During compacting we only
|
* We don't lock the journal during the whole compacting operation. During compacting we only
|
||||||
|
@ -688,18 +690,17 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
final boolean sync,
|
final boolean sync,
|
||||||
final IOCompletion callback) throws Exception {
|
final IOCompletion callback) throws Exception {
|
||||||
checkJournalIsLoaded();
|
checkJournalIsLoaded();
|
||||||
|
lineUpContext(callback);
|
||||||
|
pendingRecords.add(id);
|
||||||
|
|
||||||
|
Future<?> result = appendExecutor.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
journalLock.readLock().lock();
|
journalLock.readLock().lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
JournalInternalRecord addRecord = new JournalAddRecord(true, id, recordType, record);
|
JournalInternalRecord addRecord = new JournalAddRecord(true, id, recordType, record);
|
||||||
|
|
||||||
if (callback != null) {
|
|
||||||
callback.storeLineUp();
|
|
||||||
}
|
|
||||||
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
JournalFile usedFile = appendRecord(addRecord, false, sync, null, callback);
|
JournalFile usedFile = appendRecord(addRecord, false, sync, null, callback);
|
||||||
|
records.put(id, new JournalRecord(usedFile, addRecord.getEncodeSize()));
|
||||||
|
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("appendAddRecord::id=" + id +
|
logger.trace("appendAddRecord::id=" + id +
|
||||||
|
@ -709,13 +710,19 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
", usedFile = " +
|
", usedFile = " +
|
||||||
usedFile);
|
usedFile);
|
||||||
}
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
records.put(id, new JournalRecord(usedFile, addRecord.getEncodeSize()));
|
ActiveMQJournalLogger.LOGGER.error(e.getMessage(), e);
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
|
pendingRecords.remove(id);
|
||||||
journalLock.readLock().unlock();
|
journalLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
if (sync && callback == null) {
|
||||||
|
result.get();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void appendUpdateRecord(final long id,
|
public void appendUpdateRecord(final long id,
|
||||||
|
@ -724,96 +731,88 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
final boolean sync,
|
final boolean sync,
|
||||||
final IOCompletion callback) throws Exception {
|
final IOCompletion callback) throws Exception {
|
||||||
checkJournalIsLoaded();
|
checkJournalIsLoaded();
|
||||||
|
lineUpContext(callback);
|
||||||
|
checkKnownRecordID(id);
|
||||||
|
|
||||||
|
Future<?> result = appendExecutor.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
journalLock.readLock().lock();
|
journalLock.readLock().lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
JournalRecord jrnRecord = records.get(id);
|
JournalRecord jrnRecord = records.get(id);
|
||||||
|
|
||||||
if (jrnRecord == null) {
|
|
||||||
if (!(compactor != null && compactor.lookupRecord(id))) {
|
|
||||||
throw new IllegalStateException("Cannot find add info " + id);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
JournalInternalRecord updateRecord = new JournalAddRecord(false, id, recordType, record);
|
JournalInternalRecord updateRecord = new JournalAddRecord(false, id, recordType, record);
|
||||||
|
|
||||||
if (callback != null) {
|
|
||||||
callback.storeLineUp();
|
|
||||||
}
|
|
||||||
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
JournalFile usedFile = appendRecord(updateRecord, false, sync, null, callback);
|
JournalFile usedFile = appendRecord(updateRecord, false, sync, null, callback);
|
||||||
|
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("appendUpdateRecord::id=" + id +
|
logger.trace("appendUpdateRecord::id=" + id +
|
||||||
", userRecordType=" +
|
", userRecordType=" +
|
||||||
recordType +
|
recordType +
|
||||||
", record = " + record +
|
|
||||||
", usedFile = " +
|
", usedFile = " +
|
||||||
usedFile);
|
usedFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
// record== null here could only mean there is a compactor, and computing the delete should be done after
|
// record==null here could only mean there is a compactor
|
||||||
// compacting is done
|
// computing the delete should be done after compacting is done
|
||||||
if (jrnRecord == null) {
|
if (jrnRecord == null) {
|
||||||
compactor.addCommandUpdate(id, usedFile, updateRecord.getEncodeSize());
|
compactor.addCommandUpdate(id, usedFile, updateRecord.getEncodeSize());
|
||||||
} else {
|
} else {
|
||||||
jrnRecord.addUpdateFile(usedFile, updateRecord.getEncodeSize());
|
jrnRecord.addUpdateFile(usedFile, updateRecord.getEncodeSize());
|
||||||
}
|
}
|
||||||
}
|
} catch (Exception e) {
|
||||||
|
ActiveMQJournalLogger.LOGGER.error(e.getMessage(), e);
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.readLock().unlock();
|
journalLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
if (sync && callback == null) {
|
||||||
|
result.get();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void appendDeleteRecord(final long id, final boolean sync, final IOCompletion callback) throws Exception {
|
public void appendDeleteRecord(final long id, final boolean sync, final IOCompletion callback) throws Exception {
|
||||||
checkJournalIsLoaded();
|
checkJournalIsLoaded();
|
||||||
|
lineUpContext(callback);
|
||||||
|
checkKnownRecordID(id);
|
||||||
|
|
||||||
|
Future<?> result = appendExecutor.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
journalLock.readLock().lock();
|
journalLock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
|
|
||||||
JournalRecord record = null;
|
JournalRecord record = null;
|
||||||
|
|
||||||
if (compactor == null) {
|
if (compactor == null) {
|
||||||
record = records.remove(id);
|
record = records.remove(id);
|
||||||
|
|
||||||
if (record == null) {
|
|
||||||
throw new IllegalStateException("Cannot find add info " + id);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
if (!records.containsKey(id) && !compactor.lookupRecord(id)) {
|
|
||||||
throw new IllegalStateException("Cannot find add info " + id + " on compactor or current records");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
JournalInternalRecord deleteRecord = new JournalDeleteRecord(id);
|
JournalInternalRecord deleteRecord = new JournalDeleteRecord(id);
|
||||||
|
|
||||||
if (callback != null) {
|
|
||||||
callback.storeLineUp();
|
|
||||||
}
|
|
||||||
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
JournalFile usedFile = appendRecord(deleteRecord, false, sync, null, callback);
|
JournalFile usedFile = appendRecord(deleteRecord, false, sync, null, callback);
|
||||||
|
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("appendDeleteRecord::id=" + id + ", usedFile = " + usedFile);
|
logger.trace("appendDeleteRecord::id=" + id + ", usedFile = " + usedFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
// record== null here could only mean there is a compactor, and computing the delete should be done after
|
// record==null here could only mean there is a compactor
|
||||||
// compacting is done
|
// computing the delete should be done after compacting is done
|
||||||
if (record == null) {
|
if (record == null) {
|
||||||
compactor.addCommandDelete(id, usedFile);
|
compactor.addCommandDelete(id, usedFile);
|
||||||
} else {
|
} else {
|
||||||
record.delete(usedFile);
|
record.delete(usedFile);
|
||||||
}
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
}
|
ActiveMQJournalLogger.LOGGER.error(e.getMessage(), e);
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.readLock().unlock();
|
journalLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
if (sync && callback == null) {
|
||||||
|
result.get();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void appendAddRecordTransactional(final long txID,
|
public void appendAddRecordTransactional(final long txID,
|
||||||
|
@ -822,14 +821,16 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
final EncodingSupport record) throws Exception {
|
final EncodingSupport record) throws Exception {
|
||||||
checkJournalIsLoaded();
|
checkJournalIsLoaded();
|
||||||
|
|
||||||
journalLock.readLock().lock();
|
final JournalTransaction tx = getTransactionInfo(txID);
|
||||||
|
tx.checkErrorCondition();
|
||||||
|
|
||||||
|
appendExecutor.submit(new Runnable() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
journalLock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
JournalInternalRecord addRecord = new JournalAddRecordTX(true, txID, id, recordType, record);
|
JournalInternalRecord addRecord = new JournalAddRecordTX(true, txID, id, recordType, record);
|
||||||
|
|
||||||
JournalTransaction tx = getTransactionInfo(txID);
|
|
||||||
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
JournalFile usedFile = appendRecord(addRecord, false, false, tx, null);
|
JournalFile usedFile = appendRecord(addRecord, false, false, tx, null);
|
||||||
|
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
|
@ -844,11 +845,40 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
}
|
}
|
||||||
|
|
||||||
tx.addPositive(usedFile, id, addRecord.getEncodeSize());
|
tx.addPositive(usedFile, id, addRecord.getEncodeSize());
|
||||||
}
|
} catch (Exception e) {
|
||||||
|
ActiveMQJournalLogger.LOGGER.error(e.getMessage(), e);
|
||||||
|
setErrorCondition(tx, e);
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.readLock().unlock();
|
journalLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
private void checkKnownRecordID(final long id) throws Exception {
|
||||||
|
if (records.containsKey(id) || pendingRecords.contains(id) || (compactor != null && compactor.lookupRecord(id))) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// retry on the append thread. maybe the appender thread is not keeping up.
|
||||||
|
Future<Boolean> known = appendExecutor.submit(new Callable<Boolean>() {
|
||||||
|
@Override
|
||||||
|
public Boolean call() throws Exception {
|
||||||
|
journalLock.readLock().lock();
|
||||||
|
try {
|
||||||
|
return records.containsKey(id)
|
||||||
|
|| pendingRecords.contains(id)
|
||||||
|
|| (compactor != null && compactor.lookupRecord(id));
|
||||||
|
} finally {
|
||||||
|
journalLock.readLock().unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
if (!known.get()) {
|
||||||
|
throw new IllegalStateException("Cannot find add info " + id + " on compactor or current records");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private void checkJournalIsLoaded() {
|
private void checkJournalIsLoaded() {
|
||||||
if (state != JournalState.LOADED && state != JournalState.SYNCING) {
|
if (state != JournalState.LOADED && state != JournalState.SYNCING) {
|
||||||
|
@ -867,33 +897,40 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
final EncodingSupport record) throws Exception {
|
final EncodingSupport record) throws Exception {
|
||||||
checkJournalIsLoaded();
|
checkJournalIsLoaded();
|
||||||
|
|
||||||
|
final JournalTransaction tx = getTransactionInfo(txID);
|
||||||
|
tx.checkErrorCondition();
|
||||||
|
|
||||||
|
appendExecutor.submit(new Runnable() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
journalLock.readLock().lock();
|
journalLock.readLock().lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
JournalInternalRecord updateRecordTX = new JournalAddRecordTX(false, txID, id, recordType, record);
|
|
||||||
|
|
||||||
JournalTransaction tx = getTransactionInfo(txID);
|
JournalInternalRecord updateRecordTX = new JournalAddRecordTX( false, txID, id, recordType, record );
|
||||||
|
JournalFile usedFile = appendRecord( updateRecordTX, false, false, tx, null );
|
||||||
|
|
||||||
synchronized (lockAppend) {
|
if ( logger.isTraceEnabled() ) {
|
||||||
JournalFile usedFile = appendRecord(updateRecordTX, false, false, tx, null);
|
logger.trace( "appendUpdateRecordTransactional::txID=" + txID +
|
||||||
|
|
||||||
if (logger.isTraceEnabled()) {
|
|
||||||
logger.trace("appendUpdateRecordTransactional::txID=" + txID +
|
|
||||||
",id=" +
|
",id=" +
|
||||||
id +
|
id +
|
||||||
", userRecordType=" +
|
", userRecordType=" +
|
||||||
recordType +
|
recordType +
|
||||||
", record = " + record +
|
", record = " + record +
|
||||||
", usedFile = " +
|
", usedFile = " +
|
||||||
usedFile);
|
usedFile );
|
||||||
}
|
}
|
||||||
|
|
||||||
tx.addPositive(usedFile, id, updateRecordTX.getEncodeSize());
|
tx.addPositive( usedFile, id, updateRecordTX.getEncodeSize() );
|
||||||
}
|
} catch ( Exception e ) {
|
||||||
|
ActiveMQJournalLogger.LOGGER.error( e.getMessage(), e );
|
||||||
|
setErrorCondition( tx, e );
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.readLock().unlock();
|
journalLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void appendDeleteRecordTransactional(final long txID,
|
public void appendDeleteRecordTransactional(final long txID,
|
||||||
|
@ -901,14 +938,16 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
final EncodingSupport record) throws Exception {
|
final EncodingSupport record) throws Exception {
|
||||||
checkJournalIsLoaded();
|
checkJournalIsLoaded();
|
||||||
|
|
||||||
|
final JournalTransaction tx = getTransactionInfo(txID);
|
||||||
|
tx.checkErrorCondition();
|
||||||
|
|
||||||
|
appendExecutor.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
journalLock.readLock().lock();
|
journalLock.readLock().lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
|
||||||
JournalInternalRecord deleteRecordTX = new JournalDeleteRecordTX(txID, id, record);
|
JournalInternalRecord deleteRecordTX = new JournalDeleteRecordTX(txID, id, record);
|
||||||
|
|
||||||
JournalTransaction tx = getTransactionInfo(txID);
|
|
||||||
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
JournalFile usedFile = appendRecord(deleteRecordTX, false, false, tx, null);
|
JournalFile usedFile = appendRecord(deleteRecordTX, false, false, tx, null);
|
||||||
|
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
|
@ -920,11 +959,15 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
}
|
}
|
||||||
|
|
||||||
tx.addNegative(usedFile, id);
|
tx.addNegative(usedFile, id);
|
||||||
}
|
} catch (Exception e) {
|
||||||
|
ActiveMQJournalLogger.LOGGER.error(e.getMessage(), e);
|
||||||
|
setErrorCondition(tx, e);
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.readLock().unlock();
|
journalLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>If the system crashed after a prepare was called, it should store information that is required to bring the transaction
|
* <p>If the system crashed after a prepare was called, it should store information that is required to bring the transaction
|
||||||
|
@ -943,19 +986,17 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
final IOCompletion callback) throws Exception {
|
final IOCompletion callback) throws Exception {
|
||||||
|
|
||||||
checkJournalIsLoaded();
|
checkJournalIsLoaded();
|
||||||
|
lineUpContext(callback);
|
||||||
|
|
||||||
|
final JournalTransaction tx = getTransactionInfo(txID);
|
||||||
|
tx.checkErrorCondition();
|
||||||
|
|
||||||
|
Future<?> result = appendExecutor.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
journalLock.readLock().lock();
|
journalLock.readLock().lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
JournalTransaction tx = getTransactionInfo(txID);
|
|
||||||
|
|
||||||
JournalInternalRecord prepareRecord = new JournalCompleteRecordTX(TX_RECORD_TYPE.PREPARE, txID, transactionData);
|
JournalInternalRecord prepareRecord = new JournalCompleteRecordTX(TX_RECORD_TYPE.PREPARE, txID, transactionData);
|
||||||
|
|
||||||
if (callback != null) {
|
|
||||||
callback.storeLineUp();
|
|
||||||
}
|
|
||||||
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
JournalFile usedFile = appendRecord(prepareRecord, true, sync, tx, callback);
|
JournalFile usedFile = appendRecord(prepareRecord, true, sync, tx, callback);
|
||||||
|
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
|
@ -963,17 +1004,36 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
}
|
}
|
||||||
|
|
||||||
tx.prepare(usedFile);
|
tx.prepare(usedFile);
|
||||||
}
|
} catch (Exception e) {
|
||||||
|
ActiveMQJournalLogger.LOGGER.error(e.getMessage(), e);
|
||||||
|
setErrorCondition(tx, e);
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.readLock().unlock();
|
journalLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
if (sync && callback == null) {
|
||||||
|
result.get();
|
||||||
|
tx.checkErrorCondition();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void lineUpContext(IOCompletion callback) {
|
public void lineUpContext(IOCompletion callback) {
|
||||||
|
if (callback != null) {
|
||||||
callback.storeLineUp();
|
callback.storeLineUp();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setErrorCondition(JournalTransaction jt, Throwable t) {
|
||||||
|
if (jt != null) {
|
||||||
|
TransactionCallback callback = jt.getCurrentCallback();
|
||||||
|
if (callback != null && callback.getErrorMessage() != null) {
|
||||||
|
callback.onError(ActiveMQExceptionType.IO_ERROR.getCode(), t.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Regarding the number of operations in a given file see {@link JournalCompleteRecordTX}.
|
* Regarding the number of operations in a given file see {@link JournalCompleteRecordTX}.
|
||||||
|
@ -982,70 +1042,85 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
public void appendCommitRecord(final long txID,
|
public void appendCommitRecord(final long txID,
|
||||||
final boolean sync,
|
final boolean sync,
|
||||||
final IOCompletion callback,
|
final IOCompletion callback,
|
||||||
boolean lineUpContext) throws Exception {
|
final boolean lineUpContext) throws Exception {
|
||||||
checkJournalIsLoaded();
|
checkJournalIsLoaded();
|
||||||
|
if (lineUpContext) {
|
||||||
|
lineUpContext(callback);
|
||||||
|
}
|
||||||
|
|
||||||
journalLock.readLock().lock();
|
final JournalTransaction tx = transactions.remove(txID);
|
||||||
|
|
||||||
try {
|
|
||||||
JournalTransaction tx = transactions.remove(txID);
|
|
||||||
|
|
||||||
if (tx == null) {
|
if (tx == null) {
|
||||||
throw new IllegalStateException("Cannot find tx with id " + txID);
|
throw new IllegalStateException("Cannot find tx with id " + txID);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
tx.checkErrorCondition();
|
||||||
|
|
||||||
|
Future<?> result = appendExecutor.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
journalLock.readLock().lock();
|
||||||
|
try {
|
||||||
JournalInternalRecord commitRecord = new JournalCompleteRecordTX(TX_RECORD_TYPE.COMMIT, txID, null);
|
JournalInternalRecord commitRecord = new JournalCompleteRecordTX(TX_RECORD_TYPE.COMMIT, txID, null);
|
||||||
|
|
||||||
if (callback != null && lineUpContext) {
|
|
||||||
callback.storeLineUp();
|
|
||||||
}
|
|
||||||
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
JournalFile usedFile = appendRecord(commitRecord, true, sync, tx, callback);
|
JournalFile usedFile = appendRecord(commitRecord, true, sync, tx, callback);
|
||||||
|
|
||||||
|
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.trace("appendCommitRecord::txID=" + txID + ", usedFile = " + usedFile);
|
logger.trace("appendCommitRecord::txID=" + txID + ", usedFile = " + usedFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
tx.commit(usedFile);
|
tx.commit(usedFile);
|
||||||
}
|
} catch (Exception e) {
|
||||||
|
ActiveMQJournalLogger.LOGGER.error(e.getMessage(), e);
|
||||||
|
setErrorCondition(tx, e);
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.readLock().unlock();
|
journalLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
if (sync && callback == null) {
|
||||||
|
result.get();
|
||||||
|
tx.checkErrorCondition();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void appendRollbackRecord(final long txID, final boolean sync, final IOCompletion callback) throws Exception {
|
public void appendRollbackRecord(final long txID, final boolean sync, final IOCompletion callback) throws Exception {
|
||||||
checkJournalIsLoaded();
|
checkJournalIsLoaded();
|
||||||
|
lineUpContext(callback);
|
||||||
|
|
||||||
journalLock.readLock().lock();
|
final JournalTransaction tx = transactions.remove(txID);
|
||||||
|
|
||||||
JournalTransaction tx = null;
|
|
||||||
|
|
||||||
try {
|
|
||||||
tx = transactions.remove(txID);
|
|
||||||
|
|
||||||
if (tx == null) {
|
if (tx == null) {
|
||||||
throw new IllegalStateException("Cannot find tx with id " + txID);
|
throw new IllegalStateException("Cannot find tx with id " + txID);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
tx.checkErrorCondition();
|
||||||
|
|
||||||
|
Future<?> result = appendExecutor.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
journalLock.readLock().lock();
|
||||||
|
try {
|
||||||
JournalInternalRecord rollbackRecord = new JournalRollbackRecordTX(txID);
|
JournalInternalRecord rollbackRecord = new JournalRollbackRecordTX(txID);
|
||||||
|
|
||||||
if (callback != null) {
|
|
||||||
callback.storeLineUp();
|
|
||||||
}
|
|
||||||
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
JournalFile usedFile = appendRecord(rollbackRecord, false, sync, tx, callback);
|
JournalFile usedFile = appendRecord(rollbackRecord, false, sync, tx, callback);
|
||||||
|
|
||||||
tx.rollback(usedFile);
|
tx.rollback(usedFile);
|
||||||
}
|
} catch (Exception e) {
|
||||||
|
ActiveMQJournalLogger.LOGGER.error(e.getMessage(), e);
|
||||||
|
setErrorCondition(tx, e);
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.readLock().unlock();
|
journalLock.readLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
if (sync && callback == null) {
|
||||||
|
result.get();
|
||||||
|
tx.checkErrorCondition();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// XXX make it protected?
|
// XXX make it protected?
|
||||||
@Override
|
@Override
|
||||||
|
@ -1906,13 +1981,23 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
public void debugWait() throws InterruptedException {
|
public void debugWait() throws InterruptedException {
|
||||||
fileFactory.flush();
|
fileFactory.flush();
|
||||||
|
|
||||||
for (JournalTransaction tx : transactions.values()) {
|
if (appendExecutor != null && !appendExecutor.isShutdown()) {
|
||||||
tx.waitCallbacks();
|
// Send something to the closingExecutor, just to make sure we went until its end
|
||||||
|
final CountDownLatch latch = newLatch(1);
|
||||||
|
|
||||||
|
appendExecutor.execute(new Runnable() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
latch.countDown();
|
||||||
|
}
|
||||||
|
|
||||||
|
});
|
||||||
|
awaitLatch(latch, -1);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (filesExecutor != null && !filesExecutor.isShutdown()) {
|
if (filesExecutor != null && !filesExecutor.isShutdown()) {
|
||||||
// Send something to the closingExecutor, just to make sure we went
|
// Send something to the closingExecutor, just to make sure we went until its end
|
||||||
// until its end
|
|
||||||
final CountDownLatch latch = newLatch(1);
|
final CountDownLatch latch = newLatch(1);
|
||||||
|
|
||||||
filesExecutor.execute(new Runnable() {
|
filesExecutor.execute(new Runnable() {
|
||||||
|
@ -1985,20 +2070,52 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
// In some tests we need to force the journal to move to a next file
|
// In some tests we need to force the journal to move to a next file
|
||||||
@Override
|
@Override
|
||||||
public void forceMoveNextFile() throws Exception {
|
public void forceMoveNextFile() throws Exception {
|
||||||
journalLock.readLock().lock();
|
|
||||||
try {
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
moveNextFile(false);
|
|
||||||
debugWait();
|
debugWait();
|
||||||
}
|
journalLock.writeLock().lock();
|
||||||
|
try {
|
||||||
|
moveNextFile(false);
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.readLock().unlock();
|
journalLock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void perfBlast(final int pages) {
|
public void perfBlast(final int pages) {
|
||||||
new PerfBlast(pages).start();
|
|
||||||
|
checkJournalIsLoaded();
|
||||||
|
|
||||||
|
final ByteArrayEncoding byteEncoder = new ByteArrayEncoding(new byte[128 * 1024]);
|
||||||
|
|
||||||
|
final JournalInternalRecord blastRecord = new JournalInternalRecord() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getEncodeSize() {
|
||||||
|
return byteEncoder.getEncodeSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void encode(final ActiveMQBuffer buffer) {
|
||||||
|
byteEncoder.encode(buffer);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
appendExecutor.submit(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
journalLock.readLock().lock();
|
||||||
|
try {
|
||||||
|
|
||||||
|
for (int i = 0; i < pages; i++) {
|
||||||
|
appendRecord(blastRecord, false, false, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
} catch (Exception e) {
|
||||||
|
ActiveMQJournalLogger.LOGGER.failedToPerfBlast(e);
|
||||||
|
} finally {
|
||||||
|
journalLock.readLock().unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
// ActiveMQComponent implementation
|
// ActiveMQComponent implementation
|
||||||
|
@ -2031,6 +2148,14 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
appendExecutor = Executors.newSingleThreadExecutor(new ThreadFactory() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Thread newThread(final Runnable r) {
|
||||||
|
return new Thread(r, "JournalImpl::appendExecutor");
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
filesRepository.setExecutor(filesExecutor);
|
filesRepository.setExecutor(filesExecutor);
|
||||||
|
|
||||||
fileFactory.start();
|
fileFactory.start();
|
||||||
|
@ -2044,12 +2169,17 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
throw new IllegalStateException("Journal is already stopped");
|
throw new IllegalStateException("Journal is already stopped");
|
||||||
}
|
}
|
||||||
|
|
||||||
journalLock.writeLock().lock();
|
|
||||||
try {
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
|
|
||||||
setJournalState(JournalState.STOPPED);
|
setJournalState(JournalState.STOPPED);
|
||||||
|
|
||||||
|
// appendExecutor must be shut down first
|
||||||
|
appendExecutor.shutdown();
|
||||||
|
|
||||||
|
if (!appendExecutor.awaitTermination(60, TimeUnit.SECONDS)) {
|
||||||
|
ActiveMQJournalLogger.LOGGER.couldNotStopJournalAppendExecutor();
|
||||||
|
}
|
||||||
|
|
||||||
|
journalLock.writeLock().lock();
|
||||||
|
try {
|
||||||
compactorExecutor.shutdown();
|
compactorExecutor.shutdown();
|
||||||
|
|
||||||
if (!compactorExecutor.awaitTermination(120, TimeUnit.SECONDS)) {
|
if (!compactorExecutor.awaitTermination(120, TimeUnit.SECONDS)) {
|
||||||
|
@ -2083,7 +2213,6 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
fileFactory.stop();
|
fileFactory.stop();
|
||||||
|
|
||||||
currentFile = null;
|
currentFile = null;
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
journalLock.writeLock().unlock();
|
journalLock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
|
@ -2358,7 +2487,6 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
final boolean sync,
|
final boolean sync,
|
||||||
final JournalTransaction tx,
|
final JournalTransaction tx,
|
||||||
final IOCallback parameterCallback) throws Exception {
|
final IOCallback parameterCallback) throws Exception {
|
||||||
checkJournalIsLoaded();
|
|
||||||
|
|
||||||
final IOCallback callback;
|
final IOCallback callback;
|
||||||
|
|
||||||
|
@ -2552,46 +2680,6 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private final class PerfBlast extends Thread {
|
|
||||||
|
|
||||||
private final int pages;
|
|
||||||
|
|
||||||
private PerfBlast(final int pages) {
|
|
||||||
super("activemq-perfblast-thread");
|
|
||||||
|
|
||||||
this.pages = pages;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
synchronized (lockAppend) {
|
|
||||||
try {
|
|
||||||
|
|
||||||
final ByteArrayEncoding byteEncoder = new ByteArrayEncoding(new byte[128 * 1024]);
|
|
||||||
|
|
||||||
JournalInternalRecord blastRecord = new JournalInternalRecord() {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int getEncodeSize() {
|
|
||||||
return byteEncoder.getEncodeSize();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void encode(final ActiveMQBuffer buffer) {
|
|
||||||
byteEncoder.encode(buffer);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
for (int i = 0; i < pages; i++) {
|
|
||||||
appendRecord(blastRecord, false, false, null, null);
|
|
||||||
}
|
|
||||||
} catch (Exception e) {
|
|
||||||
ActiveMQJournalLogger.LOGGER.failedToPerfBlast(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public final void synchronizationLock() {
|
public final void synchronizationLock() {
|
||||||
compactorLock.writeLock().lock();
|
compactorLock.writeLock().lock();
|
||||||
|
@ -2624,7 +2712,7 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
||||||
long maxID = -1;
|
long maxID = -1;
|
||||||
for (long id : fileIds) {
|
for (long id : fileIds) {
|
||||||
maxID = Math.max(maxID, id);
|
maxID = Math.max(maxID, id);
|
||||||
map.put(Long.valueOf(id), filesRepository.createRemoteBackupSyncFile(id));
|
map.put(id, filesRepository.createRemoteBackupSyncFile(id));
|
||||||
}
|
}
|
||||||
filesRepository.setNextFileID(maxID);
|
filesRepository.setNextFileID(maxID);
|
||||||
return map;
|
return map;
|
||||||
|
|
|
@ -17,11 +17,13 @@
|
||||||
package org.apache.activemq.artemis.core.journal.impl;
|
package org.apache.activemq.artemis.core.journal.impl;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
|
import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
|
||||||
|
@ -45,12 +47,14 @@ public class JournalTransaction {
|
||||||
|
|
||||||
private boolean compacting = false;
|
private boolean compacting = false;
|
||||||
|
|
||||||
private Map<JournalFile, TransactionCallback> callbackList;
|
private final Map<JournalFile, TransactionCallback> callbackList = Collections.synchronizedMap(new HashMap<JournalFile, TransactionCallback>());
|
||||||
|
|
||||||
private JournalFile lastFile = null;
|
private JournalFile lastFile = null;
|
||||||
|
|
||||||
private final AtomicInteger counter = new AtomicInteger();
|
private final AtomicInteger counter = new AtomicInteger();
|
||||||
|
|
||||||
|
private CountDownLatch firstCallbackLatch;
|
||||||
|
|
||||||
public JournalTransaction(final long id, final JournalRecordProvider journal) {
|
public JournalTransaction(final long id, final JournalRecordProvider journal) {
|
||||||
this.id = id;
|
this.id = id;
|
||||||
this.journal = journal;
|
this.journal = journal;
|
||||||
|
@ -139,9 +143,7 @@ public class JournalTransaction {
|
||||||
pendingFiles.clear();
|
pendingFiles.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (callbackList != null) {
|
|
||||||
callbackList.clear();
|
callbackList.clear();
|
||||||
}
|
|
||||||
|
|
||||||
if (pos != null) {
|
if (pos != null) {
|
||||||
pos.clear();
|
pos.clear();
|
||||||
|
@ -156,6 +158,8 @@ public class JournalTransaction {
|
||||||
lastFile = null;
|
lastFile = null;
|
||||||
|
|
||||||
currentCallback = null;
|
currentCallback = null;
|
||||||
|
|
||||||
|
firstCallbackLatch = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -166,9 +170,13 @@ public class JournalTransaction {
|
||||||
data.setNumberOfRecords(getCounter(currentFile));
|
data.setNumberOfRecords(getCounter(currentFile));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public TransactionCallback getCurrentCallback() {
|
||||||
|
return currentCallback;
|
||||||
|
}
|
||||||
|
|
||||||
public TransactionCallback getCallback(final JournalFile file) throws Exception {
|
public TransactionCallback getCallback(final JournalFile file) throws Exception {
|
||||||
if (callbackList == null) {
|
if (firstCallbackLatch != null && callbackList.isEmpty()) {
|
||||||
callbackList = new HashMap<>();
|
firstCallbackLatch.countDown();
|
||||||
}
|
}
|
||||||
|
|
||||||
currentCallback = callbackList.get(file);
|
currentCallback = callbackList.get(file);
|
||||||
|
@ -178,15 +186,19 @@ public class JournalTransaction {
|
||||||
callbackList.put(file, currentCallback);
|
callbackList.put(file, currentCallback);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (currentCallback.getErrorMessage() != null) {
|
|
||||||
throw ActiveMQExceptionType.createException(currentCallback.getErrorCode(), currentCallback.getErrorMessage());
|
|
||||||
}
|
|
||||||
|
|
||||||
currentCallback.countUp();
|
currentCallback.countUp();
|
||||||
|
|
||||||
return currentCallback;
|
return currentCallback;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void checkErrorCondition() throws Exception {
|
||||||
|
if (currentCallback != null) {
|
||||||
|
if (currentCallback.getErrorMessage() != null) {
|
||||||
|
throw ActiveMQExceptionType.createException(currentCallback.getErrorCode(), currentCallback.getErrorMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public void addPositive(final JournalFile file, final long id, final int size) {
|
public void addPositive(final JournalFile file, final long id, final int size) {
|
||||||
incCounter(file);
|
incCounter(file);
|
||||||
|
|
||||||
|
@ -264,7 +276,8 @@ public class JournalTransaction {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void waitCallbacks() throws InterruptedException {
|
public void waitCallbacks() throws InterruptedException {
|
||||||
if (callbackList != null) {
|
waitFirstCallback();
|
||||||
|
synchronized (callbackList) {
|
||||||
for (TransactionCallback callback : callbackList.values()) {
|
for (TransactionCallback callback : callbackList.values()) {
|
||||||
callback.waitCompletion();
|
callback.waitCompletion();
|
||||||
}
|
}
|
||||||
|
@ -275,9 +288,16 @@ public class JournalTransaction {
|
||||||
* Wait completion at the latest file only
|
* Wait completion at the latest file only
|
||||||
*/
|
*/
|
||||||
public void waitCompletion() throws Exception {
|
public void waitCompletion() throws Exception {
|
||||||
if (currentCallback != null) {
|
waitFirstCallback();
|
||||||
currentCallback.waitCompletion();
|
currentCallback.waitCompletion();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void waitFirstCallback() throws InterruptedException {
|
||||||
|
if (currentCallback == null) {
|
||||||
|
firstCallbackLatch = new CountDownLatch(1);
|
||||||
|
firstCallbackLatch.await();
|
||||||
|
firstCallbackLatch = null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -143,7 +143,7 @@ public interface ActiveMQJournalLogger extends BasicLogger {
|
||||||
void compactReadError(JournalFile file);
|
void compactReadError(JournalFile file);
|
||||||
|
|
||||||
@LogMessage(level = Logger.Level.WARN)
|
@LogMessage(level = Logger.Level.WARN)
|
||||||
@Message(id = 142012, value = "Couldn''t find tx={0} to merge after compacting",
|
@Message(id = 142012, value = "Couldn't find tx={0} to merge after compacting",
|
||||||
format = Message.Format.MESSAGE_FORMAT)
|
format = Message.Format.MESSAGE_FORMAT)
|
||||||
void compactMergeError(Long id);
|
void compactMergeError(Long id);
|
||||||
|
|
||||||
|
@ -163,12 +163,12 @@ public interface ActiveMQJournalLogger extends BasicLogger {
|
||||||
void uncomittedTxFound(Long id);
|
void uncomittedTxFound(Long id);
|
||||||
|
|
||||||
@LogMessage(level = Logger.Level.WARN)
|
@LogMessage(level = Logger.Level.WARN)
|
||||||
@Message(id = 142016, value = "Couldn''t stop compactor executor after 120 seconds",
|
@Message(id = 142016, value = "Could not stop compactor executor after 120 seconds",
|
||||||
format = Message.Format.MESSAGE_FORMAT)
|
format = Message.Format.MESSAGE_FORMAT)
|
||||||
void couldNotStopCompactor();
|
void couldNotStopCompactor();
|
||||||
|
|
||||||
@LogMessage(level = Logger.Level.WARN)
|
@LogMessage(level = Logger.Level.WARN)
|
||||||
@Message(id = 142017, value = "Couldn''t stop journal executor after 60 seconds",
|
@Message(id = 142017, value = "Could not stop journal executor after 60 seconds",
|
||||||
format = Message.Format.MESSAGE_FORMAT)
|
format = Message.Format.MESSAGE_FORMAT)
|
||||||
void couldNotStopJournalExecutor();
|
void couldNotStopJournalExecutor();
|
||||||
|
|
||||||
|
@ -182,7 +182,7 @@ public interface ActiveMQJournalLogger extends BasicLogger {
|
||||||
void deletingOrphanedFile(String fileToDelete);
|
void deletingOrphanedFile(String fileToDelete);
|
||||||
|
|
||||||
@LogMessage(level = Logger.Level.WARN)
|
@LogMessage(level = Logger.Level.WARN)
|
||||||
@Message(id = 142020, value = "Couldn''t get lock after 60 seconds on closing Asynchronous File: {0}", format = Message.Format.MESSAGE_FORMAT)
|
@Message(id = 142020, value = "Could not get lock after 60 seconds on closing Asynchronous File: {0}", format = Message.Format.MESSAGE_FORMAT)
|
||||||
void errorClosingFile(String fileToDelete);
|
void errorClosingFile(String fileToDelete);
|
||||||
|
|
||||||
@LogMessage(level = Logger.Level.WARN)
|
@LogMessage(level = Logger.Level.WARN)
|
||||||
|
@ -241,6 +241,10 @@ public interface ActiveMQJournalLogger extends BasicLogger {
|
||||||
@Message(id = 142034, value = "Exception on submitting write", format = Message.Format.MESSAGE_FORMAT)
|
@Message(id = 142034, value = "Exception on submitting write", format = Message.Format.MESSAGE_FORMAT)
|
||||||
void errorSubmittingWrite(@Cause Throwable e);
|
void errorSubmittingWrite(@Cause Throwable e);
|
||||||
|
|
||||||
|
@LogMessage(level = Logger.Level.WARN)
|
||||||
|
@Message(id = 142035, value = "Could not stop journal append executor after 60 seconds", format = Message.Format.MESSAGE_FORMAT)
|
||||||
|
void couldNotStopJournalAppendExecutor();
|
||||||
|
|
||||||
@LogMessage(level = Logger.Level.ERROR)
|
@LogMessage(level = Logger.Level.ERROR)
|
||||||
@Message(id = 144000, value = "Failed to delete file {0}", format = Message.Format.MESSAGE_FORMAT)
|
@Message(id = 144000, value = "Failed to delete file {0}", format = Message.Format.MESSAGE_FORMAT)
|
||||||
void errorDeletingFile(Object e);
|
void errorDeletingFile(Object e);
|
||||||
|
|
|
@ -532,6 +532,8 @@ public class AlignedJournalImplTest extends ActiveMQTestBase {
|
||||||
|
|
||||||
journalImpl.appendCommitRecord(1L, false);
|
journalImpl.appendCommitRecord(1L, false);
|
||||||
|
|
||||||
|
journalImpl.debugWait();
|
||||||
|
|
||||||
System.out.println("Files = " + factory.listFiles("tt"));
|
System.out.println("Files = " + factory.listFiles("tt"));
|
||||||
|
|
||||||
SequentialFile file = factory.createSequentialFile("tt-1.tt");
|
SequentialFile file = factory.createSequentialFile("tt-1.tt");
|
||||||
|
@ -598,6 +600,8 @@ public class AlignedJournalImplTest extends ActiveMQTestBase {
|
||||||
|
|
||||||
journalImpl.appendCommitRecord(2L, false);
|
journalImpl.appendCommitRecord(2L, false);
|
||||||
|
|
||||||
|
journalImpl.debugWait();
|
||||||
|
|
||||||
SequentialFile file = factory.createSequentialFile("tt-1.tt");
|
SequentialFile file = factory.createSequentialFile("tt-1.tt");
|
||||||
|
|
||||||
file.open();
|
file.open();
|
||||||
|
@ -697,6 +701,8 @@ public class AlignedJournalImplTest extends ActiveMQTestBase {
|
||||||
|
|
||||||
journalImpl.appendCommitRecord(1L, false);
|
journalImpl.appendCommitRecord(1L, false);
|
||||||
|
|
||||||
|
journalImpl.debugWait();
|
||||||
|
|
||||||
SequentialFile file = factory.createSequentialFile("tt-1.tt");
|
SequentialFile file = factory.createSequentialFile("tt-1.tt");
|
||||||
|
|
||||||
file.open();
|
file.open();
|
||||||
|
@ -936,8 +942,7 @@ public class AlignedJournalImplTest extends ActiveMQTestBase {
|
||||||
|
|
||||||
journalImpl.forceMoveNextFile();
|
journalImpl.forceMoveNextFile();
|
||||||
|
|
||||||
// Reclaiming should still be able to reclaim a file if a transaction was
|
// Reclaiming should still be able to reclaim a file if a transaction was ignored
|
||||||
// ignored
|
|
||||||
journalImpl.checkReclaimStatus();
|
journalImpl.checkReclaimStatus();
|
||||||
|
|
||||||
Assert.assertEquals(2, factory.listFiles("tt").size());
|
Assert.assertEquals(2, factory.listFiles("tt").size());
|
||||||
|
@ -1109,7 +1114,16 @@ public class AlignedJournalImplTest extends ActiveMQTestBase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testReclaimingAfterConcurrentAddsAndDeletes() throws Exception {
|
public void testReclaimingAfterConcurrentAddsAndDeletesTx() throws Exception {
|
||||||
|
testReclaimingAfterConcurrentAddsAndDeletes(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReclaimingAfterConcurrentAddsAndDeletesNonTx() throws Exception {
|
||||||
|
testReclaimingAfterConcurrentAddsAndDeletes(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testReclaimingAfterConcurrentAddsAndDeletes(final boolean transactional) throws Exception {
|
||||||
final int JOURNAL_SIZE = 10 * 1024;
|
final int JOURNAL_SIZE = 10 * 1024;
|
||||||
|
|
||||||
setupAndLoadJournal(JOURNAL_SIZE, 1);
|
setupAndLoadJournal(JOURNAL_SIZE, 1);
|
||||||
|
@ -1131,8 +1145,14 @@ public class AlignedJournalImplTest extends ActiveMQTestBase {
|
||||||
latchReady.countDown();
|
latchReady.countDown();
|
||||||
ActiveMQTestBase.waitForLatch(latchStart);
|
ActiveMQTestBase.waitForLatch(latchStart);
|
||||||
for (int i = 0; i < NUMBER_OF_ELEMENTS; i++) {
|
for (int i = 0; i < NUMBER_OF_ELEMENTS; i++) {
|
||||||
|
|
||||||
|
if (transactional) {
|
||||||
journalImpl.appendAddRecordTransactional(i, i, (byte) 1, new SimpleEncoding(50, (byte) 1));
|
journalImpl.appendAddRecordTransactional(i, i, (byte) 1, new SimpleEncoding(50, (byte) 1));
|
||||||
journalImpl.appendCommitRecord(i, false);
|
journalImpl.appendCommitRecord(i, false);
|
||||||
|
} else {
|
||||||
|
journalImpl.appendAddRecord(i, (byte) 1, new SimpleEncoding(50, (byte) 1), false);
|
||||||
|
}
|
||||||
|
|
||||||
queueDelete.offer(i);
|
queueDelete.offer(i);
|
||||||
}
|
}
|
||||||
finishedOK.incrementAndGet();
|
finishedOK.incrementAndGet();
|
||||||
|
@ -1153,8 +1173,15 @@ public class AlignedJournalImplTest extends ActiveMQTestBase {
|
||||||
if (toDelete == null) {
|
if (toDelete == null) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (transactional) {
|
||||||
|
journalImpl.appendDeleteRecordTransactional(toDelete, toDelete, new SimpleEncoding(50, (byte) 1));
|
||||||
|
journalImpl.appendCommitRecord(i, false);
|
||||||
|
} else {
|
||||||
journalImpl.appendDeleteRecord(toDelete, false);
|
journalImpl.appendDeleteRecord(toDelete, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
}
|
||||||
finishedOK.incrementAndGet();
|
finishedOK.incrementAndGet();
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
e.printStackTrace();
|
e.printStackTrace();
|
||||||
|
|
|
@ -81,6 +81,8 @@ public class JournalAsyncTest extends ActiveMQTestBase {
|
||||||
journalImpl.appendAddRecordTransactional(1L, i, (byte) 1, new SimpleEncoding(1, (byte) 0));
|
journalImpl.appendAddRecordTransactional(1L, i, (byte) 1, new SimpleEncoding(1, (byte) 0));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
journalImpl.debugWait();
|
||||||
|
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
factory.setHoldCallbacks(false, null);
|
factory.setHoldCallbacks(false, null);
|
||||||
if (isCommit) {
|
if (isCommit) {
|
||||||
|
@ -115,8 +117,7 @@ public class JournalAsyncTest extends ActiveMQTestBase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// If a callback error already arrived, we should just throw the exception
|
// If a callback error already arrived, we should just throw the exception right away
|
||||||
// right away
|
|
||||||
@Test
|
@Test
|
||||||
public void testPreviousError() throws Exception {
|
public void testPreviousError() throws Exception {
|
||||||
final int JOURNAL_SIZE = 20000;
|
final int JOURNAL_SIZE = 20000;
|
||||||
|
@ -128,6 +129,8 @@ public class JournalAsyncTest extends ActiveMQTestBase {
|
||||||
|
|
||||||
journalImpl.appendAddRecordTransactional(1L, 1, (byte) 1, new SimpleEncoding(1, (byte) 0));
|
journalImpl.appendAddRecordTransactional(1L, 1, (byte) 1, new SimpleEncoding(1, (byte) 0));
|
||||||
|
|
||||||
|
journalImpl.debugWait();
|
||||||
|
|
||||||
factory.flushAllCallbacks();
|
factory.flushAllCallbacks();
|
||||||
|
|
||||||
factory.setGenerateErrors(false);
|
factory.setGenerateErrors(false);
|
||||||
|
@ -135,11 +138,11 @@ public class JournalAsyncTest extends ActiveMQTestBase {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
journalImpl.appendAddRecordTransactional(1L, 2, (byte) 1, new SimpleEncoding(1, (byte) 0));
|
journalImpl.appendAddRecordTransactional(1L, 2, (byte) 1, new SimpleEncoding(1, (byte) 0));
|
||||||
Assert.fail("Exception expected"); // An exception already happened in one
|
Assert.fail("Exception expected");
|
||||||
// of the elements on this transaction.
|
// An exception already happened in one of the elements on this transaction.
|
||||||
// We can't accept any more elements on
|
// We can't accept any more elements on the transaction
|
||||||
// the transaction
|
|
||||||
} catch (Exception ignored) {
|
} catch (Exception ignored) {
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue