当前位置:   article > 正文

RocketMQ5.0.0消息存储<三>_消息转发与恢复机制_duplicationenable

duplicationenable

目录

一、消息转发

1. ReputMessageService线程初始化

2. 消息转发更新ConsumeQueue

3. 消息转发更新IndexFile

二、恢复机制

1. Broker加载存储文件

2. Broker正常退出的文件恢复

3. Broker异常退出的文件恢复

三、参考资料


一、消息转发

        消息消费队列文件、索引文件都是基于CommitLog文件构建的,当消息Commit操作(将MappedFile.writeBuffer数据提交到该FileChannel文件通道内,即:提交到文件内存映射)时,ConsumeQueue、IndexFile需要及时更新,否则消息无法及时被消费,根据消息属性查找消息也会出现较大延迟。

        RocketMQ通过开启一个线程ReputMessageServcie来实时转发CommitLog文件更新事件,通过各自的任务处理器根据转发的消息及时更新ConsumeQueue、IndexFile文件。如下所示是的UML图。org.apache.rocketmq.store.CommitLogDispatcher是转发的顶层接口。需要注意的是消息转发只是将消息写入ConsumeQueue、IndexFile文件内存映射中,不是写入磁盘

1. ReputMessageService线程初始化

        Broker启动时会启动ReputMessageService线程,并初始化关键参数reputFromOffset(开始转发消息的物理偏移量)。如果允许重复转发,reputFromOffset设置为CommitLog的提交指针;如果不允许重复转发,reputFromOffset设置为Commitlog的内存中最大偏移量。如下代码所示,org.apache.rocketmq.store.DefaultMessageStore#start。

  1. /**
  2. * broker启动时,消息存储线程
  3. * BrokerController#startBasicService()
  4. * @throws Exception
  5. */
  6. @Override
  7. public void start() throws Exception {
  8. ......
  9. /*
  10. 启动ReputMessageService服务线程
  11. 重要参数reputFromOffset:从哪个物理偏移量转发消息给ConsumeQueue、IndexFile
  12. 允许重复转发:提交指针
  13. 不允许重复转发:最大偏移量
  14. 启动后,调用{@link ReputMessageService} 的run()
  15. */
  16. // 允许重复转发:提交指针
  17. if (this.getMessageStoreConfig().isDuplicationEnable()) {
  18. this.reputMessageService.setReputFromOffset(this.commitLog.getConfirmOffset());
  19. }
  20. // 不允许重复转发:最大偏移量
  21. else {
  22. // It is [recover]'s responsibility to fully dispatch the commit log data before the max offset of commit log.
  23. this.reputMessageService.setReputFromOffset(this.commitLog.getMaxOffset());
  24. }
  25. // DefaultMessageStore.ReputMessageService.run()
  26. this.reputMessageService.start();
  27. ......
  28. }

        下图所示是org.apache.rocketmq.store.DefaultMessageStore.ReputMessageService#run方法的调用链。其核心方法是org.apache.rocketmq.store.DefaultMessageStore.ReputMessageService#doReput,代码如下。注意转发消息DefaultMessageStore#doDispatch(DispatchRequest)方法并没有返回结果,而是通过重复写入次数控制,详细见下小节

  1. /**
  2. * 消息提交到Commitlog时消息转发,构建ConsumeQueue、index文件服务 的核心方法
  3. * step1:消息转发偏移量 > 最小偏移量时,则最小偏移量赋值给消息转发偏移量
  4. * step2:消息转发偏移量 <= 最小偏移量时,获取转发偏移量开始的全部有效数据
  5. * step3:循环转发每条消息,获取每条消息的转发请求对象{@link DispatchRequest}
  6. * step4:消息解析成功后,转发消息{@link DefaultMessageStore#doDispatch(DispatchRequest)}
  7. * 注意:转发消息时遍历LinkedList<CommitLogDispatcher> dispatcherList,集合中有消息队列、索引文件执行转发的实现类
  8. * step5:更新消息转发偏移量
  9. */
  10. private void doReput() {
  11. // 消息转发偏移量 > 最小偏移量时,则最小偏移量赋值给消息转发偏移量
  12. if (this.reputFromOffset < DefaultMessageStore.this.commitLog.getMinOffset()) {
  13. LOGGER.warn("The reputFromOffset={} is smaller than minPyOffset={}, this usually indicate that the dispatch behind too much and the commitlog has expired.",
  14. this.reputFromOffset, DefaultMessageStore.this.commitLog.getMinOffset());
  15. this.reputFromOffset = DefaultMessageStore.this.commitLog.getMinOffset();
  16. }
  17. // 消息转发偏移量 <= 最小偏移量时,则循环转发每条消息
  18. for (boolean doNext = true; this.isCommitLogAvailable() && doNext; ) {
  19. // 获取从消息转发偏移量开始的全部有效数据
  20. SelectMappedBufferResult result = DefaultMessageStore.this.commitLog.getData(reputFromOffset);
  21. if (result != null) {
  22. try {
  23. this.reputFromOffset = result.getStartOffset();
  24. // 循环转发每条消息
  25. for (int readSize = 0; readSize < result.getSize() && reputFromOffset < DefaultMessageStore.this.getConfirmOffset() && doNext; ) {
  26. // 获取每条消息
  27. DispatchRequest dispatchRequest =
  28. DefaultMessageStore.this.commitLog.checkMessageAndReturnSize(result.getByteBuffer(), false, false, false);
  29. // 消息大小
  30. int size = dispatchRequest.getBufferSize() == -1 ? dispatchRequest.getMsgSize() : dispatchRequest.getBufferSize();
  31. // 读取的消息是否超出提交偏移量
  32. if (reputFromOffset + size > DefaultMessageStore.this.getConfirmOffset()) {
  33. doNext = false;
  34. break;
  35. }
  36. // 解析消息成功
  37. if (dispatchRequest.isSuccess()) {
  38. if (size > 0) {
  39. // 消息转发
  40. DefaultMessageStore.this.doDispatch(dispatchRequest);
  41. // 开启长轮询时,唤醒PullRequestHoldService线程,执行被挂起的拉取消息请求
  42. if (DefaultMessageStore.this.brokerConfig.isLongPollingEnable()
  43. && DefaultMessageStore.this.messageArrivingListener != null) {
  44. // 唤醒被挂起的拉取消息请求,再次拉取消息
  45. DefaultMessageStore.this.messageArrivingListener.arriving(dispatchRequest.getTopic(),
  46. dispatchRequest.getQueueId(), dispatchRequest.getConsumeQueueOffset() + 1,
  47. dispatchRequest.getTagsCode(), dispatchRequest.getStoreTimestamp(),
  48. dispatchRequest.getBitMap(), dispatchRequest.getPropertiesMap());
  49. // 多个消息队列,唤醒被挂起的拉取消息请求,再次拉取消息
  50. notifyMessageArrive4MultiQueue(dispatchRequest);
  51. }
  52. this.reputFromOffset += size;
  53. readSize += size;
  54. if (!DefaultMessageStore.this.getMessageStoreConfig().isDuplicationEnable() &&
  55. DefaultMessageStore.this.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE) {
  56. DefaultMessageStore.this.storeStatsService
  57. .getSinglePutMessageTopicTimesTotal(dispatchRequest.getTopic()).add(1);
  58. DefaultMessageStore.this.storeStatsService
  59. .getSinglePutMessageTopicSizeTotal(dispatchRequest.getTopic())
  60. .add(dispatchRequest.getMsgSize());
  61. }
  62. } else if (size == 0) {
  63. this.reputFromOffset = DefaultMessageStore.this.commitLog.rollNextFile(this.reputFromOffset);
  64. readSize = result.getSize();
  65. }
  66. }
  67. // 解析消息失败
  68. else if (!dispatchRequest.isSuccess()) {
  69. if (size > 0) {
  70. LOGGER.error("[BUG]read total count not equals msg total size. reputFromOffset={}", reputFromOffset);
  71. this.reputFromOffset += size;
  72. } else {
  73. doNext = false;
  74. // If user open the dledger pattern or the broker is master node,
  75. // it will not ignore the exception and fix the reputFromOffset variable
  76. if (DefaultMessageStore.this.getMessageStoreConfig().isEnableDLegerCommitLog() ||
  77. DefaultMessageStore.this.brokerConfig.getBrokerId() == MixAll.MASTER_ID) {
  78. LOGGER.error("[BUG]dispatch message to consume queue error, COMMITLOG OFFSET: {}",
  79. this.reputFromOffset);
  80. this.reputFromOffset += result.getSize() - readSize;
  81. }
  82. }
  83. }
  84. }
  85. } finally {
  86. result.release();
  87. }
  88. } else {
  89. doNext = false;
  90. }
  91. }
  92. }

2. 消息转发更新ConsumeQueue

        org.apache.rocketmq.store.DefaultMessageStore.CommitLogDispatcherBuildConsumeQueue是转发任务执行更新到消费队列实现类,CommitLogDispatcherBuildConsumeQueue#dispatch的执行转发消息方法,如下所示是调用链。

        org.apache.rocketmq.store.ConsumeQueue#putMessagePositionInfoWrapper是执行转发核心方法。其中:写入消费队列最大重试次数maxRetries,默认30次,超出该值则打印异常日志;写入消费队列,只是写入到消费队列文件内存映射,并没有执行刷盘

  1. /**
  2. * 转发消息到ConsumeQueue的核心方法
  3. * step1:写入消费队列最大重试次数maxRetries,默认30次
  4. * step2:判定消费队列是否允许写入
  5. * step3:写入消费队列,只是写入到消费队列文件内存映射,不是写入磁盘
  6. * step4:超出maxRetries,打印异常日志
  7. */
  8. @Override
  9. public void putMessagePositionInfoWrapper(DispatchRequest request) {
  10. // 写入消费队列最大重试次数
  11. final int maxRetries = 30;
  12. // 是否能写
  13. boolean canWrite = this.messageStore.getRunningFlags().isCQWriteable();
  14. for (int i = 0; i < maxRetries && canWrite; i++) {
  15. long tagsCode = request.getTagsCode();
  16. if (isExtWriteEnable()) {
  17. ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit();
  18. cqExtUnit.setFilterBitMap(request.getBitMap());
  19. cqExtUnit.setMsgStoreTime(request.getStoreTimestamp());
  20. cqExtUnit.setTagsCode(request.getTagsCode());
  21. long extAddr = this.consumeQueueExt.put(cqExtUnit);
  22. if (isExtAddr(extAddr)) {
  23. tagsCode = extAddr;
  24. } else {
  25. log.warn("Save consume queue extend fail, So just save tagsCode! {}, topic:{}, queueId:{}, offset:{}", cqExtUnit,
  26. topic, queueId, request.getCommitLogOffset());
  27. }
  28. }
  29. // 写入消费队列
  30. boolean result = this.putMessagePositionInfo(request.getCommitLogOffset(),
  31. request.getMsgSize(), tagsCode, request.getConsumeQueueOffset());
  32. if (result) {
  33. if (this.messageStore.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE ||
  34. this.messageStore.getMessageStoreConfig().isEnableDLegerCommitLog()) {
  35. this.messageStore.getStoreCheckpoint().setPhysicMsgTimestamp(request.getStoreTimestamp());
  36. }
  37. this.messageStore.getStoreCheckpoint().setLogicsMsgTimestamp(request.getStoreTimestamp());
  38. if (checkMultiDispatchQueue(request)) {
  39. multiDispatchLmqQueue(request, maxRetries);
  40. }
  41. return;
  42. } else {
  43. // XXX: warn and notify me
  44. log.warn("[BUG]put commit log position info to " + topic + ":" + queueId + " " + request.getCommitLogOffset()
  45. + " failed, retry " + i + " times");
  46. try {
  47. Thread.sleep(1000);
  48. } catch (InterruptedException e) {
  49. log.warn("", e);
  50. }
  51. }
  52. }
  53. // XXX: warn and notify me
  54. log.error("[BUG]consume queue can not write, {} {}", this.topic, this.queueId);
  55. this.messageStore.getRunningFlags().makeLogicsQueueError();
  56. }

        org.apache.rocketmq.store.ConsumeQueue#putMessagePositionInfo是消息写入消费队列的文件内存映射的核心方法。代码如下所示。

  1. /**
  2. * 写入消费队列
  3. * 注意:此次写入消息,只是追加到映射内存,固定刷盘方式:异步
  4. * @param offset 消息偏移量
  5. * @param size 消息大小
  6. * @param tagsCode 消息Tag的哈希码
  7. * @param cqOffset 消费队列的偏移量
  8. * @return
  9. */
  10. private boolean putMessagePositionInfo(final long offset, final int size, final long tagsCode,
  11. final long cqOffset) {
  12. if (offset + size <= this.maxPhysicOffset) {
  13. log.warn("Maybe try to build consume queue repeatedly maxPhysicOffset={} phyOffset={}", maxPhysicOffset, offset);
  14. return true;
  15. }
  16. // 组装消费队列条目:20字节 = 8字节的Commitlog offset + 4字节的消息大小 + 8字节的Tag的哈希码
  17. this.byteBufferIndex.flip();
  18. this.byteBufferIndex.limit(CQ_STORE_UNIT_SIZE);
  19. this.byteBufferIndex.putLong(offset);
  20. this.byteBufferIndex.putInt(size);
  21. this.byteBufferIndex.putLong(tagsCode);
  22. final long expectLogicOffset = cqOffset * CQ_STORE_UNIT_SIZE;
  23. /*
  24. * 更新消费队列参数
  25. */
  26. MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile(expectLogicOffset);
  27. if (mappedFile != null) {
  28. if (mappedFile.isFirstCreateInQueue() && cqOffset != 0 && mappedFile.getWrotePosition() == 0) {
  29. this.minLogicOffset = expectLogicOffset;
  30. this.mappedFileQueue.setFlushedWhere(expectLogicOffset);
  31. this.mappedFileQueue.setCommittedWhere(expectLogicOffset);
  32. this.fillPreBlank(mappedFile, expectLogicOffset);
  33. log.info("fill pre blank space " + mappedFile.getFileName() + " " + expectLogicOffset + " "
  34. + mappedFile.getWrotePosition());
  35. }
  36. if (cqOffset != 0) {
  37. long currentLogicOffset = mappedFile.getWrotePosition() + mappedFile.getFileFromOffset();
  38. if (expectLogicOffset < currentLogicOffset) {
  39. log.warn("Build consume queue repeatedly, expectLogicOffset: {} currentLogicOffset: {} Topic: {} QID: {} Diff: {}",
  40. expectLogicOffset, currentLogicOffset, this.topic, this.queueId, expectLogicOffset - currentLogicOffset);
  41. return true;
  42. }
  43. if (expectLogicOffset != currentLogicOffset) {
  44. LOG_ERROR.warn(
  45. "[BUG]logic queue order maybe wrong, expectLogicOffset: {} currentLogicOffset: {} Topic: {} QID: {} Diff: {}",
  46. expectLogicOffset,
  47. currentLogicOffset,
  48. this.topic,
  49. this.queueId,
  50. expectLogicOffset - currentLogicOffset
  51. );
  52. }
  53. }
  54. this.maxPhysicOffset = offset + size;
  55. // 追加消息到映射,并没有刷盘(固定刷盘方式:异步)
  56. return mappedFile.appendMessage(this.byteBufferIndex.array());
  57. }
  58. return false;
  59. }

3. 消息转发更新IndexFile

        org.apache.rocketmq.store.DefaultMessageStore.CommitLogDispatcherBuildIndex是转发任务执行更新到索引文件实现类,CommitLogDispatcherBuildIndex#dispatch的执行转发消息方法,如下所示是调用链。

        org.apache.rocketmq.store.index.IndexService#buildIndex是构建索引的核心方法,如下代码所示。RocketMQ支持同一个消息建立多个索引,多个索引键空格分开,如:"key1 key2"

  1. /**
  2. * 构建消息索引文件
  3. * 注意:索引追加到映射内存
  4. * step1:获取或创建索引文件
  5. * step2:获取索引文件最大偏移量
  6. * step3:判断消息的偏移量 < 索引文件最大偏移量,说明索引文件已构建(数据重复),则返回
  7. * step4:消息唯一键不为空,则首先添加索引(唯一键索引)
  8. * step5:消息key不为空,MQ支持同一消息多个索引,用空格隔开
  9. * step6:
  10. * @param req 消息信息
  11. */
  12. public void buildIndex(DispatchRequest req) {
  13. // 获取或创建索引文件
  14. IndexFile indexFile = retryGetAndCreateIndexFile();
  15. if (indexFile != null) {
  16. // 索引文件最大偏移量
  17. long endPhyOffset = indexFile.getEndPhyOffset();
  18. DispatchRequest msg = req;
  19. String topic = msg.getTopic();
  20. String keys = msg.getKeys();
  21. // 消息的偏移量 < 索引文件最大偏移量,说明索引文件已构建(数据重复),则返回
  22. if (msg.getCommitLogOffset() < endPhyOffset) {
  23. return;
  24. }
  25. final int tranType = MessageSysFlag.getTransactionValue(msg.getSysFlag());
  26. switch (tranType) {
  27. case MessageSysFlag.TRANSACTION_NOT_TYPE:
  28. case MessageSysFlag.TRANSACTION_PREPARED_TYPE:
  29. case MessageSysFlag.TRANSACTION_COMMIT_TYPE:
  30. break;
  31. case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE:
  32. return;
  33. }
  34. // 消息唯一键不为空,则首先添加索引(唯一键索引)
  35. if (req.getUniqKey() != null) {
  36. // 写入索引文件缓冲
  37. indexFile = putKey(indexFile, msg, buildKey(topic, req.getUniqKey())); // buildKey():格式:主题#key
  38. if (indexFile == null) {
  39. LOGGER.error("putKey error commitlog {} uniqkey {}", req.getCommitLogOffset(), req.getUniqKey());
  40. return;
  41. }
  42. }
  43. // 消息key不为空,MQ支持同一消息多个索引,用空格隔开
  44. if (keys != null && keys.length() > 0) {
  45. String[] keyset = keys.split(MessageConst.KEY_SEPARATOR);
  46. for (int i = 0; i < keyset.length; i++) {
  47. String key = keyset[i];
  48. if (key.length() > 0) {
  49. indexFile = putKey(indexFile, msg, buildKey(topic, key));
  50. if (indexFile == null) {
  51. LOGGER.error("putKey error commitlog {} uniqkey {}", req.getCommitLogOffset(), req.getUniqKey());
  52. return;
  53. }
  54. }
  55. }
  56. }
  57. } else {
  58. LOGGER.error("build index error, stop building index");
  59. }
  60. }

        org.apache.rocketmq.store.index.IndexFile#putKey是索引写入IndexFile文件内存映射中的核心方法,如下代码所示。

  1. /**
  2. * 写入IndexFile文件
  3. * 追加到映射内存
  4. * @param key 消息的key
  5. * @param phyOffset 消息的物理偏移量
  6. * @param storeTimestamp 消息的存储时间戳
  7. * @return true写入成功;false写入失败或已满
  8. */
  9. public boolean putKey(final String key, final long phyOffset, final long storeTimestamp) {
  10. // 当前条目数量是否大于允许最大条目数量
  11. if (this.indexHeader.getIndexCount() < this.indexNum) {
  12. // key的hash值
  13. int keyHash = indexKeyHashMethod(key);
  14. // key的hash槽的下标
  15. int slotPos = keyHash % this.hashSlotNum;
  16. // hash槽的物理地址
  17. int absSlotPos = IndexHeader.INDEX_HEADER_SIZE + slotPos * hashSlotSize;
  18. try {
  19. /*
  20. * 相同hash,存储多个数据(槽中是否有数据)
  21. */
  22. // 获取hash槽存储的数据(最新条目的地址)
  23. int slotValue = this.mappedByteBuffer.getInt(absSlotPos);
  24. if (slotValue <= invalidIndex || slotValue > this.indexHeader.getIndexCount()) {
  25. slotValue = invalidIndex;
  26. }
  27. // 计算待存储条目 与 第一条消息时间戳 的差值,并转换成秒
  28. long timeDiff = storeTimestamp - this.indexHeader.getBeginTimestamp();
  29. timeDiff = timeDiff / 1000;
  30. if (this.indexHeader.getBeginTimestamp() <= 0) {
  31. timeDiff = 0;
  32. } else if (timeDiff > Integer.MAX_VALUE) {
  33. timeDiff = Integer.MAX_VALUE;
  34. } else if (timeDiff < 0) {
  35. timeDiff = 0;
  36. }
  37. // 存储位置absIndexPos = 头部40字节 + hash槽数量 * 单个槽的字节大小(4字节) + 当前存储条目数量 * 单个条目字节大小(20字节)
  38. int absIndexPos =
  39. IndexHeader.INDEX_HEADER_SIZE + this.hashSlotNum * hashSlotSize
  40. + this.indexHeader.getIndexCount() * indexSize;
  41. // 组装条目 = hash码 + Commitlog偏移量 + 存储时间差值 + 当前hash槽的值(待存储条目的前一个条目的偏移量)
  42. this.mappedByteBuffer.putInt(absIndexPos, keyHash);
  43. this.mappedByteBuffer.putLong(absIndexPos + 4, phyOffset);
  44. this.mappedByteBuffer.putInt(absIndexPos + 4 + 8, (int) timeDiff);
  45. this.mappedByteBuffer.putInt(absIndexPos + 4 + 8 + 4, slotValue);
  46. // 替换hash槽的值(待存储条目的偏移量absIndexPos)
  47. this.mappedByteBuffer.putInt(absSlotPos, this.indexHeader.getIndexCount());
  48. // 更新文件头信息
  49. if (this.indexHeader.getIndexCount() <= 1) {
  50. this.indexHeader.setBeginPhyOffset(phyOffset);
  51. this.indexHeader.setBeginTimestamp(storeTimestamp);
  52. }
  53. if (invalidIndex == slotValue) {
  54. this.indexHeader.incHashSlotCount();
  55. }
  56. this.indexHeader.incIndexCount();
  57. this.indexHeader.setEndPhyOffset(phyOffset);
  58. this.indexHeader.setEndTimestamp(storeTimestamp);
  59. return true;
  60. } catch (Exception e) {
  61. log.error("putKey exception, Key: " + key + " KeyHashCode: " + key.hashCode(), e);
  62. }
  63. } else {
  64. log.warn("Over index file capacity: index count = " + this.indexHeader.getIndexCount()
  65. + "; index max num = " + this.indexNum);
  66. }
  67. // 当前条目数量 >= 允许最大数量,则为false
  68. return false;
  69. }

二、恢复机制

        RocketMQ存储异步(默认)或同步提交到Commitlog文件内存映射,再异步(默认)或同步刷盘到磁盘。当消息写入Commitlog文件内存映射时,然后异步生成转发任务更新ConsumeQueue、Index 文件。如果消息成功存储到Commitlog文件中,转发任务未成功执行,此时Broker由于某个原因宕机,导致Commitlog、ConsumeQueue、IndexFile文件数据不一致。如果没有人工修复,则导致会有一部分消息即便在Commitlog文件中,但由于并没有转发到ConsumeQueue,这部分消息将永远不会被消费者消费。

        RocketMQ是如何使Commitlog、ConsumeQueue达到最终一致性的呢?RocketMQ的存储文件的加载流程中,判断上一次退出是否正常,实现机制:Broker启动时判断${ROCKETMQ_HOME}/store/abort文件是否存在,正常退出时JVM钩子函数删除abort文件,若存在abort文件,说明Broker是异常退出。根据Broker是否正常退出,来采取不同的恢复机制。

        文件恢复主要完成flushedPosition、committedWhere指针设置,消息消费队列最大偏移量加载到内存,并删除flushedPosition之后所有的文件。 如果Broker异常退出,在文件恢复过程中,RocketMQ会将最后一个有效文件中的所有消息重新转发到消息消费队列与索引文件,确保不丢失消息,但同时会带来消息重复的问题。纵观RocktMQ的整体设计思想,RocketMQ保证消息不丢失但不保证消息不会重复消费,故消息消费业务方需要实现消息消费的幕等设计

1. Broker加载存储文件

        org.apache.rocketmq.store.DefaultMessageStore#load是Broker启动时加载存储文件Commitlog、ConsumeQueue、IndexFile、Checkpoint文件的核心方法,其调用链、代码如下所示。

  1. /**
  2. * broker控制器启动初始化时,加载消息存储:如Commitlog文件、ConsumerQueue文件、IndexFile文件等
  3. * step1:判断broker上次是否正常退出,是否存在${ROCKETMQ_HOME}/store/abort文件,存在:Broker退出异常
  4. * step2:加载Commitlog文件、ConsumerQueue文件、Checkpoint文件、IndexFile索引文件
  5. * step3:恢复文件,根据Broker是否退出异常,选择不同的恢复策略
  6. * {@link DefaultMessageStore#recover(boolean)}
  7. */
  8. @Override
  9. public boolean load() {
  10. boolean result = true;
  11. try {
  12. // 判断broker上次是否正常退出
  13. boolean lastExitOK = !this.isTempFileExist();
  14. LOGGER.info("last shutdown {}, root dir: {}", lastExitOK ? "normally" : "abnormally", messageStoreConfig.getStorePathRootDir());
  15. // load Commit Log 加载消息存储文件
  16. result = result && this.commitLog.load();
  17. // load Consume Queue 加载消息消费队列文件
  18. result = result && this.consumeQueueStore.load();
  19. if (result) {
  20. // 加载检查点Checkpoint文件
  21. this.storeCheckpoint =
  22. new StoreCheckpoint(StorePathConfigHelper.getStoreCheckpoint(this.messageStoreConfig.getStorePathRootDir()));
  23. // 获取检查点中的刷盘时间点
  24. this.masterFlushedOffset = this.storeCheckpoint.getMasterFlushedOffset();
  25. // 加载IndexFile索引文件
  26. this.indexService.load(lastExitOK);
  27. // 恢复文件
  28. this.recover(lastExitOK);
  29. LOGGER.info("load over, and the max phy offset = {}", this.getMaxPhyOffset());
  30. }
  31. long maxOffset = this.getMaxPhyOffset();
  32. this.setBrokerInitMaxOffset(maxOffset);
  33. LOGGER.info("load over, and the max phy offset = {}", maxOffset);
  34. } catch (Exception e) {
  35. LOGGER.error("load exception", e);
  36. result = false;
  37. }
  38. if (!result) {
  39. this.allocateMappedFileService.shutdown();
  40. }
  41. return result;
  42. }

        org.apache.rocketmq.store.DefaultMessageStore#recover是ConsumeQueue、IndexFile文件根据Broker是否正常提出采用不同恢复策略的方法入口,如下代码所示。恢复ConsumeQueue文件后,保存每个消费队列的当前存储逻辑偏移量

  1. /**
  2. * 恢复文件
  3. * @param lastExitOK broker上次是否正常退出,true正常退出;false异常退出
  4. */
  5. private void recover(final boolean lastExitOK) {
  6. long recoverCqStart = System.currentTimeMillis();
  7. // 获取消息队列中的最大的消息物理偏移量
  8. long maxPhyOffsetOfConsumeQueue = this.recoverConsumeQueue();
  9. long recoverCqEnd = System.currentTimeMillis();
  10. // 正常退出
  11. if (lastExitOK) {
  12. this.commitLog.recoverNormally(maxPhyOffsetOfConsumeQueue);
  13. }
  14. // 异常退出
  15. else {
  16. this.commitLog.recoverAbnormally(maxPhyOffsetOfConsumeQueue);
  17. }
  18. long recoverClogEnd = System.currentTimeMillis();
  19. // 恢复ConsumeQueue文件后,保存每个消费队列的当前存储逻辑偏移量
  20. this.recoverTopicQueueTable();
  21. long recoverOffsetEnd = System.currentTimeMillis();
  22. LOGGER.info("Recover end total:{} recoverCq:{} recoverClog:{} recoverOffset:{}",
  23. recoverOffsetEnd - recoverCqStart, recoverCqEnd - recoverCqStart, recoverClogEnd - recoverCqEnd, recoverOffsetEnd - recoverClogEnd);
  24. }

2. Broker正常退出的文件恢复

        org.apache.rocketmq.store.CommitLog#recoverNormally是Broker正常退出(abort文件不存在)时的恢复核心方法,其调用链、代码如下所示。

        恢复是从ConsumeQueue文件集合的倒数第三个文件开始恢复,若没有3个则从第一个开始恢复。需要注意的是没有恢复IndexFile文件,原因是有异常的IndexFile在加载过程中被销毁,当恢复时再次转发消息来构建IndexFile,详细见org.apache.rocketmq.store.index.IndexService#load是IndexFile文件加载的核心处理方法,这里不作介绍。 

  1. /**
  2. * Broker正常停止,文件恢复
  3. * step1:倒数第三个文件开始恢复,若没有则从第一个开始恢复
  4. * step2:遍历Commitlog文件,逐个取出每一条消息进行恢复(再次转发)
  5. * step3:更新MappedFileQueue的刷盘、提交指针 并移除processOffset之后的Commitlog文件
  6. * When the normal exit, data recovery, all memory data have been flush
  7. */
  8. public void recoverNormally(long maxPhyOffsetOfConsumeQueue) {
  9. // 文件恢复时,查找消息是否验证CRC,可配置
  10. boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover();
  11. boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable();
  12. final List<MappedFile> mappedFiles = this.mappedFileQueue.getMappedFiles();
  13. if (!mappedFiles.isEmpty()) {
  14. // 倒数第三个文件开始恢复,若没有则从第一个开始恢复
  15. // Began to recover from the last third file
  16. int index = mappedFiles.size() - 3;
  17. if (index < 0) {
  18. index = 0;
  19. }
  20. MappedFile mappedFile = mappedFiles.get(index);
  21. ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
  22. // 已恢复的消息物理偏移量 = mappedFile.getFileFromOffset() + mappedFileOffset
  23. long processOffset = mappedFile.getFileFromOffset();
  24. // 当前已校验通过的offset
  25. long mappedFileOffset = 0; // 从0开始,说明该Commitlog文件的第一个消息开始
  26. long lastValidMsgPhyOffset = this.getConfirmOffset();
  27. // normal recover doesn't require dispatching
  28. boolean doDispatch = false; // 是否正常恢复
  29. // 遍历Commitlog文件,逐个取出每一条消息进行恢复
  30. while (true) {
  31. // 检查消息并返回消息大小
  32. DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover, checkDupInfo);
  33. int size = dispatchRequest.getMsgSize();
  34. // Normal data
  35. if (dispatchRequest.isSuccess() && size > 0) {
  36. lastValidMsgPhyOffset = processOffset + mappedFileOffset;
  37. mappedFileOffset += size;
  38. // 消息转发到ConsumeQueue文件、IndexFile文件
  39. this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, false);
  40. }
  41. // 消息验证通过,且消息大小为0,说明已到Commitlog文件末尾,读取下一个Commitlog文件
  42. // Come the end of the file, switch to the next file Since the
  43. // return 0 representatives met last hole,
  44. // this can not be included in truncate offset
  45. else if (dispatchRequest.isSuccess() && size == 0) {
  46. this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, true);
  47. // 读取下一文件
  48. index++;
  49. // 下一文件没有
  50. if (index >= mappedFiles.size()) {
  51. // Current branch can not happen
  52. log.info("recover last 3 physics file over, last mapped file " + mappedFile.getFileName());
  53. break;
  54. }
  55. // 若有下个Commitlog,重新设置变量
  56. else {
  57. mappedFile = mappedFiles.get(index);
  58. byteBuffer = mappedFile.sliceByteBuffer();
  59. processOffset = mappedFile.getFileFromOffset();
  60. mappedFileOffset = 0;
  61. log.info("recover next physics file, " + mappedFile.getFileName());
  62. }
  63. }
  64. // Intermediate file read error
  65. else if (!dispatchRequest.isSuccess()) {
  66. if (size > 0) {
  67. log.warn("found a half message at {}, it will be truncated.", processOffset + mappedFileOffset);
  68. }
  69. log.info("recover physics file end, " + mappedFile.getFileName());
  70. break;
  71. }
  72. }
  73. /*
  74. 更新MappedFileQueue的刷盘、提交指针
  75. */
  76. processOffset += mappedFileOffset;
  77. // Set a candidate confirm offset.
  78. // In most cases, this value will be overwritten by confirmLog.init.
  79. // It works if some confirmed messages are lost.
  80. this.setConfirmOffset(lastValidMsgPhyOffset);
  81. this.mappedFileQueue.setFlushedWhere(processOffset);
  82. this.mappedFileQueue.setCommittedWhere(processOffset);
  83. // 移除processOffset之后的所有文件
  84. this.mappedFileQueue.truncateDirtyFiles(processOffset);
  85. // Clear ConsumeQueue redundant data
  86. if (maxPhyOffsetOfConsumeQueue >= processOffset) {
  87. log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, processOffset);
  88. this.defaultMessageStore.truncateDirtyLogicFiles(processOffset);
  89. }
  90. } else {
  91. // Commitlog case files are deleted
  92. log.warn("The commitlog files are deleted, and delete the consume queue files");
  93. this.mappedFileQueue.setFlushedWhere(0);
  94. this.mappedFileQueue.setCommittedWhere(0);
  95. this.defaultMessageStore.destroyLogics();
  96. }
  97. }

3. Broker异常退出的文件恢复

        org.apache.rocketmq.store.CommitLog#recoverAbnormally是Broker异常退出(abort文件存在)时的恢复核心方法,代码如下所示。

        与Broker正常退出逻辑差不多,区别:恢复是从ConsumeQueue文件集合的最后一个文件往前遍历,找到文件第一条消息存储正常的文件。而判断Commitlog是否正常存储,判定条件:魔数是否正常(第一条消息存储时间戳 = 0时,说明该文件没有任何消息)。

  1. /**
  2. * Broker异常停止,文件恢复(大致与正常恢复一样)
  3. */
  4. @Deprecated
  5. public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) {
  6. // recover by the minimum time stamp
  7. boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover();
  8. boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable();
  9. final List<MappedFile> mappedFiles = this.mappedFileQueue.getMappedFiles();
  10. if (!mappedFiles.isEmpty()) {
  11. // 最后一个文件往前走,找到第一个消息存储正常的文件
  12. // Looking beginning to recover from which file
  13. int index = mappedFiles.size() - 1;
  14. MappedFile mappedFile = null;
  15. for (; index >= 0; index--) {
  16. mappedFile = mappedFiles.get(index);
  17. // 判断Commitlog是否正常存储(判定条件:魔数是否正常);注意第一条消息存储时间戳 = 0时,说明该文件没有任何消息
  18. if (this.isMappedFileMatchedRecover(mappedFile)) {
  19. log.info("recover from this mapped file " + mappedFile.getFileName());
  20. break;
  21. }
  22. }
  23. if (index < 0) { // 没有,则获取第一个
  24. index = 0;
  25. mappedFile = mappedFiles.get(index);
  26. }
  27. ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
  28. long processOffset = mappedFile.getFileFromOffset();
  29. long mappedFileOffset = 0;
  30. long lastValidMsgPhyOffset = this.getConfirmOffset();
  31. // abnormal recover require dispatching
  32. boolean doDispatch = true;
  33. while (true) {
  34. DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover, checkDupInfo);
  35. int size = dispatchRequest.getMsgSize();
  36. if (dispatchRequest.isSuccess()) {
  37. // Normal data
  38. if (size > 0) {
  39. lastValidMsgPhyOffset = processOffset + mappedFileOffset;
  40. mappedFileOffset += size;
  41. if (this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()) {
  42. if (dispatchRequest.getCommitLogOffset() < this.defaultMessageStore.getConfirmOffset()) {
  43. this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, false);
  44. }
  45. } else {
  46. // 消息转发到ConsumeQueue文件、IndexFile文件
  47. this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, false);
  48. }
  49. }
  50. // Come the end of the file, switch to the next file
  51. // Since the return 0 representatives met last hole, this can
  52. // not be included in truncate offset
  53. else if (size == 0) {
  54. this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, true);
  55. index++;
  56. if (index >= mappedFiles.size()) {
  57. // The current branch under normal circumstances should
  58. // not happen
  59. log.info("recover physics file over, last mapped file " + mappedFile.getFileName());
  60. break;
  61. } else {
  62. mappedFile = mappedFiles.get(index);
  63. byteBuffer = mappedFile.sliceByteBuffer();
  64. processOffset = mappedFile.getFileFromOffset();
  65. mappedFileOffset = 0;
  66. log.info("recover next physics file, " + mappedFile.getFileName());
  67. }
  68. }
  69. } else {
  70. if (size > 0) {
  71. log.warn("found a half message at {}, it will be truncated.", processOffset + mappedFileOffset);
  72. }
  73. log.info("recover physics file end, " + mappedFile.getFileName() + " pos=" + byteBuffer.position());
  74. break;
  75. }
  76. }
  77. processOffset += mappedFileOffset;
  78. // Set a candidate confirm offset.
  79. // In most cases, this value will be overwritten by confirmLog.init.
  80. // It works if some confirmed messages are lost.
  81. this.setConfirmOffset(lastValidMsgPhyOffset);
  82. this.mappedFileQueue.setFlushedWhere(processOffset);
  83. this.mappedFileQueue.setCommittedWhere(processOffset);
  84. this.mappedFileQueue.truncateDirtyFiles(processOffset);
  85. // Clear ConsumeQueue redundant data
  86. if (maxPhyOffsetOfConsumeQueue >= processOffset) {
  87. log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, processOffset);
  88. this.defaultMessageStore.truncateDirtyLogicFiles(processOffset);
  89. }
  90. }
  91. // Commitlog case files are deleted
  92. else {
  93. log.warn("The commitlog files are deleted, and delete the consume queue files");
  94. this.mappedFileQueue.setFlushedWhere(0);
  95. this.mappedFileQueue.setCommittedWhere(0);
  96. this.defaultMessageStore.destroyLogics();
  97. }
  98. }

三、参考资料

RocketMQ源码分析(十五)之文件恢复_jannals的博客-CSDN博客_rocketmq 文件恢复

RocketMQ5.0.0消息发送_爱我所爱0505的博客-CSDN博客

RocketMQ5.0.0消息存储<一>_存储文件及内存映射_爱我所爱0505的博客-CSDN博客

RocketMQ5.0.0消息存储<二>_消息存储流程_爱我所爱0505的博客-CSDN博客 

声明:本文内容由网友自发贡献,不代表【wpsshop博客】立场,版权归原作者所有,本站不承担相应法律责任。如您发现有侵权的内容,请联系我们。转载请注明出处:https://www.wpsshop.cn/w/知新_RL/article/detail/733430
推荐阅读
相关标签
  

闽ICP备14008679号