Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[#ISSUE 8927] DLedger support lmq #8944

Open
wants to merge 3 commits into
base: develop
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
68 changes: 3 additions & 65 deletions store/src/main/java/org/apache/rocketmq/store/CommitLog.java
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@
import org.apache.rocketmq.store.ha.autoswitch.AutoSwitchHAService;
import org.apache.rocketmq.store.lock.AdaptiveBackOffSpinLockImpl;
import org.apache.rocketmq.store.logfile.MappedFile;
import org.apache.rocketmq.store.queue.MultiDispatchUtils;
import org.apache.rocketmq.store.util.LibC;
import org.rocksdb.RocksDBException;

Expand Down Expand Up @@ -624,6 +625,7 @@ public DispatchRequest checkMessageAndReturnSize(java.nio.ByteBuffer byteBuffer,

return dispatchRequest;
} catch (Exception e) {
e.printStackTrace();
}

return new DispatchRequest(-1, false /* success */);
Expand Down Expand Up @@ -1850,78 +1852,14 @@ class DefaultAppendMessageCallback implements AppendMessageCallback {
this.crc32ReservedLength = messageStoreConfig.isEnabledAppendPropCRC() ? CommitLog.CRC32_RESERVED_LEN : 0;
}

public AppendMessageResult handlePropertiesForLmqMsg(ByteBuffer preEncodeBuffer,
final MessageExtBrokerInner msgInner) {
if (msgInner.isEncodeCompleted()) {
return null;
}

try {
LmqDispatch.wrapLmqDispatch(defaultMessageStore, msgInner);
} catch (ConsumeQueueException e) {
if (e.getCause() instanceof RocksDBException) {
log.error("Failed to wrap multi-dispatch", e);
return new AppendMessageResult(AppendMessageStatus.ROCKSDB_ERROR);
}
log.error("Failed to wrap multi-dispatch", e);
return new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR);
}

msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));

final byte[] propertiesData =
msgInner.getPropertiesString() == null ? null : msgInner.getPropertiesString().getBytes(MessageDecoder.CHARSET_UTF8);

boolean needAppendLastPropertySeparator = enabledAppendPropCRC && propertiesData != null && propertiesData.length > 0
&& propertiesData[propertiesData.length - 1] != MessageDecoder.PROPERTY_SEPARATOR;

final int propertiesLength = (propertiesData == null ? 0 : propertiesData.length) + (needAppendLastPropertySeparator ? 1 : 0) + crc32ReservedLength;

if (propertiesLength > Short.MAX_VALUE) {
log.warn("putMessage message properties length too long. length={}", propertiesData.length);
return new AppendMessageResult(AppendMessageStatus.PROPERTIES_SIZE_EXCEEDED);
}

int msgLenWithoutProperties = preEncodeBuffer.getInt(0);

int msgLen = msgLenWithoutProperties + 2 + propertiesLength;

// Exceeds the maximum message
if (msgLen > this.messageStoreConfig.getMaxMessageSize()) {
log.warn("message size exceeded, msg total size: " + msgLen + ", maxMessageSize: " + this.messageStoreConfig.getMaxMessageSize());
return new AppendMessageResult(AppendMessageStatus.MESSAGE_SIZE_EXCEEDED);
}

// Back filling total message length
preEncodeBuffer.putInt(0, msgLen);
// Modify position to msgLenWithoutProperties
preEncodeBuffer.position(msgLenWithoutProperties);

preEncodeBuffer.putShort((short) propertiesLength);

if (propertiesLength > crc32ReservedLength) {
preEncodeBuffer.put(propertiesData);
}

if (needAppendLastPropertySeparator) {
preEncodeBuffer.put((byte) MessageDecoder.PROPERTY_SEPARATOR);
}
// 18 CRC32
preEncodeBuffer.position(preEncodeBuffer.position() + crc32ReservedLength);

msgInner.setEncodeCompleted(true);

return null;
}

public AppendMessageResult doAppend(final long fileFromOffset, final ByteBuffer byteBuffer, final int maxBlank,
final MessageExtBrokerInner msgInner, PutMessageContext putMessageContext) {
// STORETIMESTAMP + STOREHOSTADDRESS + OFFSET <br>

ByteBuffer preEncodeBuffer = msgInner.getEncodedBuff();
boolean isMultiDispatchMsg = messageStoreConfig.isEnableLmq() && msgInner.needDispatchLMQ();
if (isMultiDispatchMsg) {
AppendMessageResult appendMessageResult = handlePropertiesForLmqMsg(preEncodeBuffer, msgInner);
AppendMessageResult appendMessageResult = MultiDispatchUtils.handlePropertiesForLmqMsg(preEncodeBuffer, msgInner, defaultMessageStore);
if (appendMessageResult != null) {
return appendMessageResult;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,8 +132,8 @@ public PutMessageResult encodeWithoutProperties(MessageExtBrokerInner msgInner)
this.byteBuf.writeInt(msgInner.getQueueId());
// 5 FLAG
this.byteBuf.writeInt(msgInner.getFlag());
// 6 QUEUEOFFSET, need update later
this.byteBuf.writeLong(0);
// 6 QUEUEOFFSET
this.byteBuf.writeLong(msgInner.getQueueOffset());
// 7 PHYSICALOFFSET, need update later
this.byteBuf.writeLong(0);
// 8 SYSFLAG
Expand Down Expand Up @@ -416,7 +416,7 @@ public void updateEncoderBufferCapacity(int newMaxMessageBodySize) {
this.byteBuf.capacity(this.maxMessageSize);
}

static class PutMessageThreadLocal {
public static class PutMessageThreadLocal {
private final MessageExtEncoder encoder;
private final StringBuilder keyBuilder;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,13 +34,17 @@
import org.apache.rocketmq.store.CommitLog;
import org.apache.rocketmq.store.DefaultMessageStore;
import org.apache.rocketmq.store.DispatchRequest;
import org.apache.rocketmq.store.LmqDispatch;
import org.apache.rocketmq.store.MessageExtEncoder;
import org.apache.rocketmq.store.MessageExtEncoder.PutMessageThreadLocal;
import org.apache.rocketmq.store.PutMessageResult;
import org.apache.rocketmq.store.PutMessageStatus;
import org.apache.rocketmq.store.SelectMappedBufferResult;
import org.apache.rocketmq.store.StoreStatsService;
import org.apache.rocketmq.store.config.MessageStoreConfig;
import org.apache.rocketmq.store.exception.ConsumeQueueException;
import org.apache.rocketmq.store.logfile.MappedFile;
import org.apache.rocketmq.store.queue.MultiDispatchUtils;
import org.rocksdb.RocksDBException;

import io.openmessaging.storage.dledger.AppendFuture;
Expand Down Expand Up @@ -566,28 +570,42 @@ public CompletableFuture<PutMessageResult> asyncPutMessage(MessageExtBrokerInner
// Back to Results
AppendMessageResult appendResult;
AppendFuture<AppendEntryResponse> dledgerFuture;
EncodeResult encodeResult;
PutMessageThreadLocal putMessageThreadLocal = this.getPutMessageThreadLocal().get();
final boolean isMultiDispatchMsg = this.getMessageStore().getMessageStoreConfig().isEnableLmq() && msg.needDispatchLMQ();

String topicQueueKey = msg.getTopic() + "-" + msg.getQueueId();
topicQueueLock.lock(topicQueueKey);
try {
defaultMessageStore.assignOffset(msg);

encodeResult = this.messageSerializer.serialize(msg);
if (encodeResult.status != AppendMessageStatus.PUT_OK) {
return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, new AppendMessageResult(encodeResult.status)));
PutMessageResult putMessageResult = putMessageThreadLocal.getEncoder().encode(msg);
if (putMessageResult != null) {
return CompletableFuture.completedFuture(putMessageResult);
}
msg.setEncodedBuff(putMessageThreadLocal.getEncoder().getEncoderBuffer());
putMessageLock.lock(); //spin or ReentrantLock ,depending on store config
long elapsedTimeInLock;
long queueOffset;
ByteBuffer preEncoder = msg.getEncodedBuff();
if (isMultiDispatchMsg) {
AppendMessageResult appendMessageResult = MultiDispatchUtils.handlePropertiesForLmqMsg(
preEncoder, msg, defaultMessageStore);
if (appendMessageResult != null) {
return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, appendMessageResult));
}
}
final int msgLen = preEncoder.getInt(0);
preEncoder.position(0);
preEncoder.limit(msgLen);
try {
beginTimeInDledgerLock = this.defaultMessageStore.getSystemClock().now();
queueOffset = getQueueOffsetByKey(msg, tranType);
encodeResult.setQueueOffsetKey(queueOffset, false);
AppendEntryRequest request = new AppendEntryRequest();
request.setGroup(dLedgerConfig.getGroup());
request.setRemoteId(dLedgerServer.getMemberState().getSelfId());
request.setBody(encodeResult.getData());
byte[] data = new byte[msgLen];
preEncoder.get(data);
request.setBody(data);
dledgerFuture = (AppendFuture<AppendEntryResponse>) dLedgerServer.handleAppend(request);
if (dledgerFuture.getPos() == -1) {
return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.OS_PAGE_CACHE_BUSY, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
Expand All @@ -599,7 +617,16 @@ public CompletableFuture<PutMessageResult> asyncPutMessage(MessageExtBrokerInner

String msgId = MessageDecoder.createMessageId(buffer, msg.getStoreHostBytes(), wroteOffset);
elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginTimeInDledgerLock;
appendResult = new AppendMessageResult(AppendMessageStatus.PUT_OK, wroteOffset, encodeResult.getData().length, msgId, System.currentTimeMillis(), queueOffset, elapsedTimeInLock);
appendResult = new AppendMessageResult(AppendMessageStatus.PUT_OK, wroteOffset, data.length, msgId, System.currentTimeMillis(), queueOffset, elapsedTimeInLock);
if (isMultiDispatchMsg) {
try {
LmqDispatch.updateLmqOffsets(defaultMessageStore, msg);
} catch (ConsumeQueueException e) {
// Increase in-memory max offset of the queue should not fail.
log.error("[BUG] DLedger update lmq offset failed");
return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
}
}
} finally {
beginTimeInDledgerLock = 0;
putMessageLock.unlock();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,16 +16,30 @@
*/
package org.apache.rocketmq.store.queue;

import java.nio.ByteBuffer;
import java.util.Map;

import org.apache.commons.lang3.StringUtils;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.constant.LoggerName;
import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageDecoder;
import org.apache.rocketmq.common.message.MessageExtBrokerInner;
import org.apache.rocketmq.common.topic.TopicValidator;
import org.apache.rocketmq.logging.org.slf4j.Logger;
import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
import org.apache.rocketmq.store.AppendMessageResult;
import org.apache.rocketmq.store.AppendMessageStatus;
import org.apache.rocketmq.store.CommitLog;
import org.apache.rocketmq.store.DefaultMessageStore;
import org.apache.rocketmq.store.DispatchRequest;
import org.apache.rocketmq.store.LmqDispatch;
import org.apache.rocketmq.store.config.MessageStoreConfig;
import org.apache.rocketmq.store.exception.ConsumeQueueException;
import org.rocksdb.RocksDBException;

public class MultiDispatchUtils {
protected static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);

public static String lmqQueueKey(String queueName) {
StringBuilder keyBuilder = new StringBuilder();
Expand Down Expand Up @@ -58,4 +72,71 @@ public static boolean checkMultiDispatchQueue(MessageStoreConfig messageStoreCon
}
return true;
}

public static AppendMessageResult handlePropertiesForLmqMsg(ByteBuffer preEncodeBuffer, final MessageExtBrokerInner msgInner,
DefaultMessageStore defaultMessageStore) {
if (msgInner.isEncodeCompleted()) {
return null;
}

boolean enabledAppendPropCRC = defaultMessageStore.getMessageStoreConfig().isEnabledAppendPropCRC();
int crc32ReservedLength = defaultMessageStore.getMessageStoreConfig().isEnabledAppendPropCRC() ? CommitLog.CRC32_RESERVED_LEN : 0;

try {
LmqDispatch.wrapLmqDispatch(defaultMessageStore, msgInner);
} catch (ConsumeQueueException e) {
if (e.getCause() instanceof RocksDBException) {
log.error("Failed to wrap multi-dispatch", e);
return new AppendMessageResult(AppendMessageStatus.ROCKSDB_ERROR);
}
log.error("Failed to wrap multi-dispatch", e);
return new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR);
}

msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));

final byte[] propertiesData =
msgInner.getPropertiesString() == null ? null : msgInner.getPropertiesString().getBytes(MessageDecoder.CHARSET_UTF8);

boolean needAppendLastPropertySeparator = enabledAppendPropCRC && propertiesData != null && propertiesData.length > 0
&& propertiesData[propertiesData.length - 1] != MessageDecoder.PROPERTY_SEPARATOR;

final int propertiesLength = (propertiesData == null ? 0 : propertiesData.length) + (needAppendLastPropertySeparator ? 1 : 0) + crc32ReservedLength;

if (propertiesLength > Short.MAX_VALUE) {
log.warn("putMessage message properties length too long. length={}", propertiesData.length);
return new AppendMessageResult(AppendMessageStatus.PROPERTIES_SIZE_EXCEEDED);
}

int msgLenWithoutProperties = preEncodeBuffer.getInt(0);

int msgLen = msgLenWithoutProperties + 2 + propertiesLength;

// Exceeds the maximum message
if (msgLen > defaultMessageStore.getMessageStoreConfig().getMaxMessageSize()) {
log.warn("message size exceeded, msg total size: " + msgLen + ", maxMessageSize: " + defaultMessageStore.getMessageStoreConfig().getMaxMessageSize());
return new AppendMessageResult(AppendMessageStatus.MESSAGE_SIZE_EXCEEDED);
}

// Back filling total message length
preEncodeBuffer.putInt(0, msgLen);
// Modify position to msgLenWithoutProperties
preEncodeBuffer.position(msgLenWithoutProperties);

preEncodeBuffer.putShort((short) propertiesLength);

if (propertiesLength > crc32ReservedLength) {
preEncodeBuffer.put(propertiesData);
}

if (needAppendLastPropertySeparator) {
preEncodeBuffer.put((byte) MessageDecoder.PROPERTY_SEPARATOR);
}
// 18 CRC32
preEncodeBuffer.position(preEncodeBuffer.position() + crc32ReservedLength);

msgInner.setEncodeCompleted(true);

return null;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@
import java.util.concurrent.Callable;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;

import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageDecoder;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.message.MessageExtBatch;
Expand Down Expand Up @@ -436,6 +438,54 @@ public void testIPv6HostMsgCommittedPos() throws Exception {
followerStore.shutdown();
}

@Test
public void testMultiDispatch() throws Exception {
String base = createBaseDir();
String peers = String.format("n0-localhost:%d", nextPort());
String group = UUID.randomUUID().toString();
DefaultMessageStore messageStore = createDledgerMessageStore(base, group, "n0", peers, null, false, 0);
messageStore.getMessageStoreConfig().setEnableLmq(true);
DLedgerCommitLog dLedgerCommitLog = (DLedgerCommitLog) messageStore.getCommitLog();
Boolean success = await().atMost(Duration.ofSeconds(4)).until(() -> dLedgerCommitLog.getdLedgerServer().getMemberState().isLeader(), item -> item);
Assert.assertTrue(success);
String topic = UUID.randomUUID().toString();

List<PutMessageResult> results = new ArrayList<>();
for (int i = 0; i < 10; i++) {
MessageExtBrokerInner msgInner = buildMessage();
msgInner.setTopic(topic);
msgInner.setQueueId(0);
msgInner.putUserProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH, "%LMQ%123,%LMQ%456");
PutMessageResult putMessageResult = messageStore.putMessage(msgInner);
results.add(putMessageResult);
Assert.assertEquals(PutMessageStatus.PUT_OK, putMessageResult.getPutMessageStatus());
Assert.assertEquals(i, putMessageResult.getAppendMessageResult().getLogicsOffset());
}
await().atMost(Duration.ofSeconds(10)).until(() -> 10 == messageStore.getMaxOffsetInQueue(topic, 0));
Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, 0));
Assert.assertEquals(10, messageStore.getMaxOffsetInQueue(topic, 0));
Assert.assertEquals(0, messageStore.dispatchBehindBytes());
GetMessageResult getMessageResult = messageStore.getMessage("group", topic, 0, 0, 32, null);
Assert.assertEquals(GetMessageStatus.FOUND, getMessageResult.getStatus());
Assert.assertEquals(10, getMessageResult.getMessageBufferList().size());
Assert.assertEquals(10, getMessageResult.getMessageMapedList().size());

for (int i = 0; i < results.size(); i++) {
ByteBuffer buffer = getMessageResult.getMessageBufferList().get(i);
MessageExt messageExt = MessageDecoder.decode(buffer);
Assert.assertEquals(i, messageExt.getQueueOffset());
Assert.assertEquals(results.get(i).getAppendMessageResult().getMsgId(), messageExt.getMsgId());
Assert.assertEquals(results.get(i).getAppendMessageResult().getWroteOffset(), messageExt.getCommitLogOffset());
}

Assert.assertEquals(10, messageStore.getConsumeQueueStore().getLmqQueueOffset("%LMQ%123", 0));
Assert.assertEquals(10, messageStore.getConsumeQueueStore().getLmqQueueOffset("%LMQ%456", 0));

messageStore.destroy();
messageStore.shutdown();
}


private Callable<Boolean> followerCatchesUp(DefaultMessageStore followerStore, String topic) {
return () -> followerStore.getMaxOffsetInQueue(topic, 0) == 1;
}
Expand Down
Loading