Skip to content

Commit edf9703

Browse files
Merge pull request alibaba#501 from suiyuzeng/issue_recover
[ISSUE alibaba#467] fix Message missed after recovering from abnormal shutdown
2 parents 3dabb6c + d3d4225 commit edf9703

File tree

4 files changed

+169
-36
lines changed

4 files changed

+169
-36
lines changed

README.md

+1-1
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ It offers a variety of features:
3131
* Docs: <https://rocketmq.apache.org/docs/quick-start/>
3232
* Issues: <https://github.com/apache/rocketmq/issues>
3333
* Ask: <https://stackoverflow.com/questions/tagged/rocketmq>
34-
* Slack: <https://rocketmq-community.slack.com/>
34+
* Slack: <https://rocketmq-invite-automation.herokuapp.com/>
3535

3636

3737
----------

store/src/main/java/org/apache/rocketmq/store/CommitLog.java

+40-31
Original file line numberDiff line numberDiff line change
@@ -158,7 +158,7 @@ public SelectMappedBufferResult getData(final long offset, final boolean returnF
158158
/**
159159
* When the normal exit, data recovery, all memory data have been flush
160160
*/
161-
public void recoverNormally() {
161+
public void recoverNormally(long maxPhyOffsetOfConsumeQueue) {
162162
boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover();
163163
final List<MappedFile> mappedFiles = this.mappedFileQueue.getMappedFiles();
164164
if (!mappedFiles.isEmpty()) {
@@ -206,6 +206,12 @@ else if (!dispatchRequest.isSuccess()) {
206206
this.mappedFileQueue.setFlushedWhere(processOffset);
207207
this.mappedFileQueue.setCommittedWhere(processOffset);
208208
this.mappedFileQueue.truncateDirtyFiles(processOffset);
209+
210+
// Clear ConsumeQueue redundant data
211+
if (maxPhyOffsetOfConsumeQueue >= processOffset) {
212+
log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, processOffset);
213+
this.defaultMessageStore.truncateDirtyLogicFiles(processOffset);
214+
}
209215
}
210216
}
211217

@@ -390,7 +396,7 @@ public void setConfirmOffset(long phyOffset) {
390396
this.confirmOffset = phyOffset;
391397
}
392398

393-
public void recoverAbnormally() {
399+
public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) {
394400
// recover by the minimum time stamp
395401
boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover();
396402
final List<MappedFile> mappedFiles = this.mappedFileQueue.getMappedFiles();
@@ -418,41 +424,41 @@ public void recoverAbnormally() {
418424
DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover);
419425
int size = dispatchRequest.getMsgSize();
420426

421-
// Normal data
422-
if (size > 0) {
423-
mappedFileOffset += size;
427+
if (dispatchRequest.isSuccess()) {
428+
// Normal data
429+
if (size > 0) {
430+
mappedFileOffset += size;
424431

425-
if (this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()) {
426-
if (dispatchRequest.getCommitLogOffset() < this.defaultMessageStore.getConfirmOffset()) {
432+
if (this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()) {
433+
if (dispatchRequest.getCommitLogOffset() < this.defaultMessageStore.getConfirmOffset()) {
434+
this.defaultMessageStore.doDispatch(dispatchRequest);
435+
}
436+
} else {
427437
this.defaultMessageStore.doDispatch(dispatchRequest);
428438
}
429-
} else {
430-
this.defaultMessageStore.doDispatch(dispatchRequest);
431439
}
432-
}
433-
// Intermediate file read error
434-
else if (size == -1) {
440+
// Come the end of the file, switch to the next file
441+
// Since the return 0 representatives met last hole, this can
442+
// not be included in truncate offset
443+
else if (size == 0) {
444+
index++;
445+
if (index >= mappedFiles.size()) {
446+
// The current branch under normal circumstances should
447+
// not happen
448+
log.info("recover physics file over, last mapped file " + mappedFile.getFileName());
449+
break;
450+
} else {
451+
mappedFile = mappedFiles.get(index);
452+
byteBuffer = mappedFile.sliceByteBuffer();
453+
processOffset = mappedFile.getFileFromOffset();
454+
mappedFileOffset = 0;
455+
log.info("recover next physics file, " + mappedFile.getFileName());
456+
}
457+
}
458+
} else {
435459
log.info("recover physics file end, " + mappedFile.getFileName());
436460
break;
437461
}
438-
// Come the end of the file, switch to the next file
439-
// Since the return 0 representatives met last hole, this can
440-
// not be included in truncate offset
441-
else if (size == 0) {
442-
index++;
443-
if (index >= mappedFiles.size()) {
444-
// The current branch under normal circumstances should
445-
// not happen
446-
log.info("recover physics file over, last mapped file " + mappedFile.getFileName());
447-
break;
448-
} else {
449-
mappedFile = mappedFiles.get(index);
450-
byteBuffer = mappedFile.sliceByteBuffer();
451-
processOffset = mappedFile.getFileFromOffset();
452-
mappedFileOffset = 0;
453-
log.info("recover next physics file, " + mappedFile.getFileName());
454-
}
455-
}
456462
}
457463

458464
processOffset += mappedFileOffset;
@@ -461,7 +467,10 @@ else if (size == 0) {
461467
this.mappedFileQueue.truncateDirtyFiles(processOffset);
462468

463469
// Clear ConsumeQueue redundant data
464-
this.defaultMessageStore.truncateDirtyLogicFiles(processOffset);
470+
if (maxPhyOffsetOfConsumeQueue >= processOffset) {
471+
log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, processOffset);
472+
this.defaultMessageStore.truncateDirtyLogicFiles(processOffset);
473+
}
465474
}
466475
// Commitlog case files are deleted
467476
else {

store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java

+10-4
Original file line numberDiff line numberDiff line change
@@ -1276,12 +1276,12 @@ private boolean loadConsumeQueue() {
12761276
}
12771277

12781278
private void recover(final boolean lastExitOK) {
1279-
this.recoverConsumeQueue();
1279+
long maxPhyOffsetOfConsumeQueue = this.recoverConsumeQueue();
12801280

12811281
if (lastExitOK) {
1282-
this.commitLog.recoverNormally();
1282+
this.commitLog.recoverNormally(maxPhyOffsetOfConsumeQueue);
12831283
} else {
1284-
this.commitLog.recoverAbnormally();
1284+
this.commitLog.recoverAbnormally(maxPhyOffsetOfConsumeQueue);
12851285
}
12861286

12871287
this.recoverTopicQueueTable();
@@ -1306,12 +1306,18 @@ private void putConsumeQueue(final String topic, final int queueId, final Consum
13061306
}
13071307
}
13081308

1309-
private void recoverConsumeQueue() {
1309+
private long recoverConsumeQueue() {
1310+
long maxPhysicOffset = -1;
13101311
for (ConcurrentMap<Integer, ConsumeQueue> maps : this.consumeQueueTable.values()) {
13111312
for (ConsumeQueue logic : maps.values()) {
13121313
logic.recover();
1314+
if (logic.getMaxPhysicOffset() > maxPhysicOffset) {
1315+
maxPhysicOffset = logic.getMaxPhysicOffset();
1316+
}
13131317
}
13141318
}
1319+
1320+
return maxPhysicOffset;
13151321
}
13161322

13171323
private void recoverTopicQueueTable() {

store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java

+118
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,12 @@
1818
package org.apache.rocketmq.store;
1919

2020
import java.io.File;
21+
import java.io.RandomAccessFile;
2122
import java.net.InetAddress;
2223
import java.net.InetSocketAddress;
2324
import java.net.SocketAddress;
25+
import java.nio.MappedByteBuffer;
26+
import java.nio.channels.FileChannel;
2427
import java.nio.channels.OverlappingFileLockException;
2528
import java.util.Map;
2629
import java.util.concurrent.atomic.AtomicInteger;
@@ -29,6 +32,7 @@
2932
import org.apache.rocketmq.common.UtilAll;
3033
import org.apache.rocketmq.store.config.FlushDiskType;
3134
import org.apache.rocketmq.store.config.MessageStoreConfig;
35+
import org.apache.rocketmq.store.config.StorePathConfigHelper;
3236
import org.junit.After;
3337
import org.apache.rocketmq.store.stats.BrokerStatsManager;
3438
import org.junit.Before;
@@ -171,6 +175,120 @@ public void testPullSize() throws Exception {
171175
assertThat(getMessageResult45.getMessageBufferList().size()).isEqualTo(10);
172176
}
173177

178+
@Test
179+
public void testRecover() throws Exception {
180+
String topic = "recoverTopic";
181+
MessageBody = StoreMessage.getBytes();
182+
for (int i = 0; i < 100; i++) {
183+
MessageExtBrokerInner messageExtBrokerInner = buildMessage();
184+
messageExtBrokerInner.setTopic(topic);
185+
messageExtBrokerInner.setQueueId(0);
186+
messageStore.putMessage(messageExtBrokerInner);
187+
}
188+
189+
Thread.sleep(100);//wait for build consumer queue
190+
long maxPhyOffset = messageStore.getMaxPhyOffset();
191+
long maxCqOffset = messageStore.getMaxOffsetInQueue(topic, 0);
192+
193+
//1.just reboot
194+
messageStore.shutdown();
195+
messageStore = buildMessageStore();
196+
boolean load = messageStore.load();
197+
assertTrue(load);
198+
messageStore.start();
199+
assertTrue(maxPhyOffset == messageStore.getMaxPhyOffset());
200+
assertTrue(maxCqOffset == messageStore.getMaxOffsetInQueue(topic, 0));
201+
202+
//2.damage commitlog and reboot normal
203+
for (int i = 0; i < 100; i++) {
204+
MessageExtBrokerInner messageExtBrokerInner = buildMessage();
205+
messageExtBrokerInner.setTopic(topic);
206+
messageExtBrokerInner.setQueueId(0);
207+
messageStore.putMessage(messageExtBrokerInner);
208+
}
209+
Thread.sleep(100);
210+
long secondLastPhyOffset = messageStore.getMaxPhyOffset();
211+
long secondLastCqOffset = messageStore.getMaxOffsetInQueue(topic, 0);
212+
213+
MessageExtBrokerInner messageExtBrokerInner = buildMessage();
214+
messageExtBrokerInner.setTopic(topic);
215+
messageExtBrokerInner.setQueueId(0);
216+
messageStore.putMessage(messageExtBrokerInner);
217+
218+
messageStore.shutdown();
219+
220+
//damage last message
221+
damageCommitlog(secondLastPhyOffset);
222+
223+
//reboot
224+
messageStore = buildMessageStore();
225+
load = messageStore.load();
226+
assertTrue(load);
227+
messageStore.start();
228+
assertTrue(secondLastPhyOffset == messageStore.getMaxPhyOffset());
229+
assertTrue(secondLastCqOffset == messageStore.getMaxOffsetInQueue(topic, 0));
230+
231+
//3.damage commitlog and reboot abnormal
232+
for (int i = 0; i < 100; i++) {
233+
messageExtBrokerInner = buildMessage();
234+
messageExtBrokerInner.setTopic(topic);
235+
messageExtBrokerInner.setQueueId(0);
236+
messageStore.putMessage(messageExtBrokerInner);
237+
}
238+
Thread.sleep(100);
239+
secondLastPhyOffset = messageStore.getMaxPhyOffset();
240+
secondLastCqOffset = messageStore.getMaxOffsetInQueue(topic, 0);
241+
242+
messageExtBrokerInner = buildMessage();
243+
messageExtBrokerInner.setTopic(topic);
244+
messageExtBrokerInner.setQueueId(0);
245+
messageStore.putMessage(messageExtBrokerInner);
246+
messageStore.shutdown();
247+
248+
//damage last message
249+
damageCommitlog(secondLastPhyOffset);
250+
//add abort file
251+
String fileName = StorePathConfigHelper.getAbortFile(((DefaultMessageStore) messageStore).getMessageStoreConfig().getStorePathRootDir());
252+
File file = new File(fileName);
253+
MappedFile.ensureDirOK(file.getParent());
254+
file.createNewFile();
255+
256+
messageStore = buildMessageStore();
257+
load = messageStore.load();
258+
assertTrue(load);
259+
messageStore.start();
260+
assertTrue(secondLastPhyOffset == messageStore.getMaxPhyOffset());
261+
assertTrue(secondLastCqOffset == messageStore.getMaxOffsetInQueue(topic, 0));
262+
263+
//message write again
264+
for (int i = 0; i < 100; i++) {
265+
messageExtBrokerInner = buildMessage();
266+
messageExtBrokerInner.setTopic(topic);
267+
messageExtBrokerInner.setQueueId(0);
268+
messageStore.putMessage(messageExtBrokerInner);
269+
}
270+
}
271+
272+
private void damageCommitlog(long offset) throws Exception {
273+
MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
274+
File file = new File(messageStoreConfig.getStorePathCommitLog() + File.separator + "00000000000000000000");
275+
276+
FileChannel fileChannel = new RandomAccessFile(file, "rw").getChannel();
277+
MappedByteBuffer mappedByteBuffer = fileChannel.map(FileChannel.MapMode.READ_WRITE, 0, 1024 * 1024 * 10);
278+
279+
int bodyLen = mappedByteBuffer.getInt((int) offset + 84);
280+
int topicLenIndex = (int) offset + 84 + bodyLen + 4;
281+
mappedByteBuffer.position(topicLenIndex);
282+
mappedByteBuffer.putInt(0);
283+
mappedByteBuffer.putInt(0);
284+
mappedByteBuffer.putInt(0);
285+
mappedByteBuffer.putInt(0);
286+
287+
mappedByteBuffer.force();
288+
fileChannel.force(true);
289+
fileChannel.close();
290+
}
291+
174292
private class MyMessageArrivingListener implements MessageArrivingListener {
175293
@Override
176294
public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime,

0 commit comments

Comments
 (0)