Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
taosdata
TDengine
提交
628bb62c
TDengine
项目概览
taosdata
/
TDengine
1 年多 前同步成功
通知
1185
Star
22016
Fork
4786
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
1
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
TDengine
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
1
Issue
1
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
提交
628bb62c
编写于
4月 04, 2023
作者:
dengyihao
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
add backpressure
上级
6ee01802
变更
12
隐藏空白更改
内联
并排
Showing
12 changed file
with
81 addition
and
24 deletion
+81
-24
.gitignore
.gitignore
+1
-0
include/libs/stream/tstream.h
include/libs/stream/tstream.h
+11
-9
include/util/taoserror.h
include/util/taoserror.h
+4
-0
include/util/tqueue.h
include/util/tqueue.h
+5
-1
source/dnode/snode/src/snode.c
source/dnode/snode/src/snode.c
+3
-2
source/dnode/vnode/src/tq/tq.c
source/dnode/vnode/src/tq/tq.c
+2
-2
source/libs/executor/src/dataDispatcher.c
source/libs/executor/src/dataDispatcher.c
+5
-1
source/libs/stream/src/stream.c
source/libs/stream/src/stream.c
+5
-1
source/libs/stream/src/streamExec.c
source/libs/stream/src/streamExec.c
+16
-5
source/libs/stream/src/streamQueue.c
source/libs/stream/src/streamQueue.c
+3
-1
source/util/src/terror.c
source/util/src/terror.c
+5
-0
source/util/src/tqueue.c
source/util/src/tqueue.c
+21
-2
未找到文件。
.gitignore
浏览文件 @
628bb62c
...
@@ -131,3 +131,4 @@ tools/BUGS
...
@@ -131,3 +131,4 @@ tools/BUGS
tools/taos-tools
tools/taos-tools
tools/taosws-rs
tools/taosws-rs
tags
tags
.clangd
include/libs/stream/tstream.h
浏览文件 @
628bb62c
...
@@ -204,7 +204,7 @@ typedef struct {
...
@@ -204,7 +204,7 @@ typedef struct {
int32_t
streamInit
();
int32_t
streamInit
();
void
streamCleanUp
();
void
streamCleanUp
();
SStreamQueue
*
streamQueueOpen
();
SStreamQueue
*
streamQueueOpen
(
int64_t
cap
);
void
streamQueueClose
(
SStreamQueue
*
queue
);
void
streamQueueClose
(
SStreamQueue
*
queue
);
static
FORCE_INLINE
void
streamQueueProcessSuccess
(
SStreamQueue
*
queue
)
{
static
FORCE_INLINE
void
streamQueueProcessSuccess
(
SStreamQueue
*
queue
)
{
...
@@ -374,7 +374,8 @@ int32_t tDecodeSStreamTask(SDecoder* pDecoder, SStreamTask* pTask);
...
@@ -374,7 +374,8 @@ int32_t tDecodeSStreamTask(SDecoder* pDecoder, SStreamTask* pTask);
void
tFreeSStreamTask
(
SStreamTask
*
pTask
);
void
tFreeSStreamTask
(
SStreamTask
*
pTask
);
static
FORCE_INLINE
int32_t
streamTaskInput
(
SStreamTask
*
pTask
,
SStreamQueueItem
*
pItem
)
{
static
FORCE_INLINE
int32_t
streamTaskInput
(
SStreamTask
*
pTask
,
SStreamQueueItem
*
pItem
)
{
int8_t
type
=
pItem
->
type
;
int32_t
code
=
0
;
int8_t
type
=
pItem
->
type
;
if
(
type
==
STREAM_INPUT__DATA_SUBMIT
)
{
if
(
type
==
STREAM_INPUT__DATA_SUBMIT
)
{
SStreamDataSubmit2
*
pSubmitClone
=
streamSubmitRefClone
((
SStreamDataSubmit2
*
)
pItem
);
SStreamDataSubmit2
*
pSubmitClone
=
streamSubmitRefClone
((
SStreamDataSubmit2
*
)
pItem
);
if
(
pSubmitClone
==
NULL
)
{
if
(
pSubmitClone
==
NULL
)
{
...
@@ -385,19 +386,20 @@ static FORCE_INLINE int32_t streamTaskInput(SStreamTask* pTask, SStreamQueueItem
...
@@ -385,19 +386,20 @@ static FORCE_INLINE int32_t streamTaskInput(SStreamTask* pTask, SStreamQueueItem
}
}
qDebug
(
"task %d %p submit enqueue %p %p %p %d %"
PRId64
,
pTask
->
taskId
,
pTask
,
pItem
,
pSubmitClone
,
qDebug
(
"task %d %p submit enqueue %p %p %p %d %"
PRId64
,
pTask
->
taskId
,
pTask
,
pItem
,
pSubmitClone
,
pSubmitClone
->
submit
.
msgStr
,
pSubmitClone
->
submit
.
msgLen
,
pSubmitClone
->
submit
.
ver
);
pSubmitClone
->
submit
.
msgStr
,
pSubmitClone
->
submit
.
msgLen
,
pSubmitClone
->
submit
.
ver
);
taosWriteQitem
(
pTask
->
inputQueue
->
queue
,
pSubmitClone
);
code
=
taosWriteQitem
(
pTask
->
inputQueue
->
queue
,
pSubmitClone
);
// qStreamInput(pTask->exec.executor, pSubmitClone);
// qStreamInput(pTask->exec.executor, pSubmitClone);
}
else
if
(
type
==
STREAM_INPUT__DATA_BLOCK
||
type
==
STREAM_INPUT__DATA_RETRIEVE
||
}
else
if
(
type
==
STREAM_INPUT__DATA_BLOCK
||
type
==
STREAM_INPUT__DATA_RETRIEVE
||
type
==
STREAM_INPUT__REF_DATA_BLOCK
)
{
type
==
STREAM_INPUT__REF_DATA_BLOCK
)
{
taosWriteQitem
(
pTask
->
inputQueue
->
queue
,
pItem
);
code
=
taosWriteQitem
(
pTask
->
inputQueue
->
queue
,
pItem
);
// qStreamInput(pTask->exec.executor, pItem);
// qStreamInput(pTask->exec.executor, pItem);
}
else
if
(
type
==
STREAM_INPUT__CHECKPOINT
)
{
}
else
if
(
type
==
STREAM_INPUT__CHECKPOINT
)
{
taosWriteQitem
(
pTask
->
inputQueue
->
queue
,
pItem
);
code
=
taosWriteQitem
(
pTask
->
inputQueue
->
queue
,
pItem
);
// qStreamInput(pTask->exec.executor, pItem);
// qStreamInput(pTask->exec.executor, pItem);
}
else
if
(
type
==
STREAM_INPUT__GET_RES
)
{
}
else
if
(
type
==
STREAM_INPUT__GET_RES
)
{
taosWriteQitem
(
pTask
->
inputQueue
->
queue
,
pItem
);
code
=
taosWriteQitem
(
pTask
->
inputQueue
->
queue
,
pItem
);
// qStreamInput(pTask->exec.executor, pItem);
// qStreamInput(pTask->exec.executor, pItem);
}
}
if
(
code
!=
0
)
return
code
;
if
(
type
!=
STREAM_INPUT__GET_RES
&&
type
!=
STREAM_INPUT__CHECKPOINT
&&
pTask
->
triggerParam
!=
0
)
{
if
(
type
!=
STREAM_INPUT__GET_RES
&&
type
!=
STREAM_INPUT__CHECKPOINT
&&
pTask
->
triggerParam
!=
0
)
{
atomic_val_compare_exchange_8
(
&
pTask
->
triggerStatus
,
TASK_TRIGGER_STATUS__INACTIVE
,
TASK_TRIGGER_STATUS__ACTIVE
);
atomic_val_compare_exchange_8
(
&
pTask
->
triggerStatus
,
TASK_TRIGGER_STATUS__INACTIVE
,
TASK_TRIGGER_STATUS__ACTIVE
);
...
@@ -637,9 +639,9 @@ typedef struct SStreamMeta {
...
@@ -637,9 +639,9 @@ typedef struct SStreamMeta {
SStreamMeta
*
streamMetaOpen
(
const
char
*
path
,
void
*
ahandle
,
FTaskExpand
expandFunc
,
int32_t
vgId
);
SStreamMeta
*
streamMetaOpen
(
const
char
*
path
,
void
*
ahandle
,
FTaskExpand
expandFunc
,
int32_t
vgId
);
void
streamMetaClose
(
SStreamMeta
*
streamMeta
);
void
streamMetaClose
(
SStreamMeta
*
streamMeta
);
int32_t
streamMetaSaveTask
(
SStreamMeta
*
pMeta
,
SStreamTask
*
pTask
);
int32_t
streamMetaSaveTask
(
SStreamMeta
*
pMeta
,
SStreamTask
*
pTask
);
int32_t
streamMetaAddTask
(
SStreamMeta
*
pMeta
,
int64_t
ver
,
SStreamTask
*
pTask
);
int32_t
streamMetaAddTask
(
SStreamMeta
*
pMeta
,
int64_t
ver
,
SStreamTask
*
pTask
);
int32_t
streamMetaAddSerializedTask
(
SStreamMeta
*
pMeta
,
int64_t
startVer
,
char
*
msg
,
int32_t
msgLen
);
int32_t
streamMetaAddSerializedTask
(
SStreamMeta
*
pMeta
,
int64_t
startVer
,
char
*
msg
,
int32_t
msgLen
);
// SStreamTask* streamMetaGetTask(SStreamMeta* pMeta, int32_t taskId);
// SStreamTask* streamMetaGetTask(SStreamMeta* pMeta, int32_t taskId);
SStreamTask
*
streamMetaAcquireTask
(
SStreamMeta
*
pMeta
,
int32_t
taskId
);
SStreamTask
*
streamMetaAcquireTask
(
SStreamMeta
*
pMeta
,
int32_t
taskId
);
...
...
include/util/taoserror.h
浏览文件 @
628bb62c
...
@@ -761,11 +761,15 @@ int32_t* taosGetErrno();
...
@@ -761,11 +761,15 @@ int32_t* taosGetErrno();
// stream
// stream
#define TSDB_CODE_STREAM_TASK_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x4100)
#define TSDB_CODE_STREAM_TASK_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x4100)
#define TSDB_CODE_STREAM_BACKPRESSURE_OUT_OF_QUEUE TAOS_DEF_ERROR_CODE(0, 0x4101)
// TDLite
// TDLite
#define TSDB_CODE_TDLITE_IVLD_OPEN_FLAGS TAOS_DEF_ERROR_CODE(0, 0x5100)
#define TSDB_CODE_TDLITE_IVLD_OPEN_FLAGS TAOS_DEF_ERROR_CODE(0, 0x5100)
#define TSDB_CODE_TDLITE_IVLD_OPEN_DIR TAOS_DEF_ERROR_CODE(0, 0x5101)
#define TSDB_CODE_TDLITE_IVLD_OPEN_DIR TAOS_DEF_ERROR_CODE(0, 0x5101)
// UTIL
#define TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY TAOS_DEF_ERROR_CODE(0, 0x6000)
#ifdef __cplusplus
#ifdef __cplusplus
}
}
#endif
#endif
...
...
include/util/tqueue.h
浏览文件 @
628bb62c
...
@@ -84,6 +84,8 @@ typedef struct STaosQueue {
...
@@ -84,6 +84,8 @@ typedef struct STaosQueue {
int64_t
memOfItems
;
int64_t
memOfItems
;
int32_t
numOfItems
;
int32_t
numOfItems
;
int64_t
threadId
;
int64_t
threadId
;
int64_t
memLimit
;
int64_t
itemLimit
;
}
STaosQueue
;
}
STaosQueue
;
typedef
struct
STaosQset
{
typedef
struct
STaosQset
{
...
@@ -106,12 +108,14 @@ void taosCloseQueue(STaosQueue *queue);
...
@@ -106,12 +108,14 @@ void taosCloseQueue(STaosQueue *queue);
void
taosSetQueueFp
(
STaosQueue
*
queue
,
FItem
itemFp
,
FItems
itemsFp
);
void
taosSetQueueFp
(
STaosQueue
*
queue
,
FItem
itemFp
,
FItems
itemsFp
);
void
*
taosAllocateQitem
(
int32_t
size
,
EQItype
itype
,
int64_t
dataSize
);
void
*
taosAllocateQitem
(
int32_t
size
,
EQItype
itype
,
int64_t
dataSize
);
void
taosFreeQitem
(
void
*
pItem
);
void
taosFreeQitem
(
void
*
pItem
);
void
taosWriteQitem
(
STaosQueue
*
queue
,
void
*
pItem
);
int32_t
taosWriteQitem
(
STaosQueue
*
queue
,
void
*
pItem
);
int32_t
taosReadQitem
(
STaosQueue
*
queue
,
void
**
ppItem
);
int32_t
taosReadQitem
(
STaosQueue
*
queue
,
void
**
ppItem
);
bool
taosQueueEmpty
(
STaosQueue
*
queue
);
bool
taosQueueEmpty
(
STaosQueue
*
queue
);
void
taosUpdateItemSize
(
STaosQueue
*
queue
,
int32_t
items
);
void
taosUpdateItemSize
(
STaosQueue
*
queue
,
int32_t
items
);
int32_t
taosQueueItemSize
(
STaosQueue
*
queue
);
int32_t
taosQueueItemSize
(
STaosQueue
*
queue
);
int64_t
taosQueueMemorySize
(
STaosQueue
*
queue
);
int64_t
taosQueueMemorySize
(
STaosQueue
*
queue
);
void
taosSetQueueCapacity
(
STaosQueue
*
queue
,
int64_t
size
);
void
taosSetQueueMemoryCapacity
(
STaosQueue
*
queue
,
int64_t
mem
);
STaosQall
*
taosAllocateQall
();
STaosQall
*
taosAllocateQall
();
void
taosFreeQall
(
STaosQall
*
qall
);
void
taosFreeQall
(
STaosQall
*
qall
);
...
...
source/dnode/snode/src/snode.c
浏览文件 @
628bb62c
...
@@ -66,8 +66,9 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) {
...
@@ -66,8 +66,9 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) {
pTask
->
refCnt
=
1
;
pTask
->
refCnt
=
1
;
pTask
->
schedStatus
=
TASK_SCHED_STATUS__INACTIVE
;
pTask
->
schedStatus
=
TASK_SCHED_STATUS__INACTIVE
;
pTask
->
inputQueue
=
streamQueueOpen
();
pTask
->
outputQueue
=
streamQueueOpen
();
pTask
->
inputQueue
=
streamQueueOpen
(
0
);
pTask
->
outputQueue
=
streamQueueOpen
(
0
);
if
(
pTask
->
inputQueue
==
NULL
||
pTask
->
outputQueue
==
NULL
)
{
if
(
pTask
->
inputQueue
==
NULL
||
pTask
->
outputQueue
==
NULL
)
{
return
-
1
;
return
-
1
;
...
...
source/dnode/vnode/src/tq/tq.c
浏览文件 @
628bb62c
...
@@ -934,8 +934,8 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) {
...
@@ -934,8 +934,8 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) {
pTask
->
refCnt
=
1
;
pTask
->
refCnt
=
1
;
pTask
->
schedStatus
=
TASK_SCHED_STATUS__INACTIVE
;
pTask
->
schedStatus
=
TASK_SCHED_STATUS__INACTIVE
;
pTask
->
inputQueue
=
streamQueueOpen
();
pTask
->
inputQueue
=
streamQueueOpen
(
128
<<
10
);
pTask
->
outputQueue
=
streamQueueOpen
();
pTask
->
outputQueue
=
streamQueueOpen
(
128
<<
10
);
if
(
pTask
->
inputQueue
==
NULL
||
pTask
->
outputQueue
==
NULL
)
{
if
(
pTask
->
inputQueue
==
NULL
||
pTask
->
outputQueue
==
NULL
)
{
return
-
1
;
return
-
1
;
...
...
source/libs/executor/src/dataDispatcher.c
浏览文件 @
628bb62c
...
@@ -125,6 +125,7 @@ static int32_t getStatus(SDataDispatchHandle* pDispatcher) {
...
@@ -125,6 +125,7 @@ static int32_t getStatus(SDataDispatchHandle* pDispatcher) {
}
}
static
int32_t
putDataBlock
(
SDataSinkHandle
*
pHandle
,
const
SInputData
*
pInput
,
bool
*
pContinue
)
{
static
int32_t
putDataBlock
(
SDataSinkHandle
*
pHandle
,
const
SInputData
*
pInput
,
bool
*
pContinue
)
{
int32_t
code
=
0
;
SDataDispatchHandle
*
pDispatcher
=
(
SDataDispatchHandle
*
)
pHandle
;
SDataDispatchHandle
*
pDispatcher
=
(
SDataDispatchHandle
*
)
pHandle
;
SDataDispatchBuf
*
pBuf
=
taosAllocateQitem
(
sizeof
(
SDataDispatchBuf
),
DEF_QITEM
,
0
);
SDataDispatchBuf
*
pBuf
=
taosAllocateQitem
(
sizeof
(
SDataDispatchBuf
),
DEF_QITEM
,
0
);
if
(
NULL
==
pBuf
)
{
if
(
NULL
==
pBuf
)
{
...
@@ -137,7 +138,10 @@ static int32_t putDataBlock(SDataSinkHandle* pHandle, const SInputData* pInput,
...
@@ -137,7 +138,10 @@ static int32_t putDataBlock(SDataSinkHandle* pHandle, const SInputData* pInput,
}
}
toDataCacheEntry
(
pDispatcher
,
pInput
,
pBuf
);
toDataCacheEntry
(
pDispatcher
,
pInput
,
pBuf
);
taosWriteQitem
(
pDispatcher
->
pDataBlocks
,
pBuf
);
code
=
taosWriteQitem
(
pDispatcher
->
pDataBlocks
,
pBuf
);
if
(
code
!=
0
)
{
return
code
;
}
int32_t
status
=
updateStatus
(
pDispatcher
);
int32_t
status
=
updateStatus
(
pDispatcher
);
*
pContinue
=
(
status
==
DS_BUF_LOW
||
status
==
DS_BUF_EMPTY
);
*
pContinue
=
(
status
==
DS_BUF_LOW
||
status
==
DS_BUF_EMPTY
);
...
...
source/libs/stream/src/stream.c
浏览文件 @
628bb62c
...
@@ -188,6 +188,7 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq,
...
@@ -188,6 +188,7 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq,
}
}
int32_t
streamTaskOutput
(
SStreamTask
*
pTask
,
SStreamDataBlock
*
pBlock
)
{
int32_t
streamTaskOutput
(
SStreamTask
*
pTask
,
SStreamDataBlock
*
pBlock
)
{
int32_t
code
=
0
;
if
(
pTask
->
outputType
==
TASK_OUTPUT__TABLE
)
{
if
(
pTask
->
outputType
==
TASK_OUTPUT__TABLE
)
{
pTask
->
tbSink
.
tbSinkFunc
(
pTask
,
pTask
->
tbSink
.
vnode
,
0
,
pBlock
->
blocks
);
pTask
->
tbSink
.
tbSinkFunc
(
pTask
,
pTask
->
tbSink
.
vnode
,
0
,
pBlock
->
blocks
);
taosArrayDestroyEx
(
pBlock
->
blocks
,
(
FDelete
)
blockDataFreeRes
);
taosArrayDestroyEx
(
pBlock
->
blocks
,
(
FDelete
)
blockDataFreeRes
);
...
@@ -198,7 +199,10 @@ int32_t streamTaskOutput(SStreamTask* pTask, SStreamDataBlock* pBlock) {
...
@@ -198,7 +199,10 @@ int32_t streamTaskOutput(SStreamTask* pTask, SStreamDataBlock* pBlock) {
taosFreeQitem
(
pBlock
);
taosFreeQitem
(
pBlock
);
}
else
{
}
else
{
ASSERT
(
pTask
->
outputType
==
TASK_OUTPUT__FIXED_DISPATCH
||
pTask
->
outputType
==
TASK_OUTPUT__SHUFFLE_DISPATCH
);
ASSERT
(
pTask
->
outputType
==
TASK_OUTPUT__FIXED_DISPATCH
||
pTask
->
outputType
==
TASK_OUTPUT__SHUFFLE_DISPATCH
);
taosWriteQitem
(
pTask
->
outputQueue
->
queue
,
pBlock
);
code
=
taosWriteQitem
(
pTask
->
outputQueue
->
queue
,
pBlock
);
if
(
code
!=
0
)
{
return
code
;
}
streamDispatch
(
pTask
);
streamDispatch
(
pTask
);
}
}
return
0
;
return
0
;
...
...
source/libs/stream/src/streamExec.c
浏览文件 @
628bb62c
...
@@ -20,7 +20,7 @@
...
@@ -20,7 +20,7 @@
static
int32_t
streamTaskExecImpl
(
SStreamTask
*
pTask
,
const
void
*
data
,
SArray
*
pRes
)
{
static
int32_t
streamTaskExecImpl
(
SStreamTask
*
pTask
,
const
void
*
data
,
SArray
*
pRes
)
{
int32_t
code
;
int32_t
code
;
void
*
exec
=
pTask
->
exec
.
executor
;
void
*
exec
=
pTask
->
exec
.
executor
;
while
(
atomic_load_8
(
&
pTask
->
taskStatus
)
!=
TASK_STATUS__NORMAL
)
{
while
(
atomic_load_8
(
&
pTask
->
taskStatus
)
!=
TASK_STATUS__NORMAL
)
{
qError
(
"stream task wait for the end of fill history"
);
qError
(
"stream task wait for the end of fill history"
);
taosMsleep
(
2
);
taosMsleep
(
2
);
continue
;
continue
;
...
@@ -105,8 +105,9 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, const void* data, SArray*
...
@@ -105,8 +105,9 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, const void* data, SArray*
}
}
int32_t
streamScanExec
(
SStreamTask
*
pTask
,
int32_t
batchSz
)
{
int32_t
streamScanExec
(
SStreamTask
*
pTask
,
int32_t
batchSz
)
{
ASSERT
(
pTask
->
taskLevel
==
TASK_LEVEL__SOURCE
)
;
int32_t
code
=
0
;
ASSERT
(
pTask
->
taskLevel
==
TASK_LEVEL__SOURCE
);
void
*
exec
=
pTask
->
exec
.
executor
;
void
*
exec
=
pTask
->
exec
.
executor
;
qSetStreamOpOpen
(
exec
);
qSetStreamOpOpen
(
exec
);
...
@@ -164,7 +165,11 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) {
...
@@ -164,7 +165,11 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) {
qRes
->
type
=
STREAM_INPUT__DATA_BLOCK
;
qRes
->
type
=
STREAM_INPUT__DATA_BLOCK
;
qRes
->
blocks
=
pRes
;
qRes
->
blocks
=
pRes
;
streamTaskOutput
(
pTask
,
qRes
);
code
=
streamTaskOutput
(
pTask
,
qRes
);
if
(
code
==
TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY
)
{
taosFreeQitem
(
pRes
);
return
code
;
}
if
(
pTask
->
outputType
==
TASK_OUTPUT__FIXED_DISPATCH
||
pTask
->
outputType
==
TASK_OUTPUT__SHUFFLE_DISPATCH
)
{
if
(
pTask
->
outputType
==
TASK_OUTPUT__FIXED_DISPATCH
||
pTask
->
outputType
==
TASK_OUTPUT__SHUFFLE_DISPATCH
)
{
qDebug
(
"task %d scan exec dispatch block num %d"
,
pTask
->
taskId
,
batchCnt
);
qDebug
(
"task %d scan exec dispatch block num %d"
,
pTask
->
taskId
,
batchCnt
);
...
@@ -214,6 +219,7 @@ int32_t streamBatchExec(SStreamTask* pTask, int32_t batchLimit) {
...
@@ -214,6 +219,7 @@ int32_t streamBatchExec(SStreamTask* pTask, int32_t batchLimit) {
#endif
#endif
int32_t
streamExecForAll
(
SStreamTask
*
pTask
)
{
int32_t
streamExecForAll
(
SStreamTask
*
pTask
)
{
int32_t
code
=
0
;
while
(
1
)
{
while
(
1
)
{
int32_t
batchCnt
=
1
;
int32_t
batchCnt
=
1
;
void
*
input
=
NULL
;
void
*
input
=
NULL
;
...
@@ -256,7 +262,10 @@ int32_t streamExecForAll(SStreamTask* pTask) {
...
@@ -256,7 +262,10 @@ int32_t streamExecForAll(SStreamTask* pTask) {
if
(
pTask
->
taskLevel
==
TASK_LEVEL__SINK
)
{
if
(
pTask
->
taskLevel
==
TASK_LEVEL__SINK
)
{
ASSERT
(((
SStreamQueueItem
*
)
input
)
->
type
==
STREAM_INPUT__DATA_BLOCK
);
ASSERT
(((
SStreamQueueItem
*
)
input
)
->
type
==
STREAM_INPUT__DATA_BLOCK
);
streamTaskOutput
(
pTask
,
input
);
code
=
streamTaskOutput
(
pTask
,
input
);
if
(
code
!=
0
)
{
// backpressure and record position
}
continue
;
continue
;
}
}
...
@@ -286,7 +295,9 @@ int32_t streamExecForAll(SStreamTask* pTask) {
...
@@ -286,7 +295,9 @@ int32_t streamExecForAll(SStreamTask* pTask) {
qRes
->
sourceVer
=
pMerged
->
ver
;
qRes
->
sourceVer
=
pMerged
->
ver
;
}
}
if
(
streamTaskOutput
(
pTask
,
qRes
)
<
0
)
{
code
=
streamTaskOutput
(
pTask
,
qRes
);
if
(
code
==
TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY
)
{
// backpressure and record position
taosArrayDestroyEx
(
pRes
,
(
FDelete
)
blockDataFreeRes
);
taosArrayDestroyEx
(
pRes
,
(
FDelete
)
blockDataFreeRes
);
streamFreeQitem
(
input
);
streamFreeQitem
(
input
);
taosFreeQitem
(
qRes
);
taosFreeQitem
(
qRes
);
...
...
source/libs/stream/src/streamQueue.c
浏览文件 @
628bb62c
...
@@ -15,7 +15,7 @@
...
@@ -15,7 +15,7 @@
#include "streamInc.h"
#include "streamInc.h"
SStreamQueue
*
streamQueueOpen
()
{
SStreamQueue
*
streamQueueOpen
(
int64_t
cap
)
{
SStreamQueue
*
pQueue
=
taosMemoryCalloc
(
1
,
sizeof
(
SStreamQueue
));
SStreamQueue
*
pQueue
=
taosMemoryCalloc
(
1
,
sizeof
(
SStreamQueue
));
if
(
pQueue
==
NULL
)
return
NULL
;
if
(
pQueue
==
NULL
)
return
NULL
;
pQueue
->
queue
=
taosOpenQueue
();
pQueue
->
queue
=
taosOpenQueue
();
...
@@ -24,6 +24,8 @@ SStreamQueue* streamQueueOpen() {
...
@@ -24,6 +24,8 @@ SStreamQueue* streamQueueOpen() {
goto
FAIL
;
goto
FAIL
;
}
}
pQueue
->
status
=
STREAM_QUEUE__SUCESS
;
pQueue
->
status
=
STREAM_QUEUE__SUCESS
;
taosSetQueueCapacity
(
pQueue
->
queue
,
cap
);
taosSetQueueMemoryCapacity
(
pQueue
->
queue
,
cap
*
1024
);
return
pQueue
;
return
pQueue
;
FAIL:
FAIL:
if
(
pQueue
->
queue
)
taosCloseQueue
(
pQueue
->
queue
);
if
(
pQueue
->
queue
)
taosCloseQueue
(
pQueue
->
queue
);
...
...
source/util/src/terror.c
浏览文件 @
628bb62c
...
@@ -632,11 +632,16 @@ TAOS_DEFINE_ERROR(TSDB_CODE_TMQ_CONSUMER_CLOSED, "Consumer closed")
...
@@ -632,11 +632,16 @@ TAOS_DEFINE_ERROR(TSDB_CODE_TMQ_CONSUMER_CLOSED, "Consumer closed")
// stream
// stream
TAOS_DEFINE_ERROR
(
TSDB_CODE_STREAM_TASK_NOT_EXIST
,
"Stream task not exist"
)
TAOS_DEFINE_ERROR
(
TSDB_CODE_STREAM_TASK_NOT_EXIST
,
"Stream task not exist"
)
TAOS_DEFINE_ERROR
(
TSDB_CODE_STREAM_BACKPRESSURE_OUT_OF_QUEUE
,
"Out of memory in stream queue"
)
// TDLite
// TDLite
TAOS_DEFINE_ERROR
(
TSDB_CODE_TDLITE_IVLD_OPEN_FLAGS
,
"Invalid TDLite open flags"
)
TAOS_DEFINE_ERROR
(
TSDB_CODE_TDLITE_IVLD_OPEN_FLAGS
,
"Invalid TDLite open flags"
)
TAOS_DEFINE_ERROR
(
TSDB_CODE_TDLITE_IVLD_OPEN_DIR
,
"Invalid TDLite open directory"
)
TAOS_DEFINE_ERROR
(
TSDB_CODE_TDLITE_IVLD_OPEN_DIR
,
"Invalid TDLite open directory"
)
// UTIL
TAOS_DEFINE_ERROR
(
TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY
,
"Queue out of memory"
)
#ifdef TAOS_ERROR_C
#ifdef TAOS_ERROR_C
};
};
#endif
#endif
...
...
source/util/src/tqueue.c
浏览文件 @
628bb62c
...
@@ -21,6 +21,9 @@
...
@@ -21,6 +21,9 @@
int64_t
tsRpcQueueMemoryAllowed
=
0
;
int64_t
tsRpcQueueMemoryAllowed
=
0
;
int64_t
tsRpcQueueMemoryUsed
=
0
;
int64_t
tsRpcQueueMemoryUsed
=
0
;
void
taosSetQueueMemoryCapacity
(
STaosQueue
*
queue
,
int64_t
cap
)
{
queue
->
memLimit
=
cap
;
}
void
taosSetQueueCapacity
(
STaosQueue
*
queue
,
int64_t
size
)
{
queue
->
itemLimit
=
size
;
}
STaosQueue
*
taosOpenQueue
()
{
STaosQueue
*
taosOpenQueue
()
{
STaosQueue
*
queue
=
taosMemoryCalloc
(
1
,
sizeof
(
STaosQueue
));
STaosQueue
*
queue
=
taosMemoryCalloc
(
1
,
sizeof
(
STaosQueue
));
if
(
queue
==
NULL
)
{
if
(
queue
==
NULL
)
{
...
@@ -153,11 +156,26 @@ void taosFreeQitem(void *pItem) {
...
@@ -153,11 +156,26 @@ void taosFreeQitem(void *pItem) {
taosMemoryFree
(
pNode
);
taosMemoryFree
(
pNode
);
}
}
void
taosWriteQitem
(
STaosQueue
*
queue
,
void
*
pItem
)
{
int32_t
taosWriteQitem
(
STaosQueue
*
queue
,
void
*
pItem
)
{
int32_t
code
=
0
;
STaosQnode
*
pNode
=
(
STaosQnode
*
)(((
char
*
)
pItem
)
-
sizeof
(
STaosQnode
));
STaosQnode
*
pNode
=
(
STaosQnode
*
)(((
char
*
)
pItem
)
-
sizeof
(
STaosQnode
));
pNode
->
next
=
NULL
;
pNode
->
next
=
NULL
;
taosThreadMutexLock
(
&
queue
->
mutex
);
taosThreadMutexLock
(
&
queue
->
mutex
);
if
(
queue
->
memLimit
>
0
&&
queue
->
memOfItems
+
pNode
->
size
>
queue
->
memLimit
)
{
code
=
TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY
;
uError
(
"item:%p failed to put into queue:%p, queue mem limit: %"
PRId64
", reason: %s"
PRId64
,
pItem
,
queue
,
queue
->
memLimit
,
tstrerror
(
code
));
taosThreadMutexUnlock
(
&
queue
->
mutex
);
return
code
;
}
else
if
(
queue
->
itemLimit
>
0
&&
queue
->
itemLimit
+
1
>
queue
->
itemLimit
)
{
code
=
TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY
;
uError
(
"item:%p failed to put into queue:%p, queue size limit: %"
PRId64
", reason: %s"
PRId64
,
pItem
,
queue
,
queue
->
itemLimit
,
tstrerror
(
code
));
taosThreadMutexUnlock
(
&
queue
->
mutex
);
return
code
;
}
if
(
queue
->
tail
)
{
if
(
queue
->
tail
)
{
queue
->
tail
->
next
=
pNode
;
queue
->
tail
->
next
=
pNode
;
...
@@ -166,15 +184,16 @@ void taosWriteQitem(STaosQueue *queue, void *pItem) {
...
@@ -166,15 +184,16 @@ void taosWriteQitem(STaosQueue *queue, void *pItem) {
queue
->
head
=
pNode
;
queue
->
head
=
pNode
;
queue
->
tail
=
pNode
;
queue
->
tail
=
pNode
;
}
}
queue
->
numOfItems
++
;
queue
->
numOfItems
++
;
queue
->
memOfItems
+=
pNode
->
size
;
queue
->
memOfItems
+=
pNode
->
size
;
if
(
queue
->
qset
)
atomic_add_fetch_32
(
&
queue
->
qset
->
numOfItems
,
1
);
if
(
queue
->
qset
)
atomic_add_fetch_32
(
&
queue
->
qset
->
numOfItems
,
1
);
uTrace
(
"item:%p is put into queue:%p, items:%d mem:%"
PRId64
,
pItem
,
queue
,
queue
->
numOfItems
,
queue
->
memOfItems
);
uTrace
(
"item:%p is put into queue:%p, items:%d mem:%"
PRId64
,
pItem
,
queue
,
queue
->
numOfItems
,
queue
->
memOfItems
);
taosThreadMutexUnlock
(
&
queue
->
mutex
);
taosThreadMutexUnlock
(
&
queue
->
mutex
);
if
(
queue
->
qset
)
tsem_post
(
&
queue
->
qset
->
sem
);
if
(
queue
->
qset
)
tsem_post
(
&
queue
->
qset
->
sem
);
return
code
;
}
}
int32_t
taosReadQitem
(
STaosQueue
*
queue
,
void
**
ppItem
)
{
int32_t
taosReadQitem
(
STaosQueue
*
queue
,
void
**
ppItem
)
{
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录