Skip to content
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.rocketmq.tieredstore.metadata.MetadataStore;
import org.apache.rocketmq.tieredstore.metadata.entity.QueueMetadata;
import org.apache.rocketmq.tieredstore.metadata.entity.TopicMetadata;
import org.apache.rocketmq.tieredstore.provider.FileSegment;
import org.apache.rocketmq.tieredstore.util.MessageFormatUtil;
import org.apache.rocketmq.tieredstore.util.MessageStoreUtil;
import org.slf4j.Logger;
Expand Down Expand Up @@ -302,9 +303,26 @@ public CompletableFuture<Long> getQueueOffsetByTimeAsync(long timestamp, Boundar
return CompletableFuture.completedFuture(cqMin);
}

// get correct consume queue file by binary search
List<FileSegment> consumeQueueFileList = this.consumeQueue.getFileSegmentList();
int low = 0, high = consumeQueueFileList.size() - 1;
int mid = low + (high - low) / 2;
while (low <= high) {
FileSegment fileSegment = consumeQueueFileList.get(mid);
if (fileSegment.getMinTimestamp() <= timestamp && timestamp <= fileSegment.getMaxTimestamp()) {
break;
} else if (timestamp < fileSegment.getMinTimestamp()) {
high = mid - 1;
} else {
low = mid + 1;
}
mid = low + (high - low) / 2;
}
FileSegment target = consumeQueueFileList.get(mid);

// binary search lower bound index in a sorted array
long minOffset = cqMin;
long maxOffset = cqMax;
long minOffset = target.getBaseOffset() / MessageFormatUtil.CONSUME_QUEUE_UNIT_SIZE;
long maxOffset = target.getCommitOffset() / MessageFormatUtil.CONSUME_QUEUE_UNIT_SIZE - 1;
List<String> queryLog = new ArrayList<>();
while (minOffset < maxOffset) {
long middle = minOffset + (maxOffset - minOffset) / 2;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -177,13 +177,35 @@ public void testBinarySearchInQueueByTime() {
// append message to consume queue
flatFile.consumeQueue.initOffset(50 * ConsumeQueue.CQ_STORE_UNIT_SIZE);

for (int i = 0; i < 5; i++) {
AppendResult appendResult = flatFile.appendConsumeQueue(new DispatchRequest(
mq.getTopic(), mq.getQueueId(), MessageFormatUtilTest.MSG_LEN * i,
MessageFormatUtilTest.MSG_LEN, 0, timestamp1, 50 + i,
AppendResult appendResult = flatFile.appendConsumeQueue(new DispatchRequest(
mq.getTopic(), mq.getQueueId(), 0,
MessageFormatUtilTest.MSG_LEN, 0, timestamp1, 50,
"", "", 0, 0, null));
Assert.assertEquals(AppendResult.SUCCESS, appendResult);
}
Assert.assertEquals(AppendResult.SUCCESS, appendResult);

appendResult = flatFile.appendConsumeQueue(new DispatchRequest(
mq.getTopic(), mq.getQueueId(), MessageFormatUtilTest.MSG_LEN,
MessageFormatUtilTest.MSG_LEN, 0, timestamp2, 51,
"", "", 0, 0, null));
Assert.assertEquals(AppendResult.SUCCESS, appendResult);

appendResult = flatFile.appendConsumeQueue(new DispatchRequest(
mq.getTopic(), mq.getQueueId(), MessageFormatUtilTest.MSG_LEN * 2,
MessageFormatUtilTest.MSG_LEN, 0, timestamp2, 52,
"", "", 0, 0, null));
Assert.assertEquals(AppendResult.SUCCESS, appendResult);

appendResult = flatFile.appendConsumeQueue(new DispatchRequest(
mq.getTopic(), mq.getQueueId(), MessageFormatUtilTest.MSG_LEN * 3,
MessageFormatUtilTest.MSG_LEN, 0, timestamp2, 53,
"", "", 0, 0, null));
Assert.assertEquals(AppendResult.SUCCESS, appendResult);

appendResult = flatFile.appendConsumeQueue(new DispatchRequest(
mq.getTopic(), mq.getQueueId(), MessageFormatUtilTest.MSG_LEN * 4,
MessageFormatUtilTest.MSG_LEN, 0, timestamp3, 54,
"", "", 0, 0, null));
Assert.assertEquals(AppendResult.SUCCESS, appendResult);

// commit message will increase max consume queue offset
Assert.assertTrue(flatFile.commitAsync().join());
Expand Down
Loading