Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
taosdata
TDengine
提交
6f29fd40
T
TDengine
项目概览
taosdata
/
TDengine
1 年多 前同步成功
通知
1187
Star
22018
Fork
4786
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
1
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
T
TDengine
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
1
Issue
1
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
提交
6f29fd40
编写于
7月 16, 2022
作者:
M
Minghao Li
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
fix(sync): snapshot begin index
上级
a760fb80
变更
6
隐藏空白更改
内联
并排
Showing
6 changed file
with
32 addition
and
84 deletion
+32
-84
source/dnode/vnode/src/vnd/vnodeOpen.c
source/dnode/vnode/src/vnd/vnodeOpen.c
+1
-1
source/libs/sync/inc/syncInt.h
source/libs/sync/inc/syncInt.h
+1
-0
source/libs/sync/src/syncAppendEntries.c
source/libs/sync/src/syncAppendEntries.c
+4
-1
source/libs/sync/src/syncAppendEntriesReply.c
source/libs/sync/src/syncAppendEntriesReply.c
+15
-1
source/libs/sync/src/syncMain.c
source/libs/sync/src/syncMain.c
+11
-0
tests/script/tsim/sync/vnodesnapshot-test.sim
tests/script/tsim/sync/vnodesnapshot-test.sim
+0
-81
未找到文件。
source/dnode/vnode/src/vnd/vnodeOpen.c
浏览文件 @
6f29fd40
...
...
@@ -119,7 +119,7 @@ SVnode *vnodeOpen(const char *path, STfs *pTfs, SMsgCb msgCb) {
taosRealPath
(
tdir
,
NULL
,
sizeof
(
tdir
));
// for test tsdb snapshot
#if
0
#if
1
pVnode
->
config
.
walCfg
.
segSize
=
200
;
pVnode
->
config
.
walCfg
.
retentionSize
=
2000
;
#endif
...
...
source/libs/sync/inc/syncInt.h
浏览文件 @
6f29fd40
...
...
@@ -223,6 +223,7 @@ void syncNodeVoteForSelf(SSyncNode* pSyncNode);
// snapshot --------------
bool
syncNodeHasSnapshot
(
SSyncNode
*
pSyncNode
);
void
syncNodeMaybeUpdateCommitBySnapshot
(
SSyncNode
*
pSyncNode
);
SyncIndex
syncNodeGetLastIndex
(
SSyncNode
*
pSyncNode
);
SyncTerm
syncNodeGetLastTerm
(
SSyncNode
*
pSyncNode
);
...
...
source/libs/sync/src/syncAppendEntries.c
浏览文件 @
6f29fd40
...
...
@@ -711,6 +711,9 @@ int32_t syncNodeOnAppendEntriesSnapshot2Cb(SSyncNode* ths, SyncAppendEntriesBatc
syncNodeEventLog
(
ths
,
logBuf
);
}
while
(
0
);
// maybe update commit index by snapshot
syncNodeMaybeUpdateCommitBySnapshot
(
ths
);
// prepare response msg
SyncAppendEntriesReply
*
pReply
=
syncAppendEntriesReplyBuild
(
ths
->
vgId
);
pReply
->
srcId
=
ths
->
myRaftId
;
...
...
@@ -718,7 +721,7 @@ int32_t syncNodeOnAppendEntriesSnapshot2Cb(SSyncNode* ths, SyncAppendEntriesBatc
pReply
->
term
=
ths
->
pRaftStore
->
currentTerm
;
pReply
->
privateTerm
=
ths
->
pNewNodeReceiver
->
privateTerm
;
pReply
->
success
=
false
;
pReply
->
matchIndex
=
SYNC_INDEX_INVALID
;
pReply
->
matchIndex
=
ths
->
commitIndex
;
// msg event log
do
{
...
...
source/libs/sync/src/syncAppendEntriesReply.c
浏览文件 @
6f29fd40
...
...
@@ -147,6 +147,15 @@ static void syncNodeStartSnapshotOnce(SSyncNode* ths, SyncIndex beginIndex, Sync
int32_t
syncNodeOnAppendEntriesReplySnapshot2Cb
(
SSyncNode
*
ths
,
SyncAppendEntriesReply
*
pMsg
)
{
int32_t
ret
=
0
;
// print log
do
{
char
logBuf
[
256
];
snprintf
(
logBuf
,
sizeof
(
logBuf
),
"recv sync-append-entries-reply, term:%lu, match:%ld, success:%d"
,
pMsg
->
term
,
pMsg
->
matchIndex
,
pMsg
->
success
);
syncNodeEventLog
(
ths
,
logBuf
);
}
while
(
0
);
// if already drop replica, do not process
if
(
!
syncNodeInRaftGroup
(
ths
,
&
(
pMsg
->
srcId
))
&&
!
ths
->
pRaftCfg
->
isStandBy
)
{
syncNodeEventLog
(
ths
,
"recv sync-append-entries-reply, maybe replica already dropped"
);
...
...
@@ -238,7 +247,7 @@ int32_t syncNodeOnAppendEntriesReplySnapshot2Cb(SSyncNode* ths, SyncAppendEntrie
SSnapshot
oldSnapshot
;
ths
->
pFsm
->
FpGetSnapshotInfo
(
ths
->
pFsm
,
&
oldSnapshot
);
SyncTerm
newSnapshotTerm
=
oldSnapshot
.
lastApplyTerm
;
syncNodeStartSnapshotOnce
(
ths
,
SYNC_INDEX_BEGIN
,
nextIndex
,
newSnapshotTerm
,
pMsg
);
syncNodeStartSnapshotOnce
(
ths
,
pMsg
->
matchIndex
+
1
,
nextIndex
,
newSnapshotTerm
,
pMsg
);
// get sender
SSyncSnapshotSender
*
pSender
=
syncNodeGetSnapshotSender
(
ths
,
&
(
pMsg
->
srcId
));
...
...
@@ -256,6 +265,11 @@ int32_t syncNodeOnAppendEntriesReplySnapshot2Cb(SSyncNode* ths, SyncAppendEntrie
}
syncIndexMgrSetIndex
(
ths
->
pNextIndex
,
&
(
pMsg
->
srcId
),
nextIndex
);
SyncIndex
oldMatchIndex
=
syncIndexMgrGetIndex
(
ths
->
pMatchIndex
,
&
(
pMsg
->
srcId
));
if
(
pMsg
->
matchIndex
>
oldMatchIndex
)
{
syncIndexMgrSetIndex
(
ths
->
pMatchIndex
,
&
(
pMsg
->
srcId
),
pMsg
->
matchIndex
);
}
// event log, update next-index
do
{
char
host
[
64
];
...
...
source/libs/sync/src/syncMain.c
浏览文件 @
6f29fd40
...
...
@@ -1083,6 +1083,17 @@ SSyncNode* syncNodeOpen(const SSyncInfo* pOldSyncInfo) {
return
pSyncNode
;
}
void
syncNodeMaybeUpdateCommitBySnapshot
(
SSyncNode
*
pSyncNode
)
{
if
(
pSyncNode
->
pFsm
!=
NULL
&&
pSyncNode
->
pFsm
->
FpGetSnapshotInfo
!=
NULL
)
{
SSnapshot
snapshot
;
int32_t
code
=
pSyncNode
->
pFsm
->
FpGetSnapshotInfo
(
pSyncNode
->
pFsm
,
&
snapshot
);
ASSERT
(
code
==
0
);
if
(
snapshot
.
lastApplyIndex
>
pSyncNode
->
commitIndex
)
{
pSyncNode
->
commitIndex
=
snapshot
.
lastApplyIndex
;
}
}
}
void
syncNodeStart
(
SSyncNode
*
pSyncNode
)
{
// start raft
if
(
pSyncNode
->
replicaNum
==
1
)
{
...
...
tests/script/tsim/sync/vnodesnapshot-test.sim
浏览文件 @
6f29fd40
...
...
@@ -170,84 +170,3 @@ if $rows != 100 then
return -1
endi
system sh/exec.sh -n dnode1 -s stop -x SIGINT
system sh/exec.sh -n dnode2 -s stop -x SIGINT
system sh/exec.sh -n dnode3 -s stop -x SIGINT
system sh/exec.sh -n dnode4 -s stop -x SIGINT
########################################################
########################################################
print ===> start dnode1 dnode3 dnode4
system sh/exec.sh -n dnode1 -s start
#system sh/exec.sh -n dnode2 -s start
system sh/exec.sh -n dnode3 -s start
system sh/exec.sh -n dnode4 -s start
sleep 7000
print =============== query data
sql connect
sql use db
sql select * from ct1
print rows: $rows
print $data00 $data01 $data02
if $rows != 100 then
return -1
endi
system sh/exec.sh -n dnode1 -s stop -x SIGINT
#system sh/exec.sh -n dnode2 -s stop -x SIGINT
system sh/exec.sh -n dnode3 -s stop -x SIGINT
system sh/exec.sh -n dnode4 -s stop -x SIGINT
########################################################
########################################################
print ===> start dnode1 dnode2 dnode4
system sh/exec.sh -n dnode1 -s start
system sh/exec.sh -n dnode2 -s start
#system sh/exec.sh -n dnode3 -s start
system sh/exec.sh -n dnode4 -s start
sleep 3000
print =============== query data
sql select * from ct1
print rows: $rows
print $data00 $data01 $data02
if $rows != 100 then
return -1
endi
system sh/exec.sh -n dnode1 -s stop -x SIGINT
system sh/exec.sh -n dnode2 -s stop -x SIGINT
#system sh/exec.sh -n dnode3 -s stop -x SIGINT
system sh/exec.sh -n dnode4 -s stop -x SIGINT
########################################################
########################################################
print ===> start dnode1 dnode2 dnode3
system sh/exec.sh -n dnode1 -s start
system sh/exec.sh -n dnode2 -s start
system sh/exec.sh -n dnode3 -s start
#system sh/exec.sh -n dnode4 -s start
sleep 3000
print =============== query data
sql select * from ct1
print rows: $rows
print $data00 $data01 $data02
if $rows != 100 then
return -1
endi
system sh/exec.sh -n dnode1 -s stop -x SIGINT
system sh/exec.sh -n dnode2 -s stop -x SIGINT
system sh/exec.sh -n dnode3 -s stop -x SIGINT
#system sh/exec.sh -n dnode4 -s stop -x SIGINT
########################################################
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录