Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
Apache RocketMQ
Rocketmq
提交
da73a1d1
R
Rocketmq
项目概览
Apache RocketMQ
/
Rocketmq
上一次同步 大约 3 年
通知
268
Star
16139
Fork
68
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
DevOps
流水线
流水线任务
计划
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
R
Rocketmq
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
DevOps
DevOps
流水线
流水线任务
计划
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
流水线任务
提交
Issue看板
提交
da73a1d1
编写于
10月 19, 2018
作者:
S
suiyuzeng
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
log recover issue
上级
a220364b
变更
3
显示空白变更内容
内联
并排
Showing
3 changed file
with
288 addition
and
151 deletion
+288
-151
store/src/main/java/org/apache/rocketmq/store/CommitLog.java
store/src/main/java/org/apache/rocketmq/store/CommitLog.java
+102
-90
store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
...n/java/org/apache/rocketmq/store/DefaultMessageStore.java
+68
-61
store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java
...va/org/apache/rocketmq/store/DefaultMessageStoreTest.java
+118
-0
未找到文件。
store/src/main/java/org/apache/rocketmq/store/CommitLog.java
浏览文件 @
da73a1d1
...
@@ -23,6 +23,7 @@ import java.util.List;
...
@@ -23,6 +23,7 @@ import java.util.List;
import
java.util.Map
;
import
java.util.Map
;
import
java.util.concurrent.CountDownLatch
;
import
java.util.concurrent.CountDownLatch
;
import
java.util.concurrent.TimeUnit
;
import
java.util.concurrent.TimeUnit
;
import
org.apache.rocketmq.common.ServiceThread
;
import
org.apache.rocketmq.common.ServiceThread
;
import
org.apache.rocketmq.common.UtilAll
;
import
org.apache.rocketmq.common.UtilAll
;
import
org.apache.rocketmq.common.constant.LoggerName
;
import
org.apache.rocketmq.common.constant.LoggerName
;
...
@@ -158,7 +159,7 @@ public class CommitLog {
...
@@ -158,7 +159,7 @@ public class CommitLog {
/**
/**
* When the normal exit, data recovery, all memory data have been flush
* When the normal exit, data recovery, all memory data have been flush
*/
*/
public
void
recoverNormally
()
{
public
void
recoverNormally
(
long
maxPhyOffsetOfConsumeQueue
)
{
boolean
checkCRCOnRecover
=
this
.
defaultMessageStore
.
getMessageStoreConfig
().
isCheckCRCOnRecover
();
boolean
checkCRCOnRecover
=
this
.
defaultMessageStore
.
getMessageStoreConfig
().
isCheckCRCOnRecover
();
final
List
<
MappedFile
>
mappedFiles
=
this
.
mappedFileQueue
.
getMappedFiles
();
final
List
<
MappedFile
>
mappedFiles
=
this
.
mappedFileQueue
.
getMappedFiles
();
if
(!
mappedFiles
.
isEmpty
())
{
if
(!
mappedFiles
.
isEmpty
())
{
...
@@ -171,6 +172,7 @@ public class CommitLog {
...
@@ -171,6 +172,7 @@ public class CommitLog {
ByteBuffer
byteBuffer
=
mappedFile
.
sliceByteBuffer
();
ByteBuffer
byteBuffer
=
mappedFile
.
sliceByteBuffer
();
long
processOffset
=
mappedFile
.
getFileFromOffset
();
long
processOffset
=
mappedFile
.
getFileFromOffset
();
long
mappedFileOffset
=
0
;
long
mappedFileOffset
=
0
;
boolean
isDamaged
=
false
;
while
(
true
)
{
while
(
true
)
{
DispatchRequest
dispatchRequest
=
this
.
checkMessageAndReturnSize
(
byteBuffer
,
checkCRCOnRecover
);
DispatchRequest
dispatchRequest
=
this
.
checkMessageAndReturnSize
(
byteBuffer
,
checkCRCOnRecover
);
int
size
=
dispatchRequest
.
getMsgSize
();
int
size
=
dispatchRequest
.
getMsgSize
();
...
@@ -198,6 +200,7 @@ public class CommitLog {
...
@@ -198,6 +200,7 @@ public class CommitLog {
// Intermediate file read error
// Intermediate file read error
else
if
(!
dispatchRequest
.
isSuccess
())
{
else
if
(!
dispatchRequest
.
isSuccess
())
{
log
.
info
(
"recover physics file end, "
+
mappedFile
.
getFileName
());
log
.
info
(
"recover physics file end, "
+
mappedFile
.
getFileName
());
isDamaged
=
true
;
break
;
break
;
}
}
}
}
...
@@ -206,6 +209,12 @@ public class CommitLog {
...
@@ -206,6 +209,12 @@ public class CommitLog {
this
.
mappedFileQueue
.
setFlushedWhere
(
processOffset
);
this
.
mappedFileQueue
.
setFlushedWhere
(
processOffset
);
this
.
mappedFileQueue
.
setCommittedWhere
(
processOffset
);
this
.
mappedFileQueue
.
setCommittedWhere
(
processOffset
);
this
.
mappedFileQueue
.
truncateDirtyFiles
(
processOffset
);
this
.
mappedFileQueue
.
truncateDirtyFiles
(
processOffset
);
// Clear ConsumeQueue redundant data
if
(
isDamaged
&&
maxPhyOffsetOfConsumeQueue
>=
processOffset
)
{
log
.
warn
(
"maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files"
,
maxPhyOffsetOfConsumeQueue
,
processOffset
);
this
.
defaultMessageStore
.
truncateDirtyLogicFiles
(
processOffset
);
}
}
}
}
}
...
@@ -390,7 +399,7 @@ public class CommitLog {
...
@@ -390,7 +399,7 @@ public class CommitLog {
this
.
confirmOffset
=
phyOffset
;
this
.
confirmOffset
=
phyOffset
;
}
}
public
void
recoverAbnormally
()
{
public
void
recoverAbnormally
(
long
maxPhyOffsetOfConsumeQueue
)
{
// recover by the minimum time stamp
// recover by the minimum time stamp
boolean
checkCRCOnRecover
=
this
.
defaultMessageStore
.
getMessageStoreConfig
().
isCheckCRCOnRecover
();
boolean
checkCRCOnRecover
=
this
.
defaultMessageStore
.
getMessageStoreConfig
().
isCheckCRCOnRecover
();
final
List
<
MappedFile
>
mappedFiles
=
this
.
mappedFileQueue
.
getMappedFiles
();
final
List
<
MappedFile
>
mappedFiles
=
this
.
mappedFileQueue
.
getMappedFiles
();
...
@@ -418,6 +427,7 @@ public class CommitLog {
...
@@ -418,6 +427,7 @@ public class CommitLog {
DispatchRequest
dispatchRequest
=
this
.
checkMessageAndReturnSize
(
byteBuffer
,
checkCRCOnRecover
);
DispatchRequest
dispatchRequest
=
this
.
checkMessageAndReturnSize
(
byteBuffer
,
checkCRCOnRecover
);
int
size
=
dispatchRequest
.
getMsgSize
();
int
size
=
dispatchRequest
.
getMsgSize
();
if
(
dispatchRequest
.
isSuccess
())
{
// Normal data
// Normal data
if
(
size
>
0
)
{
if
(
size
>
0
)
{
mappedFileOffset
+=
size
;
mappedFileOffset
+=
size
;
...
@@ -430,11 +440,6 @@ public class CommitLog {
...
@@ -430,11 +440,6 @@ public class CommitLog {
this
.
defaultMessageStore
.
doDispatch
(
dispatchRequest
);
this
.
defaultMessageStore
.
doDispatch
(
dispatchRequest
);
}
}
}
}
// Intermediate file read error
else
if
(
size
==
-
1
)
{
log
.
info
(
"recover physics file end, "
+
mappedFile
.
getFileName
());
break
;
}
// Come the end of the file, switch to the next file
// Come the end of the file, switch to the next file
// Since the return 0 representatives met last hole, this can
// Since the return 0 representatives met last hole, this can
// not be included in truncate offset
// not be included in truncate offset
...
@@ -453,6 +458,10 @@ public class CommitLog {
...
@@ -453,6 +458,10 @@ public class CommitLog {
log
.
info
(
"recover next physics file, "
+
mappedFile
.
getFileName
());
log
.
info
(
"recover next physics file, "
+
mappedFile
.
getFileName
());
}
}
}
}
}
else
{
log
.
info
(
"recover physics file end, "
+
mappedFile
.
getFileName
());
break
;
}
}
}
processOffset
+=
mappedFileOffset
;
processOffset
+=
mappedFileOffset
;
...
@@ -461,8 +470,11 @@ public class CommitLog {
...
@@ -461,8 +470,11 @@ public class CommitLog {
this
.
mappedFileQueue
.
truncateDirtyFiles
(
processOffset
);
this
.
mappedFileQueue
.
truncateDirtyFiles
(
processOffset
);
// Clear ConsumeQueue redundant data
// Clear ConsumeQueue redundant data
if
(
maxPhyOffsetOfConsumeQueue
>=
processOffset
)
{
log
.
warn
(
"maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files"
,
maxPhyOffsetOfConsumeQueue
,
processOffset
);
this
.
defaultMessageStore
.
truncateDirtyLogicFiles
(
processOffset
);
this
.
defaultMessageStore
.
truncateDirtyLogicFiles
(
processOffset
);
}
}
}
// Commitlog case files are deleted
// Commitlog case files are deleted
else
{
else
{
this
.
mappedFileQueue
.
setFlushedWhere
(
0
);
this
.
mappedFileQueue
.
setFlushedWhere
(
0
);
...
...
store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
浏览文件 @
da73a1d1
...
@@ -35,6 +35,7 @@ import java.util.concurrent.Executors;
...
@@ -35,6 +35,7 @@ import java.util.concurrent.Executors;
import
java.util.concurrent.ScheduledExecutorService
;
import
java.util.concurrent.ScheduledExecutorService
;
import
java.util.concurrent.TimeUnit
;
import
java.util.concurrent.TimeUnit
;
import
java.util.concurrent.atomic.AtomicLong
;
import
java.util.concurrent.atomic.AtomicLong
;
import
org.apache.rocketmq.common.BrokerConfig
;
import
org.apache.rocketmq.common.BrokerConfig
;
import
org.apache.rocketmq.common.MixAll
;
import
org.apache.rocketmq.common.MixAll
;
import
org.apache.rocketmq.common.ServiceThread
;
import
org.apache.rocketmq.common.ServiceThread
;
...
@@ -1276,12 +1277,12 @@ public class DefaultMessageStore implements MessageStore {
...
@@ -1276,12 +1277,12 @@ public class DefaultMessageStore implements MessageStore {
}
}
private
void
recover
(
final
boolean
lastExitOK
)
{
private
void
recover
(
final
boolean
lastExitOK
)
{
this
.
recoverConsumeQueue
();
long
maxPhyOffsetOfConsumeQueue
=
this
.
recoverConsumeQueue
();
if
(
lastExitOK
)
{
if
(
lastExitOK
)
{
this
.
commitLog
.
recoverNormally
();
this
.
commitLog
.
recoverNormally
(
maxPhyOffsetOfConsumeQueue
);
}
else
{
}
else
{
this
.
commitLog
.
recoverAbnormally
();
this
.
commitLog
.
recoverAbnormally
(
maxPhyOffsetOfConsumeQueue
);
}
}
this
.
recoverTopicQueueTable
();
this
.
recoverTopicQueueTable
();
...
@@ -1306,12 +1307,18 @@ public class DefaultMessageStore implements MessageStore {
...
@@ -1306,12 +1307,18 @@ public class DefaultMessageStore implements MessageStore {
}
}
}
}
private
void
recoverConsumeQueue
()
{
private
long
recoverConsumeQueue
()
{
long
maxPhysicOffset
=
-
1
;
for
(
ConcurrentMap
<
Integer
,
ConsumeQueue
>
maps
:
this
.
consumeQueueTable
.
values
())
{
for
(
ConcurrentMap
<
Integer
,
ConsumeQueue
>
maps
:
this
.
consumeQueueTable
.
values
())
{
for
(
ConsumeQueue
logic
:
maps
.
values
())
{
for
(
ConsumeQueue
logic
:
maps
.
values
())
{
logic
.
recover
();
logic
.
recover
();
if
(
logic
.
getMaxPhysicOffset
()
>
maxPhysicOffset
)
{
maxPhysicOffset
=
logic
.
getMaxPhysicOffset
();
}
}
}
}
}
return
maxPhysicOffset
;
}
}
private
void
recoverTopicQueueTable
()
{
private
void
recoverTopicQueueTable
()
{
...
...
store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java
浏览文件 @
da73a1d1
...
@@ -18,9 +18,12 @@
...
@@ -18,9 +18,12 @@
package
org.apache.rocketmq.store
;
package
org.apache.rocketmq.store
;
import
java.io.File
;
import
java.io.File
;
import
java.io.RandomAccessFile
;
import
java.net.InetAddress
;
import
java.net.InetAddress
;
import
java.net.InetSocketAddress
;
import
java.net.InetSocketAddress
;
import
java.net.SocketAddress
;
import
java.net.SocketAddress
;
import
java.nio.MappedByteBuffer
;
import
java.nio.channels.FileChannel
;
import
java.nio.channels.OverlappingFileLockException
;
import
java.nio.channels.OverlappingFileLockException
;
import
java.util.Map
;
import
java.util.Map
;
import
java.util.concurrent.atomic.AtomicInteger
;
import
java.util.concurrent.atomic.AtomicInteger
;
...
@@ -29,6 +32,7 @@ import org.apache.rocketmq.common.BrokerConfig;
...
@@ -29,6 +32,7 @@ import org.apache.rocketmq.common.BrokerConfig;
import
org.apache.rocketmq.common.UtilAll
;
import
org.apache.rocketmq.common.UtilAll
;
import
org.apache.rocketmq.store.config.FlushDiskType
;
import
org.apache.rocketmq.store.config.FlushDiskType
;
import
org.apache.rocketmq.store.config.MessageStoreConfig
;
import
org.apache.rocketmq.store.config.MessageStoreConfig
;
import
org.apache.rocketmq.store.config.StorePathConfigHelper
;
import
org.junit.After
;
import
org.junit.After
;
import
org.apache.rocketmq.store.stats.BrokerStatsManager
;
import
org.apache.rocketmq.store.stats.BrokerStatsManager
;
import
org.junit.Before
;
import
org.junit.Before
;
...
@@ -171,6 +175,120 @@ public class DefaultMessageStoreTest {
...
@@ -171,6 +175,120 @@ public class DefaultMessageStoreTest {
assertThat
(
getMessageResult45
.
getMessageBufferList
().
size
()).
isEqualTo
(
10
);
assertThat
(
getMessageResult45
.
getMessageBufferList
().
size
()).
isEqualTo
(
10
);
}
}
@Test
public
void
testRecover
()
throws
Exception
{
String
topic
=
"recoverTopic"
;
MessageBody
=
StoreMessage
.
getBytes
();
for
(
int
i
=
0
;
i
<
100
;
i
++)
{
MessageExtBrokerInner
messageExtBrokerInner
=
buildMessage
();
messageExtBrokerInner
.
setTopic
(
topic
);
messageExtBrokerInner
.
setQueueId
(
0
);
messageStore
.
putMessage
(
messageExtBrokerInner
);
}
Thread
.
sleep
(
100
);
//wait for build consumer queue
long
maxPhyOffset
=
messageStore
.
getMaxPhyOffset
();
long
maxCqOffset
=
messageStore
.
getMaxOffsetInQueue
(
topic
,
0
);
//1.just reboot
messageStore
.
shutdown
();
messageStore
=
buildMessageStore
();
boolean
load
=
messageStore
.
load
();
assertTrue
(
load
);
messageStore
.
start
();
assertTrue
(
maxPhyOffset
==
messageStore
.
getMaxPhyOffset
());
assertTrue
(
maxCqOffset
==
messageStore
.
getMaxOffsetInQueue
(
topic
,
0
));
//2.damage commitlog and reboot normal
for
(
int
i
=
0
;
i
<
100
;
i
++)
{
MessageExtBrokerInner
messageExtBrokerInner
=
buildMessage
();
messageExtBrokerInner
.
setTopic
(
topic
);
messageExtBrokerInner
.
setQueueId
(
0
);
messageStore
.
putMessage
(
messageExtBrokerInner
);
}
Thread
.
sleep
(
100
);
long
secondLastPhyOffset
=
messageStore
.
getMaxPhyOffset
();
long
secondLastCqOffset
=
messageStore
.
getMaxOffsetInQueue
(
topic
,
0
);
MessageExtBrokerInner
messageExtBrokerInner
=
buildMessage
();
messageExtBrokerInner
.
setTopic
(
topic
);
messageExtBrokerInner
.
setQueueId
(
0
);
messageStore
.
putMessage
(
messageExtBrokerInner
);
messageStore
.
shutdown
();
//damage last message
damageCommitlog
(
secondLastPhyOffset
);
//reboot
messageStore
=
buildMessageStore
();
load
=
messageStore
.
load
();
assertTrue
(
load
);
messageStore
.
start
();
assertTrue
(
secondLastPhyOffset
==
messageStore
.
getMaxPhyOffset
());
assertTrue
(
secondLastCqOffset
==
messageStore
.
getMaxOffsetInQueue
(
topic
,
0
));
//3.damage commitlog and reboot abnormal
for
(
int
i
=
0
;
i
<
100
;
i
++)
{
messageExtBrokerInner
=
buildMessage
();
messageExtBrokerInner
.
setTopic
(
topic
);
messageExtBrokerInner
.
setQueueId
(
0
);
messageStore
.
putMessage
(
messageExtBrokerInner
);
}
Thread
.
sleep
(
100
);
secondLastPhyOffset
=
messageStore
.
getMaxPhyOffset
();
secondLastCqOffset
=
messageStore
.
getMaxOffsetInQueue
(
topic
,
0
);
messageExtBrokerInner
=
buildMessage
();
messageExtBrokerInner
.
setTopic
(
topic
);
messageExtBrokerInner
.
setQueueId
(
0
);
messageStore
.
putMessage
(
messageExtBrokerInner
);
messageStore
.
shutdown
();
//damage last message
damageCommitlog
(
secondLastPhyOffset
);
//add abort file
String
fileName
=
StorePathConfigHelper
.
getAbortFile
(((
DefaultMessageStore
)
messageStore
).
getMessageStoreConfig
().
getStorePathRootDir
());
File
file
=
new
File
(
fileName
);
MappedFile
.
ensureDirOK
(
file
.
getParent
());
file
.
createNewFile
();
messageStore
=
buildMessageStore
();
load
=
messageStore
.
load
();
assertTrue
(
load
);
messageStore
.
start
();
assertTrue
(
secondLastPhyOffset
==
messageStore
.
getMaxPhyOffset
());
assertTrue
(
secondLastCqOffset
==
messageStore
.
getMaxOffsetInQueue
(
topic
,
0
));
//message write again
for
(
int
i
=
0
;
i
<
100
;
i
++)
{
messageExtBrokerInner
=
buildMessage
();
messageExtBrokerInner
.
setTopic
(
topic
);
messageExtBrokerInner
.
setQueueId
(
0
);
messageStore
.
putMessage
(
messageExtBrokerInner
);
}
}
private
void
damageCommitlog
(
long
offset
)
throws
Exception
{
MessageStoreConfig
messageStoreConfig
=
new
MessageStoreConfig
();
File
file
=
new
File
(
messageStoreConfig
.
getStorePathCommitLog
()
+
File
.
separator
+
"00000000000000000000"
);
FileChannel
fileChannel
=
new
RandomAccessFile
(
file
,
"rw"
).
getChannel
();
MappedByteBuffer
mappedByteBuffer
=
fileChannel
.
map
(
FileChannel
.
MapMode
.
READ_WRITE
,
0
,
1024
*
1024
*
10
);
int
bodyLen
=
mappedByteBuffer
.
getInt
((
int
)
offset
+
84
);
int
topicLenIndex
=
(
int
)
offset
+
84
+
bodyLen
+
4
;
mappedByteBuffer
.
position
(
topicLenIndex
);
mappedByteBuffer
.
putInt
(
0
);
mappedByteBuffer
.
putInt
(
0
);
mappedByteBuffer
.
putInt
(
0
);
mappedByteBuffer
.
putInt
(
0
);
mappedByteBuffer
.
force
();
fileChannel
.
force
(
true
);
fileChannel
.
close
();
}
private
class
MyMessageArrivingListener
implements
MessageArrivingListener
{
private
class
MyMessageArrivingListener
implements
MessageArrivingListener
{
@Override
@Override
public
void
arriving
(
String
topic
,
int
queueId
,
long
logicOffset
,
long
tagsCode
,
long
msgStoreTime
,
public
void
arriving
(
String
topic
,
int
queueId
,
long
logicOffset
,
long
tagsCode
,
long
msgStoreTime
,
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录