Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
taosdata
TDengine
提交
0716a4ae
T
TDengine
项目概览
taosdata
/
TDengine
1 年多 前同步成功
通知
1185
Star
22016
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看板
提交
0716a4ae
编写于
4月 06, 2023
作者:
L
liuyao
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
feat:add check point
上级
5aff5aa9
变更
8
显示空白变更内容
内联
并排
Showing
8 changed file
with
91 addition
and
77 deletion
+91
-77
include/common/tcommon.h
include/common/tcommon.h
+4
-4
include/libs/stream/streamState.h
include/libs/stream/streamState.h
+0
-1
include/libs/stream/tstreamFileState.h
include/libs/stream/tstreamFileState.h
+1
-1
source/common/src/tglobal.c
source/common/src/tglobal.c
+1
-1
source/libs/executor/inc/executorimpl.h
source/libs/executor/inc/executorimpl.h
+2
-0
source/libs/executor/src/timewindowoperator.c
source/libs/executor/src/timewindowoperator.c
+17
-47
source/libs/stream/src/streamState.c
source/libs/stream/src/streamState.c
+2
-20
source/libs/stream/src/tstreamFileState.c
source/libs/stream/src/tstreamFileState.c
+64
-3
未找到文件。
include/common/tcommon.h
浏览文件 @
0716a4ae
...
@@ -58,15 +58,15 @@ static inline int winKeyCmprImpl(const void* pKey1, const void* pKey2) {
...
@@ -58,15 +58,15 @@ static inline int winKeyCmprImpl(const void* pKey1, const void* pKey2) {
SWinKey
*
pWin1
=
(
SWinKey
*
)
pKey1
;
SWinKey
*
pWin1
=
(
SWinKey
*
)
pKey1
;
SWinKey
*
pWin2
=
(
SWinKey
*
)
pKey2
;
SWinKey
*
pWin2
=
(
SWinKey
*
)
pKey2
;
if
(
pWin1
->
groupId
>
pWin2
->
groupId
)
{
if
(
pWin1
->
ts
>
pWin2
->
ts
)
{
return
1
;
return
1
;
}
else
if
(
pWin1
->
groupId
<
pWin2
->
groupId
)
{
}
else
if
(
pWin1
->
ts
<
pWin2
->
ts
)
{
return
-
1
;
return
-
1
;
}
}
if
(
pWin1
->
ts
>
pWin2
->
ts
)
{
if
(
pWin1
->
groupId
>
pWin2
->
groupId
)
{
return
1
;
return
1
;
}
else
if
(
pWin1
->
ts
<
pWin2
->
ts
)
{
}
else
if
(
pWin1
->
groupId
<
pWin2
->
groupId
)
{
return
-
1
;
return
-
1
;
}
}
...
...
include/libs/stream/streamState.h
浏览文件 @
0716a4ae
...
@@ -109,7 +109,6 @@ int32_t streamStateAddIfNotExist(SStreamState* pState, const SWinKey* key, void*
...
@@ -109,7 +109,6 @@ int32_t streamStateAddIfNotExist(SStreamState* pState, const SWinKey* key, void*
int32_t
streamStateReleaseBuf
(
SStreamState
*
pState
,
const
SWinKey
*
key
,
void
*
pVal
);
int32_t
streamStateReleaseBuf
(
SStreamState
*
pState
,
const
SWinKey
*
key
,
void
*
pVal
);
void
streamFreeVal
(
void
*
val
);
void
streamFreeVal
(
void
*
val
);
SStreamStateCur
*
streamStateGetCur
(
SStreamState
*
pState
,
const
SWinKey
*
key
);
SStreamStateCur
*
streamStateGetAndCheckCur
(
SStreamState
*
pState
,
SWinKey
*
key
);
SStreamStateCur
*
streamStateGetAndCheckCur
(
SStreamState
*
pState
,
SWinKey
*
key
);
SStreamStateCur
*
streamStateSeekKeyNext
(
SStreamState
*
pState
,
const
SWinKey
*
key
);
SStreamStateCur
*
streamStateSeekKeyNext
(
SStreamState
*
pState
,
const
SWinKey
*
key
);
SStreamStateCur
*
streamStateFillSeekKeyNext
(
SStreamState
*
pState
,
const
SWinKey
*
key
);
SStreamStateCur
*
streamStateFillSeekKeyNext
(
SStreamState
*
pState
,
const
SWinKey
*
key
);
...
...
include/libs/stream/tstreamFileState.h
浏览文件 @
0716a4ae
...
@@ -47,7 +47,7 @@ int32_t getRowBuffByPos(SStreamFileState* pFileState, SRowBuffPos* pPos, void**
...
@@ -47,7 +47,7 @@ int32_t getRowBuffByPos(SStreamFileState* pFileState, SRowBuffPos* pPos, void**
bool
hasRowBuff
(
SStreamFileState
*
pFileState
,
void
*
pKey
,
int32_t
keyLen
);
bool
hasRowBuff
(
SStreamFileState
*
pFileState
,
void
*
pKey
,
int32_t
keyLen
);
SStreamSnapshot
*
getSnapshot
(
SStreamFileState
*
pFileState
);
SStreamSnapshot
*
getSnapshot
(
SStreamFileState
*
pFileState
);
int32_t
flushSnapshot
(
void
*
pFile
,
SStreamSnapshot
*
pSnapshot
,
int32_t
rowSiz
e
);
int32_t
flushSnapshot
(
SStreamFileState
*
pFileState
,
SStreamSnapshot
*
pSnapshot
,
bool
flushStat
e
);
int32_t
recoverSnapshot
(
SStreamFileState
*
pFileState
);
int32_t
recoverSnapshot
(
SStreamFileState
*
pFileState
);
#ifdef __cplusplus
#ifdef __cplusplus
...
...
source/common/src/tglobal.c
浏览文件 @
0716a4ae
...
@@ -197,7 +197,7 @@ char tsUdfdResFuncs[512] = ""; // udfd resident funcs that teardown when udf
...
@@ -197,7 +197,7 @@ char tsUdfdResFuncs[512] = ""; // udfd resident funcs that teardown when udf
char
tsUdfdLdLibPath
[
512
]
=
""
;
char
tsUdfdLdLibPath
[
512
]
=
""
;
bool
tsDisableStream
=
false
;
bool
tsDisableStream
=
false
;
int64_t
tsStreamBufferSize
=
128
*
1024
*
1024
;
int64_t
tsStreamBufferSize
=
128
*
1024
*
1024
;
int64_t
tsCheckpointInterval
=
24
*
60
*
60
*
1000
;
int64_t
tsCheckpointInterval
=
3
*
60
*
60
*
1000
;
#ifndef _STORAGE
#ifndef _STORAGE
int32_t
taosSetTfsCfg
(
SConfig
*
pCfg
)
{
int32_t
taosSetTfsCfg
(
SConfig
*
pCfg
)
{
...
...
source/libs/executor/inc/executorimpl.h
浏览文件 @
0716a4ae
...
@@ -426,6 +426,8 @@ typedef struct STimeWindowAggSupp {
...
@@ -426,6 +426,8 @@ typedef struct STimeWindowAggSupp {
int64_t
waterMark
;
int64_t
waterMark
;
TSKEY
maxTs
;
TSKEY
maxTs
;
TSKEY
minTs
;
TSKEY
minTs
;
TSKEY
checkPointTs
;
TSKEY
checkPointInterval
;
SColumnInfoData
timeWindowData
;
// query time window info for scalar function execution.
SColumnInfoData
timeWindowData
;
// query time window info for scalar function execution.
}
STimeWindowAggSupp
;
}
STimeWindowAggSupp
;
...
...
source/libs/executor/src/timewindowoperator.c
浏览文件 @
0716a4ae
...
@@ -869,22 +869,8 @@ static void removeResults(SArray* pWins, SSHashObj* pUpdatedMap) {
...
@@ -869,22 +869,8 @@ static void removeResults(SArray* pWins, SSHashObj* pUpdatedMap) {
}
}
int32_t
compareWinKey
(
void
*
pKey
,
void
*
data
,
int32_t
index
)
{
int32_t
compareWinKey
(
void
*
pKey
,
void
*
data
,
int32_t
index
)
{
SArray
*
res
=
(
SArray
*
)
data
;
void
*
pDataPos
=
taosArrayGet
((
SArray
*
)
data
,
index
);
SWinKey
*
pDataPos
=
taosArrayGet
(
res
,
index
);
return
winKeyCmprImpl
(
pKey
,
pDataPos
);
SWinKey
*
pWKey
=
(
SWinKey
*
)
pKey
;
if
(
pWKey
->
groupId
>
pDataPos
->
groupId
)
{
return
1
;
}
else
if
(
pWKey
->
groupId
<
pDataPos
->
groupId
)
{
return
-
1
;
}
if
(
pWKey
->
ts
>
pDataPos
->
ts
)
{
return
1
;
}
else
if
(
pWKey
->
ts
<
pDataPos
->
ts
)
{
return
-
1
;
}
return
0
;
}
}
static
void
removeDeleteResults
(
SSHashObj
*
pUpdatedMap
,
SArray
*
pDelWins
)
{
static
void
removeDeleteResults
(
SSHashObj
*
pUpdatedMap
,
SArray
*
pDelWins
)
{
...
@@ -1497,30 +1483,6 @@ static void deleteIntervalDiscBuf(SStreamState* pState, SHashObj* pPullDataMap,
...
@@ -1497,30 +1483,6 @@ static void deleteIntervalDiscBuf(SStreamState* pState, SHashObj* pPullDataMap,
break
;
break
;
}
}
}
}
// for debug
if
(
qDebugFlag
&
DEBUG_DEBUG
&&
mark
>
0
)
{
SStreamStateCur
*
pCur
=
streamStateGetCur
(
pState
,
key
);
int32_t
code
=
streamStateCurPrev
(
pState
,
pCur
);
if
(
code
==
TSDB_CODE_SUCCESS
)
{
SWinKey
tmpKey
=
{
0
};
code
=
streamStateGetKVByCur
(
pCur
,
&
tmpKey
,
NULL
,
0
);
if
(
code
==
TSDB_CODE_SUCCESS
)
{
STimeWindow
tw
=
getFinalTimeWindow
(
tmpKey
.
ts
,
pInterval
);
qDebug
(
"===stream===error stream state first key:%"
PRId64
"-%"
PRId64
",%"
PRIu64
",mark %"
PRId64
,
tw
.
skey
,
tw
.
ekey
,
tmpKey
.
groupId
,
mark
);
}
else
{
STimeWindow
tw
=
getFinalTimeWindow
(
key
->
ts
,
pInterval
);
qDebug
(
"===stream===stream state first key:%"
PRId64
"-%"
PRId64
",%"
PRIu64
",mark %"
PRId64
,
tw
.
skey
,
tw
.
ekey
,
key
->
groupId
,
mark
);
}
}
else
{
STimeWindow
tw
=
getFinalTimeWindow
(
key
->
ts
,
pInterval
);
qDebug
(
"===stream===stream state first key:%"
PRId64
"-%"
PRId64
",%"
PRIu64
",mark %"
PRId64
,
tw
.
skey
,
tw
.
ekey
,
key
->
groupId
,
mark
);
}
streamStateFreeCur
(
pCur
);
}
}
}
static
void
closeChildIntervalWindow
(
SOperatorInfo
*
pOperator
,
SArray
*
pChildren
,
TSKEY
maxTs
)
{
static
void
closeChildIntervalWindow
(
SOperatorInfo
*
pOperator
,
SArray
*
pChildren
,
TSKEY
maxTs
)
{
...
@@ -2641,7 +2603,10 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) {
...
@@ -2641,7 +2603,10 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) {
}
else
{
}
else
{
deleteIntervalDiscBuf
(
pInfo
->
pState
,
pInfo
->
pPullDataMap
,
pInfo
->
twAggSup
.
maxTs
-
pInfo
->
twAggSup
.
deleteMark
,
deleteIntervalDiscBuf
(
pInfo
->
pState
,
pInfo
->
pPullDataMap
,
pInfo
->
twAggSup
.
maxTs
-
pInfo
->
twAggSup
.
deleteMark
,
&
pInfo
->
interval
,
&
pInfo
->
delKey
);
&
pInfo
->
interval
,
&
pInfo
->
delKey
);
streamStateCommit
(
pTaskInfo
->
streamInfo
.
pState
);
if
(
pInfo
->
twAggSup
.
maxTs
-
pInfo
->
twAggSup
.
checkPointInterval
>
pInfo
->
twAggSup
.
checkPointTs
)
{
streamStateCommit
(
pInfo
->
pState
);
pInfo
->
twAggSup
.
checkPointTs
=
pInfo
->
twAggSup
.
maxTs
;
}
}
}
return
NULL
;
return
NULL
;
}
else
{
}
else
{
...
@@ -2833,6 +2798,8 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream,
...
@@ -2833,6 +2798,8 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream,
.
deleteMark
=
getDeleteMark
(
pIntervalPhyNode
),
.
deleteMark
=
getDeleteMark
(
pIntervalPhyNode
),
.
deleteMarkSaved
=
0
,
.
deleteMarkSaved
=
0
,
.
calTriggerSaved
=
0
,
.
calTriggerSaved
=
0
,
.
checkPointTs
=
0
,
.
checkPointInterval
=
convertTimePrecision
(
tsCheckpointInterval
,
TSDB_TIME_PRECISION_MILLI
,
pInfo
->
interval
.
precision
),
};
};
ASSERTS
(
pInfo
->
twAggSup
.
calTrigger
!=
STREAM_TRIGGER_MAX_DELAY
,
"trigger type should not be max delay"
);
ASSERTS
(
pInfo
->
twAggSup
.
calTrigger
!=
STREAM_TRIGGER_MAX_DELAY
,
"trigger type should not be max delay"
);
pInfo
->
primaryTsIndex
=
((
SColumnNode
*
)
pIntervalPhyNode
->
window
.
pTspk
)
->
slotId
;
pInfo
->
primaryTsIndex
=
((
SColumnNode
*
)
pIntervalPhyNode
->
window
.
pTspk
)
->
slotId
;
...
@@ -4887,7 +4854,10 @@ static SSDataBlock* doStreamIntervalAgg(SOperatorInfo* pOperator) {
...
@@ -4887,7 +4854,10 @@ static SSDataBlock* doStreamIntervalAgg(SOperatorInfo* pOperator) {
deleteIntervalDiscBuf
(
pInfo
->
pState
,
NULL
,
pInfo
->
twAggSup
.
maxTs
-
pInfo
->
twAggSup
.
deleteMark
,
&
pInfo
->
interval
,
deleteIntervalDiscBuf
(
pInfo
->
pState
,
NULL
,
pInfo
->
twAggSup
.
maxTs
-
pInfo
->
twAggSup
.
deleteMark
,
&
pInfo
->
interval
,
&
pInfo
->
delKey
);
&
pInfo
->
delKey
);
setOperatorCompleted
(
pOperator
);
setOperatorCompleted
(
pOperator
);
streamStateCommit
(
pTaskInfo
->
streamInfo
.
pState
);
if
(
pInfo
->
twAggSup
.
maxTs
-
pInfo
->
twAggSup
.
checkPointInterval
>
pInfo
->
twAggSup
.
checkPointTs
)
{
streamStateCommit
(
pInfo
->
pState
);
pInfo
->
twAggSup
.
checkPointTs
=
pInfo
->
twAggSup
.
maxTs
;
}
return
NULL
;
return
NULL
;
}
}
...
@@ -5000,7 +4970,7 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys
...
@@ -5000,7 +4970,7 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys
SExprInfo
*
pExprInfo
=
createExprInfo
(
pIntervalPhyNode
->
window
.
pFuncs
,
NULL
,
&
numOfCols
);
SExprInfo
*
pExprInfo
=
createExprInfo
(
pIntervalPhyNode
->
window
.
pFuncs
,
NULL
,
&
numOfCols
);
SSDataBlock
*
pResBlock
=
createDataBlockFromDescNode
(
pPhyNode
->
pOutputDataBlockDesc
);
SSDataBlock
*
pResBlock
=
createDataBlockFromDescNode
(
pPhyNode
->
pOutputDataBlockDesc
);
SInterval
interval
=
{
pInfo
->
interval
=
(
SInterval
)
{
.
interval
=
pIntervalPhyNode
->
interval
,
.
interval
=
pIntervalPhyNode
->
interval
,
.
sliding
=
pIntervalPhyNode
->
sliding
,
.
sliding
=
pIntervalPhyNode
->
sliding
,
.
intervalUnit
=
pIntervalPhyNode
->
intervalUnit
,
.
intervalUnit
=
pIntervalPhyNode
->
intervalUnit
,
...
@@ -5009,19 +4979,19 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys
...
@@ -5009,19 +4979,19 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys
.
precision
=
((
SColumnNode
*
)
pIntervalPhyNode
->
window
.
pTspk
)
->
node
.
resType
.
precision
,
.
precision
=
((
SColumnNode
*
)
pIntervalPhyNode
->
window
.
pTspk
)
->
node
.
resType
.
precision
,
};
};
STimeWindowAggSupp
twAggSupp
=
{
pInfo
->
twAggSup
=
(
STimeWindowAggSupp
)
{
.
waterMark
=
pIntervalPhyNode
->
window
.
watermark
,
.
waterMark
=
pIntervalPhyNode
->
window
.
watermark
,
.
calTrigger
=
pIntervalPhyNode
->
window
.
triggerType
,
.
calTrigger
=
pIntervalPhyNode
->
window
.
triggerType
,
.
maxTs
=
INT64_MIN
,
.
maxTs
=
INT64_MIN
,
.
minTs
=
INT64_MAX
,
.
minTs
=
INT64_MAX
,
.
deleteMark
=
getDeleteMark
(
pIntervalPhyNode
),
.
deleteMark
=
getDeleteMark
(
pIntervalPhyNode
),
.
checkPointTs
=
0
,
.
checkPointInterval
=
convertTimePrecision
(
tsCheckpointInterval
,
TSDB_TIME_PRECISION_MILLI
,
pInfo
->
interval
.
precision
),
};
};
ASSERTS
(
twAggSup
p
.
calTrigger
!=
STREAM_TRIGGER_MAX_DELAY
,
"trigger type should not be max delay"
);
ASSERTS
(
pInfo
->
twAggSu
p
.
calTrigger
!=
STREAM_TRIGGER_MAX_DELAY
,
"trigger type should not be max delay"
);
pOperator
->
pTaskInfo
=
pTaskInfo
;
pOperator
->
pTaskInfo
=
pTaskInfo
;
pInfo
->
interval
=
interval
;
pInfo
->
twAggSup
=
twAggSupp
;
pInfo
->
ignoreExpiredData
=
pIntervalPhyNode
->
window
.
igExpired
;
pInfo
->
ignoreExpiredData
=
pIntervalPhyNode
->
window
.
igExpired
;
pInfo
->
isFinal
=
false
;
pInfo
->
isFinal
=
false
;
...
...
source/libs/stream/src/streamState.c
浏览文件 @
0716a4ae
...
@@ -247,6 +247,8 @@ int32_t streamStateBegin(SStreamState* pState) {
...
@@ -247,6 +247,8 @@ int32_t streamStateBegin(SStreamState* pState) {
int32_t
streamStateCommit
(
SStreamState
*
pState
)
{
int32_t
streamStateCommit
(
SStreamState
*
pState
)
{
#ifdef USE_ROCKSDB
#ifdef USE_ROCKSDB
SStreamSnapshot
*
pShot
=
getSnapshot
(
pState
->
pFileState
);
flushSnapshot
(
pState
->
pFileState
,
pShot
,
true
);
return
0
;
return
0
;
#else
#else
if
(
tdbCommit
(
pState
->
pTdbState
->
db
,
pState
->
pTdbState
->
txn
)
<
0
)
{
if
(
tdbCommit
(
pState
->
pTdbState
->
db
,
pState
->
pTdbState
->
txn
)
<
0
)
{
...
@@ -410,26 +412,6 @@ int32_t streamStateReleaseBuf(SStreamState* pState, const SWinKey* key, void* pV
...
@@ -410,26 +412,6 @@ int32_t streamStateReleaseBuf(SStreamState* pState, const SWinKey* key, void* pV
return
0
;
return
0
;
}
}
SStreamStateCur
*
streamStateGetCur
(
SStreamState
*
pState
,
const
SWinKey
*
key
)
{
#ifdef USE_ROCKSDB
return
streamStateGetCur_rocksdb
(
pState
,
key
);
#else
SStreamStateCur
*
pCur
=
taosMemoryCalloc
(
1
,
sizeof
(
SStreamStateCur
));
if
(
pCur
==
NULL
)
return
NULL
;
tdbTbcOpen
(
pState
->
pTdbState
->
pStateDb
,
&
pCur
->
pCur
,
NULL
);
int32_t
c
=
0
;
SStateKey
sKey
=
{.
key
=
*
key
,
.
opNum
=
pState
->
number
};
tdbTbcMoveTo
(
pCur
->
pCur
,
&
sKey
,
sizeof
(
SStateKey
),
&
c
);
if
(
c
!=
0
)
{
streamStateFreeCur
(
pCur
);
return
NULL
;
}
pCur
->
number
=
pState
->
number
;
return
pCur
;
#endif
}
SStreamStateCur
*
streamStateFillGetCur
(
SStreamState
*
pState
,
const
SWinKey
*
key
)
{
SStreamStateCur
*
streamStateFillGetCur
(
SStreamState
*
pState
,
const
SWinKey
*
key
)
{
#ifdef USE_ROCKSDB
#ifdef USE_ROCKSDB
return
streamStateFillGetCur_rocksdb
(
pState
,
key
);
return
streamStateFillGetCur_rocksdb
(
pState
,
key
);
...
...
source/libs/stream/src/tstreamFileState.c
浏览文件 @
0716a4ae
...
@@ -71,6 +71,7 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t rowSize, GetTsFu
...
@@ -71,6 +71,7 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t rowSize, GetTsFu
pFileState
->
curRowCount
=
0
;
pFileState
->
curRowCount
=
0
;
pFileState
->
deleteMark
=
delMark
;
pFileState
->
deleteMark
=
delMark
;
pFileState
->
flushMark
=
-
1
;
pFileState
->
flushMark
=
-
1
;
recoverSnapshot
(
pFileState
);
return
pFileState
;
return
pFileState
;
_error:
_error:
...
@@ -152,7 +153,7 @@ int32_t flushRowBuff(SStreamFileState* pFileState) {
...
@@ -152,7 +153,7 @@ int32_t flushRowBuff(SStreamFileState* pFileState) {
i
++
;
i
++
;
}
}
}
}
flushSnapshot
(
pFileState
->
pFileStore
,
pFlushList
,
pFileState
->
rowSiz
e
);
flushSnapshot
(
pFileState
,
pFlushList
,
fals
e
);
return
TSDB_CODE_SUCCESS
;
return
TSDB_CODE_SUCCESS
;
}
}
...
@@ -272,7 +273,18 @@ SStreamSnapshot* getSnapshot(SStreamFileState* pFileState) {
...
@@ -272,7 +273,18 @@ SStreamSnapshot* getSnapshot(SStreamFileState* pFileState) {
return
pFileState
->
usedBuffs
;
return
pFileState
->
usedBuffs
;
}
}
int32_t
flushSnapshot
(
void
*
pFile
,
SStreamSnapshot
*
pSnapshot
,
int32_t
rowSize
)
{
void
streamFileStateDecode
(
SStreamFileState
*
pFileState
,
void
*
pBuff
,
int32_t
len
)
{
pBuff
=
taosDecodeFixedI64
(
pBuff
,
&
pFileState
->
flushMark
);
}
void
streamFileStateEncode
(
SStreamFileState
*
pFileState
,
void
**
pVal
,
int32_t
*
pLen
)
{
*
pLen
=
sizeof
(
TSKEY
);
*
pVal
=
taosMemoryCalloc
(
1
,
*
pLen
);
void
**
buff
=
pVal
;
taosEncodeFixedI64
(
buff
,
pFileState
->
flushMark
);
}
int32_t
flushSnapshot
(
SStreamFileState
*
pFileState
,
SStreamSnapshot
*
pSnapshot
,
bool
flushState
)
{
int32_t
code
=
TSDB_CODE_SUCCESS
;
int32_t
code
=
TSDB_CODE_SUCCESS
;
SListIter
iter
=
{
0
};
SListIter
iter
=
{
0
};
tdListInitIter
(
pSnapshot
,
&
iter
,
TD_LIST_FORWARD
);
tdListInitIter
(
pSnapshot
,
&
iter
,
TD_LIST_FORWARD
);
...
@@ -280,11 +292,60 @@ int32_t flushSnapshot(void* pFile, SStreamSnapshot* pSnapshot, int32_t rowSize)
...
@@ -280,11 +292,60 @@ int32_t flushSnapshot(void* pFile, SStreamSnapshot* pSnapshot, int32_t rowSize)
SListNode
*
pNode
=
NULL
;
SListNode
*
pNode
=
NULL
;
while
((
pNode
=
tdListNext
(
&
iter
))
!=
NULL
&&
code
==
TSDB_CODE_SUCCESS
)
{
while
((
pNode
=
tdListNext
(
&
iter
))
!=
NULL
&&
code
==
TSDB_CODE_SUCCESS
)
{
SRowBuffPos
*
pPos
=
*
(
SRowBuffPos
**
)
pNode
->
data
;
SRowBuffPos
*
pPos
=
*
(
SRowBuffPos
**
)
pNode
->
data
;
code
=
streamStatePut_rocksdb
(
pFile
,
pPos
->
pKey
,
pPos
->
pRowBuff
,
rowSize
);
code
=
streamStatePut_rocksdb
(
pFileState
->
pFileStore
,
pPos
->
pKey
,
pPos
->
pRowBuff
,
pFileState
->
rowSize
);
}
if
(
flushState
)
{
int32_t
len
=
0
;
void
*
buff
=
NULL
;
streamFileStateEncode
(
pFileState
,
&
buff
,
&
len
);
SWinKey
key
=
{.
ts
=
-
1
,
.
groupId
=
0
};
// dengyihao
streamStatePut_rocksdb
(
pFileState
->
pFileStore
,
&
key
,
buff
,
len
);
}
}
return
code
;
return
code
;
}
}
int32_t
recoverSnapshot
(
SStreamFileState
*
pFileState
)
{
int32_t
recoverSnapshot
(
SStreamFileState
*
pFileState
)
{
int32_t
code
=
TSDB_CODE_SUCCESS
;
SWinKey
stkey
=
{.
ts
=
-
1
,
.
groupId
=
0
};
// dengyihao
void
*
pStVal
=
NULL
;
int32_t
len
=
0
;
code
=
streamStateGet_rocksdb
(
pFileState
->
pFileStore
,
&
stkey
,
&
pStVal
,
&
len
);
if
(
code
==
TSDB_CODE_SUCCESS
)
{
streamFileStateDecode
(
pFileState
,
pStVal
,
len
);
}
else
{
return
TSDB_CODE_FAILED
;
}
SWinKey
key
=
{.
groupId
=
0
,
.
ts
=
0
};
SStreamStateCur
*
pCur
=
streamStateGetCur_rocksdb
(
pFileState
->
pFileStore
,
&
key
);
if
(
!
pCur
)
{
return
TSDB_CODE_FAILED
;
}
code
=
streamStateSeekLast
(
pFileState
->
pFileStore
,
pCur
);
if
(
code
!=
TSDB_CODE_SUCCESS
)
{
return
code
;
}
while
(
code
==
TSDB_CODE_SUCCESS
)
{
if
(
pFileState
->
curRowCount
==
pFileState
->
maxRowCount
)
{
break
;
}
void
*
pVal
=
NULL
;
int32_t
pVLen
=
0
;
SRowBuffPos
*
pNewPos
=
getNewRowPos
(
pFileState
);
code
=
streamStateGetKVByCur_rocksdb
(
pCur
,
pNewPos
->
pKey
,
(
const
void
**
)
&
pVal
,
&
pVLen
);
if
(
code
!=
TSDB_CODE_SUCCESS
||
pFileState
->
getTs
(
pNewPos
->
pKey
)
<
pFileState
->
flushMark
)
{
destroyRowBuffPos
(
pNewPos
);
break
;
}
memcpy
(
pNewPos
->
pRowBuff
,
pVal
,
pVLen
);
taosMemoryFree
(
pVal
);
code
=
tSimpleHashPut
(
pFileState
->
rowBuffMap
,
pNewPos
->
pKey
,
pFileState
->
rowSize
,
&
pNewPos
,
POINTER_BYTES
);
if
(
code
!=
TSDB_CODE_SUCCESS
)
{
destroyRowBuffPos
(
pNewPos
);
break
;
}
code
=
streamStateCurPrev_rocksdb
(
pFileState
->
pFileStore
,
pCur
);
}
return
TSDB_CODE_SUCCESS
;
return
TSDB_CODE_SUCCESS
;
}
}
\ No newline at end of file
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录