Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
taosdata
TDengine
提交
1e01f902
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看板
提交
1e01f902
编写于
3月 11, 2023
作者:
H
Haojun Liao
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
refactor(tmq): do some internal refactor.
上级
9d680a09
变更
1
隐藏空白更改
内联
并排
Showing
1 changed file
with
124 addition
and
122 deletion
+124
-122
source/client/src/clientTmq.c
source/client/src/clientTmq.c
+124
-122
未找到文件。
source/client/src/clientTmq.c
浏览文件 @
1e01f902
...
...
@@ -214,6 +214,8 @@ typedef struct {
}
SMqCommitCbParam
;
static
int32_t
tmqAskEp
(
tmq_t
*
tmq
,
bool
async
);
static
int32_t
makeTopicVgroupKey
(
char
*
dst
,
const
char
*
topicName
,
int32_t
vg
);
static
int32_t
tmqCommitDone
(
SMqCommitCbParamSet
*
pParamSet
);
tmq_conf_t
*
tmq_conf_new
()
{
tmq_conf_t
*
conf
=
taosMemoryCalloc
(
1
,
sizeof
(
tmq_conf_t
));
...
...
@@ -380,42 +382,6 @@ char** tmq_list_to_c_array(const tmq_list_t* list) {
return
container
->
pData
;
}
static
int32_t
makeTopicVgroupKey
(
char
*
dst
,
const
char
*
topicName
,
int32_t
vg
)
{
return
sprintf
(
dst
,
"%s:%d"
,
topicName
,
vg
);
}
int32_t
tmqCommitDone
(
SMqCommitCbParamSet
*
pParamSet
)
{
tmq_t
*
tmq
=
taosAcquireRef
(
tmqMgmt
.
rsetId
,
pParamSet
->
refId
);
if
(
tmq
==
NULL
)
{
if
(
!
pParamSet
->
async
)
{
tsem_destroy
(
&
pParamSet
->
rspSem
);
}
taosMemoryFree
(
pParamSet
);
terrno
=
TSDB_CODE_TMQ_CONSUMER_CLOSED
;
return
-
1
;
}
// if no more waiting rsp
if
(
pParamSet
->
async
)
{
// call async cb func
if
(
pParamSet
->
automatic
&&
tmq
->
commitCb
)
{
tmq
->
commitCb
(
tmq
,
pParamSet
->
rspErr
,
tmq
->
commitCbUserParam
);
}
else
if
(
!
pParamSet
->
automatic
&&
pParamSet
->
userCb
)
{
// sem post
pParamSet
->
userCb
(
tmq
,
pParamSet
->
rspErr
,
pParamSet
->
userParam
);
}
taosMemoryFree
(
pParamSet
);
}
else
{
tsem_post
(
&
pParamSet
->
rspSem
);
}
#if 0
taosArrayDestroyP(pParamSet->successfulOffsets, taosMemoryFree);
taosArrayDestroyP(pParamSet->failedOffsets, taosMemoryFree);
#endif
return
0
;
}
static
void
tmqCommitRspCountDown
(
SMqCommitCbParamSet
*
pParamSet
)
{
int32_t
waitingRspNum
=
atomic_sub_fetch_32
(
&
pParamSet
->
waitingRspNum
,
1
);
if
(
waitingRspNum
==
0
)
{
...
...
@@ -728,8 +694,8 @@ static int32_t tmqCommitConsumerImpl(tmq_t* tmq, int8_t automatic, int8_t async,
return
code
;
}
int32_t
tmqCommitInner
(
tmq_t
*
tmq
,
const
TAOS_RES
*
msg
,
int8_t
automatic
,
int8_t
async
,
tmq_commit_cb
*
userCb
,
void
*
userParam
)
{
static
int32_t
tmqCommitInner
(
tmq_t
*
tmq
,
const
TAOS_RES
*
msg
,
int8_t
automatic
,
int8_t
async
,
tmq_commit_cb
*
userCb
,
void
*
userParam
)
{
if
(
msg
)
{
return
tmqCommitMsgImpl
(
tmq
,
msg
,
async
,
userCb
,
userParam
);
}
else
{
...
...
@@ -1550,90 +1516,6 @@ END:
return
code
;
}
int32_t
tmqAskEp
(
tmq_t
*
tmq
,
bool
async
)
{
int32_t
code
=
TSDB_CODE_SUCCESS
;
#if 0
int8_t epStatus = atomic_val_compare_exchange_8(&tmq->epStatus, 0, 1);
if (epStatus == 1) {
int32_t epSkipCnt = atomic_add_fetch_32(&tmq->epSkipCnt, 1);
tscTrace("consumer:0x%" PRIx64 ", skip ask ep cnt %d", tmq->consumerId, epSkipCnt);
if (epSkipCnt < 5000) return 0;
}
atomic_store_32(&tmq->epSkipCnt, 0);
#endif
SMqAskEpReq
req
=
{
0
};
req
.
consumerId
=
tmq
->
consumerId
;
req
.
epoch
=
tmq
->
epoch
;
strcpy
(
req
.
cgroup
,
tmq
->
groupId
);
int32_t
tlen
=
tSerializeSMqAskEpReq
(
NULL
,
0
,
&
req
);
if
(
tlen
<
0
)
{
tscError
(
"consumer:0x%"
PRIx64
", tSerializeSMqAskEpReq failed"
,
tmq
->
consumerId
);
return
-
1
;
}
void
*
pReq
=
taosMemoryCalloc
(
1
,
tlen
);
if
(
pReq
==
NULL
)
{
tscError
(
"consumer:0x%"
PRIx64
", failed to malloc askEpReq msg, size:%d"
,
tmq
->
consumerId
,
tlen
);
terrno
=
TSDB_CODE_OUT_OF_MEMORY
;
return
-
1
;
}
if
(
tSerializeSMqAskEpReq
(
pReq
,
tlen
,
&
req
)
<
0
)
{
tscError
(
"consumer:0x%"
PRIx64
", tSerializeSMqAskEpReq %d failed"
,
tmq
->
consumerId
,
tlen
);
taosMemoryFree
(
pReq
);
return
-
1
;
}
SMqAskEpCbParam
*
pParam
=
taosMemoryCalloc
(
1
,
sizeof
(
SMqAskEpCbParam
));
if
(
pParam
==
NULL
)
{
tscError
(
"consumer:0x%"
PRIx64
", failed to malloc subscribe param"
,
tmq
->
consumerId
);
taosMemoryFree
(
pReq
);
return
-
1
;
}
pParam
->
refId
=
tmq
->
refId
;
pParam
->
epoch
=
tmq
->
epoch
;
pParam
->
async
=
async
;
tsem_init
(
&
pParam
->
rspSem
,
0
,
0
);
SMsgSendInfo
*
sendInfo
=
taosMemoryCalloc
(
1
,
sizeof
(
SMsgSendInfo
));
if
(
sendInfo
==
NULL
)
{
tsem_destroy
(
&
pParam
->
rspSem
);
taosMemoryFree
(
pParam
);
taosMemoryFree
(
pReq
);
return
-
1
;
}
sendInfo
->
msgInfo
=
(
SDataBuf
){
.
pData
=
pReq
,
.
len
=
tlen
,
.
handle
=
NULL
,
};
sendInfo
->
requestId
=
generateRequestId
();
sendInfo
->
requestObjRefId
=
0
;
sendInfo
->
param
=
pParam
;
sendInfo
->
fp
=
tmqAskEpCb
;
sendInfo
->
msgType
=
TDMT_MND_TMQ_ASK_EP
;
SEpSet
epSet
=
getEpSet_s
(
&
tmq
->
pTscObj
->
pAppInfo
->
mgmtEp
);
tscDebug
(
"consumer:0x%"
PRIx64
" ask ep from mnode, async:%d, reqId:0x%"
PRIx64
,
tmq
->
consumerId
,
async
,
sendInfo
->
requestId
);
int64_t
transporterId
=
0
;
asyncSendMsgToServer
(
tmq
->
pTscObj
->
pAppInfo
->
pTransporter
,
&
epSet
,
&
transporterId
,
sendInfo
);
if
(
!
async
)
{
tsem_wait
(
&
pParam
->
rspSem
);
code
=
pParam
->
code
;
taosMemoryFree
(
pParam
);
}
return
code
;
}
void
tmqBuildConsumeReqImpl
(
SMqPollReq
*
pReq
,
tmq_t
*
tmq
,
int64_t
timeout
,
SMqClientTopic
*
pTopic
,
SMqClientVg
*
pVg
)
{
int32_t
groupLen
=
strlen
(
tmq
->
groupId
);
memcpy
(
pReq
->
subKey
,
tmq
->
groupId
,
groupLen
);
...
...
@@ -2136,3 +2018,123 @@ void tmq_commit_async(tmq_t* tmq, const TAOS_RES* msg, tmq_commit_cb* cb, void*
int32_t
tmq_commit_sync
(
tmq_t
*
tmq
,
const
TAOS_RES
*
msg
)
{
return
tmqCommitInner
(
tmq
,
msg
,
0
,
0
,
NULL
,
NULL
);
}
int32_t
tmqAskEp
(
tmq_t
*
tmq
,
bool
async
)
{
int32_t
code
=
TSDB_CODE_SUCCESS
;
#if 0
int8_t epStatus = atomic_val_compare_exchange_8(&tmq->epStatus, 0, 1);
if (epStatus == 1) {
int32_t epSkipCnt = atomic_add_fetch_32(&tmq->epSkipCnt, 1);
tscTrace("consumer:0x%" PRIx64 ", skip ask ep cnt %d", tmq->consumerId, epSkipCnt);
if (epSkipCnt < 5000) return 0;
}
atomic_store_32(&tmq->epSkipCnt, 0);
#endif
SMqAskEpReq
req
=
{
0
};
req
.
consumerId
=
tmq
->
consumerId
;
req
.
epoch
=
tmq
->
epoch
;
strcpy
(
req
.
cgroup
,
tmq
->
groupId
);
int32_t
tlen
=
tSerializeSMqAskEpReq
(
NULL
,
0
,
&
req
);
if
(
tlen
<
0
)
{
tscError
(
"consumer:0x%"
PRIx64
", tSerializeSMqAskEpReq failed"
,
tmq
->
consumerId
);
return
-
1
;
}
void
*
pReq
=
taosMemoryCalloc
(
1
,
tlen
);
if
(
pReq
==
NULL
)
{
tscError
(
"consumer:0x%"
PRIx64
", failed to malloc askEpReq msg, size:%d"
,
tmq
->
consumerId
,
tlen
);
terrno
=
TSDB_CODE_OUT_OF_MEMORY
;
return
-
1
;
}
if
(
tSerializeSMqAskEpReq
(
pReq
,
tlen
,
&
req
)
<
0
)
{
tscError
(
"consumer:0x%"
PRIx64
", tSerializeSMqAskEpReq %d failed"
,
tmq
->
consumerId
,
tlen
);
taosMemoryFree
(
pReq
);
return
-
1
;
}
SMqAskEpCbParam
*
pParam
=
taosMemoryCalloc
(
1
,
sizeof
(
SMqAskEpCbParam
));
if
(
pParam
==
NULL
)
{
tscError
(
"consumer:0x%"
PRIx64
", failed to malloc subscribe param"
,
tmq
->
consumerId
);
taosMemoryFree
(
pReq
);
return
-
1
;
}
pParam
->
refId
=
tmq
->
refId
;
pParam
->
epoch
=
tmq
->
epoch
;
pParam
->
async
=
async
;
tsem_init
(
&
pParam
->
rspSem
,
0
,
0
);
SMsgSendInfo
*
sendInfo
=
taosMemoryCalloc
(
1
,
sizeof
(
SMsgSendInfo
));
if
(
sendInfo
==
NULL
)
{
tsem_destroy
(
&
pParam
->
rspSem
);
taosMemoryFree
(
pParam
);
taosMemoryFree
(
pReq
);
return
-
1
;
}
sendInfo
->
msgInfo
=
(
SDataBuf
){
.
pData
=
pReq
,
.
len
=
tlen
,
.
handle
=
NULL
,
};
sendInfo
->
requestId
=
generateRequestId
();
sendInfo
->
requestObjRefId
=
0
;
sendInfo
->
param
=
pParam
;
sendInfo
->
fp
=
tmqAskEpCb
;
sendInfo
->
msgType
=
TDMT_MND_TMQ_ASK_EP
;
SEpSet
epSet
=
getEpSet_s
(
&
tmq
->
pTscObj
->
pAppInfo
->
mgmtEp
);
tscDebug
(
"consumer:0x%"
PRIx64
" ask ep from mnode, async:%d, reqId:0x%"
PRIx64
,
tmq
->
consumerId
,
async
,
sendInfo
->
requestId
);
int64_t
transporterId
=
0
;
asyncSendMsgToServer
(
tmq
->
pTscObj
->
pAppInfo
->
pTransporter
,
&
epSet
,
&
transporterId
,
sendInfo
);
if
(
!
async
)
{
tsem_wait
(
&
pParam
->
rspSem
);
code
=
pParam
->
code
;
taosMemoryFree
(
pParam
);
}
return
code
;
}
int32_t
makeTopicVgroupKey
(
char
*
dst
,
const
char
*
topicName
,
int32_t
vg
)
{
return
sprintf
(
dst
,
"%s:%d"
,
topicName
,
vg
);
}
int32_t
tmqCommitDone
(
SMqCommitCbParamSet
*
pParamSet
)
{
tmq_t
*
tmq
=
taosAcquireRef
(
tmqMgmt
.
rsetId
,
pParamSet
->
refId
);
if
(
tmq
==
NULL
)
{
if
(
!
pParamSet
->
async
)
{
tsem_destroy
(
&
pParamSet
->
rspSem
);
}
taosMemoryFree
(
pParamSet
);
terrno
=
TSDB_CODE_TMQ_CONSUMER_CLOSED
;
return
-
1
;
}
// if no more waiting rsp
if
(
pParamSet
->
async
)
{
// call async cb func
if
(
pParamSet
->
automatic
&&
tmq
->
commitCb
)
{
tmq
->
commitCb
(
tmq
,
pParamSet
->
rspErr
,
tmq
->
commitCbUserParam
);
}
else
if
(
!
pParamSet
->
automatic
&&
pParamSet
->
userCb
)
{
// sem post
pParamSet
->
userCb
(
tmq
,
pParamSet
->
rspErr
,
pParamSet
->
userParam
);
}
taosMemoryFree
(
pParamSet
);
}
else
{
tsem_post
(
&
pParamSet
->
rspSem
);
}
#if 0
taosArrayDestroyP(pParamSet->successfulOffsets, taosMemoryFree);
taosArrayDestroyP(pParamSet->failedOffsets, taosMemoryFree);
#endif
return
0
;
}
\ No newline at end of file
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录