ARTEMIS-1089 Fixing Replication catchup slow

This commit is contained in:
Clebert Suconic 2017-04-02 19:20:42 -04:00
parent 739dd82f31
commit 0b62f698c3
7 changed files with 88 additions and 65 deletions

View File

@ -332,6 +332,7 @@ public class PageCursorProviderImpl implements PageCursorProvider {
@Override @Override
public void resumeCleanup() { public void resumeCleanup() {
this.cleanupEnabled = true; this.cleanupEnabled = true;
scheduleCleanup();
} }
@Override @Override

View File

@ -1123,6 +1123,8 @@ public class PagingStoreImpl implements PagingStore {
@Override @Override
public Collection<Integer> getCurrentIds() throws Exception { public Collection<Integer> getCurrentIds() throws Exception {
lock.writeLock().lock();
try {
List<Integer> ids = new ArrayList<>(); List<Integer> ids = new ArrayList<>();
if (fileFactory != null) { if (fileFactory != null) {
for (String fileName : fileFactory.listFiles("page")) { for (String fileName : fileFactory.listFiles("page")) {
@ -1130,23 +1132,22 @@ public class PagingStoreImpl implements PagingStore {
} }
} }
return ids; return ids;
}
@Override
public void sendPages(ReplicationManager replicator, Collection<Integer> pageIds) throws Exception {
lock.writeLock().lock();
try {
for (Integer id : pageIds) {
SequentialFile sFile = fileFactory.createSequentialFile(createFileName(id));
if (!sFile.exists()) {
continue;
}
replicator.syncPages(sFile, id, getAddress());
}
} finally { } finally {
lock.writeLock().unlock(); lock.writeLock().unlock();
} }
} }
@Override
public void sendPages(ReplicationManager replicator, Collection<Integer> pageIds) throws Exception {
for (Integer id : pageIds) {
SequentialFile sFile = fileFactory.createSequentialFile(createFileName(id));
if (!sFile.exists()) {
continue;
}
ActiveMQServerLogger.LOGGER.replicaSyncFile(sFile, sFile.size());
replicator.syncPages(sFile, id, getAddress());
}
}
// Inner classes ------------------------------------------------- // Inner classes -------------------------------------------------
} }

View File

@ -587,10 +587,10 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
stopReplication(); stopReplication();
throw e; throw e;
} finally { } finally {
pagingManager.resumeCleanup();
// Re-enable compact and reclaim of journal files // Re-enable compact and reclaim of journal files
originalBindingsJournal.replicationSyncFinished(); originalBindingsJournal.replicationSyncFinished();
originalMessageJournal.replicationSyncFinished(); originalMessageJournal.replicationSyncFinished();
pagingManager.resumeCleanup();
} }
} }

View File

@ -16,11 +16,11 @@
*/ */
package org.apache.activemq.artemis.core.protocol.core.impl.wireformat; package org.apache.activemq.artemis.core.protocol.core.impl.wireformat;
import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.Set; import java.util.Set;
import io.netty.buffer.ByteBuf;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer; import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.SimpleString; import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.persistence.impl.journal.AbstractJournalStorageManager; import org.apache.activemq.artemis.core.persistence.impl.journal.AbstractJournalStorageManager;
@ -42,7 +42,7 @@ public final class ReplicationSyncFileMessage extends PacketImpl {
*/ */
private long fileId; private long fileId;
private int dataSize; private int dataSize;
private ByteBuffer byteBuffer; private ByteBuf byteBuffer;
private byte[] byteArray; private byte[] byteArray;
private SimpleString pageStoreName; private SimpleString pageStoreName;
private FileType fileType; private FileType fileType;
@ -78,7 +78,7 @@ public final class ReplicationSyncFileMessage extends PacketImpl {
SimpleString storeName, SimpleString storeName,
long id, long id,
int size, int size,
ByteBuffer buffer) { ByteBuf buffer) {
this(); this();
this.byteBuffer = buffer; this.byteBuffer = buffer;
this.pageStoreName = storeName; this.pageStoreName = storeName;
@ -124,7 +124,12 @@ public final class ReplicationSyncFileMessage extends PacketImpl {
* (which might receive appends) * (which might receive appends)
*/ */
if (dataSize > 0) { if (dataSize > 0) {
buffer.writeBytes(byteBuffer); buffer.writeBytes(byteBuffer, 0, byteBuffer.writerIndex());
}
if (byteBuffer != null) {
byteBuffer.release();
byteBuffer = null;
} }
} }

View File

@ -411,7 +411,7 @@ public final class ReplicationEndpoint implements ChannelHandler, ActiveMQCompon
if (!channel1.isOpen()) { if (!channel1.isOpen()) {
channel1.open(); channel1.open();
} }
channel1.writeDirect(ByteBuffer.wrap(data), true); channel1.writeDirect(ByteBuffer.wrap(data), false);
} }
/** /**

View File

@ -25,8 +25,11 @@ import java.util.Map;
import java.util.Queue; import java.util.Queue;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.PooledByteBufAllocator;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer; import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
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;
@ -122,6 +125,8 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
private final ExecutorFactory executorFactory; private final ExecutorFactory executorFactory;
private final Executor replicationStream;
private SessionFailureListener failureListener; private SessionFailureListener failureListener;
private CoreRemotingConnection remotingConnection; private CoreRemotingConnection remotingConnection;
@ -141,6 +146,7 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
this.executorFactory = executorFactory; this.executorFactory = executorFactory;
this.replicatingChannel = remotingConnection.getChannel(CHANNEL_ID.REPLICATION.id, -1); this.replicatingChannel = remotingConnection.getChannel(CHANNEL_ID.REPLICATION.id, -1);
this.remotingConnection = remotingConnection; this.remotingConnection = remotingConnection;
this.replicationStream = executorFactory.getExecutor();
this.timeout = timeout; this.timeout = timeout;
} }
@ -178,7 +184,7 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
boolean sync, boolean sync,
final boolean lineUp) throws Exception { final boolean lineUp) throws Exception {
if (enabled) { if (enabled) {
sendReplicatePacket(new ReplicationCommitMessage(journalID, false, txID), lineUp); sendReplicatePacket(new ReplicationCommitMessage(journalID, false, txID), lineUp, true);
} }
} }
@ -343,15 +349,15 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
} }
private OperationContext sendReplicatePacket(final Packet packet) { private OperationContext sendReplicatePacket(final Packet packet) {
return sendReplicatePacket(packet, true); return sendReplicatePacket(packet, true, true);
} }
private OperationContext sendReplicatePacket(final Packet packet, boolean lineUp) { private OperationContext sendReplicatePacket(final Packet packet, boolean lineUp, boolean useExecutor) {
if (!enabled) if (!enabled)
return null; return null;
boolean runItNow = false; boolean runItNow = false;
OperationContext repliToken = OperationContextImpl.getContext(executorFactory); final OperationContext repliToken = OperationContextImpl.getContext(executorFactory);
if (lineUp) { if (lineUp) {
repliToken.replicationLineUp(); repliToken.replicationLineUp();
} }
@ -359,10 +365,17 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
synchronized (replicationLock) { synchronized (replicationLock) {
if (enabled) { if (enabled) {
pendingTokens.add(repliToken); pendingTokens.add(repliToken);
if (!flowControl()) { if (useExecutor) {
return repliToken; replicationStream.execute(() -> {
} if (enabled) {
flowControl();
replicatingChannel.send(packet); replicatingChannel.send(packet);
}
});
} else {
flowControl();
replicatingChannel.send(packet);
}
} else { } else {
// Already replicating channel failed, so just play the action now // Already replicating channel failed, so just play the action now
runItNow = true; runItNow = true;
@ -383,11 +396,12 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
* In case you refactor this in any way, this method must hold a lock on replication lock. . * In case you refactor this in any way, this method must hold a lock on replication lock. .
*/ */
private boolean flowControl() { private boolean flowControl() {
synchronized (replicationLock) {
// synchronized (replicationLock) { -- I'm not adding this because the caller already has it // synchronized (replicationLock) { -- I'm not adding this because the caller already has it
// future maintainers of this code please be aware that the intention here is hold the lock on replication lock // future maintainers of this code please be aware that the intention here is hold the lock on replication lock
if (!replicatingChannel.getConnection().isWritable(this)) { if (!replicatingChannel.getConnection().isWritable(this)) {
try { try {
logger.trace("flowControl waiting on writable"); logger.trace("flowControl waiting on writable replication");
writable.set(false); writable.set(false);
//don't wait for ever as this may hang tests etc, we've probably been closed anyway //don't wait for ever as this may hang tests etc, we've probably been closed anyway
long now = System.currentTimeMillis(); long now = System.currentTimeMillis();
@ -396,7 +410,7 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
replicationLock.wait(deadline - now); replicationLock.wait(deadline - now);
now = System.currentTimeMillis(); now = System.currentTimeMillis();
} }
logger.trace("flow control done"); logger.trace("flow control done on replication");
if (!writable.get()) { if (!writable.get()) {
ActiveMQServerLogger.LOGGER.slowReplicationResponse(); ActiveMQServerLogger.LOGGER.slowReplicationResponse();
@ -412,6 +426,7 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
throw new ActiveMQInterruptedException(e); throw new ActiveMQInterruptedException(e);
} }
} }
}
return true; return true;
} }
@ -515,7 +530,7 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
} }
SequentialFile file = jf.getFile().cloneFile(); SequentialFile file = jf.getFile().cloneFile();
try { try {
ActiveMQServerLogger.LOGGER.journalSynch(jf, file.size(), file); ActiveMQServerLogger.LOGGER.replicaSyncFile(file, file.size());
sendLargeFile(content, null, jf.getFileID(), file, Long.MAX_VALUE); sendLargeFile(content, null, jf.getFileID(), file, Long.MAX_VALUE);
} finally { } finally {
if (file.isOpen()) if (file.isOpen())
@ -560,10 +575,11 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
// We can afford having a single buffer here for this entire loop // We can afford having a single buffer here for this entire loop
// because sendReplicatePacket will encode the packet as a NettyBuffer // because sendReplicatePacket will encode the packet as a NettyBuffer
// through ActiveMQBuffer class leaving this buffer free to be reused on the next copy // through ActiveMQBuffer class leaving this buffer free to be reused on the next copy
final ByteBuffer buffer = ByteBuffer.allocate(1 << 17); // 1 << 17 == 131072 == 128 * 1024 int size = 1 << 17;
while (true) { while (true) {
buffer.clear(); final ByteBuf buffer = PooledByteBufAllocator.DEFAULT.directBuffer(size);
final int bytesRead = channel.read(buffer); ByteBuffer byteBuffer = buffer.writerIndex(size).readerIndex(0).nioBuffer();
final int bytesRead = channel.read(byteBuffer);
int toSend = bytesRead; int toSend = bytesRead;
if (bytesRead > 0) { if (bytesRead > 0) {
if (bytesRead >= maxBytesToSend) { if (bytesRead >= maxBytesToSend) {
@ -572,12 +588,13 @@ public final class ReplicationManager implements ActiveMQComponent, ReadyListene
} else { } else {
maxBytesToSend = maxBytesToSend - bytesRead; maxBytesToSend = maxBytesToSend - bytesRead;
} }
buffer.limit(toSend);
} }
buffer.rewind(); logger.debug("sending " + buffer.writerIndex() + " bytes on file " + file.getFileName());
// sending -1 or 0 bytes will close the file at the backup // sending -1 or 0 bytes will close the file at the backup
sendReplicatePacket(new ReplicationSyncFileMessage(content, pageStore, id, toSend, buffer)); // We cannot simply send everything of a file through the executor,
// otherwise we would run out of memory.
// so we don't use the executor here
sendReplicatePacket(new ReplicationSyncFileMessage(content, pageStore, id, toSend, buffer), true, false);
if (bytesRead == -1 || bytesRead == 0 || maxBytesToSend == 0) if (bytesRead == -1 || bytesRead == 0 || maxBytesToSend == 0)
break; break;
} }

View File

@ -47,7 +47,6 @@ import org.apache.activemq.artemis.core.client.impl.ServerLocatorInternal;
import org.apache.activemq.artemis.core.config.Configuration; import org.apache.activemq.artemis.core.config.Configuration;
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.journal.impl.JournalFile;
import org.apache.activemq.artemis.core.paging.cursor.PagePosition; import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
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.OperationContext;
@ -189,8 +188,8 @@ public interface ActiveMQServerLogger extends BasicLogger {
void backupServerSynched(ActiveMQServerImpl server); void backupServerSynched(ActiveMQServerImpl server);
@LogMessage(level = Logger.Level.INFO) @LogMessage(level = Logger.Level.INFO)
@Message(id = 221025, value = "Replication: sending {0} (size={1}) to backup. {2}", format = Message.Format.MESSAGE_FORMAT) @Message(id = 221025, value = "Replication: sending {0} (size={1}) to replica.", format = Message.Format.MESSAGE_FORMAT)
void journalSynch(JournalFile jf, Long size, SequentialFile file); void replicaSyncFile(SequentialFile jf, Long size);
@LogMessage(level = Logger.Level.INFO) @LogMessage(level = Logger.Level.INFO)
@Message( @Message(