Merge pull request #23157 from taosdata/fix/liaohj

refactor(stream): fix bugs and do some internal refactor.
This commit is contained in:
Haojun Liao 2023-10-11 10:30:16 +08:00 committed by GitHub
commit 6982935df7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
46 changed files with 1241 additions and 704 deletions

View File

@ -3090,6 +3090,13 @@ typedef struct {
int32_t taskId; int32_t taskId;
} SVDropStreamTaskReq; } SVDropStreamTaskReq;
typedef struct {
SMsgHead head;
int64_t streamId;
int32_t taskId;
int64_t dataVer;
} SVStreamTaskVerUpdateReq;
typedef struct { typedef struct {
int8_t reserved; int8_t reserved;
} SVDropStreamTaskRsp; } SVDropStreamTaskRsp;
@ -3281,7 +3288,7 @@ typedef struct {
SMsgHead head; SMsgHead head;
int64_t streamId; int64_t streamId;
int32_t taskId; int32_t taskId;
} SVPauseStreamTaskReq; } SVPauseStreamTaskReq, SVResetStreamTaskReq;
typedef struct { typedef struct {
int8_t reserved; int8_t reserved;

View File

@ -302,11 +302,11 @@ enum { // WARN: new msg should be appended to segment tail
TD_DEF_MSG_TYPE(TDMT_SYNC_FORCE_FOLLOWER, "sync-force-become-follower", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_SYNC_FORCE_FOLLOWER, "sync-force-become-follower", NULL, NULL)
TD_NEW_MSG_SEG(TDMT_VND_STREAM_MSG) TD_NEW_MSG_SEG(TDMT_VND_STREAM_MSG)
// TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TRIGGER, "vnode-stream-trigger", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_STREAM_SCAN_HISTORY, "vnode-stream-scan-history", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_SCAN_HISTORY, "vnode-stream-scan-history", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_STREAM_SCAN_HISTORY_FINISH, "vnode-stream-scan-history-finish", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_SCAN_HISTORY_FINISH, "vnode-stream-scan-history-finish", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_STREAM_CHECK_POINT_SOURCE, "vnode-stream-checkpoint-source", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_CHECK_POINT_SOURCE, "vnode-stream-checkpoint-source", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_UPDATE, "vnode-stream-update", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_UPDATE, "vnode-stream-update", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_RESET, "vnode-stream-reset", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_CHECK, "vnode-stream-task-check", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_CHECK, "vnode-stream-task-check", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_STREAM_MAX_MSG, "vnd-stream-max", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_MAX_MSG, "vnd-stream-max", NULL, NULL)

View File

@ -44,7 +44,8 @@ extern "C" {
#define NODE_ROLE_LEADER 0x2 #define NODE_ROLE_LEADER 0x2
#define NODE_ROLE_FOLLOWER 0x3 #define NODE_ROLE_FOLLOWER 0x3
typedef struct SStreamTask SStreamTask; typedef struct SStreamTask SStreamTask;
typedef struct SStreamQueue SStreamQueue;
#define SSTREAM_TASK_VER 2 #define SSTREAM_TASK_VER 2
enum { enum {
@ -190,13 +191,6 @@ int32_t streamQueuePush(SStreamQueue1* pQueue, SStreamQueueItem* pItem);
SStreamQueueRes streamQueueGetRes(SStreamQueue1* pQueue); SStreamQueueRes streamQueueGetRes(SStreamQueue1* pQueue);
#endif #endif
typedef struct {
STaosQueue* pQueue;
STaosQall* qall;
void* qItem;
int8_t status;
} SStreamQueue;
int32_t streamInit(); int32_t streamInit();
void streamCleanUp(); void streamCleanUp();
@ -265,8 +259,9 @@ typedef struct SStreamTaskId {
typedef struct SCheckpointInfo { typedef struct SCheckpointInfo {
int64_t startTs; int64_t startTs;
int64_t checkpointId; int64_t checkpointId;
int64_t checkpointVer; // latest checkpointId version int64_t checkpointVer; // latest checkpointId version
int64_t nextProcessVer; // current offset in WAL, not serialize it int64_t nextProcessVer; // current offset in WAL, not serialize it
int64_t failedId; // record the latest failed checkpoint id
} SCheckpointInfo; } SCheckpointInfo;
typedef struct SStreamStatus { typedef struct SStreamStatus {
@ -277,6 +272,7 @@ typedef struct SStreamStatus {
bool appendTranstateBlock; // has append the transfer state data block already, todo: remove it bool appendTranstateBlock; // has append the transfer state data block already, todo: remove it
int8_t pauseAllowed; // allowed task status to be set to be paused int8_t pauseAllowed; // allowed task status to be set to be paused
int32_t timerActive; // timer is active int32_t timerActive; // timer is active
int32_t inScanHistorySentinel;
} SStreamStatus; } SStreamStatus;
typedef struct SDataRange { typedef struct SDataRange {
@ -296,6 +292,8 @@ typedef struct SSTaskBasicInfo {
} SSTaskBasicInfo; } SSTaskBasicInfo;
typedef struct SStreamDispatchReq SStreamDispatchReq; typedef struct SStreamDispatchReq SStreamDispatchReq;
typedef struct STokenBucket STokenBucket;
typedef struct SMetaHbInfo SMetaHbInfo;
typedef struct SDispatchMsgInfo { typedef struct SDispatchMsgInfo {
SStreamDispatchReq* pData; // current dispatch data SStreamDispatchReq* pData; // current dispatch data
@ -307,14 +305,13 @@ typedef struct SDispatchMsgInfo {
void* pTimer; // used to dispatch data after a given time duration void* pTimer; // used to dispatch data after a given time duration
} SDispatchMsgInfo; } SDispatchMsgInfo;
typedef struct STaskOutputInfo { typedef struct STaskOutputQueue {
int8_t type;
int8_t status; int8_t status;
SStreamQueue* queue; SStreamQueue* queue;
} STaskOutputInfo; } STaskOutputQueue;
typedef struct STaskInputInfo { typedef struct STaskInputInfo {
int8_t status; int8_t status;
SStreamQueue* queue; SStreamQueue* queue;
} STaskInputInfo; } STaskInputInfo;
@ -354,29 +351,7 @@ typedef struct SHistoryTaskInfo {
int32_t waitInterval; int32_t waitInterval;
} SHistoryTaskInfo; } SHistoryTaskInfo;
typedef struct STokenBucket STokenBucket; typedef struct STaskOutputInfo {
typedef struct SMetaHbInfo SMetaHbInfo;
struct SStreamTask {
int64_t ver;
SStreamTaskId id;
SSTaskBasicInfo info;
STaskOutputInfo outputInfo;
STaskInputInfo inputInfo;
STaskSchedInfo schedInfo;
SDispatchMsgInfo msgInfo;
SStreamStatus status;
SCheckpointInfo chkInfo;
STaskExec exec;
SDataRange dataRange;
SHistoryTaskInfo hTaskInfo;
STaskId streamTaskId;
STaskExecStatisInfo execInfo;
SArray* pReadyMsgList; // SArray<SStreamChkptReadyInfo*>
TdThreadMutex lock; // secure the operation of set task status and puting data into inputQ
SArray* pUpstreamInfoList;
// output
union { union {
STaskDispatcherFixed fixedDispatcher; STaskDispatcherFixed fixedDispatcher;
STaskDispatcherShuffle shuffleDispatcher; STaskDispatcherShuffle shuffleDispatcher;
@ -384,12 +359,37 @@ struct SStreamTask {
STaskSinkSma smaSink; STaskSinkSma smaSink;
STaskSinkFetch fetchSink; STaskSinkFetch fetchSink;
}; };
int8_t type;
STokenBucket* pTokenBucket; STokenBucket* pTokenBucket;
SMsgCb* pMsgCb; // msg handle } STaskOutputInfo;
SStreamState* pState; // state backend
SArray* pRspMsgList;
typedef struct SUpstreamInfo {
SArray* pList;
int32_t numOfClosed;
} SUpstreamInfo;
struct SStreamTask {
int64_t ver;
SStreamTaskId id;
SSTaskBasicInfo info;
STaskOutputQueue outputq;
STaskInputInfo inputInfo;
STaskSchedInfo schedInfo;
STaskOutputInfo outputInfo;
SDispatchMsgInfo msgInfo;
SStreamStatus status;
SCheckpointInfo chkInfo;
STaskExec exec;
SDataRange dataRange;
SHistoryTaskInfo hTaskInfo;
STaskId streamTaskId;
STaskExecStatisInfo execInfo;
SArray* pReadyMsgList; // SArray<SStreamChkptReadyInfo*>
TdThreadMutex lock; // secure the operation of set task status and puting data into inputQ
SMsgCb* pMsgCb; // msg handle
SStreamState* pState; // state backend
SArray* pRspMsgList;
SUpstreamInfo upstreamInfo;
// the followings attributes don't be serialized // the followings attributes don't be serialized
int32_t notReadyTasks; int32_t notReadyTasks;
int32_t numOfWaitingUpstream; int32_t numOfWaitingUpstream;
@ -406,7 +406,8 @@ struct SStreamTask {
}; };
typedef struct STaskStartInfo { typedef struct STaskStartInfo {
int64_t ts; int64_t startTs;
int64_t readyTs;
int32_t startedAfterNodeUpdate; int32_t startedAfterNodeUpdate;
SHashObj* pReadyTaskSet; // tasks that are all ready for running stream processing SHashObj* pReadyTaskSet; // tasks that are all ready for running stream processing
int32_t elapsedTime; int32_t elapsedTime;
@ -450,8 +451,8 @@ typedef struct SStreamMeta {
int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo);
int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo); int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo);
SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHistory, int64_t triggerParam, SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory, int64_t triggerParam,
SArray* pTaskList); SArray* pTaskList, bool hasFillhistory);
int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask); int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask);
int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask); int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask);
void tFreeStreamTask(SStreamTask* pTask); void tFreeStreamTask(SStreamTask* pTask);
@ -463,7 +464,7 @@ int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId);
int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem); int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem);
int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock); int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock);
int32_t streamTaskPutTranstateIntoInputQ(SStreamTask* pTask); int32_t streamTaskPutTranstateIntoInputQ(SStreamTask* pTask);
bool streamQueueIsFull(const STaosQueue* pQueue, bool inputQ); bool streamQueueIsFull(const SStreamQueue* pQueue);
typedef struct { typedef struct {
SMsgHead head; SMsgHead head;
@ -602,6 +603,15 @@ typedef struct STaskStatusEntry {
int32_t status; int32_t status;
int32_t stage; int32_t stage;
int32_t nodeId; int32_t nodeId;
int64_t verStart; // start version in WAL, only valid for source task
int64_t verEnd; // end version in WAL, only valid for source task
int64_t processedVer; // only valid for source task
int64_t activeCheckpointId; // current active checkpoint id
bool checkpointFailed; // denote if the checkpoint is failed or not
double inputQUsed; // in MiB
double inputRate;
double sinkQuota; // existed quota size for sink task
double sinkDataSize; // sink to dest data size
} STaskStatusEntry; } STaskStatusEntry;
typedef struct SStreamHbMsg { typedef struct SStreamHbMsg {
@ -667,7 +677,6 @@ void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq);
int32_t streamSetupScheduleTrigger(SStreamTask* pTask); int32_t streamSetupScheduleTrigger(SStreamTask* pTask);
int32_t streamProcessRunReq(SStreamTask* pTask);
int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg); int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg);
int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code); int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code);
@ -675,7 +684,7 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieve
SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId); SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId);
void streamTaskInputFail(SStreamTask* pTask); void streamTaskInputFail(SStreamTask* pTask);
int32_t streamTryExec(SStreamTask* pTask); int32_t streamExecTask(SStreamTask* pTask);
int32_t streamSchedExec(SStreamTask* pTask); int32_t streamSchedExec(SStreamTask* pTask);
bool streamTaskShouldStop(const SStreamStatus* pStatus); bool streamTaskShouldStop(const SStreamStatus* pStatus);
bool streamTaskShouldPause(const SStreamStatus* pStatus); bool streamTaskShouldPause(const SStreamStatus* pStatus);
@ -687,11 +696,12 @@ char* createStreamTaskIdStr(int64_t streamId, int32_t taskId);
// recover and fill history // recover and fill history
void streamTaskCheckDownstream(SStreamTask* pTask); void streamTaskCheckDownstream(SStreamTask* pTask);
int32_t streamTaskLaunchScanHistory(SStreamTask* pTask); int32_t streamTaskStartScanHistory(SStreamTask* pTask);
int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage); int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage);
int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList); int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList);
void streamTaskResetUpstreamStageInfo(SStreamTask* pTask); void streamTaskResetUpstreamStageInfo(SStreamTask* pTask);
int8_t streamTaskSetSchedStatusWait(SStreamTask* pTask); bool streamTaskAllUpstreamClosed(SStreamTask* pTask);
bool streamTaskSetSchedStatusWait(SStreamTask* pTask);
int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask); int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask);
int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask); int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask);
@ -704,6 +714,7 @@ int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask);
int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated); int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated);
bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t latestVer); bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t latestVer);
int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue); int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue);
int32_t streamQueueGetAvailableSpace(const SStreamQueue* pQueue, int32_t* availNum, double* availSize);
// common // common
int32_t streamRestoreParam(SStreamTask* pTask); int32_t streamRestoreParam(SStreamTask* pTask);
@ -724,6 +735,9 @@ int32_t streamTaskReloadState(SStreamTask* pTask);
void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId); void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId);
void streamTaskOpenAllUpstreamInput(SStreamTask* pTask); void streamTaskOpenAllUpstreamInput(SStreamTask* pTask);
void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask);
void streamTaskStatusCopy(STaskStatusEntry* pDst, const STaskStatusEntry* pSrc);
// source level // source level
int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow); int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow);
int32_t streamSetParamForStreamScannerStep2(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow); int32_t streamSetParamForStreamScannerStep2(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow);
@ -758,6 +772,7 @@ void streamMetaInitForSnode(SStreamMeta* pMeta);
// checkpoint // checkpoint
int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq);
int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask); int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask);
void streamTaskClearCheckInfo(SStreamTask* pTask);
int32_t streamAlignTransferState(SStreamTask* pTask); int32_t streamAlignTransferState(SStreamTask* pTask);
int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId); int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId);

View File

@ -161,9 +161,12 @@ static const SSysDbTableSchema streamTaskSchema[] = {
{.name = "task_id", .bytes = 16 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "task_id", .bytes = 16 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false},
{.name = "node_type", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "node_type", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false},
{.name = "node_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "node_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false},
{.name = "level", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "level", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false},
{.name = "status", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "status", .bytes = 15 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false},
{.name = "stage", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "stage", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false},
{.name = "in_queue", .bytes = 20, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false},
// {.name = "out_queue", .bytes = 20, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false},
{.name = "info", .bytes = 25, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false},
}; };
static const SSysDbTableSchema userTblsSchema[] = { static const SSysDbTableSchema userTblsSchema[] = {

View File

@ -240,7 +240,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch
// internal // internal
int32_t tsTransPullupInterval = 2; int32_t tsTransPullupInterval = 2;
int32_t tsMqRebalanceInterval = 2; int32_t tsMqRebalanceInterval = 2;
int32_t tsStreamCheckpointTickInterval = 600; int32_t tsStreamCheckpointTickInterval = 300;
int32_t tsStreamNodeCheckInterval = 30; int32_t tsStreamNodeCheckInterval = 30;
int32_t tsTtlUnit = 86400; int32_t tsTtlUnit = 86400;
int32_t tsTtlPushIntervalSec = 10; int32_t tsTtlPushIntervalSec = 10;
@ -263,8 +263,6 @@ char tsS3BucketName[TSDB_FQDN_LEN] = "<bucketname>";
char tsS3AppId[TSDB_FQDN_LEN] = "<appid>"; char tsS3AppId[TSDB_FQDN_LEN] = "<appid>";
int8_t tsS3Enabled = false; int8_t tsS3Enabled = false;
int32_t tsCheckpointInterval = 20;
#ifndef _STORAGE #ifndef _STORAGE
int32_t taosSetTfsCfg(SConfig *pCfg) { int32_t taosSetTfsCfg(SConfig *pCfg) {
SConfigItem *pItem = cfgGetItem(pCfg, "dataDir"); SConfigItem *pItem = cfgGetItem(pCfg, "dataDir");
@ -641,7 +639,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) {
if (cfgAddBool(pCfg, "disableStream", tsDisableStream, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddBool(pCfg, "disableStream", tsDisableStream, CFG_SCOPE_SERVER) != 0) return -1;
if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER) != 0) return -1;
if (cfgAddInt64(pCfg, "checkpointInterval", tsCheckpointInterval, 0, INT64_MAX, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt64(pCfg, "checkpointInterval", tsStreamCheckpointTickInterval, 60, 1200, CFG_SCOPE_SERVER) != 0) return -1;
if (cfgAddInt32(pCfg, "cacheLazyLoadThreshold", tsCacheLazyLoadThreshold, 0, 100000, CFG_SCOPE_SERVER) != 0) if (cfgAddInt32(pCfg, "cacheLazyLoadThreshold", tsCacheLazyLoadThreshold, 0, 100000, CFG_SCOPE_SERVER) != 0)
return -1; return -1;

View File

@ -212,6 +212,7 @@ SArray *mmGetMsgHandles() {
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_STOP_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_STOP_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_CHECK_POINT_SOURCE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_CHECK_POINT_SOURCE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_RESET_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT, mmPutMsgToReadQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_CONFIG_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_CONFIG_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER;

View File

@ -794,13 +794,13 @@ SArray *vmGetMsgHandles() {
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_SCAN_HISTORY_FINISH_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_SCAN_HISTORY_FINISH_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_CHECK, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_CHECK, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_CHECK_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_CHECK_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
// if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TRIGGER, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_PAUSE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_PAUSE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RESUME, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RESUME, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_STOP, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_STOP, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_CHECK_POINT_SOURCE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_CHECK_POINT_SOURCE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECKPOINT_READY, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECKPOINT_READY, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_RESET, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_REPLICA, vmPutMsgToMgmtQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_REPLICA, vmPutMsgToMgmtQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_CONFIG, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_CONFIG, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;

View File

@ -27,8 +27,8 @@
#define SINK_NODE_LEVEL (0) #define SINK_NODE_LEVEL (0)
extern bool tsDeployOnSnode; extern bool tsDeployOnSnode;
static int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, static int32_t doAddSinkTask(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup,
SVgObj* pVgroup, SEpSet* pEpset, int32_t fillHistory); SEpSet* pEpset, bool isFillhistory);
int32_t mndConvertRsmaTask(char** pDst, int32_t* pDstLen, const char* ast, int64_t uid, int8_t triggerType, int32_t mndConvertRsmaTask(char** pDst, int32_t* pDstLen, const char* ast, int64_t uid, int8_t triggerType,
int64_t watermark, int64_t deleteMark) { int64_t watermark, int64_t deleteMark) {
@ -87,15 +87,17 @@ END:
} }
int32_t mndSetSinkTaskInfo(SStreamObj* pStream, SStreamTask* pTask) { int32_t mndSetSinkTaskInfo(SStreamObj* pStream, SStreamTask* pTask) {
STaskOutputInfo* pInfo = &pTask->outputInfo;
if (pStream->smaId != 0) { if (pStream->smaId != 0) {
pTask->outputInfo.type = TASK_OUTPUT__SMA; pInfo->type = TASK_OUTPUT__SMA;
pTask->smaSink.smaId = pStream->smaId; pInfo->smaSink.smaId = pStream->smaId;
} else { } else {
pTask->outputInfo.type = TASK_OUTPUT__TABLE; pInfo->type = TASK_OUTPUT__TABLE;
pTask->tbSink.stbUid = pStream->targetStbUid; pInfo->tbSink.stbUid = pStream->targetStbUid;
memcpy(pTask->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); memcpy(pInfo->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN);
pTask->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema); pInfo->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema);
if (pTask->tbSink.pSchemaWrapper == NULL) { if (pInfo->tbSink.pSchemaWrapper == NULL) {
return TSDB_CODE_OUT_OF_MEMORY; return TSDB_CODE_OUT_OF_MEMORY;
} }
} }
@ -113,7 +115,7 @@ int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SAr
isShuffle = true; isShuffle = true;
pTask->outputInfo.type = TASK_OUTPUT__SHUFFLE_DISPATCH; pTask->outputInfo.type = TASK_OUTPUT__SHUFFLE_DISPATCH;
pTask->msgInfo.msgType = TDMT_STREAM_TASK_DISPATCH; pTask->msgInfo.msgType = TDMT_STREAM_TASK_DISPATCH;
if (mndExtractDbInfo(pMnode, pDb, &pTask->shuffleDispatcher.dbInfo, NULL) < 0) { if (mndExtractDbInfo(pMnode, pDb, &pTask->outputInfo.shuffleDispatcher.dbInfo, NULL) < 0) {
return -1; return -1;
} }
} }
@ -124,8 +126,8 @@ int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SAr
int32_t numOfSinkNodes = taosArrayGetSize(pSinkNodeList); int32_t numOfSinkNodes = taosArrayGetSize(pSinkNodeList);
if (isShuffle) { if (isShuffle) {
memcpy(pTask->shuffleDispatcher.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); memcpy(pTask->outputInfo.shuffleDispatcher.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN);
SArray* pVgs = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* pVgs = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
int32_t numOfVgroups = taosArrayGetSize(pVgs); int32_t numOfVgroups = taosArrayGetSize(pVgs);
for (int32_t i = 0; i < numOfVgroups; i++) { for (int32_t i = 0; i < numOfVgroups; i++) {
@ -207,8 +209,7 @@ SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) {
} }
// create sink node for each vgroup. // create sink node for each vgroup.
int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStream, SEpSet* pEpset, int32_t doAddShuffleSinkTask(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStream, SEpSet* pEpset, bool fillHistory) {
int32_t fillHistory) {
SSdb* pSdb = pMnode->pSdb; SSdb* pSdb = pMnode->pSdb;
void* pIter = NULL; void* pIter = NULL;
@ -224,17 +225,17 @@ int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStrea
continue; continue;
} }
mndAddSinkTaskToStream(pStream, pTaskList, pMnode, pVgroup->vgId, pVgroup, pEpset, fillHistory); doAddSinkTask(pStream, pTaskList, pMnode, pVgroup->vgId, pVgroup, pEpset, fillHistory);
sdbRelease(pSdb, pVgroup); sdbRelease(pSdb, pVgroup);
} }
return 0; return 0;
} }
int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, int32_t doAddSinkTask(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup,
SEpSet* pEpset, int32_t fillHistory) { SEpSet* pEpset, bool isFillhistory) {
int64_t uid = (fillHistory == 0)? pStream->uid:pStream->hTaskUid; int64_t uid = (isFillhistory)? pStream->hTaskUid:pStream->uid;
SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, fillHistory, 0, pTaskList); SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, isFillhistory, 0, pTaskList, pStream->conf.fillHistory);
if (pTask == NULL) { if (pTask == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1; return -1;
@ -248,17 +249,16 @@ int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* p
return 0; return 0;
} }
static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList, static int32_t addSourceTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList,
SStreamObj* pStream, SSubplan* plan, uint64_t uid, SEpSet* pEpset, SStreamObj* pStream, SSubplan* plan, uint64_t uid, SEpSet* pEpset, bool fillHistory,
int8_t fillHistory, bool hasExtraSink, int64_t firstWindowSkey) { bool hasExtraSink, int64_t firstWindowSkey, bool hasFillHistory) {
SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, pStream->conf.triggerParam, pTaskList); SStreamTask* pTask =
tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, pStream->conf.triggerParam, pTaskList, hasFillHistory);
if (pTask == NULL) { if (pTask == NULL) {
return terrno; return terrno;
} }
epsetAssign(&pTask->info.mnodeEpset, pEpset); epsetAssign(&pTask->info.mnodeEpset, pEpset);
// todo set the correct ts, which should be last key of queried table.
STimeWindow* pWindow = &pTask->dataRange.window; STimeWindow* pWindow = &pTask->dataRange.window;
pWindow->skey = INT64_MIN; pWindow->skey = INT64_MIN;
@ -345,8 +345,8 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan*
// new stream task // new stream task
SArray** pSinkTaskList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); SArray** pSinkTaskList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL);
int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, pEpset, int32_t code = addSourceTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, pEpset,
0, hasExtraSink, nextWindowSkey); false, hasExtraSink, nextWindowSkey, pStream->conf.fillHistory);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
sdbRelease(pSdb, pVgroup); sdbRelease(pSdb, pVgroup);
return -1; return -1;
@ -354,8 +354,8 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan*
if (pStream->conf.fillHistory) { if (pStream->conf.fillHistory) {
SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL); SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL);
code = addSourceStreamTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, code = addSourceTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid,
pEpset, 1, hasExtraSink, nextWindowSkey); pEpset, true, hasExtraSink, nextWindowSkey, true);
} }
sdbRelease(pSdb, pVgroup); sdbRelease(pSdb, pVgroup);
@ -371,10 +371,10 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan*
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t uid, SStreamTask* pDownstreamTask, static int32_t doAddSourceTask(SArray* pTaskList, bool isFillhistory, int64_t uid, SStreamTask* pDownstreamTask,
SMnode* pMnode, SSubplan* pPlan, SVgObj* pVgroup, SEpSet* pEpset, SMnode* pMnode, SSubplan* pPlan, SVgObj* pVgroup, SEpSet* pEpset,
int64_t nextWindowSkey) { int64_t nextWindowSkey, bool hasFillHistory) {
SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, 0, pTaskList); SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, isFillhistory, 0, pTaskList, hasFillHistory);
if (pTask == NULL) { if (pTask == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1; return -1;
@ -400,8 +400,8 @@ static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t ui
} }
static int32_t doAddAggTask(uint64_t uid, SArray* pTaskList, SArray* pSinkNodeList, SMnode* pMnode, SStreamObj* pStream, static int32_t doAddAggTask(uint64_t uid, SArray* pTaskList, SArray* pSinkNodeList, SMnode* pMnode, SStreamObj* pStream,
SEpSet* pEpset, int32_t fillHistory, SStreamTask** pAggTask) { SEpSet* pEpset, bool fillHistory, SStreamTask** pAggTask, bool hasFillhistory) {
*pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, fillHistory, pStream->conf.triggerParam, pTaskList); *pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, fillHistory, pStream->conf.triggerParam, pTaskList, hasFillhistory);
if (*pAggTask == NULL) { if (*pAggTask == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1; return -1;
@ -432,7 +432,8 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan
*pAggTask = NULL; *pAggTask = NULL;
SArray* pSinkNodeList = taosArrayGetP(pStream->tasks, SINK_NODE_LEVEL); SArray* pSinkNodeList = taosArrayGetP(pStream->tasks, SINK_NODE_LEVEL);
int32_t code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, pEpset, 0, pAggTask); int32_t code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, pEpset, false, pAggTask,
pStream->conf.fillHistory);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
return -1; return -1;
} }
@ -461,7 +462,7 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan
*pHAggTask = NULL; *pHAggTask = NULL;
code = doAddAggTask(pStream->hTaskUid, pHAggTaskList, pHSinkNodeList, pMnode, pStream, pEpset, pStream->conf.fillHistory, code = doAddAggTask(pStream->hTaskUid, pHAggTaskList, pHSinkNodeList, pMnode, pStream, pEpset, pStream->conf.fillHistory,
pHAggTask); pHAggTask, pStream->conf.fillHistory);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
if (pSnode != NULL) { if (pSnode != NULL) {
sdbRelease(pSdb, pSnode); sdbRelease(pSdb, pSnode);
@ -520,8 +521,8 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl
continue; continue;
} }
int32_t code = int32_t code = doAddSourceTask(pSourceTaskList, false, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup, pEpset,
doAddSourceTask(pSourceTaskList, 0, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup, pEpset, nextWindowSkey); nextWindowSkey, pStream->conf.fillHistory);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
sdbRelease(pSdb, pVgroup); sdbRelease(pSdb, pVgroup);
terrno = code; terrno = code;
@ -529,8 +530,8 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl
} }
if (pStream->conf.fillHistory) { if (pStream->conf.fillHistory) {
code = doAddSourceTask(pHSourceTaskList, 1, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup, code = doAddSourceTask(pHSourceTaskList, true, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup, pEpset,
pEpset, nextWindowSkey); nextWindowSkey, pStream->conf.fillHistory);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
sdbRelease(pSdb, pVgroup); sdbRelease(pSdb, pVgroup);
return code; return code;
@ -548,16 +549,16 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl
} }
static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStream, SArray** pCreatedTaskList, static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStream, SArray** pCreatedTaskList,
SEpSet* pEpset, int32_t fillHistory) { SEpSet* pEpset, bool fillHistory) {
SArray* pSinkTaskList = addNewTaskList(pTasksList); SArray* pSinkTaskList = addNewTaskList(pTasksList);
if (pStream->fixedSinkVgId == 0) { if (pStream->fixedSinkVgId == 0) {
if (mndAddShuffleSinkTasksToStream(pMnode, pSinkTaskList, pStream, pEpset, fillHistory) < 0) { if (doAddShuffleSinkTask(pMnode, pSinkTaskList, pStream, pEpset, fillHistory) < 0) {
// TODO free // TODO free
return -1; return -1;
} }
} else { } else {
if (mndAddSinkTaskToStream(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, if (doAddSinkTask(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, pEpset,
pEpset, fillHistory) < 0) { fillHistory) < 0) {
// TODO free // TODO free
return -1; return -1;
} }

View File

@ -44,7 +44,8 @@ typedef struct SNodeEntry {
typedef struct SStreamExecNodeInfo { typedef struct SStreamExecNodeInfo {
SArray *pNodeEntryList; SArray *pNodeEntryList;
int64_t ts; // snapshot ts int64_t ts; // snapshot ts
int64_t activeCheckpoint; // active check point id
SHashObj *pTaskMap; SHashObj *pTaskMap;
SArray *pTaskList; SArray *pTaskList;
TdThreadMutex lock; TdThreadMutex lock;
@ -77,14 +78,18 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in
static int32_t mndProcessNodeCheck(SRpcMsg *pReq); static int32_t mndProcessNodeCheck(SRpcMsg *pReq);
static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg); static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg);
static SArray *doExtractNodeListFromStream(SMnode *pMnode); static SArray *extractNodeListFromStream(SMnode *pMnode);
static SArray *mndTakeVgroupSnapshot(SMnode *pMnode); static SArray *mndTakeVgroupSnapshot(SMnode *pMnode);
static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList); static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList);
static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans);
static STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, const char *name);
static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans);
static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset); static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset);
static int32_t createStreamUpdateTrans(SMnode *pMnode, SStreamObj *pStream, SVgroupChangeInfo *pInfo);
static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode); static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode);
static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode); static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode);
static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot);
int32_t mndInitStream(SMnode *pMnode) { int32_t mndInitStream(SMnode *pMnode) {
SSdbTable table = { SSdbTable table = {
@ -107,6 +112,7 @@ int32_t mndInitStream(SMnode *pMnode) {
mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_RESUME_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_RESUME_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_STOP_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_STOP_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_VND_STREAM_TASK_UPDATE_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_VND_STREAM_TASK_UPDATE_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_VND_STREAM_TASK_RESET_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_VND_STREAM_CHECK_POINT_SOURCE_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_VND_STREAM_CHECK_POINT_SOURCE_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_MND_STREAM_CHECKPOINT_TIMER, mndProcessStreamCheckpointTmr); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_CHECKPOINT_TIMER, mndProcessStreamCheckpointTmr);
@ -579,21 +585,6 @@ int32_t mndPersistDropStreamLog(SMnode *pMnode, STrans *pTrans, SStreamObj *pStr
return 0; return 0;
} }
static int32_t mndSetStreamRecover(SMnode *pMnode, STrans *pTrans, const SStreamObj *pStream) {
SStreamObj streamObj = {0};
memcpy(streamObj.name, pStream->name, TSDB_STREAM_FNAME_LEN);
streamObj.status = STREAM_STATUS__RECOVER;
SSdbRaw *pCommitRaw = mndStreamActionEncode(&streamObj);
if (pCommitRaw == NULL) return -1;
if (mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) {
mError("stream trans:%d, failed to append commit log since %s", pTrans->id, terrstr());
return -1;
}
(void)sdbSetRawStatus(pCommitRaw, SDB_STATUS_READY);
return 0;
}
static int32_t mndCreateStbForStream(SMnode *pMnode, STrans *pTrans, const SStreamObj *pStream, const char *user) { static int32_t mndCreateStbForStream(SMnode *pMnode, STrans *pTrans, const SStreamObj *pStream, const char *user) {
SStbObj *pStb = NULL; SStbObj *pStb = NULL;
SDbObj *pDb = NULL; SDbObj *pDb = NULL;
@ -1147,7 +1138,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) {
execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList);
} }
execNodeList.pNodeEntryList = doExtractNodeListFromStream(pMnode); execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode);
} }
if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) {
@ -1564,7 +1555,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock
} }
pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++);
colDataSetVal(pColInfo, numOfRows, (const char *)&level, false); colDataSetVal(pColInfo, numOfRows, (const char *)level, false);
// status // status
char status[20 + VARSTR_HEADER_SIZE] = {0}; char status[20 + VARSTR_HEADER_SIZE] = {0};
@ -1578,12 +1569,45 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock
const char* pStatus = streamGetTaskStatusStr(pe->status); const char* pStatus = streamGetTaskStatusStr(pe->status);
STR_TO_VARSTR(status, pStatus); STR_TO_VARSTR(status, pStatus);
// status
pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++);
colDataSetVal(pColInfo, numOfRows, (const char *)&status, false); colDataSetVal(pColInfo, numOfRows, (const char *)status, false);
// stage
pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++);
colDataSetVal(pColInfo, numOfRows, (const char *)&pe->stage, false); colDataSetVal(pColInfo, numOfRows, (const char *)&pe->stage, false);
// input queue
char vbuf[30] = {0};
char buf[25] = {0};
const char* queueInfoStr = "%4.2fMiB (%5.2f%)";
sprintf(buf, queueInfoStr, pe->inputQUsed, pe->inputRate);
STR_TO_VARSTR(vbuf, buf);
pColInfo = taosArrayGet(pBlock->pDataBlock, cols++);
colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false);
// output queue
// sprintf(buf, queueInfoStr, pe->outputQUsed, pe->outputRate);
// STR_TO_VARSTR(vbuf, buf);
// pColInfo = taosArrayGet(pBlock->pDataBlock, cols++);
// colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false);
if (pTask->info.taskLevel == TASK_LEVEL__SINK) {
const char* sinkStr = "%.2fMiB";
sprintf(buf, sinkStr, pe->sinkDataSize);
} else if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
// offset info
const char *offsetStr = "%" PRId64 " [%" PRId64 ", %" PRId64 "]";
sprintf(buf, offsetStr, pe->processedVer, pe->verStart, pe->verEnd);
}
STR_TO_VARSTR(vbuf, buf);
pColInfo = taosArrayGet(pBlock->pDataBlock, cols++);
colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false);
numOfRows++; numOfRows++;
} }
} }
@ -1625,7 +1649,9 @@ static int32_t mndPauseStreamTask(STrans *pTrans, SStreamTask *pTask) {
return 0; return 0;
} }
int32_t mndPauseAllStreamTaskImpl(STrans *pTrans, SArray *tasks) { int32_t mndPauseAllStreamTasks(STrans *pTrans, SStreamObj *pStream) {
SArray* tasks = pStream->tasks;
int32_t size = taosArrayGetSize(tasks); int32_t size = taosArrayGetSize(tasks);
for (int32_t i = 0; i < size; i++) { for (int32_t i = 0; i < size; i++) {
SArray *pTasks = taosArrayGetP(tasks, i); SArray *pTasks = taosArrayGetP(tasks, i);
@ -1645,16 +1671,6 @@ int32_t mndPauseAllStreamTaskImpl(STrans *pTrans, SArray *tasks) {
return 0; return 0;
} }
int32_t mndPauseAllStreamTasks(STrans *pTrans, SStreamObj *pStream) {
int32_t code = mndPauseAllStreamTaskImpl(pTrans, pStream->tasks);
if (code != 0) {
return code;
}
// pStream->pHTasksList is null
// code = mndPauseAllStreamTaskImpl(pTrans, pStream->pHTasksList);
return code;
}
static int32_t mndPersistStreamLog(STrans *pTrans, const SStreamObj *pStream, int8_t status) { static int32_t mndPersistStreamLog(STrans *pTrans, const SStreamObj *pStream, int8_t status) {
SStreamObj streamObj = {0}; SStreamObj streamObj = {0};
memcpy(streamObj.name, pStream->name, TSDB_STREAM_FNAME_LEN); memcpy(streamObj.name, pStream->name, TSDB_STREAM_FNAME_LEN);
@ -1708,6 +1724,7 @@ static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) {
sdbRelease(pMnode->pSdb, pStream); sdbRelease(pMnode->pSdb, pStream);
return -1; return -1;
} }
mInfo("trans:%d, used to pause stream:%s", pTrans->id, pauseReq.name); mInfo("trans:%d, used to pause stream:%s", pTrans->id, pauseReq.name);
mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb); mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb);
@ -1719,7 +1736,7 @@ static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) {
// pause all tasks // pause all tasks
if (mndPauseAllStreamTasks(pTrans, pStream) < 0) { if (mndPauseAllStreamTasks(pTrans, pStream) < 0) {
mError("stream:%s, failed to drop task since %s", pauseReq.name, terrstr()); mError("stream:%s, failed to pause task since %s", pauseReq.name, terrstr());
sdbRelease(pMnode->pSdb, pStream); sdbRelease(pMnode->pSdb, pStream);
mndTransDrop(pTrans); mndTransDrop(pTrans);
return -1; return -1;
@ -1946,20 +1963,9 @@ void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_
// todo extract method: traverse stream tasks // todo extract method: traverse stream tasks
// build trans to update the epset // build trans to update the epset
static int32_t createStreamUpdateTrans(SMnode *pMnode, SStreamObj *pStream, SVgroupChangeInfo *pInfo) { static int32_t createStreamUpdateTrans(SMnode *pMnode, SStreamObj *pStream, SVgroupChangeInfo *pInfo) {
STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, TRN_CONFLICT_DB, NULL, "stream-task-update"); STrans* pTrans = doCreateTrans(pMnode, pStream, "stream-task-update");
if (pTrans == NULL) { if (pTrans == NULL) {
mError("failed to build stream task DAG update, reason: %s", tstrerror(TSDB_CODE_OUT_OF_MEMORY)); return terrno;
return -1;
}
mDebug("start to build stream:0x%" PRIx64 " task DAG update", pStream->uid);
mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb);
if (mndTransCheckConflict(pMnode, pTrans) != 0) {
mError("failed to build stream:0x%" PRIx64 " task DAG update, code:%s", pStream->uid,
tstrerror(TSDB_CODE_MND_TRANS_CONFLICT));
mndTransDrop(pTrans);
return -1;
} }
taosWLockLatch(&pStream->lock); taosWLockLatch(&pStream->lock);
@ -2120,7 +2126,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange
return 0; return 0;
} }
static SArray *doExtractNodeListFromStream(SMnode *pMnode) { static SArray *extractNodeListFromStream(SMnode *pMnode) {
SSdb *pSdb = pMnode->pSdb; SSdb *pSdb = pMnode->pSdb;
SStreamObj *pStream = NULL; SStreamObj *pStream = NULL;
void *pIter = NULL; void *pIter = NULL;
@ -2141,11 +2147,9 @@ static SArray *doExtractNodeListFromStream(SMnode *pMnode) {
int32_t numOfTasks = taosArrayGetSize(pLevel); int32_t numOfTasks = taosArrayGetSize(pLevel);
for (int32_t k = 0; k < numOfTasks; ++k) { for (int32_t k = 0; k < numOfTasks; ++k) {
SStreamTask *pTask = taosArrayGetP(pLevel, k); SStreamTask *pTask = taosArrayGetP(pLevel, k);
SNodeEntry entry = {0};
epsetAssign(&entry.epset, &pTask->info.epSet);
entry.nodeId = pTask->info.nodeId;
entry.hbTimestamp = -1;
SNodeEntry entry = {.hbTimestamp = -1, .nodeId = pTask->info.nodeId};
epsetAssign(&entry.epset, &pTask->info.epSet);
taosHashPut(pHash, &entry.nodeId, sizeof(entry.nodeId), &entry, sizeof(entry)); taosHashPut(pHash, &entry.nodeId, sizeof(entry.nodeId), &entry, sizeof(entry));
} }
} }
@ -2202,24 +2206,28 @@ static int32_t doRemoveFromTask(SStreamExecNodeInfo* pExecNode, STaskId* pRemove
return 0; return 0;
} }
static int32_t removeInvalidStreamTask(SArray *pNodeSnapshot) { static bool taskNodeExists(SArray* pList, int32_t nodeId) {
size_t num = taosArrayGetSize(pList);
for(int32_t i = 0; i < num; ++i) {
SNodeEntry* pEntry = taosArrayGet(pList, i);
if (pEntry->nodeId == nodeId) {
return true;
}
}
return false;
}
int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) {
SArray* pRemoveTaskList = taosArrayInit(4, sizeof(STaskId)); SArray* pRemoveTaskList = taosArrayInit(4, sizeof(STaskId));
int32_t numOfTask = taosArrayGetSize(execNodeList.pTaskList); int32_t numOfTask = taosArrayGetSize(execNodeList.pTaskList);
int32_t numOfVgroups = taosArrayGetSize(pNodeSnapshot);
for(int32_t i = 0; i < numOfTask; ++i) { for(int32_t i = 0; i < numOfTask; ++i) {
STaskId* pId = taosArrayGet(execNodeList.pTaskList, i); STaskId* pId = taosArrayGet(execNodeList.pTaskList, i);
STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, pId, sizeof(*pId)); STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, pId, sizeof(*pId));
bool existed = false; bool existed = taskNodeExists(pNodeSnapshot, pEntry->nodeId);
for(int32_t j = 0; j < numOfVgroups; ++j) {
SNodeEntry* pNodeEntry = taosArrayGet(pNodeSnapshot, j);
if (pNodeEntry->nodeId == pEntry->nodeId) {
existed = true;
break;
}
}
if (!existed) { if (!existed) {
taosArrayPush(pRemoveTaskList, pId); taosArrayPush(pRemoveTaskList, pId);
} }
@ -2230,15 +2238,18 @@ static int32_t removeInvalidStreamTask(SArray *pNodeSnapshot) {
doRemoveFromTask(&execNodeList, pId); doRemoveFromTask(&execNodeList, pId);
} }
mDebug("remove invalid stream tasks:%d, remain:%d", (int32_t)taosArrayGetSize(pRemoveTaskList),
(int32_t) taosArrayGetSize(execNodeList.pTaskList));
int32_t size = taosArrayGetSize(pNodeSnapshot); int32_t size = taosArrayGetSize(pNodeSnapshot);
SArray* pValidNodeEntryList = taosArrayInit(4, sizeof(SNodeEntry)); SArray* pValidNodeEntryList = taosArrayInit(4, sizeof(SNodeEntry));
for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) {
SNodeEntry* pExisted = taosArrayGet(execNodeList.pNodeEntryList, i); SNodeEntry* p = taosArrayGet(execNodeList.pNodeEntryList, i);
for(int32_t j = 0; j < size; ++j) { for(int32_t j = 0; j < size; ++j) {
SNodeEntry* pEntry = taosArrayGet(pNodeSnapshot, j); SNodeEntry* pEntry = taosArrayGet(pNodeSnapshot, j);
if (pEntry->nodeId == pExisted->nodeId) { if (pEntry->nodeId == p->nodeId) {
taosArrayPush(pValidNodeEntryList, pExisted); taosArrayPush(pValidNodeEntryList, p);
break; break;
} }
} }
@ -2269,7 +2280,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) {
execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList);
} }
execNodeList.pNodeEntryList = doExtractNodeListFromStream(pMnode); execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode);
} }
if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) {
@ -2282,7 +2293,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) {
SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode);
taosThreadMutexLock(&execNodeList.lock); taosThreadMutexLock(&execNodeList.lock);
removeInvalidStreamTask(pNodeSnapshot); removeExpirednodeEntryAndTask(pNodeSnapshot);
SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot);
if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) {
@ -2310,9 +2321,13 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) {
} }
typedef struct SMStreamNodeCheckMsg { typedef struct SMStreamNodeCheckMsg {
int8_t holder; // // to fix windows compile error, define place holder int8_t placeHolder; // // to fix windows compile error, define place holder
} SMStreamNodeCheckMsg; } SMStreamNodeCheckMsg;
typedef struct SMStreamTaskResetMsg {
int8_t placeHolder;
} SMStreamTaskResetMsg;
static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { static int32_t mndProcessNodeCheck(SRpcMsg *pReq) {
SMnode *pMnode = pReq->info.node; SMnode *pMnode = pReq->info.node;
SSdb *pSdb = pMnode->pSdb; SSdb *pSdb = pMnode->pSdb;
@ -2321,14 +2336,16 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) {
} }
SMStreamNodeCheckMsg *pMsg = rpcMallocCont(sizeof(SMStreamNodeCheckMsg)); SMStreamNodeCheckMsg *pMsg = rpcMallocCont(sizeof(SMStreamNodeCheckMsg));
SRpcMsg rpcMsg = {
.msgType = TDMT_MND_STREAM_NODECHANGE_CHECK, .pCont = pMsg, .contLen = sizeof(SMStreamNodeCheckMsg)}; SRpcMsg rpcMsg = {
.msgType = TDMT_MND_STREAM_NODECHANGE_CHECK, .pCont = pMsg, .contLen = sizeof(SMStreamNodeCheckMsg)};
tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg);
return 0; return 0;
} }
void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) { void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) {
int32_t level = taosArrayGetSize(pStream->tasks); int32_t level = taosArrayGetSize(pStream->tasks);
for (int32_t i = 0; i < level; i++) { for (int32_t i = 0; i < level; i++) {
SArray *pLevel = taosArrayGetP(pStream->tasks, i); SArray *pLevel = taosArrayGetP(pStream->tasks, i);
@ -2339,11 +2356,9 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) {
STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId};
void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id));
if (p == NULL) { if (p == NULL) {
STaskStatusEntry entry = {.id.streamId = pTask->id.streamId, STaskStatusEntry entry = {0};
.id.taskId = pTask->id.taskId, streamTaskStatusInit(&entry, pTask);
.stage = -1,
.nodeId = pTask->info.nodeId,
.status = TASK_STATUS__STOP};
taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry)); taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry));
taosArrayPush(pExecNode->pTaskList, &id); taosArrayPush(pExecNode->pTaskList, &id);
mInfo("s-task:0x%x add into task buffer, total:%d", (int32_t)entry.id.taskId, mInfo("s-task:0x%x add into task buffer, total:%d", (int32_t)entry.id.taskId,
@ -2384,11 +2399,150 @@ void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode
ASSERT(taosHashGetSize(pExecNode->pTaskMap) == taosArrayGetSize(pExecNode->pTaskList)); ASSERT(taosHashGetSize(pExecNode->pTaskMap) == taosArrayGetSize(pExecNode->pTaskList));
} }
// todo: this process should be executed by the write queue worker of the mnode static STrans* doCreateTrans(SMnode* pMnode, SStreamObj* pStream, const char* name) {
STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, TRN_CONFLICT_DB_INSIDE, NULL, name);
if (pTrans == NULL) {
mError("failed to build trans:%s, reason: %s", name, tstrerror(TSDB_CODE_OUT_OF_MEMORY));
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}
mDebug("start to build stream:0x%" PRIx64 " task DAG update", pStream->uid);
mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb);
if (mndTransCheckConflict(pMnode, pTrans) != 0) {
terrno = TSDB_CODE_MND_TRANS_CONFLICT;
mError("failed to build trans:%s for stream:0x%" PRIx64 " code:%s", name, pStream->uid, tstrerror(terrno));
mndTransDrop(pTrans);
return NULL;
}
terrno = 0;
return pTrans;
}
int32_t createStreamResetStatusTrans(SMnode* pMnode, SStreamObj* pStream) {
STrans *pTrans = doCreateTrans(pMnode, pStream, "stream-task-reset");
if (pTrans == NULL) {
return terrno;
}
taosWLockLatch(&pStream->lock);
int32_t numOfLevels = taosArrayGetSize(pStream->tasks);
for (int32_t j = 0; j < numOfLevels; ++j) {
SArray *pLevel = taosArrayGetP(pStream->tasks, j);
int32_t numOfTasks = taosArrayGetSize(pLevel);
for (int32_t k = 0; k < numOfTasks; ++k) {
SStreamTask *pTask = taosArrayGetP(pLevel, k);
// todo extract method, with pause stream task
SVResetStreamTaskReq* pReq = taosMemoryCalloc(1, sizeof(SVResetStreamTaskReq));
if (pReq == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
mError("failed to malloc in reset stream, size:%" PRIzu ", code:%s", sizeof(SVResetStreamTaskReq),
tstrerror(TSDB_CODE_OUT_OF_MEMORY));
return terrno;
}
pReq->head.vgId = htonl(pTask->info.nodeId);
pReq->taskId = pTask->id.taskId;
pReq->streamId = pTask->id.streamId;
STransAction action = {0};
initTransAction(&action, pReq, sizeof(SVResetStreamTaskReq), TDMT_VND_STREAM_TASK_RESET, &pTask->info.epSet);
if (mndTransAppendRedoAction(pTrans, &action) != 0) {
taosMemoryFree(pReq);
taosWUnLockLatch(&pStream->lock);
mndTransDrop(pTrans);
return terrno;
}
}
}
taosWUnLockLatch(&pStream->lock);
int32_t code = mndPersistTransLog(pStream, pTrans);
if (code != TSDB_CODE_SUCCESS) {
sdbRelease(pMnode->pSdb, pStream);
return -1;
}
if (mndTransPrepare(pMnode, pTrans) != 0) {
mError("trans:%d, failed to prepare update stream trans since %s", pTrans->id, terrstr());
sdbRelease(pMnode->pSdb, pStream);
mndTransDrop(pTrans);
return -1;
}
sdbRelease(pMnode->pSdb, pStream);
mndTransDrop(pTrans);
return TSDB_CODE_ACTION_IN_PROGRESS;
}
int32_t mndResetFromCheckpoint(SMnode* pMnode) {
// find the checkpoint trans id
int32_t transId = 0;
{
SSdb *pSdb = pMnode->pSdb;
STrans *pTrans = NULL;
void* pIter = NULL;
while (1) {
pIter = sdbFetch(pSdb, SDB_TRANS, pIter, (void **)&pTrans);
if (pIter == NULL) {
break;
}
if (strncmp(pTrans->opername, MND_STREAM_CHECKPOINT_NAME, tListLen(pTrans->opername) - 1) == 0) {
transId = pTrans->id;
sdbRelease(pSdb, pTrans);
sdbCancelFetch(pSdb, pIter);
break;
}
sdbRelease(pSdb, pTrans);
}
}
if (transId == 0) {
mError("failed to find the checkpoint trans, reset not executed");
return TSDB_CODE_SUCCESS;
}
STrans* pTrans = mndAcquireTrans(pMnode, transId);
mndKillTrans(pMnode, pTrans);
// set all tasks status to be normal, refactor later to be stream level, instead of vnode level.
SSdb *pSdb = pMnode->pSdb;
SStreamObj *pStream = NULL;
void *pIter = NULL;
while (1) {
pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream);
if (pIter == NULL) {
break;
}
mDebug("stream:%s (0x%" PRIx64 ") reset checkpoint procedure, create reset trans", pStream->name, pStream->uid);
int32_t code = createStreamResetStatusTrans(pMnode, pStream);
if (code != TSDB_CODE_SUCCESS) {
sdbCancelFetch(pSdb, pIter);
return code;
}
}
return 0;
}
int32_t mndProcessStreamHb(SRpcMsg *pReq) { int32_t mndProcessStreamHb(SRpcMsg *pReq) {
SMnode *pMnode = pReq->info.node; SMnode *pMnode = pReq->info.node;
SStreamHbMsg req = {0}; SStreamHbMsg req = {0};
bool checkpointFailed = false;
int64_t activeCheckpointId = 0;
SDecoder decoder = {0}; SDecoder decoder = {0};
tDecoderInit(&decoder, pReq->pCont, pReq->contLen); tDecoderInit(&decoder, pReq->pCont, pReq->contLen);
@ -2409,9 +2563,9 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) {
for (int32_t i = 0; i < req.numOfTasks; ++i) { for (int32_t i = 0; i < req.numOfTasks; ++i) {
STaskStatusEntry *p = taosArrayGet(req.pTaskStatus, i); STaskStatusEntry *p = taosArrayGet(req.pTaskStatus, i);
STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, &p->id, sizeof(p->id)); STaskStatusEntry *pEntry = taosHashGet(execNodeList.pTaskMap, &p->id, sizeof(p->id));
if (pEntry == NULL) { if (pEntry == NULL) {
mError("s-task:0x%"PRIx64" not found in mnode task list", p->id.taskId); mError("s-task:0x%" PRIx64 " not found in mnode task list", p->id.taskId);
continue; continue;
} }
@ -2429,7 +2583,18 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) {
} }
} }
} else { } else {
pEntry->stage = p->stage; streamTaskStatusCopy(pEntry, p);
if (p->activeCheckpointId != 0) {
if (activeCheckpointId != 0) {
ASSERT(activeCheckpointId == p->activeCheckpointId);
} else {
activeCheckpointId = p->activeCheckpointId;
}
if (p->checkpointFailed) {
checkpointFailed = p->checkpointFailed;
}
}
} }
pEntry->status = p->status; pEntry->status = p->status;
@ -2438,6 +2603,18 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) {
} }
} }
// current checkpoint is failed, rollback from the checkpoint trans
// kill the checkpoint trans and then set all tasks status to be normal
if (checkpointFailed && activeCheckpointId != 0) {
if (execNodeList.activeCheckpoint != activeCheckpointId) {
mInfo("checkpointId:%"PRId64" failed, issue task-reset trans to reset all tasks status", activeCheckpointId);
execNodeList.activeCheckpoint = activeCheckpointId;
mndResetFromCheckpoint(pMnode);
} else {
mDebug("checkpoint:%"PRId64" reset has issued already, ignore it", activeCheckpointId);
}
}
taosThreadMutexUnlock(&execNodeList.lock); taosThreadMutexUnlock(&execNodeList.lock);
taosArrayDestroy(req.pTaskStatus); taosArrayDestroy(req.pTaskStatus);

View File

@ -1695,7 +1695,6 @@ static int32_t mndRetrieveTrans(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBl
int32_t numOfRows = 0; int32_t numOfRows = 0;
STrans *pTrans = NULL; STrans *pTrans = NULL;
int32_t cols = 0; int32_t cols = 0;
char *pWrite;
while (numOfRows < rows) { while (numOfRows < rows) {
pShow->pIter = sdbFetch(pSdb, SDB_TRANS, pShow->pIter, (void **)&pTrans); pShow->pIter = sdbFetch(pSdb, SDB_TRANS, pShow->pIter, (void **)&pTrans);

View File

@ -53,9 +53,9 @@ FAIL:
taosFreeQitem(pMsg); taosFreeQitem(pMsg);
} }
int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProcessVer) {
ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG && taosArrayGetSize(pTask->pUpstreamInfoList) != 0); ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG && taosArrayGetSize(pTask->upstreamInfo.pList) != 0);
int32_t code = streamTaskInit(pTask, pSnode->pMeta, &pSnode->msgCb, ver); int32_t code = streamTaskInit(pTask, pSnode->pMeta, &pSnode->msgCb, nextProcessVer);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
return code; return code;
} }
@ -70,7 +70,7 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) {
qDebug("s-task:%s state:%p", pTask->id.idStr, pTask->pState); qDebug("s-task:%s state:%p", pTask->id.idStr, pTask->pState);
} }
int32_t numOfChildEp = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t numOfChildEp = taosArrayGetSize(pTask->upstreamInfo.pList);
SReadHandle handle = { .vnode = NULL, .numOfVgroups = numOfChildEp, .pStateBackend = pTask->pState, .fillHistory = pTask->info.fillHistory }; SReadHandle handle = { .vnode = NULL, .numOfVgroups = numOfChildEp, .pStateBackend = pTask->pState, .fillHistory = pTask->info.fillHistory };
initStreamStateAPI(&handle.api); initStreamStateAPI(&handle.api);
@ -203,7 +203,7 @@ int32_t sndProcessTaskRunReq(SSnode *pSnode, SRpcMsg *pMsg) {
SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, pReq->streamId, pReq->taskId); SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, pReq->streamId, pReq->taskId);
if (pTask) { if (pTask) {
streamProcessRunReq(pTask); streamExecTask(pTask);
streamMetaReleaseTask(pSnode->pMeta, pTask); streamMetaReleaseTask(pSnode->pMeta, pTask);
return 0; return 0;
} else { } else {

View File

@ -224,9 +224,10 @@ int tqPushMsg(STQ*, tmsg_t msgType);
int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg);
int tqUnregisterPushHandle(STQ* pTq, void* pHandle); int tqUnregisterPushHandle(STQ* pTq, void* pHandle);
int tqScanWalAsync(STQ* pTq, bool ckPause); int tqScanWalAsync(STQ* pTq, bool ckPause);
int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp);
int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg);
int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg);
int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg);
int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq); int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq);
int tqCommit(STQ*); int tqCommit(STQ*);
@ -246,7 +247,7 @@ int32_t tqProcessVgCommittedInfoReq(STQ* pTq, SRpcMsg* pMsg);
// tq-stream // tq-stream
int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen);
int32_t tqProcessTaskDropReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen);
int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen);
int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen);
int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg);

View File

@ -733,11 +733,11 @@ end:
void freePtr(void* ptr) { taosMemoryFree(*(void**)ptr); } void freePtr(void* ptr) { taosMemoryFree(*(void**)ptr); }
int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) {
int32_t vgId = TD_VID(pTq->pVnode); int32_t vgId = TD_VID(pTq->pVnode);
tqDebug("s-task:0x%x start to expand task", pTask->id.taskId); tqDebug("s-task:0x%x start to expand task", pTask->id.taskId);
int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, ver); int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, nextProcessVer);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
return code; return code;
} }
@ -797,7 +797,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) {
tqDebug("s-task:%s state:%p", pTask->id.idStr, pTask->pState); tqDebug("s-task:%s state:%p", pTask->id.idStr, pTask->pState);
} }
int32_t numOfVgroups = (int32_t)taosArrayGetSize(pTask->pUpstreamInfoList); int32_t numOfVgroups = (int32_t)taosArrayGetSize(pTask->upstreamInfo.pList);
SReadHandle handle = { SReadHandle handle = {
.checkpointId = pTask->chkInfo.checkpointId, .checkpointId = pTask->chkInfo.checkpointId,
.vnode = NULL, .vnode = NULL,
@ -818,27 +818,27 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) {
// sink // sink
if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { if (pTask->outputInfo.type == TASK_OUTPUT__SMA) {
pTask->smaSink.vnode = pTq->pVnode; pTask->outputInfo.smaSink.vnode = pTq->pVnode;
pTask->smaSink.smaSink = smaHandleRes; pTask->outputInfo.smaSink.smaSink = smaHandleRes;
} else if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { } else if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) {
pTask->tbSink.vnode = pTq->pVnode; pTask->outputInfo.tbSink.vnode = pTq->pVnode;
pTask->tbSink.tbSinkFunc = tqSinkDataIntoDstTable; pTask->outputInfo.tbSink.tbSinkFunc = tqSinkDataIntoDstTable;
int32_t ver1 = 1; int32_t ver1 = 1;
SMetaInfo info = {0}; SMetaInfo info = {0};
code = metaGetInfo(pTq->pVnode->pMeta, pTask->tbSink.stbUid, &info, NULL); code = metaGetInfo(pTq->pVnode->pMeta, pTask->outputInfo.tbSink.stbUid, &info, NULL);
if (code == TSDB_CODE_SUCCESS) { if (code == TSDB_CODE_SUCCESS) {
ver1 = info.skmVer; ver1 = info.skmVer;
} }
SSchemaWrapper* pschemaWrapper = pTask->tbSink.pSchemaWrapper; SSchemaWrapper* pschemaWrapper = pTask->outputInfo.tbSink.pSchemaWrapper;
pTask->tbSink.pTSchema = tBuildTSchema(pschemaWrapper->pSchema, pschemaWrapper->nCols, ver1); pTask->outputInfo.tbSink.pTSchema = tBuildTSchema(pschemaWrapper->pSchema, pschemaWrapper->nCols, ver1);
if (pTask->tbSink.pTSchema == NULL) { if (pTask->outputInfo.tbSink.pTSchema == NULL) {
return -1; return -1;
} }
pTask->tbSink.pTblInfo = tSimpleHashInit(10240, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT)); pTask->outputInfo.tbSink.pTblInfo = tSimpleHashInit(10240, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT));
tSimpleHashSetFreeFp(pTask->tbSink.pTblInfo, freePtr); tSimpleHashSetFreeFp(pTask->outputInfo.tbSink.pTblInfo, freePtr);
} }
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
@ -973,18 +973,17 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms
if (tsDisableStream) { if (tsDisableStream) {
tqInfo("vgId:%d stream disabled, not deploy stream tasks", vgId); tqInfo("vgId:%d stream disabled, not deploy stream tasks", vgId);
return 0; return code;
} }
tqDebug("vgId:%d receive new stream task deploy msg, start to build stream task", vgId); tqDebug("vgId:%d receive new stream task deploy msg, start to build stream task", vgId);
// 1.deserialize msg and build task // 1.deserialize msg and build task
SStreamTask* pTask = taosMemoryCalloc(1, sizeof(SStreamTask)); int32_t size = sizeof(SStreamTask);
SStreamTask* pTask = taosMemoryCalloc(1, size);
if (pTask == NULL) { if (pTask == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; tqError("vgId:%d failed to create stream task due to out of memory, alloc size:%d", vgId, size);
tqError("vgId:%d failed to create stream task due to out of memory, alloc size:%d", vgId, return TSDB_CODE_OUT_OF_MEMORY;
(int32_t)sizeof(SStreamTask));
return -1;
} }
SDecoder decoder; SDecoder decoder;
@ -992,9 +991,9 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms
code = tDecodeStreamTask(&decoder, pTask); code = tDecodeStreamTask(&decoder, pTask);
tDecoderClear(&decoder); tDecoderClear(&decoder);
if (code < 0) { if (code != TSDB_CODE_SUCCESS) {
taosMemoryFree(pTask); taosMemoryFree(pTask);
return -1; return TSDB_CODE_INVALID_MSG;
} }
SStreamMeta* pStreamMeta = pTq->pStreamMeta; SStreamMeta* pStreamMeta = pTq->pStreamMeta;
@ -1010,9 +1009,9 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms
taosWUnLockLatch(&pStreamMeta->lock); taosWUnLockLatch(&pStreamMeta->lock);
if (code < 0) { if (code < 0) {
tqError("vgId:%d failed to add s-task:0x%x, total:%d, code:%s", vgId, taskId, numOfTasks, tstrerror(code)); tqError("failed to add s-task:0x%x into vgId:%d meta, total:%d, code:%s", vgId, taskId, numOfTasks, tstrerror(code));
tFreeStreamTask(pTask); tFreeStreamTask(pTask);
return -1; return code;
} }
// added into meta store, pTask cannot be reference since it may have been destroyed by other threads already now if // added into meta store, pTask cannot be reference since it may have been destroyed by other threads already now if
@ -1044,17 +1043,18 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms
tFreeStreamTask(pTask); tFreeStreamTask(pTask);
} }
return 0; return code;
} }
// this function should be executed by only one thread
int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
SStreamScanHistoryReq* pReq = (SStreamScanHistoryReq*)pMsg->pCont; SStreamScanHistoryReq* pReq = (SStreamScanHistoryReq*)pMsg->pCont;
SStreamMeta* pMeta = pTq->pStreamMeta; SStreamMeta* pMeta = pTq->pStreamMeta;
int32_t code = TSDB_CODE_SUCCESS;
int32_t code = TSDB_CODE_SUCCESS;
SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId);
if (pTask == NULL) { if (pTask == NULL) {
tqError("vgId:%d failed to acquire stream task:0x%x during stream recover, task may have been destroyed", tqError("vgId:%d failed to acquire stream task:0x%x during scan history data, task may have been destroyed",
pMeta->vgId, pReq->taskId); pMeta->vgId, pReq->taskId);
return -1; return -1;
} }
@ -1062,25 +1062,47 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
// do recovery step1 // do recovery step1
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus);
tqDebug("s-task:%s start scan-history stage(step 1), status:%s", id, pStatus);
// avoid multi-thread exec
while(1) {
int32_t sentinel = atomic_val_compare_exchange_32(&pTask->status.inScanHistorySentinel, 0, 1);
if (sentinel != 0) {
tqDebug("s-task:%s already in scan-history func, wait for 100ms, and try again", id);
taosMsleep(100);
} else {
break;
}
}
if (pTask->execInfo.step1Start == 0) { if (pTask->execInfo.step1Start == 0) {
ASSERT(pTask->status.pauseAllowed == false); ASSERT(pTask->status.pauseAllowed == false);
pTask->execInfo.step1Start = taosGetTimestampMs(); int64_t ts = taosGetTimestampMs();
pTask->execInfo.step1Start = ts;
tqDebug("s-task:%s start scan-history stage(step 1), status:%s, step1 startTs:%" PRId64, id, pStatus, ts);
// NOTE: in case of stream task, scan-history data in wal is not allowed to pause
if (pTask->info.fillHistory == 1) { if (pTask->info.fillHistory == 1) {
streamTaskEnablePause(pTask); streamTaskEnablePause(pTask);
} }
} else { } else {
tqDebug("s-task:%s resume from paused, start ts:%" PRId64, pTask->id.idStr, pTask->execInfo.step1Start); if (pTask->execInfo.step2Start == 0) {
tqDebug("s-task:%s resume from paused, original step1 startTs:%" PRId64, id, pTask->execInfo.step1Start);
} else {
tqDebug("s-task:%s already in step2, no need to scan-history data, step2 starTs:%"PRId64, id, pTask->execInfo.step2Start);
atomic_store_32(&pTask->status.inScanHistorySentinel, 0);
streamMetaReleaseTask(pMeta, pTask);
return 0;
}
} }
// we have to continue retrying to successfully execute the scan history task. // we have to continue retrying to successfully execute the scan history task.
int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); if (!streamTaskSetSchedStatusWait(pTask)) {
if (schedStatus != TASK_SCHED_STATUS__INACTIVE) {
tqError( tqError(
"s-task:%s failed to start scan-history in first stream time window since already started, unexpected " "s-task:%s failed to start scan-history in first stream time window since already started, unexpected "
"sched-status:%d", "sched-status:%d",
id, schedStatus); id, pTask->status.schedStatus);
atomic_store_32(&pTask->status.inScanHistorySentinel, 0);
streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pTask);
return 0; return 0;
} }
@ -1090,16 +1112,18 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
} }
streamScanHistoryData(pTask); streamScanHistoryData(pTask);
double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0;
if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { if (pTask->status.taskStatus == TASK_STATUS__PAUSE) {
double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0;
int8_t status = streamTaskSetSchedStatusInActive(pTask); int8_t status = streamTaskSetSchedStatusInActive(pTask);
tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, status); tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, status);
atomic_store_32(&pTask->status.inScanHistorySentinel, 0);
streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pTask);
return 0; return 0;
} }
// the following procedure should be executed, no matter status is stop/pause or not // the following procedure should be executed, no matter status is stop/pause or not
double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0;
tqDebug("s-task:%s scan-history stage(step 1) ended, elapsed time:%.2fs", id, el); tqDebug("s-task:%s scan-history stage(step 1) ended, elapsed time:%.2fs", id, el);
if (pTask->info.fillHistory) { if (pTask->info.fillHistory) {
@ -1117,6 +1141,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
tqDebug("s-task:%s fill-history task set status to be dropping", id); tqDebug("s-task:%s fill-history task set status to be dropping", id);
streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id);
atomic_store_32(&pTask->status.inScanHistorySentinel, 0);
streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pTask);
return -1; return -1;
} }
@ -1133,14 +1159,13 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
} }
// now we can stop the stream task execution // now we can stop the stream task execution
int64_t latestVer = 0; int64_t nextProcessedVer = 0;
while (1) { while (1) {
taosThreadMutexLock(&pStreamTask->lock); taosThreadMutexLock(&pStreamTask->lock);
int8_t status = pStreamTask->status.taskStatus; int8_t status = pStreamTask->status.taskStatus;
if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) {
// return; // return; do nothing
// do nothing
} }
if (status == TASK_STATUS__HALT) { if (status == TASK_STATUS__HALT) {
@ -1171,9 +1196,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
pStreamTask->status.keepTaskStatus = status; pStreamTask->status.keepTaskStatus = status;
pStreamTask->status.taskStatus = TASK_STATUS__HALT; pStreamTask->status.taskStatus = TASK_STATUS__HALT;
tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, nextProcessedVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader);
pStreamTask->info.taskLevel, pStreamTask->status.schedStatus, id);
latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); tqDebug("s-task:%s level:%d nextProcessedVer:%" PRId64 ", sched-status:%d is halt by fill-history task:%s",
pStreamTask->id.idStr, pStreamTask->info.taskLevel, nextProcessedVer, pStreamTask->status.schedStatus,
id);
taosThreadMutexUnlock(&pStreamTask->lock); taosThreadMutexUnlock(&pStreamTask->lock);
break; break;
@ -1181,10 +1208,10 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
// if it's an source task, extract the last version in wal. // if it's an source task, extract the last version in wal.
pRange = &pTask->dataRange.range; pRange = &pTask->dataRange.range;
done = streamHistoryTaskSetVerRangeStep2(pTask, latestVer); done = streamHistoryTaskSetVerRangeStep2(pTask, nextProcessedVer);
pTask->execInfo.step2Start = taosGetTimestampMs();
if (done) { if (done) {
pTask->execInfo.step2Start = taosGetTimestampMs();
qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0); qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0);
streamTaskPutTranstateIntoInputQ(pTask); streamTaskPutTranstateIntoInputQ(pTask);
@ -1194,7 +1221,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus)); streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus));
} }
streamTryExec(pTask); // exec directly streamExecTask(pTask); // exec directly
} else { } else {
STimeWindow* pWindow = &pTask->dataRange.window; STimeWindow* pWindow = &pTask->dataRange.window;
tqDebug("s-task:%s level:%d verRange:%" PRId64 " - %" PRId64 " window:%" PRId64 "-%" PRId64 tqDebug("s-task:%s level:%d verRange:%" PRId64 " - %" PRId64 " window:%" PRId64 "-%" PRId64
@ -1203,27 +1230,26 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
pStreamTask->id.idStr); pStreamTask->id.idStr);
ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING);
pTask->execInfo.step2Start = taosGetTimestampMs();
streamSetParamForStreamScannerStep2(pTask, pRange, pWindow); streamSetParamForStreamScannerStep2(pTask, pRange, pWindow);
int64_t dstVer = pTask->dataRange.range.minVer; int64_t dstVer = pTask->dataRange.range.minVer;
pTask->chkInfo.nextProcessVer = dstVer; pTask->chkInfo.nextProcessVer = dstVer;
walReaderSetSkipToVersion(pTask->exec.pWalReader, dstVer); walReaderSetSkipToVersion(pTask->exec.pWalReader, dstVer);
tqDebug("s-task:%s wal reader start scan WAL verRange:%" PRId64 "-%" PRId64 ", set sched-status:%d", id, dstVer, tqDebug("s-task:%s wal reader start scan WAL verRange:%" PRId64 "-%" PRId64 ", set sched-status:%d", id, dstVer,
pTask->dataRange.range.maxVer, TASK_SCHED_STATUS__INACTIVE); pTask->dataRange.range.maxVer, TASK_SCHED_STATUS__INACTIVE);
/*int8_t status = */streamTaskSetSchedStatusInActive(pTask); /*int8_t status = */streamTaskSetSchedStatusInActive(pTask);
// set the fill-history task to be normal // the fill-history task starts to process data in wal, let's set it status to be normal now
if (pTask->info.fillHistory == 1 && !streamTaskShouldStop(&pTask->status)) { if (pTask->info.fillHistory == 1 && !streamTaskShouldStop(&pTask->status)) {
streamSetStatusNormal(pTask); streamSetStatusNormal(pTask);
} }
tqScanWalAsync(pTq, false); tqScanWalAsync(pTq, false);
} }
streamMetaReleaseTask(pMeta, pTask);
streamMetaReleaseTask(pMeta, pStreamTask); streamMetaReleaseTask(pMeta, pStreamTask);
} else { } else {
STimeWindow* pWindow = &pTask->dataRange.window; STimeWindow* pWindow = &pTask->dataRange.window;
@ -1244,14 +1270,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
} }
code = streamTaskScanHistoryDataComplete(pTask); code = streamTaskScanHistoryDataComplete(pTask);
streamMetaReleaseTask(pMeta, pTask);
// when all source task complete to scan history data in stream time window, they are allowed to handle stream data
// at the same time.
return code;
} }
return 0; atomic_store_32(&pTask->status.inScanHistorySentinel, 0);
streamMetaReleaseTask(pMeta, pTask);
return code;
} }
// only the agg tasks and the sink tasks will receive this message from upstream tasks // only the agg tasks and the sink tasks will receive this message from upstream tasks
@ -1339,7 +1362,7 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) {
if (st == TASK_STATUS__NORMAL || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) { if (st == TASK_STATUS__NORMAL || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) {
tqDebug("vgId:%d s-task:%s start to process block from inputQ, next checked ver:%" PRId64, vgId, pTask->id.idStr, tqDebug("vgId:%d s-task:%s start to process block from inputQ, next checked ver:%" PRId64, vgId, pTask->id.idStr,
pTask->chkInfo.nextProcessVer); pTask->chkInfo.nextProcessVer);
streamProcessRunReq(pTask); streamExecTask(pTask);
} else { } else {
int8_t status = streamTaskSetSchedStatusInActive(pTask); int8_t status = streamTaskSetSchedStatusInActive(pTask);
tqDebug("vgId:%d s-task:%s ignore run req since not in ready state, status:%s, sched-status:%d", vgId, tqDebug("vgId:%d s-task:%s ignore run req since not in ready state, status:%s, sched-status:%d", vgId,
@ -1407,20 +1430,36 @@ int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg) {
} }
} }
int32_t tqProcessTaskDropReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen) {
SVDropStreamTaskReq* pReq = (SVDropStreamTaskReq*)msg; SVDropStreamTaskReq* pReq = (SVDropStreamTaskReq*)msg;
tqDebug("vgId:%d receive msg to drop stream task:0x%x", TD_VID(pTq->pVnode), pReq->taskId);
streamMetaUnregisterTask(pTq->pStreamMeta, pReq->streamId, pReq->taskId); int32_t vgId = TD_VID(pTq->pVnode);
SStreamMeta* pMeta = pTq->pStreamMeta;
tqDebug("vgId:%d receive msg to drop s-task:0x%x", vgId, pReq->taskId);
SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId);
if (pTask != NULL) {
// drop the related fill-history task firstly
if (pTask->hTaskInfo.id.taskId != 0) {
STaskId* pHTaskId = &pTask->hTaskInfo.id;
streamMetaUnregisterTask(pMeta, pHTaskId->streamId, pHTaskId->taskId);
tqDebug("vgId:%d drop fill-history task:0x%x dropped firstly", vgId, (int32_t)pHTaskId->taskId);
}
streamMetaReleaseTask(pMeta, pTask);
}
// drop the stream task now
streamMetaUnregisterTask(pMeta, pReq->streamId, pReq->taskId);
// commit the update // commit the update
taosWLockLatch(&pTq->pStreamMeta->lock); taosWLockLatch(&pMeta->lock);
int32_t numOfTasks = streamMetaGetNumOfTasks(pTq->pStreamMeta); int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta);
tqDebug("vgId:%d task:0x%x dropped, remain tasks:%d", TD_VID(pTq->pVnode), pReq->taskId, numOfTasks); tqDebug("vgId:%d task:0x%x dropped, remain tasks:%d", vgId, pReq->taskId, numOfTasks);
if (streamMetaCommit(pTq->pStreamMeta) < 0) { if (streamMetaCommit(pMeta) < 0) {
// persist to disk // persist to disk
} }
taosWUnLockLatch(&pTq->pStreamMeta->lock); taosWUnLockLatch(&pMeta->lock);
return 0; return 0;
} }
@ -1495,7 +1534,7 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion,
if (level == TASK_LEVEL__SOURCE && pTask->info.fillHistory && if (level == TASK_LEVEL__SOURCE && pTask->info.fillHistory &&
pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) {
streamStartScanHistoryAsync(pTask, igUntreated); streamStartScanHistoryAsync(pTask, igUntreated);
} else if (level == TASK_LEVEL__SOURCE && (taosQueueItemSize(pTask->inputInfo.queue->pQueue) == 0)) { } else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0)) {
tqScanWalAsync(pTq, false); tqScanWalAsync(pTq, false);
} else { } else {
streamSchedExec(pTask); streamSchedExec(pTask);
@ -1508,14 +1547,15 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion,
int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) {
SVResumeStreamTaskReq* pReq = (SVResumeStreamTaskReq*)msg; SVResumeStreamTaskReq* pReq = (SVResumeStreamTaskReq*)msg;
SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->streamId, pReq->taskId);
int32_t code = tqProcessTaskResumeImpl(pTq, pTask, sversion, pReq->igUntreated); SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->streamId, pReq->taskId);
int32_t code = tqProcessTaskResumeImpl(pTq, pTask, sversion, pReq->igUntreated);
if (code != 0) { if (code != 0) {
return code; return code;
} }
SStreamTask* pHistoryTask = STaskId* pHTaskId = &pTask->hTaskInfo.id;
streamMetaAcquireTask(pTq->pStreamMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId); SStreamTask* pHistoryTask = streamMetaAcquireTask(pTq->pStreamMeta, pHTaskId->streamId, pHTaskId->taskId);
if (pHistoryTask) { if (pHistoryTask) {
code = tqProcessTaskResumeImpl(pTq, pHistoryTask, sversion, pReq->igUntreated); code = tqProcessTaskResumeImpl(pTq, pHistoryTask, sversion, pReq->igUntreated);
} }
@ -1534,9 +1574,11 @@ int32_t tqProcessTaskRetrieveReq(STQ* pTq, SRpcMsg* pMsg) {
tDecodeStreamRetrieveReq(&decoder, &req); tDecodeStreamRetrieveReq(&decoder, &req);
tDecoderClear(&decoder); tDecoderClear(&decoder);
int32_t vgId = pTq->pStreamMeta->vgId;
SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, req.dstTaskId); SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, req.dstTaskId);
if (pTask == NULL) { if (pTask == NULL) {
// tDeleteStreamDispatchReq(&req); tqError("vgId:%d process retrieve req, failed to acquire task:0x%x, it may have been dropped already", vgId,
req.dstTaskId);
return -1; return -1;
} }
@ -1636,16 +1678,17 @@ FAIL:
} }
// todo error code cannot be return, since this is invoked by an mnode-launched transaction. // todo error code cannot be return, since this is invoked by an mnode-launched transaction.
int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) { int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) {
int32_t vgId = TD_VID(pTq->pVnode); int32_t vgId = TD_VID(pTq->pVnode);
SStreamMeta* pMeta = pTq->pStreamMeta; SStreamMeta* pMeta = pTq->pStreamMeta;
char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead));
int32_t len = pMsg->contLen - sizeof(SMsgHead); int32_t len = pMsg->contLen - sizeof(SMsgHead);
int32_t code = 0; int32_t code = 0;
// disable auto rsp to source // disable auto rsp to mnode
pRsp->info.handle = NULL; pRsp->info.handle = NULL;
// todo: add counter to make sure other tasks would not be trapped in checkpoint state
SStreamCheckpointSourceReq req = {0}; SStreamCheckpointSourceReq req = {0};
if (!vnodeIsRoleLeader(pTq->pVnode)) { if (!vnodeIsRoleLeader(pTq->pVnode)) {
tqDebug("vgId:%d not leader, ignore checkpoint-source msg", vgId); tqDebug("vgId:%d not leader, ignore checkpoint-source msg", vgId);
@ -1655,6 +1698,14 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
if (!pTq->pVnode->restored) {
tqDebug("vgId:%d checkpoint-source msg received during restoring, ignore it", vgId);
SRpcMsg rsp = {0};
buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0);
tmsgSendRsp(&rsp); // error occurs
return TSDB_CODE_SUCCESS;
}
SDecoder decoder; SDecoder decoder;
tDecoderInit(&decoder, (uint8_t*)msg, len); tDecoderInit(&decoder, (uint8_t*)msg, len);
if (tDecodeStreamCheckpointSourceReq(&decoder, &req) < 0) { if (tDecodeStreamCheckpointSourceReq(&decoder, &req) < 0) {
@ -1680,6 +1731,9 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs
// downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req. // downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req.
if (pTask->status.downstreamReady != 1) { if (pTask->status.downstreamReady != 1) {
pTask->chkInfo.failedId = req.checkpointId; // record the latest failed checkpoint id
pTask->checkpointingId = req.checkpointId;
qError("s-task:%s not ready for checkpoint, since downstream not ready, ignore this checkpoint:%" PRId64 qError("s-task:%s not ready for checkpoint, since downstream not ready, ignore this checkpoint:%" PRId64
", set it failure", pTask->id.idStr, req.checkpointId); ", set it failure", pTask->id.idStr, req.checkpointId);
streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pTask);
@ -1729,8 +1783,6 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs
return code; return code;
} }
// todo: when generating checkpoint, no new tasks are allowed to add into current Vnode
// todo: when generating checkpoint, leader of mnode has transfer to other DNode?
streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pTask);
return code; return code;
} }
@ -1896,3 +1948,27 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) {
taosArrayDestroy(req.pNodeList); taosArrayDestroy(req.pNodeList);
return rsp.code; return rsp.code;
} }
int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg) {
SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*) pMsg->pCont;
SStreamMeta* pMeta = pTq->pStreamMeta;
SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId);
if (pTask == NULL) {
tqError("vgId:%d process task-reset req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId,
pReq->taskId);
return TSDB_CODE_SUCCESS;
}
tqDebug("s-task:%s receive task-reset msg from mnode, reset status and ready for data processing", pTask->id.idStr);
// clear flag set during do checkpoint, and open inputQ for all upstream tasks
if (pTask->status.taskStatus == TASK_STATUS__CK) {
streamTaskClearCheckInfo(pTask);
taosArrayClear(pTask->pReadyMsgList);
streamSetStatusNormal(pTask);
}
streamMetaReleaseTask(pMeta, pTask);
return TSDB_CODE_SUCCESS;
}

View File

@ -309,7 +309,7 @@ int32_t extractMsgFromWal(SWalReader* pReader, void** pItem, int64_t maxVer, con
SWalCont* pCont = &pReader->pHead->head; SWalCont* pCont = &pReader->pHead->head;
int64_t ver = pCont->version; int64_t ver = pCont->version;
if (ver > maxVer) { if (ver > maxVer) {
tqDebug("maxVer in WAL:%" PRId64 " reached current:%" PRId64 ", do not scan wal anymore, %s", maxVer, ver, id); tqDebug("maxVer in WAL:%" PRId64 " reached, current:%" PRId64 ", do not scan wal anymore, %s", maxVer, ver, id);
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }

View File

@ -24,6 +24,7 @@ typedef struct STableSinkInfo {
tstr name; tstr name;
} STableSinkInfo; } STableSinkInfo;
static bool hasOnlySubmitData(const SArray* pBlocks, int32_t numOfBlocks);
static int32_t tsAscendingSortFn(const void* p1, const void* p2); static int32_t tsAscendingSortFn(const void* p1, const void* p2);
static int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDataBlock, char* stbFullName, static int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDataBlock, char* stbFullName,
SSubmitTbData* pTableData); SSubmitTbData* pTableData);
@ -141,7 +142,7 @@ static int32_t doBuildAndSendCreateTableMsg(SVnode* pVnode, char* stbFullName, S
int64_t suid) { int64_t suid) {
tqDebug("s-task:%s build create table msg", pTask->id.idStr); tqDebug("s-task:%s build create table msg", pTask->id.idStr);
STSchema* pTSchema = pTask->tbSink.pTSchema; STSchema* pTSchema = pTask->outputInfo.tbSink.pTSchema;
int32_t rows = pDataBlock->info.rows; int32_t rows = pDataBlock->info.rows;
SArray* tagArray = NULL; SArray* tagArray = NULL;
int32_t code = 0; int32_t code = 0;
@ -587,7 +588,7 @@ int32_t doConvertRows(SSubmitTbData* pTableData, STSchema* pTSchema, SSDataBlock
int32_t doWaitForDstTableCreated(SVnode* pVnode, SStreamTask* pTask, STableSinkInfo* pTableSinkInfo, int32_t doWaitForDstTableCreated(SVnode* pVnode, SStreamTask* pTask, STableSinkInfo* pTableSinkInfo,
const char* dstTableName, int64_t* uid) { const char* dstTableName, int64_t* uid) {
int32_t vgId = TD_VID(pVnode); int32_t vgId = TD_VID(pVnode);
int64_t suid = pTask->tbSink.stbUid; int64_t suid = pTask->outputInfo.tbSink.stbUid;
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
while (pTableSinkInfo->uid == 0) { while (pTableSinkInfo->uid == 0) {
@ -630,12 +631,12 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat
char* dstTableName = pDataBlock->info.parTbName; char* dstTableName = pDataBlock->info.parTbName;
int32_t numOfRows = pDataBlock->info.rows; int32_t numOfRows = pDataBlock->info.rows;
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
int64_t suid = pTask->tbSink.stbUid; int64_t suid = pTask->outputInfo.tbSink.stbUid;
STSchema* pTSchema = pTask->tbSink.pTSchema; STSchema* pTSchema = pTask->outputInfo.tbSink.pTSchema;
int32_t vgId = TD_VID(pVnode); int32_t vgId = TD_VID(pVnode);
STableSinkInfo* pTableSinkInfo = NULL; STableSinkInfo* pTableSinkInfo = NULL;
bool alreadyCached = tqGetTableInfo(pTask->tbSink.pTblInfo, groupId, &pTableSinkInfo); bool alreadyCached = tqGetTableInfo(pTask->outputInfo.tbSink.pTblInfo, groupId, &pTableSinkInfo);
if (alreadyCached) { if (alreadyCached) {
if (dstTableName[0] == 0) { // data block does not set the destination table name if (dstTableName[0] == 0) { // data block does not set the destination table name
@ -701,7 +702,7 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat
} }
pTableSinkInfo->uid = 0; pTableSinkInfo->uid = 0;
doPutIntoCache(pTask->tbSink.pTblInfo, pTableSinkInfo, groupId, id); doPutIntoCache(pTask->outputInfo.tbSink.pTblInfo, pTableSinkInfo, groupId, id);
} else { } else {
bool isValid = isValidDstChildTable(&mr, vgId, dstTableName, suid); bool isValid = isValidDstChildTable(&mr, vgId, dstTableName, suid);
if (!isValid) { if (!isValid) {
@ -715,7 +716,7 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat
pTableSinkInfo->uid = mr.me.uid; pTableSinkInfo->uid = mr.me.uid;
metaReaderClear(&mr); metaReaderClear(&mr);
doPutIntoCache(pTask->tbSink.pTblInfo, pTableSinkInfo, groupId, id); doPutIntoCache(pTask->outputInfo.tbSink.pTblInfo, pTableSinkInfo, groupId, id);
} }
} }
} }
@ -729,11 +730,11 @@ int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlo
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
tqDebug("s-task:%s sink data pipeline, build submit msg from %dth resBlock, including %d rows, dst suid:%" PRId64, tqDebug("s-task:%s sink data pipeline, build submit msg from %dth resBlock, including %d rows, dst suid:%" PRId64,
id, blockIndex + 1, numOfRows, pTask->tbSink.stbUid); id, blockIndex + 1, numOfRows, pTask->outputInfo.tbSink.stbUid);
char* dstTableName = pDataBlock->info.parTbName; char* dstTableName = pDataBlock->info.parTbName;
// convert all rows // convert all rows
int32_t code = doConvertRows(pTableData, pTask->tbSink.pTSchema, pDataBlock, id); int32_t code = doConvertRows(pTableData, pTask->outputInfo.tbSink.pTSchema, pDataBlock, id);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
tqError("s-task:%s failed to convert rows from result block, code:%s", id, tstrerror(terrno)); tqError("s-task:%s failed to convert rows from result block, code:%s", id, tstrerror(terrno));
return code; return code;
@ -744,30 +745,29 @@ int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlo
return code; return code;
} }
bool hasOnlySubmitData(const SArray* pBlocks, int32_t numOfBlocks) {
for(int32_t i = 0; i < numOfBlocks; ++i) {
SSDataBlock* p = taosArrayGet(pBlocks, i);
if (p->info.type == STREAM_DELETE_RESULT || p->info.type == STREAM_CREATE_CHILD_TABLE) {
return false;
}
}
return true;
}
void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) {
const SArray* pBlocks = (const SArray*)data; const SArray* pBlocks = (const SArray*)data;
SVnode* pVnode = (SVnode*)vnode; SVnode* pVnode = (SVnode*)vnode;
int64_t suid = pTask->tbSink.stbUid; int64_t suid = pTask->outputInfo.tbSink.stbUid;
char* stbFullName = pTask->tbSink.stbFullName; char* stbFullName = pTask->outputInfo.tbSink.stbFullName;
STSchema* pTSchema = pTask->tbSink.pTSchema; STSchema* pTSchema = pTask->outputInfo.tbSink.pTSchema;
int32_t vgId = TD_VID(pVnode); int32_t vgId = TD_VID(pVnode);
int32_t numOfBlocks = taosArrayGetSize(pBlocks); int32_t numOfBlocks = taosArrayGetSize(pBlocks);
int32_t code = TSDB_CODE_SUCCESS; int32_t code = TSDB_CODE_SUCCESS;
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
if (pTask->execInfo.start == 0) { bool onlySubmitData = hasOnlySubmitData(pBlocks, numOfBlocks);
pTask->execInfo.start = taosGetTimestampMs();
}
bool onlySubmitData = true;
for(int32_t i = 0; i < numOfBlocks; ++i) {
SSDataBlock* p = taosArrayGet(pBlocks, i);
if (p->info.type == STREAM_DELETE_RESULT || p->info.type == STREAM_CREATE_CHILD_TABLE) {
onlySubmitData = false;
break;
}
}
if (!onlySubmitData) { if (!onlySubmitData) {
tqDebug("vgId:%d, s-task:%s write %d stream resBlock(s) into table, has delete block, submit one-by-one", vgId, id, tqDebug("vgId:%d, s-task:%s write %d stream resBlock(s) into table, has delete block, submit one-by-one", vgId, id,
numOfBlocks); numOfBlocks);

View File

@ -17,10 +17,11 @@
#include "vnd.h" #include "vnd.h"
#define MAX_REPEAT_SCAN_THRESHOLD 3 #define MAX_REPEAT_SCAN_THRESHOLD 3
#define SCAN_WAL_IDLE_DURATION 100
static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle); static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle);
static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId);
static void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); static bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver);
// extract data blocks(submit/delete) from WAL, and add them into the input queue for all the sources tasks. // extract data blocks(submit/delete) from WAL, and add them into the input queue for all the sources tasks.
int32_t tqScanWal(STQ* pTq) { int32_t tqScanWal(STQ* pTq) {
@ -36,12 +37,10 @@ int32_t tqScanWal(STQ* pTq) {
bool shouldIdle = true; bool shouldIdle = true;
doScanWalForAllTasks(pTq->pStreamMeta, &shouldIdle); doScanWalForAllTasks(pTq->pStreamMeta, &shouldIdle);
int32_t times = 0;
if (shouldIdle) { if (shouldIdle) {
taosWLockLatch(&pMeta->lock); taosWLockLatch(&pMeta->lock);
times = (--pMeta->walScanCounter); int32_t times = (--pMeta->walScanCounter);
ASSERT(pMeta->walScanCounter >= 0); ASSERT(pMeta->walScanCounter >= 0);
if (pMeta->walScanCounter <= 0) { if (pMeta->walScanCounter <= 0) {
@ -50,8 +49,10 @@ int32_t tqScanWal(STQ* pTq) {
} }
taosWUnLockLatch(&pMeta->lock); taosWUnLockLatch(&pMeta->lock);
tqDebug("vgId:%d scan wal for stream tasks for %d times", vgId, times); tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION);
} }
taosMsleep(SCAN_WAL_IDLE_DURATION);
} }
int64_t el = (taosGetTimestampMs() - st); int64_t el = (taosGetTimestampMs() - st);
@ -73,7 +74,7 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) {
taosWLockLatch(&pMeta->lock); taosWLockLatch(&pMeta->lock);
pTaskList = taosArrayDup(pMeta->pTaskList, NULL); pTaskList = taosArrayDup(pMeta->pTaskList, NULL);
taosHashClear(pMeta->startInfo.pReadyTaskSet); taosHashClear(pMeta->startInfo.pReadyTaskSet);
pMeta->startInfo.ts = taosGetTimestampMs(); pMeta->startInfo.startTs = taosGetTimestampMs();
taosWUnLockLatch(&pMeta->lock); taosWUnLockLatch(&pMeta->lock);
// broadcast the check downstream tasks msg // broadcast the check downstream tasks msg
@ -297,7 +298,7 @@ int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId) {
} }
// todo handle memory error // todo handle memory error
void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) {
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
int64_t maxVer = pTask->dataRange.range.maxVer; int64_t maxVer = pTask->dataRange.range.maxVer;
@ -310,12 +311,95 @@ void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) {
double el = (taosGetTimestampMs() - pTask->execInfo.step2Start) / 1000.0; double el = (taosGetTimestampMs() - pTask->execInfo.step2Start) / 1000.0;
qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, el); qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, el);
/*int32_t code = */streamTaskPutTranstateIntoInputQ(pTask); /*int32_t code = */streamTaskPutTranstateIntoInputQ(pTask);
/*int32_t code = */streamSchedExec(pTask); return true;
} else { } else {
qWarn("s-task:%s fill-history scan WAL, nextProcessVer:%" PRId64 " out of the maximum ver:%" PRId64 ", not scan wal", qWarn("s-task:%s fill-history scan WAL, nextProcessVer:%" PRId64 " out of the maximum ver:%" PRId64 ", not scan wal",
id, ver, maxVer); id, ver, maxVer);
} }
} }
return false;
}
static bool taskReadyForDataFromWal(SStreamTask* pTask) {
// non-source or fill-history tasks don't need to response the WAL scan action.
if ((pTask->info.taskLevel != TASK_LEVEL__SOURCE) || (pTask->status.downstreamReady == 0)) {
return false;
}
// not in ready state, do not handle the data from wal
int32_t status = pTask->status.taskStatus;
if (status != TASK_STATUS__NORMAL) {
tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status));
return false;
}
// fill-history task has entered into the last phase, no need to anything
if ((pTask->info.fillHistory == 1) && pTask->status.appendTranstateBlock) {
ASSERT(status == TASK_STATUS__NORMAL);
// the maximum version of data in the WAL has reached already, the step2 is done
tqDebug("s-task:%s fill-history reach the maximum ver:%" PRId64 ", not scan wal anymore", pTask->id.idStr,
pTask->dataRange.range.maxVer);
return false;
}
// check if input queue is full or not
if (streamQueueIsFull(pTask->inputInfo.queue)) {
tqTrace("s-task:%s input queue is full, do nothing", pTask->id.idStr);
return false;
}
// the input queue of downstream task is full, so the output is blocked, stopped for a while
if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) {
tqDebug("s-task:%s inputQ is blocked, do nothing", pTask->id.idStr);
return false;
}
return true;
}
static bool doPutDataIntoInputQFromWal(SStreamTask* pTask, int64_t maxVer, int32_t* numOfItems) {
const char* id = pTask->id.idStr;
int32_t numOfNewItems = 0;
while(1) {
if ((pTask->info.fillHistory == 1) && pTask->status.appendTranstateBlock) {
*numOfItems += numOfNewItems;
return numOfNewItems > 0;
}
SStreamQueueItem* pItem = NULL;
int32_t code = extractMsgFromWal(pTask->exec.pWalReader, (void**)&pItem, maxVer, id);
if (code != TSDB_CODE_SUCCESS || pItem == NULL) { // failed, continue
int64_t currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader);
bool itemInFillhistory = handleFillhistoryScanComplete(pTask, currentVer);
if (itemInFillhistory) {
numOfNewItems += 1;
}
break;
}
if (pItem != NULL) {
code = streamTaskPutDataIntoInputQ(pTask, pItem);
if (code == TSDB_CODE_SUCCESS) {
numOfNewItems += 1;
int64_t ver = walReaderGetCurrentVer(pTask->exec.pWalReader);
pTask->chkInfo.nextProcessVer = ver;
tqDebug("s-task:%s set the ver:%" PRId64 " from WALReader after extract block from WAL", id, ver);
bool itemInFillhistory = handleFillhistoryScanComplete(pTask, ver);
if (itemInFillhistory) {
break;
}
} else {
tqError("s-task:%s append input queue failed, code: too many items, ver:%" PRId64, id, pTask->chkInfo.nextProcessVer);
break;
}
}
}
*numOfItems += numOfNewItems;
return numOfNewItems > 0;
} }
int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) {
@ -340,45 +424,13 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) {
numOfTasks = taosArrayGetSize(pTaskList); numOfTasks = taosArrayGetSize(pTaskList);
for (int32_t i = 0; i < numOfTasks; ++i) { for (int32_t i = 0; i < numOfTasks; ++i) {
SStreamTaskId* pTaskId = taosArrayGet(pTaskList, i); STaskId* pTaskId = taosArrayGet(pTaskList, i);
SStreamTask* pTask = streamMetaAcquireTask(pStreamMeta, pTaskId->streamId, pTaskId->taskId); SStreamTask* pTask = streamMetaAcquireTask(pStreamMeta, pTaskId->streamId, pTaskId->taskId);
if (pTask == NULL) { if (pTask == NULL) {
continue; continue;
} }
int32_t status = pTask->status.taskStatus; if (!taskReadyForDataFromWal(pTask)) {
// non-source or fill-history tasks don't need to response the WAL scan action.
if ((pTask->info.taskLevel != TASK_LEVEL__SOURCE) || (pTask->status.downstreamReady == 0)) {
streamMetaReleaseTask(pStreamMeta, pTask);
continue;
}
const char* pStatus = streamGetTaskStatusStr(status);
if (status != TASK_STATUS__NORMAL) {
tqTrace("s-task:%s not ready for new submit block from wal, status:%s", pTask->id.idStr, pStatus);
streamMetaReleaseTask(pStreamMeta, pTask);
continue;
}
if ((pTask->info.fillHistory == 1) && pTask->status.appendTranstateBlock) {
ASSERT(status == TASK_STATUS__NORMAL);
// the maximum version of data in the WAL has reached already, the step2 is done
tqDebug("s-task:%s fill-history reach the maximum ver:%" PRId64 ", not scan wal anymore", pTask->id.idStr,
pTask->dataRange.range.maxVer);
streamMetaReleaseTask(pStreamMeta, pTask);
continue;
}
if (streamQueueIsFull(pTask->inputInfo.queue->pQueue, true)) {
tqTrace("s-task:%s input queue is full, do nothing", pTask->id.idStr);
streamMetaReleaseTask(pStreamMeta, pTask);
continue;
}
// downstream task has blocked the output, stopped for a while
if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) {
tqDebug("s-task:%s inputQ is blocked, do nothing", pTask->id.idStr);
streamMetaReleaseTask(pStreamMeta, pTask); streamMetaReleaseTask(pStreamMeta, pTask);
continue; continue;
} }
@ -397,7 +449,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) {
taosThreadMutexLock(&pTask->lock); taosThreadMutexLock(&pTask->lock);
pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus);
if (pTask->status.taskStatus != TASK_STATUS__NORMAL) { if (pTask->status.taskStatus != TASK_STATUS__NORMAL) {
tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pStatus); tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pStatus);
taosThreadMutexUnlock(&pTask->lock); taosThreadMutexUnlock(&pTask->lock);
@ -405,33 +457,11 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) {
continue; continue;
} }
SStreamQueueItem* pItem = NULL; bool hasNewData = doPutDataIntoInputQFromWal(pTask, maxVer, &numOfItems);
code = extractMsgFromWal(pTask->exec.pWalReader, (void**)&pItem, maxVer, pTask->id.idStr);
if ((code != TSDB_CODE_SUCCESS || pItem == NULL) && (numOfItems == 0)) { // failed, continue
handleFillhistoryScanComplete(pTask, walReaderGetCurrentVer(pTask->exec.pWalReader));
streamMetaReleaseTask(pStreamMeta, pTask);
taosThreadMutexUnlock(&pTask->lock);
continue;
}
if (pItem != NULL) {
noDataInWal = false;
code = streamTaskPutDataIntoInputQ(pTask, pItem);
if (code == TSDB_CODE_SUCCESS) {
int64_t ver = walReaderGetCurrentVer(pTask->exec.pWalReader);
pTask->chkInfo.nextProcessVer = ver;
handleFillhistoryScanComplete(pTask, ver);
tqDebug("s-task:%s set the ver:%" PRId64 " from WALReader after extract block from WAL", pTask->id.idStr, ver);
} else {
tqError("s-task:%s append input queue failed, too many in inputQ, ver:%" PRId64, pTask->id.idStr,
pTask->chkInfo.nextProcessVer);
}
}
taosThreadMutexUnlock(&pTask->lock); taosThreadMutexUnlock(&pTask->lock);
if ((code == TSDB_CODE_SUCCESS) || (numOfItems > 0)) { if ((numOfItems > 0) || hasNewData) {
noDataInWal = false;
code = streamSchedExec(pTask); code = streamSchedExec(pTask);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
streamMetaReleaseTask(pStreamMeta, pTask); streamMetaReleaseTask(pStreamMeta, pTask);

View File

@ -710,6 +710,25 @@ static FORCE_INLINE int32_t tLDataIterDescCmprFn(const SRBTreeNode *p1, const SR
return -1 * tLDataIterCmprFn(p1, p2); return -1 * tLDataIterCmprFn(p1, p2);
} }
static void adjustValidLDataIters(SArray *pLDIterList, int32_t numOfFileObj) {
int32_t size = taosArrayGetSize(pLDIterList);
if (size < numOfFileObj) {
int32_t inc = numOfFileObj - size;
for (int32_t k = 0; k < inc; ++k) {
SLDataIter *pIter = taosMemoryCalloc(1, sizeof(SLDataIter));
taosArrayPush(pLDIterList, &pIter);
}
} else if (size > numOfFileObj) { // remove unused LDataIter
int32_t inc = size - numOfFileObj;
for (int i = 0; i < inc; ++i) {
SLDataIter *pIter = taosArrayPop(pLDIterList);
destroyLDataIter(pIter);
}
}
}
int32_t tMergeTreeOpen2(SMergeTree *pMTree, SMergeTreeConf *pConf) { int32_t tMergeTreeOpen2(SMergeTree *pMTree, SMergeTreeConf *pConf) {
int32_t code = TSDB_CODE_SUCCESS; int32_t code = TSDB_CODE_SUCCESS;
@ -725,45 +744,33 @@ int32_t tMergeTreeOpen2(SMergeTree *pMTree, SMergeTreeConf *pConf) {
pMTree->ignoreEarlierTs = false; pMTree->ignoreEarlierTs = false;
int32_t size = ((STFileSet *)pConf->pCurrentFileset)->lvlArr->size; // no data exists, go to end
if (size == 0) { int32_t numOfLevels = ((STFileSet *)pConf->pCurrentFileset)->lvlArr->size;
if (numOfLevels == 0) {
goto _end; goto _end;
} }
// add the list/iter placeholder // add the list/iter placeholder
while (taosArrayGetSize(pConf->pSttFileBlockIterArray) < size) { while (taosArrayGetSize(pConf->pSttFileBlockIterArray) < numOfLevels) {
SArray *pList = taosArrayInit(4, POINTER_BYTES); SArray *pList = taosArrayInit(4, POINTER_BYTES);
taosArrayPush(pConf->pSttFileBlockIterArray, &pList); taosArrayPush(pConf->pSttFileBlockIterArray, &pList);
} }
for (int32_t j = 0; j < size; ++j) { for (int32_t j = 0; j < numOfLevels; ++j) {
SSttLvl *pSttLevel = ((STFileSet *)pConf->pCurrentFileset)->lvlArr->data[j]; SSttLvl *pSttLevel = ((STFileSet *)pConf->pCurrentFileset)->lvlArr->data[j];
ASSERT(pSttLevel->level == j); SArray *pList = taosArrayGetP(pConf->pSttFileBlockIterArray, j);
SArray *pList = taosArrayGetP(pConf->pSttFileBlockIterArray, j); int32_t numOfFileObj = TARRAY2_SIZE(pSttLevel->fobjArr);
int32_t numOfIter = taosArrayGetSize(pList); adjustValidLDataIters(pList, numOfFileObj);
if (numOfIter < TARRAY2_SIZE(pSttLevel->fobjArr)) { for (int32_t i = 0; i < numOfFileObj; ++i) { // open all last file
int32_t inc = TARRAY2_SIZE(pSttLevel->fobjArr) - numOfIter;
for (int32_t k = 0; k < inc; ++k) {
SLDataIter *pIter = taosMemoryCalloc(1, sizeof(SLDataIter));
taosArrayPush(pList, &pIter);
}
} else if (numOfIter > TARRAY2_SIZE(pSttLevel->fobjArr)){
int32_t inc = numOfIter - TARRAY2_SIZE(pSttLevel->fobjArr);
for (int i = 0; i < inc; ++i) {
SLDataIter *pIter = taosArrayPop(pList);
destroyLDataIter(pIter);
}
}
for (int32_t i = 0; i < TARRAY2_SIZE(pSttLevel->fobjArr); ++i) { // open all last file
SLDataIter *pIter = taosArrayGetP(pList, i); SLDataIter *pIter = taosArrayGetP(pList, i);
SSttFileReader *pSttFileReader = pIter->pReader; SSttFileReader *pSttFileReader = pIter->pReader;
SSttBlockLoadInfo *pLoadInfo = pIter->pBlockLoadInfo; SSttBlockLoadInfo *pLoadInfo = pIter->pBlockLoadInfo;
// open stt file reader if not // open stt file reader if not opened yet
// if failed to open this stt file, ignore the error and try next one
if (pSttFileReader == NULL) { if (pSttFileReader == NULL) {
SSttFileReaderConfig conf = {.tsdb = pConf->pTsdb, .szPage = pConf->pTsdb->pVnode->config.tsdbPageSize}; SSttFileReaderConfig conf = {.tsdb = pConf->pTsdb, .szPage = pConf->pTsdb->pVnode->config.tsdbPageSize};
conf.file[0] = *pSttLevel->fobjArr->data[i]->f; conf.file[0] = *pSttLevel->fobjArr->data[i]->f;

View File

@ -1652,8 +1652,8 @@ static int32_t doMergeFileBlockAndLastBlock(SLastBlockReader* pLastBlockReader,
// create local variable to hold the row value // create local variable to hold the row value
TSDBROW fRow = {.iRow = pRow->iRow, .type = TSDBROW_COL_FMT, .pBlockData = pRow->pBlockData}; TSDBROW fRow = {.iRow = pRow->iRow, .type = TSDBROW_COL_FMT, .pBlockData = pRow->pBlockData};
tsdbTrace("fRow ptr:%p, %d, uid:%" PRIu64 ", %s", pRow->pBlockData, pRow->iRow, pLastBlockReader->uid, tsdbTrace("fRow ptr:%p, %d, uid:%" PRIu64 ", ts:%" PRId64 " %s", pRow->pBlockData, pRow->iRow, pLastBlockReader->uid,
pReader->idStr); fRow.pBlockData->aTSKEY[fRow.iRow], pReader->idStr);
// only last block exists // only last block exists
if ((!mergeBlockData) || (tsLastBlock != pBlockData->aTSKEY[pDumpInfo->rowIndex])) { if ((!mergeBlockData) || (tsLastBlock != pBlockData->aTSKEY[pDumpInfo->rowIndex])) {
@ -4605,14 +4605,14 @@ static SSDataBlock* doRetrieveDataBlock(STsdbReader* pReader) {
code = doLoadFileBlockData(pReader, &pStatus->blockIter, &pStatus->fileBlockData, pBlockScanInfo->uid); code = doLoadFileBlockData(pReader, &pStatus->blockIter, &pStatus->fileBlockData, pBlockScanInfo->uid);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
tBlockDataDestroy(&pStatus->fileBlockData); tBlockDataReset(&pStatus->fileBlockData);
terrno = code; terrno = code;
return NULL; return NULL;
} }
code = copyBlockDataToSDataBlock(pReader); code = copyBlockDataToSDataBlock(pReader);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
tBlockDataDestroy(&pStatus->fileBlockData); tBlockDataReset(&pStatus->fileBlockData);
terrno = code; terrno = code;
return NULL; return NULL;
} }

View File

@ -468,7 +468,6 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg
void *ptr = NULL; void *ptr = NULL;
void *pReq; void *pReq;
int32_t len; int32_t len;
int32_t ret;
if (ver <= pVnode->state.applied) { if (ver <= pVnode->state.applied) {
vError("vgId:%d, duplicate write request. ver: %" PRId64 ", applied: %" PRId64 "", TD_VID(pVnode), ver, vError("vgId:%d, duplicate write request. ver: %" PRId64 ", applied: %" PRId64 "", TD_VID(pVnode), ver,
@ -561,12 +560,14 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg
} }
break; break;
case TDMT_STREAM_TASK_DEPLOY: { case TDMT_STREAM_TASK_DEPLOY: {
if (tqProcessTaskDeployReq(pVnode->pTq, ver, pReq, len) < 0) { int32_t code = tqProcessTaskDeployReq(pVnode->pTq, ver, pReq, len);
if (code != TSDB_CODE_SUCCESS) {
terrno = code;
goto _err; goto _err;
} }
} break; } break;
case TDMT_STREAM_TASK_DROP: { case TDMT_STREAM_TASK_DROP: {
if (tqProcessTaskDropReq(pVnode->pTq, ver, pMsg->pCont, pMsg->contLen) < 0) { if (tqProcessTaskDropReq(pVnode->pTq, pMsg->pCont, pMsg->contLen) < 0) {
goto _err; goto _err;
} }
} break; } break;
@ -582,13 +583,17 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg
goto _err; goto _err;
} }
} break; } break;
case TDMT_VND_STREAM_TASK_RESET: {
if (pVnode->restored/* && vnodeIsLeader(pVnode)*/) {
tqProcessTaskResetReq(pVnode->pTq, pMsg);
}
} break;
case TDMT_VND_ALTER_CONFIRM: case TDMT_VND_ALTER_CONFIRM:
needCommit = pVnode->config.hashChange; needCommit = pVnode->config.hashChange;
if (vnodeProcessAlterConfirmReq(pVnode, ver, pReq, len, pRsp) < 0) { if (vnodeProcessAlterConfirmReq(pVnode, ver, pReq, len, pRsp) < 0) {
goto _err; goto _err;
} }
break; break;
case TDMT_VND_ALTER_CONFIG: case TDMT_VND_ALTER_CONFIG:
vnodeProcessAlterConfigReq(pVnode, ver, pReq, len, pRsp); vnodeProcessAlterConfigReq(pVnode, ver, pReq, len, pRsp);
break; break;
@ -602,7 +607,7 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg
vnodeProcessDropIndexReq(pVnode, ver, pReq, len, pRsp); vnodeProcessDropIndexReq(pVnode, ver, pReq, len, pRsp);
break; break;
case TDMT_VND_STREAM_CHECK_POINT_SOURCE: case TDMT_VND_STREAM_CHECK_POINT_SOURCE:
tqProcessStreamCheckPointSourceReq(pVnode->pTq, pMsg, pRsp); tqProcessTaskCheckPointSourceReq(pVnode->pTq, pMsg, pRsp);
break; break;
case TDMT_VND_STREAM_TASK_UPDATE: case TDMT_VND_STREAM_TASK_UPDATE:
tqProcessTaskUpdateReq(pVnode->pTq, pMsg); tqProcessTaskUpdateReq(pVnode->pTq, pMsg);

View File

@ -1317,8 +1317,9 @@ static STimeWindow getSlidingWindow(TSKEY* startTsCol, TSKEY* endTsCol, uint64_t
} }
static SSDataBlock* doRangeScan(SStreamScanInfo* pInfo, SSDataBlock* pSDB, int32_t tsColIndex, int32_t* pRowIndex) { static SSDataBlock* doRangeScan(SStreamScanInfo* pInfo, SSDataBlock* pSDB, int32_t tsColIndex, int32_t* pRowIndex) {
qInfo("do stream range scan. windows index:%d", *pRowIndex); qDebug("do stream range scan. windows index:%d", *pRowIndex);
bool prepareRes = true; bool prepareRes = true;
while (1) { while (1) {
SSDataBlock* pResult = NULL; SSDataBlock* pResult = NULL;
pResult = doTableScan(pInfo->pTableScanOp); pResult = doTableScan(pInfo->pTableScanOp);

View File

@ -370,10 +370,25 @@ static void doBuildDeleteResult(SStreamIntervalOperatorInfo* pInfo, SArray* pWin
} }
} }
void clearGroupResInfo(SGroupResInfo* pGroupResInfo) {
if (pGroupResInfo->freeItem) {
int32_t size = taosArrayGetSize(pGroupResInfo->pRows);
for (int32_t i = pGroupResInfo->index; i < size; i++) {
void* pVal = taosArrayGetP(pGroupResInfo->pRows, i);
taosMemoryFree(pVal);
}
pGroupResInfo->freeItem = false;
}
pGroupResInfo->pRows = taosArrayDestroy(pGroupResInfo->pRows);
pGroupResInfo->index = 0;
}
void destroyStreamFinalIntervalOperatorInfo(void* param) { void destroyStreamFinalIntervalOperatorInfo(void* param) {
SStreamIntervalOperatorInfo* pInfo = (SStreamIntervalOperatorInfo*)param; SStreamIntervalOperatorInfo* pInfo = (SStreamIntervalOperatorInfo*)param;
cleanupBasicInfo(&pInfo->binfo); cleanupBasicInfo(&pInfo->binfo);
cleanupAggSup(&pInfo->aggSup); cleanupAggSup(&pInfo->aggSup);
clearGroupResInfo(&pInfo->groupResInfo);
// it should be empty. // it should be empty.
void* pIte = NULL; void* pIte = NULL;
while ((pIte = taosHashIterate(pInfo->pPullDataMap, pIte)) != NULL) { while ((pIte = taosHashIterate(pInfo->pPullDataMap, pIte)) != NULL) {
@ -390,7 +405,6 @@ void destroyStreamFinalIntervalOperatorInfo(void* param) {
nodesDestroyNode((SNode*)pInfo->pPhyNode); nodesDestroyNode((SNode*)pInfo->pPhyNode);
colDataDestroy(&pInfo->twAggSup.timeWindowData); colDataDestroy(&pInfo->twAggSup.timeWindowData);
pInfo->groupResInfo.pRows = taosArrayDestroy(pInfo->groupResInfo.pRows);
cleanupExprSupp(&pInfo->scalarSupp); cleanupExprSupp(&pInfo->scalarSupp);
tSimpleHashCleanup(pInfo->pUpdatedMap); tSimpleHashCleanup(pInfo->pUpdatedMap);
pInfo->pUpdatedMap = NULL; pInfo->pUpdatedMap = NULL;
@ -1536,6 +1550,7 @@ void destroyStreamSessionAggOperatorInfo(void* param) {
cleanupBasicInfo(&pInfo->binfo); cleanupBasicInfo(&pInfo->binfo);
destroyStreamAggSupporter(&pInfo->streamAggSup); destroyStreamAggSupporter(&pInfo->streamAggSup);
cleanupExprSupp(&pInfo->scalarSupp); cleanupExprSupp(&pInfo->scalarSupp);
clearGroupResInfo(&pInfo->groupResInfo);
if (pInfo->pChildren != NULL) { if (pInfo->pChildren != NULL) {
int32_t size = taosArrayGetSize(pInfo->pChildren); int32_t size = taosArrayGetSize(pInfo->pChildren);
@ -2862,7 +2877,7 @@ void destroyStreamStateOperatorInfo(void* param) {
SStreamStateAggOperatorInfo* pInfo = (SStreamStateAggOperatorInfo*)param; SStreamStateAggOperatorInfo* pInfo = (SStreamStateAggOperatorInfo*)param;
cleanupBasicInfo(&pInfo->binfo); cleanupBasicInfo(&pInfo->binfo);
destroyStreamAggSupporter(&pInfo->streamAggSup); destroyStreamAggSupporter(&pInfo->streamAggSup);
cleanupGroupResInfo(&pInfo->groupResInfo); clearGroupResInfo(&pInfo->groupResInfo);
cleanupExprSupp(&pInfo->scalarSupp); cleanupExprSupp(&pInfo->scalarSupp);
if (pInfo->pChildren != NULL) { if (pInfo->pChildren != NULL) {
int32_t size = taosArrayGetSize(pInfo->pChildren); int32_t size = taosArrayGetSize(pInfo->pChildren);

View File

@ -40,6 +40,9 @@ extern "C" {
#define META_HB_SEND_IDLE_COUNTER 25 // send hb every 5 sec #define META_HB_SEND_IDLE_COUNTER 25 // send hb every 5 sec
#define STREAM_TASK_KEY_LEN ((sizeof(int64_t)) << 1) #define STREAM_TASK_KEY_LEN ((sizeof(int64_t)) << 1)
#define STREAM_TASK_QUEUE_CAPACITY 20480
#define STREAM_TASK_QUEUE_CAPACITY_IN_SIZE (30)
// clang-format off // clang-format off
#define stFatal(...) do { if (stDebugFlag & DEBUG_FATAL) { taosPrintLog("STM FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); }} while(0) #define stFatal(...) do { if (stDebugFlag & DEBUG_FATAL) { taosPrintLog("STM FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); }} while(0)
#define stError(...) do { if (stDebugFlag & DEBUG_ERROR) { taosPrintLog("STM ERROR ", DEBUG_ERROR, 255, __VA_ARGS__); }} while(0) #define stError(...) do { if (stDebugFlag & DEBUG_ERROR) { taosPrintLog("STM ERROR ", DEBUG_ERROR, 255, __VA_ARGS__); }} while(0)
@ -64,12 +67,19 @@ struct STokenBucket {
int32_t numCapacity; // total capacity, available token per second int32_t numCapacity; // total capacity, available token per second
int32_t numOfToken; // total available tokens int32_t numOfToken; // total available tokens
int32_t numRate; // number of token per second int32_t numRate; // number of token per second
double bytesCapacity; // available capacity for maximum input size, KiloBytes per Second double quotaCapacity; // available capacity for maximum input size, KiloBytes per Second
double bytesRemain; // not consumed bytes per second double quotaRemain; // not consumed bytes per second
double bytesRate; // number of token per second double quotaRate; // number of token per second
int64_t fillTimestamp; // fill timestamp int64_t fillTimestamp; // fill timestamp
}; };
struct SStreamQueue {
STaosQueue* pQueue;
STaosQall* qall;
void* qItem;
int8_t status;
};
extern SStreamGlobalEnv streamEnv; extern SStreamGlobalEnv streamEnv;
extern int32_t streamBackendId; extern int32_t streamBackendId;
extern int32_t streamBackendCfWrapperId; extern int32_t streamBackendCfWrapperId;
@ -100,7 +110,6 @@ int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask);
int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask); int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask);
int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks, int32_t* blockSize); int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks, int32_t* blockSize);
int32_t streamQueueGetNumOfItemsInQueue(const SStreamQueue* pQueue);
int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem); int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem);
void streamQueueItemIncSize(const SStreamQueueItem* pItem, int32_t size); void streamQueueItemIncSize(const SStreamQueueItem* pItem, int32_t size);
const char* streamQueueItemGetTypeStr(int32_t type); const char* streamQueueItemGetTypeStr(int32_t type);
@ -113,19 +122,20 @@ int32_t streamNotifyUpstreamContinue(SStreamTask* pTask);
int32_t streamTaskFillHistoryFinished(SStreamTask* pTask); int32_t streamTaskFillHistoryFinished(SStreamTask* pTask);
int32_t streamTransferStateToStreamTask(SStreamTask* pTask); int32_t streamTransferStateToStreamTask(SStreamTask* pTask);
int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t bytesRate); int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t quotaRate);
STaskId streamTaskExtractKey(const SStreamTask* pTask); STaskId streamTaskExtractKey(const SStreamTask* pTask);
void streamTaskInitForLaunchHTask(SHistoryTaskInfo* pInfo); void streamTaskInitForLaunchHTask(SHistoryTaskInfo* pInfo);
void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo); void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo);
void streamMetaResetStartInfo(STaskStartInfo* pMeta);
SStreamQueue* streamQueueOpen(int64_t cap); SStreamQueue* streamQueueOpen(int64_t cap);
void streamQueueClose(SStreamQueue* pQueue, int32_t taskId); void streamQueueClose(SStreamQueue* pQueue, int32_t taskId);
void streamQueueProcessSuccess(SStreamQueue* queue); void streamQueueProcessSuccess(SStreamQueue* queue);
void streamQueueProcessFail(SStreamQueue* queue); void streamQueueProcessFail(SStreamQueue* queue);
void* streamQueueNextItem(SStreamQueue* pQueue); void* streamQueueNextItem(SStreamQueue* pQueue);
void streamFreeQitem(SStreamQueueItem* data); void streamFreeQitem(SStreamQueueItem* data);
int32_t streamQueueGetItemSize(const SStreamQueue* pQueue);
#ifdef __cplusplus #ifdef __cplusplus
} }

View File

@ -58,39 +58,54 @@ char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) {
static void streamSchedByTimer(void* param, void* tmrId) { static void streamSchedByTimer(void* param, void* tmrId) {
SStreamTask* pTask = (void*)param; SStreamTask* pTask = (void*)param;
const char* id = pTask->id.idStr;
int32_t nextTrigger = (int32_t)pTask->info.triggerParam;
int8_t status = atomic_load_8(&pTask->schedInfo.status); int8_t status = atomic_load_8(&pTask->schedInfo.status);
stDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", pTask->id.idStr, status, (int32_t)pTask->info.triggerParam); stDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", id, status, nextTrigger);
if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) {
stDebug("s-task:%s jump out of schedTimer", pTask->id.idStr); stDebug("s-task:%s jump out of schedTimer", id);
return; return;
} }
if (status == TASK_TRIGGER_STATUS__ACTIVE) { if (pTask->status.taskStatus == TASK_STATUS__CK) {
SStreamTrigger* pTrigger = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0); stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger);
if (pTrigger == NULL) { } else {
return; if (status == TASK_TRIGGER_STATUS__ACTIVE) {
} SStreamTrigger* pTrigger = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0);
if (pTrigger == NULL) {
stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory",
nextTrigger);
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamEnv.timer, &pTask->schedInfo.pTimer);
return;
}
pTrigger->type = STREAM_INPUT__GET_RES; pTrigger->type = STREAM_INPUT__GET_RES;
pTrigger->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); pTrigger->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock));
if (pTrigger->pBlock == NULL) { if (pTrigger->pBlock == NULL) {
taosFreeQitem(pTrigger); taosFreeQitem(pTrigger);
return;
}
atomic_store_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE); stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory",
pTrigger->pBlock->info.type = STREAM_GET_ALL; nextTrigger);
if (streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger) < 0) { taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamEnv.timer, &pTask->schedInfo.pTimer);
taosTmrReset(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamEnv.timer, &pTask->schedInfo.pTimer); return;
return; }
}
streamSchedExec(pTask); atomic_store_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE);
pTrigger->pBlock->info.type = STREAM_GET_ALL;
int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger);
if (code != TSDB_CODE_SUCCESS) {
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamEnv.timer, &pTask->schedInfo.pTimer);
return;
}
streamSchedExec(pTask);
}
} }
taosTmrReset(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamEnv.timer, &pTask->schedInfo.pTimer); taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamEnv.timer, &pTask->schedInfo.pTimer);
} }
int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { int32_t streamSetupScheduleTrigger(SStreamTask* pTask) {
@ -108,8 +123,7 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask) {
} }
int32_t streamSchedExec(SStreamTask* pTask) { int32_t streamSchedExec(SStreamTask* pTask) {
int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); if (streamTaskSetSchedStatusWait(pTask)) {
if (schedStatus == TASK_SCHED_STATUS__INACTIVE) {
SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq));
if (pRunReq == NULL) { if (pRunReq == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
@ -127,7 +141,7 @@ int32_t streamSchedExec(SStreamTask* pTask) {
SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)};
tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg);
} else { } else {
stDebug("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); stTrace("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus);
} }
return 0; return 0;
@ -240,8 +254,9 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S
status = TASK_INPUT_STATUS__BLOCKED; status = TASK_INPUT_STATUS__BLOCKED;
} else { } else {
// This task has received the checkpoint req from the upstream task, from which all the messages should be // This task has received the checkpoint req from the upstream task, from which all the messages should be
// blocked // blocked. Note that there is no race condition here.
if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) {
atomic_add_fetch_32(&pTask->upstreamInfo.numOfClosed, 1);
streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId);
stDebug("s-task:%s close inputQ for upstream:0x%x, msgId:%d", id, pReq->upstreamTaskId, pReq->msgId); stDebug("s-task:%s close inputQ for upstream:0x%x, msgId:%d", id, pReq->upstreamTaskId, pReq->msgId);
} }
@ -275,13 +290,6 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S
return 0; return 0;
} }
int32_t streamProcessRunReq(SStreamTask* pTask) {
if (streamTryExec(pTask) < 0) {
return -1;
}
return 0;
}
int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pRsp) { int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pRsp) {
streamTaskEnqueueRetrieve(pTask, pReq, pRsp); streamTaskEnqueueRetrieve(pTask, pReq, pRsp);
ASSERT(pTask->info.taskLevel != TASK_LEVEL__SINK); ASSERT(pTask->info.taskLevel != TASK_LEVEL__SINK);
@ -292,15 +300,17 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, S
void streamTaskInputFail(SStreamTask* pTask) { atomic_store_8(&pTask->inputInfo.status, TASK_INPUT_STATUS__FAILED); } void streamTaskInputFail(SStreamTask* pTask) { atomic_store_8(&pTask->inputInfo.status, TASK_INPUT_STATUS__FAILED); }
void streamTaskOpenAllUpstreamInput(SStreamTask* pTask) { void streamTaskOpenAllUpstreamInput(SStreamTask* pTask) {
int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList);
if (num == 0) { if (num == 0) {
return; return;
} }
for (int32_t i = 0; i < num; ++i) { for (int32_t i = 0; i < num; ++i) {
SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i);
pInfo->dataAllowed = true; pInfo->dataAllowed = true;
} }
pTask->upstreamInfo.numOfClosed = 0;
} }
void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) { void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) {
@ -311,9 +321,9 @@ void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) {
} }
SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId) { SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId) {
int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList);
for (int32_t i = 0; i < num; ++i) { for (int32_t i = 0; i < num; ++i) {
SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i);
if (pInfo->taskId == taskId) { if (pInfo->taskId == taskId) {
return pInfo; return pInfo;
} }

View File

@ -92,7 +92,7 @@ int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointRea
} }
static int32_t streamAlignCheckpoint(SStreamTask* pTask) { static int32_t streamAlignCheckpoint(SStreamTask* pTask) {
int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList);
int64_t old = atomic_val_compare_exchange_32(&pTask->checkpointAlignCnt, 0, num); int64_t old = atomic_val_compare_exchange_32(&pTask->checkpointAlignCnt, 0, num);
if (old == 0) { if (old == 0) {
stDebug("s-task:%s set initial align upstream num:%d", pTask->id.idStr, num); stDebug("s-task:%s set initial align upstream num:%d", pTask->id.idStr, num);
@ -143,8 +143,8 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo
pTask->execInfo.checkpoint += 1; pTask->execInfo.checkpoint += 1;
// 2. let's dispatch checkpoint msg to downstream task directly and do nothing else. put the checkpoint block into // 2. Put the checkpoint block into inputQ, to make sure all blocks with less version have been handled by this task
// inputQ, to make sure all blocks with less version have been handled by this task already. // already.
int32_t code = appendCheckpointIntoInputQ(pTask, STREAM_INPUT__CHECKPOINT_TRIGGER); int32_t code = appendCheckpointIntoInputQ(pTask, STREAM_INPUT__CHECKPOINT_TRIGGER);
return code; return code;
} }
@ -153,7 +153,7 @@ static int32_t continueDispatchCheckpointBlock(SStreamDataBlock* pBlock, SStream
pBlock->srcTaskId = pTask->id.taskId; pBlock->srcTaskId = pTask->id.taskId;
pBlock->srcVgId = pTask->pMeta->vgId; pBlock->srcVgId = pTask->pMeta->vgId;
int32_t code = taosWriteQitem(pTask->outputInfo.queue->pQueue, pBlock); int32_t code = taosWriteQitem(pTask->outputq.queue->pQueue, pBlock);
if (code == 0) { if (code == 0) {
streamDispatchStreamBlock(pTask); streamDispatchStreamBlock(pTask);
} else { } else {
@ -192,14 +192,14 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc
int32_t taskLevel = pTask->info.taskLevel; int32_t taskLevel = pTask->info.taskLevel;
if (taskLevel == TASK_LEVEL__SOURCE) { if (taskLevel == TASK_LEVEL__SOURCE) {
if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
stDebug("s-task:%s set childIdx:%d, and add checkpoint block into outputQ", id, pTask->info.selfChildId); stDebug("s-task:%s set childIdx:%d, and add checkpoint-trigger block into outputQ", id, pTask->info.selfChildId);
continueDispatchCheckpointBlock(pBlock, pTask); continueDispatchCheckpointBlock(pBlock, pTask);
} else { // only one task exists, no need to dispatch downstream info } else { // only one task exists, no need to dispatch downstream info
streamProcessCheckpointReadyMsg(pTask); streamProcessCheckpointReadyMsg(pTask);
streamFreeQitem((SStreamQueueItem*)pBlock); streamFreeQitem((SStreamQueueItem*)pBlock);
} }
} else if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) { } else if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) {
ASSERT(taosArrayGetSize(pTask->pUpstreamInfoList) > 0); ASSERT(taosArrayGetSize(pTask->upstreamInfo.pList) > 0);
if (pTask->chkInfo.startTs == 0) { if (pTask->chkInfo.startTs == 0) {
pTask->chkInfo.startTs = taosGetTimestampMs(); pTask->chkInfo.startTs = taosGetTimestampMs();
pTask->execInfo.checkpoint += 1; pTask->execInfo.checkpoint += 1;
@ -210,7 +210,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc
// there are still some upstream tasks not send checkpoint request, do nothing and wait for then // there are still some upstream tasks not send checkpoint request, do nothing and wait for then
int32_t notReady = streamAlignCheckpoint(pTask); int32_t notReady = streamAlignCheckpoint(pTask);
int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList);
if (notReady > 0) { if (notReady > 0) {
stDebug("s-task:%s received checkpoint block, idx:%d, %d upstream tasks not send checkpoint info yet, total:%d", stDebug("s-task:%s received checkpoint block, idx:%d, %d upstream tasks not send checkpoint info yet, total:%d",
id, pTask->info.selfChildId, notReady, num); id, pTask->info.selfChildId, notReady, num);
@ -264,6 +264,15 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask) {
return 0; return 0;
} }
void streamTaskClearCheckInfo(SStreamTask* pTask) {
pTask->checkpointingId = 0; // clear the checkpoint id
pTask->chkInfo.failedId = 0;
pTask->chkInfo.startTs = 0; // clear the recorded start time
pTask->checkpointNotReadyTasks = 0;
pTask->checkpointAlignCnt = 0;
streamTaskOpenAllUpstreamInput(pTask); // open inputQ for all upstream tasks
}
int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) {
taosWLockLatch(&pMeta->lock); taosWLockLatch(&pMeta->lock);
@ -283,11 +292,11 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) {
ASSERT(p->chkInfo.checkpointId < p->checkpointingId && p->checkpointingId == checkpointId); ASSERT(p->chkInfo.checkpointId < p->checkpointingId && p->checkpointingId == checkpointId);
p->chkInfo.checkpointId = p->checkpointingId; p->chkInfo.checkpointId = p->checkpointingId;
streamTaskClearCheckInfo(p);
streamSetStatusNormal(p); streamSetStatusNormal(p);
// save the task // save the task
streamMetaSaveTask(pMeta, p); streamMetaSaveTask(pMeta, p);
streamTaskOpenAllUpstreamInput(p); // open inputQ for all upstream tasks
stDebug( stDebug(
"vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, "
@ -318,8 +327,6 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) {
ASSERT(remain >= 0); ASSERT(remain >= 0);
double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0;
pTask->chkInfo.startTs = 0; // clear the recorded start time
if (remain == 0) { // all tasks are ready if (remain == 0) { // all tasks are ready
stDebug("s-task:%s all downstreams are ready, ready for do checkpoint", pTask->id.idStr); stDebug("s-task:%s all downstreams are ready, ready for do checkpoint", pTask->id.idStr);
streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); streamBackendDoCheckpoint(pMeta, pTask->checkpointingId);

View File

@ -195,11 +195,11 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock)
.retrieveLen = dataStrLen, .retrieveLen = dataStrLen,
}; };
int32_t sz = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t sz = taosArrayGetSize(pTask->upstreamInfo.pList);
ASSERT(sz > 0); ASSERT(sz > 0);
for (int32_t i = 0; i < sz; i++) { for (int32_t i = 0; i < sz; i++) {
req.reqId = tGenIdPI64(); req.reqId = tGenIdPI64();
SStreamChildEpInfo* pEpInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); SStreamChildEpInfo* pEpInfo = taosArrayGetP(pTask->upstreamInfo.pList, i);
req.dstNodeId = pEpInfo->nodeId; req.dstNodeId = pEpInfo->nodeId;
req.dstTaskId = pEpInfo->taskId; req.dstTaskId = pEpInfo->taskId;
int32_t len; int32_t len;
@ -288,7 +288,7 @@ void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups) {
int32_t getNumOfDispatchBranch(SStreamTask* pTask) { int32_t getNumOfDispatchBranch(SStreamTask* pTask) {
return (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) return (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH)
? 1 ? 1
: taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); : taosArrayGetSize(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos);
} }
static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pData) { static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pData) {
@ -301,7 +301,7 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD
if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) {
SStreamDispatchReq* pReq = taosMemoryCalloc(1, sizeof(SStreamDispatchReq)); SStreamDispatchReq* pReq = taosMemoryCalloc(1, sizeof(SStreamDispatchReq));
int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; int32_t downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId;
code = tInitStreamDispatchReq(pReq, pTask, pData->srcVgId, numOfBlocks, downstreamTaskId, pData->type); code = tInitStreamDispatchReq(pReq, pTask, pData->srcVgId, numOfBlocks, downstreamTaskId, pData->type);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
return code; return code;
@ -318,10 +318,10 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD
pTask->msgInfo.pData = pReq; pTask->msgInfo.pData = pReq;
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
int32_t rspCnt = atomic_load_32(&pTask->shuffleDispatcher.waitingRspCnt); int32_t rspCnt = atomic_load_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt);
ASSERT(rspCnt == 0); ASSERT(rspCnt == 0);
SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
int32_t numOfVgroups = taosArrayGetSize(vgInfo); int32_t numOfVgroups = taosArrayGetSize(vgInfo);
SStreamDispatchReq* pReqs = taosMemoryCalloc(numOfVgroups, sizeof(SStreamDispatchReq)); SStreamDispatchReq* pReqs = taosMemoryCalloc(numOfVgroups, sizeof(SStreamDispatchReq));
@ -352,7 +352,7 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD
} }
if (pReqs[j].blockNum == 0) { if (pReqs[j].blockNum == 0) {
atomic_add_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); atomic_add_fetch_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 1);
} }
pReqs[j].blockNum++; pReqs[j].blockNum++;
@ -381,16 +381,16 @@ static int32_t sendDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pDispatch
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) {
int32_t vgId = pTask->fixedDispatcher.nodeId; int32_t vgId = pTask->outputInfo.fixedDispatcher.nodeId;
SEpSet* pEpSet = &pTask->fixedDispatcher.epSet; SEpSet* pEpSet = &pTask->outputInfo.fixedDispatcher.epSet;
int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; int32_t downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId;
stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id,
pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId);
code = doSendDispatchMsg(pTask, pDispatchMsg, vgId, pEpSet); code = doSendDispatchMsg(pTask, pDispatchMsg, vgId, pEpSet);
} else { } else {
SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
int32_t numOfVgroups = taosArrayGetSize(vgInfo); int32_t numOfVgroups = taosArrayGetSize(vgInfo);
stDebug("s-task:%s (child taskId:%d) start to shuffle-dispatch blocks to %d vgroup(s), msgId:%d", stDebug("s-task:%s (child taskId:%d) start to shuffle-dispatch blocks to %d vgroup(s), msgId:%d",
@ -421,12 +421,12 @@ static void doRetryDispatchData(void* param, void* tmrId) {
int32_t msgId = pTask->execInfo.dispatch; int32_t msgId = pTask->execInfo.dispatch;
if (streamTaskShouldStop(&pTask->status)) { if (streamTaskShouldStop(&pTask->status)) {
int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1);
stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref);
return; return;
} }
ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); ASSERT(pTask->outputq.status == TASK_OUTPUT_STATUS__WAIT);
int32_t code = 0; int32_t code = 0;
{ {
@ -436,7 +436,7 @@ static void doRetryDispatchData(void* param, void* tmrId) {
SStreamDispatchReq *pReq = pTask->msgInfo.pData; SStreamDispatchReq *pReq = pTask->msgInfo.pData;
if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
int32_t numOfVgroups = taosArrayGetSize(vgInfo); int32_t numOfVgroups = taosArrayGetSize(vgInfo);
int32_t numOfFailed = taosArrayGetSize(pList); int32_t numOfFailed = taosArrayGetSize(pList);
@ -462,9 +462,9 @@ static void doRetryDispatchData(void* param, void* tmrId) {
stDebug("s-task:%s complete re-try shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfFailed, msgId); stDebug("s-task:%s complete re-try shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfFailed, msgId);
} else { } else {
int32_t vgId = pTask->fixedDispatcher.nodeId; int32_t vgId = pTask->outputInfo.fixedDispatcher.nodeId;
SEpSet* pEpSet = &pTask->fixedDispatcher.epSet; SEpSet* pEpSet = &pTask->outputInfo.fixedDispatcher.epSet;
int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; int32_t downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId;
stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id,
pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId);
@ -476,7 +476,7 @@ static void doRetryDispatchData(void* param, void* tmrId) {
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
if (!streamTaskShouldStop(&pTask->status)) { if (!streamTaskShouldStop(&pTask->status)) {
// stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); // stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr);
// atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); // atomic_store_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 0);
if (streamTaskShouldPause(&pTask->status)) { if (streamTaskShouldPause(&pTask->status)) {
streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS * 10); streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS * 10);
} else { } else {
@ -487,7 +487,7 @@ static void doRetryDispatchData(void* param, void* tmrId) {
stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref);
} }
} else { } else {
int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1);
stDebug("s-task:%s send success, jump out of timer, ref:%d", pTask->id.idStr, ref); stDebug("s-task:%s send success, jump out of timer, ref:%d", pTask->id.idStr, ref);
} }
} }
@ -508,7 +508,7 @@ void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration) {
int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, SSDataBlock* pDataBlock, int32_t vgSz, int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, SSDataBlock* pDataBlock, int32_t vgSz,
int64_t groupId) { int64_t groupId) {
uint32_t hashValue = 0; uint32_t hashValue = 0;
SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
if (pTask->pNameMap == NULL) { if (pTask->pNameMap == NULL) {
pTask->pNameMap = tSimpleHashInit(1024, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT)); pTask->pNameMap = tSimpleHashInit(1024, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT));
} }
@ -528,14 +528,14 @@ int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, S
} }
if (pDataBlock->info.parTbName[0]) { if (pDataBlock->info.parTbName[0]) {
snprintf(ctbName, TSDB_TABLE_NAME_LEN, "%s.%s", pTask->shuffleDispatcher.dbInfo.db, pDataBlock->info.parTbName); snprintf(ctbName, TSDB_TABLE_NAME_LEN, "%s.%s", pTask->outputInfo.shuffleDispatcher.dbInfo.db, pDataBlock->info.parTbName);
} else { } else {
buildCtbNameByGroupIdImpl(pTask->shuffleDispatcher.stbFullName, groupId, pDataBlock->info.parTbName); buildCtbNameByGroupIdImpl(pTask->outputInfo.shuffleDispatcher.stbFullName, groupId, pDataBlock->info.parTbName);
snprintf(ctbName, TSDB_TABLE_NAME_LEN, "%s.%s", pTask->shuffleDispatcher.dbInfo.db, pDataBlock->info.parTbName); snprintf(ctbName, TSDB_TABLE_NAME_LEN, "%s.%s", pTask->outputInfo.shuffleDispatcher.dbInfo.db, pDataBlock->info.parTbName);
} }
/*uint32_t hashValue = MurmurHash3_32(ctbName, strlen(ctbName));*/ /*uint32_t hashValue = MurmurHash3_32(ctbName, strlen(ctbName));*/
SUseDbRsp* pDbInfo = &pTask->shuffleDispatcher.dbInfo; SUseDbRsp* pDbInfo = &pTask->outputInfo.shuffleDispatcher.dbInfo;
hashValue = hashValue =
taosGetTbHashVal(ctbName, strlen(ctbName), pDbInfo->hashMethod, pDbInfo->hashPrefix, pDbInfo->hashSuffix); taosGetTbHashVal(ctbName, strlen(ctbName), pDbInfo->hashMethod, pDbInfo->hashPrefix, pDbInfo->hashSuffix);
taosMemoryFree(ctbName); taosMemoryFree(ctbName);
@ -560,7 +560,7 @@ int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, S
} }
if (pReqs[j].blockNum == 0) { if (pReqs[j].blockNum == 0) {
atomic_add_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); atomic_add_fetch_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 1);
} }
pReqs[j].blockNum++; pReqs[j].blockNum++;
@ -576,34 +576,33 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) {
ASSERT((pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH)); ASSERT((pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH));
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
int32_t numOfElems = streamQueueGetNumOfItems(pTask->outputInfo.queue); int32_t numOfElems = streamQueueGetNumOfItems(pTask->outputq.queue);
if (numOfElems > 0) { if (numOfElems > 0) {
double size = SIZE_IN_MiB(taosQueueMemorySize(pTask->outputInfo.queue->pQueue)); double size = SIZE_IN_MiB(taosQueueMemorySize(pTask->outputq.queue->pQueue));
stDebug("s-task:%s start to dispatch intermediate block to downstream, elem in outputQ:%d, size:%.2fMiB", id, numOfElems, size); stDebug("s-task:%s start to dispatch intermediate block to downstream, elem in outputQ:%d, size:%.2fMiB", id, numOfElems, size);
} }
// to make sure only one dispatch is running // to make sure only one dispatch is running
int8_t old = int8_t old =
atomic_val_compare_exchange_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL, TASK_OUTPUT_STATUS__WAIT); atomic_val_compare_exchange_8(&pTask->outputq.status, TASK_OUTPUT_STATUS__NORMAL, TASK_OUTPUT_STATUS__WAIT);
if (old != TASK_OUTPUT_STATUS__NORMAL) { if (old != TASK_OUTPUT_STATUS__NORMAL) {
stDebug("s-task:%s wait for dispatch rsp, not dispatch now, output status:%d", id, old); stDebug("s-task:%s wait for dispatch rsp, not dispatch now, output status:%d", id, old);
return 0; return 0;
} }
ASSERT(pTask->msgInfo.pData == NULL); ASSERT(pTask->msgInfo.pData == NULL);
stDebug("s-task:%s start to dispatch msg, set output status:%d", id, pTask->outputInfo.status); stDebug("s-task:%s start to dispatch msg, set output status:%d", id, pTask->outputq.status);
SStreamDataBlock* pBlock = streamQueueNextItem(pTask->outputInfo.queue); SStreamDataBlock* pBlock = streamQueueNextItem(pTask->outputq.queue);
if (pBlock == NULL) { if (pBlock == NULL) {
atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); atomic_store_8(&pTask->outputq.status, TASK_OUTPUT_STATUS__NORMAL);
stDebug("s-task:%s not dispatch since no elems in outputQ, output status:%d", id, pTask->outputInfo.status); stDebug("s-task:%s not dispatch since no elems in outputQ, output status:%d", id, pTask->outputq.status);
return 0; return 0;
} }
ASSERT(pBlock->type == STREAM_INPUT__DATA_BLOCK || pBlock->type == STREAM_INPUT__CHECKPOINT_TRIGGER || ASSERT(pBlock->type == STREAM_INPUT__DATA_BLOCK || pBlock->type == STREAM_INPUT__CHECKPOINT_TRIGGER ||
pBlock->type == STREAM_INPUT__TRANS_STATE); pBlock->type == STREAM_INPUT__TRANS_STATE);
int32_t retryCount = 0;
pTask->execInfo.dispatch += 1; pTask->execInfo.dispatch += 1;
pTask->msgInfo.startTs = taosGetTimestampMs(); pTask->msgInfo.startTs = taosGetTimestampMs();
@ -613,6 +612,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) {
} else { // todo handle build dispatch msg failed } else { // todo handle build dispatch msg failed
} }
int32_t retryCount = 0;
while (1) { while (1) {
code = sendDispatchMsg(pTask, pTask->msgInfo.pData); code = sendDispatchMsg(pTask, pTask->msgInfo.pData);
if (code == TSDB_CODE_SUCCESS) { if (code == TSDB_CODE_SUCCESS) {
@ -620,10 +620,10 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) {
} }
stDebug("s-task:%s failed to dispatch msg:%d to downstream, code:%s, output status:%d, retry cnt:%d", id, stDebug("s-task:%s failed to dispatch msg:%d to downstream, code:%s, output status:%d, retry cnt:%d", id,
pTask->execInfo.dispatch, tstrerror(terrno), pTask->outputInfo.status, retryCount); pTask->execInfo.dispatch, tstrerror(terrno), pTask->outputq.status, retryCount);
// todo deal with only partially success dispatch case // todo deal with only partially success dispatch case
atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); atomic_store_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 0);
if (terrno == TSDB_CODE_APP_IS_STOPPING) { // in case of this error, do not retry anymore if (terrno == TSDB_CODE_APP_IS_STOPPING) { // in case of this error, do not retry anymore
destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask));
pTask->msgInfo.pData = NULL; pTask->msgInfo.pData = NULL;
@ -631,7 +631,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) {
} }
if (++retryCount > MAX_CONTINUE_RETRY_COUNT) { // add to timer to retry if (++retryCount > MAX_CONTINUE_RETRY_COUNT) { // add to timer to retry
int8_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1);
stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d",
pTask->id.idStr, retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); pTask->id.idStr, retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref);
@ -654,11 +654,11 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) {
// serialize // serialize
if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) {
req.downstreamTaskId = pTask->fixedDispatcher.taskId; req.downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId;
pTask->notReadyTasks = 1; pTask->notReadyTasks = 1;
doDispatchScanHistoryFinishMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); doDispatchScanHistoryFinishMsg(pTask, &req, pTask->outputInfo.fixedDispatcher.nodeId, &pTask->outputInfo.fixedDispatcher.epSet);
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
int32_t numOfVgs = taosArrayGetSize(vgInfo); int32_t numOfVgs = taosArrayGetSize(vgInfo);
pTask->notReadyTasks = numOfVgs; pTask->notReadyTasks = numOfVgs;
@ -680,7 +680,7 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) {
// this function is usually invoked by sink/agg task // this function is usually invoked by sink/agg task
int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask) { int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask) {
int32_t num = taosArrayGetSize(pTask->pReadyMsgList); int32_t num = taosArrayGetSize(pTask->pReadyMsgList);
ASSERT(taosArrayGetSize(pTask->pUpstreamInfoList) == num); ASSERT(taosArrayGetSize(pTask->upstreamInfo.pList) == num);
for (int32_t i = 0; i < num; ++i) { for (int32_t i = 0; i < num; ++i) {
SStreamChkptReadyInfo* pInfo = taosArrayGet(pTask->pReadyMsgList, i); SStreamChkptReadyInfo* pInfo = taosArrayGet(pTask->pReadyMsgList, i);
@ -1049,7 +1049,7 @@ static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId
} }
// now ready for next data output // now ready for next data output
atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); atomic_store_8(&pTask->outputq.status, TASK_OUTPUT_STATUS__NORMAL);
// otherwise, continue dispatch the first block to down stream task in pipeline // otherwise, continue dispatch the first block to down stream task in pipeline
streamDispatchStreamBlock(pTask); streamDispatchStreamBlock(pTask);
@ -1061,11 +1061,13 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i
int32_t vgId = pTask->pMeta->vgId; int32_t vgId = pTask->pMeta->vgId;
int32_t msgId = pTask->execInfo.dispatch; int32_t msgId = pTask->execInfo.dispatch;
// follower not handle the dispatch rsp
if ((pTask->pMeta->role == NODE_ROLE_FOLLOWER) || (pTask->status.downstreamReady != 1)) { if ((pTask->pMeta->role == NODE_ROLE_FOLLOWER) || (pTask->status.downstreamReady != 1)) {
stError("s-task:%s vgId:%d is follower or task just re-launched, not handle the dispatch rsp, discard it", id, vgId); stError("s-task:%s vgId:%d is follower or task just re-launched, not handle the dispatch rsp, discard it", id, vgId);
return TSDB_CODE_STREAM_TASK_NOT_EXIST; return TSDB_CODE_STREAM_TASK_NOT_EXIST;
} }
// discard invalid dispatch rsp msg
if ((pRsp->msgId != msgId) || (pRsp->stage != pTask->pMeta->stage)) { if ((pRsp->msgId != msgId) || (pRsp->stage != pTask->pMeta->stage)) {
stError("s-task:%s vgId:%d not expect rsp, expected: msgId:%d, stage:%" PRId64 " actual msgId:%d, stage:%" PRId64 stError("s-task:%s vgId:%d not expect rsp, expected: msgId:%d, stage:%" PRId64 " actual msgId:%d, stage:%" PRId64
" discard it", " discard it",
@ -1107,7 +1109,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i
int32_t leftRsp = 0; int32_t leftRsp = 0;
if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); leftRsp = atomic_sub_fetch_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 1);
ASSERT(leftRsp >= 0); ASSERT(leftRsp >= 0);
if (leftRsp > 0) { if (leftRsp > 0) {
@ -1127,18 +1129,17 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i
// all msg rsp already, continue // all msg rsp already, continue
if (leftRsp == 0) { if (leftRsp == 0) {
ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); ASSERT(pTask->outputq.status == TASK_OUTPUT_STATUS__WAIT);
stDebug("s-task:%s output status is set to:%d", id, pTask->outputInfo.status);
// we need to re-try send dispatch msg to downstream tasks // we need to re-try send dispatch msg to downstream tasks
int32_t numOfFailed = taosArrayGetSize(pTask->msgInfo.pRetryList); int32_t numOfFailed = taosArrayGetSize(pTask->msgInfo.pRetryList);
if (numOfFailed > 0) { if (numOfFailed > 0) {
if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, numOfFailed); atomic_store_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, numOfFailed);
stDebug("s-task:%s waiting rsp set to be %d", id, pTask->shuffleDispatcher.waitingRspCnt); stDebug("s-task:%s waiting rsp set to be %d", id, pTask->outputInfo.shuffleDispatcher.waitingRspCnt);
} }
int8_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1);
stDebug("s-task:%s failed to dispatch msg to downstream code:%s, add timer to retry in %dms, ref:%d", stDebug("s-task:%s failed to dispatch msg to downstream code:%s, add timer to retry in %dms, ref:%d",
pTask->id.idStr, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); pTask->id.idStr, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref);
@ -1156,7 +1157,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i
} }
// now ready for next data output // now ready for next data output
atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); atomic_store_8(&pTask->outputq.status, TASK_OUTPUT_STATUS__NORMAL);
} else { } else {
handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId);
} }

View File

@ -17,8 +17,7 @@
// maximum allowed processed block batches. One block may include several submit blocks // maximum allowed processed block batches. One block may include several submit blocks
#define MAX_STREAM_EXEC_BATCH_NUM 32 #define MAX_STREAM_EXEC_BATCH_NUM 32
#define MIN_STREAM_EXEC_BATCH_NUM 4 #define STREAM_RESULT_DUMP_THRESHOLD 300
#define STREAM_RESULT_DUMP_THRESHOLD 100
#define STREAM_RESULT_DUMP_SIZE_THRESHOLD (1048576 * 1) #define STREAM_RESULT_DUMP_SIZE_THRESHOLD (1048576 * 1)
static int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask); static int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask);
@ -37,10 +36,10 @@ static int32_t doOutputResultBlockImpl(SStreamTask* pTask, SStreamDataBlock* pBl
int32_t code = 0; int32_t code = 0;
int32_t type = pTask->outputInfo.type; int32_t type = pTask->outputInfo.type;
if (type == TASK_OUTPUT__TABLE) { if (type == TASK_OUTPUT__TABLE) {
pTask->tbSink.tbSinkFunc(pTask, pTask->tbSink.vnode, pBlock->blocks); pTask->outputInfo.tbSink.tbSinkFunc(pTask, pTask->outputInfo.tbSink.vnode, pBlock->blocks);
destroyStreamDataBlock(pBlock); destroyStreamDataBlock(pBlock);
} else if (type == TASK_OUTPUT__SMA) { } else if (type == TASK_OUTPUT__SMA) {
pTask->smaSink.smaSink(pTask->smaSink.vnode, pTask->smaSink.smaId, pBlock->blocks); pTask->outputInfo.smaSink.smaSink(pTask->outputInfo.smaSink.vnode, pTask->outputInfo.smaSink.smaId, pBlock->blocks);
destroyStreamDataBlock(pBlock); destroyStreamDataBlock(pBlock);
} else { } else {
ASSERT(type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH); ASSERT(type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH);
@ -488,7 +487,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock
// agg task should dispatch trans-state msg to sink task, to flush all data to sink task. // agg task should dispatch trans-state msg to sink task, to flush all data to sink task.
if (level == TASK_LEVEL__AGG || level == TASK_LEVEL__SOURCE) { if (level == TASK_LEVEL__AGG || level == TASK_LEVEL__SOURCE) {
pBlock->srcVgId = pTask->pMeta->vgId; pBlock->srcVgId = pTask->pMeta->vgId;
code = taosWriteQitem(pTask->outputInfo.queue->pQueue, pBlock); code = taosWriteQitem(pTask->outputq.queue->pQueue, pBlock);
if (code == 0) { if (code == 0) {
streamDispatchStreamBlock(pTask); streamDispatchStreamBlock(pTask);
} else { } else {
@ -608,7 +607,7 @@ bool streamTaskIsIdle(const SStreamTask* pTask) {
pTask->status.taskStatus == TASK_STATUS__DROPPING); pTask->status.taskStatus == TASK_STATUS__DROPPING);
} }
int32_t streamTryExec(SStreamTask* pTask) { int32_t streamExecTask(SStreamTask* pTask) {
// this function may be executed by multi-threads, so status check is required. // this function may be executed by multi-threads, so status check is required.
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
@ -616,13 +615,13 @@ int32_t streamTryExec(SStreamTask* pTask) {
if (schedStatus == TASK_SCHED_STATUS__WAITING) { if (schedStatus == TASK_SCHED_STATUS__WAITING) {
while (1) { while (1) {
int32_t code = streamExecForAll(pTask); int32_t code = streamExecForAll(pTask);
if (code < 0) { // todo this status shoudl be removed if (code < 0) { // todo this status should be removed
atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__FAILED); atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__FAILED);
return -1; return -1;
} }
taosThreadMutexLock(&pTask->lock); taosThreadMutexLock(&pTask->lock);
if (taosQueueEmpty(pTask->inputInfo.queue->pQueue) || streamTaskShouldStop(&pTask->status) || if ((streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0) || streamTaskShouldStop(&pTask->status) ||
streamTaskShouldPause(&pTask->status)) { streamTaskShouldPause(&pTask->status)) {
atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE);
taosThreadMutexUnlock(&pTask->lock); taosThreadMutexUnlock(&pTask->lock);
@ -664,7 +663,7 @@ int32_t streamTaskReloadState(SStreamTask* pTask) {
} }
int32_t streamAlignTransferState(SStreamTask* pTask) { int32_t streamAlignTransferState(SStreamTask* pTask) {
int32_t numOfUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t numOfUpstream = taosArrayGetSize(pTask->upstreamInfo.pList);
int32_t old = atomic_val_compare_exchange_32(&pTask->transferStateAlignCnt, 0, numOfUpstream); int32_t old = atomic_val_compare_exchange_32(&pTask->transferStateAlignCnt, 0, numOfUpstream);
if (old == 0) { if (old == 0) {
stDebug("s-task:%s set the transfer state aligncnt %d", pTask->id.idStr, numOfUpstream); stDebug("s-task:%s set the transfer state aligncnt %d", pTask->id.idStr, numOfUpstream);

View File

@ -20,6 +20,7 @@
#include "tref.h" #include "tref.h"
#include "tstream.h" #include "tstream.h"
#include "ttimer.h" #include "ttimer.h"
#include "wal.h"
static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT;
@ -393,29 +394,34 @@ int32_t streamMetaRemoveTask(SStreamMeta* pMeta, STaskId* pTaskId) {
int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded) { int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded) {
*pAdded = false; *pAdded = false;
STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; STaskId id = streamTaskExtractKey(pTask);
void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
if (p == NULL) { if (p != NULL) {
if (pMeta->expandFunc(pMeta->ahandle, pTask, ver) < 0) {
tFreeStreamTask(pTask);
return -1;
}
taosArrayPush(pMeta->pTaskList, &pTask->id);
if (streamMetaSaveTask(pMeta, pTask) < 0) {
tFreeStreamTask(pTask);
return -1;
}
if (streamMetaCommit(pMeta) < 0) {
tFreeStreamTask(pTask);
return -1;
}
} else {
return 0; return 0;
} }
if (pTask->info.fillHistory == 1) {
stDebug("s-task:0x%x initial nextProcessVer is set to 1 for fill-history task", pTask->id.taskId);
ver = 1;
}
if (pMeta->expandFunc(pMeta->ahandle, pTask, ver) < 0) {
tFreeStreamTask(pTask);
return -1;
}
taosArrayPush(pMeta->pTaskList, &pTask->id);
if (streamMetaSaveTask(pMeta, pTask) < 0) {
tFreeStreamTask(pTask);
return -1;
}
if (streamMetaCommit(pMeta) < 0) {
tFreeStreamTask(pTask);
return -1;
}
taosHashPut(pMeta->pTasksMap, &id, sizeof(id), &pTask, POINTER_BYTES); taosHashPut(pMeta->pTasksMap, &id, sizeof(id), &pTask, POINTER_BYTES);
if (pTask->info.fillHistory == 0) { if (pTask->info.fillHistory == 0) {
atomic_add_fetch_32(&pMeta->numOfStreamTasks, 1); atomic_add_fetch_32(&pMeta->numOfStreamTasks, 1);
@ -708,9 +714,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) {
STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId};
void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
if (p == NULL) { if (p == NULL) {
// pTask->chkInfo.checkpointVer may be 0, when a follower is become a leader if (pMeta->expandFunc(pMeta->ahandle, pTask, pTask->chkInfo.checkpointVer + 1) < 0) {
// In this case, we try not to start fill-history task anymore.
if (pMeta->expandFunc(pMeta->ahandle, pTask, pTask->chkInfo.checkpointVer) < 0) {
doClear(pKey, pVal, pCur, pRecycleList); doClear(pKey, pVal, pCur, pRecycleList);
tFreeStreamTask(pTask); tFreeStreamTask(pTask);
return -1; return -1;
@ -718,6 +722,8 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) {
taosArrayPush(pMeta->pTaskList, &pTask->id); taosArrayPush(pMeta->pTaskList, &pTask->id);
} else { } else {
// todo this should replace the existed object put by replay creating stream task msg from mnode
stError("s-task:0x%x already added into table meta by replaying WAL, need check", pTask->id.taskId);
tdbFree(pKey); tdbFree(pKey);
tdbFree(pVal); tdbFree(pVal);
taosMemoryFree(pTask); taosMemoryFree(pTask);
@ -776,6 +782,15 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) {
if (tEncodeI32(pEncoder, ps->status) < 0) return -1; if (tEncodeI32(pEncoder, ps->status) < 0) return -1;
if (tEncodeI32(pEncoder, ps->stage) < 0) return -1; if (tEncodeI32(pEncoder, ps->stage) < 0) return -1;
if (tEncodeI32(pEncoder, ps->nodeId) < 0) return -1; if (tEncodeI32(pEncoder, ps->nodeId) < 0) return -1;
if (tEncodeDouble(pEncoder, ps->inputQUsed) < 0) return -1;
if (tEncodeDouble(pEncoder, ps->inputRate) < 0) return -1;
if (tEncodeDouble(pEncoder, ps->sinkQuota) < 0) return -1;
if (tEncodeDouble(pEncoder, ps->sinkDataSize) < 0) return -1;
if (tEncodeI64(pEncoder, ps->processedVer) < 0) return -1;
if (tEncodeI64(pEncoder, ps->verStart) < 0) return -1;
if (tEncodeI64(pEncoder, ps->verEnd) < 0) return -1;
if (tEncodeI64(pEncoder, ps->activeCheckpointId) < 0) return -1;
if (tEncodeI8(pEncoder, ps->checkpointFailed) < 0) return -1;
} }
tEndEncode(pEncoder); tEndEncode(pEncoder);
return pEncoder->pos; return pEncoder->pos;
@ -796,6 +811,15 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) {
if (tDecodeI32(pDecoder, &entry.status) < 0) return -1; if (tDecodeI32(pDecoder, &entry.status) < 0) return -1;
if (tDecodeI32(pDecoder, &entry.stage) < 0) return -1; if (tDecodeI32(pDecoder, &entry.stage) < 0) return -1;
if (tDecodeI32(pDecoder, &entry.nodeId) < 0) return -1; if (tDecodeI32(pDecoder, &entry.nodeId) < 0) return -1;
if (tDecodeDouble(pDecoder, &entry.inputQUsed) < 0) return -1;
if (tDecodeDouble(pDecoder, &entry.inputRate) < 0) return -1;
if (tDecodeDouble(pDecoder, &entry.sinkQuota) < 0) return -1;
if (tDecodeDouble(pDecoder, &entry.sinkDataSize) < 0) return -1;
if (tDecodeI64(pDecoder, &entry.processedVer) < 0) return -1;
if (tDecodeI64(pDecoder, &entry.verStart) < 0) return -1;
if (tDecodeI64(pDecoder, &entry.verEnd) < 0) return -1;
if (tDecodeI64(pDecoder, &entry.activeCheckpointId) < 0) return -1;
if (tDecodeI8(pDecoder, (int8_t*)&entry.checkpointFailed) < 0) return -1;
entry.id.taskId = taskId; entry.id.taskId = taskId;
taosArrayPush(pReq->pTaskStatus, &entry); taosArrayPush(pReq->pTaskStatus, &entry);
@ -863,12 +887,36 @@ void metaHbToMnode(void* param, void* tmrId) {
STaskId* pId = taosArrayGet(pMeta->pTaskList, i); STaskId* pId = taosArrayGet(pMeta->pTaskList, i);
SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId));
// not report the status of fill-history task
if ((*pTask)->info.fillHistory == 1) { if ((*pTask)->info.fillHistory == 1) {
continue; continue;
} }
STaskStatusEntry entry = { STaskStatusEntry entry = {
.id = *pId, .status = (*pTask)->status.taskStatus, .nodeId = pMeta->vgId, .stage = pMeta->stage}; .id = *pId,
.status = (*pTask)->status.taskStatus,
.nodeId = pMeta->vgId,
.stage = pMeta->stage,
.inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputInfo.queue)),
};
entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE;
if ((*pTask)->info.taskLevel == TASK_LEVEL__SINK) {
entry.sinkQuota = (*pTask)->outputInfo.pTokenBucket->quotaRate;
entry.sinkDataSize = SIZE_IN_MiB((*pTask)->execInfo.sink.dataSize);
}
if ((*pTask)->checkpointingId != 0) {
entry.checkpointFailed = ((*pTask)->chkInfo.failedId >= (*pTask)->checkpointingId);
entry.activeCheckpointId = (*pTask)->checkpointingId;
}
if ((*pTask)->exec.pWalReader != NULL) {
entry.processedVer = (*pTask)->chkInfo.nextProcessVer - 1;
walReaderValidVersionRange((*pTask)->exec.pWalReader, &entry.verStart, &entry.verEnd);
}
taosArrayPush(hbMsg.pTaskStatus, &entry); taosArrayPush(hbMsg.pTaskStatus, &entry);
if (!hasValEpset) { if (!hasValEpset) {
@ -1005,3 +1053,8 @@ void streamMetaInitForSnode(SStreamMeta* pMeta) {
pMeta->stage = 0; pMeta->stage = 0;
pMeta->role = NODE_ROLE_LEADER; pMeta->role = NODE_ROLE_LEADER;
} }
void streamMetaResetStartInfo(STaskStartInfo* pStartInfo) {
taosHashClear(pStartInfo->pReadyTaskSet);
pStartInfo->startedAfterNodeUpdate = 0;
}

View File

@ -16,11 +16,9 @@
#include "streamInt.h" #include "streamInt.h"
#define MAX_STREAM_EXEC_BATCH_NUM 32 #define MAX_STREAM_EXEC_BATCH_NUM 32
#define MIN_STREAM_EXEC_BATCH_NUM 4 #define MAX_SMOOTH_BURST_RATIO 5 // 5 sec
#define STREAM_TASK_QUEUE_CAPACITY 20480 #define WAIT_FOR_DURATION 40
#define STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE (30) #define SINK_TASK_IDLE_DURATION 200 // 200 ms
#define STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE (50)
#define MAX_SMOOTH_BURST_RATIO 5 // 20 sec
// todo refactor: // todo refactor:
// read data from input queue // read data from input queue
@ -31,7 +29,7 @@ typedef struct SQueueReader {
int32_t waitDuration; // maximum wait time to format several block into a batch to process, unit: ms int32_t waitDuration; // maximum wait time to format several block into a batch to process, unit: ms
} SQueueReader; } SQueueReader;
static bool streamTaskExtractAvailableToken(STokenBucket* pBucket); static bool streamTaskExtractAvailableToken(STokenBucket* pBucket, const char* id);
static void streamTaskPutbackToken(STokenBucket* pBucket); static void streamTaskPutbackToken(STokenBucket* pBucket);
static void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes); static void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes);
@ -105,15 +103,13 @@ void streamQueueProcessFail(SStreamQueue* queue) {
atomic_store_8(&queue->status, STREAM_QUEUE__FAILED); atomic_store_8(&queue->status, STREAM_QUEUE__FAILED);
} }
bool streamQueueIsFull(const STaosQueue* pQueue, bool inputQ) { bool streamQueueIsFull(const SStreamQueue* pQueue) {
bool isFull = taosQueueItemSize((STaosQueue*)pQueue) >= STREAM_TASK_QUEUE_CAPACITY; int32_t numOfItems = streamQueueGetNumOfItems(pQueue);
if (isFull) { if (numOfItems >= STREAM_TASK_QUEUE_CAPACITY) {
return true; return true;
} }
int32_t threahold = (inputQ) ? STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE : STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; return (SIZE_IN_MiB(taosQueueMemorySize(pQueue->pQueue)) >= STREAM_TASK_QUEUE_CAPACITY_IN_SIZE);
double size = SIZE_IN_MiB(taosQueueMemorySize((STaosQueue*)pQueue));
return (size >= threahold);
} }
int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) { int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) {
@ -123,8 +119,17 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) {
return numOfItems1 + numOfItems2; return numOfItems1 + numOfItems2;
} }
int32_t streamQueueGetNumOfItemsInQueue(const SStreamQueue* pQueue) { int32_t streamQueueGetAvailableSpace(const SStreamQueue* pQueue, int32_t* availNum, double* availSize) {
return taosQueueItemSize(pQueue->pQueue); int32_t num = streamQueueGetNumOfItems(pQueue);
*availNum = STREAM_TASK_QUEUE_CAPACITY - num;
*availSize = STREAM_TASK_QUEUE_CAPACITY_IN_SIZE - taosQueueMemorySize(pQueue->pQueue);
return 0;
}
// todo: fix it: data in Qall is not included here
int32_t streamQueueGetItemSize(const SStreamQueue* pQueue) {
return taosQueueMemorySize(pQueue->pQueue);
} }
int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem) { int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem) {
@ -162,8 +167,8 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
*blockSize = 0; *blockSize = 0;
// no available token in bucket for sink task, let's wait for a little bit // no available token in bucket for sink task, let's wait for a little bit
if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->pTokenBucket))) { if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->outputInfo.pTokenBucket, pTask->id.idStr))) {
stDebug("s-task:%s no available token in bucket for sink data, wait", id); stDebug("s-task:%s no available token in bucket for sink data, wait for 50ms", id);
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
@ -176,7 +181,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue);
if (qItem == NULL) { if (qItem == NULL) {
if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) { if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) {
taosMsleep(10); taosMsleep(WAIT_FOR_DURATION);
continue; continue;
} }
@ -184,10 +189,10 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
if (*numOfBlocks > 0) { if (*numOfBlocks > 0) {
*blockSize = streamQueueItemGetSize(*pInput); *blockSize = streamQueueItemGetSize(*pInput);
if (taskLevel == TASK_LEVEL__SINK) { if (taskLevel == TASK_LEVEL__SINK) {
streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize);
} }
} else { } else {
streamTaskPutbackToken(pTask->pTokenBucket); streamTaskPutbackToken(pTask->outputInfo.pTokenBucket);
} }
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
@ -203,7 +208,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
stDebug("s-task:%s %s msg extracted, start to process immediately", id, p); stDebug("s-task:%s %s msg extracted, start to process immediately", id, p);
// restore the token to bucket in case of checkpoint/trans-state msg // restore the token to bucket in case of checkpoint/trans-state msg
streamTaskPutbackToken(pTask->pTokenBucket); streamTaskPutbackToken(pTask->outputInfo.pTokenBucket);
*blockSize = 0; *blockSize = 0;
*numOfBlocks = 1; *numOfBlocks = 1;
*pInput = qItem; *pInput = qItem;
@ -212,7 +217,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
stDebug("s-task:%s %s msg extracted, handle previous blocks, numOfBlocks:%d", id, p, *numOfBlocks); stDebug("s-task:%s %s msg extracted, handle previous blocks, numOfBlocks:%d", id, p, *numOfBlocks);
*blockSize = streamQueueItemGetSize(*pInput); *blockSize = streamQueueItemGetSize(*pInput);
if (taskLevel == TASK_LEVEL__SINK) { if (taskLevel == TASK_LEVEL__SINK) {
streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize);
} }
streamQueueProcessFail(pTask->inputInfo.queue); streamQueueProcessFail(pTask->inputInfo.queue);
@ -233,7 +238,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
*blockSize = streamQueueItemGetSize(*pInput); *blockSize = streamQueueItemGetSize(*pInput);
if (taskLevel == TASK_LEVEL__SINK) { if (taskLevel == TASK_LEVEL__SINK) {
streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize);
} }
streamQueueProcessFail(pTask->inputInfo.queue); streamQueueProcessFail(pTask->inputInfo.queue);
@ -251,7 +256,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
*blockSize = streamQueueItemGetSize(*pInput); *blockSize = streamQueueItemGetSize(*pInput);
if (taskLevel == TASK_LEVEL__SINK) { if (taskLevel == TASK_LEVEL__SINK) {
streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize);
} }
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
@ -267,11 +272,11 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem)
if (type == STREAM_INPUT__DATA_SUBMIT) { if (type == STREAM_INPUT__DATA_SUBMIT) {
SStreamDataSubmit* px = (SStreamDataSubmit*)pItem; SStreamDataSubmit* px = (SStreamDataSubmit*)pItem;
if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && streamQueueIsFull(pQueue, true)) { if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && streamQueueIsFull(pTask->inputInfo.queue)) {
double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue));
stTrace( stTrace(
"s-task:%s inputQ is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data", "s-task:%s inputQ is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data",
pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_QUEUE_CAPACITY_IN_SIZE, total, size);
streamDataSubmitDestroy(px); streamDataSubmitDestroy(px);
taosFreeQitem(pItem); taosFreeQitem(pItem);
return -1; return -1;
@ -294,11 +299,11 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem)
msgLen, ver, total, size + SIZE_IN_MiB(msgLen)); msgLen, ver, total, size + SIZE_IN_MiB(msgLen));
} 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) {
if (streamQueueIsFull(pQueue, true)) { if (streamQueueIsFull(pTask->inputInfo.queue)) {
double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue));
stTrace("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", stTrace("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort",
pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_QUEUE_CAPACITY_IN_SIZE, total, size);
destroyStreamDataBlock((SStreamDataBlock*)pItem); destroyStreamDataBlock((SStreamDataBlock*)pItem);
return -1; return -1;
} }
@ -346,15 +351,15 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem)
// the result should be put into the outputQ in any cases, otherwise, the result may be lost // the result should be put into the outputQ in any cases, otherwise, the result may be lost
int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) { int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) {
STaosQueue* pQueue = pTask->outputInfo.queue->pQueue; STaosQueue* pQueue = pTask->outputq.queue->pQueue;
while (streamQueueIsFull(pQueue, false)) { while (streamQueueIsFull(pTask->outputq.queue)) {
if (streamTaskShouldStop(&pTask->status)) { if (streamTaskShouldStop(&pTask->status)) {
stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr);
return TSDB_CODE_STREAM_EXEC_CANCELLED; return TSDB_CODE_STREAM_EXEC_CANCELLED;
} }
int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); int32_t total = streamQueueGetNumOfItems(pTask->outputq.queue);
double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue));
// let's wait for there are enough space to hold this result pBlock // let's wait for there are enough space to hold this result pBlock
stDebug("s-task:%s outputQ is full, wait for 500ms and retry, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, stDebug("s-task:%s outputQ is full, wait for 500ms and retry, outputQ items:%d, size:%.2fMiB", pTask->id.idStr,
@ -364,7 +369,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc
int32_t code = taosWriteQitem(pQueue, pBlock); int32_t code = taosWriteQitem(pQueue, pBlock);
int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); int32_t total = streamQueueGetNumOfItems(pTask->outputq.queue);
double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue));
if (code != 0) { if (code != 0) {
stError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost", stError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost",
@ -376,7 +381,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t bytesRate) { int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t quotaRate) {
if (numCap < 10 || numRate < 10 || pBucket == NULL) { if (numCap < 10 || numRate < 10 || pBucket == NULL) {
stError("failed to init sink task bucket, cap:%d, rate:%d", numCap, numRate); stError("failed to init sink task bucket, cap:%d, rate:%d", numCap, numRate);
return TSDB_CODE_INVALID_PARA; return TSDB_CODE_INVALID_PARA;
@ -386,15 +391,15 @@ int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t
pBucket->numOfToken = numCap; pBucket->numOfToken = numCap;
pBucket->numRate = numRate; pBucket->numRate = numRate;
pBucket->bytesRate = bytesRate; pBucket->quotaRate = quotaRate;
pBucket->bytesCapacity = bytesRate * MAX_SMOOTH_BURST_RATIO; pBucket->quotaCapacity = quotaRate * MAX_SMOOTH_BURST_RATIO;
pBucket->bytesRemain = pBucket->bytesCapacity; pBucket->quotaRemain = pBucket->quotaCapacity;
pBucket->fillTimestamp = taosGetTimestampMs(); pBucket->fillTimestamp = taosGetTimestampMs();
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
static void fillTokenBucket(STokenBucket* pBucket) { static void fillTokenBucket(STokenBucket* pBucket, const char* id) {
int64_t now = taosGetTimestampMs(); int64_t now = taosGetTimestampMs();
int64_t delta = now - pBucket->fillTimestamp; int64_t delta = now - pBucket->fillTimestamp;
ASSERT(pBucket->numOfToken >= 0); ASSERT(pBucket->numOfToken >= 0);
@ -406,23 +411,23 @@ static void fillTokenBucket(STokenBucket* pBucket) {
} }
// increase the new available quota as time goes on // increase the new available quota as time goes on
double incSize = (delta / 1000.0) * pBucket->bytesRate; double incSize = (delta / 1000.0) * pBucket->quotaRate;
if (incSize > 0) { if (incSize > 0) {
pBucket->bytesRemain = TMIN(pBucket->bytesRemain + incSize, pBucket->bytesCapacity); pBucket->quotaRemain = TMIN(pBucket->quotaRemain + incSize, pBucket->quotaCapacity);
} }
if (incNum > 0) { if (incNum > 0 || incSize > 0) {
stDebug("new token and capacity available, current token:%d inc:%d, current quota:%.2fMiB inc:%.2fMiB, ts:%" PRId64 stDebug("new token and capacity available, current token:%d inc:%d, current quota:%.2fMiB inc:%.2fMiB, ts:%" PRId64
" wait for %.2f Sec", " idle for %.2f Sec, %s",
pBucket->numOfToken, incNum, pBucket->bytesRemain, incSize, now, delta / 1000.0); pBucket->numOfToken, incNum, pBucket->quotaRemain, incSize, now, delta / 1000.0, id);
} }
} }
bool streamTaskExtractAvailableToken(STokenBucket* pBucket) { bool streamTaskExtractAvailableToken(STokenBucket* pBucket, const char* id) {
fillTokenBucket(pBucket); fillTokenBucket(pBucket, id);
if (pBucket->numOfToken > 0) { if (pBucket->numOfToken > 0) {
if (pBucket->bytesRemain > 0) { if (pBucket->quotaRemain > 0) {
pBucket->numOfToken -= 1; pBucket->numOfToken -= 1;
return true; return true;
} else { // no available size quota now } else { // no available size quota now
@ -439,5 +444,5 @@ void streamTaskPutbackToken(STokenBucket* pBucket) {
// size in KB // size in KB
void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes) { void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes) {
pBucket->bytesRemain -= SIZE_IN_MiB(bytes); pBucket->quotaRemain -= SIZE_IN_MiB(bytes);
} }

View File

@ -40,7 +40,7 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) {
int32_t vgId = pMeta->vgId; int32_t vgId = pMeta->vgId;
if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) {
pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->upstreamInfo.pList);
stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s",
pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream,
streamGetTaskStatusStr(pTask->status.taskStatus)); streamGetTaskStatusStr(pTask->status.taskStatus));
@ -58,17 +58,27 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) {
STaskId id = streamTaskExtractKey(pTask); STaskId id = streamTaskExtractKey(pTask);
taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0); taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0);
int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta);
if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) { if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) {
// reset value for next time start STaskStartInfo* pStartInfo = &pMeta->startInfo;
taosHashClear(pMeta->startInfo.pReadyTaskSet); pStartInfo->readyTs = pTask->execInfo.start;
pMeta->startInfo.startedAfterNodeUpdate = 0;
pMeta->startInfo.elapsedTime = pTask->execInfo.start - pMeta->startInfo.ts;
stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, total elapsed time:%.2f sec", if (pStartInfo->startTs != 0) {
vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pMeta->startInfo.elapsedTime / 1000.0); pStartInfo->elapsedTime = pStartInfo->readyTs - pStartInfo->startTs;
} else {
pStartInfo->elapsedTime = 0;
}
streamMetaResetStartInfo(pStartInfo);
stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, startTs:%" PRId64
", readyTs:%" PRId64 " total elapsed time:%.2fs",
vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pStartInfo->startTs, pStartInfo->readyTs,
pStartInfo->elapsedTime / 1000.0);
} }
taosWUnLockLatch(&pMeta->lock); taosWUnLockLatch(&pMeta->lock);
} }
@ -92,7 +102,7 @@ int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) {
return 0; return 0;
} }
static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { static int32_t doStartScanHistoryTask(SStreamTask* pTask) {
SVersionRange* pRange = &pTask->dataRange.range; SVersionRange* pRange = &pTask->dataRange.range;
if (pTask->info.fillHistory) { if (pTask->info.fillHistory) {
streamSetParamForScanHistory(pTask); streamSetParamForScanHistory(pTask);
@ -103,15 +113,18 @@ static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) {
return code; return code;
} }
int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { int32_t streamTaskStartScanHistory(SStreamTask* pTask) {
ASSERT(pTask->status.downstreamReady == 1);
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) {
return doLaunchScanHistoryTask(pTask); return doStartScanHistoryTask(pTask);
} else { } else {
ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL); ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL);
stDebug("s-task:%s no need to scan-history-data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr, stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr,
streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus,
walReaderGetCurrentVer(pTask->exec.pWalReader)); walReaderGetCurrentVer(pTask->exec.pWalReader));
streamTaskEnablePause(pTask);
} }
} else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) {
if (pTask->info.fillHistory) { if (pTask->info.fillHistory) {
@ -119,7 +132,9 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) {
streamTaskEnablePause(pTask); streamTaskEnablePause(pTask);
} }
} else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) {
stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) {
stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr);
}
} }
return 0; return 0;
} }
@ -140,18 +155,18 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) {
// serialize streamProcessScanHistoryFinishRsp // serialize streamProcessScanHistoryFinishRsp
if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) {
req.reqId = tGenIdPI64(); req.reqId = tGenIdPI64();
req.downstreamNodeId = pTask->fixedDispatcher.nodeId; req.downstreamNodeId = pTask->outputInfo.fixedDispatcher.nodeId;
req.downstreamTaskId = pTask->fixedDispatcher.taskId; req.downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId;
pTask->checkReqId = req.reqId; pTask->checkReqId = req.reqId;
stDebug("s-task:%s stage:%" PRId64 " check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 stDebug("s-task:%s (vgId:%d) stage:%" PRId64 " check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64
" window:%" PRId64 "-%" PRId64 " req:0x%" PRIx64, " window:%" PRId64 "-%" PRId64 " req:0x%" PRIx64,
pTask->id.idStr, req.reqId, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, pTask->id.idStr, pTask->info.nodeId, req.stage, req.downstreamTaskId, req.downstreamNodeId,
pRange->range.maxVer, pWindow->skey, pWindow->ekey, req.reqId); pRange->range.minVer, pRange->range.maxVer, pWindow->skey, pWindow->ekey, req.reqId);
streamSendCheckMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); streamSendCheckMsg(pTask, &req, pTask->outputInfo.fixedDispatcher.nodeId, &pTask->outputInfo.fixedDispatcher.epSet);
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
int32_t numOfVgs = taosArrayGetSize(vgInfo); int32_t numOfVgs = taosArrayGetSize(vgInfo);
pTask->notReadyTasks = numOfVgs; pTask->notReadyTasks = numOfVgs;
@ -175,7 +190,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) {
streamTaskSetReady(pTask, 0); streamTaskSetReady(pTask, 0);
streamTaskSetRangeStreamCalc(pTask); streamTaskSetRangeStreamCalc(pTask);
streamTaskLaunchScanHistory(pTask); streamTaskStartScanHistory(pTask);
streamLaunchFillHistoryTask(pTask); streamLaunchFillHistoryTask(pTask);
} }
@ -221,9 +236,9 @@ static void recheckDownstreamTasks(void* param, void* tmrId) {
if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) {
stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr,
pTask->info.nodeId, pReq->downstreamTaskId, pReq->downstreamNodeId, pReq->stage); pTask->info.nodeId, pReq->downstreamTaskId, pReq->downstreamNodeId, pReq->stage);
streamSendCheckMsg(pTask, pReq, pReq->downstreamNodeId, &pTask->fixedDispatcher.epSet); streamSendCheckMsg(pTask, pReq, pReq->downstreamNodeId, &pTask->outputInfo.fixedDispatcher.epSet);
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
int32_t numOfVgs = taosArrayGetSize(vgInfo); int32_t numOfVgs = taosArrayGetSize(vgInfo);
for (int32_t i = 0; i < numOfVgs; i++) { for (int32_t i = 0; i < numOfVgs; i++) {
@ -237,7 +252,7 @@ static void recheckDownstreamTasks(void* param, void* tmrId) {
} }
destroyRecheckInfo(pInfo); destroyRecheckInfo(pInfo);
int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1);
stDebug("s-task:%s complete send check in timer, ref:%d", pTask->id.idStr, ref); stDebug("s-task:%s complete send check in timer, ref:%d", pTask->id.idStr, ref);
} }
@ -285,8 +300,9 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) {
if (status == TASK_STATUS__SCAN_HISTORY) { if (status == TASK_STATUS__SCAN_HISTORY) {
stDebug("s-task:%s enter into scan-history data stage, status:%s", id, str); stDebug("s-task:%s enter into scan-history data stage, status:%s", id, str);
streamTaskLaunchScanHistory(pTask); streamTaskStartScanHistory(pTask);
} else { } else {
// fill-history tasks are not allowed to reach here.
if (pTask->info.fillHistory == 1) { if (pTask->info.fillHistory == 1) {
stDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); stDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id);
pTask->status.taskStatus = TASK_STATUS__DROPPING; pTask->status.taskStatus = TASK_STATUS__DROPPING;
@ -301,7 +317,6 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) {
streamLaunchFillHistoryTask(pTask); streamLaunchFillHistoryTask(pTask);
} }
// todo handle error
int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) {
ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); ASSERT(pTask->id.taskId == pRsp->upstreamTaskId);
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
@ -337,7 +352,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs
doProcessDownstreamReadyRsp(pTask, numOfReqs); doProcessDownstreamReadyRsp(pTask, numOfReqs);
} else { } else {
int32_t total = taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); int32_t total = taosArrayGetSize(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos);
stDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, stDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id,
pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left); pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left);
} }
@ -363,7 +378,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs
} else { } else {
STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp);
int8_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1);
stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms, ref:%d ", id, stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms, ref:%d ", id,
pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref); pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref);
@ -524,7 +539,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory
ASSERT(left >= 0); ASSERT(left >= 0);
if (left == 0) { if (left == 0) {
int32_t numOfTasks = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t numOfTasks = taosArrayGetSize(pTask->upstreamInfo.pList);
stDebug( stDebug(
"s-task:%s all %d upstream tasks finish scan-history data, set param for agg task for stream data and send " "s-task:%s all %d upstream tasks finish scan-history data, set param for agg task for stream data and send "
"rsp to all upstream tasks", "rsp to all upstream tasks",
@ -580,15 +595,18 @@ int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) {
} }
static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) { static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) {
pHTask->dataRange.range.minVer = 0; SDataRange* pRange = &pHTask->dataRange;
// the query version range should be limited to the already processed data // the query version range should be limited to the already processed data
pHTask->dataRange.range.maxVer = pTask->chkInfo.nextProcessVer - 1; pRange->range.minVer = 0;
pRange->range.maxVer = pTask->chkInfo.nextProcessVer - 1;
pHTask->execInfo.init = taosGetTimestampMs();
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
stDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64 stDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64
" ver range:%" PRId64 " - %" PRId64, " ver range:%" PRId64 " - %" PRId64", init:%"PRId64,
pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, pTask->id.idStr, pHTask->id.idStr, pRange->window.skey, pRange->window.ekey,
pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); pRange->range.minVer, pRange->range.maxVer, pHTask->execInfo.init);
} else { } else {
stDebug("s-task:%s no fill history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr); stDebug("s-task:%s no fill history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr);
} }
@ -633,7 +651,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) {
} }
if (pHTaskInfo->retryTimes > MAX_RETRY_LAUNCH_HISTORY_TASK) { if (pHTaskInfo->retryTimes > MAX_RETRY_LAUNCH_HISTORY_TASK) {
int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1);
streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pTask);
stError("s-task:%s max retry:%d reached, quit from retrying launch related fill-history task:0x%x, ref:%d", stError("s-task:%s max retry:%d reached, quit from retrying launch related fill-history task:0x%x, ref:%d",
@ -665,7 +683,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) {
} }
// not in timer anymore // not in timer anymore
int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1);
stDebug("s-task:0x%x fill-history task launch completed, retry times:%d, ref:%d", (int32_t)pInfo->id.taskId, stDebug("s-task:0x%x fill-history task launch completed, retry times:%d, ref:%d", (int32_t)pInfo->id.taskId,
pHTaskInfo->retryTimes, ref); pHTaskInfo->retryTimes, ref);
streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pTask);
@ -719,7 +737,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) {
if (pTask->hTaskInfo.pTimer == NULL) { if (pTask->hTaskInfo.pTimer == NULL) {
int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1);
pTask->hTaskInfo.pTimer = taosTmrStart(tryLaunchHistoryTask, WAIT_FOR_MINIMAL_INTERVAL, pInfo, streamEnv.timer); pTask->hTaskInfo.pTimer = taosTmrStart(tryLaunchHistoryTask, WAIT_FOR_MINIMAL_INTERVAL, pInfo, streamEnv.timer);
if (pTask->hTaskInfo.pTimer == NULL) { // todo failed to create timer if (pTask->hTaskInfo.pTimer == NULL) {
atomic_sub_fetch_32(&pTask->status.timerActive, 1); atomic_sub_fetch_32(&pTask->status.timerActive, 1);
stError("s-task:%s failed to start timer, related fill-history task not launched, ref:%d", pTask->id.idStr, stError("s-task:%s failed to start timer, related fill-history task not launched, ref:%d", pTask->id.idStr,
pTask->status.timerActive); pTask->status.timerActive);
@ -774,24 +792,24 @@ int32_t streamTaskFillHistoryFinished(SStreamTask* pTask) {
return qStreamInfoResetTimewindowFilter(exec); return qStreamInfoResetTimewindowFilter(exec);
} }
bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t latestVer) { bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t nextProcessVer) {
SVersionRange* pRange = &pTask->dataRange.range; SVersionRange* pRange = &pTask->dataRange.range;
ASSERT(latestVer >= pRange->maxVer); ASSERT(nextProcessVer >= pRange->maxVer);
int64_t nextStartVer = pRange->maxVer + 1; int64_t walScanStartVer = pRange->maxVer + 1;
if (nextStartVer > latestVer - 1) { if (walScanStartVer > nextProcessVer - 1) {
// no input data yet. no need to execute the secondardy scan while stream task halt // no input data yet. no need to execute the secondary scan while stream task halt
streamTaskFillHistoryFinished(pTask); streamTaskFillHistoryFinished(pTask);
stDebug( stDebug(
"s-task:%s no need to perform secondary scan-history data(step 2), since no data ingest during step1 scan, " "s-task:%s no need to perform secondary scan-history data(step 2), since no data ingest during step1 scan, "
"related stream task currentVer:%" PRId64, "related stream task currentVer:%" PRId64,
pTask->id.idStr, latestVer); pTask->id.idStr, nextProcessVer);
return true; return true;
} else { } else {
// 2. do secondary scan of the history data, the time window remain, and the version range is updated to // 2. do secondary scan of the history data, the time window remain, and the version range is updated to
// [pTask->dataRange.range.maxVer, ver1] // [pTask->dataRange.range.maxVer, ver1]
pRange->minVer = nextStartVer; pRange->minVer = walScanStartVer;
pRange->maxVer = latestVer - 1; pRange->maxVer = nextProcessVer - 1;
return false; return false;
} }
} }
@ -877,20 +895,19 @@ int32_t tDecodeStreamScanHistoryFinishReq(SDecoder* pDecoder, SStreamScanHistory
} }
void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { void streamTaskSetRangeStreamCalc(SStreamTask* pTask) {
SDataRange* pRange = &pTask->dataRange;
if (pTask->hTaskInfo.id.taskId == 0) { if (pTask->hTaskInfo.id.taskId == 0) {
SDataRange* pRange = &pTask->dataRange;
if (pTask->info.fillHistory == 1) { if (pTask->info.fillHistory == 1) {
stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64,
"-%" PRId64, pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer);
pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer);
} else { } else {
stDebug("s-task:%s no related fill-history task, stream time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 stDebug(
"-%" PRId64, "s-task:%s no related fill-history task, stream time window and verRange are not set. default stream time "
pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); "window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64,
pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer);
} }
} else { } else {
SDataRange* pRange = &pTask->dataRange;
int64_t ekey = 0; int64_t ekey = 0;
if (pRange->window.ekey < INT64_MAX) { if (pRange->window.ekey < INT64_MAX) {
ekey = pRange->window.ekey + 1; ekey = pRange->window.ekey + 1;
@ -906,9 +923,9 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) {
pRange->range.maxVer = ver; pRange->range.maxVer = ver;
stDebug("s-task:%s level:%d related fill-history task exists, update stream calc time window:%" PRId64 " - %" PRId64 stDebug("s-task:%s level:%d related fill-history task exists, update stream calc time window:%" PRId64 " - %" PRId64
", verRang:%" PRId64 " - %" PRId64, ", verRang:%" PRId64 " - %" PRId64,
pTask->id.idStr, pTask->info.taskLevel, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pTask->id.idStr, pTask->info.taskLevel, pRange->window.skey, pRange->window.ekey, pRange->range.minVer,
pRange->range.maxVer); pRange->range.maxVer);
} }
} }

View File

@ -20,6 +20,8 @@
#include "ttimer.h" #include "ttimer.h"
#include "wal.h" #include "wal.h"
static void streamTaskDestroyUpstreamInfo(SUpstreamInfo* pUpstreamInfo);
static int32_t addToTaskset(SArray* pArray, SStreamTask* pTask) { static int32_t addToTaskset(SArray* pArray, SStreamTask* pTask) {
int32_t childId = taosArrayGetSize(pArray); int32_t childId = taosArrayGetSize(pArray);
pTask->info.selfChildId = childId; pTask->info.selfChildId = childId;
@ -27,8 +29,8 @@ static int32_t addToTaskset(SArray* pArray, SStreamTask* pTask) {
return 0; return 0;
} }
SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHistory, int64_t triggerParam, SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory, int64_t triggerParam,
SArray* pTaskList) { SArray* pTaskList, bool hasFillhistory) {
SStreamTask* pTask = (SStreamTask*)taosMemoryCalloc(1, sizeof(SStreamTask)); SStreamTask* pTask = (SStreamTask*)taosMemoryCalloc(1, sizeof(SStreamTask));
if (pTask == NULL) { if (pTask == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
@ -46,9 +48,13 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHisto
pTask->id.idStr = taosStrdup(buf); pTask->id.idStr = taosStrdup(buf);
pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE;
pTask->status.taskStatus = TASK_STATUS__SCAN_HISTORY; pTask->status.taskStatus = (fillHistory || hasFillhistory)? TASK_STATUS__SCAN_HISTORY:TASK_STATUS__NORMAL;
pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL;
pTask->outputInfo.status = TASK_OUTPUT_STATUS__NORMAL; pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL;
if (fillHistory) {
ASSERT(hasFillhistory);
}
addToTaskset(pTaskList, pTask); addToTaskset(pTaskList, pTask);
return pTask; return pTask;
@ -109,10 +115,10 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) {
if (tEncodeI64(pEncoder, pTask->dataRange.window.skey)) return -1; if (tEncodeI64(pEncoder, pTask->dataRange.window.skey)) return -1;
if (tEncodeI64(pEncoder, pTask->dataRange.window.ekey)) return -1; if (tEncodeI64(pEncoder, pTask->dataRange.window.ekey)) return -1;
int32_t epSz = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t epSz = taosArrayGetSize(pTask->upstreamInfo.pList);
if (tEncodeI32(pEncoder, epSz) < 0) return -1; if (tEncodeI32(pEncoder, epSz) < 0) return -1;
for (int32_t i = 0; i < epSz; i++) { for (int32_t i = 0; i < epSz; i++) {
SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i);
if (tEncodeStreamEpInfo(pEncoder, pInfo) < 0) return -1; if (tEncodeStreamEpInfo(pEncoder, pInfo) < 0) return -1;
} }
@ -121,20 +127,20 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) {
} }
if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) {
if (tEncodeI64(pEncoder, pTask->tbSink.stbUid) < 0) return -1; if (tEncodeI64(pEncoder, pTask->outputInfo.tbSink.stbUid) < 0) return -1;
if (tEncodeCStr(pEncoder, pTask->tbSink.stbFullName) < 0) return -1; if (tEncodeCStr(pEncoder, pTask->outputInfo.tbSink.stbFullName) < 0) return -1;
if (tEncodeSSchemaWrapper(pEncoder, pTask->tbSink.pSchemaWrapper) < 0) return -1; if (tEncodeSSchemaWrapper(pEncoder, pTask->outputInfo.tbSink.pSchemaWrapper) < 0) return -1;
} else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { } else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) {
if (tEncodeI64(pEncoder, pTask->smaSink.smaId) < 0) return -1; if (tEncodeI64(pEncoder, pTask->outputInfo.smaSink.smaId) < 0) return -1;
} else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) {
if (tEncodeI8(pEncoder, pTask->fetchSink.reserved) < 0) return -1; if (tEncodeI8(pEncoder, pTask->outputInfo.fetchSink.reserved) < 0) return -1;
} else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) {
if (tEncodeI32(pEncoder, pTask->fixedDispatcher.taskId) < 0) return -1; if (tEncodeI32(pEncoder, pTask->outputInfo.fixedDispatcher.taskId) < 0) return -1;
if (tEncodeI32(pEncoder, pTask->fixedDispatcher.nodeId) < 0) return -1; if (tEncodeI32(pEncoder, pTask->outputInfo.fixedDispatcher.nodeId) < 0) return -1;
if (tEncodeSEpSet(pEncoder, &pTask->fixedDispatcher.epSet) < 0) return -1; if (tEncodeSEpSet(pEncoder, &pTask->outputInfo.fixedDispatcher.epSet) < 0) return -1;
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
if (tSerializeSUseDbRspImp(pEncoder, &pTask->shuffleDispatcher.dbInfo) < 0) return -1; if (tSerializeSUseDbRspImp(pEncoder, &pTask->outputInfo.shuffleDispatcher.dbInfo) < 0) return -1;
if (tEncodeCStr(pEncoder, pTask->shuffleDispatcher.stbFullName) < 0) return -1; if (tEncodeCStr(pEncoder, pTask->outputInfo.shuffleDispatcher.stbFullName) < 0) return -1;
} }
if (tEncodeI64(pEncoder, pTask->info.triggerParam) < 0) return -1; if (tEncodeI64(pEncoder, pTask->info.triggerParam) < 0) return -1;
if (tEncodeCStrWithLen(pEncoder, pTask->reserve, sizeof(pTask->reserve) - 1) < 0) return -1; if (tEncodeCStrWithLen(pEncoder, pTask->reserve, sizeof(pTask->reserve) - 1) < 0) return -1;
@ -185,7 +191,7 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) {
int32_t epSz = -1; int32_t epSz = -1;
if (tDecodeI32(pDecoder, &epSz) < 0) return -1; if (tDecodeI32(pDecoder, &epSz) < 0) return -1;
pTask->pUpstreamInfoList = taosArrayInit(epSz, POINTER_BYTES); pTask->upstreamInfo.pList = taosArrayInit(epSz, POINTER_BYTES);
for (int32_t i = 0; i < epSz; i++) { for (int32_t i = 0; i < epSz; i++) {
SStreamChildEpInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamChildEpInfo)); SStreamChildEpInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamChildEpInfo));
if (pInfo == NULL) return -1; if (pInfo == NULL) return -1;
@ -193,7 +199,7 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) {
taosMemoryFreeClear(pInfo); taosMemoryFreeClear(pInfo);
return -1; return -1;
} }
taosArrayPush(pTask->pUpstreamInfoList, &pInfo); taosArrayPush(pTask->upstreamInfo.pList, &pInfo);
} }
if (pTask->info.taskLevel != TASK_LEVEL__SINK) { if (pTask->info.taskLevel != TASK_LEVEL__SINK) {
@ -201,22 +207,22 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) {
} }
if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) {
if (tDecodeI64(pDecoder, &pTask->tbSink.stbUid) < 0) return -1; if (tDecodeI64(pDecoder, &pTask->outputInfo.tbSink.stbUid) < 0) return -1;
if (tDecodeCStrTo(pDecoder, pTask->tbSink.stbFullName) < 0) return -1; if (tDecodeCStrTo(pDecoder, pTask->outputInfo.tbSink.stbFullName) < 0) return -1;
pTask->tbSink.pSchemaWrapper = taosMemoryCalloc(1, sizeof(SSchemaWrapper)); pTask->outputInfo.tbSink.pSchemaWrapper = taosMemoryCalloc(1, sizeof(SSchemaWrapper));
if (pTask->tbSink.pSchemaWrapper == NULL) return -1; if (pTask->outputInfo.tbSink.pSchemaWrapper == NULL) return -1;
if (tDecodeSSchemaWrapper(pDecoder, pTask->tbSink.pSchemaWrapper) < 0) return -1; if (tDecodeSSchemaWrapper(pDecoder, pTask->outputInfo.tbSink.pSchemaWrapper) < 0) return -1;
} else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { } else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) {
if (tDecodeI64(pDecoder, &pTask->smaSink.smaId) < 0) return -1; if (tDecodeI64(pDecoder, &pTask->outputInfo.smaSink.smaId) < 0) return -1;
} else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) {
if (tDecodeI8(pDecoder, &pTask->fetchSink.reserved) < 0) return -1; if (tDecodeI8(pDecoder, &pTask->outputInfo.fetchSink.reserved) < 0) return -1;
} else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) {
if (tDecodeI32(pDecoder, &pTask->fixedDispatcher.taskId) < 0) return -1; if (tDecodeI32(pDecoder, &pTask->outputInfo.fixedDispatcher.taskId) < 0) return -1;
if (tDecodeI32(pDecoder, &pTask->fixedDispatcher.nodeId) < 0) return -1; if (tDecodeI32(pDecoder, &pTask->outputInfo.fixedDispatcher.nodeId) < 0) return -1;
if (tDecodeSEpSet(pDecoder, &pTask->fixedDispatcher.epSet) < 0) return -1; if (tDecodeSEpSet(pDecoder, &pTask->outputInfo.fixedDispatcher.epSet) < 0) return -1;
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
if (tDeserializeSUseDbRspImp(pDecoder, &pTask->shuffleDispatcher.dbInfo) < 0) return -1; if (tDeserializeSUseDbRspImp(pDecoder, &pTask->outputInfo.shuffleDispatcher.dbInfo) < 0) return -1;
if (tDecodeCStrTo(pDecoder, pTask->shuffleDispatcher.stbFullName) < 0) return -1; if (tDecodeCStrTo(pDecoder, pTask->outputInfo.shuffleDispatcher.stbFullName) < 0) return -1;
} }
if (tDecodeI64(pDecoder, &pTask->info.triggerParam) < 0) return -1; if (tDecodeI64(pDecoder, &pTask->info.triggerParam) < 0) return -1;
if (tDecodeCStrTo(pDecoder, pTask->reserve) < 0) return -1; if (tDecodeCStrTo(pDecoder, pTask->reserve) < 0) return -1;
@ -327,8 +333,8 @@ void tFreeStreamTask(SStreamTask* pTask) {
streamQueueClose(pTask->inputInfo.queue, pTask->id.taskId); streamQueueClose(pTask->inputInfo.queue, pTask->id.taskId);
} }
if (pTask->outputInfo.queue) { if (pTask->outputq.queue) {
streamQueueClose(pTask->outputInfo.queue, pTask->id.taskId); streamQueueClose(pTask->outputq.queue, pTask->id.taskId);
} }
if (pTask->exec.qmsg) { if (pTask->exec.qmsg) {
@ -352,11 +358,11 @@ void tFreeStreamTask(SStreamTask* pTask) {
} }
if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) {
tDeleteSchemaWrapper(pTask->tbSink.pSchemaWrapper); tDeleteSchemaWrapper(pTask->outputInfo.tbSink.pSchemaWrapper);
taosMemoryFree(pTask->tbSink.pTSchema); taosMemoryFree(pTask->outputInfo.tbSink.pTSchema);
tSimpleHashCleanup(pTask->tbSink.pTblInfo); tSimpleHashCleanup(pTask->outputInfo.tbSink.pTblInfo);
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
taosArrayDestroy(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); taosArrayDestroy(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos);
pTask->checkReqIds = taosArrayDestroy(pTask->checkReqIds); pTask->checkReqIds = taosArrayDestroy(pTask->checkReqIds);
} }
@ -378,13 +384,10 @@ void tFreeStreamTask(SStreamTask* pTask) {
pTask->pRspMsgList = NULL; pTask->pRspMsgList = NULL;
} }
if (pTask->pUpstreamInfoList != NULL) { streamTaskDestroyUpstreamInfo(&pTask->upstreamInfo);
taosArrayDestroyEx(pTask->pUpstreamInfoList, freeUpstreamItem);
pTask->pUpstreamInfoList = NULL;
}
pTask->msgInfo.pRetryList = taosArrayDestroy(pTask->msgInfo.pRetryList); pTask->msgInfo.pRetryList = taosArrayDestroy(pTask->msgInfo.pRetryList);
taosMemoryFree(pTask->pTokenBucket); taosMemoryFree(pTask->outputInfo.pTokenBucket);
taosThreadMutexDestroy(&pTask->lock); taosThreadMutexDestroy(&pTask->lock);
taosMemoryFree(pTask); taosMemoryFree(pTask);
@ -397,33 +400,34 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i
pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE;
pTask->status.timerActive = 0; pTask->status.timerActive = 0;
pTask->inputInfo.queue = streamQueueOpen(512 << 10); pTask->inputInfo.queue = streamQueueOpen(512 << 10);
pTask->outputInfo.queue = streamQueueOpen(512 << 10); pTask->outputq.queue = streamQueueOpen(512 << 10);
if (pTask->inputInfo.queue == NULL || pTask->outputInfo.queue == NULL) { if (pTask->inputInfo.queue == NULL || pTask->outputq.queue == NULL) {
stError("s-task:%s failed to prepare the input/output queue, initialize task failed", pTask->id.idStr); stError("s-task:%s failed to prepare the input/output queue, initialize task failed", pTask->id.idStr);
return TSDB_CODE_OUT_OF_MEMORY; return TSDB_CODE_OUT_OF_MEMORY;
} }
pTask->execInfo.created = taosGetTimestampMs(); pTask->execInfo.created = taosGetTimestampMs();
pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL;
pTask->outputInfo.status = TASK_OUTPUT_STATUS__NORMAL; pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL;
pTask->pMeta = pMeta; pTask->pMeta = pMeta;
pTask->chkInfo.checkpointVer = ver - 1;
pTask->chkInfo.nextProcessVer = ver; pTask->chkInfo.nextProcessVer = ver;
pTask->dataRange.range.maxVer = ver; pTask->dataRange.range.maxVer = ver;
pTask->dataRange.range.minVer = ver; pTask->dataRange.range.minVer = ver;
pTask->pMsgCb = pMsgCb; pTask->pMsgCb = pMsgCb;
pTask->msgInfo.pRetryList = taosArrayInit(4, sizeof(int32_t)); pTask->msgInfo.pRetryList = taosArrayInit(4, sizeof(int32_t));
pTask->pTokenBucket = taosMemoryCalloc(1, sizeof(STokenBucket)); pTask->outputInfo.pTokenBucket = taosMemoryCalloc(1, sizeof(STokenBucket));
if (pTask->pTokenBucket == NULL) { if (pTask->outputInfo.pTokenBucket == NULL) {
stError("s-task:%s failed to prepare the tokenBucket, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); stError("s-task:%s failed to prepare the tokenBucket, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY));
return TSDB_CODE_OUT_OF_MEMORY; return TSDB_CODE_OUT_OF_MEMORY;
} }
// 2MiB per second for sink task // 2MiB per second for sink task
// 50 times sink operator per second // 50 times sink operator per second
streamTaskInitTokenBucket(pTask->pTokenBucket, 50, 50, 2); streamTaskInitTokenBucket(pTask->outputInfo.pTokenBucket, 50, 50, 2);
TdThreadMutexAttr attr = {0}; TdThreadMutexAttr attr = {0};
int code = taosThreadMutexAttrInit(&attr); int code = taosThreadMutexAttrInit(&attr);
@ -452,7 +456,7 @@ int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask) {
if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__TABLE) { if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__TABLE) {
return 1; return 1;
} else { } else {
SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
return taosArrayGetSize(vgInfo); return taosArrayGetSize(vgInfo);
} }
} }
@ -480,11 +484,11 @@ int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstre
return TSDB_CODE_OUT_OF_MEMORY; return TSDB_CODE_OUT_OF_MEMORY;
} }
if (pTask->pUpstreamInfoList == NULL) { if (pTask->upstreamInfo.pList == NULL) {
pTask->pUpstreamInfoList = taosArrayInit(4, POINTER_BYTES); pTask->upstreamInfo.pList = taosArrayInit(4, POINTER_BYTES);
} }
taosArrayPush(pTask->pUpstreamInfoList, &pEpInfo); taosArrayPush(pTask->upstreamInfo.pList, &pEpInfo);
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
@ -492,9 +496,9 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS
char buf[512] = {0}; char buf[512] = {0};
EPSET_TO_STR(pEpSet, buf); EPSET_TO_STR(pEpSet, buf);
int32_t numOfUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t numOfUpstream = taosArrayGetSize(pTask->upstreamInfo.pList);
for (int32_t i = 0; i < numOfUpstream; ++i) { for (int32_t i = 0; i < numOfUpstream; ++i) {
SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i);
if (pInfo->nodeId == nodeId) { if (pInfo->nodeId == nodeId) {
epsetAssign(&pInfo->epSet, pEpSet); epsetAssign(&pInfo->epSet, pEpSet);
stDebug("s-task:0x%x update the upstreamInfo taskId:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, stDebug("s-task:0x%x update the upstreamInfo taskId:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId,
@ -504,8 +508,16 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS
} }
} }
void streamTaskDestroyUpstreamInfo(SUpstreamInfo* pUpstreamInfo) {
if (pUpstreamInfo->pList != NULL) {
taosArrayDestroyEx(pUpstreamInfo->pList, freeUpstreamItem);
pUpstreamInfo->numOfClosed = 0;
pUpstreamInfo->pList = NULL;
}
}
void streamTaskSetFixedDownstreamInfo(SStreamTask* pTask, const SStreamTask* pDownstreamTask) { void streamTaskSetFixedDownstreamInfo(SStreamTask* pTask, const SStreamTask* pDownstreamTask) {
STaskDispatcherFixed* pDispatcher = &pTask->fixedDispatcher; STaskDispatcherFixed* pDispatcher = &pTask->outputInfo.fixedDispatcher;
pDispatcher->taskId = pDownstreamTask->id.taskId; pDispatcher->taskId = pDownstreamTask->id.taskId;
pDispatcher->nodeId = pDownstreamTask->info.nodeId; pDispatcher->nodeId = pDownstreamTask->info.nodeId;
pDispatcher->epSet = pDownstreamTask->info.epSet; pDispatcher->epSet = pDownstreamTask->info.epSet;
@ -520,7 +532,7 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE
int8_t type = pTask->outputInfo.type; int8_t type = pTask->outputInfo.type;
if (type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
SArray* pVgs = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; SArray* pVgs = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
int32_t numOfVgroups = taosArrayGetSize(pVgs); int32_t numOfVgroups = taosArrayGetSize(pVgs);
for (int32_t i = 0; i < numOfVgroups; i++) { for (int32_t i = 0; i < numOfVgroups; i++) {
@ -534,7 +546,7 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE
} }
} }
} else if (type == TASK_OUTPUT__FIXED_DISPATCH) { } else if (type == TASK_OUTPUT__FIXED_DISPATCH) {
STaskDispatcherFixed* pDispatcher = &pTask->fixedDispatcher; STaskDispatcherFixed* pDispatcher = &pTask->outputInfo.fixedDispatcher;
if (pDispatcher->nodeId == nodeId) { if (pDispatcher->nodeId == nodeId) {
epsetAssign(&pDispatcher->epSet, pEpSet); epsetAssign(&pDispatcher->epSet, pEpSet);
stDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpSet:%s", pTask->id.taskId, stDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpSet:%s", pTask->id.taskId,
@ -615,24 +627,33 @@ void streamTaskResetUpstreamStageInfo(SStreamTask* pTask) {
return; return;
} }
int32_t size = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t size = taosArrayGetSize(pTask->upstreamInfo.pList);
for (int32_t i = 0; i < size; ++i) { for (int32_t i = 0; i < size; ++i) {
SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i);
pInfo->stage = -1; pInfo->stage = -1;
} }
stDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr); stDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr);
} }
int8_t streamTaskSetSchedStatusWait(SStreamTask* pTask) { bool streamTaskAllUpstreamClosed(SStreamTask* pTask) {
taosThreadMutexLock(&pTask->lock); return pTask->upstreamInfo.numOfClosed == taosArrayGetSize(pTask->upstreamInfo.pList);
int8_t status = pTask->status.schedStatus; }
if (status == TASK_SCHED_STATUS__INACTIVE) {
pTask->status.schedStatus = TASK_SCHED_STATUS__WAITING;
}
taosThreadMutexUnlock(&pTask->lock);
return status; bool streamTaskSetSchedStatusWait(SStreamTask* pTask) {
bool ret = false;
// double check
if (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE) {
taosThreadMutexLock(&pTask->lock);
if (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE) {
pTask->status.schedStatus = TASK_SCHED_STATUS__WAITING;
ret = true;
}
taosThreadMutexUnlock(&pTask->lock);
}
return ret;
} }
int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask) { int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask) {
@ -711,3 +732,24 @@ const char* streamGetTaskStatusStr(int32_t status) {
default:return ""; default:return "";
} }
} }
void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask) {
pEntry->id.streamId = pTask->id.streamId;
pEntry->id.taskId = pTask->id.taskId;
pEntry->stage = -1;
pEntry->nodeId = pTask->info.nodeId;
pEntry->status = TASK_STATUS__STOP;
}
void streamTaskStatusCopy(STaskStatusEntry* pDst, const STaskStatusEntry* pSrc) {
pDst->stage = pSrc->stage;
pDst->inputQUsed = pSrc->inputQUsed;
pDst->inputRate = pSrc->inputRate;
pDst->processedVer = pSrc->processedVer;
pDst->verStart = pSrc->verStart;
pDst->verEnd = pSrc->verEnd;
pDst->sinkQuota = pSrc->sinkQuota;
pDst->sinkDataSize = pSrc->sinkDataSize;
pDst->activeCheckpointId = pSrc->activeCheckpointId;
pDst->checkpointFailed = pSrc->checkpointFailed;
}

View File

@ -137,6 +137,7 @@ echo "idxDebugFlag 143" >> $TAOS_CFG
echo "udfDebugFlag 143" >> $TAOS_CFG echo "udfDebugFlag 143" >> $TAOS_CFG
echo "smaDebugFlag 143" >> $TAOS_CFG echo "smaDebugFlag 143" >> $TAOS_CFG
echo "metaDebugFlag 143" >> $TAOS_CFG echo "metaDebugFlag 143" >> $TAOS_CFG
echo "stDebugFlag 143" >> $TAOS_CFG
echo "numOfLogLines 20000000" >> $TAOS_CFG echo "numOfLogLines 20000000" >> $TAOS_CFG
echo "asyncLog 0" >> $TAOS_CFG echo "asyncLog 0" >> $TAOS_CFG
echo "locale en_US.UTF-8" >> $TAOS_CFG echo "locale en_US.UTF-8" >> $TAOS_CFG

View File

@ -22,6 +22,8 @@ sql create table t2 using st tags(2,2,2);
sql create stable result.streamt0(ts timestamp,a int,b int) tags(ta int,tb varchar(100),tc int); sql create stable result.streamt0(ts timestamp,a int,b int) tags(ta int,tb varchar(100),tc int);
sql create stream streams0 trigger at_once into result.streamt0 tags(tb) as select _wstart, count(*) c1, max(a) c2 from st partition by tbname tb interval(10s); sql create stream streams0 trigger at_once into result.streamt0 tags(tb) as select _wstart, count(*) c1, max(a) c2 from st partition by tbname tb interval(10s);
sleep 500
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,2,2,3); sql insert into t2 values(1648791213000,2,2,3);
@ -106,6 +108,8 @@ sql create table t2 using st tags(2,2,2);
sql create stable result1.streamt1(ts timestamp,a int,b int,c int) tags(ta varchar(100),tb int,tc int); sql create stable result1.streamt1(ts timestamp,a int,b int,c int) tags(ta varchar(100),tb int,tc int);
sql create stream streams1 trigger at_once into result1.streamt1(ts,c,a,b) tags(ta) as select _wstart, count(*) c1, max(a),min(b) c2 from st partition by tbname as ta interval(10s); sql create stream streams1 trigger at_once into result1.streamt1(ts,c,a,b) tags(ta) as select _wstart, count(*) c1, max(a),min(b) c2 from st partition by tbname as ta interval(10s);
sleep 500
sql insert into t1 values(1648791213000,10,20,30); sql insert into t1 values(1648791213000,10,20,30);
sql insert into t2 values(1648791213000,40,50,60); sql insert into t2 values(1648791213000,40,50,60);
@ -194,7 +198,7 @@ sql_error create stream streams2 trigger at_once into result2.streamt2 as selec
# column dest 3, source 2 # column dest 3, source 2
sql create stream streams2 trigger at_once into result2.streamt2(ts, a) tags(ta) as select _wstart, count(*) c1 from st partition by tbname as ta interval(10s); sql create stream streams2 trigger at_once into result2.streamt2(ts, a) tags(ta) as select _wstart, count(*) c1 from st partition by tbname as ta interval(10s);
sleep 500
print ===== step5 print ===== step5
@ -211,6 +215,7 @@ sql create table t2 using st tags(4,5,6);
sql create stable result3.streamt3(ts timestamp,a int,b int,c int, d int) tags(ta int,tb int,tc int); sql create stable result3.streamt3(ts timestamp,a int,b int,c int, d int) tags(ta int,tb int,tc int);
sql create stream streams3 trigger at_once into result3.streamt3(ts,c,a,b) as select _wstart, count(*) c1, max(a),min(b) c2 from st interval(10s); sql create stream streams3 trigger at_once into result3.streamt3(ts,c,a,b) as select _wstart, count(*) c1, max(a),min(b) c2 from st interval(10s);
sleep 500
sql insert into t1 values(1648791213000,10,20,30); sql insert into t1 values(1648791213000,10,20,30);
sql insert into t2 values(1648791213000,40,50,60); sql insert into t2 values(1648791213000,40,50,60);
@ -290,6 +295,7 @@ sql create table t2 using st tags(4,5,6);
sql create stable result4.streamt4(ts timestamp,a int,b int,c int, d int) tags(tg1 int,tg2 int,tg3 int); sql create stable result4.streamt4(ts timestamp,a int,b int,c int, d int) tags(tg1 int,tg2 int,tg3 int);
sql create stream streams4 trigger at_once into result4.streamt4(ts,c,a,b) tags(tg2, tg3, tg1) subtable( concat("tbl-", cast(tg1 as varchar(10)) ) ) as select _wstart, count(*) c1, max(a),min(b) c2 from st partition by ta+1 as tg1, cast(tb as bigint) as tg2, tc as tg3 interval(10s); sql create stream streams4 trigger at_once into result4.streamt4(ts,c,a,b) tags(tg2, tg3, tg1) subtable( concat("tbl-", cast(tg1 as varchar(10)) ) ) as select _wstart, count(*) c1, max(a),min(b) c2 from st partition by ta+1 as tg1, cast(tb as bigint) as tg2, tc as tg3 interval(10s);
sleep 500
sql insert into t1 values(1648791213000,10,20,30); sql insert into t1 values(1648791213000,10,20,30);
sql insert into t2 values(1648791213000,40,50,60); sql insert into t2 values(1648791213000,40,50,60);
@ -374,6 +380,7 @@ sql create table t2 using st tags(4,5,6);
sql create stable result5.streamt5(ts timestamp,a int,b int,c int, d int) tags(tg1 int,tg2 int,tg3 int); sql create stable result5.streamt5(ts timestamp,a int,b int,c int, d int) tags(tg1 int,tg2 int,tg3 int);
sql create stream streams5 trigger at_once into result5.streamt5(ts,c,a,b) tags(tg2, tg3, tg1) subtable( concat("tbl-", cast(tg3 as varchar(10)) ) ) as select _wstart, count(*) c1, max(a),min(b) c2 from st partition by ta+1 as tg1, cast(tb as bigint) as tg2, a as tg3 session(ts, 10s); sql create stream streams5 trigger at_once into result5.streamt5(ts,c,a,b) tags(tg2, tg3, tg1) subtable( concat("tbl-", cast(tg3 as varchar(10)) ) ) as select _wstart, count(*) c1, max(a),min(b) c2 from st partition by ta+1 as tg1, cast(tb as bigint) as tg2, a as tg3 session(ts, 10s);
sleep 500
sql insert into t1 values(1648791213000,NULL,NULL,NULL); sql insert into t1 values(1648791213000,NULL,NULL,NULL);
@ -458,6 +465,7 @@ sql create stream streams8 trigger at_once into streamt8 as select _wstart as
sql drop stream streams8; sql drop stream streams8;
sql create stream streams71 trigger at_once into streamt8(ts, c2) tags(group_id)as select _wstart, count(*) from t1 partition by tbname as group_id interval(10s); sql create stream streams71 trigger at_once into streamt8(ts, c2) tags(group_id)as select _wstart, count(*) from t1 partition by tbname as group_id interval(10s);
sleep 500
sql insert into t1 values(1648791233000,1,2,3,1.0); sql insert into t1 values(1648791233000,1,2,3,1.0);

View File

@ -15,6 +15,8 @@ sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int);
sql create table t1 using st tags(1,1,1); sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams1 trigger at_once into streamt1 as select _wstart, count(*) c1, count(a) c2 from st interval(1s) ; sql create stream streams1 trigger at_once into streamt1 as select _wstart, count(*) c1, count(a) c2 from st interval(1s) ;
sleep 500
sql insert into t1 values(1648791211000,1,2,3); sql insert into t1 values(1648791211000,1,2,3);
sql insert into t1 values(1648791212000,2,2,3); sql insert into t1 values(1648791212000,2,2,3);
@ -44,6 +46,7 @@ sql alter table streamt1 add column c3 double;
print create stream streams1 trigger at_once into streamt1 as select _wstart, count(*) c1, count(a) c2, avg(b) c3 from st interval(1s) ; print create stream streams1 trigger at_once into streamt1 as select _wstart, count(*) c1, count(a) c2, avg(b) c3 from st interval(1s) ;
sql create stream streams1 trigger at_once into streamt1 as select _wstart, count(*) c1, count(a) c2, avg(b) c3 from st interval(1s) ; sql create stream streams1 trigger at_once into streamt1 as select _wstart, count(*) c1, count(a) c2, avg(b) c3 from st interval(1s) ;
sleep 500
sql insert into t2 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,1,2,3);
sql insert into t1 values(1648791214000,1,2,3); sql insert into t1 values(1648791214000,1,2,3);

View File

@ -17,6 +17,7 @@ sql create database test vgroups 1;
sql use test; sql use test;
sql create table t1(ts timestamp, a int, b int , c int, d double); sql create table t1(ts timestamp, a int, b int , c int, d double);
sql create stream streams0 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _wstart c1, count(*) c2, max(a) c3 from t1 interval(10s); sql create stream streams0 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _wstart c1, count(*) c2, max(a) c3 from t1 interval(10s);
sleep 500
sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL);
sleep 1000 sleep 1000
@ -194,6 +195,7 @@ sql create stable st(ts timestamp, a int, b int, c int, d double) tags(ta int,tb
sql create table t1 using st tags(1,1,1); sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt2 as select _wstart c1, count(*) c2, max(a) c3 from st interval(10s); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt2 as select _wstart c1, count(*) c2, max(a) c3 from st interval(10s);
sleep 500
sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL);
sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL);
@ -420,6 +422,7 @@ sql create stable st(ts timestamp, a int, b int, c int, d double) tags(ta int,tb
sql create table t1 using st tags(1,1,1); sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt3 as select _wstart c1, count(*) c2, max(a) c3 from st interval(10s); sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt3 as select _wstart c1, count(*) c2, max(a) c3 from st interval(10s);
sleep 500
sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL);
sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL);

View File

@ -18,6 +18,8 @@ sql use test;
sql create table t1(ts timestamp, a int, b int , c int, d double); sql create table t1(ts timestamp, a int, b int , c int, d double);
sql create stream streams0 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _wstart c1, count(*) c2, max(a) c3 from t1 session(ts, 5s); sql create stream streams0 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _wstart c1, count(*) c2, max(a) c3 from t1 session(ts, 5s);
sleep 2000
sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL);
sleep 1000 sleep 1000
sql delete from t1 where ts = 1648791213000; sql delete from t1 where ts = 1648791213000;
@ -193,6 +195,7 @@ sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt2 as select _wstart c1, count(*) c2, max(a) c3 from st session(ts,5s); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt2 as select _wstart c1, count(*) c2, max(a) c3 from st session(ts,5s);
sleep 2000
sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL);
sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL);
@ -423,6 +426,7 @@ sql create stable st(ts timestamp, a int, b int, c int, d double) tags(ta int,tb
sql create table t1 using st tags(1,1,1); sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt3 as select _wstart c1, count(*) c2, max(a) c3 from st session(ts,5s); sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt3 as select _wstart c1, count(*) c2, max(a) c3 from st session(ts,5s);
sleep 2000
sql insert into t1 values(1648791210000,1,1,1,NULL); sql insert into t1 values(1648791210000,1,1,1,NULL);
sql insert into t1 values(1648791210001,2,2,2,NULL); sql insert into t1 values(1648791210001,2,2,2,NULL);
@ -534,6 +538,7 @@ sql create table t2 using st tags(2,2,2);
print create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _wstart, count(*) c1 from st partition by tbname session(ts, 2s); print create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _wstart, count(*) c1 from st partition by tbname session(ts, 2s);
sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _wstart, count(*) c1 from st partition by tbname session(ts, 2s); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _wstart, count(*) c1 from st partition by tbname session(ts, 2s);
sleep 1000
sql insert into t1 values(1648791210000,1,2,3); sql insert into t1 values(1648791210000,1,2,3);
sql insert into t1 values(1648791220000,2,2,3); sql insert into t1 values(1648791220000,2,2,3);

View File

@ -439,6 +439,7 @@ sql create table ts1 using st tags(1,1,1);
sql create table ts2 using st tags(2,2,2); sql create table ts2 using st tags(2,2,2);
sql create stream stream_t2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 watermark 20s into streamtST1 as select _wstart, count(*) c1, count(a) c2 , sum(a) c3 , max(b) c5, min(c) c6 from st interval(10s) ; sql create stream stream_t2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 watermark 20s into streamtST1 as select _wstart, count(*) c1, count(a) c2 , sum(a) c3 , max(b) c5, min(c) c6 from st interval(10s) ;
sleep 2000
sql insert into ts1 values(1648791211000,1,2,3); sql insert into ts1 values(1648791211000,1,2,3);
sql insert into ts1 values(1648791222001,2,2,3); sql insert into ts1 values(1648791222001,2,2,3);
sql insert into ts2 values(1648791211000,1,2,3); sql insert into ts2 values(1648791211000,1,2,3);

View File

@ -284,6 +284,7 @@ sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt2 as select _wstart c1, count(*) c2, max(a) c3 from st partition by a session(ts, 5s); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt2 as select _wstart c1, count(*) c2, max(a) c3 from st partition by a session(ts, 5s);
sleep 1000
sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL);
sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL);
sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL);
@ -480,6 +481,7 @@ sql create table t3 using st tags(2,2,2);
sql create table t4 using st tags(2,2,2); sql create table t4 using st tags(2,2,2);
sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt4 as select _wstart c1, count(*) c2, max(a) c3 from st partition by a session(ts, 5s); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt4 as select _wstart c1, count(*) c2, max(a) c3 from st partition by a session(ts, 5s);
sleep 2000
sql insert into t1 values(1648791213000,2,2,3,1.0); sql insert into t1 values(1648791213000,2,2,3,1.0);
sql insert into t2 values(1648791213000,2,2,3,1.0); sql insert into t2 values(1648791213000,2,2,3,1.0);
sql insert into t3 values(1648791213000,2,2,3,1.0); sql insert into t3 values(1648791213000,2,2,3,1.0);

View File

@ -22,6 +22,8 @@ sql create table t2 using st tags(2,2,2);
#sql_error create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE("aaa") as select _wstart, count(*) c1 from st interval(10s); #sql_error create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE("aaa") as select _wstart, count(*) c1 from st interval(10s);
sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE(concat("aaa-", tbname)) as select _wstart, count(*) c1 from st partition by tbname interval(10s); sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE(concat("aaa-", tbname)) as select _wstart, count(*) c1 from st partition by tbname interval(10s);
sleep 1000
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,1,2,3);
@ -89,10 +91,11 @@ sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result2.streamt2 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st partition by concat("tag-", tbname) as cc interval(10s); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result2.streamt2 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st partition by concat("tag-", tbname) as cc interval(10s);
sleep 1000
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,1,2,3);
$loop_count = 0 $loop_count = 0
loop2: loop2:
@ -174,10 +177,11 @@ sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result3.streamt3 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", tbname)) as select _wstart, count(*) c1 from st partition by concat("tag-", tbname) as dd, tbname interval(10s); sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result3.streamt3 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", tbname)) as select _wstart, count(*) c1 from st partition by concat("tag-", tbname) as dd, tbname interval(10s);
sleep 1000
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,1,2,3);
$loop_count = 0 $loop_count = 0
loop4: loop4:
@ -286,8 +290,9 @@ sql create table t2 using st tags(2,2,2);
sql create table t3 using st tags(3,3,3); sql create table t3 using st tags(3,3,3);
sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result4.streamt4 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", tbname)) as select _wstart, count(*) c1 from st partition by concat("tag-", tbname) as dd, tbname interval(10s); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result4.streamt4 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", tbname)) as select _wstart, count(*) c1 from st partition by concat("tag-", tbname) as dd, tbname interval(10s);
sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); sleep 1000
sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3);
$loop_count = 0 $loop_count = 0
loop7: loop7:
@ -405,8 +410,9 @@ sql create table t2 using st tags("2",2,2);
sql create table t3 using st tags("3",3,3); sql create table t3 using st tags("3",3,3);
sql create stream streams6 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result6.streamt6 TAGS(dd int) as select _wstart, count(*) c1 from st partition by concat(ta, "0") as dd, tbname interval(10s); sql create stream streams6 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result6.streamt6 TAGS(dd int) as select _wstart, count(*) c1 from st partition by concat(ta, "0") as dd, tbname interval(10s);
sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); sleep 1000
sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3);
$loop_count = 0 $loop_count = 0
loop9: loop9:

View File

@ -22,6 +22,8 @@ sql create table t2 using st tags(2,2,2);
#sql_error create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE("aaa") as select _wstart, count(*) c1 from st interval(10s); #sql_error create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE("aaa") as select _wstart, count(*) c1 from st interval(10s);
sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE( concat("aaa-", cast(a as varchar(10) ) ) ) as select _wstart, count(*) c1 from st partition by a interval(10s); sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE( concat("aaa-", cast(a as varchar(10) ) ) ) as select _wstart, count(*) c1 from st partition by a interval(10s);
sleep 2000
print ===== insert into 1 print ===== insert into 1
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,2,2,3); sql insert into t2 values(1648791213000,2,2,3);
@ -88,11 +90,12 @@ sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result2.streamt2 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st partition by concat("col-", cast(a as varchar(10) ) ) as cc interval(10s); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result2.streamt2 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st partition by concat("col-", cast(a as varchar(10) ) ) as cc interval(10s);
sleep 2000
print ===== insert into 2 print ===== insert into 2
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,2,2,3); sql insert into t2 values(1648791213000,2,2,3);
$loop_count = 0 $loop_count = 0
loop2: loop2:
@ -172,6 +175,8 @@ sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result3.streamt3 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", cast(a as varchar(10) ) ) ) as select _wstart, count(*) c1 from st partition by concat("col-", cast(a as varchar(10) ) ) as dd, a interval(10s); sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result3.streamt3 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", cast(a as varchar(10) ) ) ) as select _wstart, count(*) c1 from st partition by concat("col-", cast(a as varchar(10) ) ) as dd, a interval(10s);
sleep 2000
print ===== insert into 3 print ===== insert into 3
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,2,2,3); sql insert into t2 values(1648791213000,2,2,3);
@ -284,8 +289,9 @@ sql create table t2 using st tags(2,2,2);
sql create table t3 using st tags(3,3,3); sql create table t3 using st tags(3,3,3);
sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result4.streamt4 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", dd)) as select _wstart, count(*) c1 from st partition by concat("t", cast(a as varchar(10) ) ) as dd interval(10s); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result4.streamt4 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", dd)) as select _wstart, count(*) c1 from st partition by concat("t", cast(a as varchar(10) ) ) as dd interval(10s);
sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); sleep 2000
sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3);
$loop_count = 0 $loop_count = 0
loop7: loop7:

View File

@ -21,6 +21,8 @@ sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE("aaa") as select _wstart, count(*) c1 from st interval(10s); sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE("aaa") as select _wstart, count(*) c1 from st interval(10s);
sleep 2000
print ===== insert into 1 print ===== insert into 1
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,2,2,3); sql insert into t2 values(1648791213000,2,2,3);
@ -94,11 +96,12 @@ sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result2.streamt2 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st interval(10s); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result2.streamt2 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st interval(10s);
sleep 2000
print ===== insert into 2 print ===== insert into 2
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,2,2,3); sql insert into t2 values(1648791213000,2,2,3);
$loop_count = 0 $loop_count = 0
loop2: loop2:
@ -186,21 +189,20 @@ print ===== step4
print ===== column name + table name print ===== column name + table name
sql create database result3 vgroups 1; sql create database result3 vgroups 1;
sql create database test3 vgroups 4; sql create database test3 vgroups 4;
sql use test3; sql use test3;
sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int);
sql create table t1 using st tags(1,1,1); sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2); sql create table t2 using st tags(2,2,2);
sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result3.streamt3 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", "1") ) as select _wstart, count(*) c1 from st interval(10s); sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result3.streamt3 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", "1") ) as select _wstart, count(*) c1 from st interval(10s);
sleep 2000
print ===== insert into 3 print ===== insert into 3
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,2,2,3); sql insert into t2 values(1648791213000,2,2,3);
$loop_count = 0 $loop_count = 0
loop4: loop4:
@ -306,8 +308,9 @@ sql create table t2 using st tags(2,2,2);
sql create table t3 using st tags(3,3,3); sql create table t3 using st tags(3,3,3);
sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result4.streamt4 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", "1")) as select _wstart, count(*) c1 from st interval(10s); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result4.streamt4 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", "1")) as select _wstart, count(*) c1 from st interval(10s);
sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); sleep 2000
sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3);
$loop_count = 0 $loop_count = 0
loop7: loop7:
@ -379,6 +382,8 @@ sql create stream streams51 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 in
sql create stream streams52 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result5.streamt52 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st interval(10s); sql create stream streams52 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result5.streamt52 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st interval(10s);
sql create stream streams53 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result5.streamt53 TAGS(dd varchar(100)) SUBTABLE(concat("aaa-", "1") ) as select _wstart, count(*) c1 from st interval(10s); sql create stream streams53 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result5.streamt53 TAGS(dd varchar(100)) SUBTABLE(concat("aaa-", "1") ) as select _wstart, count(*) c1 from st interval(10s);
sleep 2000
sql insert into t1 values(1648791213000,1,2,3); sql insert into t1 values(1648791213000,1,2,3);
sql insert into t2 values(1648791213000,2,2,3); sql insert into t2 values(1648791213000,2,2,3);

View File

@ -217,7 +217,7 @@ class TDTestCase:
tdSql.checkEqual(20470,len(tdSql.queryResult)) tdSql.checkEqual(20470,len(tdSql.queryResult))
tdSql.query("select * from information_schema.ins_columns where db_name ='information_schema'") tdSql.query("select * from information_schema.ins_columns where db_name ='information_schema'")
tdSql.checkEqual(194, len(tdSql.queryResult)) tdSql.checkEqual(196, len(tdSql.queryResult))
tdSql.query("select * from information_schema.ins_columns where db_name ='performance_schema'") tdSql.query("select * from information_schema.ins_columns where db_name ='performance_schema'")
tdSql.checkEqual(54, len(tdSql.queryResult)) tdSql.checkEqual(54, len(tdSql.queryResult))

View File

@ -70,6 +70,9 @@ class TDTestCase:
fill_value='VALUE,1,2,3,4,5,6,7,8,9,10,11' fill_value='VALUE,1,2,3,4,5,6,7,8,9,10,11'
self.tdCom.create_stream(stream_name=f'{self.tb_name}{self.tdCom.stream_suffix}', des_table=self.tb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.tb_source_select_str} from {self.tb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=tb_subtable_value, fill_value=fill_value, fill_history_value=fill_history_value) self.tdCom.create_stream(stream_name=f'{self.tb_name}{self.tdCom.stream_suffix}', des_table=self.tb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.tb_source_select_str} from {self.tb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=tb_subtable_value, fill_value=fill_value, fill_history_value=fill_history_value)
start_time = self.tdCom.date_time start_time = self.tdCom.date_time
time.sleep(1)
for i in range(self.tdCom.range_count): for i in range(self.tdCom.range_count):
ts_value = str(self.tdCom.date_time+self.tdCom.dataDict["interval"])+f'+{i*10}s' ts_value = str(self.tdCom.date_time+self.tdCom.dataDict["interval"])+f'+{i*10}s'
ts_cast_delete_value = self.tdCom.time_cast(ts_value) ts_cast_delete_value = self.tdCom.time_cast(ts_value)

View File

@ -92,6 +92,8 @@ class TDTestCase:
else: else:
range_count = self.tdCom.range_count range_count = self.tdCom.range_count
time.sleep(1)
for i in range(range_count): for i in range(range_count):
latency = 0 latency = 0
tag_value_list = list() tag_value_list = list()

View File

@ -35,6 +35,9 @@ class TDTestCase:
# create stb/ctb/tb stream # create stb/ctb/tb stream
self.tdCom.create_stream(stream_name=f'{self.stb_name}{self.tdCom.stream_suffix}', des_table=self.stb_stream_des_table, source_sql=source_sql, ignore_expired=ignore_expired) self.tdCom.create_stream(stream_name=f'{self.stb_name}{self.tdCom.stream_suffix}', des_table=self.stb_stream_des_table, source_sql=source_sql, ignore_expired=ignore_expired)
time.sleep(1)
# insert data # insert data
count = 1 count = 1
step_count = 1 step_count = 1

View File

@ -56,6 +56,9 @@ class TDTestCase:
self.tdCom.create_stream(stream_name=f'{self.stb_name}{self.tdCom.stream_suffix}', des_table=self.stb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.stb_source_select_str} from {self.stb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=stb_subtable_value, fill_history_value=fill_history_value) self.tdCom.create_stream(stream_name=f'{self.stb_name}{self.tdCom.stream_suffix}', des_table=self.stb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.stb_source_select_str} from {self.stb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=stb_subtable_value, fill_history_value=fill_history_value)
self.tdCom.create_stream(stream_name=f'{self.ctb_name}{self.tdCom.stream_suffix}', des_table=self.tdCom.ctb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.stb_source_select_str} from {self.ctb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=ctb_subtable_value, fill_history_value=fill_history_value) self.tdCom.create_stream(stream_name=f'{self.ctb_name}{self.tdCom.stream_suffix}', des_table=self.tdCom.ctb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.stb_source_select_str} from {self.ctb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=ctb_subtable_value, fill_history_value=fill_history_value)
self.tdCom.create_stream(stream_name=f'{self.tb_name}{self.tdCom.stream_suffix}', des_table=self.tdCom.tb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.tb_source_select_str} from {self.tb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=tb_subtable_value, fill_history_value=fill_history_value) self.tdCom.create_stream(stream_name=f'{self.tb_name}{self.tdCom.stream_suffix}', des_table=self.tdCom.tb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.tb_source_select_str} from {self.tb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=tb_subtable_value, fill_history_value=fill_history_value)
time.sleep(1)
for i in range(range_count): for i in range(range_count):
ts_value = str(date_time+self.tdCom.dataDict["interval"])+f'+{i*10}s' ts_value = str(date_time+self.tdCom.dataDict["interval"])+f'+{i*10}s'
ts_cast_delete_value = self.tdCom.time_cast(ts_value) ts_cast_delete_value = self.tdCom.time_cast(ts_value)
@ -75,6 +78,9 @@ class TDTestCase:
partition_elm = f'partition by {partition}' partition_elm = f'partition by {partition}'
else: else:
partition_elm = "" partition_elm = ""
time.sleep(1)
# if i == int(range_count/2): # if i == int(range_count/2):
if i > 2 and i % 3 == 0: if i > 2 and i % 3 == 0:
for stream_name in [f'{self.stb_name}{self.tdCom.stream_suffix}', f'{self.ctb_name}{self.tdCom.stream_suffix}', f'{self.tb_name}{self.tdCom.stream_suffix}']: for stream_name in [f'{self.stb_name}{self.tdCom.stream_suffix}', f'{self.ctb_name}{self.tdCom.stream_suffix}', f'{self.tb_name}{self.tdCom.stream_suffix}']:

View File

@ -43,6 +43,9 @@ class TDTestCase:
watermark_value = None watermark_value = None
# create stb/ctb/tb stream # create stb/ctb/tb stream
self.tdCom.create_stream(stream_name=f'{self.stb_name}{self.tdCom.stream_suffix}', des_table=self.tdCom.ext_stb_stream_des_table, source_sql=f'select _wstart AS wstart, {partitial_tb_source_str} from {self.stb_name} session(ts, {self.tdCom.dataDict["session"]}s)', trigger_mode="window_close", watermark=watermark_value, subtable_value=stb_subtable_value, fill_history_value=fill_history_value, stb_field_name_value=stb_field_name_value, tag_value=tag_value, use_exist_stb=use_exist_stb) self.tdCom.create_stream(stream_name=f'{self.stb_name}{self.tdCom.stream_suffix}', des_table=self.tdCom.ext_stb_stream_des_table, source_sql=f'select _wstart AS wstart, {partitial_tb_source_str} from {self.stb_name} session(ts, {self.tdCom.dataDict["session"]}s)', trigger_mode="window_close", watermark=watermark_value, subtable_value=stb_subtable_value, fill_history_value=fill_history_value, stb_field_name_value=stb_field_name_value, tag_value=tag_value, use_exist_stb=use_exist_stb)
time.sleep(1)
for i in range(self.tdCom.range_count): for i in range(self.tdCom.range_count):
if i == 0: if i == 0:
window_close_ts = self.tdCom.cal_watermark_window_close_session_endts(self.tdCom.date_time, self.tdCom.dataDict['watermark'], self.tdCom.dataDict['session']) window_close_ts = self.tdCom.cal_watermark_window_close_session_endts(self.tdCom.date_time, self.tdCom.dataDict['watermark'], self.tdCom.dataDict['session'])