Merge pull request #23343 from taosdata/refact/streamsm

refactor(stream):  add fsm for managing task status.
This commit is contained in:
Haojun Liao 2023-10-25 22:31:56 +08:00 committed by GitHub
commit b974561a79
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 1558 additions and 859 deletions

View File

@ -38,16 +38,25 @@ extern "C" {
#define TASK_DOWNSTREAM_READY 0x0
#define TASK_DOWNSTREAM_NOT_READY 0x1
#define TASK_DOWNSTREAM_NOT_LEADER 0x2
#define TASK_SELF_NEW_STAGE 0x3
#define TASK_UPSTREAM_NEW_STAGE 0x3
#define NODE_ROLE_UNINIT 0x1
#define NODE_ROLE_LEADER 0x2
#define NODE_ROLE_FOLLOWER 0x3
typedef struct SStreamTask SStreamTask;
typedef struct SStreamQueue SStreamQueue;
#define HAS_RELATED_FILLHISTORY_TASK(_t) ((_t)->hTaskInfo.id.taskId != 0)
#define CLEAR_RELATED_FILLHISTORY_TASK(_t) \
do { \
(_t)->hTaskInfo.id.taskId = 0; \
(_t)->hTaskInfo.id.streamId = 0; \
} while (0)
typedef struct SStreamTask SStreamTask;
typedef struct SStreamQueue SStreamQueue;
typedef struct SStreamTaskSM SStreamTaskSM;
#define SSTREAM_TASK_VER 2
enum {
STREAM_STATUS__NORMAL = 0,
STREAM_STATUS__STOP,
@ -58,7 +67,7 @@ enum {
};
typedef enum ETaskStatus {
TASK_STATUS__NORMAL = 0,
TASK_STATUS__READY = 0,
TASK_STATUS__DROPPING,
TASK_STATUS__UNINIT, // not used, an placeholder
TASK_STATUS__STOP,
@ -66,6 +75,7 @@ typedef enum ETaskStatus {
TASK_STATUS__HALT, // pause, but not be manipulated by user command
TASK_STATUS__PAUSE, // pause
TASK_STATUS__CK, // stream task is in checkpoint status, no data are allowed to put into inputQ anymore
TASK_STATUS__STREAM_SCAN_HISTORY,
} ETaskStatus;
enum {
@ -118,6 +128,22 @@ enum {
STREAM_META_OK_TO_STOP = 2,
};
typedef enum EStreamTaskEvent {
TASK_EVENT_INIT = 0x1,
TASK_EVENT_INIT_SCANHIST = 0x2,
TASK_EVENT_INIT_STREAM_SCANHIST = 0x3,
TASK_EVENT_SCANHIST_DONE = 0x4,
TASK_EVENT_STOP = 0x5,
TASK_EVENT_GEN_CHECKPOINT = 0x6,
TASK_EVENT_CHECKPOINT_DONE = 0x7,
TASK_EVENT_PAUSE = 0x8,
TASK_EVENT_RESUME = 0x9,
TASK_EVENT_HALT = 0xA,
TASK_EVENT_DROPPING = 0xB,
TASK_EVENT_SCAN_TSDB = 0xC,
TASK_EVENT_SCAN_WAL = 0xD,
} EStreamTaskEvent;
typedef struct {
int8_t type;
} SStreamQueueItem;
@ -155,11 +181,6 @@ typedef struct {
SSDataBlock* pBlock;
} SStreamRefDataBlock;
typedef struct {
int8_t type;
SSDataBlock* pBlock;
} SStreamTrigger;
typedef struct SStreamQueueNode SStreamQueueNode;
struct SStreamQueueNode {
@ -265,14 +286,15 @@ typedef struct SCheckpointInfo {
} SCheckpointInfo;
typedef struct SStreamStatus {
int8_t taskStatus;
int8_t downstreamReady; // downstream tasks are all ready now, if this flag is set
int8_t schedStatus;
int8_t keepTaskStatus;
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
int32_t timerActive; // timer is active
int32_t inScanHistorySentinel;
SStreamTaskSM* pSM;
int8_t taskStatus;
int8_t downstreamReady; // downstream tasks are all ready now, if this flag is set
int8_t schedStatus;
int8_t keepTaskStatus;
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
int32_t timerActive; // timer is active
int32_t inScanHistorySentinel;
} SStreamStatus;
typedef struct SDataRange {
@ -349,6 +371,7 @@ typedef struct SHistoryTaskInfo {
int32_t tickCount;
int32_t retryTimes;
int32_t waitInterval;
int64_t haltVer; // offset in wal when halt the stream task
} SHistoryTaskInfo;
typedef struct STaskOutputInfo {
@ -413,33 +436,38 @@ typedef struct STaskStartInfo {
int32_t elapsedTime;
} STaskStartInfo;
typedef struct STaskUpdateInfo {
SHashObj* pTasks;
int32_t transId;
} STaskUpdateInfo;
// meta
typedef struct SStreamMeta {
char* path;
TDB* db;
TTB* pTaskDb;
TTB* pCheckpointDb;
SHashObj* pTasksMap;
SArray* pTaskList; // SArray<STaskId*>
void* ahandle;
TXN* txn;
FTaskExpand* expandFunc;
int32_t vgId;
int64_t stage;
int32_t role;
STaskStartInfo startInfo;
SRWLatch lock;
int32_t walScanCounter;
void* streamBackend;
int64_t streamBackendRid;
SHashObj* pTaskBackendUnique;
TdThreadMutex backendMutex;
SMetaHbInfo* pHbInfo;
SHashObj* pUpdateTaskSet;
int32_t numOfStreamTasks; // this value should be increased when a new task is added into the meta
int32_t numOfPausedTasks;
int32_t chkptNotReadyTasks;
int64_t rid;
char* path;
TDB* db;
TTB* pTaskDb;
TTB* pCheckpointDb;
SHashObj* pTasksMap;
SArray* pTaskList; // SArray<STaskId*>
void* ahandle;
TXN* txn;
FTaskExpand* expandFunc;
int32_t vgId;
int64_t stage;
int32_t role;
STaskStartInfo startInfo;
SRWLatch lock;
int32_t walScanCounter;
void* streamBackend;
int64_t streamBackendRid;
SHashObj* pTaskBackendUnique;
TdThreadMutex backendMutex;
SMetaHbInfo* pHbInfo;
STaskUpdateInfo updateInfo;
int32_t numOfStreamTasks; // this value should be increased when a new task is added into the meta
int32_t numOfPausedTasks;
int32_t chkptNotReadyTasks;
int64_t rid;
int64_t chkpId;
SArray* chkpSaved;
@ -641,6 +669,7 @@ typedef struct SNodeUpdateInfo {
} SNodeUpdateInfo;
typedef struct SStreamTaskNodeUpdateMsg {
int32_t transId; // to identify the msg
int64_t streamId;
int32_t taskId;
SArray* pNodeList; // SArray<SNodeUpdateInfo>
@ -686,24 +715,37 @@ SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t
void streamTaskInputFail(SStreamTask* pTask);
int32_t streamExecTask(SStreamTask* pTask);
int32_t streamSchedExec(SStreamTask* pTask);
bool streamTaskShouldStop(const SStreamStatus* pStatus);
bool streamTaskShouldPause(const SStreamStatus* pStatus);
bool streamTaskShouldStop(const SStreamTask* pStatus);
bool streamTaskShouldPause(const SStreamTask* pStatus);
bool streamTaskIsIdle(const SStreamTask* pTask);
bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus);
char* createStreamTaskIdStr(int64_t streamId, int32_t taskId);
ETaskStatus streamTaskGetStatus(const SStreamTask* pTask, char** pStr);
const char* streamTaskGetStatusStr(ETaskStatus status);
void streamTaskResetStatus(SStreamTask* pTask);
void streamTaskSetStatusReady(SStreamTask* pTask);
void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen);
char* createStreamTaskIdStr(int64_t streamId, int32_t taskId);
// recover and fill history
void streamTaskCheckDownstream(SStreamTask* pTask);
int32_t streamTaskStartScanHistory(SStreamTask* pTask);
int32_t onNormalTaskReady(SStreamTask* pTask);
int32_t onScanhistoryTaskReady(SStreamTask* pTask);
//int32_t streamTaskStartScanHistory(SStreamTask* pTask);
int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage);
int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList);
void streamTaskResetUpstreamStageInfo(SStreamTask* pTask);
bool streamTaskAllUpstreamClosed(SStreamTask* pTask);
bool streamTaskSetSchedStatusWait(SStreamTask* pTask);
int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask);
int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask);
int8_t streamTaskSetSchedStatusInactive(SStreamTask* pTask);
int32_t streamTaskClearHTaskAttr(SStreamTask* pTask);
int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event);
int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM);
void streamTaskRestoreStatus(SStreamTask* pTask);
int32_t streamTaskStop(SStreamTask* pTask);
int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp,
@ -717,12 +759,8 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue);
// common
int32_t streamRestoreParam(SStreamTask* pTask);
int32_t streamSetStatusNormal(SStreamTask* pTask);
int32_t streamSetStatusUnint(SStreamTask* pTask);
const char* streamGetTaskStatusStr(int32_t status);
void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta);
void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta);
void streamTaskResumeFromHalt(SStreamTask* pTask);
void streamTaskResume(SStreamTask* pTask);
void streamTaskDisablePause(SStreamTask* pTask);
void streamTaskEnablePause(SStreamTask* pTask);
int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask);
@ -767,6 +805,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta);
void streamMetaNotifyClose(SStreamMeta* pMeta);
void streamMetaStartHb(SStreamMeta* pMeta);
void streamMetaInitForSnode(SStreamMeta* pMeta);
bool streamMetaTaskInTimer(SStreamMeta* pMeta);
// checkpoint
int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq);

View File

@ -101,6 +101,9 @@ struct STaosQall {
STaosQnode *current;
STaosQnode *start;
int32_t numOfItems;
int64_t memOfItems;
int32_t unAccessedNumOfItems;
int64_t unAccessMemOfItems;
};
STaosQueue *taosOpenQueue();
@ -123,6 +126,9 @@ int32_t taosReadAllQitems(STaosQueue *queue, STaosQall *qall);
int32_t taosGetQitem(STaosQall *qall, void **ppItem);
void taosResetQitems(STaosQall *qall);
int32_t taosQallItemSize(STaosQall *qall);
int64_t taosQallMemSize(STaosQall *qll);
int64_t taosQallUnAccessedItemSize(STaosQall *qall);
int64_t taosQallUnAccessedMemSize(STaosQall *qall);
STaosQset *taosOpenQset();
void taosCloseQset(STaosQset *qset);
@ -135,8 +141,6 @@ int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, SQueueInfo *qinfo)
int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, SQueueInfo *qinfo);
void taosResetQsetThread(STaosQset *qset, void *pItem);
extern int64_t tsRpcQueueMemoryAllowed;
#ifdef __cplusplus
}
#endif

View File

@ -244,7 +244,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch
// internal
int32_t tsTransPullupInterval = 2;
int32_t tsMqRebalanceInterval = 2;
int32_t tsStreamCheckpointInterval = 300;
int32_t tsStreamCheckpointInterval = 60;
float tsSinkDataRate = 2.0;
int32_t tsStreamNodeCheckInterval = 30;
int32_t tsTtlUnit = 86400;

View File

@ -8422,7 +8422,7 @@ void tDestroySubmitTbData(SSubmitTbData *pTbData, int32_t flag) {
} else {
tDestroySVCreateTbReq(pTbData->pCreateTbReq, TSDB_MSG_FLG_DECODE);
}
taosMemoryFree(pTbData->pCreateTbReq);
taosMemoryFreeClear(pTbData->pCreateTbReq);
}
if (pTbData->flags & SUBMIT_REQ_COLUMN_DATA_FORMAT) {

View File

@ -569,6 +569,10 @@ static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStr
}
static void setSinkTaskUpstreamInfo(SArray* pTasksList, const SStreamTask* pUpstreamTask) {
if (taosArrayGetSize(pTasksList) < SINK_NODE_LEVEL || pUpstreamTask == NULL) {
return;
}
SArray* pSinkTaskList = taosArrayGetP(pTasksList, SINK_NODE_LEVEL);
for(int32_t i = 0; i < taosArrayGetSize(pSinkTaskList); ++i) {
SStreamTask* pSinkTask = taosArrayGetP(pSinkTaskList, i);
@ -628,7 +632,9 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan*
}
setSinkTaskUpstreamInfo(pStream->tasks, pAggTask);
setSinkTaskUpstreamInfo(pStream->pHTasksList, pHAggTask);
if (pHAggTask != NULL) {
setSinkTaskUpstreamInfo(pStream->pHTasksList, pHAggTask);
}
// source level
return addSourceTasksForMultiLevelStream(pMnode, pPlan, pStream, pAggTask, pHAggTask, pEpset, nextWindowSkey);

View File

@ -42,22 +42,22 @@ typedef struct SNodeEntry {
int64_t hbTimestamp; // second
} SNodeEntry;
typedef struct SStreamExecNodeInfo {
typedef struct SStreamExecInfo {
SArray *pNodeEntryList;
int64_t ts; // snapshot ts
int64_t activeCheckpoint; // active check point id
SHashObj *pTaskMap;
SArray *pTaskList;
TdThreadMutex lock;
} SStreamExecNodeInfo;
} SStreamExecInfo;
typedef struct SVgroupChangeInfo {
SHashObj *pDBMap;
SArray *pUpdateNodeList; // SArray<SNodeUpdateInfo>
} SVgroupChangeInfo;
static int32_t mndNodeCheckSentinel = 0;
static SStreamExecNodeInfo execNodeList;
static int32_t mndNodeCheckSentinel = 0;
static SStreamExecInfo execInfo;
static int32_t mndStreamActionInsert(SSdb *pSdb, SStreamObj *pStream);
static int32_t mndStreamActionDelete(SSdb *pSdb, SStreamObj *pStream);
@ -77,20 +77,20 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in
int64_t streamId, int32_t taskId);
static int32_t mndProcessNodeCheck(SRpcMsg *pReq);
static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg);
static SArray *extractNodeListFromStream(SMnode *pMnode);
static SArray *mndTakeVgroupSnapshot(SMnode *pMnode);
static SArray *extractNodeListFromStream(SMnode *pMnode);
static SArray *mndTakeVgroupSnapshot(SMnode *pMnode);
static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList);
static STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, const char *name);
static STrans *doCreateTrans1(SMnode *pMnode, const char *name, const char* pDbName);
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,
int32_t retryCode);
static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *pInfo, STrans *pTrans);
static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode);
static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode);
static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode);
static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode);
static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot);
static int32_t doKillActiveCheckpointTrans(SMnode *pMnode);
int32_t mndInitStream(SMnode *pMnode) {
SSdbTable table = {
@ -130,18 +130,18 @@ int32_t mndInitStream(SMnode *pMnode) {
mndAddShowRetrieveHandle(pMnode, TSDB_MGMT_TABLE_STREAM_TASKS, mndRetrieveStreamTask);
mndAddShowFreeIterHandle(pMnode, TSDB_MGMT_TABLE_STREAM_TASKS, mndCancelGetNextStreamTask);
taosThreadMutexInit(&execNodeList.lock, NULL);
execNodeList.pTaskMap = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR), true, HASH_NO_LOCK);
execNodeList.pTaskList = taosArrayInit(4, sizeof(STaskId));
taosThreadMutexInit(&execInfo.lock, NULL);
execInfo.pTaskMap = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR), true, HASH_NO_LOCK);
execInfo.pTaskList = taosArrayInit(4, sizeof(STaskId));
return sdbSetTable(pMnode->pSdb, table);
}
void mndCleanupStream(SMnode *pMnode) {
taosArrayDestroy(execNodeList.pTaskList);
taosHashCleanup(execNodeList.pTaskMap);
taosThreadMutexDestroy(&execNodeList.lock);
mDebug("mnd stream cleanup");
taosArrayDestroy(execInfo.pTaskList);
taosHashCleanup(execInfo.pTaskMap);
taosThreadMutexDestroy(&execInfo.lock);
mDebug("mnd stream exec info cleanup");
}
SSdbRaw *mndStreamActionEncode(SStreamObj *pStream) {
@ -517,7 +517,7 @@ int32_t mndPersistTaskDeployReq(STrans *pTrans, SStreamTask *pTask) {
STransAction action = {0};
action.mTraceId = pTrans->mTraceId;
initTransAction(&action, buf, tlen, TDMT_STREAM_TASK_DEPLOY, &pTask->info.epSet);
initTransAction(&action, buf, tlen, TDMT_STREAM_TASK_DEPLOY, &pTask->info.epSet, 0);
if (mndTransAppendRedoAction(pTrans, &action) != 0) {
taosMemoryFree(buf);
return -1;
@ -689,7 +689,7 @@ static int32_t mndPersistTaskDropReq(STrans *pTrans, SStreamTask *pTask) {
pReq->streamId = pTask->id.streamId;
STransAction action = {0};
initTransAction(&action, pReq, sizeof(SVDropStreamTaskReq), TDMT_STREAM_TASK_DROP, &pTask->info.epSet);
initTransAction(&action, pReq, sizeof(SVDropStreamTaskReq), TDMT_STREAM_TASK_DROP, &pTask->info.epSet, 0);
if (mndTransAppendRedoAction(pTrans, &action) != 0) {
taosMemoryFree(pReq);
return -1;
@ -848,10 +848,10 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) {
mndTransDrop(pTrans);
taosThreadMutexLock(&execNodeList.lock);
taosThreadMutexLock(&execInfo.lock);
mDebug("register to stream task node list");
keepStreamTasksInBuf(&streamObj, &execNodeList);
taosThreadMutexUnlock(&execNodeList.lock);
keepStreamTasksInBuf(&streamObj, &execInfo);
taosThreadMutexUnlock(&execInfo.lock);
code = TSDB_CODE_ACTION_IN_PROGRESS;
@ -883,9 +883,8 @@ static int32_t mndProcessStreamCheckpointTmr(SRpcMsg *pReq) {
return 0;
}
int64_t checkpointId = taosGetTimestampMs();
SMStreamDoCheckpointMsg *pMsg = rpcMallocCont(sizeof(SMStreamDoCheckpointMsg));
pMsg->checkpointId = checkpointId;
pMsg->checkpointId = taosGetTimestampMs();
int32_t size = sizeof(SMStreamDoCheckpointMsg);
SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_BEGIN_CHECKPOINT, .pCont = pMsg, .contLen = size};
@ -1070,7 +1069,7 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream
STransAction action = {0};
SEpSet epset = mndGetVgroupEpset(pMnode, pVgObj);
initTransAction(&action, buf, tlen, TDMT_VND_STREAM_CHECK_POINT_SOURCE, &epset);
initTransAction(&action, buf, tlen, TDMT_VND_STREAM_CHECK_POINT_SOURCE, &epset, TSDB_CODE_SYN_PROPOSE_NOT_READY);
mndReleaseVgroup(pMnode, pVgObj);
if (mndTransAppendRedoAction(pTrans, &action) != 0) {
@ -1085,6 +1084,7 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream
pStream->checkpointId = chkptId;
pStream->checkpointFreq = taosGetTimestampMs();
pStream->currentTick = 0;
// 3. commit log: stream checkpoint info
pStream->version = pStream->version + 1;
@ -1134,22 +1134,22 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) {
{ // check if the node update happens or not
int64_t ts = taosGetTimestampSec();
if (execNodeList.pNodeEntryList == NULL || (taosArrayGetSize(execNodeList.pNodeEntryList) == 0)) {
if (execNodeList.pNodeEntryList != NULL) {
execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList);
if (execInfo.pNodeEntryList == NULL || (taosArrayGetSize(execInfo.pNodeEntryList) == 0)) {
if (execInfo.pNodeEntryList != NULL) {
execInfo.pNodeEntryList = taosArrayDestroy(execInfo.pNodeEntryList);
}
execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode);
execInfo.pNodeEntryList = extractNodeListFromStream(pMnode);
}
if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) {
if (taosArrayGetSize(execInfo.pNodeEntryList) == 0) {
mDebug("stream task node change checking done, no vgroups exist, do nothing");
execNodeList.ts = ts;
execInfo.ts = ts;
return 0;
}
for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) {
SNodeEntry* pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, i);
for(int32_t i = 0; i < taosArrayGetSize(execInfo.pNodeEntryList); ++i) {
SNodeEntry* pNodeEntry = taosArrayGet(execInfo.pNodeEntryList, i);
if (pNodeEntry->stageUpdated) {
mDebug("stream task not ready due to node update detected, checkpoint not issued");
return 0;
@ -1158,7 +1158,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) {
SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode);
SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot);
SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execInfo.pNodeEntryList, pNodeSnapshot);
bool nodeUpdated = (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0);
taosArrayDestroy(changeInfo.pUpdateNodeList);
taosHashCleanup(changeInfo.pDBMap);
@ -1170,26 +1170,25 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) {
}
}
{ // check if all tasks are in TASK_STATUS__NORMAL status
{ // check if all tasks are in TASK_STATUS__READY status
bool ready = true;
taosThreadMutexLock(&execNodeList.lock);
for (int32_t i = 0; i < taosArrayGetSize(execNodeList.pTaskList); ++i) {
STaskId *p = taosArrayGet(execNodeList.pTaskList, i);
STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, p, sizeof(*p));
taosThreadMutexLock(&execInfo.lock);
for (int32_t i = 0; i < taosArrayGetSize(execInfo.pTaskList); ++i) {
STaskId *p = taosArrayGet(execInfo.pTaskList, i);
STaskStatusEntry* pEntry = taosHashGet(execInfo.pTaskMap, p, sizeof(*p));
if (pEntry == NULL) {
continue;
}
if (pEntry->status != TASK_STATUS__NORMAL) {
if (pEntry->status != TASK_STATUS__READY) {
mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, checkpoint msg not issued",
pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamGetTaskStatusStr(pEntry->status));
pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamTaskGetStatusStr(pEntry->status));
ready = false;
break;
}
}
taosThreadMutexUnlock(&execNodeList.lock);
taosThreadMutexUnlock(&execInfo.lock);
if (!ready) {
return 0;
}
@ -1203,7 +1202,8 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) {
mError("failed to trigger checkpoint, reason: %s", tstrerror(TSDB_CODE_OUT_OF_MEMORY));
return -1;
}
mDebug("start to trigger checkpoint, checkpointId: %" PRId64 "", checkpointId);
mDebug("start to trigger checkpoint, checkpointId: %" PRId64, checkpointId);
const char *pDb = mndGetStreamDB(pMnode);
mndTransSetDbName(pTrans, pDb, "checkpoint");
@ -1229,11 +1229,16 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) {
if (code == 0) {
if (mndTransPrepare(pMnode, pTrans) != 0) {
mError("failed to prepre trans rebalance since %s", terrstr());
mError("failed to prepare trans rebalance since %s", terrstr());
}
}
mndTransDrop(pTrans);
// only one trans here
taosThreadMutexLock(&execInfo.lock);
execInfo.activeCheckpoint = checkpointId;
taosThreadMutexUnlock(&execInfo.lock);
return code;
}
@ -1311,7 +1316,7 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) {
return -1;
}
removeStreamTasksInBuf(pStream, &execNodeList);
removeStreamTasksInBuf(pStream, &execInfo);
SName name = {0};
tNameFromString(&name, dropReq.name, T_NAME_ACCT | T_NAME_DB);
@ -1562,12 +1567,12 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock
char status[20 + VARSTR_HEADER_SIZE] = {0};
STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId};
STaskStatusEntry* pe = taosHashGet(execNodeList.pTaskMap, &id, sizeof(id));
STaskStatusEntry* pe = taosHashGet(execInfo.pTaskMap, &id, sizeof(id));
if (pe == NULL) {
continue;
}
const char* pStatus = streamGetTaskStatusStr(pe->status);
const char* pStatus = streamTaskGetStatusStr(pe->status);
STR_TO_VARSTR(status, pStatus);
// status
@ -1642,7 +1647,7 @@ static int32_t mndPauseStreamTask(STrans *pTrans, SStreamTask *pTask) {
pReq->streamId = pTask->id.streamId;
STransAction action = {0};
initTransAction(&action, pReq, sizeof(SVPauseStreamTaskReq), TDMT_STREAM_TASK_PAUSE, &pTask->info.epSet);
initTransAction(&action, pReq, sizeof(SVPauseStreamTaskReq), TDMT_STREAM_TASK_PAUSE, &pTask->info.epSet, 0);
if (mndTransAppendRedoAction(pTrans, &action) != 0) {
taosMemoryFree(pReq);
return -1;
@ -1775,7 +1780,7 @@ static int32_t mndResumeStreamTask(STrans *pTrans, SStreamTask *pTask, int8_t ig
pReq->igUntreated = igUntreated;
STransAction action = {0};
initTransAction(&action, pReq, sizeof(SVResumeStreamTaskReq), TDMT_STREAM_TASK_RESUME, &pTask->info.epSet);
initTransAction(&action, pReq, sizeof(SVResumeStreamTaskReq), TDMT_STREAM_TASK_RESUME, &pTask->info.epSet, 0);
if (mndTransAppendRedoAction(pTrans, &action) != 0) {
taosMemoryFree(pReq);
return -1;
@ -1879,18 +1884,19 @@ static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) {
return TSDB_CODE_ACTION_IN_PROGRESS;
}
static void initNodeUpdateMsg(SStreamTaskNodeUpdateMsg *pMsg, const SVgroupChangeInfo *pInfo, int64_t streamId,
int32_t taskId) {
pMsg->streamId = streamId;
pMsg->taskId = taskId;
static void initNodeUpdateMsg(SStreamTaskNodeUpdateMsg *pMsg, const SVgroupChangeInfo *pInfo, SStreamTaskId *pId,
int32_t transId) {
pMsg->streamId = pId->streamId;
pMsg->taskId = pId->taskId;
pMsg->transId = transId;
pMsg->pNodeList = taosArrayInit(taosArrayGetSize(pInfo->pUpdateNodeList), sizeof(SNodeUpdateInfo));
taosArrayAddAll(pMsg->pNodeList, pInfo->pUpdateNodeList);
}
static int32_t doBuildStreamTaskUpdateMsg(void **pBuf, int32_t *pLen, SVgroupChangeInfo *pInfo, int32_t nodeId,
int64_t streamId, int32_t taskId) {
SStreamTaskId* pId, int32_t transId) {
SStreamTaskNodeUpdateMsg req = {0};
initNodeUpdateMsg(&req, pInfo, streamId, taskId);
initNodeUpdateMsg(&req, pInfo, pId, transId);
int32_t code = 0;
int32_t blen;
@ -1954,19 +1960,21 @@ int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans) {
return 0;
}
void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset) {
void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset,
int32_t retryCode) {
pAction->epSet = *pEpset;
pAction->contLen = contLen;
pAction->pCont = pCont;
pAction->msgType = msgType;
pAction->retryCode = retryCode;
}
// todo extract method: traverse stream tasks
// build trans to update the epset
static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *pInfo, STrans* pTrans) {
mDebug("start to build stream:0x%" PRIx64 " task DAG update", pStream->uid);
static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *pInfo, STrans *pTrans) {
mDebug("start to build stream:0x%" PRIx64 " tasks epset update", pStream->uid);
taosWLockLatch(&pStream->lock);
taosWLockLatch(&pStream->lock);
int32_t numOfLevels = taosArrayGetSize(pStream->tasks);
for (int32_t j = 0; j < numOfLevels; ++j) {
@ -1979,10 +1987,10 @@ static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *p
void *pBuf = NULL;
int32_t len = 0;
streamTaskUpdateEpsetInfo(pTask, pInfo->pUpdateNodeList);
doBuildStreamTaskUpdateMsg(&pBuf, &len, pInfo, pTask->info.nodeId, pTask->id.streamId, pTask->id.taskId);
doBuildStreamTaskUpdateMsg(&pBuf, &len, pInfo, pTask->info.nodeId, &pTask->id, pTrans->id);
STransAction action = {0};
initTransAction(&action, pBuf, len, TDMT_VND_STREAM_TASK_UPDATE, &pTask->info.epSet);
initTransAction(&action, pBuf, len, TDMT_VND_STREAM_TASK_UPDATE, &pTask->info.epSet, 0);
if (mndTransAppendRedoAction(pTrans, &action) != 0) {
taosMemoryFree(pBuf);
taosWUnLockLatch(&pStream->lock);
@ -2030,8 +2038,8 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP
char buf[256] = {0};
EPSET_TO_STR(&pCurrent->epset, buf);
mDebug("nodeId:%d epset changed detected, old:%s:%d -> new:%s", pCurrent->nodeId, pPrevEp->fqdn,
pPrevEp->port, buf);
mDebug("nodeId:%d restart/epset changed detected, old:%s:%d -> new:%s, stageUpdate:%d", pCurrent->nodeId,
pPrevEp->fqdn, pPrevEp->port, buf, pPrevEntry->stageUpdated);
SNodeUpdateInfo updateInfo = {.nodeId = pPrevEntry->nodeId};
epsetAssign(&updateInfo.prevEp, &pPrevEntry->epset);
@ -2069,6 +2077,9 @@ static SArray *mndTakeVgroupSnapshot(SMnode *pMnode) {
entry.nodeId = pVgroup->vgId;
entry.hbTimestamp = -1;
char buf[256] = {0};
EPSET_TO_STR(&entry.epset, buf);
mDebug("take node snapshot, nodeId:%d %s", entry.nodeId, buf);
taosArrayPush(pVgroupListSnapshot, &entry);
sdbRelease(pSdb, pVgroup);
}
@ -2082,8 +2093,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange
// check all streams that involved this vnode should update the epset info
SStreamObj *pStream = NULL;
void *pIter = NULL;
STrans *pTrans = NULL;
STrans *pTrans = NULL;
while (1) {
pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream);
@ -2091,6 +2101,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange
break;
}
// here create only one trans
if (pTrans == NULL) {
pTrans = doCreateTrans(pMnode, pStream, "stream-task-update");
if (pTrans == NULL) {
@ -2137,8 +2148,6 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange
sdbRelease(pMnode->pSdb, pStream);
mndTransDrop(pTrans);
// return TSDB_CODE_ACTION_IN_PROGRESS;
return 0;
}
@ -2181,6 +2190,10 @@ static SArray *extractNodeListFromStream(SMnode *pMnode) {
while ((pIter = taosHashIterate(pHash, pIter)) != NULL) {
SNodeEntry *pEntry = (SNodeEntry *)pIter;
taosArrayPush(plist, pEntry);
char buf[256] = {0};
EPSET_TO_STR(&pEntry->epset, buf);
mDebug("extract nodeInfo from stream obj, nodeId:%d, %s", pEntry->nodeId, buf);
}
taosHashCleanup(pHash);
@ -2198,12 +2211,12 @@ static void doExtractTasksFromStream(SMnode *pMnode) {
break;
}
keepStreamTasksInBuf(pStream, &execNodeList);
keepStreamTasksInBuf(pStream, &execInfo);
sdbRelease(pSdb, pStream);
}
}
static int32_t doRemoveFromTask(SStreamExecNodeInfo* pExecNode, STaskId* pRemovedId) {
static int32_t doRemoveTasks(SStreamExecInfo* pExecNode, STaskId* pRemovedId) {
void *p = taosHashGet(pExecNode->pTaskMap, pRemovedId, sizeof(*pRemovedId));
if (p != NULL) {
@ -2236,31 +2249,31 @@ static bool taskNodeExists(SArray* pList, int32_t nodeId) {
}
int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) {
SArray* pRemoveTaskList = taosArrayInit(4, sizeof(STaskId));
SArray* pRemovedTasks = taosArrayInit(4, sizeof(STaskId));
int32_t numOfTask = taosArrayGetSize(execNodeList.pTaskList);
int32_t numOfTask = taosArrayGetSize(execInfo.pTaskList);
for(int32_t i = 0; i < numOfTask; ++i) {
STaskId* pId = taosArrayGet(execNodeList.pTaskList, i);
STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, pId, sizeof(*pId));
STaskId* pId = taosArrayGet(execInfo.pTaskList, i);
STaskStatusEntry* pEntry = taosHashGet(execInfo.pTaskMap, pId, sizeof(*pId));
bool existed = taskNodeExists(pNodeSnapshot, pEntry->nodeId);
if (!existed) {
taosArrayPush(pRemoveTaskList, pId);
taosArrayPush(pRemovedTasks, pId);
}
}
for(int32_t i = 0; i < taosArrayGetSize(pRemoveTaskList); ++i) {
STaskId* pId = taosArrayGet(pRemoveTaskList, i);
doRemoveFromTask(&execNodeList, pId);
for(int32_t i = 0; i < taosArrayGetSize(pRemovedTasks); ++i) {
STaskId* pId = taosArrayGet(pRemovedTasks, i);
doRemoveTasks(&execInfo, pId);
}
mDebug("remove invalid stream tasks:%d, remain:%d", (int32_t)taosArrayGetSize(pRemoveTaskList),
(int32_t) taosArrayGetSize(execNodeList.pTaskList));
mDebug("remove invalid stream tasks:%d, remain:%d", (int32_t)taosArrayGetSize(pRemovedTasks),
(int32_t) taosArrayGetSize(execInfo.pTaskList));
int32_t size = taosArrayGetSize(pNodeSnapshot);
SArray* pValidNodeEntryList = taosArrayInit(4, sizeof(SNodeEntry));
for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) {
SNodeEntry* p = taosArrayGet(execNodeList.pNodeEntryList, i);
for(int32_t i = 0; i < taosArrayGetSize(execInfo.pNodeEntryList); ++i) {
SNodeEntry* p = taosArrayGet(execInfo.pNodeEntryList, i);
for(int32_t j = 0; j < size; ++j) {
SNodeEntry* pEntry = taosArrayGet(pNodeSnapshot, j);
@ -2271,10 +2284,11 @@ int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) {
}
}
execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList);
execNodeList.pNodeEntryList = pValidNodeEntryList;
execInfo.pNodeEntryList = taosArrayDestroy(execInfo.pNodeEntryList);
execInfo.pNodeEntryList = pValidNodeEntryList;
taosArrayDestroy(pRemoveTaskList);
mDebug("remain %d valid node entries", (int32_t) taosArrayGetSize(pValidNodeEntryList));
taosArrayDestroy(pRemovedTasks);
return 0;
}
@ -2291,43 +2305,48 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) {
int64_t ts = taosGetTimestampSec();
SMnode *pMnode = pMsg->info.node;
if (execNodeList.pNodeEntryList == NULL || (taosArrayGetSize(execNodeList.pNodeEntryList) == 0)) {
if (execNodeList.pNodeEntryList != NULL) {
execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList);
if (execInfo.pNodeEntryList == NULL || (taosArrayGetSize(execInfo.pNodeEntryList) == 0)) {
if (execInfo.pNodeEntryList != NULL) {
execInfo.pNodeEntryList = taosArrayDestroy(execInfo.pNodeEntryList);
}
execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode);
execInfo.pNodeEntryList = extractNodeListFromStream(pMnode);
}
if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) {
if (taosArrayGetSize(execInfo.pNodeEntryList) == 0) {
mDebug("end to do stream task node change checking, no vgroup exists, do nothing");
execNodeList.ts = ts;
execInfo.ts = ts;
atomic_store_32(&mndNodeCheckSentinel, 0);
return 0;
}
SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode);
taosThreadMutexLock(&execNodeList.lock);
taosThreadMutexLock(&execInfo.lock);
removeExpirednodeEntryAndTask(pNodeSnapshot);
SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot);
SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execInfo.pNodeEntryList, pNodeSnapshot);
if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) {
// kill current active checkpoint transaction, since the transaction is vnode wide.
doKillActiveCheckpointTrans(pMnode);
code = mndProcessVgroupChange(pMnode, &changeInfo);
// keep the new vnode snapshot
if (code == TSDB_CODE_SUCCESS || code == TSDB_CODE_ACTION_IN_PROGRESS) {
mDebug("create trans successfully, update cached node list");
taosArrayDestroy(execNodeList.pNodeEntryList);
execNodeList.pNodeEntryList = pNodeSnapshot;
execNodeList.ts = ts;
taosArrayDestroy(execInfo.pNodeEntryList);
execInfo.pNodeEntryList = pNodeSnapshot;
execInfo.ts = ts;
} else {
mDebug("unexpect code during create nodeUpdate trans, code:%s", tstrerror(code));
taosArrayDestroy(pNodeSnapshot);
}
} else {
mDebug("no update found in nodeList");
taosArrayDestroy(pNodeSnapshot);
}
taosThreadMutexUnlock(&execNodeList.lock);
taosThreadMutexUnlock(&execInfo.lock);
taosArrayDestroy(changeInfo.pUpdateNodeList);
taosHashCleanup(changeInfo.pDBMap);
@ -2359,7 +2378,7 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) {
return 0;
}
void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) {
void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) {
int32_t level = taosArrayGetSize(pStream->tasks);
for (int32_t i = 0; i < level; i++) {
@ -2384,7 +2403,7 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) {
}
}
void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode) {
void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecInfo * pExecNode) {
int32_t level = taosArrayGetSize(pStream->tasks);
for (int32_t i = 0; i < level; i++) {
SArray *pLevel = taosArrayGetP(pStream->tasks, i);
@ -2467,7 +2486,7 @@ int32_t createStreamResetStatusTrans(SMnode* pMnode, SStreamObj* pStream) {
pReq->streamId = pTask->id.streamId;
STransAction action = {0};
initTransAction(&action, pReq, sizeof(SVResetStreamTaskReq), TDMT_VND_STREAM_TASK_RESET, &pTask->info.epSet);
initTransAction(&action, pReq, sizeof(SVResetStreamTaskReq), TDMT_VND_STREAM_TASK_RESET, &pTask->info.epSet, 0);
if (mndTransAppendRedoAction(pTrans, &action) != 0) {
taosMemoryFree(pReq);
taosWUnLockLatch(&pStream->lock);
@ -2498,29 +2517,26 @@ int32_t createStreamResetStatusTrans(SMnode* pMnode, SStreamObj* pStream) {
return TSDB_CODE_ACTION_IN_PROGRESS;
}
int32_t mndResetFromCheckpoint(SMnode* pMnode) {
// find the checkpoint trans id
int32_t doKillActiveCheckpointTrans(SMnode *pMnode) {
int32_t transId = 0;
SSdb *pSdb = pMnode->pSdb;
STrans *pTrans = NULL;
void *pIter = NULL;
{
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);
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) {
@ -2528,8 +2544,16 @@ int32_t mndResetFromCheckpoint(SMnode* pMnode) {
return TSDB_CODE_SUCCESS;
}
STrans* pTrans = mndAcquireTrans(pMnode, transId);
pTrans = mndAcquireTrans(pMnode, transId);
mInfo("kill checkpoint trans:%d", transId);
mndKillTrans(pMnode, pTrans);
mndReleaseTrans(pMnode, pTrans);
return TSDB_CODE_SUCCESS;
}
int32_t mndResetFromCheckpoint(SMnode* pMnode) {
doKillActiveCheckpointTrans(pMnode);
// set all tasks status to be normal, refactor later to be stream level, instead of vnode level.
SSdb *pSdb = pMnode->pSdb;
@ -2541,6 +2565,7 @@ int32_t mndResetFromCheckpoint(SMnode* pMnode) {
break;
}
// todo this transaction should exist be only one
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) {
@ -2571,24 +2596,24 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) {
mTrace("receive stream-meta hb from vgId:%d, active numOfTasks:%d", req.vgId, req.numOfTasks);
taosThreadMutexLock(&execNodeList.lock);
int32_t numOfExisted = taosHashGetSize(execNodeList.pTaskMap);
taosThreadMutexLock(&execInfo.lock);
int32_t numOfExisted = taosHashGetSize(execInfo.pTaskMap);
if (numOfExisted == 0) {
doExtractTasksFromStream(pMnode);
}
for (int32_t i = 0; i < req.numOfTasks; ++i) {
STaskStatusEntry *p = taosArrayGet(req.pTaskStatus, i);
STaskStatusEntry *pEntry = taosHashGet(execNodeList.pTaskMap, &p->id, sizeof(p->id));
STaskStatusEntry *pEntry = taosHashGet(execInfo.pTaskMap, &p->id, sizeof(p->id));
if (pEntry == NULL) {
mError("s-task:0x%" PRIx64 " not found in mnode task list", p->id.taskId);
continue;
}
if (p->stage != pEntry->stage && pEntry->stage != -1) {
int32_t numOfNodes = taosArrayGetSize(execNodeList.pNodeEntryList);
int32_t numOfNodes = taosArrayGetSize(execInfo.pNodeEntryList);
for(int32_t j = 0; j < numOfNodes; ++j) {
SNodeEntry* pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, j);
SNodeEntry* pNodeEntry = taosArrayGet(execInfo.pNodeEntryList, j);
if (pNodeEntry->nodeId == pEntry->nodeId) {
mInfo("vgId:%d stage updated, from %d to %d, nodeUpdate trigger by s-task:0x%" PRIx64,
pEntry->nodeId, pEntry->stage, p->stage, pEntry->id.taskId);
@ -2614,24 +2639,20 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) {
}
pEntry->status = p->status;
if (p->status != TASK_STATUS__NORMAL) {
mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamGetTaskStatusStr(p->status));
if (p->status != TASK_STATUS__READY) {
mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamTaskGetStatusStr(p->status));
}
}
// 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);
}
// if the execInfo.activeCheckpoint == 0, the checkpoint is restoring from wal
mInfo("checkpointId:%" PRId64 " failed, issue task-reset trans to reset all tasks status", execInfo.activeCheckpoint);
mndResetFromCheckpoint(pMnode);
}
taosThreadMutexUnlock(&execNodeList.lock);
taosThreadMutexUnlock(&execInfo.lock);
taosArrayDestroy(req.pTaskStatus);
return TSDB_CODE_SUCCESS;

View File

@ -92,11 +92,13 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProcessVer
}
}
qInfo("snode:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64
" child id:%d, level:%d, status:%s fill-history:%d, trigger:%" PRId64 " ms",
char* p = NULL;
streamTaskGetStatus(pTask, &p);
qInfo("snode:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64
" nextProcessVer:%" PRId64 " child id:%d, level:%d, status:%s fill-history:%d, trigger:%" PRId64 " ms",
SNODE_HANDLE, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer,
pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus),
pTask->info.fillHistory, pTask->info.triggerParam);
pTask->info.selfChildId, pTask->info.taskLevel, p, pTask->info.fillHistory, pTask->info.triggerParam);
return 0;
}
@ -174,9 +176,15 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) {
int32_t numOfTasks = streamMetaGetNumOfTasks(pSnode->pMeta);
taosWUnLockLatch(&pSnode->pMeta->lock);
qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE, pTask->id.idStr,
streamGetTaskStatusStr(pTask->status.taskStatus), numOfTasks);
char* p = NULL;
streamTaskGetStatus(pTask, &p);
qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE,
pTask->id.idStr, p, numOfTasks);
EStreamTaskEvent event = (HAS_RELATED_FILLHISTORY_TASK(pTask)) ? TASK_EVENT_INIT_STREAM_SCANHIST : TASK_EVENT_INIT;
streamTaskHandleEvent(pTask->status.pSM, event);
streamTaskCheckDownstream(pTask);
return 0;
}
@ -352,10 +360,10 @@ int32_t sndProcessStreamTaskCheckReq(SSnode *pSnode, SRpcMsg *pMsg) {
if (pTask != NULL) {
rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage);
streamMetaReleaseTask(pSnode->pMeta, pTask);
const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus);
char* p = NULL;
streamTaskGetStatus(pTask, &p);
qDebug("s-task:%s status:%s, recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d",
pTask->id.idStr, pStatus, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status);
pTask->id.idStr, p, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status);
} else {
rsp.status = TASK_DOWNSTREAM_NOT_READY;
qDebug("recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 ") from task:0x%x (vgId:%d), rsp status %d",

View File

@ -125,7 +125,6 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR
int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, SMqDataRsp* pRsp, int32_t numOfCols, int8_t precision);
int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataRsp* pRsp,
int32_t type, int32_t vgId);
//int32_t tqPushDataRsp(STqHandle* pHandle, int32_t vgId);
int32_t tqPushEmptyDataRsp(STqHandle* pHandle, int32_t vgId);
// tqMeta
@ -133,7 +132,6 @@ int32_t tqMetaOpen(STQ* pTq);
int32_t tqMetaClose(STQ* pTq);
int32_t tqMetaSaveHandle(STQ* pTq, const char* key, const STqHandle* pHandle);
int32_t tqMetaDeleteHandle(STQ* pTq, const char* key);
//int32_t tqMetaRestoreHandle(STQ* pTq);
int32_t tqMetaSaveCheckInfo(STQ* pTq, const char* key, const void* value, int32_t vLen);
int32_t tqMetaDeleteCheckInfo(STQ* pTq, const char* key);
int32_t tqMetaRestoreCheckInfo(STQ* pTq);
@ -159,7 +157,7 @@ int32_t tqOffsetRestoreFromFile(STqOffsetStore* pStore, const char* fname);
// tqStream
int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver);
int32_t tqScanWal(STQ* pTq);
int32_t tqCheckAndRunStreamTask(STQ* pTq);
int32_t tqStartStreamTask(STQ* pTq);
int32_t tqStartStreamTasks(STQ* pTq);
int32_t tqStopStreamTasks(STQ* pTq);
@ -170,6 +168,12 @@ int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp*
int32_t type, int64_t sver, int64_t ever);
int32_t tqInitDataRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset);
void tqUpdateNodeStage(STQ* pTq, bool isLeader);
int32_t setDstTableDataPayload(uint64_t suid, const STSchema* pTSchema, int32_t blockIndex, SSDataBlock* pDataBlock,
SSubmitTbData* pTableData, const char* id);
int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, const char* id);
SVCreateTbReq* buildAutoCreateTableReq(const char* stbFullName, int64_t suid, int32_t numOfCols,
SSDataBlock* pDataBlock, SArray* pTagArray);
#ifdef __cplusplus
}

View File

@ -35,8 +35,8 @@ int32_t tdProcessTSmaInsert(SSma *pSma, int64_t indexUid, const char *msg) {
return code;
}
int32_t tdProcessTSmaCreate(SSma *pSma, int64_t version, const char *msg) {
int32_t code = tdProcessTSmaCreateImpl(pSma, version, msg);
int32_t tdProcessTSmaCreate(SSma *pSma, int64_t ver, const char *msg) {
int32_t code = tdProcessTSmaCreateImpl(pSma, ver, msg);
return code;
}
@ -109,7 +109,7 @@ _exit:
* @param pMsg
* @return int32_t
*/
static int32_t tdProcessTSmaCreateImpl(SSma *pSma, int64_t version, const char *pMsg) {
static int32_t tdProcessTSmaCreateImpl(SSma *pSma, int64_t ver, const char *pMsg) {
int32_t code = 0;
int32_t lino = 0;
SSmaCfg *pCfg = (SSmaCfg *)pMsg;
@ -118,7 +118,7 @@ static int32_t tdProcessTSmaCreateImpl(SSma *pSma, int64_t version, const char *
if (TD_VID(pSma->pVnode) == pCfg->dstVgId) {
// create tsma meta in dstVgId
if (metaCreateTSma(SMA_META(pSma), version, pCfg) < 0) {
if (metaCreateTSma(SMA_META(pSma), ver, pCfg) < 0) {
code = terrno;
TSDB_CHECK_CODE(code, lino, _exit);
}
@ -130,7 +130,7 @@ static int32_t tdProcessTSmaCreateImpl(SSma *pSma, int64_t version, const char *
pReq.schemaRow = pCfg->schemaRow;
pReq.schemaTag = pCfg->schemaTag;
if (metaCreateSTable(SMA_META(pSma), version, &pReq) < 0) {
if (metaCreateSTable(SMA_META(pSma), ver, &pReq) < 0) {
code = terrno;
TSDB_CHECK_CODE(code, lino, _exit);
}
@ -154,94 +154,36 @@ _exit:
return code;
}
int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema *pTSchema,
SSchemaWrapper *pTagSchemaWrapper, bool createTb, int64_t suid, const char *stbFullName,
SBatchDeleteReq *pDeleteReq, void **ppData, int32_t *pLen) {
int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema *pTSchema, int64_t suid,
const char *stbFullName, SBatchDeleteReq *pDeleteReq, void **ppData, int32_t *pLen) {
int32_t code = 0;
int32_t lino = 0;
void *pBuf = NULL;
int32_t len = 0;
SSubmitReq2 *pReq = NULL;
SArray *tagArray = NULL;
SArray *createTbArray = NULL;
SArray *pVals = NULL;
int32_t sz = taosArrayGetSize(pBlocks);
int32_t numOfBlocks = taosArrayGetSize(pBlocks);
tagArray = taosArrayInit(1, sizeof(STagVal));
createTbArray = taosArrayInit(sz, POINTER_BYTES);
pReq = taosMemoryCalloc(1, sizeof(SSubmitReq2));
pReq->aSubmitTbData = taosArrayInit(1, sizeof(SSubmitTbData));
if(!tagArray || !createTbArray || !pReq || !pReq->aSubmitTbData) {
if (!tagArray || !pReq) {
code = terrno == TSDB_CODE_SUCCESS ? TSDB_CODE_OUT_OF_MEMORY : terrno;
TSDB_CHECK_CODE(code, lino, _exit);
}
// create table req
if (createTb) {
for (int32_t i = 0; i < sz; ++i) {
SSDataBlock *pDataBlock = taosArrayGet(pBlocks, i);
SVCreateTbReq *pCreateTbReq = NULL;
if (pDataBlock->info.type == STREAM_DELETE_RESULT) {
taosArrayPush(createTbArray, &pCreateTbReq);
continue;
}
if (!(pCreateTbReq = taosMemoryCalloc(1, sizeof(SVCreateStbReq)))) {
code = TSDB_CODE_OUT_OF_MEMORY;
TSDB_CHECK_CODE(code, lino, _exit);
};
// don't move to the end of loop as to destroy in the end of func when error occur
taosArrayPush(createTbArray, &pCreateTbReq);
// set const
pCreateTbReq->flags = 0;
pCreateTbReq->type = TSDB_CHILD_TABLE;
pCreateTbReq->ctb.suid = suid;
// set super table name
SName name = {0};
tNameFromString(&name, stbFullName, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE);
pCreateTbReq->ctb.stbName = taosStrdup((char *)tNameGetTableName(&name)); // taosStrdup(stbFullName);
// set tag content
taosArrayClear(tagArray);
STagVal tagVal = {
.cid = taosArrayGetSize(pDataBlock->pDataBlock) + 1,
.type = TSDB_DATA_TYPE_UBIGINT,
.i64 = (int64_t)pDataBlock->info.id.groupId,
};
taosArrayPush(tagArray, &tagVal);
pCreateTbReq->ctb.tagNum = taosArrayGetSize(tagArray);
STag *pTag = NULL;
tTagNew(tagArray, 1, false, &pTag);
if (pTag == NULL) {
code = TSDB_CODE_OUT_OF_MEMORY;
TSDB_CHECK_CODE(code, lino, _exit);
}
pCreateTbReq->ctb.pTag = (uint8_t *)pTag;
// set tag name
SArray *tagName = taosArrayInit(1, TSDB_COL_NAME_LEN);
char tagNameStr[TSDB_COL_NAME_LEN] = {0};
strcpy(tagNameStr, "group_id");
taosArrayPush(tagName, tagNameStr);
pCreateTbReq->ctb.tagName = tagName;
// set table name
if (pDataBlock->info.parTbName[0]) {
pCreateTbReq->name = taosStrdup(pDataBlock->info.parTbName);
} else {
pCreateTbReq->name = buildCtbNameByGroupId(stbFullName, pDataBlock->info.id.groupId);
}
}
pReq->aSubmitTbData = taosArrayInit(1, sizeof(SSubmitTbData));
if (pReq->aSubmitTbData == NULL) {
code = terrno == TSDB_CODE_SUCCESS ? TSDB_CODE_OUT_OF_MEMORY : terrno;
TSDB_CHECK_CODE(code, lino, _exit);
}
SHashObj *pTableIndexMap =
taosHashInit(numOfBlocks, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false, HASH_NO_LOCK);
// SSubmitTbData req
for (int32_t i = 0; i < sz; ++i) {
for (int32_t i = 0; i < numOfBlocks; ++i) {
SSDataBlock *pDataBlock = taosArrayGet(pBlocks, i);
if (pDataBlock->info.type == STREAM_DELETE_RESULT) {
pDeleteReq->suid = suid;
@ -250,62 +192,42 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema *
continue;
}
int32_t rows = pDataBlock->info.rows;
SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version, .flags = SUBMIT_REQ_AUTO_CREATE_TABLE,};
SSubmitTbData tbData = {0};
int32_t cid = taosArrayGetSize(pDataBlock->pDataBlock) + 1;
tbData.pCreateTbReq = buildAutoCreateTableReq(stbFullName, suid, cid, pDataBlock, tagArray);
if (!(tbData.aRowP = taosArrayInit(rows, sizeof(SRow *)))) {
code = terrno;
TSDB_CHECK_CODE(code, lino, _exit);
}
tbData.suid = suid;
tbData.uid = 0; // uid is assigned by vnode
tbData.sver = pTSchema->version;
{
uint64_t groupId = pDataBlock->info.id.groupId;
if (createTb) {
tbData.pCreateTbReq = taosArrayGetP(createTbArray, i);
if (tbData.pCreateTbReq) tbData.flags = SUBMIT_REQ_AUTO_CREATE_TABLE;
}
int32_t *index = taosHashGet(pTableIndexMap, &groupId, sizeof(groupId));
if (index == NULL) { // no data yet, append it
code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, "");
if (code != TSDB_CODE_SUCCESS) {
continue;
}
if (!pVals && !(pVals = taosArrayInit(pTSchema->numOfCols, sizeof(SColVal)))) {
taosArrayDestroy(tbData.aRowP);
code = terrno;
TSDB_CHECK_CODE(code, lino, _exit);
}
taosArrayPush(pReq->aSubmitTbData, &tbData);
for (int32_t j = 0; j < rows; ++j) {
taosArrayClear(pVals);
for (int32_t k = 0; k < pTSchema->numOfCols; k++) {
const STColumn *pCol = &pTSchema->columns[k];
SColumnInfoData *pColData = taosArrayGet(pDataBlock->pDataBlock, k);
if (colDataIsNull_s(pColData, j)) {
SColVal cv = COL_VAL_NULL(pCol->colId, pCol->type);
taosArrayPush(pVals, &cv);
} else {
void *data = colDataGetData(pColData, j);
if (IS_STR_DATA_TYPE(pCol->type)) {
SValue sv = (SValue){.nData = varDataLen(data), .pData = varDataVal(data)}; // address copy, no value
SColVal cv = COL_VAL_VALUE(pCol->colId, pCol->type, sv);
taosArrayPush(pVals, &cv);
} else {
SValue sv;
memcpy(&sv.val, data, tDataTypes[pCol->type].bytes);
SColVal cv = COL_VAL_VALUE(pCol->colId, pCol->type, sv);
taosArrayPush(pVals, &cv);
}
int32_t size = (int32_t)taosArrayGetSize(pReq->aSubmitTbData) - 1;
taosHashPut(pTableIndexMap, &groupId, sizeof(groupId), &size, sizeof(size));
} else {
code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, "");
if (code != TSDB_CODE_SUCCESS) {
continue;
}
SSubmitTbData *pExisted = taosArrayGet(pReq->aSubmitTbData, *index);
code = doMergeExistedRows(pExisted, &tbData, "id");
if (code != TSDB_CODE_SUCCESS) {
continue;
}
}
SRow *pRow = NULL;
if ((code = tRowBuild(pVals, (STSchema *)pTSchema, &pRow)) < 0) {
tDestroySubmitTbData(&tbData, TSDB_MSG_FLG_ENCODE);
TSDB_CHECK_CODE(code, lino, _exit);
}
taosArrayPush(tbData.aRowP, &pRow);
}
taosArrayPush(pReq->aSubmitTbData, &tbData);
}
taosHashCleanup(pTableIndexMap);
// encode
tEncodeSize(tEncodeSubmitReq, pReq, len, code);
if (TSDB_CODE_SUCCESS == code) {
@ -327,11 +249,10 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema *
}
tEncoderClear(&encoder);
}
_exit:
taosArrayDestroy(createTbArray);
taosArrayDestroy(tagArray);
taosArrayDestroy(pVals);
if (pReq) {
if (pReq != NULL) {
tDestroySubmitReq(pReq, TSDB_MSG_FLG_ENCODE);
taosMemoryFree(pReq);
}
@ -442,8 +363,8 @@ static int32_t tdProcessTSmaInsertImpl(SSma *pSma, int64_t indexUid, const char
void *pSubmitReq = NULL;
int32_t contLen = 0;
code = smaBlockToSubmit(pSma->pVnode, (const SArray *)msg, pTsmaStat->pTSchema, &pTsmaStat->pTSma->schemaTag, true,
pTsmaStat->pTSma->dstTbUid, pTsmaStat->pTSma->dstTbName, &deleteReq, &pSubmitReq, &contLen);
code = smaBlockToSubmit(pSma->pVnode, (const SArray *)msg, pTsmaStat->pTSchema, pTsmaStat->pTSma->dstTbUid,
pTsmaStat->pTSma->dstTbName, &deleteReq, &pSubmitReq, &contLen);
TSDB_CHECK_CODE(code, lino, _exit);
if ((terrno = tsmaProcessDelReq(pSma, indexUid, &deleteReq)) != 0) {

View File

@ -20,6 +20,12 @@ typedef struct {
int8_t inited;
} STqMgmt;
typedef struct STaskUpdateEntry {
int64_t streamId;
int32_t taskId;
int32_t transId;
} STaskUpdateEntry;
static STqMgmt tqMgmt = {0};
// 0: not init
@ -817,28 +823,29 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) {
}
// sink
if (pTask->outputInfo.type == TASK_OUTPUT__SMA) {
pTask->outputInfo.smaSink.vnode = pTq->pVnode;
pTask->outputInfo.smaSink.smaSink = smaHandleRes;
} else if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) {
pTask->outputInfo.tbSink.vnode = pTq->pVnode;
pTask->outputInfo.tbSink.tbSinkFunc = tqSinkDataIntoDstTable;
STaskOutputInfo* pOutputInfo = &pTask->outputInfo;
if (pOutputInfo->type == TASK_OUTPUT__SMA) {
pOutputInfo->smaSink.vnode = pTq->pVnode;
pOutputInfo->smaSink.smaSink = smaHandleRes;
} else if (pOutputInfo->type == TASK_OUTPUT__TABLE) {
pOutputInfo->tbSink.vnode = pTq->pVnode;
pOutputInfo->tbSink.tbSinkFunc = tqSinkDataIntoDstTable;
int32_t ver1 = 1;
SMetaInfo info = {0};
code = metaGetInfo(pTq->pVnode->pMeta, pTask->outputInfo.tbSink.stbUid, &info, NULL);
code = metaGetInfo(pTq->pVnode->pMeta, pOutputInfo->tbSink.stbUid, &info, NULL);
if (code == TSDB_CODE_SUCCESS) {
ver1 = info.skmVer;
}
SSchemaWrapper* pschemaWrapper = pTask->outputInfo.tbSink.pSchemaWrapper;
pTask->outputInfo.tbSink.pTSchema = tBuildTSchema(pschemaWrapper->pSchema, pschemaWrapper->nCols, ver1);
if (pTask->outputInfo.tbSink.pTSchema == NULL) {
SSchemaWrapper* pschemaWrapper = pOutputInfo->tbSink.pSchemaWrapper;
pOutputInfo->tbSink.pTSchema = tBuildTSchema(pschemaWrapper->pSchema, pschemaWrapper->nCols, ver1);
if (pOutputInfo->tbSink.pTSchema == NULL) {
return -1;
}
pTask->outputInfo.tbSink.pTblInfo = tSimpleHashInit(10240, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT));
tSimpleHashSetFreeFp(pTask->outputInfo.tbSink.pTblInfo, freePtr);
pOutputInfo->tbSink.pTblInfo = tSimpleHashInit(10240, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT));
tSimpleHashSetFreeFp(pOutputInfo->tbSink.pTblInfo, freePtr);
}
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
@ -846,11 +853,11 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) {
pTask->exec.pWalReader = walOpenReader(pTq->pVnode->pWal, &cond, pTask->id.taskId);
}
// reset the task status from unfinished transaction
if (pTask->status.taskStatus == TASK_STATUS__PAUSE) {
tqWarn("s-task:%s reset task status to be normal, status kept in taskMeta: Paused", pTask->id.idStr);
pTask->status.taskStatus = TASK_STATUS__NORMAL;
}
// // reset the task status from unfinished transaction
// if (pTask->status.taskStatus == TASK_STATUS__PAUSE) {
// tqWarn("s-task:%s reset task status to be normal, status kept in taskMeta: Paused", pTask->id.idStr);
// pTask->status.taskStatus = TASK_STATUS__READY;
// }
streamTaskResetUpstreamStageInfo(pTask);
streamSetupScheduleTrigger(pTask);
@ -863,20 +870,23 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) {
pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer);
}
char* p = NULL;
streamTaskGetStatus(pTask, &p);
if (pTask->info.fillHistory) {
tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64
" nextProcessVer:%" PRId64
" child id:%d, level:%d, status:%s fill-history:%d, related stream task:0x%x trigger:%" PRId64 " ms",
vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer,
pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus),
pTask->info.fillHistory, (int32_t)pTask->streamTaskId.taskId, pTask->info.triggerParam);
pTask->info.selfChildId, pTask->info.taskLevel, p, pTask->info.fillHistory,
(int32_t)pTask->streamTaskId.taskId, pTask->info.triggerParam);
} else {
tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64
" nextProcessVer:%" PRId64
" child id:%d, level:%d, status:%s fill-history:%d, related fill-task:0x%x trigger:%" PRId64 " ms",
vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer,
pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus),
pTask->info.fillHistory, (int32_t)pTask->hTaskInfo.id.taskId, pTask->info.triggerParam);
pTask->info.selfChildId, pTask->info.taskLevel, p, pTask->info.fillHistory,
(int32_t)pTask->hTaskInfo.id.taskId, pTask->info.triggerParam);
}
return 0;
@ -918,9 +928,10 @@ int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) {
rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage);
streamMetaReleaseTask(pMeta, pTask);
const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus);
char* p = NULL;
streamTaskGetStatus(pTask, &p);
tqDebug("s-task:%s status:%s, stage:%d recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), check_status:%d",
pTask->id.idStr, pStatus, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status);
pTask->id.idStr, p, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status);
} else {
rsp.status = TASK_DOWNSTREAM_NOT_READY;
tqDebug("tq recv task check(taskId:0x%" PRIx64 "-0x%x not built yet) req(reqId:0x%" PRIx64
@ -954,6 +965,12 @@ int32_t tqProcessTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) {
tqDebug("tq task:0x%x (vgId:%d) recv check rsp(reqId:0x%" PRIx64 ") from 0x%x (vgId:%d) status %d",
rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.status);
if (!vnodeIsRoleLeader(pTq->pVnode)) {
tqError("vgId:%d not leader, task:0x%x not handle the check rsp, downstream:0x%x (vgId:%d)", vgId,
rsp.upstreamTaskId, rsp.downstreamTaskId, rsp.downstreamNodeId);
return code;
}
SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, rsp.streamId, rsp.upstreamTaskId);
if (pTask == NULL) {
tqError("tq failed to locate the stream task:0x%" PRIx64 "-0x%x (vgId:%d), it may have been destroyed or stopped",
@ -1023,11 +1040,9 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms
SStreamTask* p = streamMetaAcquireTask(pStreamMeta, streamId, taskId);
bool restored = pTq->pVnode->restored;
if (p != NULL && restored) {
p->execInfo.init = taosGetTimestampMs();
tqDebug("s-task:%s set the init ts:%"PRId64, p->id.idStr, p->execInfo.init);
streamTaskCheckDownstream(p);
if (p != NULL && restored && p->info.fillHistory == 0) {
EStreamTaskEvent event = (HAS_RELATED_FILLHISTORY_TASK(p)) ? TASK_EVENT_INIT_STREAM_SCANHIST : TASK_EVENT_INIT;
streamTaskHandleEvent(p->status.pSM, event);
} else if (!restored) {
tqWarn("s-task:%s not launched since vnode(vgId:%d) not ready", p->id.idStr, vgId);
}
@ -1061,7 +1076,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
// do recovery step1
const char* id = pTask->id.idStr;
const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus);
char* pStatus = NULL;
streamTaskGetStatus(pTask, &pStatus);
// avoid multi-thread exec
while(1) {
@ -1114,8 +1130,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
streamScanHistoryData(pTask);
double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0;
if (pTask->status.taskStatus == TASK_STATUS__PAUSE) {
int8_t status = streamTaskSetSchedStatusInActive(pTask);
if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__PAUSE) {
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);
atomic_store_32(&pTask->status.inScanHistorySentinel, 0);
@ -1124,12 +1140,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
}
// the following procedure should be executed, no matter status is stop/pause or not
tqDebug("s-task:%s scan-history stage(step 1) ended, elapsed time:%.2fs", id, el);
tqDebug("s-task:%s scan-history(step 1) ended, elapsed time:%.2fs", id, el);
if (pTask->info.fillHistory) {
SVersionRange* pRange = NULL;
SStreamTask* pStreamTask = NULL;
bool done = false;
// 1. get the related stream task
pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId);
@ -1148,10 +1163,10 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
}
ASSERT(pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE);
#if 0
// 2. it cannot be paused, when the stream task in TASK_STATUS__SCAN_HISTORY status. Let's wait for the
// stream task get ready for scan history data
while (pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) {
while (streamTaskGetStatus(pStreamTask, NULL) == TASK_STATUS__SCAN_HISTORY) {
tqDebug(
"s-task:%s level:%d related stream task:%s(status:%s) not ready for halt, wait for it and recheck in 100ms",
id, pTask->info.taskLevel, pStreamTask->id.idStr, streamGetTaskStatusStr(pStreamTask->status.taskStatus));
@ -1209,21 +1224,26 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
taosThreadMutexUnlock(&pStreamTask->lock);
break;
}
#endif
streamTaskHandleEvent(pStreamTask->status.pSM, TASK_EVENT_HALT);
int64_t nextProcessedVer = pStreamTask->hTaskInfo.haltVer;
// if it's an source task, extract the last version in wal.
pRange = &pTask->dataRange.range;
done = streamHistoryTaskSetVerRangeStep2(pTask, nextProcessedVer);
bool done = streamHistoryTaskSetVerRangeStep2(pTask, nextProcessedVer);
pTask->execInfo.step2Start = taosGetTimestampMs();
if (done) {
qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0);
streamTaskPutTranstateIntoInputQ(pTask);
// streamTaskRestoreStatus(pTask);
if (pTask->status.taskStatus == TASK_STATUS__PAUSE) {
pTask->status.keepTaskStatus = TASK_STATUS__NORMAL;
qDebug("s-task:%s prev status is %s, update the kept status to be:%s when after step 2", id,
streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus));
}
// if (pTask->status.taskStatus == TASK_STATUS__PAUSE) {
// pTask->status.keepTaskStatus = TASK_STATUS__READY;
// qDebug("s-task:%s prev status is %s, update the kept status to be:%s when after step 2", id,
// streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus));
// }
streamExecTask(pTask); // exec directly
} else {
@ -1243,35 +1263,30 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
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);
/*int8_t status = */streamTaskSetSchedStatusInActive(pTask);
/*int8_t status = */streamTaskSetSchedStatusInactive(pTask);
#if 0
// 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)) {
streamSetStatusNormal(pTask);
}
#endif
// now the fill-history task starts to scan data from wal files.
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE);
tqScanWalAsync(pTq, false);
}
streamMetaReleaseTask(pMeta, pStreamTask);
} else {
STimeWindow* pWindow = &pTask->dataRange.window;
ASSERT(HAS_RELATED_FILLHISTORY_TASK(pTask));
if (pTask->hTaskInfo.id.taskId == 0) {
*pWindow = (STimeWindow){INT64_MIN, INT64_MAX};
tqDebug(
"s-task:%s scan-history in stream time window completed, no related fill-history task, reset the time "
"window:%" PRId64 " - %" PRId64,
id, pWindow->skey, pWindow->ekey);
qStreamInfoResetTimewindowFilter(pTask->exec.pExecutor);
} else {
// when related fill-history task exists, update the fill-history time window only when the
// state transfer is completed.
tqDebug(
"s-task:%s scan-history in stream time window completed, now start to handle data from WAL, start "
"ver:%" PRId64 ", window:%" PRId64 " - %" PRId64,
id, pTask->chkInfo.nextProcessVer, pWindow->skey, pWindow->ekey);
}
// Not update the fill-history time window until the state transfer is completed if the related fill-history task
// exists.
tqDebug(
"s-task:%s scan-history in stream time window completed, now start to handle data from WAL, startVer:%" PRId64
", window:%" PRId64 " - %" PRId64,
id, pTask->chkInfo.nextProcessVer, pWindow->skey, pWindow->ekey);
code = streamTaskScanHistoryDataComplete(pTask);
}
@ -1350,7 +1365,7 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) {
int32_t vgId = TD_VID(pTq->pVnode);
if (taskId == STREAM_EXEC_TASK_STATUS_CHECK_ID) {
tqCheckAndRunStreamTask(pTq);
tqStartStreamTask(pTq);
return 0;
}
@ -1360,17 +1375,16 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) {
}
SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->streamId, taskId);
if (pTask != NULL) {
// even in halt status, the data in inputQ must be processed
int8_t st = pTask->status.taskStatus;
if (st == TASK_STATUS__NORMAL || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) {
if (pTask != NULL) { // even in halt status, the data in inputQ must be processed
char* p = NULL;
if (streamTaskReadyToRun(pTask, &p)) {
tqDebug("vgId:%d s-task:%s start to process block from inputQ, next checked ver:%" PRId64, vgId, pTask->id.idStr,
pTask->chkInfo.nextProcessVer);
streamExecTask(pTask);
} 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,
pTask->id.idStr, streamGetTaskStatusStr(st), status);
pTask->id.idStr, p, status);
}
streamMetaReleaseTask(pTq->pStreamMeta, pTask);
@ -1444,7 +1458,7 @@ int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen) {
SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId);
if (pTask != NULL) {
// drop the related fill-history task firstly
if (pTask->hTaskInfo.id.taskId != 0) {
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
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);
@ -1484,7 +1498,7 @@ int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg
streamTaskPause(pTask, pMeta);
SStreamTask* pHistoryTask = NULL;
if (pTask->hTaskInfo.id.taskId != 0) {
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
pHistoryTask = streamMetaAcquireTask(pMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId);
if (pHistoryTask == NULL) {
tqError("vgId:%d process pause req, failed to acquire fill-history task:0x%" PRIx64
@ -1512,8 +1526,7 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion,
return -1;
}
// todo: handle the case: resume from halt to pause/ from halt to normal/ from pause to normal
streamTaskResume(pTask, pTq->pStreamMeta);
streamTaskResume(pTask);
int32_t level = pTask->info.taskLevel;
if (level == TASK_LEVEL__SINK) {
@ -1521,8 +1534,8 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion,
return 0;
}
int8_t status = pTask->status.taskStatus;
if (status == TASK_STATUS__NORMAL || status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__CK) {
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
if (status == TASK_STATUS__READY || status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__CK) {
// no lock needs to secure the access of the version
if (igUntreated && level == TASK_LEVEL__SOURCE && !pTask->info.fillHistory) {
// discard all the data when the stream task is suspended.
@ -1535,8 +1548,7 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion,
vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer, sversion, pTask->status.schedStatus);
}
if (level == TASK_LEVEL__SOURCE && pTask->info.fillHistory &&
pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) {
if (level == TASK_LEVEL__SOURCE && pTask->info.fillHistory && status == TASK_STATUS__SCAN_HISTORY) {
streamStartScanHistoryAsync(pTask, igUntreated);
} else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0)) {
tqScanWalAsync(pTq, false);
@ -1681,7 +1693,6 @@ FAIL:
return -1;
}
// todo error code cannot be return, since this is invoked by an mnode-launched transaction.
int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) {
int32_t vgId = TD_VID(pTq->pVnode);
SStreamMeta* pMeta = pTq->pStreamMeta;
@ -1692,7 +1703,6 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp)
// disable auto rsp to mnode
pRsp->info.handle = NULL;
// todo: add counter to make sure other tasks would not be trapped in checkpoint state
SStreamCheckpointSourceReq req = {0};
if (!vnodeIsRoleLeader(pTq->pVnode)) {
tqDebug("vgId:%d not leader, ignore checkpoint-source msg", vgId);
@ -1723,6 +1733,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp)
}
tDecoderClear(&decoder);
// todo handle failure to reset from checkpoint procedure
SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.taskId);
if (pTask == NULL) {
tqError("vgId:%d failed to find s-task:0x%x, ignore checkpoint msg. it may have been destroyed already", vgId,
@ -1733,6 +1744,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp)
return TSDB_CODE_SUCCESS;
}
// todo handle failure to reset from checkpoint procedure
// downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req.
if (pTask->status.downstreamReady != 1) {
pTask->chkInfo.failedId = req.checkpointId; // record the latest failed checkpoint id
@ -1748,8 +1760,11 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp)
return TSDB_CODE_SUCCESS;
}
// todo save the checkpoint failed info
taosThreadMutexLock(&pTask->lock);
if (pTask->status.taskStatus == TASK_STATUS__HALT) {
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
if (status == TASK_STATUS__HALT || status == TASK_STATUS__PAUSE) {
qError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 ", set it failure",
pTask->id.idStr, req.checkpointId);
taosThreadMutexUnlock(&pTask->lock);
@ -1830,7 +1845,6 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) {
char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead));
int32_t len = pMsg->contLen - sizeof(SMsgHead);
SRpcMsg rsp = {.info = pMsg->info, .code = TSDB_CODE_SUCCESS};
bool allStopped = false;
SStreamTaskNodeUpdateMsg req = {0};
@ -1861,17 +1875,37 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) {
}
SStreamTask* pTask = *ppTask;
tqDebug("s-task:%s receive nodeEp update msg from mnode", pTask->id.idStr);
if (pMeta->updateInfo.transId != req.transId) {
pMeta->updateInfo.transId = req.transId;
tqDebug("s-task:%s receive new trans to update nodeEp msg from mnode, transId:%d", pTask->id.idStr, req.transId);
// info needs to be kept till the new trans to update the nodeEp arrived.
taosHashClear(pMeta->updateInfo.pTasks);
} else {
tqDebug("s-task:%s recv trans to update nodeEp from mnode, transId:%d", pTask->id.idStr, req.transId);
}
STaskUpdateEntry entry = {.streamId = req.streamId, .taskId = req.taskId, .transId = req.transId};
void* exist = taosHashGet(pMeta->updateInfo.pTasks, &entry, sizeof(STaskUpdateEntry));
if (exist != NULL) {
tqDebug("s-task:%s (vgId:%d) already update in trans:%d, discard the nodeEp update msg", pTask->id.idStr, vgId,
req.transId);
rsp.code = TSDB_CODE_SUCCESS;
taosWUnLockLatch(&pMeta->lock);
taosArrayDestroy(req.pNodeList);
return rsp.code;
}
streamTaskUpdateEpsetInfo(pTask, req.pNodeList);
streamSetStatusNormal(pTask);
streamTaskResetStatus(pTask);
SStreamTask** ppHTask = NULL;
if (pTask->hTaskInfo.id.taskId != 0) {
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
ppHTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id));
if (ppHTask == NULL || *ppHTask == NULL) {
tqError("vgId:%d failed to acquire fill-history task:0x%x when handling update, it may have been dropped already",
pMeta->vgId, req.taskId);
CLEAR_RELATED_FILLHISTORY_TASK(pTask);
} else {
tqDebug("s-task:%s fill-history task update nodeEp along with stream task", (*ppHTask)->id.idStr);
streamTaskUpdateEpsetInfo(*ppHTask, req.pNodeList);
@ -1890,12 +1924,14 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) {
}
streamTaskStop(pTask);
taosHashPut(pMeta->pUpdateTaskSet, &pTask->id, sizeof(pTask->id), NULL, 0);
// keep the already handled info
taosHashPut(pMeta->updateInfo.pTasks, &entry, sizeof(entry), NULL, 0);
if (ppHTask != NULL) {
streamTaskStop(*ppHTask);
tqDebug("s-task:%s task nodeEp update completed, streamTask and related fill-history task closed", pTask->id.idStr);
taosHashPut(pMeta->pUpdateTaskSet, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0);
taosHashPut(pMeta->updateInfo.pTasks, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0);
} else {
tqDebug("s-task:%s task nodeEp update completed, streamTask closed", pTask->id.idStr);
}
@ -1904,7 +1940,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) {
// possibly only handle the stream task.
int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta);
int32_t updateTasks = taosHashGetSize(pMeta->pUpdateTaskSet);
int32_t updateTasks = taosHashGetSize(pMeta->updateInfo.pTasks);
pMeta->startInfo.startedAfterNodeUpdate = 1;
@ -1913,16 +1949,21 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) {
updateTasks, (numOfTasks - updateTasks));
taosWUnLockLatch(&pMeta->lock);
} else {
taosHashClear(pMeta->pUpdateTaskSet);
if (!pTq->pVnode->restored) {
tqDebug("vgId:%d vnode restore not completed, not restart the tasks, clear the start after nodeUpdate flag", vgId);
pMeta->startInfo.startedAfterNodeUpdate = 0;
taosWUnLockLatch(&pMeta->lock);
} else {
tqDebug("vgId:%d tasks are all updated and stopped, restart them", vgId);
terrno = 0;
taosWUnLockLatch(&pMeta->lock);
while (streamMetaTaskInTimer(pMeta)) {
qDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pMeta->vgId);
taosMsleep(100);
}
taosWLockLatch(&pMeta->lock);
int32_t code = streamMetaReopen(pMeta);
if (code != 0) {
tqError("vgId:%d failed to reopen stream meta", vgId);
@ -1968,10 +2009,10 @@ int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg) {
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) {
if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__CK) {
streamTaskClearCheckInfo(pTask);
taosArrayClear(pTask->pReadyMsgList);
streamSetStatusNormal(pTask);
streamTaskSetStatusReady(pTask);
}
streamMetaReleaseTask(pMeta, pTask);

View File

@ -13,6 +13,7 @@
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include <common/tmsg.h>
#include "tcommon.h"
#include "tmsg.h"
#include "tq.h"
@ -28,19 +29,19 @@ static bool hasOnlySubmitData(const SArray* pBlocks, int32_t numOfBlocks);
static int32_t tsAscendingSortFn(const void* p1, const void* p2);
static int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDataBlock, char* stbFullName,
SSubmitTbData* pTableData);
static int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlock* pDataBlock,
SSubmitTbData* pTableData);
static int32_t doBuildAndSendDeleteMsg(SVnode* pVnode, char* stbFullName, SSDataBlock* pDataBlock, SStreamTask* pTask,
int64_t suid);
static int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* pReq, int32_t numOfBlocks);
static int32_t buildSubmitMsgImpl(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen);
static int32_t doConvertRows(SSubmitTbData* pTableData, STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id);
static int32_t doConvertRows(SSubmitTbData* pTableData, const STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id);
static int32_t doWaitForDstTableCreated(SVnode* pVnode, SStreamTask* pTask, STableSinkInfo* pTableSinkInfo,
const char* dstTableName, int64_t* uid);
static int32_t doPutIntoCache(SSHashObj* pSinkTableMap, STableSinkInfo* pTableSinkInfo, uint64_t groupId, const char* id);
static int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, const char* id);
static bool isValidDstChildTable(SMetaReader* pReader, int32_t vgId, const char* ctbName, int64_t suid);
static SVCreateTbReq* buildAutoCreateTableReq(char* stbFullName, int64_t suid, int32_t numOfCols, SSDataBlock* pDataBlock);
static int32_t initCreateTableMsg(SVCreateTbReq* pCreateTableReq, uint64_t suid, const char* stbFullName, int32_t numOfTags);
static SArray* createDefaultTagColName();
static void setCreateTableMsgTableName(SVCreateTbReq* pCreateTableReq, SSDataBlock* pDataBlock, const char* stbFullName,
int64_t gid);
int32_t tqBuildDeleteReq(const char* stbFullName, const SSDataBlock* pDataBlock, SBatchDeleteReq* deleteReq,
const char* pIdStr) {
@ -138,61 +139,68 @@ static int32_t tqPutReqToQueue(SVnode* pVnode, SVCreateTbBatchReq* pReqs) {
return TSDB_CODE_SUCCESS;
}
int32_t initCreateTableMsg(SVCreateTbReq* pCreateTableReq, uint64_t suid, const char* stbFullName, int32_t numOfTags) {
pCreateTableReq->flags = 0;
pCreateTableReq->type = TSDB_CHILD_TABLE;
pCreateTableReq->ctb.suid = suid;
// set super table name
SName name = {0};
tNameFromString(&name, stbFullName, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE);
pCreateTableReq->ctb.stbName = taosStrdup((char*)tNameGetTableName(&name));
pCreateTableReq->ctb.tagNum = numOfTags;
return TSDB_CODE_SUCCESS;
}
SArray* createDefaultTagColName() {
SArray* pTagColNameList = taosArrayInit(1, TSDB_COL_NAME_LEN);
char tagNameStr[TSDB_COL_NAME_LEN] = "group_id";
taosArrayPush(pTagColNameList, tagNameStr);
return pTagColNameList;
}
void setCreateTableMsgTableName(SVCreateTbReq* pCreateTableReq, SSDataBlock* pDataBlock, const char* stbFullName,
int64_t gid) {
if (pDataBlock->info.parTbName[0]) {
pCreateTableReq->name = taosStrdup(pDataBlock->info.parTbName);
} else {
pCreateTableReq->name = buildCtbNameByGroupId(stbFullName, gid);
}
}
static int32_t doBuildAndSendCreateTableMsg(SVnode* pVnode, char* stbFullName, SSDataBlock* pDataBlock, SStreamTask* pTask,
int64_t suid) {
tqDebug("s-task:%s build create table msg", pTask->id.idStr);
STSchema* pTSchema = pTask->outputInfo.tbSink.pTSchema;
int32_t rows = pDataBlock->info.rows;
SArray* tagArray = NULL;
SArray* tagArray = taosArrayInit(4, sizeof(STagVal));;
int32_t code = 0;
SVCreateTbBatchReq reqs = {0};
SArray* crTblArray = reqs.pArray = taosArrayInit(1, sizeof(SVCreateTbReq));
if (NULL == reqs.pArray) {
tqError("s-task:%s failed to init create table msg, code:%s", pTask->id.idStr, tstrerror(terrno));
goto _end;
}
for (int32_t rowId = 0; rowId < rows; rowId++) {
SVCreateTbReq* pCreateTbReq = &((SVCreateTbReq){0});
// set const
pCreateTbReq->flags = 0;
pCreateTbReq->type = TSDB_CHILD_TABLE;
pCreateTbReq->ctb.suid = suid;
// set super table name
SName name = {0};
tNameFromString(&name, stbFullName, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE);
pCreateTbReq->ctb.stbName = taosStrdup((char*)tNameGetTableName(&name)); // taosStrdup(stbFullName);
// set tag content
int32_t size = taosArrayGetSize(pDataBlock->pDataBlock);
if (size == 2) {
tagArray = taosArrayInit(1, sizeof(STagVal));
if (!tagArray) {
tdDestroySVCreateTbReq(pCreateTbReq);
goto _end;
}
int32_t numOfTags = TMAX(size - UD_TAG_COLUMN_INDEX, 1);
initCreateTableMsg(pCreateTbReq, suid, stbFullName, numOfTags);
taosArrayClear(tagArray);
if (size == 2) {
STagVal tagVal = {
.cid = pTSchema->numOfCols + 1, .type = TSDB_DATA_TYPE_UBIGINT, .i64 = pDataBlock->info.id.groupId};
taosArrayPush(tagArray, &tagVal);
// set tag name
SArray* tagName = taosArrayInit(1, TSDB_COL_NAME_LEN);
char tagNameStr[TSDB_COL_NAME_LEN] = "group_id";
taosArrayPush(tagName, tagNameStr);
pCreateTbReq->ctb.tagName = tagName;
pCreateTbReq->ctb.tagName = createDefaultTagColName();
} else {
tagArray = taosArrayInit(size - 1, sizeof(STagVal));
if (!tagArray) {
tdDestroySVCreateTbReq(pCreateTbReq);
goto _end;
}
for (int32_t tagId = UD_TAG_COLUMN_INDEX, step = 1; tagId < size; tagId++, step++) {
SColumnInfoData* pTagData = taosArrayGet(pDataBlock->pDataBlock, tagId);
@ -209,29 +217,26 @@ static int32_t doBuildAndSendCreateTableMsg(SVnode* pVnode, char* stbFullName, S
taosArrayPush(tagArray, &tagVal);
}
}
pCreateTbReq->ctb.tagNum = TMAX(size - UD_TAG_COLUMN_INDEX, 1);
STag* pTag = NULL;
tTagNew(tagArray, 1, false, &pTag);
tTagNew(tagArray, 1, false, (STag**)&pCreateTbReq->ctb.pTag);
tagArray = taosArrayDestroy(tagArray);
if (pTag == NULL) {
if (pCreateTbReq->ctb.pTag == NULL) {
tdDestroySVCreateTbReq(pCreateTbReq);
code = TSDB_CODE_OUT_OF_MEMORY;
goto _end;
}
pCreateTbReq->ctb.pTag = (uint8_t*)pTag;
// set table name
if (!pDataBlock->info.parTbName[0]) {
uint64_t gid = pDataBlock->info.id.groupId;
if (taosArrayGetSize(pDataBlock->pDataBlock) > UD_GROUPID_COLUMN_INDEX) {
SColumnInfoData* pGpIdColInfo = taosArrayGet(pDataBlock->pDataBlock, UD_GROUPID_COLUMN_INDEX);
// todo remove this
void* pGpIdData = colDataGetData(pGpIdColInfo, rowId);
pCreateTbReq->name = buildCtbNameByGroupId(stbFullName, *(uint64_t*)pGpIdData);
} else {
pCreateTbReq->name = taosStrdup(pDataBlock->info.parTbName);
ASSERT(gid == *(int64_t*)pGpIdData);
}
setCreateTableMsgTableName(pCreateTbReq, pDataBlock, stbFullName, gid);
taosArrayPush(reqs.pArray, pCreateTbReq);
tqDebug("s-task:%s build create table:%s msg complete", pTask->id.idStr, pCreateTbReq->name);
}
@ -390,60 +395,33 @@ bool isValidDstChildTable(SMetaReader* pReader, int32_t vgId, const char* ctbNam
return true;
}
SVCreateTbReq* buildAutoCreateTableReq(char* stbFullName, int64_t suid, int32_t numOfCols, SSDataBlock* pDataBlock) {
char* ctbName = pDataBlock->info.parTbName;
SVCreateTbReq* pCreateTbReq = taosMemoryCalloc(1, sizeof(SVCreateStbReq));
SVCreateTbReq* buildAutoCreateTableReq(const char* stbFullName, int64_t suid, int32_t numOfCols,
SSDataBlock* pDataBlock, SArray* pTagArray) {
SVCreateTbReq* pCreateTbReq = taosMemoryCalloc(1, sizeof(SVCreateTbReq));
if (pCreateTbReq == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}
// set tag content
SArray* tagArray = taosArrayInit(1, sizeof(STagVal));
if (tagArray == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
tdDestroySVCreateTbReq(pCreateTbReq);
taosMemoryFreeClear(pCreateTbReq);
return NULL;
}
// set const
pCreateTbReq->flags = 0;
pCreateTbReq->type = TSDB_CHILD_TABLE;
pCreateTbReq->ctb.suid = suid;
// set super table name
SName name = {0};
tNameFromString(&name, stbFullName, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE);
pCreateTbReq->ctb.stbName = taosStrdup((char*)tNameGetTableName(&name));
taosArrayClear(pTagArray);
initCreateTableMsg(pCreateTbReq, suid, stbFullName, 1);
STagVal tagVal = { .cid = numOfCols, .type = TSDB_DATA_TYPE_UBIGINT, .i64 = pDataBlock->info.id.groupId};
taosArrayPush(tagArray, &tagVal);
pCreateTbReq->ctb.tagNum = taosArrayGetSize(tagArray);
taosArrayPush(pTagArray, &tagVal);
STag* pTag = NULL;
tTagNew(tagArray, 1, false, &pTag);
taosArrayDestroy(tagArray);
tTagNew(pTagArray, 1, false, (STag**) &pCreateTbReq->ctb.pTag);
if (pTag == NULL) {
if (pCreateTbReq->ctb.pTag == NULL) {
tdDestroySVCreateTbReq(pCreateTbReq);
taosMemoryFreeClear(pCreateTbReq);
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}
pCreateTbReq->ctb.pTag = (uint8_t*)pTag;
// set tag name
SArray* tagName = taosArrayInit(1, TSDB_COL_NAME_LEN);
char k[TSDB_COL_NAME_LEN] = "group_id";
taosArrayPush(tagName, k);
pCreateTbReq->ctb.tagName = tagName;
pCreateTbReq->ctb.tagName = createDefaultTagColName();
// set table name
pCreateTbReq->name = taosStrdup(ctbName);
setCreateTableMsgTableName(pCreateTbReq, pDataBlock, stbFullName, pDataBlock->info.id.groupId);
return pCreateTbReq;
}
@ -514,7 +492,7 @@ int32_t tsAscendingSortFn(const void* p1, const void* p2) {
}
}
int32_t doConvertRows(SSubmitTbData* pTableData, STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id) {
int32_t doConvertRows(SSubmitTbData* pTableData, const STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id) {
int32_t numOfRows = pDataBlock->info.rows;
int32_t code = TSDB_CODE_SUCCESS;
@ -592,7 +570,7 @@ int32_t doWaitForDstTableCreated(SVnode* pVnode, SStreamTask* pTask, STableSinkI
const char* id = pTask->id.idStr;
while (pTableSinkInfo->uid == 0) {
if (streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldStop(pTask)) {
tqDebug("s-task:%s task will stop, quit from waiting for table:%s create", id, dstTableName);
return TSDB_CODE_STREAM_EXEC_CANCELLED;
}
@ -693,8 +671,13 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat
tqDebug("s-task:%s stream write into table:%s, table auto created", id, dstTableName);
SArray* pTagArray = taosArrayInit(pTSchema->numOfCols + 1, sizeof(STagVal));
pTableData->flags = SUBMIT_REQ_AUTO_CREATE_TABLE;
pTableData->pCreateTbReq = buildAutoCreateTableReq(stbFullName, suid, pTSchema->numOfCols + 1, pDataBlock);
pTableData->pCreateTbReq =
buildAutoCreateTableReq(stbFullName, suid, pTSchema->numOfCols + 1, pDataBlock, pTagArray);
taosArrayDestroy(pTagArray);
if (pTableData->pCreateTbReq == NULL) {
tqError("s-task:%s failed to build auto create table req, code:%s", id, tstrerror(terrno));
taosMemoryFree(pTableSinkInfo);
@ -724,17 +707,16 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat
return TSDB_CODE_SUCCESS;
}
int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlock* pDataBlock,
SSubmitTbData* pTableData) {
int32_t numOfRows = pDataBlock->info.rows;
const char* id = pTask->id.idStr;
int32_t setDstTableDataPayload(uint64_t suid, const STSchema *pTSchema, int32_t blockIndex, SSDataBlock* pDataBlock,
SSubmitTbData* pTableData, const char* id) {
int32_t numOfRows = pDataBlock->info.rows;
tqDebug("s-task:%s sink data pipeline, build submit msg from %dth resBlock, including %d rows, dst suid:%" PRId64,
id, blockIndex + 1, numOfRows, pTask->outputInfo.tbSink.stbUid);
id, blockIndex + 1, numOfRows, suid);
char* dstTableName = pDataBlock->info.parTbName;
// convert all rows
int32_t code = doConvertRows(pTableData, pTask->outputInfo.tbSink.pTSchema, pDataBlock, id);
int32_t code = doConvertRows(pTableData, pTSchema, pDataBlock, id);
if (code != TSDB_CODE_SUCCESS) {
tqError("s-task:%s failed to convert rows from result block, code:%s", id, tstrerror(terrno));
return code;
@ -773,7 +755,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) {
numOfBlocks);
for(int32_t i = 0; i < numOfBlocks; ++i) {
if (streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldStop(pTask)) {
return;
}
@ -800,7 +782,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) {
continue;
}
code = setDstTableDataPayload(pTask, i, pDataBlock, &tbData);
code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, id);
if (code != TSDB_CODE_SUCCESS) {
continue;
}
@ -823,7 +805,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) {
bool hasSubmit = false;
for (int32_t i = 0; i < numOfBlocks; i++) {
if (streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldStop(pTask)) {
taosHashCleanup(pTableIndexMap);
tDestroySubmitReq(&submitReq, TSDB_MSG_FLG_ENCODE);
return;
@ -847,7 +829,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) {
continue;
}
code = setDstTableDataPayload(pTask, i, pDataBlock, &tbData);
code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, id);
if (code != TSDB_CODE_SUCCESS) {
continue;
}
@ -857,7 +839,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) {
int32_t size = (int32_t)taosArrayGetSize(submitReq.aSubmitTbData) - 1;
taosHashPut(pTableIndexMap, &groupId, sizeof(groupId), &size, sizeof(size));
} else {
code = setDstTableDataPayload(pTask, i, pDataBlock, &tbData);
code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, id);
if (code != TSDB_CODE_SUCCESS) {
continue;
}

View File

@ -60,7 +60,7 @@ int32_t tqScanWal(STQ* pTq) {
return 0;
}
int32_t tqCheckAndRunStreamTask(STQ* pTq) {
int32_t tqStartStreamTask(STQ* pTq) {
int32_t vgId = TD_VID(pTq->pVnode);
SStreamMeta* pMeta = pTq->pStreamMeta;
@ -92,19 +92,18 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) {
}
if (pTask->status.downstreamReady == 1) {
tqDebug("s-task:%s downstream ready, no need to check downstream, check only related fill-history task",
pTask->id.idStr);
streamLaunchFillHistoryTask(pTask);
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
tqDebug("s-task:%s downstream ready, no need to check downstream, check only related fill-history task",
pTask->id.idStr);
streamLaunchFillHistoryTask(pTask);
}
streamMetaReleaseTask(pMeta, pTask);
continue;
}
pTask->execInfo.init = taosGetTimestampMs();
tqDebug("s-task:%s start check downstream tasks, set the init ts:%"PRId64, pTask->id.idStr, pTask->execInfo.init);
streamSetStatusNormal(pTask);
streamTaskCheckDownstream(pTask);
EStreamTaskEvent event = (HAS_RELATED_FILLHISTORY_TASK(pTask)) ? TASK_EVENT_INIT_STREAM_SCANHIST : TASK_EVENT_INIT;
streamTaskHandleEvent(pTask->status.pSM, event);
streamMetaReleaseTask(pMeta, pTask);
}
@ -113,8 +112,8 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) {
}
int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq) {
int32_t vgId = TD_VID(pTq->pVnode);
SStreamMeta* pMeta = pTq->pStreamMeta;
int32_t vgId = pMeta->vgId;
int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList);
if (numOfTasks == 0) {
@ -244,9 +243,8 @@ int32_t tqStartStreamTasks(STQ* pTq) {
STaskId id = {.streamId = pTaskId->streamId, .taskId = pTaskId->taskId};
SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
int8_t status = (*pTask)->status.taskStatus;
if (status == TASK_STATUS__STOP && (*pTask)->info.fillHistory != 1) {
streamSetStatusNormal(*pTask);
if ((*pTask)->info.fillHistory != 1) {
streamTaskResetStatus(*pTask);
}
}
@ -328,15 +326,17 @@ static bool taskReadyForDataFromWal(SStreamTask* pTask) {
}
// 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));
// int32_t status = pTask->status.taskStatus;
char* p = NULL;
int32_t status = streamTaskGetStatus(pTask, &p);
if (streamTaskGetStatus(pTask, &p) != TASK_STATUS__READY) {
tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, p);
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);
ASSERT(status == TASK_STATUS__READY);
// 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);
@ -449,9 +449,10 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) {
taosThreadMutexLock(&pTask->lock);
const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus);
if (pTask->status.taskStatus != TASK_STATUS__NORMAL) {
tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pStatus);
char* p = NULL;
ETaskStatus status = streamTaskGetStatus(pTask, &p);
if (status != TASK_STATUS__READY) {
tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, p);
taosThreadMutexUnlock(&pTask->lock);
streamMetaReleaseTask(pStreamMeta, pTask);
continue;

View File

@ -584,7 +584,7 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg
}
} break;
case TDMT_VND_STREAM_TASK_RESET: {
if (pVnode->restored/* && vnodeIsLeader(pVnode)*/) {
if (pVnode->restored && vnodeIsLeader(pVnode)) {
tqProcessTaskResetReq(pVnode->pTq, pMsg);
}
} break;

View File

@ -52,6 +52,11 @@ extern "C" {
#define stTrace(...) do { if (stDebugFlag & DEBUG_TRACE) { taosPrintLog("STM ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0)
// clang-format on
typedef struct {
int8_t type;
SSDataBlock* pBlock;
} SStreamTrigger;
typedef struct SStreamGlobalEnv {
int8_t inited;
void* timer;

View File

@ -0,0 +1,73 @@
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef TDENGINE_STREAMSM_H
#define TDENGINE_STREAMSM_H
#include "tstream.h"
#ifdef __cplusplus
extern "C" {
#endif
// moore finite state machine for stream task
typedef struct SStreamTaskState {
ETaskStatus state;
char* name;
} SStreamTaskState;
typedef int32_t (*__state_trans_fn)(SStreamTask*);
typedef int32_t (*__state_trans_succ_fn)(SStreamTask*);
typedef struct SAttachedEventInfo {
ETaskStatus status; // required status that this event can be handled
EStreamTaskEvent event; // the delayed handled event
} SAttachedEventInfo;
typedef struct STaskStateTrans {
bool autoInvokeEndFn;
SStreamTaskState state;
EStreamTaskEvent event;
SStreamTaskState next;
__state_trans_fn pAction;
__state_trans_succ_fn pSuccAction;
SAttachedEventInfo attachEvent;
} STaskStateTrans;
struct SStreamTaskSM {
SStreamTask* pTask;
STaskStateTrans* pActiveTrans;
int64_t startTs;
SStreamTaskState current;
struct {
SStreamTaskState state;
EStreamTaskEvent evt;
} prev;
// register the next handled event, if current state is not allowed to handle this event
SArray* pWaitingEventList;
};
typedef struct SStreamEventInfo {
EStreamTaskEvent event;
const char* name;
} SStreamEventInfo;
SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask);
void* streamDestroyStateMachine(SStreamTaskSM* pSM);
#ifdef __cplusplus
}
#endif
#endif // TDENGINE_STREAMSM_H

View File

@ -62,14 +62,14 @@ static void streamSchedByTimer(void* param, void* tmrId) {
int32_t nextTrigger = (int32_t)pTask->info.triggerParam;
int8_t status = atomic_load_8(&pTask->schedInfo.status);
stDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", id, status, nextTrigger);
stTrace("s-task:%s in scheduler, trigger status:%d, next:%dms", id, status, nextTrigger);
if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) {
if (streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) {
stDebug("s-task:%s jump out of schedTimer", id);
return;
}
if (pTask->status.taskStatus == TASK_STATUS__CK) {
if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__CK) {
stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger);
} else {
if (status == TASK_TRIGGER_STATUS__ACTIVE) {
@ -127,7 +127,7 @@ int32_t streamSchedExec(SStreamTask* pTask) {
SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq));
if (pRunReq == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
/*int8_t status = */streamTaskSetSchedStatusInActive(pTask);
/*int8_t status = */streamTaskSetSchedStatusInactive(pTask);
stError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr);
return -1;
}
@ -267,8 +267,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S
}
// disable the data from upstream tasks
int8_t st = pTask->status.taskStatus;
if (st == TASK_STATUS__HALT) {
if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__HALT) {
status = TASK_INPUT_STATUS__BLOCKED;
}

View File

@ -2007,14 +2007,12 @@ int32_t streamStateAddIfNotExist_rocksdb(SStreamState* pState, const SWinKey* ke
return 0;
}
int32_t streamStateCurPrev_rocksdb(SStreamStateCur* pCur) {
stDebug("streamStateCurPrev_rocksdb");
if (!pCur) return -1;
rocksdb_iter_prev(pCur->iter);
return 0;
}
int32_t streamStateGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) {
stDebug("streamStateGetKVByCur_rocksdb");
if (!pCur) return -1;
SStateKey tkey;
SStateKey* pKtmp = &tkey;
@ -2039,7 +2037,6 @@ int32_t streamStateGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, cons
return -1;
}
SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey* key) {
stDebug("streamStateGetAndCheckCur_rocksdb");
SStreamStateCur* pCur = streamStateFillGetCur_rocksdb(pState, key);
if (pCur) {
int32_t code = streamStateGetGroupKVByCur_rocksdb(pCur, key, NULL, 0);
@ -2050,7 +2047,6 @@ SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey
}
SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) {
stDebug("streamStateSeekKeyNext_rocksdb");
SStreamStateCur* pCur = createStreamStateCursor();
if (pCur == NULL) {
return NULL;
@ -2089,7 +2085,6 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin
}
SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState) {
stDebug("streamStateGetCur_rocksdb");
int32_t code = 0;
const SStateKey maxStateKey = {.key = {.groupId = UINT64_MAX, .ts = INT64_MAX}, .opNum = INT64_MAX};
@ -2130,7 +2125,6 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState) {
}
SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key) {
stDebug("streamStateGetCur_rocksdb");
SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper;
SStreamStateCur* pCur = createStreamStateCursor();

View File

@ -135,12 +135,13 @@ static int32_t appendCheckpointIntoInputQ(SStreamTask* pTask, int32_t checkpoint
int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq) {
ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE);
// todo this status may not be set here.
// 1. set task status to be prepared for check point, no data are allowed to put into inputQ.
pTask->status.taskStatus = TASK_STATUS__CK;
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_GEN_CHECKPOINT);
pTask->checkpointingId = pReq->checkpointId;
pTask->checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask);
pTask->chkInfo.startTs = taosGetTimestampMs();
pTask->execInfo.checkpoint += 1;
// 2. Put the checkpoint block into inputQ, to make sure all blocks with less version have been handled by this task
@ -171,11 +172,11 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc
const char* id = pTask->id.idStr;
int32_t code = TSDB_CODE_SUCCESS;
// set the task status
pTask->checkpointingId = checkpointId;
// set task status
pTask->status.taskStatus = TASK_STATUS__CK;
if (streamTaskGetStatus(pTask, NULL) != TASK_STATUS__CK) {
pTask->checkpointingId = checkpointId;
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_GEN_CHECKPOINT);
}
{ // todo: remove this when the pipeline checkpoint generating is used.
SStreamMeta* pMeta = pTask->pMeta;
@ -195,6 +196,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc
stDebug("s-task:%s set childIdx:%d, and add checkpoint-trigger block into outputQ", id, pTask->info.selfChildId);
continueDispatchCheckpointBlock(pBlock, pTask);
} else { // only one task exists, no need to dispatch downstream info
atomic_add_fetch_32(&pTask->checkpointNotReadyTasks, 1);
streamProcessCheckpointReadyMsg(pTask);
streamFreeQitem((SStreamQueueItem*)pBlock);
}
@ -274,6 +276,8 @@ void streamTaskClearCheckInfo(SStreamTask* pTask) {
}
int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) {
int32_t vgId = pMeta->vgId;
taosWLockLatch(&pMeta->lock);
for (int32_t i = 0; i < taosArrayGetSize(pMeta->pTaskList); ++i) {
@ -288,27 +292,34 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) {
continue;
}
int8_t prev = p->status.taskStatus;
ASSERT(p->chkInfo.checkpointId < p->checkpointingId && p->checkpointingId == checkpointId);
p->chkInfo.checkpointId = p->checkpointingId;
streamTaskClearCheckInfo(p);
streamSetStatusNormal(p);
// save the task
streamMetaSaveTask(pMeta, p);
char* str = NULL;
streamTaskGetStatus(p, &str);
int32_t code = streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE);
if (code != TSDB_CODE_SUCCESS) {
stDebug("s-task:%s vgId:%d save task status failed, since handle event failed", p->id.idStr, vgId);
taosWUnLockLatch(&pMeta->lock);
return -1;
} else { // save the task
streamMetaSaveTask(pMeta, p);
}
stDebug(
"vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, "
"checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s",
pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer,
streamGetTaskStatusStr(prev));
str);
}
if (streamMetaCommit(pMeta) < 0) {
taosWUnLockLatch(&pMeta->lock);
stError("vgId:%d failed to commit stream meta after do checkpoint, checkpointId:%" PRId64 ", since %s", pMeta->vgId,
checkpointId, terrstr());
checkpointId, terrstr());
return -1;
} else {
taosWUnLockLatch(&pMeta->lock);

View File

@ -420,7 +420,7 @@ static void doRetryDispatchData(void* param, void* tmrId) {
const char* id = pTask->id.idStr;
int32_t msgId = pTask->execInfo.dispatch;
if (streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldStop(pTask)) {
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);
return;
@ -474,10 +474,10 @@ static void doRetryDispatchData(void* param, void* tmrId) {
}
if (code != TSDB_CODE_SUCCESS) {
if (!streamTaskShouldStop(&pTask->status)) {
if (!streamTaskShouldStop(pTask)) {
// stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr);
// atomic_store_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 0);
if (streamTaskShouldPause(&pTask->status)) {
if (streamTaskShouldPause(pTask)) {
streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS * 10);
} else {
streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS);
@ -662,8 +662,10 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) {
int32_t numOfVgs = taosArrayGetSize(vgInfo);
pTask->notReadyTasks = numOfVgs;
char* p = NULL;
streamTaskGetStatus(pTask, &p);
stDebug("s-task:%s send scan-history data complete msg to downstream (shuffle-dispatch) %d tasks, status:%s", pTask->id.idStr,
numOfVgs, streamGetTaskStatusStr(pTask->status.taskStatus));
numOfVgs, p);
for (int32_t i = 0; i < numOfVgs; i++) {
SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i);
req.downstreamTaskId = pVgInfo->taskId;
@ -775,8 +777,9 @@ int32_t doDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamScanHist
initRpcMsg(&msg, TDMT_VND_STREAM_SCAN_HISTORY_FINISH, buf, tlen + sizeof(SMsgHead));
tmsgSendReq(pEpSet, &msg);
const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus);
stDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pStatus,
char* p = NULL;
streamTaskGetStatus(pTask, &p);
stDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, p,
pReq->downstreamTaskId, vgId);
return 0;
}
@ -1008,7 +1011,7 @@ int32_t streamAddEndScanHistoryMsg(SStreamTask* pTask, SRpcHandleInfo* pRpcInfo,
taosThreadMutexUnlock(&pTask->lock);
int32_t num = taosArrayGetSize(pTask->pRspMsgList);
stDebug("s-task:%s add scan history finish rsp msg for task:0x%x, total:%d", pTask->id.idStr, pReq->upstreamTaskId,
stDebug("s-task:%s add scan-history finish rsp msg for task:0x%x, total:%d", pTask->id.idStr, pReq->upstreamTaskId,
num);
return TSDB_CODE_SUCCESS;
}
@ -1024,7 +1027,7 @@ int32_t streamNotifyUpstreamContinue(SStreamTask* pTask) {
SStreamContinueExecInfo* pInfo = taosArrayGet(pTask->pRspMsgList, i);
tmsgSendRsp(&pInfo->msg);
stDebug("s-task:%s level:%d notify upstream:0x%x continuing scan data in WAL", id, level, pInfo->taskId);
stDebug("s-task:%s level:%d notify upstream:0x%x continuing handle data in WAL", id, level, pInfo->taskId);
}
taosArrayClear(pTask->pRspMsgList);
@ -1113,16 +1116,16 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i
ASSERT(leftRsp >= 0);
if (leftRsp > 0) {
stDebug( "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d rsp",
id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp);
stDebug( "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%s, waiting for %d rsp",
id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, tstrerror(code), leftRsp);
} else {
stDebug(
"s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp",
id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code);
"s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%s, all rsp",
id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, tstrerror(code));
}
} else {
stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d",
id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code);
stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%s",
id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, tstrerror(code));
}
ASSERT(leftRsp >= 0);
@ -1181,6 +1184,9 @@ int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpda
if (tEncodeSEpSet(pEncoder, &pInfo->prevEp) < 0) return -1;
if (tEncodeSEpSet(pEncoder, &pInfo->newEp) < 0) return -1;
}
// todo this new attribute will be result in being incompatible with previous version
if (tEncodeI32(pEncoder, pMsg->transId) < 0) return -1;
tEndEncode(pEncoder);
return pEncoder->pos;
}
@ -1201,6 +1207,8 @@ int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg*
taosArrayPush(pMsg->pNodeList, &info);
}
if (tDecodeI32(pDecoder, &pMsg->transId) < 0) return -1;
tEndDecode(pDecoder);
return 0;
}

View File

@ -22,14 +22,13 @@
static int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask);
bool streamTaskShouldStop(const SStreamStatus* pStatus) {
int32_t status = atomic_load_8((int8_t*)&pStatus->taskStatus);
return (status == TASK_STATUS__STOP) || (status == TASK_STATUS__DROPPING);
bool streamTaskShouldStop(const SStreamTask* pTask) {
ETaskStatus s = streamTaskGetStatus(pTask, NULL);
return (s == TASK_STATUS__STOP) || (s == TASK_STATUS__DROPPING);
}
bool streamTaskShouldPause(const SStreamStatus* pStatus) {
int32_t status = atomic_load_8((int8_t*)&pStatus->taskStatus);
return (status == TASK_STATUS__PAUSE);
bool streamTaskShouldPause(const SStreamTask* pTask) {
return (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__PAUSE);
}
static int32_t doOutputResultBlockImpl(SStreamTask* pTask, SStreamDataBlock* pBlock) {
@ -102,7 +101,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i
pRes = taosArrayInit(4, sizeof(SSDataBlock));
}
if (streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldStop(pTask)) {
taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes);
return 0;
}
@ -198,7 +197,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) {
qSetStreamOpOpen(exec);
while (!finished) {
if (streamTaskShouldPause(&pTask->status)) {
if (streamTaskShouldPause(pTask)) {
double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0;
stDebug("s-task:%s paused from the scan-history task, elapsed time:%.2fsec", pTask->id.idStr, el);
break;
@ -213,7 +212,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) {
int32_t size = 0;
int32_t numOfBlocks = 0;
while (1) {
if (streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldStop(pTask)) {
taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes);
return 0;
}
@ -309,21 +308,19 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) {
pStreamTask->id.idStr);
}
ASSERT(((pStreamTask->status.taskStatus == TASK_STATUS__STOP) ||
(pStreamTask->hTaskInfo.id.taskId == pTask->id.taskId)) &&
ETaskStatus status = streamTaskGetStatus(pStreamTask, NULL);
ASSERT(((status == TASK_STATUS__DROPPING) || (pStreamTask->hTaskInfo.id.taskId == pTask->id.taskId)) &&
pTask->status.appendTranstateBlock == true);
STimeWindow* pTimeWindow = &pStreamTask->dataRange.window;
// todo. the dropping status should be append to the status after the halt completed.
// It must be halted for a source stream task, since when the related scan-history-data task start scan the history
// for the step 2.
int8_t status = pStreamTask->status.taskStatus;
if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) {
ASSERT(status == TASK_STATUS__HALT || status == TASK_STATUS__DROPPING);
ASSERT(status == TASK_STATUS__HALT || status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP);
} else {
ASSERT(status == TASK_STATUS__NORMAL);
pStreamTask->status.taskStatus = TASK_STATUS__HALT;
ASSERT(status == TASK_STATUS__READY|| status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP);
streamTaskHandleEvent(pStreamTask->status.pSM, TASK_EVENT_HALT);
stDebug("s-task:%s halt by related fill-history task:%s", pStreamTask->id.idStr, pTask->id.idStr);
}
@ -333,13 +330,14 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) {
// In case of sink tasks, no need to halt them.
// In case of source tasks and agg tasks, we should HALT them, and wait for them to be idle. And then, it's safe to
// start the task state transfer procedure.
// When a task is idle with halt status, all data in inputQ are consumed.
char* p = NULL;
streamTaskGetStatus(pStreamTask, &p);
if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) {
// update the scan data range for source task.
stDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " update to %" PRId64 " - %" PRId64
", status:%s, sched-status:%d",
pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN,
pTimeWindow->ekey, streamGetTaskStatusStr(TASK_STATUS__NORMAL), pStreamTask->status.schedStatus);
", status:%s, sched-status:%d",
pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN,
pTimeWindow->ekey, p, pStreamTask->status.schedStatus);
} else {
stDebug("s-task:%s no need to update time window for non-source task", pStreamTask->id.idStr);
}
@ -354,7 +352,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) {
// 3. resume the state of stream task, after this function, the stream task will run immidately. But it can not be
// pause, since the pause allowed attribute is not set yet.
streamTaskResumeFromHalt(pStreamTask);
streamTaskResume(pStreamTask); // todo refactor: use streamTaskResume.
stDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr);
@ -362,20 +360,21 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) {
streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id);
// 5. clear the link between fill-history task and stream task info
pStreamTask->hTaskInfo.id.taskId = 0;
pStreamTask->hTaskInfo.id.streamId = 0;
// CLEAR_RELATED_FILLHISTORY_TASK(pStreamTask);
// 6. save to disk
taosWLockLatch(&pMeta->lock);
streamMetaSaveTask(pMeta, pStreamTask);
if (streamMetaCommit(pMeta) < 0) {
pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask, NULL);
// streamMetaSaveTask(pMeta, pStreamTask);
// if (streamMetaCommit(pMeta) < 0) {
// persist to disk
}
// }
taosWUnLockLatch(&pMeta->lock);
// 7. pause allowed.
streamTaskEnablePause(pStreamTask);
if (taosQueueEmpty(pStreamTask->inputInfo.queue->pQueue)) {
if ((pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) && taosQueueEmpty(pStreamTask->inputInfo.queue->pQueue)) {
SStreamRefDataBlock* pItem = taosAllocateQitem(sizeof(SStreamRefDataBlock), DEF_QITEM, 0);
SSDataBlock* pDelBlock = createSpecialDataBlock(STREAM_DELETE_DATA);
@ -492,7 +491,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock
code = taosWriteQitem(pTask->outputq.queue->pQueue, pBlock);
if (code == 0) {
streamDispatchStreamBlock(pTask);
} else {
} else { // todo put into queue failed, retry
streamFreeQitem((SStreamQueueItem*)pBlock);
}
} else { // level == TASK_LEVEL__SINK
@ -500,12 +499,12 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock
}
} else { // non-dispatch task, do task state transfer directly
streamFreeQitem((SStreamQueueItem*)pBlock);
stDebug("s-task:%s non-dispatch task, start to transfer state directly", id);
stDebug("s-task:%s non-dispatch task, level:%d start to transfer state directly", id, pTask->info.taskLevel);
ASSERT(pTask->info.fillHistory == 1);
code = streamTransferStateToStreamTask(pTask);
code = streamTransferStateToStreamTask(pTask);
if (code != TSDB_CODE_SUCCESS) {
/*int8_t status = */ streamTaskSetSchedStatusInActive(pTask);
/*int8_t status = */ streamTaskSetSchedStatusInactive(pTask);
}
}
@ -526,7 +525,7 @@ int32_t streamExecForAll(SStreamTask* pTask) {
int32_t blockSize = 0;
int32_t numOfBlocks = 0;
SStreamQueueItem* pInput = NULL;
if (streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldStop(pTask)) {
stDebug("s-task:%s stream task is stopped", id);
break;
}
@ -592,8 +591,9 @@ int32_t streamExecForAll(SStreamTask* pTask) {
// todo other thread may change the status
// do nothing after sync executor state to storage backend, untill the vnode-level checkpoint is completed.
if (type == STREAM_INPUT__CHECKPOINT) {
stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr,
streamGetTaskStatusStr(pTask->status.taskStatus));
char* p = NULL;
streamTaskGetStatus(pTask, &p);
stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, p);
streamTaskBuildCheckpoint(pTask);
return 0;
}
@ -605,8 +605,15 @@ int32_t streamExecForAll(SStreamTask* pTask) {
// the task may be set dropping/stopping, while it is still in the task queue, therefore, the sched-status can not
// be updated by tryExec function, therefore, the schedStatus will always be the TASK_SCHED_STATUS__WAITING.
bool streamTaskIsIdle(const SStreamTask* pTask) {
return (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE || pTask->status.taskStatus == TASK_STATUS__STOP ||
pTask->status.taskStatus == TASK_STATUS__DROPPING);
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
return (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE || status == TASK_STATUS__STOP ||
status == TASK_STATUS__DROPPING);
}
bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) {
ETaskStatus st = streamTaskGetStatus(pTask, NULL);
return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__STREAM_SCAN_HISTORY ||
st == TASK_STATUS__CK);
}
int32_t streamExecTask(SStreamTask* pTask) {
@ -623,20 +630,23 @@ int32_t streamExecTask(SStreamTask* pTask) {
}
taosThreadMutexLock(&pTask->lock);
if ((streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0) || streamTaskShouldStop(&pTask->status) ||
streamTaskShouldPause(&pTask->status)) {
if ((streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0) || streamTaskShouldStop(pTask) ||
streamTaskShouldPause(pTask)) {
atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE);
taosThreadMutexUnlock(&pTask->lock);
stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id,
streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus);
char* p = NULL;
streamTaskGetStatus(pTask, &p);
stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, p, pTask->status.schedStatus);
return 0;
}
taosThreadMutexUnlock(&pTask->lock);
}
} else {
stDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id,
streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus);
char* p = NULL;
streamTaskGetStatus(pTask, &p);
stDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, p,
pTask->status.schedStatus);
}
return 0;

View File

@ -143,8 +143,8 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF
goto _err;
}
pMeta->pUpdateTaskSet = taosHashInit(64, fp, false, HASH_NO_LOCK);
if (pMeta->pUpdateTaskSet == NULL) {
pMeta->updateInfo.pTasks = taosHashInit(64, fp, false, HASH_NO_LOCK);
if (pMeta->updateInfo.pTasks == NULL) {
goto _err;
}
@ -219,7 +219,7 @@ _err:
if (pMeta->pCheckpointDb) tdbTbClose(pMeta->pCheckpointDb);
if (pMeta->db) tdbClose(pMeta->db);
if (pMeta->pHbInfo) taosMemoryFreeClear(pMeta->pHbInfo);
if (pMeta->pUpdateTaskSet) taosHashCleanup(pMeta->pUpdateTaskSet);
if (pMeta->updateInfo.pTasks) taosHashCleanup(pMeta->updateInfo.pTasks);
if (pMeta->startInfo.pReadyTaskSet) taosHashCleanup(pMeta->startInfo.pReadyTaskSet);
taosMemoryFree(pMeta);
@ -230,9 +230,9 @@ _err:
int32_t streamMetaReopen(SStreamMeta* pMeta) {
streamMetaClear(pMeta);
// NOTE: role should not be changed during reopen meta
pMeta->streamBackendRid = -1;
pMeta->streamBackend = NULL;
pMeta->role = NODE_ROLE_UNINIT;
char* defaultPath = taosMemoryCalloc(1, strlen(pMeta->path) + 128);
sprintf(defaultPath, "%s%s%s", pMeta->path, TD_DIRSEP, "state");
@ -274,6 +274,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) {
}
void streamMetaClear(SStreamMeta* pMeta) {
// remove all existed tasks in this vnode
void* pIter = NULL;
while ((pIter = taosHashIterate(pMeta->pTasksMap, pIter)) != NULL) {
SStreamTask* p = *(SStreamTask**)pIter;
@ -299,6 +300,9 @@ void streamMetaClear(SStreamMeta* pMeta) {
taosArrayClear(pMeta->chkpInUse);
pMeta->numOfStreamTasks = 0;
pMeta->numOfPausedTasks = 0;
pMeta->chkptNotReadyTasks = 0;
streamMetaResetStartInfo(&pMeta->startInfo);
}
void streamMetaClose(SStreamMeta* pMeta) {
@ -336,7 +340,7 @@ void streamMetaCloseImpl(void* arg) {
taosHashCleanup(pMeta->pTasksMap);
taosHashCleanup(pMeta->pTaskBackendUnique);
taosHashCleanup(pMeta->pUpdateTaskSet);
taosHashCleanup(pMeta->updateInfo.pTasks);
taosHashCleanup(pMeta->startInfo.pReadyTaskSet);
taosMemoryFree(pMeta->pHbInfo);
@ -348,6 +352,7 @@ void streamMetaCloseImpl(void* arg) {
stDebug("end to close stream meta");
}
// todo let's check the status for each task
int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) {
void* buf = NULL;
int32_t len;
@ -461,7 +466,7 @@ SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t
STaskId id = {.streamId = streamId, .taskId = taskId};
SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
if (ppTask != NULL) {
if (!streamTaskShouldStop(&(*ppTask)->status)) {
if (!streamTaskShouldStop(*ppTask)) {
int32_t ref = atomic_add_fetch_32(&(*ppTask)->refCnt, 1);
taosRUnLockLatch(&pMeta->lock);
stTrace("s-task:%s acquire task, ref:%d", (*ppTask)->id.idStr, ref);
@ -478,7 +483,7 @@ void streamMetaReleaseTask(SStreamMeta* UNUSED_PARAM(pMeta), SStreamTask* pTask)
if (ref > 0) {
stTrace("s-task:%s release task, ref:%d", pTask->id.idStr, ref);
} else if (ref == 0) {
ASSERT(streamTaskShouldStop(&pTask->status));
ASSERT(streamTaskShouldStop(pTask));
stTrace("s-task:%s all refs are gone, free it", pTask->id.idStr);
tFreeStreamTask(pTask);
} else if (ref < 0) {
@ -506,11 +511,15 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t
SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
if (ppTask) {
pTask = *ppTask;
if (streamTaskShouldPause(&pTask->status)) {
// desc the paused task counter
if (streamTaskShouldPause(pTask)) {
int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1);
stInfo("vgId:%d s-task:%s drop stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num);
}
atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING);
// handle the dropping event
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_DROPPING);
} else {
stDebug("vgId:%d failed to find the task:0x%x, it may be dropped already", pMeta->vgId, taskId);
taosWUnLockLatch(&pMeta->lock);
@ -518,13 +527,12 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t
}
taosWUnLockLatch(&pMeta->lock);
stDebug("s-task:0x%x set task status:%s and start to unregister it", taskId,
streamGetTaskStatusStr(TASK_STATUS__DROPPING));
stDebug("s-task:0x%x set task status:dropping and start to unregister it", taskId);
while (1) {
taosRLockLatch(&pMeta->lock);
ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
if (ppTask) {
if ((*ppTask)->status.timerActive == 0) {
taosRUnLockLatch(&pMeta->lock);
@ -544,23 +552,19 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t
taosWLockLatch(&pMeta->lock);
ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
if (ppTask) {
pTask = *ppTask;
// it is an fill-history task, remove the related stream task's id that points to it
if ((*ppTask)->info.fillHistory == 1) {
STaskId streamTaskId = {.streamId = (*ppTask)->streamTaskId.streamId, .taskId = (*ppTask)->streamTaskId.taskId};
SStreamTask** ppStreamTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &streamTaskId, sizeof(streamTaskId));
if (ppStreamTask != NULL) {
(*ppStreamTask)->hTaskInfo.id.taskId = 0;
(*ppStreamTask)->hTaskInfo.id.streamId = 0;
}
if (pTask->info.fillHistory == 1) {
streamTaskClearHTaskAttr(pTask);
} else {
atomic_sub_fetch_32(&pMeta->numOfStreamTasks, 1);
}
taosHashRemove(pMeta->pTasksMap, &id, sizeof(id));
atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING);
doRemoveIdFromList(pMeta, (int32_t)taosArrayGetSize(pMeta->pTaskList), &pTask->id);
ASSERT(pTask->status.timerActive == 0);
doRemoveIdFromList(pMeta, (int32_t)taosArrayGetSize(pMeta->pTaskList), &pTask->id);
if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) {
stDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", pTask->id.idStr, pTask->refCnt);
@ -691,8 +695,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) {
tFreeStreamTask(pTask);
stError(
"vgId:%d stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild "
"stream "
"manually",
"stream manually",
vgId, tsDataDir);
return -1;
}
@ -703,8 +706,8 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) {
tFreeStreamTask(pTask);
STaskId id = streamTaskExtractKey(pTask);
taosArrayPush(pRecycleList, &id);
int32_t total = taosArrayGetSize(pRecycleList);
stDebug("s-task:0x%x is already dropped, add into recycle list, total:%d", taskId, total);
continue;
@ -740,7 +743,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) {
atomic_add_fetch_32(&pMeta->numOfStreamTasks, 1);
}
if (streamTaskShouldPause(&pTask->status)) {
if (streamTaskShouldPause(pTask)) {
atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1);
}
@ -854,8 +857,8 @@ void metaHbToMnode(void* param, void* tmrId) {
}
// not leader not send msg
if (pMeta->role == NODE_ROLE_FOLLOWER) {
stInfo("vgId:%d follower not send hb to mnode", pMeta->vgId);
if (pMeta->role != NODE_ROLE_LEADER) {
stInfo("vgId:%d role:%d not leader not send hb to mnode", pMeta->vgId, pMeta->role);
taosReleaseRef(streamMetaId, rid);
pMeta->pHbInfo->hbStart = 0;
return;
@ -895,7 +898,7 @@ void metaHbToMnode(void* param, void* tmrId) {
STaskStatusEntry entry = {
.id = *pId,
.status = (*pTask)->status.taskStatus,
.status = streamTaskGetStatus(*pTask, NULL),
.nodeId = pMeta->vgId,
.stage = pMeta->stage,
.inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputInfo.queue)),
@ -977,9 +980,8 @@ void metaHbToMnode(void* param, void* tmrId) {
taosReleaseRef(streamMetaId, rid);
}
static bool hasStreamTaskInTimer(SStreamMeta* pMeta) {
bool streamMetaTaskInTimer(SStreamMeta* pMeta) {
bool inTimer = false;
taosWLockLatch(&pMeta->lock);
void* pIter = NULL;
@ -1033,7 +1035,7 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) {
stDebug("vgId:%d start to check all tasks", vgId);
int64_t st = taosGetTimestampMs();
while (hasStreamTaskInTimer(pMeta)) {
while (streamMetaTaskInTimer(pMeta)) {
stDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pMeta->vgId);
taosMsleep(100);
}

View File

@ -119,9 +119,8 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) {
return numOfItems1 + numOfItems2;
}
// todo: fix it: data in Qall is not included here
int32_t streamQueueGetItemSize(const SStreamQueue* pQueue) {
return taosQueueMemorySize(pQueue->pQueue);
return taosQueueMemorySize(pQueue->pQueue) + taosQallUnAccessedMemSize(pQueue->qall);
}
int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem) {
@ -165,7 +164,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
}
while (1) {
if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldPause(pTask) || streamTaskShouldStop(pTask)) {
stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks);
return TSDB_CODE_SUCCESS;
}
@ -346,7 +345,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc
STaosQueue* pQueue = pTask->outputq.queue->pQueue;
while (streamQueueIsFull(pTask->outputq.queue)) {
if (streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldStop(pTask)) {
stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr);
return TSDB_CODE_STREAM_EXEC_CANCELLED;
}

View File

@ -17,6 +17,7 @@
#include "trpc.h"
#include "ttimer.h"
#include "wal.h"
#include "streamsm.h"
typedef struct SLaunchHTaskInfo {
SStreamMeta* pMeta;
@ -34,16 +35,18 @@ static void streamTaskSetRangeStreamCalc(SStreamTask* pTask);
static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated);
static SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, int32_t taskId);
static void tryLaunchHistoryTask(void* param, void* tmrId);
static int32_t updateTaskReadyInMeta(SStreamTask* pTask);
static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) {
SStreamMeta* pMeta = pTask->pMeta;
int32_t vgId = pMeta->vgId;
int32_t streamTaskSetReady(SStreamTask* pTask) {
char* p = NULL;
int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask);
ETaskStatus status = streamTaskGetStatus(pTask, &p);
if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) {
if ((status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__STREAM_SCAN_HISTORY) &&
pTask->info.taskLevel != TASK_LEVEL__SOURCE) {
pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->upstreamInfo.pList);
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,
streamGetTaskStatusStr(pTask->status.taskStatus));
pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, p);
}
ASSERT(pTask->status.downstreamReady == 0);
@ -52,34 +55,10 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) {
pTask->execInfo.start = taosGetTimestampMs();
int64_t el = (pTask->execInfo.start - pTask->execInfo.init);
stDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%" PRId64 "ms, task status:%s",
pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus));
pTask->id.idStr, numOfDowns, el, p);
taosWLockLatch(&pMeta->lock);
STaskId id = streamTaskExtractKey(pTask);
taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0);
int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta);
if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) {
STaskStartInfo* pStartInfo = &pMeta->startInfo;
pStartInfo->readyTs = pTask->execInfo.start;
if (pStartInfo->startTs != 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);
updateTaskReadyInMeta(pTask);
return TSDB_CODE_SUCCESS;
}
int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) {
@ -114,28 +93,23 @@ static int32_t doStartScanHistoryTask(SStreamTask* pTask) {
}
int32_t streamTaskStartScanHistory(SStreamTask* pTask) {
ASSERT(pTask->status.downstreamReady == 1);
int32_t level = pTask->info.taskLevel;
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) {
return doStartScanHistoryTask(pTask);
} else {
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,
streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus,
walReaderGetCurrentVer(pTask->exec.pWalReader));
streamTaskEnablePause(pTask);
}
} else if (pTask->info.taskLevel == TASK_LEVEL__AGG) {
ASSERT(pTask->status.downstreamReady == 1 &&
((status == TASK_STATUS__SCAN_HISTORY) || (status == TASK_STATUS__STREAM_SCAN_HISTORY)));
if (level == TASK_LEVEL__SOURCE) {
return doStartScanHistoryTask(pTask);
} else if (level == TASK_LEVEL__AGG) {
if (pTask->info.fillHistory) {
streamSetParamForScanHistory(pTask);
streamTaskEnablePause(pTask);
}
} else if (pTask->info.taskLevel == TASK_LEVEL__SINK) {
if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) {
stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr);
}
} else if (level == TASK_LEVEL__SINK) {
stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr);
}
return 0;
}
@ -152,6 +126,8 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) {
.stage = pTask->pMeta->stage,
};
ASSERT(pTask->status.downstreamReady == 0);
// serialize streamProcessScanHistoryFinishRsp
if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) {
req.reqId = tGenIdPI64();
@ -187,11 +163,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) {
}
} else {
stDebug("s-task:%s (vgId:%d) set downstream ready, since no downstream", pTask->id.idStr, pTask->info.nodeId);
streamTaskSetReady(pTask, 0);
streamTaskSetRangeStreamCalc(pTask);
streamTaskStartScanHistory(pTask);
streamLaunchFillHistoryTask(pTask);
streamTaskOnHandleEventSuccess(pTask->status.pSM);
}
return 0;
@ -280,22 +252,76 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_
}
if (pInfo->stage != stage) {
return TASK_SELF_NEW_STAGE;
return TASK_UPSTREAM_NEW_STAGE;
} else if (pTask->status.downstreamReady != 1) {
stDebug("s-task:%s vgId:%d leader:%d, downstream not ready", id, vgId, (pTask->pMeta->role == NODE_ROLE_LEADER));
return TASK_DOWNSTREAM_NOT_READY;
} else {
return TASK_DOWNSTREAM_READY;
}
}
static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) {
streamTaskSetReady(pTask, numOfReqs);
int32_t onNormalTaskReady(SStreamTask* pTask) {
const char* id = pTask->id.idStr;
streamTaskSetReady(pTask);
streamTaskSetRangeStreamCalc(pTask);
char* p = NULL;
ETaskStatus status = streamTaskGetStatus(pTask, &p);
ASSERT(status == TASK_STATUS__READY);
// todo refactor: remove this later
// 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);
// pTask->status.taskStatus = TASK_STATUS__DROPPING;
// ASSERT(pTask->hTaskInfo.id.taskId == 0);
// }
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ready for data from wal ver:%" PRId64,
id, p, pTask->status.schedStatus, walReaderGetCurrentVer(pTask->exec.pWalReader));
} else {
stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p, pTask->status.schedStatus);
}
streamTaskEnablePause(pTask);
return TSDB_CODE_SUCCESS;
}
int32_t onScanhistoryTaskReady(SStreamTask* pTask) {
const char* id = pTask->id.idStr;
// set the state to be ready
streamTaskSetReady(pTask);
streamTaskSetRangeStreamCalc(pTask);
char* p = NULL;
ETaskStatus status = streamTaskGetStatus(pTask, &p);
ASSERT(status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__STREAM_SCAN_HISTORY);
stDebug("s-task:%s enter into scan-history data stage, status:%s", id, p);
streamTaskStartScanHistory(pTask);
// start the related fill-history task, when current task is ready
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
streamLaunchFillHistoryTask(pTask);
}
return TSDB_CODE_SUCCESS;
}
// todo: refactor this function.
static void doProcessDownstreamReadyRsp(SStreamTask* pTask) {
streamTaskOnHandleEventSuccess(pTask->status.pSM);
#if 0
const char* id = pTask->id.idStr;
int8_t status = pTask->status.taskStatus;
const char* str = streamGetTaskStatusStr(status);
ASSERT(status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__NORMAL);
ASSERT(status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__READY);
streamTaskSetRangeStreamCalc(pTask);
if (status == TASK_STATUS__SCAN_HISTORY) {
@ -314,13 +340,14 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) {
streamTaskEnablePause(pTask);
}
}
#endif
}
int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) {
ASSERT(pTask->id.taskId == pRsp->upstreamTaskId);
const char* id = pTask->id.idStr;
if (streamTaskShouldStop(&pTask->status)) {
if (streamTaskShouldStop(pTask)) {
stDebug("s-task:%s should stop, do not do check downstream again", id);
return TSDB_CODE_SUCCESS;
}
@ -349,7 +376,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs
taosArrayDestroy(pTask->checkReqIds);
pTask->checkReqIds = NULL;
doProcessDownstreamReadyRsp(pTask, numOfReqs);
doProcessDownstreamReadyRsp(pTask);
} else {
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,
@ -361,28 +388,29 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs
return -1;
}
doProcessDownstreamReadyRsp(pTask, 1);
doProcessDownstreamReadyRsp(pTask);
}
} else { // not ready, wait for 100ms and retry
if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) {
if (pRsp->status == TASK_UPSTREAM_NEW_STAGE) {
stError(
"s-task:%s downstream taskId:0x%x (vgId:%d) vnode-transfer/leader-change detected, not send check again, "
"roll-back needed",
id, pRsp->downstreamTaskId, pRsp->downstreamNodeId);
"s-task:%s vgId:%d self vnode-transfer/leader-change/restart detected, old stage:%d, current stage:%d, "
"not check wait for downstream task nodeUpdate, and all tasks restart",
id, pRsp->upstreamNodeId, pRsp->oldStage, (int32_t)pTask->pMeta->stage);
} else {
if (pRsp->status == TASK_SELF_NEW_STAGE) {
if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) {
stError(
"s-task:%s vnode-transfer/leader-change/restart detected, old stage:%d, current stage:%d, continue check "
"till downstream nodeUpdate",
id, pRsp->oldStage, (int32_t)pTask->pMeta->stage);
"s-task:%s downstream taskId:0x%x (vgId:%d) not leader, self dispatch epset needs to be updated, not check "
"downstream again, nodeUpdate needed",
id, pRsp->downstreamTaskId, pRsp->downstreamNodeId);
return 0;
}
STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp);
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,
pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref);
taosTmrReset(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer, &pInfo->checkTimer);
pInfo->checkTimer = taosTmrStart(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer);
}
}
@ -426,30 +454,6 @@ int32_t streamRestoreParam(SStreamTask* pTask) {
return qRestoreStreamOperatorOption(pTask->exec.pExecutor);
}
int32_t streamSetStatusNormal(SStreamTask* pTask) {
int32_t status = atomic_load_8(&pTask->status.taskStatus);
if (status == TASK_STATUS__DROPPING) {
stError("s-task:%s cannot be set normal, since in dropping state", pTask->id.idStr);
return -1;
} else {
stDebug("s-task:%s set task status to be normal, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status));
atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__NORMAL);
return 0;
}
}
int32_t streamSetStatusUnint(SStreamTask* pTask) {
int32_t status = atomic_load_8(&pTask->status.taskStatus);
if (status == TASK_STATUS__DROPPING) {
stError("s-task:%s cannot be set uninit, since in dropping state", pTask->id.idStr);
return -1;
} else {
stDebug("s-task:%s set task status to be uninit, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status));
atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__UNINIT);
return 0;
}
}
// source
int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) {
return qStreamSourceScanParamForHistoryScanStep1(pTask->exec.pExecutor, pVerRange, pWindow);
@ -515,9 +519,13 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory
int32_t taskLevel = pTask->info.taskLevel;
ASSERT(taskLevel == TASK_LEVEL__AGG || taskLevel == TASK_LEVEL__SINK);
if (pTask->status.taskStatus != TASK_STATUS__SCAN_HISTORY) {
const char* id = pTask->id.idStr;
char* p = NULL;
ETaskStatus status = streamTaskGetStatus(pTask, &p);
if (status != TASK_STATUS__SCAN_HISTORY && status != TASK_STATUS__STREAM_SCAN_HISTORY) {
stError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly",
pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), pReq->upstreamTaskId);
id, p, pReq->upstreamTaskId);
void* pBuf = NULL;
int32_t len = 0;
@ -527,8 +535,8 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory
initRpcMsg(&msg, 0, pBuf, sizeof(SMsgHead) + len);
tmsgSendRsp(&msg);
stDebug("s-task:%s level:%d notify upstream:0x%x(vgId:%d) to continue process data in WAL", pTask->id.idStr,
pTask->info.taskLevel, pReq->upstreamTaskId, pReq->upstreamNodeId);
stDebug("s-task:%s level:%d notify upstream:0x%x(vgId:%d) to continue process data in WAL", id,
taskLevel, pReq->upstreamTaskId, pReq->upstreamNodeId);
return 0;
}
@ -540,13 +548,15 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory
if (left == 0) {
int32_t numOfTasks = taosArrayGetSize(pTask->upstreamInfo.pList);
stDebug(
"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",
pTask->id.idStr, numOfTasks);
if (pTask->info.taskLevel == TASK_LEVEL__AGG) {
if (taskLevel == TASK_LEVEL__AGG) {
stDebug(
"s-task:%s all %d upstream tasks finish scan-history data, set param for agg task for stream data processing "
"and send rsp to all upstream tasks",
id, numOfTasks);
streamAggUpstreamScanHistoryFinish(pTask);
} else {
stDebug("s-task:%s all %d upstream task(s) finish scan-history data, and rsp to all upstream tasks", id,
numOfTasks);
}
// all upstream tasks have completed the scan-history task in the stream time window, let's start to extract data
@ -557,26 +567,25 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory
if (taskLevel == TASK_LEVEL__AGG) {
/*int32_t code = */streamTaskScanHistoryDataComplete(pTask);
} else { // for sink task, set normal
if (pTask->status.taskStatus != TASK_STATUS__PAUSE && pTask->status.taskStatus != TASK_STATUS__STOP &&
pTask->status.taskStatus != TASK_STATUS__DROPPING) {
streamSetStatusNormal(pTask);
}
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE);
}
} else {
stDebug("s-task:%s receive scan-history data finish msg from upstream:0x%x(index:%d), unfinished:%d",
pTask->id.idStr, pReq->upstreamTaskId, pReq->childId, left);
id, pReq->upstreamTaskId, pReq->childId, left);
}
return 0;
}
int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) {
ASSERT(pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY);
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
ASSERT(status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__STREAM_SCAN_HISTORY);
SStreamMeta* pMeta = pTask->pMeta;
// execute in the scan history complete call back msg, ready to process data from inputQ
streamSetStatusNormal(pTask);
streamTaskSetSchedStatusInActive(pTask);
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE);
streamTaskSetSchedStatusInactive(pTask);
taosWLockLatch(&pMeta->lock);
streamMetaSaveTask(pMeta, pTask);
@ -604,15 +613,15 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask)
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
stDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64
" ver range:%" PRId64 " - %" PRId64", init:%"PRId64,
" verRange:%" PRId64 " - %" PRId64", init:%"PRId64,
pTask->id.idStr, pHTask->id.idStr, pRange->window.skey, pRange->window.ekey,
pRange->range.minVer, pRange->range.maxVer, pHTask->execInfo.init);
} 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);
}
// check if downstream tasks have been ready
doCheckDownstreamStatus(pHTask);
streamTaskHandleEvent(pHTask->status.pSM, TASK_EVENT_INIT_SCANHIST);
}
static void tryLaunchHistoryTask(void* param, void* tmrId) {
@ -624,12 +633,12 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) {
if (ppTask) {
ASSERT((*ppTask)->status.timerActive >= 1);
if (streamTaskShouldStop(&(*ppTask)->status)) {
const char* pStatus = streamGetTaskStatusStr((*ppTask)->status.taskStatus);
if (streamTaskShouldStop(*ppTask)) {
char* p = NULL;
streamTaskGetStatus((*ppTask), &p);
int32_t ref = atomic_sub_fetch_32(&(*ppTask)->status.timerActive, 1);
stDebug("s-task:%s status:%s should stop, quit launch fill-history task timer, retry:%d, ref:%d",
(*ppTask)->id.idStr, pStatus, (*ppTask)->hTaskInfo.retryTimes, ref);
(*ppTask)->id.idStr, p, (*ppTask)->hTaskInfo.retryTimes, ref);
taosMemoryFree(pInfo);
taosWUnLockLatch(&pMeta->lock);
@ -665,9 +674,11 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) {
// abort the timer if intend to stop task
SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pHTaskInfo->id.streamId, pHTaskInfo->id.taskId);
if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) {
const char* p = streamGetTaskStatusStr(pTask->status.taskStatus);
int32_t hTaskId = pHTaskInfo->id.taskId;
if (pHTask == NULL && (!streamTaskShouldStop(pTask))) {
char* p = NULL;
int32_t hTaskId = pHTaskInfo->id.taskId;
streamTaskGetStatus(pTask, &p);
stDebug(
"s-task:%s status:%s failed to launch fill-history task:0x%x, retry launch:%dms, retryCount:%d",
pTask->id.idStr, p, hTaskId, pHTaskInfo->waitInterval, pHTaskInfo->retryTimes);
@ -713,11 +724,8 @@ SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, in
int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) {
SStreamMeta* pMeta = pTask->pMeta;
int32_t hTaskId = pTask->hTaskInfo.id.taskId;
if (hTaskId == 0) {
return TSDB_CODE_SUCCESS;
}
ASSERT(pTask->status.downstreamReady == 1);
ASSERT((hTaskId != 0) && (pTask->status.downstreamReady == 1));
stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr,
pTask->hTaskInfo.id.streamId, hTaskId);
@ -765,7 +773,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) {
}
int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) {
if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) {
if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__DROPPING) {
return 0;
}
@ -897,7 +905,7 @@ int32_t tDecodeStreamScanHistoryFinishReq(SDecoder* pDecoder, SStreamScanHistory
void streamTaskSetRangeStreamCalc(SStreamTask* pTask) {
SDataRange* pRange = &pTask->dataRange;
if (pTask->hTaskInfo.id.taskId == 0) {
if (!HAS_RELATED_FILLHISTORY_TASK(pTask)) {
if (pTask->info.fillHistory == 1) {
stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64,
pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer);
@ -931,19 +939,17 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) {
// only the downstream tasks are ready, set the task to be ready to work.
void streamTaskCheckDownstream(SStreamTask* pTask) {
if (pTask->info.fillHistory) {
stDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr);
return;
}
// if (pTask->info.fillHistory) {
// ASSERT(0);
// stDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr);
// return;
// }
ASSERT(pTask->status.downstreamReady == 0);
doCheckDownstreamStatus(pTask);
}
// normal -> pause, pause/stop/dropping -> pause, halt -> pause, scan-history -> pause
void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) {
int64_t st = taosGetTimestampMs();
#if 0
int8_t status = pTask->status.taskStatus;
if (status == TASK_STATUS__DROPPING) {
stDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr);
@ -1001,43 +1007,54 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) {
stInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num);
taosWUnLockLatch(&pMeta->lock);
#endif
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_PAUSE);
int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1);
stInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num);
// in case of fill-history task, stop the tsdb file scan operation.
if (pTask->info.fillHistory == 1) {
void* pExecutor = pTask->exec.pExecutor;
qKillTask(pExecutor, TSDB_CODE_SUCCESS);
}
int64_t el = taosGetTimestampMs() - st;
stDebug("vgId:%d s-task:%s set pause flag, prev:%s, pause elapsed time:%dms", pMeta->vgId, pTask->id.idStr,
streamGetTaskStatusStr(pTask->status.keepTaskStatus), (int32_t)el);
stDebug("vgId:%d s-task:%s set pause flag and pause task", pMeta->vgId, pTask->id.idStr);
}
void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) {
int8_t status = pTask->status.taskStatus;
if (status == TASK_STATUS__PAUSE) {
pTask->status.taskStatus = pTask->status.keepTaskStatus;
pTask->status.keepTaskStatus = TASK_STATUS__NORMAL;
int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1);
stInfo("vgId:%d s-task:%s resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num);
} else if (pTask->info.taskLevel == TASK_LEVEL__SINK) {
int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1);
stInfo("vgId:%d s-task:%s sink task.resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num);
void streamTaskResume(SStreamTask* pTask) {
char* p = NULL;
ETaskStatus status = streamTaskGetStatus(pTask, &p);
SStreamMeta* pMeta = pTask->pMeta;
if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__HALT) {
streamTaskRestoreStatus(pTask);
char* pNew = NULL;
streamTaskGetStatus(pTask, &pNew);
if (status == TASK_STATUS__PAUSE) {
int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1);
stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, p, num);
} else {
stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, p);
}
} else {
stError("s-task:%s not in pause, failed to resume, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status));
stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, p);
}
}
// todo fix race condition
void streamTaskDisablePause(SStreamTask* pTask) {
// pre-condition check
const char* id = pTask->id.idStr;
while (pTask->status.taskStatus == TASK_STATUS__PAUSE) {
stDebug("s-task:%s already in pause, wait for pause being cancelled, and set pause disabled, recheck in 100ms", id);
taosMsleep(100);
}
stDebug("s-task:%s disable task pause", id);
pTask->status.pauseAllowed = 0;
// const char* id = pTask->id.idStr;
// while (pTask->status.taskStatus == TASK_STATUS__PAUSE) {
// stDebug("s-task:%s already in pause, wait for pause being cancelled, and set pause disabled, recheck in 100ms", id);
// taosMsleep(100);
// }
//
// stDebug("s-task:%s disable task pause", id);
// pTask->status.pauseAllowed = 0;
}
void streamTaskEnablePause(SStreamTask* pTask) {
@ -1045,15 +1062,34 @@ void streamTaskEnablePause(SStreamTask* pTask) {
pTask->status.pauseAllowed = 1;
}
void streamTaskResumeFromHalt(SStreamTask* pTask) {
const char* id = pTask->id.idStr;
int8_t status = pTask->status.taskStatus;
if (status != TASK_STATUS__HALT) {
stError("s-task:%s not in halt status, status:%s", id, streamGetTaskStatusStr(status));
return;
int32_t updateTaskReadyInMeta(SStreamTask* pTask) {
SStreamMeta* pMeta = pTask->pMeta;
taosWLockLatch(&pMeta->lock);
STaskId id = streamTaskExtractKey(pTask);
taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0);
int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta);
if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) {
STaskStartInfo* pStartInfo = &pMeta->startInfo;
pStartInfo->readyTs = pTask->execInfo.start;
if (pStartInfo->startTs != 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",
pMeta->vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pStartInfo->startTs, pStartInfo->readyTs,
pStartInfo->elapsedTime / 1000.0);
}
pTask->status.taskStatus = pTask->status.keepTaskStatus;
pTask->status.keepTaskStatus = TASK_STATUS__NORMAL;
stDebug("s-task:%s resume from halt, current status:%s", id, streamGetTaskStatusStr(pTask->status.taskStatus));
taosWUnLockLatch(&pMeta->lock);
return TSDB_CODE_SUCCESS;
}

View File

@ -19,6 +19,7 @@
#include "tstream.h"
#include "ttimer.h"
#include "wal.h"
#include "streamsm.h"
static void streamTaskDestroyUpstreamInfo(SUpstreamInfo* pUpstreamInfo);
@ -34,8 +35,11 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory
SStreamTask* pTask = (SStreamTask*)taosMemoryCalloc(1, sizeof(SStreamTask));
if (pTask == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
stError("s-task:0x%" PRIx64 " failed malloc new stream task, size:%d, code:%s", streamId,
(int32_t)sizeof(SStreamTask), tstrerror(terrno));
return NULL;
}
pTask->ver = SSTREAM_TASK_VER;
pTask->id.taskId = tGenIdPI32();
pTask->id.streamId = streamId;
@ -43,12 +47,18 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory
pTask->info.fillHistory = fillHistory;
pTask->info.triggerParam = triggerParam;
pTask->status.pSM = streamCreateStateMachine(pTask);
if (pTask->status.pSM == NULL) {
taosMemoryFreeClear(pTask);
return NULL;
}
char buf[128] = {0};
sprintf(buf, "0x%" PRIx64 "-%d", pTask->id.streamId, pTask->id.taskId);
pTask->id.idStr = taosStrdup(buf);
pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE;
pTask->status.taskStatus = (fillHistory || hasFillhistory)? TASK_STATUS__SCAN_HISTORY:TASK_STATUS__NORMAL;
pTask->status.taskStatus = (fillHistory || hasFillhistory) ? TASK_STATUS__SCAN_HISTORY : TASK_STATUS__READY;
pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL;
pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL;
@ -296,9 +306,7 @@ void tFreeStreamTask(SStreamTask* pTask) {
int32_t taskId = pTask->id.taskId;
STaskExecStatisInfo* pStatis = &pTask->execInfo;
stDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState,
streamGetTaskStatusStr(pTask->status.taskStatus));
stDebug("start to free s-task:0x%x, %p, state:%p", taskId, pTask, pTask->pState);
stDebug("s-task:0x%x task exec summary: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64
", updateCount:%d latestUpdate:%" PRId64 ", latestCheckPoint:%" PRId64 ", ver:%" PRId64
@ -309,8 +317,8 @@ void tFreeStreamTask(SStreamTask* pTask) {
// remove the ref by timer
while (pTask->status.timerActive > 0) {
stDebug("s-task:%s wait for task stop timer activities", pTask->id.idStr);
taosMsleep(10);
stDebug("s-task:%s wait for task stop timer activities, ref:%d", pTask->id.idStr, pTask->status.timerActive);
taosMsleep(100);
}
if (pTask->schedInfo.pTimer != NULL) {
@ -384,6 +392,8 @@ void tFreeStreamTask(SStreamTask* pTask) {
pTask->pRspMsgList = NULL;
}
pTask->status.pSM = streamDestroyStateMachine(pTask->status.pSM);
streamTaskDestroyUpstreamInfo(&pTask->upstreamInfo);
pTask->msgInfo.pRetryList = taosArrayDestroy(pTask->msgInfo.pRetryList);
@ -407,6 +417,13 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i
return TSDB_CODE_OUT_OF_MEMORY;
}
pTask->status.pSM = streamCreateStateMachine(pTask);
if (pTask->status.pSM == NULL) {
stError("s-task:%s failed create state-machine for stream task, initialization failed, code:%s", pTask->id.idStr,
tstrerror(terrno));
return terrno;
}
pTask->execInfo.created = taosGetTimestampMs();
pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL;
pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL;
@ -453,7 +470,9 @@ int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask) {
return 0;
} else {
int32_t type = pTask->outputInfo.type;
if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__TABLE) {
if (type == TASK_OUTPUT__TABLE) {
return 0;
} else if (type == TASK_OUTPUT__FIXED_DISPATCH) {
return 1;
} else {
SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
@ -562,13 +581,7 @@ int32_t streamTaskStop(SStreamTask* pTask) {
int64_t st = taosGetTimestampMs();
const char* id = pTask->id.idStr;
taosThreadMutexLock(&pTask->lock);
if (pTask->status.taskStatus == TASK_STATUS__CK) {
stDebug("s-task:%s in checkpoint will be discarded since task is stopped", id);
}
pTask->status.taskStatus = TASK_STATUS__STOP;
taosThreadMutexUnlock(&pTask->lock);
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_STOP);
qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS);
while (/*pTask->status.schedStatus != TASK_SCHED_STATUS__INACTIVE */ !streamTaskIsIdle(pTask)) {
stDebug("s-task:%s level:%d wait for task to be idle and then close, check again in 100ms", id,
@ -667,7 +680,7 @@ int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask) {
return status;
}
int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask) {
int8_t streamTaskSetSchedStatusInactive(SStreamTask* pTask) {
taosThreadMutexLock(&pTask->lock);
int8_t status = pTask->status.schedStatus;
ASSERT(status == TASK_SCHED_STATUS__WAITING || status == TASK_SCHED_STATUS__ACTIVE ||
@ -678,6 +691,25 @@ int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask) {
return status;
}
int32_t streamTaskClearHTaskAttr(SStreamTask* pTask) {
SStreamMeta* pMeta = pTask->pMeta;
if (pTask->info.fillHistory == 0) {
return TSDB_CODE_SUCCESS;
}
STaskId sTaskId = {.streamId = pTask->streamTaskId.streamId, .taskId = pTask->streamTaskId.taskId};
SStreamTask** ppStreamTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &sTaskId, sizeof(sTaskId));
if (ppStreamTask != NULL) {
CLEAR_RELATED_FILLHISTORY_TASK((*ppStreamTask));
streamMetaSaveTask(pMeta, *ppStreamTask);
stDebug("s-task:%s clear the related stream task:0x%x attr to fill-history task", pTask->id.idStr,
(int32_t)sTaskId.taskId);
}
return TSDB_CODE_SUCCESS;
}
int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId) {
SVDropStreamTaskReq *pReq = rpcMallocCont(sizeof(SVDropStreamTaskReq));
if (pReq == NULL) {
@ -696,7 +728,7 @@ int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskI
return code;
}
stDebug("vgId:%d build and send drop table:0x%x msg", vgId, pTaskId->taskId);
stDebug("vgId:%d build and send drop task:0x%x msg", vgId, pTaskId->taskId);
return code;
}
@ -719,20 +751,6 @@ void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo) {
pInfo->retryTimes += 1;
}
const char* streamGetTaskStatusStr(int32_t status) {
switch(status) {
case TASK_STATUS__NORMAL: return "normal";
case TASK_STATUS__SCAN_HISTORY: return "scan-history";
case TASK_STATUS__HALT: return "halt";
case TASK_STATUS__PAUSE: return "paused";
case TASK_STATUS__CK: return "check-point";
case TASK_STATUS__DROPPING: return "dropping";
case TASK_STATUS__STOP: return "stop";
case TASK_STATUS__UNINIT: return "uninitialized";
default:return "";
}
}
void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask) {
pEntry->id.streamId = pTask->id.streamId;
pEntry->id.taskId = pTask->id.taskId;

View File

@ -0,0 +1,497 @@
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include <streamsm.h>
#include "streamInt.h"
#include "streamsm.h"
#include "tmisce.h"
#include "tstream.h"
#include "ttimer.h"
#include "wal.h"
SStreamTaskState StreamTaskStatusList[9] = {
{.state = TASK_STATUS__READY, .name = "ready"},
{.state = TASK_STATUS__DROPPING, .name = "dropped"},
{.state = TASK_STATUS__UNINIT, .name = "uninit"},
{.state = TASK_STATUS__STOP, .name = "stop"},
{.state = TASK_STATUS__SCAN_HISTORY, .name = "scan-history"},
{.state = TASK_STATUS__HALT, .name = "halt"},
{.state = TASK_STATUS__PAUSE, .name = "paused"},
{.state = TASK_STATUS__CK, .name = "checkpoint"},
{.state = TASK_STATUS__STREAM_SCAN_HISTORY, .name = "stream-scan-history"},
};
SStreamEventInfo StreamTaskEventList[12] = {
{.event = 0, .name = ""}, // dummy event, place holder
{.event = TASK_EVENT_INIT, .name = "initialize"},
{.event = TASK_EVENT_INIT_SCANHIST, .name = "scan-history-init"},
{.event = TASK_EVENT_INIT_STREAM_SCANHIST, .name = "stream-scan-history-init"},
{.event = TASK_EVENT_SCANHIST_DONE, .name = "scan-history-completed"},
{.event = TASK_EVENT_STOP, .name = "stopping"},
{.event = TASK_EVENT_GEN_CHECKPOINT, .name = "checkpoint"},
{.event = TASK_EVENT_CHECKPOINT_DONE, .name = "checkpoint-done"},
{.event = TASK_EVENT_PAUSE, .name = "pausing"},
{.event = TASK_EVENT_RESUME, .name = "resuming"},
{.event = TASK_EVENT_HALT, .name = "halting"},
{.event = TASK_EVENT_DROPPING, .name = "dropping"},
};
static TdThreadOnce streamTaskStateMachineInit = PTHREAD_ONCE_INIT;
static SArray* streamTaskSMTrans = NULL;
static int32_t streamTaskInitStatus(SStreamTask* pTask);
static int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask);
static int32_t initStateTransferTable();
static void doInitStateTransferTable(void);
static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event,
__state_trans_fn fn, __state_trans_succ_fn succFn,
SAttachedEventInfo* pEventInfo, bool autoInvoke);
static int32_t dummyFn(SStreamTask* UNUSED_PARAM(p)) { return TSDB_CODE_SUCCESS; }
static int32_t attachEvent(SStreamTask* pTask, SAttachedEventInfo* pEvtInfo) {
char* p = NULL;
streamTaskGetStatus(pTask, &p);
stDebug("s-task:%s status:%s attach event:%s required status:%s, since not allowed to handle it", pTask->id.idStr, p,
StreamTaskEventList[pEvtInfo->event].name, StreamTaskStatusList[pEvtInfo->status].name);
taosArrayPush(pTask->status.pSM->pWaitingEventList, pEvtInfo);
return 0;
}
int32_t streamTaskInitStatus(SStreamTask* pTask) {
pTask->execInfo.init = taosGetTimestampMs();
stDebug("s-task:%s start init, and check downstream tasks, set the init ts:%" PRId64, pTask->id.idStr,
pTask->execInfo.init);
streamTaskCheckDownstream(pTask);
return 0;
}
int32_t streamTaskSetReadyForWal(SStreamTask* pTask) {
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
stDebug("s-task:%s ready for extract data from wal", pTask->id.idStr);
}
return TSDB_CODE_SUCCESS;
}
static int32_t streamTaskDoCheckpoint(SStreamTask* pTask) {
stDebug("s-task:%s start to do checkpoint", pTask->id.idStr);
return 0;
}
int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask) {
ASSERT(HAS_RELATED_FILLHISTORY_TASK(pTask));
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
pTask->hTaskInfo.haltVer = walReaderGetCurrentVer(pTask->exec.pWalReader);
if (pTask->hTaskInfo.haltVer == -1) {
pTask->hTaskInfo.haltVer = pTask->dataRange.range.maxVer + 1;
}
}
return TSDB_CODE_SUCCESS;
}
// todo optimize the perf of find the trans objs by using hash table
static STaskStateTrans* streamTaskFindTransform(const SStreamTaskSM* pState, const EStreamTaskEvent event) {
int32_t numOfTrans = taosArrayGetSize(streamTaskSMTrans);
for (int32_t i = 0; i < numOfTrans; ++i) {
STaskStateTrans* pTrans = taosArrayGet(streamTaskSMTrans, i);
if (pTrans->state.state == pState->current.state && pTrans->event == event) {
return pTrans;
}
}
if (event == TASK_EVENT_CHECKPOINT_DONE && pState->current.state == TASK_STATUS__STOP) {
} else {
ASSERT(0);
}
return NULL;
}
void streamTaskRestoreStatus(SStreamTask* pTask) {
SStreamTaskSM* pSM = pTask->status.pSM;
taosThreadMutexLock(&pTask->lock);
ASSERT(pSM->pActiveTrans == NULL);
ASSERT(pSM->current.state == TASK_STATUS__PAUSE || pSM->current.state == TASK_STATUS__HALT);
SStreamTaskState state = pSM->current;
pSM->current = pSM->prev.state;
pSM->prev.state = state;
pSM->prev.evt = 0;
pSM->startTs = taosGetTimestampMs();
taosThreadMutexUnlock(&pTask->lock);
stDebug("s-task:%s restore status, %s -> %s", pTask->id.idStr, pSM->prev.state.name, pSM->current.name);
}
SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) {
initStateTransferTable();
const char* id = pTask->id.idStr;
SStreamTaskSM* pSM = taosMemoryCalloc(1, sizeof(SStreamTaskSM));
if (pSM == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
stError("s-task:%s failed to create task stateMachine, size:%d, code:%s", id, (int32_t)sizeof(SStreamTaskSM),
tstrerror(terrno));
return NULL;
}
pSM->pTask = pTask;
pSM->pWaitingEventList = taosArrayInit(4, sizeof(SAttachedEventInfo));
if (pSM->pWaitingEventList == NULL) {
taosMemoryFree(pSM);
terrno = TSDB_CODE_OUT_OF_MEMORY;
stError("s-task:%s failed to create task stateMachine, size:%d, code:%s", id, (int32_t)sizeof(SStreamTaskSM),
tstrerror(terrno));
return NULL;
}
// set the initial state for the state-machine of stream task
pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT];
pSM->startTs = taosGetTimestampMs();
return pSM;
}
void* streamDestroyStateMachine(SStreamTaskSM* pSM) {
if (pSM == NULL) {
return NULL;
}
taosArrayDestroy(pSM->pWaitingEventList);
taosMemoryFree(pSM);
return NULL;
}
int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) {
SStreamTask* pTask = pSM->pTask;
taosThreadMutexLock(&pTask->lock);
STaskStateTrans* pTrans = streamTaskFindTransform(pSM, event);
if (pTrans == NULL) {
stWarn("s-task:%s status:%s not allowed handle event:%s", pTask->id.idStr, pSM->current.name, StreamTaskEventList[event].name);
return -1;
} else {
stDebug("s-task:%s start to handle event:%s, state:%s", pTask->id.idStr, StreamTaskEventList[event].name,
pSM->current.name);
}
if (pTrans->attachEvent.event != 0) {
attachEvent(pTask, &pTrans->attachEvent);
taosThreadMutexUnlock(&pTask->lock);
while (1) {
// wait for the task to be here
taosThreadMutexLock(&pTask->lock);
ETaskStatus s = streamTaskGetStatus(pTask, NULL);
taosThreadMutexUnlock(&pTask->lock);
if ((s == pTrans->next.state) && (pSM->prev.evt == pTrans->event)) {
stDebug("s-task:%s attached event:%s handled", pTask->id.idStr, StreamTaskEventList[pTrans->event].name);
return TSDB_CODE_SUCCESS;
} else {// this event has been handled already
stDebug("s-task:%s not handle event:%s yet, wait for 100ms and recheck", pTask->id.idStr,
StreamTaskEventList[event].name);
taosMsleep(100);
}
}
} else {
if (pSM->pActiveTrans != NULL) {
ASSERT(!pSM->pActiveTrans->autoInvokeEndFn);
stWarn("s-task:%s status:%s handle event:%s is interrupted, handle the new event:%s", pTask->id.idStr,
pSM->current.name, StreamTaskEventList[pSM->pActiveTrans->event].name, StreamTaskEventList[event].name);
}
pSM->pActiveTrans = pTrans;
pSM->startTs = taosGetTimestampMs();
taosThreadMutexUnlock(&pTask->lock);
int32_t code = pTrans->pAction(pTask);
// todo handle error code;
if (pTrans->autoInvokeEndFn) {
streamTaskOnHandleEventSuccess(pSM);
}
}
return TSDB_CODE_SUCCESS;
}
static void keepPrevInfo(SStreamTaskSM* pSM) {
STaskStateTrans* pTrans = pSM->pActiveTrans;
pSM->prev.state = pSM->current;
pSM->prev.evt = pTrans->event;
}
int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) {
SStreamTask* pTask = pSM->pTask;
// do update the task status
taosThreadMutexLock(&pTask->lock);
STaskStateTrans* pTrans = pSM->pActiveTrans;
if (pTrans == NULL) {
ETaskStatus s = pSM->current.state;
ASSERT(s == TASK_STATUS__DROPPING || s == TASK_STATUS__PAUSE || s == TASK_STATUS__STOP);
// the pSM->prev.evt may be 0, so print string is not appropriate.
stDebug("status not handled success, current status:%s, trigger event:%d, %s", pSM->current.name, pSM->prev.evt,
pTask->id.idStr);
taosThreadMutexUnlock(&pTask->lock);
return TSDB_CODE_INVALID_PARA;
}
keepPrevInfo(pSM);
pSM->current = pTrans->next;
pSM->pActiveTrans = NULL;
// on success callback, add into lock if necessary, or maybe we should add an option for this?
pTrans->pSuccAction(pTask);
if (taosArrayGetSize(pSM->pWaitingEventList) > 0) {
int64_t el = (taosGetTimestampMs() - pSM->startTs);
stDebug("s-task:%s handle event:%s completed, elapsed time:%" PRId64 "ms state:%s -> %s", pTask->id.idStr,
StreamTaskEventList[pTrans->event].name, el, pSM->prev.state.name, pSM->current.name);
SAttachedEventInfo* pEvtInfo = taosArrayPop(pSM->pWaitingEventList);
// OK, let's handle the attached event, since the task has reached the required status now
if (pSM->current.state == pEvtInfo->status) {
stDebug("s-task:%s handle the attached event:%s, state:%s", pTask->id.idStr,
StreamTaskEventList[pEvtInfo->event].name, pSM->current.name);
STaskStateTrans* pNextTrans = streamTaskFindTransform(pSM, pEvtInfo->event);
ASSERT(pSM->pActiveTrans == NULL && pNextTrans != NULL);
pSM->pActiveTrans = pNextTrans;
pSM->startTs = taosGetTimestampMs();
taosThreadMutexUnlock(&pTask->lock);
int32_t code = pNextTrans->pAction(pSM->pTask);
if (pNextTrans->autoInvokeEndFn) {
return streamTaskOnHandleEventSuccess(pSM);
} else {
return code;
}
}
} else {
taosThreadMutexUnlock(&pTask->lock);
int64_t el = (taosGetTimestampMs() - pSM->startTs);
stDebug("s-task:%s handle event:%s completed, elapsed time:%" PRId64 "ms state:%s -> %s", pTask->id.idStr,
StreamTaskEventList[pTrans->event].name, el, pSM->prev.state.name, pSM->current.name);
}
return TSDB_CODE_SUCCESS;
}
ETaskStatus streamTaskGetStatus(const SStreamTask* pTask, char** pStr) {
SStreamTaskState s = pTask->status.pSM->current; // copy one obj in case of multi-thread environment
if (pStr != NULL) {
*pStr = s.name;
}
return s.state;
}
const char* streamTaskGetStatusStr(ETaskStatus status) {
return StreamTaskStatusList[status].name;
}
void streamTaskResetStatus(SStreamTask* pTask) {
SStreamTaskSM* pSM = pTask->status.pSM;
taosThreadMutexLock(&pTask->lock);
pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT];
pSM->pActiveTrans = NULL;
taosArrayClear(pSM->pWaitingEventList);
taosThreadMutexUnlock(&pTask->lock);
// clear the downstream ready status
pTask->status.downstreamReady = 0;
}
void streamTaskSetStatusReady(SStreamTask* pTask) {
SStreamTaskSM* pSM = pTask->status.pSM;
if (pSM->current.state == TASK_STATUS__DROPPING) {
stError("s-task:%s task in dropping state, cannot be set ready", pTask->id.idStr);
return;
}
taosThreadMutexLock(&pTask->lock);
pSM->prev.state = pSM->current;
pSM->prev.evt = 0;
pSM->current = StreamTaskStatusList[TASK_STATUS__READY];
pSM->startTs = taosGetTimestampMs();
pSM->pActiveTrans = NULL;
taosArrayClear(pSM->pWaitingEventList);
taosThreadMutexUnlock(&pTask->lock);
}
STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn,
__state_trans_succ_fn succFn, SAttachedEventInfo* pEventInfo, bool autoInvoke) {
STaskStateTrans trans = {0};
trans.state = StreamTaskStatusList[current];
trans.next = StreamTaskStatusList[next];
trans.event = event;
if (pEventInfo != NULL) {
trans.attachEvent = *pEventInfo;
} else {
trans.attachEvent.event = 0;
trans.attachEvent.status = 0;
}
trans.pAction = (fn != NULL) ? fn : dummyFn;
trans.pSuccAction = (succFn != NULL) ? succFn : dummyFn;
trans.autoInvokeEndFn = autoInvoke;
return trans;
}
int32_t initStateTransferTable() {
taosThreadOnce(&streamTaskStateMachineInit, doInitStateTransferTable);
return TSDB_CODE_SUCCESS;
}
void doInitStateTransferTable(void) {
streamTaskSMTrans = taosArrayInit(8, sizeof(STaskStateTrans));
// initialization event handle
STaskStateTrans trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__READY, TASK_EVENT_INIT,
streamTaskInitStatus, onNormalTaskReady, false, false);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__SCAN_HISTORY, TASK_EVENT_INIT_SCANHIST,
streamTaskInitStatus, onScanhistoryTaskReady, false, false);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__STREAM_SCAN_HISTORY, TASK_EVENT_INIT_STREAM_SCANHIST,
streamTaskInitStatus, onScanhistoryTaskReady, false, false);
taosArrayPush(streamTaskSMTrans, &trans);
// scan-history related event
trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE,
streamTaskSetReadyForWal, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE,
streamTaskSetReadyForWal, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
// halt stream task, from other task status
trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL,
streamTaskKeepCurrentVerInWal, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
SAttachedEventInfo info = {.status = TASK_STATUS__READY, .event = TASK_EVENT_HALT};
trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL,
streamTaskKeepCurrentVerInWal, &info, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal,
&info, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL,
streamTaskKeepCurrentVerInWal, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
// checkpoint related event
trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__CK, TASK_EVENT_GEN_CHECKPOINT, NULL,
streamTaskDoCheckpoint, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans =
createStateTransform(TASK_STATUS__CK, TASK_STATUS__READY, TASK_EVENT_CHECKPOINT_DONE, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
// pause & resume related event handle
trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
info = (SAttachedEventInfo){.status = TASK_STATUS__READY, .event = TASK_EVENT_PAUSE};
trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__STOP, TASK_EVENT_PAUSE, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__DROPPING, TASK_EVENT_PAUSE, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
// resume is completed by restore status of state-machine
// stop related event
trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
// dropping related event
trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true);
taosArrayPush(streamTaskSMTrans, &trans);
}

View File

@ -191,7 +191,7 @@ void* taosArrayGet(const SArray* pArray, size_t index) {
}
if (index >= pArray->size) {
uError("index is out of range, current:%" PRIzu " max:%d", index, pArray->capacity);
uError("index is out of range, current:%" PRIzu " max:%"PRIzu, index, pArray->size);
return NULL;
}

View File

@ -242,6 +242,11 @@ int32_t taosReadAllQitems(STaosQueue *queue, STaosQall *qall) {
qall->current = queue->head;
qall->start = queue->head;
qall->numOfItems = queue->numOfItems;
qall->memOfItems = queue->memOfItems;
qall->unAccessedNumOfItems = queue->numOfItems;
qall->unAccessMemOfItems = queue->memOfItems;
numOfItems = qall->numOfItems;
queue->head = NULL;
@ -274,6 +279,10 @@ int32_t taosGetQitem(STaosQall *qall, void **ppItem) {
if (pNode) {
*ppItem = pNode->item;
num = 1;
qall->unAccessedNumOfItems -= 1;
qall->unAccessMemOfItems -= pNode->dataSize;
uTrace("item:%p is fetched", *ppItem);
} else {
*ppItem = NULL;
@ -449,6 +458,8 @@ int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, SQueueInfo *
qall->current = queue->head;
qall->start = queue->head;
qall->numOfItems = queue->numOfItems;
qall->memOfItems = queue->memOfItems;
code = qall->numOfItems;
qinfo->ahandle = queue->ahandle;
qinfo->fp = queue->itemsFp;
@ -476,6 +487,11 @@ int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, SQueueInfo *
}
int32_t taosQallItemSize(STaosQall *qall) { return qall->numOfItems; }
int64_t taosQallMemSize(STaosQall *qall) { return qall->memOfItems; }
int64_t taosQallUnAccessedItemSize(STaosQall *qall) {return qall->unAccessedNumOfItems;}
int64_t taosQallUnAccessedMemSize(STaosQall *qall) {return qall->unAccessMemOfItems;}
void taosResetQitems(STaosQall *qall) { qall->current = qall->start; }
int32_t taosGetQueueNumber(STaosQset *qset) { return qset->numOfQueues; }

View File

@ -135,6 +135,8 @@ class TDDnode:
"vDebugFlag": "143",
"tqDebugFlag": "143",
"cDebugFlag": "143",
"stDebugFlag": "143",
"smaDebugFlag": "143",
"jniDebugFlag": "143",
"qDebugFlag": "143",
"rpcDebugFlag": "143",

View File

@ -267,6 +267,7 @@ sql create table t1 using st tags(1,1,1);
sql create table t2 using st tags(2,2,2);
sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 delete_mark 20s into streamt1 as select _wstart as c0, count(*) c1, count(a) c2 from st interval(10s) ;
sleep 1000
sql insert into t1 values(1648791211000,1,2,3);
sql insert into t1 values(1262275200000,2,2,3);

View File

@ -23,7 +23,8 @@ from util.cases import *
from util.sql import *
class TDTestCase:
updatecfgDict = {'vdebugFlag': 143, 'qdebugflag':135, 'tqdebugflag':135, 'udebugflag':135, 'rpcdebugflag':135,
'asynclog': 0, 'stdebugflag':135}
# random string
def random_string(self, count):
letters = string.ascii_letters

View File

@ -7,7 +7,7 @@ from util.common import *
class TDTestCase:
updatecfgDict = {'vdebugFlag': 143, 'qdebugflag':135, 'tqdebugflag':135, 'udebugflag':135, 'rpcdebugflag':135,
'asynclog': 0}
'asynclog': 0, 'stdebugflag':135}
def init(self, conn, logSql, replicaVar=1):
self.replicaVar = int(replicaVar)
tdLog.debug("start to execute %s" % __file__)

View File

@ -398,7 +398,7 @@ if __name__ == "__main__":
tdDnodes.setValgrind(valgrind)
tdDnodes.stopAll()
for dnode in tdDnodes.dnodes:
tdDnodes.deploy(dnode.index,{})
tdDnodes.deploy(dnode.index, updateCfgDict)
for dnode in tdDnodes.dnodes:
tdDnodes.starttaosd(dnode.index)
tdCases.logSql(logSql)