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 #8764] Implement consume lag estimation in cq rocksdb store #8800

Merged
merged 7 commits into from
Oct 15, 2024
Merged
Show file tree
Hide file tree
Changes from 5 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
Original file line number Diff line number Diff line change
Expand Up @@ -168,12 +168,6 @@ public void run() {
}
}

@Override
public long estimateMessageCount(String topic, int queueId, long from, long to, MessageFilter filter) {
// todo
return 0;
}

@Override
public void initMetrics(Meter meter, Supplier<AttributesBuilder> attributesBuilderSupplier) {
DefaultStoreMetricsManager.init(meter, attributesBuilderSupplier, this);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -222,10 +222,48 @@ public void increaseQueueOffset(QueueOffsetOperator queueOffsetOperator, Message

@Override
public long estimateMessageCount(long from, long to, MessageFilter filter) {
// todo
return 0;
// Check from and to offset validity
Pair<CqUnit, Long> fromUnit = getCqUnitAndStoreTime(from);
Pair<CqUnit, Long> toUnit = getCqUnitAndStoreTime(to);
if (toUnit == null || fromUnit == null) {
return -1;
}

if (from >= to) {
return -1;
}

if (to > getMaxOffsetInQueue()) {
to = getMaxOffsetInQueue();
}

int maxSampleSize = messageStore.getMessageStoreConfig().getMaxConsumeQueueScan();
int sampleSize = to - from > maxSampleSize ? maxSampleSize : (int) (to - from);

int matchThreshold = messageStore.getMessageStoreConfig().getSampleCountThreshold();
int matchSize = 0;

for (int i = 0; i < sampleSize; i++) {
long index = from + i;
Pair<CqUnit, Long> pair = getCqUnitAndStoreTime(index);
if (pair == null) {
continue;
}
CqUnit cqUnit = pair.getObject1();
if (filter.isMatchedByConsumeQueue(cqUnit.getTagsCode(), cqUnit.getCqExtUnit())) {
matchSize++;
// if matchSize is plenty, early exit estimate
if (matchSize > matchThreshold) {
sampleSize = i;
break;
}
}
}
// Make sure the second half is a floating point number, otherwise it will be truncated to 0
return sampleSize == 0 ? 0 : (long) ((to - from) * (matchSize / (sampleSize * 1.0)));
}


@Override
public long getMinOffsetInQueue() {
return this.messageStore.getMinOffsetInQueue(this.topic, this.queueId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.rocketmq.store.DispatchRequest;
import org.apache.rocketmq.store.MessageFilter;
import org.apache.rocketmq.store.MessageStore;
import org.apache.rocketmq.store.RocksDBMessageStore;
import org.apache.rocketmq.store.config.MessageStoreConfig;
import org.apache.rocketmq.store.stats.BrokerStatsManager;
import org.junit.Assert;
Expand Down Expand Up @@ -84,7 +85,26 @@ messageStoreConfig, new BrokerStatsManager(brokerConfig),
return master;
}

protected void putMsg(DefaultMessageStore messageStore) throws Exception {
protected RocksDBMessageStore genRocksdbMessageStore() throws Exception {
MessageStoreConfig messageStoreConfig = buildStoreConfig(
COMMIT_LOG_FILE_SIZE, CQ_FILE_SIZE, true, CQ_EXT_FILE_SIZE
);

BrokerConfig brokerConfig = new BrokerConfig();

RocksDBMessageStore master = new RocksDBMessageStore(
messageStoreConfig, new BrokerStatsManager(brokerConfig),
(topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> {
}, brokerConfig, new ConcurrentHashMap<>());

assertThat(master.load()).isTrue();

master.start();

return master;
}

protected void putMsg(MessageStore messageStore) {
int totalMsgs = 200;
for (int i = 0; i < totalMsgs; i++) {
MessageExtBrokerInner message = buildMessage();
Expand Down Expand Up @@ -184,9 +204,30 @@ public void testIterator() throws Exception {

@Test
public void testEstimateMessageCountInEmptyConsumeQueue() {
DefaultMessageStore master = null;
DefaultMessageStore messageStore = null;
try {
messageStore = gen();
doTestEstimateMessageCountInEmptyConsumeQueue(messageStore);
} catch (Exception e) {
e.printStackTrace();
assertThat(Boolean.FALSE).isTrue();
}
}

@Test
public void testEstimateRocksdbMessageCountInEmptyConsumeQueue() {
DefaultMessageStore messageStore = null;
try {
messageStore = genRocksdbMessageStore();
doTestEstimateMessageCountInEmptyConsumeQueue(messageStore);
} catch (Exception e) {
e.printStackTrace();
assertThat(Boolean.FALSE).isTrue();
}
}

public void doTestEstimateMessageCountInEmptyConsumeQueue(MessageStore master) {
try {
master = gen();
ConsumeQueueInterface consumeQueue = master.findConsumeQueue(TOPIC, QUEUE_ID);
MessageFilter filter = new MessageFilter() {
@Override
Expand Down Expand Up @@ -219,16 +260,31 @@ public boolean isMatchedByCommitLog(ByteBuffer msgBuffer, Map<String, String> pr
}
}

@Test
public void testEstimateRocksdbMessageCount() {
DefaultMessageStore messageStore = null;
try {
messageStore = genRocksdbMessageStore();
doTestEstimateMessageCount(messageStore);
} catch (Exception e) {
e.printStackTrace();
assertThat(Boolean.FALSE).isTrue();
}
}

@Test
public void testEstimateMessageCount() {
DefaultMessageStore messageStore = null;
try {
messageStore = gen();
doTestEstimateMessageCount(messageStore);
} catch (Exception e) {
e.printStackTrace();
assertThat(Boolean.FALSE).isTrue();
}
}

public void doTestEstimateMessageCount(MessageStore messageStore) {
try {
try {
putMsg(messageStore);
Expand Down Expand Up @@ -265,15 +321,31 @@ public boolean isMatchedByCommitLog(ByteBuffer msgBuffer, Map<String, String> pr
}
}

@Test
public void testEstimateRocksdbMessageCountSample() {
DefaultMessageStore messageStore = null;
try {
messageStore = genRocksdbMessageStore();
doTestEstimateMessageCountSample(messageStore);
} catch (Exception e) {
e.printStackTrace();
assertThat(Boolean.FALSE).isTrue();
}
}

@Test
public void testEstimateMessageCountSample() {
DefaultMessageStore messageStore = null;
try {
messageStore = gen();
doTestEstimateMessageCountSample(messageStore);
} catch (Exception e) {
e.printStackTrace();
assertThat(Boolean.FALSE).isTrue();
}
}

public void doTestEstimateMessageCountSample(MessageStore messageStore) {

try {
try {
Expand Down
Loading