Merge pull request #24356 from taosdata/fix/3_liaohj
enh(stream): remove sleep to opt perf and avoid deadlock
This commit is contained in:
commit
5866fc2919
|
@ -18,6 +18,7 @@
|
|||
|
||||
// message process
|
||||
int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart);
|
||||
int32_t tqStreamOneTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId);
|
||||
int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored);
|
||||
int32_t tqStreamTaskProcessDispatchReq(SStreamMeta* pMeta, SRpcMsg* pMsg);
|
||||
int32_t tqStreamTaskProcessDispatchRsp(SStreamMeta* pMeta, SRpcMsg* pMsg);
|
||||
|
@ -27,10 +28,11 @@ int32_t tqStreamTaskProcessScanHistoryFinishRsp(SStreamMeta* pMeta, SRpcMsg* pMs
|
|||
int32_t tqStreamTaskProcessCheckReq(SStreamMeta* pMeta, SRpcMsg* pMsg);
|
||||
int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader);
|
||||
int32_t tqStreamTaskProcessCheckpointReadyMsg(SStreamMeta* pMeta, SRpcMsg* pMsg);
|
||||
int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, int64_t sversion, char* msg, int32_t msgLen, bool isLeader, bool restored);
|
||||
int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sversion, char* msg, int32_t msgLen,
|
||||
bool isLeader, bool restored);
|
||||
int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen);
|
||||
int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader);
|
||||
int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta);
|
||||
int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta, int32_t* numOfTasks);
|
||||
int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta);
|
||||
int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg);
|
||||
int32_t tqStreamTaskProcessTaskPauseReq(SStreamMeta* pMeta, char* pMsg);
|
||||
|
|
|
@ -210,6 +210,7 @@ void* qExtractReaderFromStreamScanner(void* scanner);
|
|||
int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner);
|
||||
|
||||
int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo);
|
||||
int32_t qResetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo);
|
||||
int32_t qStreamSourceScanParamForHistoryScanStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow);
|
||||
int32_t qStreamSourceScanParamForHistoryScanStep2(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow);
|
||||
int32_t qStreamRecoverFinish(qTaskInfo_t tinfo);
|
||||
|
|
|
@ -50,10 +50,13 @@ extern "C" {
|
|||
(_t)->hTaskInfo.id.streamId = 0; \
|
||||
} while (0)
|
||||
|
||||
#define STREAM_EXEC_EXTRACT_DATA_IN_WAL_ID (-1)
|
||||
#define STREAM_EXEC_START_ALL_TASKS_ID (-2)
|
||||
#define STREAM_EXEC_RESTART_ALL_TASKS_ID (-3)
|
||||
#define STREAM_EXEC_STOP_ALL_TASKS_ID (-4)
|
||||
#define STREAM_EXEC_T_EXTRACT_WAL_DATA (-1)
|
||||
#define STREAM_EXEC_T_START_ALL_TASKS (-2)
|
||||
#define STREAM_EXEC_T_START_ONE_TASK (-3)
|
||||
#define STREAM_EXEC_T_RESTART_ALL_TASKS (-4)
|
||||
#define STREAM_EXEC_T_STOP_ALL_TASKS (-5)
|
||||
#define STREAM_EXEC_T_RESUME_TASK (-6)
|
||||
#define STREAM_EXEC_T_UPDATE_TASK_EPSET (-7)
|
||||
|
||||
typedef struct SStreamTask SStreamTask;
|
||||
typedef struct SStreamQueue SStreamQueue;
|
||||
|
@ -81,14 +84,12 @@ 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 {
|
||||
TASK_SCHED_STATUS__INACTIVE = 1,
|
||||
TASK_SCHED_STATUS__WAITING,
|
||||
TASK_SCHED_STATUS__ACTIVE,
|
||||
TASK_SCHED_STATUS__FAILED,
|
||||
TASK_SCHED_STATUS__DROPPING,
|
||||
};
|
||||
|
||||
|
@ -322,10 +323,11 @@ 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
|
||||
int32_t schedIdleTime; // idle time before invoke again
|
||||
int64_t lastExecTs; // last exec time stamp
|
||||
int8_t statusBackup;
|
||||
bool appendTranstateBlock; // has append the transfer state data block already
|
||||
int32_t timerActive; // timer is active
|
||||
int8_t allowedAddInTimer; // allowed to add into timer
|
||||
int32_t inScanHistorySentinel;
|
||||
} SStreamStatus;
|
||||
|
||||
|
@ -366,7 +368,8 @@ typedef struct STaskQueue {
|
|||
|
||||
typedef struct STaskSchedInfo {
|
||||
int8_t status;
|
||||
void* pTimer;
|
||||
tmr_h pDelayTimer;
|
||||
tmr_h pIdleTimer;
|
||||
} STaskSchedInfo;
|
||||
|
||||
typedef struct SSinkRecorder {
|
||||
|
@ -481,6 +484,11 @@ typedef struct STaskUpdateInfo {
|
|||
int32_t transId;
|
||||
} STaskUpdateInfo;
|
||||
|
||||
typedef struct SScanWalInfo {
|
||||
int32_t scanCounter;
|
||||
tmr_h scanTimer;
|
||||
} SScanWalInfo;
|
||||
|
||||
// meta
|
||||
typedef struct SStreamMeta {
|
||||
char* path;
|
||||
|
@ -498,7 +506,7 @@ typedef struct SStreamMeta {
|
|||
bool sendMsgBeforeClosing; // send hb to mnode before close all tasks when switch to follower.
|
||||
STaskStartInfo startInfo;
|
||||
TdThreadRwlock lock;
|
||||
int32_t walScanCounter;
|
||||
SScanWalInfo scanInfo;
|
||||
void* streamBackend;
|
||||
int64_t streamBackendRid;
|
||||
SHashObj* pTaskDbUnique;
|
||||
|
@ -541,6 +549,7 @@ typedef struct {
|
|||
SMsgHead head;
|
||||
int64_t streamId;
|
||||
int32_t taskId;
|
||||
int32_t reqType;
|
||||
} SStreamTaskRunReq;
|
||||
|
||||
struct SStreamDispatchReq {
|
||||
|
@ -728,6 +737,11 @@ typedef struct SStreamTaskNodeUpdateMsg {
|
|||
int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpdateMsg* pMsg);
|
||||
int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* pMsg);
|
||||
|
||||
typedef struct SStreamTaskState {
|
||||
ETaskStatus state;
|
||||
char* name;
|
||||
} SStreamTaskState;
|
||||
|
||||
typedef struct {
|
||||
int64_t streamId;
|
||||
int32_t downstreamTaskId;
|
||||
|
@ -764,17 +778,18 @@ SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t
|
|||
|
||||
void streamTaskInputFail(SStreamTask* pTask);
|
||||
int32_t streamExecTask(SStreamTask* pTask);
|
||||
int32_t streamResumeTask(SStreamTask* pTask);
|
||||
int32_t streamSchedExec(SStreamTask* pTask);
|
||||
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);
|
||||
char* createStreamTaskIdStr(int64_t streamId, int32_t taskId);
|
||||
SStreamTaskState* streamTaskGetStatus(const SStreamTask* pTask);
|
||||
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);
|
||||
|
||||
|
@ -810,6 +825,7 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue);
|
|||
|
||||
// common
|
||||
int32_t streamRestoreParam(SStreamTask* pTask);
|
||||
int32_t streamResetParamForScanHistory(SStreamTask* pTask);
|
||||
void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta);
|
||||
void streamTaskResume(SStreamTask* pTask);
|
||||
int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask);
|
||||
|
@ -867,6 +883,7 @@ void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool i
|
|||
int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta);
|
||||
int32_t streamMetaStartAllTasks(SStreamMeta* pMeta);
|
||||
int32_t streamMetaStopAllTasks(SStreamMeta* pMeta);
|
||||
int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId);
|
||||
|
||||
// checkpoint
|
||||
int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq);
|
||||
|
|
|
@ -939,8 +939,8 @@ TEST(clientCase, agg_query_tables) {
|
|||
}
|
||||
taos_free_result(pRes);
|
||||
|
||||
int64_t st = 1685959293000;
|
||||
for (int32_t i = 0; i < 10000000; ++i) {
|
||||
int64_t st = 1685959293299;
|
||||
for (int32_t i = 0; i < 5; ++i) {
|
||||
char s[256] = {0};
|
||||
|
||||
while (1) {
|
||||
|
@ -954,16 +954,16 @@ TEST(clientCase, agg_query_tables) {
|
|||
}
|
||||
}
|
||||
|
||||
while (1) {
|
||||
sprintf(s, "insert into t2 values(%ld, %d)", st + i, i);
|
||||
pRes = taos_query(pConn, s);
|
||||
int32_t ret = taos_errno(pRes);
|
||||
|
||||
taos_free_result(pRes);
|
||||
if (ret == 0) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
// while (1) {
|
||||
// sprintf(s, "insert into t2 values(%ld, %d)", st + i, i);
|
||||
// pRes = taos_query(pConn, s);
|
||||
// int32_t ret = taos_errno(pRes);
|
||||
//
|
||||
// taos_free_result(pRes);
|
||||
// if (ret == 0) {
|
||||
// break;
|
||||
// }
|
||||
// }
|
||||
}
|
||||
|
||||
// pRes = taos_query(pConn, "show table distributed tup");
|
||||
|
|
|
@ -28,9 +28,6 @@
|
|||
} \
|
||||
} while (0)
|
||||
|
||||
extern int32_t streamTimerInit();
|
||||
extern void streamTimerCleanUp();
|
||||
|
||||
static SDnode globalDnode = {0};
|
||||
|
||||
SDnode *dmInstance() { return &globalDnode; }
|
||||
|
@ -169,7 +166,6 @@ int32_t dmInit() {
|
|||
#if defined(USE_S3)
|
||||
if (s3Begin() != 0) return -1;
|
||||
#endif
|
||||
if (streamTimerInit() != 0) return -1;
|
||||
|
||||
dInfo("dnode env is initialized");
|
||||
return 0;
|
||||
|
@ -196,10 +192,10 @@ void dmCleanup() {
|
|||
udfStopUdfd();
|
||||
taosStopCacheRefreshWorker();
|
||||
dmDiskClose();
|
||||
|
||||
#if defined(USE_S3)
|
||||
s3End();
|
||||
#endif
|
||||
streamTimerCleanUp();
|
||||
|
||||
dInfo("dnode env is cleaned up");
|
||||
|
||||
|
|
|
@ -90,6 +90,7 @@ int32_t dmInitDnode(SDnode *pDnode) {
|
|||
goto _OVER;
|
||||
}
|
||||
#endif
|
||||
|
||||
indexInit(tsNumOfCommitThreads);
|
||||
streamMetaInit();
|
||||
|
||||
|
@ -108,7 +109,9 @@ _OVER:
|
|||
}
|
||||
|
||||
void dmCleanupDnode(SDnode *pDnode) {
|
||||
if (pDnode == NULL) return;
|
||||
if (pDnode == NULL) {
|
||||
return;
|
||||
}
|
||||
|
||||
dmCleanupClient(pDnode);
|
||||
dmCleanupStatusClient(pDnode);
|
||||
|
|
|
@ -71,7 +71,8 @@ int32_t mndPersistStream(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream
|
|||
|
||||
int32_t mndStreamRegisterTrans(STrans* pTrans, const char* pTransName, int64_t streamUid);
|
||||
int32_t mndAddtoCheckpointWaitingList(SStreamObj *pStream, int64_t checkpointId);
|
||||
bool streamTransConflictOtherTrans(SMnode *pMnode, int64_t streamUid, const char *pTransName, bool lock);
|
||||
bool mndStreamTransConflictCheck(SMnode *pMnode, int64_t streamUid, const char *pTransName, bool lock);
|
||||
int32_t mndStreamGetRelCheckpointTrans(SMnode *pMnode, int64_t streamUid);
|
||||
|
||||
// for sma
|
||||
// TODO refactor
|
||||
|
|
|
@ -80,7 +80,9 @@ static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *p
|
|||
static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode);
|
||||
static void saveStreamTasksInfo(SStreamObj *pStream, SStreamExecInfo *pExecNode);
|
||||
static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot);
|
||||
static int32_t killActiveCheckpointTrans(SMnode *pMnode, const char *pDbName, size_t len);
|
||||
static int32_t doKillCheckpointTrans(SMnode *pMnode, const char *pDbName, size_t len);
|
||||
static void killCheckpointTransImpl(SMnode *pMnode, int32_t transId, const char *pDbName);
|
||||
|
||||
static int32_t setNodeEpsetExpiredFlag(const SArray *pNodeList);
|
||||
static void freeCheckpointCandEntry(void *);
|
||||
|
||||
|
@ -1047,7 +1049,7 @@ static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStre
|
|||
return -1;
|
||||
}
|
||||
|
||||
bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_CHECKPOINT_NAME, true);
|
||||
bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_CHECKPOINT_NAME, true);
|
||||
if (conflict) {
|
||||
mndAddtoCheckpointWaitingList(pStream, checkpointId);
|
||||
mWarn("checkpoint conflict with other trans in %s, ignore the checkpoint for stream:%s %" PRIx64, pStream->sourceDb,
|
||||
|
@ -1344,6 +1346,7 @@ static int32_t mndProcessStreamCheckpointInCandid(SRpcMsg *pReq) {
|
|||
if (ps == NULL) {
|
||||
continue;
|
||||
}
|
||||
|
||||
mDebug("start to launch checkpoint for stream:%s %" PRIx64 " in candidate list", pEntry->pName, pEntry->streamId);
|
||||
|
||||
code = mndProcessStreamCheckpointTrans(pMnode, ps, pEntry->checkpointId);
|
||||
|
@ -1399,7 +1402,7 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) {
|
|||
}
|
||||
|
||||
// check if it is conflict with other trans in both sourceDb and targetDb.
|
||||
bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_DROP_NAME, true);
|
||||
bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_DROP_NAME, true);
|
||||
if (conflict) {
|
||||
sdbRelease(pMnode->pSdb, pStream);
|
||||
tFreeMDropStreamReq(&dropReq);
|
||||
|
@ -1451,6 +1454,12 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) {
|
|||
return -1;
|
||||
}
|
||||
|
||||
// kill the related checkpoint trans
|
||||
int32_t transId = mndStreamGetRelCheckpointTrans(pMnode, pStream->uid);
|
||||
if (transId != 0) {
|
||||
killCheckpointTransImpl(pMnode, transId, pStream->sourceDb);
|
||||
}
|
||||
|
||||
removeStreamTasksInBuf(pStream, &execInfo);
|
||||
|
||||
SName name = {0};
|
||||
|
@ -1491,6 +1500,16 @@ int32_t mndDropStreamByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb) {
|
|||
return -1;
|
||||
}
|
||||
#endif
|
||||
|
||||
// kill the related checkpoint trans
|
||||
int32_t transId = mndStreamGetRelCheckpointTrans(pMnode, pStream->uid);
|
||||
if (transId != 0) {
|
||||
killCheckpointTransImpl(pMnode, transId, pStream->sourceDb);
|
||||
}
|
||||
|
||||
// drop the stream obj in execInfo
|
||||
removeStreamTasksInBuf(pStream, &execInfo);
|
||||
|
||||
if (mndPersistDropStreamLog(pMnode, pTrans, pStream) < 0) {
|
||||
sdbRelease(pSdb, pStream);
|
||||
sdbCancelFetch(pSdb, pIter);
|
||||
|
@ -1623,10 +1642,18 @@ static void mndCancelGetNextStream(SMnode *pMnode, void *pIter) {
|
|||
sdbCancelFetch(pSdb, pIter);
|
||||
}
|
||||
|
||||
static void setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SSDataBlock *pBlock, int32_t numOfRows) {
|
||||
static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SSDataBlock *pBlock, int32_t numOfRows) {
|
||||
SColumnInfoData *pColInfo;
|
||||
int32_t cols = 0;
|
||||
|
||||
STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId};
|
||||
|
||||
STaskStatusEntry *pe = taosHashGet(execInfo.pTaskMap, &id, sizeof(id));
|
||||
if (pe == NULL) {
|
||||
mError("task:0x%" PRIx64 " not exists in vnode, no valid status/stage info", id.taskId);
|
||||
return -1;
|
||||
}
|
||||
|
||||
// stream name
|
||||
char streamName[TSDB_TABLE_NAME_LEN + VARSTR_HEADER_SIZE] = {0};
|
||||
STR_WITH_MAXSIZE_TO_VARSTR(streamName, mndGetDbStr(pStream->name), sizeof(streamName));
|
||||
|
@ -1677,14 +1704,7 @@ static void setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SSDat
|
|||
colDataSetVal(pColInfo, numOfRows, (const char *)level, false);
|
||||
|
||||
// status
|
||||
char status[20 + VARSTR_HEADER_SIZE] = {0};
|
||||
STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId};
|
||||
|
||||
STaskStatusEntry *pe = taosHashGet(execInfo.pTaskMap, &id, sizeof(id));
|
||||
if (pe == NULL) {
|
||||
mError("task:0x%" PRIx64 " not exists in vnode, no valid status/stage info", id.taskId);
|
||||
return;
|
||||
}
|
||||
char status[20 + VARSTR_HEADER_SIZE] = {0};
|
||||
|
||||
const char *pStatus = streamTaskGetStatusStr(pe->status);
|
||||
STR_TO_VARSTR(status, pStatus);
|
||||
|
@ -1727,6 +1747,8 @@ static void setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SSDat
|
|||
|
||||
pColInfo = taosArrayGet(pBlock->pDataBlock, cols++);
|
||||
colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false);
|
||||
|
||||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
|
||||
static int32_t getNumOfTasks(SArray *pTaskList) {
|
||||
|
@ -1768,8 +1790,10 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock
|
|||
int32_t numOfLevels = taosArrayGetSize(pLevel);
|
||||
for (int32_t j = 0; j < numOfLevels; j++) {
|
||||
SStreamTask *pTask = taosArrayGetP(pLevel, j);
|
||||
setTaskAttrInResBlock(pStream, pTask, pBlock, numOfRows);
|
||||
numOfRows++;
|
||||
int32_t code = setTaskAttrInResBlock(pStream, pTask, pBlock, numOfRows);
|
||||
if (code == TSDB_CODE_SUCCESS) {
|
||||
numOfRows++;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1838,7 +1862,7 @@ int32_t mndPauseAllStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStre
|
|||
}
|
||||
|
||||
if (atomic_load_8(&pTask->status.taskStatus) != TASK_STATUS__PAUSE) {
|
||||
atomic_store_8(&pTask->status.keepTaskStatus, pTask->status.taskStatus);
|
||||
atomic_store_8(&pTask->status.statusBackup, pTask->status.taskStatus);
|
||||
atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__PAUSE);
|
||||
}
|
||||
}
|
||||
|
@ -1894,7 +1918,7 @@ static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) {
|
|||
}
|
||||
|
||||
// check if it is conflict with other trans in both sourceDb and targetDb.
|
||||
bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_PAUSE_NAME, true);
|
||||
bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_PAUSE_NAME, true);
|
||||
if (conflict) {
|
||||
sdbRelease(pMnode->pSdb, pStream);
|
||||
return -1;
|
||||
|
@ -1996,7 +2020,7 @@ int32_t mndResumeAllStreamTasks(STrans *pTrans, SMnode *pMnode, SStreamObj *pStr
|
|||
}
|
||||
|
||||
if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__PAUSE) {
|
||||
atomic_store_8(&pTask->status.taskStatus, pTask->status.keepTaskStatus);
|
||||
atomic_store_8(&pTask->status.taskStatus, pTask->status.statusBackup);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2037,7 +2061,7 @@ static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) {
|
|||
}
|
||||
|
||||
// check if it is conflict with other trans in both sourceDb and targetDb.
|
||||
bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_RESUME_NAME, true);
|
||||
bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_RESUME_NAME, true);
|
||||
if (conflict) {
|
||||
sdbRelease(pMnode->pSdb, pStream);
|
||||
return -1;
|
||||
|
@ -2343,7 +2367,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange
|
|||
break;
|
||||
}
|
||||
|
||||
bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_TASK_UPDATE_NAME, false);
|
||||
bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_TASK_UPDATE_NAME, false);
|
||||
sdbRelease(pSdb, pStream);
|
||||
|
||||
if (conflict) {
|
||||
|
@ -2567,7 +2591,7 @@ static void killAllCheckpointTrans(SMnode *pMnode, SVgroupChangeInfo *pChangeInf
|
|||
|
||||
size_t len = 0;
|
||||
void * pKey = taosHashGetKey(pDb, &len);
|
||||
killActiveCheckpointTrans(pMnode, pKey, len);
|
||||
doKillCheckpointTrans(pMnode, pKey, len);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2812,7 +2836,16 @@ int32_t createStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) {
|
|||
return TSDB_CODE_ACTION_IN_PROGRESS;
|
||||
}
|
||||
|
||||
int32_t killActiveCheckpointTrans(SMnode *pMnode, const char *pDBName, size_t len) {
|
||||
void killCheckpointTransImpl(SMnode* pMnode, int32_t transId, const char* pDbName) {
|
||||
STrans *pTrans = mndAcquireTrans(pMnode, transId);
|
||||
if (pTrans != NULL) {
|
||||
mInfo("kill checkpoint transId:%d in Db:%s", transId, pDbName);
|
||||
mndKillTrans(pMnode, pTrans);
|
||||
mndReleaseTrans(pMnode, pTrans);
|
||||
}
|
||||
}
|
||||
|
||||
int32_t doKillCheckpointTrans(SMnode *pMnode, const char *pDBName, size_t len) {
|
||||
// data in the hash table will be removed automatically, no need to remove it here.
|
||||
SStreamTransInfo *pTransInfo = taosHashGet(execInfo.transMgmt.pDBTrans, pDBName, len);
|
||||
if (pTransInfo == NULL) {
|
||||
|
@ -2825,15 +2858,9 @@ int32_t killActiveCheckpointTrans(SMnode *pMnode, const char *pDBName, size_t le
|
|||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
|
||||
STrans *pTrans = mndAcquireTrans(pMnode, pTransInfo->transId);
|
||||
if (pTrans != NULL) {
|
||||
char* pDupDBName = strndup(pDBName, len);
|
||||
mInfo("kill checkpoint transId:%d in Db:%s", pTransInfo->transId, pDupDBName);
|
||||
taosMemoryFree(pDupDBName);
|
||||
|
||||
mndKillTrans(pMnode, pTrans);
|
||||
mndReleaseTrans(pMnode, pTrans);
|
||||
}
|
||||
char* pDupDBName = strndup(pDBName, len);
|
||||
killCheckpointTransImpl(pMnode, pTransInfo->transId, pDupDBName);
|
||||
taosMemoryFree(pDupDBName);
|
||||
|
||||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
|
@ -2855,7 +2882,7 @@ static int32_t mndResetStatusFromCheckpoint(SMnode *pMnode, int64_t streamId, in
|
|||
code = TSDB_CODE_STREAM_TASK_NOT_EXIST;
|
||||
mError("failed to acquire the streamObj:0x%" PRIx64 " to reset checkpoint, may have been dropped", pStream->uid);
|
||||
} else {
|
||||
bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_TASK_RESET_NAME, false);
|
||||
bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_TASK_RESET_NAME, false);
|
||||
if (conflict) {
|
||||
mError("stream:%s other trans exists in DB:%s, dstTable:%s failed to start reset-status trans", pStream->name,
|
||||
pStream->sourceDb, pStream->targetSTbName);
|
||||
|
|
|
@ -65,7 +65,7 @@ int32_t clearFinishedTrans(SMnode* pMnode) {
|
|||
return 0;
|
||||
}
|
||||
|
||||
bool streamTransConflictOtherTrans(SMnode* pMnode, int64_t streamUid, const char* pTransName, bool lock) {
|
||||
bool mndStreamTransConflictCheck(SMnode* pMnode, int64_t streamUid, const char* pTransName, bool lock) {
|
||||
if (lock) {
|
||||
taosThreadMutexLock(&execInfo.lock);
|
||||
}
|
||||
|
@ -113,6 +113,30 @@ bool streamTransConflictOtherTrans(SMnode* pMnode, int64_t streamUid, const char
|
|||
return false;
|
||||
}
|
||||
|
||||
int32_t mndStreamGetRelCheckpointTrans(SMnode* pMnode, int64_t streamUid) {
|
||||
taosThreadMutexLock(&execInfo.lock);
|
||||
int32_t num = taosHashGetSize(execInfo.transMgmt.pDBTrans);
|
||||
if (num <= 0) {
|
||||
taosThreadMutexUnlock(&execInfo.lock);
|
||||
return 0;
|
||||
}
|
||||
|
||||
clearFinishedTrans(pMnode);
|
||||
SStreamTransInfo* pEntry = taosHashGet(execInfo.transMgmt.pDBTrans, &streamUid, sizeof(streamUid));
|
||||
if (pEntry != NULL) {
|
||||
SStreamTransInfo tInfo = *pEntry;
|
||||
taosThreadMutexUnlock(&execInfo.lock);
|
||||
|
||||
if (strcmp(tInfo.name, MND_STREAM_CHECKPOINT_NAME) == 0) {
|
||||
return tInfo.transId;
|
||||
}
|
||||
} else {
|
||||
taosThreadMutexUnlock(&execInfo.lock);
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
int32_t mndAddtoCheckpointWaitingList(SStreamObj* pStream, int64_t checkpointId) {
|
||||
SCheckpointCandEntry* pEntry = taosHashGet(execInfo.transMgmt.pWaitingList, &pStream->uid, sizeof(pStream->uid));
|
||||
if (pEntry == NULL) {
|
||||
|
|
|
@ -91,9 +91,7 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProcessVer
|
|||
pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer);
|
||||
}
|
||||
|
||||
char *p = NULL;
|
||||
streamTaskGetStatus(pTask, &p);
|
||||
|
||||
char* p = streamTaskGetStatus(pTask)->name;
|
||||
if (pTask->info.fillHistory) {
|
||||
sndInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64
|
||||
" nextProcessVer:%" PRId64
|
||||
|
@ -147,7 +145,8 @@ FAIL:
|
|||
}
|
||||
|
||||
int32_t sndInit(SSnode *pSnode) {
|
||||
tqStreamTaskResetStatus(pSnode->pMeta);
|
||||
int32_t numOfTasks = 0;
|
||||
tqStreamTaskResetStatus(pSnode->pMeta, &numOfTasks);
|
||||
streamMetaStartAllTasks(pSnode->pMeta);
|
||||
return 0;
|
||||
}
|
||||
|
@ -195,7 +194,7 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) {
|
|||
case TDMT_STREAM_TASK_DEPLOY: {
|
||||
void * pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead));
|
||||
int32_t len = pMsg->contLen - sizeof(SMsgHead);
|
||||
return tqStreamTaskProcessDeployReq(pSnode->pMeta, pMsg->info.conn.applyIndex, pReq, len, true, true);
|
||||
return tqStreamTaskProcessDeployReq(pSnode->pMeta, &pSnode->msgCb,pMsg->info.conn.applyIndex, pReq, len, true, true);
|
||||
}
|
||||
|
||||
case TDMT_STREAM_TASK_DROP:
|
||||
|
|
|
@ -107,6 +107,7 @@ struct STQ {
|
|||
TTB* pExecStore;
|
||||
TTB* pCheckStore;
|
||||
SStreamMeta* pStreamMeta;
|
||||
void* tqTimer;
|
||||
};
|
||||
|
||||
int32_t tEncodeSTqHandle(SEncoder* pEncoder, const STqHandle* pHandle);
|
||||
|
|
|
@ -19,13 +19,25 @@
|
|||
|
||||
// 0: not init
|
||||
// 1: already inited
|
||||
// 2: wait to be inited or cleaup
|
||||
// 2: wait to be inited or cleanup
|
||||
static int32_t tqInitialize(STQ* pTq);
|
||||
|
||||
static FORCE_INLINE bool tqIsHandleExec(STqHandle* pHandle) { return TMQ_HANDLE_STATUS_EXEC == pHandle->status; }
|
||||
static FORCE_INLINE void tqSetHandleExec(STqHandle* pHandle) { pHandle->status = TMQ_HANDLE_STATUS_EXEC; }
|
||||
static FORCE_INLINE void tqSetHandleIdle(STqHandle* pHandle) { pHandle->status = TMQ_HANDLE_STATUS_IDLE; }
|
||||
|
||||
static int32_t tqTimerInit(STQ* pTq) {
|
||||
pTq->tqTimer = taosTmrInit(100, 100, 1000, "TQ");
|
||||
if (pTq->tqTimer == NULL) {
|
||||
return -1;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
static void tqTimerCleanUp(STQ* pTq) {
|
||||
taosTmrCleanUp(pTq->tqTimer);
|
||||
}
|
||||
|
||||
void tqDestroyTqHandle(void* data) {
|
||||
STqHandle* pData = (STqHandle*)data;
|
||||
qDestroyTask(pData->execHandle.task);
|
||||
|
@ -106,6 +118,7 @@ int32_t tqInitialize(STQ* pTq) {
|
|||
return -1;
|
||||
}
|
||||
|
||||
tqTimerInit(pTq);
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
@ -136,6 +149,8 @@ void tqClose(STQ* pTq) {
|
|||
taosMemoryFree(pTq->path);
|
||||
tqMetaClose(pTq);
|
||||
streamMetaClose(pTq->pStreamMeta);
|
||||
tqTimerCleanUp(pTq);
|
||||
|
||||
qDebug("end to close tq");
|
||||
taosMemoryFree(pTq);
|
||||
}
|
||||
|
@ -842,8 +857,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) {
|
|||
pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer);
|
||||
}
|
||||
|
||||
char* p = NULL;
|
||||
streamTaskGetStatus(pTask, &p);
|
||||
char* p = streamTaskGetStatus(pTask)->name;
|
||||
|
||||
if (pTask->info.fillHistory) {
|
||||
tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64
|
||||
|
@ -873,7 +887,8 @@ int32_t tqProcessTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) {
|
|||
}
|
||||
|
||||
int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) {
|
||||
return tqStreamTaskProcessDeployReq(pTq->pStreamMeta, sversion, msg, msgLen, vnodeIsRoleLeader(pTq->pVnode), pTq->pVnode->restored);
|
||||
return tqStreamTaskProcessDeployReq(pTq->pStreamMeta, &pTq->pVnode->msgCb, sversion, msg, msgLen,
|
||||
vnodeIsRoleLeader(pTq->pVnode), pTq->pVnode->restored);
|
||||
}
|
||||
|
||||
static void doStartFillhistoryStep2(SStreamTask* pTask, SStreamTask* pStreamTask, STQ* pTq) {
|
||||
|
@ -932,8 +947,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
|
|||
|
||||
// do recovery step1
|
||||
const char* id = pTask->id.idStr;
|
||||
char* pStatus = NULL;
|
||||
streamTaskGetStatus(pTask, &pStatus);
|
||||
char* pStatus = streamTaskGetStatus(pTask)->name;
|
||||
|
||||
// avoid multi-thread exec
|
||||
while (1) {
|
||||
|
@ -990,15 +1004,15 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
|
|||
if (retInfo.ret == TASK_SCANHISTORY_REXEC) {
|
||||
streamReExecScanHistoryFuture(pTask, retInfo.idleTime);
|
||||
} else {
|
||||
char* p = NULL;
|
||||
ETaskStatus s = streamTaskGetStatus(pTask, &p);
|
||||
SStreamTaskState* p = streamTaskGetStatus(pTask);
|
||||
ETaskStatus s = p->state;
|
||||
|
||||
if (s == TASK_STATUS__PAUSE) {
|
||||
tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs total:%.2fs, sched-status:%d", pTask->id.idStr,
|
||||
el, pTask->execInfo.step1El, status);
|
||||
} else if (s == TASK_STATUS__STOP || s == TASK_STATUS__DROPPING) {
|
||||
tqDebug("s-task:%s status:%p not continue scan-history data, total elapsed time:%.2fs quit", pTask->id.idStr, p,
|
||||
pTask->execInfo.step1El);
|
||||
tqDebug("s-task:%s status:%p not continue scan-history data, total elapsed time:%.2fs quit", pTask->id.idStr,
|
||||
p->name, pTask->execInfo.step1El);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1038,15 +1052,6 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) {
|
|||
streamMetaReleaseTask(pMeta, pStreamTask);
|
||||
} else {
|
||||
ASSERT(0);
|
||||
// STimeWindow* pWindow = &pTask->dataRange.window;
|
||||
// ASSERT(HAS_RELATED_FILLHISTORY_TASK(pTask) || streamTaskShouldStop(pTask));
|
||||
//
|
||||
// // Not update the fill-history time window until the state transfer is completed.
|
||||
// tqDebug("s-task:%s scan-history in stream time window completed, start to handle data from WAL, startVer:%" PRId64
|
||||
// ", window:%" PRId64 " - %" PRId64,
|
||||
// id, pTask->chkInfo.nextProcessVer, pWindow->skey, pWindow->ekey);
|
||||
//
|
||||
// code = streamTaskScanHistoryDataComplete(pTask);
|
||||
}
|
||||
|
||||
atomic_store_32(&pTask->status.inScanHistorySentinel, 0);
|
||||
|
@ -1066,15 +1071,16 @@ int32_t tqProcessTaskScanHistoryFinishRsp(STQ* pTq, SRpcMsg* pMsg) {
|
|||
int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) {
|
||||
SStreamTaskRunReq* pReq = pMsg->pCont;
|
||||
|
||||
int32_t taskId = pReq->taskId;
|
||||
|
||||
if (taskId == STREAM_EXEC_EXTRACT_DATA_IN_WAL_ID) { // all tasks are extracted submit data from the wal
|
||||
// extracted submit data from wal files for all tasks
|
||||
if (pReq->reqType == STREAM_EXEC_T_EXTRACT_WAL_DATA) {
|
||||
tqScanWal(pTq);
|
||||
return 0;
|
||||
}
|
||||
|
||||
int32_t code = tqStreamTaskProcessRunReq(pTq->pStreamMeta, pMsg, vnodeIsRoleLeader(pTq->pVnode));
|
||||
if(code == 0 && taskId > 0){
|
||||
|
||||
// let's continue scan data in the wal files
|
||||
if(code == 0 && pReq->reqType >= 0){
|
||||
tqScanWalAsync(pTq, false);
|
||||
}
|
||||
|
||||
|
@ -1178,7 +1184,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp)
|
|||
|
||||
// todo save the checkpoint failed info
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask)->state;
|
||||
|
||||
if (status == TASK_STATUS__HALT || status == TASK_STATUS__PAUSE) {
|
||||
tqError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 ", set it failure",
|
||||
|
@ -1265,14 +1271,8 @@ int32_t tqProcessTaskDropHTask(STQ* pTq, SRpcMsg* pMsg) {
|
|||
}
|
||||
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
|
||||
// if (status == TASK_STATUS__STREAM_SCAN_HISTORY) {
|
||||
// streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE);
|
||||
// }
|
||||
|
||||
SStreamTaskId id = {.streamId = pTask->hTaskInfo.id.streamId, .taskId = pTask->hTaskInfo.id.taskId};
|
||||
streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &id);
|
||||
|
||||
taosThreadMutexUnlock(&pTask->lock);
|
||||
|
||||
// clear the scheduler status
|
||||
|
|
|
@ -24,6 +24,7 @@ static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId);
|
|||
static bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver);
|
||||
static bool taskReadyForDataFromWal(SStreamTask* pTask);
|
||||
static bool doPutDataIntoInputQFromWal(SStreamTask* pTask, int64_t maxVer, int32_t* numOfItems);
|
||||
static int32_t tqScanWalInFuture(STQ* pTq, int32_t numOfTasks, int32_t idleDuration);
|
||||
|
||||
// extract data blocks(submit/delete) from WAL, and add them into the input queue for all the sources tasks.
|
||||
int32_t tqScanWal(STQ* pTq) {
|
||||
|
@ -31,35 +32,78 @@ int32_t tqScanWal(STQ* pTq) {
|
|||
SStreamMeta* pMeta = pTq->pStreamMeta;
|
||||
int64_t st = taosGetTimestampMs();
|
||||
|
||||
while (1) {
|
||||
int32_t scan = pMeta->walScanCounter;
|
||||
tqDebug("vgId:%d continue check if data in wal are available, walScanCounter:%d", vgId, scan);
|
||||
tqDebug("vgId:%d continue to check if data in wal are available, scanCounter:%d", vgId, pMeta->scanInfo.scanCounter);
|
||||
|
||||
// check all tasks
|
||||
bool shouldIdle = true;
|
||||
doScanWalForAllTasks(pTq->pStreamMeta, &shouldIdle);
|
||||
// check all tasks
|
||||
int32_t numOfTasks = 0;
|
||||
bool shouldIdle = true;
|
||||
doScanWalForAllTasks(pMeta, &shouldIdle);
|
||||
|
||||
// if (shouldIdle) {
|
||||
streamMetaWLock(pMeta);
|
||||
int32_t times = (--pMeta->walScanCounter);
|
||||
ASSERT(pMeta->walScanCounter >= 0);
|
||||
streamMetaWUnLock(pMeta);
|
||||
streamMetaWLock(pMeta);
|
||||
int32_t times = (--pMeta->scanInfo.scanCounter);
|
||||
ASSERT(pMeta->scanInfo.scanCounter >= 0);
|
||||
|
||||
if (times <= 0) {
|
||||
break;
|
||||
} else {
|
||||
tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION);
|
||||
}
|
||||
// }
|
||||
|
||||
taosMsleep(SCAN_WAL_IDLE_DURATION);
|
||||
}
|
||||
numOfTasks = taosArrayGetSize(pMeta->pTaskList);
|
||||
streamMetaWUnLock(pMeta);
|
||||
|
||||
int64_t el = (taosGetTimestampMs() - st);
|
||||
tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 " ms", vgId, el);
|
||||
|
||||
if (times > 0) {
|
||||
tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION);
|
||||
tqScanWalInFuture(pTq, numOfTasks, SCAN_WAL_IDLE_DURATION);
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
typedef struct SBuildScanWalMsgParam {
|
||||
STQ* pTq;
|
||||
int32_t numOfTasks;
|
||||
} SBuildScanWalMsgParam;
|
||||
|
||||
static void doStartScanWal(void* param, void* tmrId) {
|
||||
SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*) param;
|
||||
|
||||
int32_t vgId = pParam->pTq->pStreamMeta->vgId;
|
||||
|
||||
SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq));
|
||||
if (pRunReq == NULL) {
|
||||
taosMemoryFree(pParam);
|
||||
terrno = TSDB_CODE_OUT_OF_MEMORY;
|
||||
tqError("vgId:%d failed to create msg to start wal scanning to launch stream tasks, code:%s", vgId, terrstr());
|
||||
return;
|
||||
}
|
||||
|
||||
tqDebug("vgId:%d create msg to start wal scan, numOfTasks:%d, vnd restored:%d", vgId, pParam->numOfTasks,
|
||||
pParam->pTq->pVnode->restored);
|
||||
|
||||
pRunReq->head.vgId = vgId;
|
||||
pRunReq->streamId = 0;
|
||||
pRunReq->taskId = 0;
|
||||
pRunReq->reqType = STREAM_EXEC_T_EXTRACT_WAL_DATA;
|
||||
|
||||
SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)};
|
||||
tmsgPutToQueue(&pParam->pTq->pVnode->msgCb, STREAM_QUEUE, &msg);
|
||||
|
||||
taosMemoryFree(pParam);
|
||||
}
|
||||
|
||||
int32_t tqScanWalInFuture(STQ* pTq, int32_t numOfTasks, int32_t idleDuration) {
|
||||
SStreamMeta* pMeta = pTq->pStreamMeta;
|
||||
|
||||
SBuildScanWalMsgParam* pParam = taosMemoryMalloc(sizeof(SBuildScanWalMsgParam));
|
||||
|
||||
pParam->pTq = pTq;
|
||||
pParam->numOfTasks = numOfTasks;
|
||||
if (pMeta->scanInfo.scanTimer == NULL) {
|
||||
pMeta->scanInfo.scanTimer = taosTmrStart(doStartScanWal, idleDuration, pParam, pTq->tqTimer);
|
||||
} else {
|
||||
taosTmrReset(doStartScanWal, idleDuration, pParam, pTq->tqTimer, &pMeta->scanInfo.scanTimer);
|
||||
}
|
||||
|
||||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
|
||||
int32_t tqScanWalAsync(STQ* pTq, bool ckPause) {
|
||||
int32_t vgId = TD_VID(pTq->pVnode);
|
||||
SStreamMeta* pMeta = pTq->pStreamMeta;
|
||||
|
@ -80,23 +124,23 @@ int32_t tqScanWalAsync(STQ* pTq, bool ckPause) {
|
|||
return 0;
|
||||
}
|
||||
|
||||
pMeta->walScanCounter += 1;
|
||||
if (pMeta->walScanCounter > MAX_REPEAT_SCAN_THRESHOLD) {
|
||||
pMeta->walScanCounter = MAX_REPEAT_SCAN_THRESHOLD;
|
||||
pMeta->scanInfo.scanCounter += 1;
|
||||
if (pMeta->scanInfo.scanCounter > MAX_REPEAT_SCAN_THRESHOLD) {
|
||||
pMeta->scanInfo.scanCounter = MAX_REPEAT_SCAN_THRESHOLD;
|
||||
}
|
||||
|
||||
if (pMeta->walScanCounter > 1) {
|
||||
tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->walScanCounter);
|
||||
if (pMeta->scanInfo.scanCounter > 1) {
|
||||
tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->scanInfo.scanCounter);
|
||||
streamMetaWUnLock(pMeta);
|
||||
return 0;
|
||||
}
|
||||
|
||||
int32_t numOfPauseTasks = pTq->pStreamMeta->numOfPausedTasks;
|
||||
int32_t numOfPauseTasks = pMeta->numOfPausedTasks;
|
||||
if (ckPause && numOfTasks == numOfPauseTasks) {
|
||||
tqDebug("vgId:%d ignore all submit, all streams had been paused, reset the walScanCounter", vgId);
|
||||
|
||||
// reset the counter value, since we do not launch the scan wal operation.
|
||||
pMeta->walScanCounter = 0;
|
||||
pMeta->scanInfo.scanCounter = 0;
|
||||
streamMetaWUnLock(pMeta);
|
||||
return 0;
|
||||
}
|
||||
|
@ -114,7 +158,8 @@ int32_t tqScanWalAsync(STQ* pTq, bool ckPause) {
|
|||
|
||||
pRunReq->head.vgId = vgId;
|
||||
pRunReq->streamId = 0;
|
||||
pRunReq->taskId = STREAM_EXEC_EXTRACT_DATA_IN_WAL_ID;
|
||||
pRunReq->taskId = 0;
|
||||
pRunReq->reqType = STREAM_EXEC_T_EXTRACT_WAL_DATA;
|
||||
|
||||
SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)};
|
||||
tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg);
|
||||
|
@ -138,7 +183,8 @@ int32_t tqStopStreamTasksAsync(STQ* pTq) {
|
|||
|
||||
pRunReq->head.vgId = vgId;
|
||||
pRunReq->streamId = 0;
|
||||
pRunReq->taskId = STREAM_EXEC_STOP_ALL_TASKS_ID;
|
||||
pRunReq->taskId = 0;
|
||||
pRunReq->reqType = STREAM_EXEC_T_STOP_ALL_TASKS;
|
||||
|
||||
SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)};
|
||||
tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg);
|
||||
|
@ -220,16 +266,15 @@ bool taskReadyForDataFromWal(SStreamTask* pTask) {
|
|||
}
|
||||
|
||||
// not in ready state, do not handle the data from wal
|
||||
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);
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pTask);
|
||||
if (pState->state != TASK_STATUS__READY) {
|
||||
tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, pState->name);
|
||||
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__READY);
|
||||
ASSERT(pState->state == 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);
|
||||
|
@ -342,10 +387,9 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) {
|
|||
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
|
||||
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);
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pTask);
|
||||
if (pState->state != TASK_STATUS__READY) {
|
||||
tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pState->name);
|
||||
taosThreadMutexUnlock(&pTask->lock);
|
||||
streamMetaReleaseTask(pStreamMeta, pTask);
|
||||
continue;
|
||||
|
|
|
@ -25,7 +25,6 @@ typedef struct STaskUpdateEntry {
|
|||
|
||||
int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart) {
|
||||
int32_t vgId = pMeta->vgId;
|
||||
|
||||
int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList);
|
||||
if (numOfTasks == 0) {
|
||||
tqDebug("vgId:%d no stream tasks existed to run", vgId);
|
||||
|
@ -42,7 +41,62 @@ int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart) {
|
|||
tqDebug("vgId:%d start all %d stream task(s) async", vgId, numOfTasks);
|
||||
pRunReq->head.vgId = vgId;
|
||||
pRunReq->streamId = 0;
|
||||
pRunReq->taskId = restart ? STREAM_EXEC_RESTART_ALL_TASKS_ID : STREAM_EXEC_START_ALL_TASKS_ID;
|
||||
pRunReq->taskId = 0;
|
||||
pRunReq->reqType = restart ? STREAM_EXEC_T_RESTART_ALL_TASKS : STREAM_EXEC_T_START_ALL_TASKS;
|
||||
|
||||
SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)};
|
||||
tmsgPutToQueue(cb, STREAM_QUEUE, &msg);
|
||||
return 0;
|
||||
}
|
||||
|
||||
int32_t tqStreamOneTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId) {
|
||||
int32_t vgId = pMeta->vgId;
|
||||
|
||||
int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList);
|
||||
if (numOfTasks == 0) {
|
||||
tqDebug("vgId:%d no stream tasks existed to run", vgId);
|
||||
return 0;
|
||||
}
|
||||
|
||||
SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq));
|
||||
if (pRunReq == NULL) {
|
||||
terrno = TSDB_CODE_OUT_OF_MEMORY;
|
||||
tqError("vgId:%d failed to create msg to start task:0x%x, code:%s", vgId, taskId, terrstr());
|
||||
return -1;
|
||||
}
|
||||
|
||||
tqDebug("vgId:%d start task:0x%x async", vgId, taskId);
|
||||
pRunReq->head.vgId = vgId;
|
||||
pRunReq->streamId = streamId;
|
||||
pRunReq->taskId = taskId;
|
||||
pRunReq->reqType = STREAM_EXEC_T_START_ONE_TASK;
|
||||
|
||||
SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)};
|
||||
tmsgPutToQueue(cb, STREAM_QUEUE, &msg);
|
||||
return 0;
|
||||
}
|
||||
|
||||
int32_t tqUpdateNodeEpsetAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId) {
|
||||
int32_t vgId = pMeta->vgId;
|
||||
|
||||
int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList);
|
||||
if (numOfTasks == 0) {
|
||||
tqDebug("vgId:%d no stream tasks existed to run", vgId);
|
||||
return 0;
|
||||
}
|
||||
|
||||
SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq));
|
||||
if (pRunReq == NULL) {
|
||||
terrno = TSDB_CODE_OUT_OF_MEMORY;
|
||||
tqError("vgId:%d failed to create msg to start task:0x%x, code:%s", vgId, taskId, terrstr());
|
||||
return -1;
|
||||
}
|
||||
|
||||
tqDebug("vgId:%d update s-task:0x%x nodeEpset async", vgId, taskId);
|
||||
pRunReq->head.vgId = vgId;
|
||||
pRunReq->streamId = streamId;
|
||||
pRunReq->taskId = taskId;
|
||||
pRunReq->reqType = STREAM_EXEC_T_UPDATE_TASK_EPSET;
|
||||
|
||||
SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)};
|
||||
tmsgPutToQueue(cb, STREAM_QUEUE, &msg);
|
||||
|
@ -54,6 +108,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM
|
|||
char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead));
|
||||
int32_t len = pMsg->contLen - sizeof(SMsgHead);
|
||||
SRpcMsg rsp = {.info = pMsg->info, .code = TSDB_CODE_SUCCESS};
|
||||
int64_t st = taosGetTimestampMs();
|
||||
|
||||
SStreamTaskNodeUpdateMsg req = {0};
|
||||
|
||||
|
@ -75,7 +130,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM
|
|||
STaskId id = {.streamId = req.streamId, .taskId = req.taskId};
|
||||
SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
|
||||
if (ppTask == NULL || *ppTask == NULL) {
|
||||
tqError("vgId:%d failed to acquire task:0x%x when handling update, it may have been dropped already", pMeta->vgId,
|
||||
tqError("vgId:%d failed to acquire task:0x%x when handling update, it may have been dropped already", vgId,
|
||||
req.taskId);
|
||||
rsp.code = TSDB_CODE_SUCCESS;
|
||||
streamMetaWUnLock(pMeta);
|
||||
|
@ -85,43 +140,38 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM
|
|||
}
|
||||
|
||||
SStreamTask* pTask = *ppTask;
|
||||
const char* idstr = pTask->id.idStr;
|
||||
|
||||
if (pMeta->updateInfo.transId != req.transId) {
|
||||
pMeta->updateInfo.transId = req.transId;
|
||||
tqInfo("s-task:%s receive new trans to update nodeEp msg from mnode, transId:%d", pTask->id.idStr, req.transId);
|
||||
tqInfo("s-task:%s receive new trans to update nodeEp msg from mnode, transId:%d", 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);
|
||||
tqDebug("s-task:%s recv trans to update nodeEp from mnode, transId:%d", 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);
|
||||
|
||||
void* pReqTask = taosHashGet(pMeta->updateInfo.pTasks, &entry, sizeof(STaskUpdateEntry));
|
||||
if (pReqTask != NULL) {
|
||||
tqDebug("s-task:%s (vgId:%d) already update in trans:%d, discard the nodeEp update msg", idstr, vgId, req.transId);
|
||||
rsp.code = TSDB_CODE_SUCCESS;
|
||||
streamMetaWUnLock(pMeta);
|
||||
taosArrayDestroy(req.pNodeList);
|
||||
return rsp.code;
|
||||
}
|
||||
|
||||
// streamMetaWUnLock(pMeta);
|
||||
|
||||
// todo for test purpose
|
||||
// the following two functions should not be executed within the scope of meta lock to avoid deadlock
|
||||
streamTaskUpdateEpsetInfo(pTask, req.pNodeList);
|
||||
streamTaskResetStatus(pTask);
|
||||
|
||||
// continue after lock the meta again
|
||||
// streamMetaWLock(pMeta);
|
||||
|
||||
SStreamTask** ppHTask = NULL;
|
||||
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);
|
||||
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);
|
||||
|
@ -129,17 +179,23 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM
|
|||
}
|
||||
}
|
||||
|
||||
{
|
||||
if (restored) {
|
||||
tqDebug("s-task:%s vgId:%d start to save task", idstr, vgId);
|
||||
streamMetaSaveTask(pMeta, pTask);
|
||||
if (ppHTask != NULL) {
|
||||
streamMetaSaveTask(pMeta, *ppHTask);
|
||||
}
|
||||
|
||||
#if 0
|
||||
if (streamMetaCommit(pMeta) < 0) {
|
||||
// persist to disk
|
||||
}
|
||||
#endif
|
||||
} else {
|
||||
tqDebug("s-task:%s vgId:%d not save since restore not finish", idstr, vgId);
|
||||
}
|
||||
|
||||
tqDebug("s-task:%s vgId:%d start to stop task after save task", idstr, vgId);
|
||||
streamTaskStop(pTask);
|
||||
|
||||
// keep the already handled info
|
||||
|
@ -147,10 +203,15 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM
|
|||
|
||||
if (ppHTask != NULL) {
|
||||
streamTaskStop(*ppHTask);
|
||||
tqDebug("s-task:%s task nodeEp update completed, streamTask and related fill-history task closed", pTask->id.idStr);
|
||||
|
||||
int64_t now = taosGetTimestampMs();
|
||||
tqDebug("s-task:%s vgId:%d task nodeEp update completed, streamTask/fill-history closed, elapsed:%" PRId64 " ms",
|
||||
idstr, vgId, now - st);
|
||||
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);
|
||||
int64_t now = taosGetTimestampMs();
|
||||
tqDebug("s-task:%s vgId:%d, task nodeEp update completed, streamTask closed, elapsed time:%" PRId64 "ms", idstr,
|
||||
vgId, now - st);
|
||||
}
|
||||
|
||||
rsp.code = 0;
|
||||
|
@ -171,11 +232,12 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM
|
|||
pMeta->startInfo.tasksWillRestart = 0;
|
||||
streamMetaWUnLock(pMeta);
|
||||
} else {
|
||||
tqDebug("vgId:%d all %d task(s) nodeEp updated and closed", vgId, numOfTasks);
|
||||
tqDebug("vgId:%d all %d task(s) nodeEp updated and closed, transId:%d", vgId, numOfTasks, req.transId);
|
||||
#if 0
|
||||
// for test purpose, to trigger the leader election
|
||||
taosMSleep(5000);
|
||||
#endif
|
||||
|
||||
tqStreamTaskStartAsync(pMeta, cb, true);
|
||||
streamMetaWUnLock(pMeta);
|
||||
}
|
||||
|
@ -399,11 +461,11 @@ int32_t tqStreamTaskProcessCheckReq(SStreamMeta* pMeta, SRpcMsg* pMsg) {
|
|||
rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage, &rsp.oldStage);
|
||||
streamMetaReleaseTask(pMeta, pTask);
|
||||
|
||||
char* p = NULL;
|
||||
streamTaskGetStatus(pTask, &p);
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pTask);
|
||||
tqDebug("s-task:%s status:%s, stage:%" PRId64 " recv task check req(reqId:0x%" PRIx64
|
||||
") task:0x%x (vgId:%d), check_status:%d",
|
||||
pTask->id.idStr, p, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status);
|
||||
pTask->id.idStr, pState->name, 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
|
||||
|
@ -415,12 +477,22 @@ int32_t tqStreamTaskProcessCheckReq(SStreamMeta* pMeta, SRpcMsg* pMsg) {
|
|||
return streamSendCheckRsp(pMeta, &req, &rsp, &pMsg->info, taskId);
|
||||
}
|
||||
|
||||
static void setParam(SStreamTask* pTask, int64_t* initTs, bool* hasHTask, STaskId* pId) {
|
||||
*initTs = pTask->execInfo.init;
|
||||
|
||||
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
|
||||
*hasHTask = true;
|
||||
pId->streamId = pTask->hTaskInfo.id.streamId;
|
||||
pId->taskId = pTask->hTaskInfo.id.taskId;
|
||||
}
|
||||
}
|
||||
|
||||
int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader) {
|
||||
char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead));
|
||||
int32_t len = pMsg->contLen - sizeof(SMsgHead);
|
||||
int32_t vgId = pMeta->vgId;
|
||||
int32_t code = TSDB_CODE_SUCCESS;
|
||||
|
||||
int32_t code;
|
||||
SStreamTaskCheckRsp rsp;
|
||||
|
||||
SDecoder decoder;
|
||||
|
@ -437,20 +509,62 @@ int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLe
|
|||
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);
|
||||
|
||||
int64_t initTs = 0;
|
||||
int64_t now = taosGetTimestampMs();
|
||||
STaskId id = {.streamId = rsp.streamId, .taskId = rsp.upstreamTaskId};
|
||||
STaskId fId = {0};
|
||||
bool hasHistoryTask = false;
|
||||
|
||||
// todo extract method
|
||||
if (!isLeader) {
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, 0, taosGetTimestampMs(), false);
|
||||
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);
|
||||
// this task may have been stopped, so acquire task may failed. Retrieve it directly from the task hash map.
|
||||
streamMetaRLock(pMeta);
|
||||
|
||||
SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
|
||||
if (ppTask != NULL) {
|
||||
setParam(*ppTask, &initTs, &hasHistoryTask, &fId);
|
||||
streamMetaRUnLock(pMeta);
|
||||
|
||||
if (hasHistoryTask) {
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, fId.streamId, fId.taskId, initTs, now, false);
|
||||
}
|
||||
|
||||
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);
|
||||
} else {
|
||||
streamMetaRUnLock(pMeta);
|
||||
|
||||
tqError("tq failed to locate the stream task:0x%" PRIx64 "-0x%x (vgId:%d), it may have been destroyed or stopped",
|
||||
rsp.streamId, rsp.upstreamTaskId, vgId);
|
||||
code = terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST;
|
||||
}
|
||||
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, initTs, now, false);
|
||||
return code;
|
||||
}
|
||||
|
||||
SStreamTask* pTask = streamMetaAcquireTask(pMeta, rsp.streamId, rsp.upstreamTaskId);
|
||||
if (pTask == NULL) {
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, 0, taosGetTimestampMs(), false);
|
||||
streamMetaRLock(pMeta);
|
||||
|
||||
SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
|
||||
if (ppTask != NULL) {
|
||||
setParam(*ppTask, &initTs, &hasHistoryTask, &fId);
|
||||
streamMetaRUnLock(pMeta);
|
||||
|
||||
if (hasHistoryTask) {
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, fId.streamId, fId.taskId, initTs, now, false);
|
||||
}
|
||||
} else {
|
||||
streamMetaRUnLock(pMeta);
|
||||
}
|
||||
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, initTs, now, false);
|
||||
tqError("tq failed to locate the stream task:0x%" PRIx64 "-0x%x (vgId:%d), it may have been destroyed or stopped",
|
||||
rsp.streamId, rsp.upstreamTaskId, vgId);
|
||||
terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST;
|
||||
return -1;
|
||||
|
||||
code = terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST;
|
||||
return code;
|
||||
}
|
||||
|
||||
code = streamProcessCheckRsp(pTask, &rsp);
|
||||
|
@ -489,8 +603,8 @@ int32_t tqStreamTaskProcessCheckpointReadyMsg(SStreamMeta* pMeta, SRpcMsg* pMsg)
|
|||
return code;
|
||||
}
|
||||
|
||||
int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, int64_t sversion, char* msg, int32_t msgLen, bool isLeader,
|
||||
bool restored) {
|
||||
int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sversion, char* msg, int32_t msgLen,
|
||||
bool isLeader, bool restored) {
|
||||
int32_t code = 0;
|
||||
int32_t vgId = pMeta->vgId;
|
||||
|
||||
|
@ -542,18 +656,19 @@ int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, int64_t sversion, char*
|
|||
// only handled in the leader node
|
||||
if (isLeader) {
|
||||
tqDebug("vgId:%d s-task:0x%x is deployed and add into meta, numOfTasks:%d", vgId, taskId, numOfTasks);
|
||||
SStreamTask* p = streamMetaAcquireTask(pMeta, streamId, taskId);
|
||||
|
||||
if (p != NULL && restored && p->info.fillHistory == 0) {
|
||||
EStreamTaskEvent event = 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);
|
||||
if (restored) {
|
||||
SStreamTask* p = streamMetaAcquireTask(pMeta, streamId, taskId);
|
||||
if (p != NULL && (p->info.fillHistory == 0)) {
|
||||
tqStreamOneTaskStartAsync(pMeta, cb, streamId, taskId);
|
||||
}
|
||||
if (p != NULL) {
|
||||
streamMetaReleaseTask(pMeta, p);
|
||||
}
|
||||
} else {
|
||||
tqWarn("s-task:0x%x not launched since vnode(vgId:%d) not ready", taskId, vgId);
|
||||
}
|
||||
|
||||
if (p != NULL) {
|
||||
streamMetaReleaseTask(pMeta, p);
|
||||
}
|
||||
} else {
|
||||
tqDebug("vgId:%d not leader, not launch stream task s-task:0x%x", vgId, taskId);
|
||||
}
|
||||
|
@ -598,21 +713,25 @@ int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen
|
|||
return 0;
|
||||
}
|
||||
|
||||
int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta) {
|
||||
int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta, int32_t* numOfTasks) {
|
||||
int32_t vgId = pMeta->vgId;
|
||||
int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList);
|
||||
*numOfTasks = taosArrayGetSize(pMeta->pTaskList);
|
||||
|
||||
tqDebug("vgId:%d reset all %d stream task(s) status to be uninit", vgId, numOfTasks);
|
||||
if (numOfTasks == 0) {
|
||||
tqDebug("vgId:%d reset all %d stream task(s) status to be uninit", vgId, *numOfTasks);
|
||||
if (*numOfTasks == 0) {
|
||||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
|
||||
for (int32_t i = 0; i < numOfTasks; ++i) {
|
||||
for (int32_t i = 0; i < (*numOfTasks); ++i) {
|
||||
SStreamTaskId* pTaskId = taosArrayGet(pMeta->pTaskList, i);
|
||||
|
||||
STaskId id = {.streamId = pTaskId->streamId, .taskId = pTaskId->taskId};
|
||||
SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
|
||||
streamTaskResetStatus(*pTask);
|
||||
|
||||
// if ((*pTask)->info.fillHistory == 1) {
|
||||
// streamResetParamForScanHistory(*pTask);
|
||||
// }
|
||||
}
|
||||
|
||||
return 0;
|
||||
|
@ -658,9 +777,18 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) {
|
|||
return code;
|
||||
}
|
||||
|
||||
{
|
||||
STaskStartInfo* pStartInfo = &pMeta->startInfo;
|
||||
taosHashClear(pStartInfo->pReadyTaskSet);
|
||||
taosHashClear(pStartInfo->pFailedTaskSet);
|
||||
pStartInfo->readyTs = 0;
|
||||
}
|
||||
|
||||
if (isLeader && !tsDisableStream) {
|
||||
tqStreamTaskResetStatus(pMeta);
|
||||
int32_t numOfTasks = 0;
|
||||
tqStreamTaskResetStatus(pMeta, &numOfTasks);
|
||||
streamMetaWUnLock(pMeta);
|
||||
|
||||
streamMetaStartAllTasks(pMeta);
|
||||
} else {
|
||||
streamMetaResetStartInfo(&pMeta->startInfo);
|
||||
|
@ -675,21 +803,36 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) {
|
|||
int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader) {
|
||||
SStreamTaskRunReq* pReq = pMsg->pCont;
|
||||
|
||||
int32_t taskId = pReq->taskId;
|
||||
int32_t type = pReq->reqType;
|
||||
int32_t vgId = pMeta->vgId;
|
||||
|
||||
if (taskId == STREAM_EXEC_START_ALL_TASKS_ID) {
|
||||
if (type == STREAM_EXEC_T_START_ONE_TASK) {
|
||||
streamMetaStartOneTask(pMeta, pReq->streamId, pReq->taskId);
|
||||
return 0;
|
||||
} else if (type == STREAM_EXEC_T_START_ALL_TASKS) {
|
||||
streamMetaStartAllTasks(pMeta);
|
||||
return 0;
|
||||
} else if (taskId == STREAM_EXEC_RESTART_ALL_TASKS_ID) {
|
||||
} else if (type == STREAM_EXEC_T_RESTART_ALL_TASKS) {
|
||||
restartStreamTasks(pMeta, isLeader);
|
||||
return 0;
|
||||
} else if (taskId == STREAM_EXEC_STOP_ALL_TASKS_ID) {
|
||||
} else if (type == STREAM_EXEC_T_STOP_ALL_TASKS) {
|
||||
streamMetaStopAllTasks(pMeta);
|
||||
return 0;
|
||||
} else if (type == STREAM_EXEC_T_RESUME_TASK) { // task resume to run after idle for a while
|
||||
SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId);
|
||||
if (pTask != NULL) {
|
||||
ASSERT(streamTaskReadyToRun(pTask, NULL));
|
||||
int64_t execTs = pTask->status.lastExecTs;
|
||||
int32_t idle = taosGetTimestampMs() - execTs;
|
||||
tqDebug("s-task:%s task resume to run after idle for:%dms from:%" PRId64, pTask->id.idStr, idle, execTs);
|
||||
|
||||
streamResumeTask(pTask);
|
||||
streamMetaReleaseTask(pMeta, pTask);
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, taskId);
|
||||
SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId);
|
||||
if (pTask != NULL) { // even in halt status, the data in inputQ must be processed
|
||||
char* p = NULL;
|
||||
if (streamTaskReadyToRun(pTask, &p)) {
|
||||
|
@ -706,29 +849,34 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead
|
|||
return 0;
|
||||
} else { // NOTE: pTask->status.schedStatus is not updated since it is not be handled by the run exec.
|
||||
// todo add one function to handle this
|
||||
tqError("vgId:%d failed to found s-task, taskId:0x%x may have been dropped", vgId, taskId);
|
||||
tqError("vgId:%d failed to found s-task, taskId:0x%x may have been dropped", vgId, pReq->taskId);
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta) {
|
||||
STaskStartInfo* pStartInfo = &pMeta->startInfo;
|
||||
int32_t vgId = pMeta->vgId;
|
||||
|
||||
streamMetaWLock(pMeta);
|
||||
if (pStartInfo->taskStarting == 1) {
|
||||
tqDebug("vgId:%d already in start tasks procedure in other thread, restartCounter:%d, do nothing", vgId,
|
||||
pMeta->startInfo.restartCount);
|
||||
} else { // not in starting procedure
|
||||
if (pStartInfo->restartCount > 0) {
|
||||
pStartInfo->restartCount -= 1;
|
||||
tqDebug("vgId:%d role:%d need to restart all tasks again, restartCounter:%d", vgId, pMeta->role,
|
||||
pStartInfo->restartCount);
|
||||
|
||||
if (pStartInfo->restartCount > 0) {
|
||||
pStartInfo->restartCount -= 1;
|
||||
|
||||
ASSERT(pStartInfo->taskStarting == 0);
|
||||
tqDebug("vgId:%d role:%d need to restart all tasks again, restartCounter:%d", pMeta->vgId, pMeta->role,
|
||||
pStartInfo->restartCount);
|
||||
|
||||
streamMetaWUnLock(pMeta);
|
||||
restartStreamTasks(pMeta, (pMeta->role == NODE_ROLE_LEADER));
|
||||
} else {
|
||||
streamMetaWUnLock(pMeta);
|
||||
tqDebug("vgId:%d start all tasks completed", pMeta->vgId);
|
||||
streamMetaWUnLock(pMeta);
|
||||
restartStreamTasks(pMeta, (pMeta->role == NODE_ROLE_LEADER));
|
||||
return TSDB_CODE_SUCCESS;
|
||||
} else {
|
||||
tqDebug("vgId:%d start all tasks completed in callbackFn", pMeta->vgId);
|
||||
}
|
||||
}
|
||||
|
||||
streamMetaWUnLock(pMeta);
|
||||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
|
||||
|
@ -745,7 +893,7 @@ int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, 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 (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__CK) {
|
||||
if (streamTaskGetStatus(pTask)->state == TASK_STATUS__CK) {
|
||||
streamTaskClearCheckInfo(pTask, true);
|
||||
streamTaskSetStatusReady(pTask);
|
||||
}
|
||||
|
@ -799,7 +947,7 @@ static int32_t tqProcessTaskResumeImpl(void* handle, SStreamTask* pTask, int64_t
|
|||
}
|
||||
|
||||
streamTaskResume(pTask);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask)->state;
|
||||
|
||||
int32_t level = pTask->info.taskLevel;
|
||||
if (level == TASK_LEVEL__SINK) {
|
||||
|
|
|
@ -567,29 +567,30 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx)
|
|||
if (vnodeIsRoleLeader(pVnode)) {
|
||||
// start to restore all stream tasks
|
||||
if (tsDisableStream) {
|
||||
streamMetaWUnLock(pMeta);
|
||||
vInfo("vgId:%d, sync restore finished, not launch stream tasks, since stream tasks are disabled", vgId);
|
||||
} else {
|
||||
vInfo("vgId:%d sync restore finished, start to launch stream tasks", pVnode->config.vgId);
|
||||
tqStreamTaskResetStatus(pVnode->pTq->pStreamMeta);
|
||||
vInfo("vgId:%d sync restore finished, start to launch stream task(s)", pVnode->config.vgId);
|
||||
int32_t numOfTasks = 0;
|
||||
tqStreamTaskResetStatus(pMeta, &numOfTasks);
|
||||
|
||||
{
|
||||
if (numOfTasks > 0) {
|
||||
if (pMeta->startInfo.taskStarting == 1) {
|
||||
pMeta->startInfo.restartCount += 1;
|
||||
tqDebug("vgId:%d in start tasks procedure, inc restartCounter by 1, remaining restart:%d", vgId,
|
||||
pMeta->startInfo.restartCount);
|
||||
streamMetaWUnLock(pMeta);
|
||||
} else {
|
||||
pMeta->startInfo.taskStarting = 1;
|
||||
streamMetaWUnLock(pMeta);
|
||||
tqStreamTaskStartAsync(pMeta, &pVnode->msgCb, false);
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
streamMetaWUnLock(pMeta);
|
||||
vInfo("vgId:%d, sync restore finished, not launch stream tasks since not leader", vgId);
|
||||
}
|
||||
|
||||
streamMetaWUnLock(pMeta);
|
||||
}
|
||||
|
||||
static void vnodeBecomeFollower(const SSyncFSM *pFsm) {
|
||||
|
|
|
@ -1023,6 +1023,57 @@ int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo) {
|
|||
return 0;
|
||||
}
|
||||
|
||||
int32_t qResetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo) {
|
||||
SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo;
|
||||
SOperatorInfo* pOperator = pTaskInfo->pRoot;
|
||||
|
||||
while (1) {
|
||||
int32_t type = pOperator->operatorType;
|
||||
if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL || type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL ||
|
||||
type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) {
|
||||
SStreamIntervalOperatorInfo* pInfo = pOperator->info;
|
||||
STimeWindowAggSupp* pSup = &pInfo->twAggSup;
|
||||
|
||||
pSup->calTriggerSaved = 0;
|
||||
pSup->deleteMarkSaved = 0;
|
||||
qInfo("reset stream param for interval: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark);
|
||||
|
||||
} else if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION ||
|
||||
type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION ||
|
||||
type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) {
|
||||
SStreamSessionAggOperatorInfo* pInfo = pOperator->info;
|
||||
STimeWindowAggSupp* pSup = &pInfo->twAggSup;
|
||||
|
||||
pSup->calTriggerSaved = 0;
|
||||
pSup->deleteMarkSaved = 0;
|
||||
qInfo("reset stream param for session: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark);
|
||||
|
||||
} else if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE) {
|
||||
SStreamStateAggOperatorInfo* pInfo = pOperator->info;
|
||||
STimeWindowAggSupp* pSup = &pInfo->twAggSup;
|
||||
|
||||
pSup->calTriggerSaved = 0;
|
||||
pSup->deleteMarkSaved = 0;
|
||||
qInfo("reset stream param for state: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark);
|
||||
|
||||
} else if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT) {
|
||||
SStreamEventAggOperatorInfo* pInfo = pOperator->info;
|
||||
STimeWindowAggSupp* pSup = &pInfo->twAggSup;
|
||||
|
||||
pSup->calTriggerSaved = 0;
|
||||
pSup->deleteMarkSaved = 0;
|
||||
qInfo("save stream param for state: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark);
|
||||
}
|
||||
|
||||
// iterate operator tree
|
||||
if (pOperator->numOfDownstream != 1 || pOperator->pDownstream[0] == NULL) {
|
||||
return 0;
|
||||
} else {
|
||||
pOperator = pOperator->pDownstream[0];
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
int32_t qRestoreStreamOperatorOption(qTaskInfo_t tinfo) {
|
||||
SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo;
|
||||
const char* id = GET_TASKID(pTaskInfo);
|
||||
|
|
|
@ -92,6 +92,9 @@ extern int32_t streamBackendId;
|
|||
extern int32_t streamBackendCfWrapperId;
|
||||
extern int32_t taskDbWrapperId;
|
||||
|
||||
int32_t streamTimerInit();
|
||||
void streamTimerCleanUp();
|
||||
|
||||
void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration);
|
||||
int32_t streamDispatchStreamBlock(SStreamTask* pTask);
|
||||
void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups);
|
||||
|
|
|
@ -23,11 +23,6 @@ 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*);
|
||||
|
||||
|
|
|
@ -21,13 +21,18 @@ void* streamTimer = NULL;
|
|||
int32_t streamTimerInit() {
|
||||
streamTimer = taosTmrInit(1000, 100, 10000, "STREAM");
|
||||
if (streamTimer == NULL) {
|
||||
stError("init stream timer failed, code:%s", tstrerror(terrno));
|
||||
return -1;
|
||||
}
|
||||
|
||||
stInfo("init stream timer, %p", streamTimer);
|
||||
return 0;
|
||||
}
|
||||
|
||||
void streamTimerCleanUp() {
|
||||
stInfo("cleanup stream timer, %p", streamTimer);
|
||||
taosTmrCleanUp(streamTimer);
|
||||
streamTimer = NULL;
|
||||
}
|
||||
|
||||
char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) {
|
||||
|
@ -49,7 +54,7 @@ static void streamSchedByTimer(void* param, void* tmrId) {
|
|||
return;
|
||||
}
|
||||
|
||||
if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__CK) {
|
||||
if (streamTaskGetStatus(pTask)->state == TASK_STATUS__CK) {
|
||||
stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger);
|
||||
} else {
|
||||
if (status == TASK_TRIGGER_STATUS__ACTIVE) {
|
||||
|
@ -57,7 +62,7 @@ static void streamSchedByTimer(void* param, void* tmrId) {
|
|||
if (pTrigger == NULL) {
|
||||
stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory",
|
||||
nextTrigger);
|
||||
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pTimer);
|
||||
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -68,7 +73,7 @@ static void streamSchedByTimer(void* param, void* tmrId) {
|
|||
|
||||
stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory",
|
||||
nextTrigger);
|
||||
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pTimer);
|
||||
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -77,7 +82,7 @@ static void streamSchedByTimer(void* param, void* tmrId) {
|
|||
|
||||
int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger);
|
||||
if (code != TSDB_CODE_SUCCESS) {
|
||||
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pTimer);
|
||||
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -85,17 +90,17 @@ static void streamSchedByTimer(void* param, void* tmrId) {
|
|||
}
|
||||
}
|
||||
|
||||
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pTimer);
|
||||
taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer);
|
||||
}
|
||||
|
||||
int32_t streamSetupScheduleTrigger(SStreamTask* pTask) {
|
||||
if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) {
|
||||
int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1);
|
||||
ASSERT(ref == 2 && pTask->schedInfo.pTimer == NULL);
|
||||
ASSERT(ref == 2 && pTask->schedInfo.pDelayTimer == NULL);
|
||||
|
||||
stDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam);
|
||||
|
||||
pTask->schedInfo.pTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamTimer);
|
||||
pTask->schedInfo.pDelayTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamTimer);
|
||||
pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE;
|
||||
}
|
||||
|
||||
|
@ -247,7 +252,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S
|
|||
}
|
||||
|
||||
// disable the data from upstream tasks
|
||||
if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__HALT) {
|
||||
if (streamTaskGetStatus(pTask)->state == TASK_STATUS__HALT) {
|
||||
status = TASK_INPUT_STATUS__BLOCKED;
|
||||
}
|
||||
|
||||
|
|
|
@ -187,7 +187,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc
|
|||
int32_t code = TSDB_CODE_SUCCESS;
|
||||
|
||||
// set task status
|
||||
if (streamTaskGetStatus(pTask, NULL) != TASK_STATUS__CK) {
|
||||
if (streamTaskGetStatus(pTask)->state != TASK_STATUS__CK) {
|
||||
pTask->chkInfo.checkpointingId = checkpointId;
|
||||
code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_GEN_CHECKPOINT);
|
||||
if (code != TSDB_CODE_SUCCESS) {
|
||||
|
@ -309,8 +309,7 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) {
|
|||
p->chkInfo.checkpointVer = p->chkInfo.processedVer;
|
||||
|
||||
streamTaskClearCheckInfo(p, false);
|
||||
char* str = NULL;
|
||||
streamTaskGetStatus(p, &str);
|
||||
SStreamTaskState* pState = streamTaskGetStatus(p);
|
||||
|
||||
code = streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE);
|
||||
taosThreadMutexUnlock(&p->lock);
|
||||
|
@ -322,7 +321,7 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) {
|
|||
|
||||
stDebug("vgId:%d s-task:%s level:%d open upstream inputQ, save status after checkpoint, checkpointId:%" PRId64
|
||||
", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status: normal, prev:%s",
|
||||
vgId, id, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, str);
|
||||
vgId, id, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, pState->name);
|
||||
|
||||
// save the task if not sink task
|
||||
if (p->info.taskLevel != TASK_LEVEL__SINK) {
|
||||
|
|
|
@ -696,10 +696,9 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) {
|
|||
int32_t numOfVgs = taosArrayGetSize(vgInfo);
|
||||
pTask->notReadyTasks = numOfVgs;
|
||||
|
||||
char* p = NULL;
|
||||
streamTaskGetStatus(pTask, &p);
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pTask);
|
||||
stDebug("s-task:%s send scan-history data complete msg to downstream (shuffle-dispatch) %d tasks, status:%s",
|
||||
pTask->id.idStr, numOfVgs, p);
|
||||
pTask->id.idStr, numOfVgs, pState->name);
|
||||
for (int32_t i = 0; i < numOfVgs; i++) {
|
||||
SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i);
|
||||
req.downstreamTaskId = pVgInfo->taskId;
|
||||
|
@ -819,9 +818,9 @@ int32_t doDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamScanHist
|
|||
initRpcMsg(&msg, TDMT_VND_STREAM_SCAN_HISTORY_FINISH, buf, tlen + sizeof(SMsgHead));
|
||||
|
||||
tmsgSendReq(pEpSet, &msg);
|
||||
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,
|
||||
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pTask);
|
||||
stDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pState->name,
|
||||
pReq->downstreamTaskId, vgId);
|
||||
return 0;
|
||||
}
|
||||
|
|
|
@ -24,12 +24,12 @@
|
|||
static int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask);
|
||||
|
||||
bool streamTaskShouldStop(const SStreamTask* pTask) {
|
||||
ETaskStatus s = streamTaskGetStatus(pTask, NULL);
|
||||
return (s == TASK_STATUS__STOP) || (s == TASK_STATUS__DROPPING);
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pTask);
|
||||
return (pState->state == TASK_STATUS__STOP) || (pState->state == TASK_STATUS__DROPPING);
|
||||
}
|
||||
|
||||
bool streamTaskShouldPause(const SStreamTask* pTask) {
|
||||
return (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__PAUSE);
|
||||
return (streamTaskGetStatus(pTask)->state == TASK_STATUS__PAUSE);
|
||||
}
|
||||
|
||||
static int32_t doOutputResultBlockImpl(SStreamTask* pTask, SStreamDataBlock* pBlock) {
|
||||
|
@ -107,12 +107,6 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i
|
|||
return 0;
|
||||
}
|
||||
|
||||
if (pTask->inputq.status == TASK_INPUT_STATUS__BLOCKED) {
|
||||
stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry exec task", pTask->id.idStr);
|
||||
taosMsleep(1000);
|
||||
continue;
|
||||
}
|
||||
|
||||
SSDataBlock* output = NULL;
|
||||
uint64_t ts = 0;
|
||||
if ((code = qExecTask(pExecutor, &output, &ts)) < 0) {
|
||||
|
@ -344,11 +338,14 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) {
|
|||
streamMetaWUnLock(pMeta);
|
||||
return TSDB_CODE_STREAM_TASK_NOT_EXIST;
|
||||
} else {
|
||||
stDebug("s-task:%s fill-history task end, update related stream task:%s info, transfer exec state", id,
|
||||
pStreamTask->id.idStr);
|
||||
double el = (taosGetTimestampMs() - pTask->execInfo.step2Start) / 1000.;
|
||||
stDebug(
|
||||
"s-task:%s fill-history task end, scal wal elapsed time:%.2fSec,update related stream task:%s info, transfer "
|
||||
"exec state",
|
||||
id, el, pStreamTask->id.idStr);
|
||||
}
|
||||
|
||||
ETaskStatus status = streamTaskGetStatus(pStreamTask, NULL);
|
||||
ETaskStatus status = streamTaskGetStatus(pStreamTask)->state;
|
||||
STimeWindow* pTimeWindow = &pStreamTask->dataRange.window;
|
||||
|
||||
// It must be halted for a source stream task, since when the related scan-history-data task start scan the history
|
||||
|
@ -374,8 +371,10 @@ 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.
|
||||
char* p = NULL;
|
||||
status = streamTaskGetStatus(pStreamTask, &p);
|
||||
// char* p = NULL;
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pStreamTask);
|
||||
status = pState->state;
|
||||
char* p = pState->name;
|
||||
if (status == TASK_STATUS__STOP || status == TASK_STATUS__DROPPING) {
|
||||
stError("s-task:%s failed to transfer state from fill-history task:%s, status:%s", id, pStreamTask->id.idStr, p);
|
||||
streamMetaReleaseTask(pMeta, pStreamTask);
|
||||
|
@ -409,7 +408,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) {
|
|||
streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id);
|
||||
|
||||
// 5. save to disk
|
||||
pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask, NULL);
|
||||
pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask)->state;
|
||||
|
||||
// 6. add empty delete block
|
||||
if ((pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) && taosQueueEmpty(pStreamTask->inputq.queue->pQueue)) {
|
||||
|
@ -549,6 +548,18 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock
|
|||
return code;
|
||||
}
|
||||
|
||||
static void setTaskSchedInfo(SStreamTask* pTask, int32_t idleTime) {
|
||||
SStreamStatus* pStatus = &pTask->status;
|
||||
|
||||
pStatus->schedIdleTime = idleTime;
|
||||
pStatus->lastExecTs = taosGetTimestampMs();
|
||||
}
|
||||
|
||||
static void clearTaskSchedInfo(SStreamTask* pTask) {
|
||||
SStreamStatus* pStatus = &pTask->status;
|
||||
pStatus->schedIdleTime = 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* todo: the batch of blocks should be tuned dynamic, according to the total elapsed time of each batch of blocks, the
|
||||
* appropriate batch of blocks should be handled in 5 to 10 sec.
|
||||
|
@ -563,11 +574,23 @@ int32_t doStreamExecTask(SStreamTask* pTask) {
|
|||
int32_t blockSize = 0;
|
||||
int32_t numOfBlocks = 0;
|
||||
SStreamQueueItem* pInput = NULL;
|
||||
if (streamTaskShouldStop(pTask) || (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__UNINIT)) {
|
||||
if (streamTaskShouldStop(pTask) || (streamTaskGetStatus(pTask)->state == TASK_STATUS__UNINIT)) {
|
||||
stDebug("s-task:%s stream task is stopped", id);
|
||||
break;
|
||||
}
|
||||
|
||||
if (streamQueueIsFull(pTask->outputq.queue)) {
|
||||
stWarn("s-task:%s outputQ is full, idle for 500ms and retry", id);
|
||||
setTaskSchedInfo(pTask, 500);
|
||||
break;
|
||||
}
|
||||
|
||||
if (pTask->inputq.status == TASK_INPUT_STATUS__BLOCKED) {
|
||||
stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry", pTask->id.idStr);
|
||||
setTaskSchedInfo(pTask, 1000);
|
||||
break;
|
||||
}
|
||||
|
||||
/*int32_t code = */ streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks, &blockSize);
|
||||
if (pInput == NULL) {
|
||||
ASSERT(numOfBlocks == 0);
|
||||
|
@ -582,7 +605,7 @@ int32_t doStreamExecTask(SStreamTask* pTask) {
|
|||
continue;
|
||||
}
|
||||
|
||||
if (pInput->type == STREAM_INPUT__TRANS_STATE) {
|
||||
if (type == STREAM_INPUT__TRANS_STATE) {
|
||||
streamProcessTranstateBlock(pTask, (SStreamDataBlock*)pInput);
|
||||
continue;
|
||||
}
|
||||
|
@ -634,10 +657,9 @@ int32_t doStreamExecTask(SStreamTask* pTask) {
|
|||
if (type == STREAM_INPUT__CHECKPOINT) {
|
||||
|
||||
// todo add lock
|
||||
char* p = NULL;
|
||||
ETaskStatus s = streamTaskGetStatus(pTask, &p);
|
||||
if (s == TASK_STATUS__CK) {
|
||||
stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, p);
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pTask);
|
||||
if (pState->state == TASK_STATUS__CK) {
|
||||
stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, pState->name);
|
||||
streamTaskBuildCheckpoint(pTask);
|
||||
} else {
|
||||
// todo refactor
|
||||
|
@ -665,15 +687,112 @@ int32_t doStreamExecTask(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) {
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask)->state;
|
||||
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);
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pTask);
|
||||
|
||||
ETaskStatus st = pState->state;
|
||||
if (pStatus != NULL) {
|
||||
*pStatus = pState->name;
|
||||
}
|
||||
|
||||
return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK);
|
||||
}
|
||||
|
||||
static void doStreamExecTaskHelper(void* param, void* tmrId) {
|
||||
SStreamTask* pTask = (SStreamTask*)param;
|
||||
|
||||
SStreamTaskState* p = streamTaskGetStatus(pTask);
|
||||
if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) {
|
||||
streamTaskSetSchedStatusInactive(pTask);
|
||||
|
||||
int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1);
|
||||
stDebug("s-task:%s status:%s not resume task, ref:%d", pTask->id.idStr, p->name, ref);
|
||||
|
||||
streamMetaReleaseTask(pTask->pMeta, pTask);
|
||||
return;
|
||||
}
|
||||
|
||||
// task resume running
|
||||
SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq));
|
||||
if (pRunReq == NULL) {
|
||||
terrno = TSDB_CODE_OUT_OF_MEMORY;
|
||||
/*int8_t status = */streamTaskSetSchedStatusInactive(pTask);
|
||||
|
||||
int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1);
|
||||
stError("failed to create msg to resume s-task:%s, reason out of memory, ref:%d", pTask->id.idStr, ref);
|
||||
|
||||
streamMetaReleaseTask(pTask->pMeta, pTask);
|
||||
return;
|
||||
}
|
||||
|
||||
pRunReq->head.vgId = pTask->info.nodeId;
|
||||
pRunReq->streamId = pTask->id.streamId;
|
||||
pRunReq->taskId = pTask->id.taskId;
|
||||
pRunReq->reqType = STREAM_EXEC_T_RESUME_TASK;
|
||||
|
||||
int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1);
|
||||
stDebug("trigger to resume s-task:%s after being idled for %dms, ref:%d", pTask->id.idStr, pTask->status.schedIdleTime, ref);
|
||||
|
||||
SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)};
|
||||
tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg);
|
||||
|
||||
// release the task ref count
|
||||
clearTaskSchedInfo(pTask);
|
||||
streamMetaReleaseTask(pTask->pMeta, pTask);
|
||||
}
|
||||
|
||||
static int32_t schedTaskInFuture(SStreamTask* pTask) {
|
||||
int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1);
|
||||
stDebug("s-task:%s task should idle, add into timer to retry in %dms, ref:%d", pTask->id.idStr,
|
||||
pTask->status.schedIdleTime, ref);
|
||||
|
||||
// add one ref count for task
|
||||
// todo this may be failed, and add ref may be failed.
|
||||
SStreamTask* pAddRefTask = streamMetaAcquireTask(pTask->pMeta, pTask->id.streamId, pTask->id.taskId);
|
||||
|
||||
if (pTask->schedInfo.pIdleTimer == NULL) {
|
||||
pTask->schedInfo.pIdleTimer = taosTmrStart(doStreamExecTaskHelper, pTask->status.schedIdleTime, pTask, streamTimer);
|
||||
} else {
|
||||
taosTmrReset(doStreamExecTaskHelper, pTask->status.schedIdleTime, pTask, streamTimer, &pTask->schedInfo.pIdleTimer);
|
||||
}
|
||||
|
||||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
|
||||
int32_t streamResumeTask(SStreamTask* pTask) {
|
||||
ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__ACTIVE);
|
||||
const char* id = pTask->id.idStr;
|
||||
|
||||
while (1) {
|
||||
/*int32_t code = */doStreamExecTask(pTask);
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
|
||||
// check if this task needs to be idle for a while
|
||||
if (pTask->status.schedIdleTime > 0) {
|
||||
stDebug("s-task:%s idled, and will be invoked in %dms", id, pTask->status.schedIdleTime);
|
||||
schedTaskInFuture(pTask);
|
||||
|
||||
taosThreadMutexUnlock(&pTask->lock);
|
||||
return 0;
|
||||
} else {
|
||||
int32_t numOfItems = streamQueueGetNumOfItems(pTask->inputq.queue);
|
||||
if ((numOfItems == 0) || streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) {
|
||||
atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE);
|
||||
taosThreadMutexUnlock(&pTask->lock);
|
||||
|
||||
char* p = streamTaskGetStatus(pTask)->name;
|
||||
stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, p, pTask->status.schedStatus);
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
taosThreadMutexUnlock(&pTask->lock);
|
||||
}
|
||||
}
|
||||
|
||||
int32_t streamExecTask(SStreamTask* pTask) {
|
||||
|
@ -682,29 +801,9 @@ int32_t streamExecTask(SStreamTask* pTask) {
|
|||
|
||||
int8_t schedStatus = streamTaskSetSchedStatusActive(pTask);
|
||||
if (schedStatus == TASK_SCHED_STATUS__WAITING) {
|
||||
while (1) {
|
||||
int32_t code = doStreamExecTask(pTask);
|
||||
if (code < 0) { // todo this status should be removed
|
||||
atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__FAILED);
|
||||
return -1;
|
||||
}
|
||||
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
if ((streamQueueGetNumOfItems(pTask->inputq.queue) == 0) || streamTaskShouldStop(pTask) ||
|
||||
streamTaskShouldPause(pTask)) {
|
||||
atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE);
|
||||
taosThreadMutexUnlock(&pTask->lock);
|
||||
|
||||
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);
|
||||
}
|
||||
streamResumeTask(pTask);
|
||||
} else {
|
||||
char* p = NULL;
|
||||
streamTaskGetStatus(pTask, &p);
|
||||
char* p = streamTaskGetStatus(pTask)->name;
|
||||
stDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, p,
|
||||
pTask->status.schedStatus);
|
||||
}
|
||||
|
|
|
@ -61,9 +61,10 @@ static void streamMetaEnvInit() {
|
|||
streamMetaId = taosOpenRef(64, streamMetaCloseImpl);
|
||||
|
||||
metaRefMgtInit();
|
||||
streamTimerInit();
|
||||
}
|
||||
|
||||
void streamMetaInit() { taosThreadOnce(&streamMetaModuleInit, streamMetaEnvInit); }
|
||||
void streamMetaInit() { taosThreadOnce(&streamMetaModuleInit, streamMetaEnvInit);}
|
||||
|
||||
void streamMetaCleanup() {
|
||||
taosCloseRef(streamBackendId);
|
||||
|
@ -71,6 +72,7 @@ void streamMetaCleanup() {
|
|||
taosCloseRef(streamMetaId);
|
||||
|
||||
metaRefMgtCleanup();
|
||||
streamTimerCleanUp();
|
||||
}
|
||||
|
||||
void metaRefMgtInit() {
|
||||
|
@ -359,7 +361,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF
|
|||
goto _err;
|
||||
}
|
||||
|
||||
pMeta->walScanCounter = 0;
|
||||
pMeta->scanInfo.scanCounter = 0;
|
||||
pMeta->vgId = vgId;
|
||||
pMeta->ahandle = ahandle;
|
||||
pMeta->expandFunc = expandFunc;
|
||||
|
@ -452,7 +454,7 @@ void streamMetaClear(SStreamMeta* pMeta) {
|
|||
// release the ref by timer
|
||||
if (p->info.triggerParam != 0 && p->info.fillHistory == 0) { // one more ref in timer
|
||||
stDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", p->id.idStr, p->refCnt);
|
||||
taosTmrStop(p->schedInfo.pTimer);
|
||||
taosTmrStop(p->schedInfo.pDelayTimer);
|
||||
p->info.triggerParam = 0;
|
||||
streamMetaReleaseTask(pMeta, p);
|
||||
}
|
||||
|
@ -467,6 +469,7 @@ void streamMetaClear(SStreamMeta* pMeta) {
|
|||
taosArrayClear(pMeta->pTaskList);
|
||||
taosArrayClear(pMeta->chkpSaved);
|
||||
taosArrayClear(pMeta->chkpInUse);
|
||||
|
||||
pMeta->numOfStreamTasks = 0;
|
||||
pMeta->numOfPausedTasks = 0;
|
||||
|
||||
|
@ -730,7 +733,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t
|
|||
|
||||
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);
|
||||
taosTmrStop(pTask->schedInfo.pTimer);
|
||||
taosTmrStop(pTask->schedInfo.pDelayTimer);
|
||||
pTask->info.triggerParam = 0;
|
||||
streamMetaReleaseTask(pMeta, pTask);
|
||||
}
|
||||
|
@ -1093,7 +1096,7 @@ static int32_t metaHeartbeatToMnodeImpl(SStreamMeta* pMeta) {
|
|||
|
||||
STaskStatusEntry entry = {
|
||||
.id = *pId,
|
||||
.status = streamTaskGetStatus(*pTask, NULL),
|
||||
.status = streamTaskGetStatus(*pTask)->state,
|
||||
.nodeId = hbMsg.vgId,
|
||||
.stage = pMeta->stage,
|
||||
.inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputq.queue)),
|
||||
|
@ -1262,11 +1265,11 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) {
|
|||
|
||||
// wait for the stream meta hb function stopping
|
||||
if (pMeta->role == NODE_ROLE_LEADER) {
|
||||
pMeta->pHbInfo->stopFlag = STREAM_META_WILL_STOP;
|
||||
while (pMeta->pHbInfo->stopFlag != STREAM_META_OK_TO_STOP) {
|
||||
taosMsleep(100);
|
||||
stDebug("vgId:%d wait for meta to stop timer", pMeta->vgId);
|
||||
}
|
||||
// pMeta->pHbInfo->stopFlag = STREAM_META_WILL_STOP;
|
||||
// while (pMeta->pHbInfo->stopFlag != STREAM_META_OK_TO_STOP) {
|
||||
// taosMsleep(100);
|
||||
// stDebug("vgId:%d wait for meta to stop timer", pMeta->vgId);
|
||||
// }
|
||||
}
|
||||
|
||||
stDebug("vgId:%d start to check all tasks", vgId);
|
||||
|
@ -1362,13 +1365,13 @@ SArray* streamMetaSendMsgBeforeCloseTasks(SStreamMeta* pMeta) {
|
|||
}
|
||||
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
char* p = NULL;
|
||||
ETaskStatus s = streamTaskGetStatus(pTask, &p);
|
||||
if (s == TASK_STATUS__CK) {
|
||||
|
||||
SStreamTaskState* pState = streamTaskGetStatus(pTask);
|
||||
if (pState->state == TASK_STATUS__CK) {
|
||||
streamTaskSetCheckpointFailedId(pTask);
|
||||
stDebug("s-task:%s mark the checkpoint:%"PRId64" failed", pTask->id.idStr, pTask->chkInfo.checkpointingId);
|
||||
} else {
|
||||
stDebug("s-task:%s status:%s not reset the checkpoint", pTask->id.idStr, p);
|
||||
stDebug("s-task:%s status:%s not reset the checkpoint", pTask->id.idStr, pState->name);
|
||||
}
|
||||
|
||||
taosThreadMutexUnlock(&pTask->lock);
|
||||
|
@ -1467,9 +1470,12 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) {
|
|||
continue;
|
||||
}
|
||||
|
||||
// todo: may be we should find the related fill-history task and set it failed.
|
||||
|
||||
// fill-history task can only be launched by related stream tasks.
|
||||
STaskExecStatisInfo* pInfo = &pTask->execInfo;
|
||||
if (pTask->info.fillHistory == 1) {
|
||||
stDebug("s-task:%s fill-history task wait related stream task start", pTask->id.idStr);
|
||||
streamMetaReleaseTask(pMeta, pTask);
|
||||
continue;
|
||||
}
|
||||
|
@ -1486,13 +1492,16 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) {
|
|||
continue;
|
||||
}
|
||||
|
||||
EStreamTaskEvent event = /*(HAS_RELATED_FILLHISTORY_TASK(pTask)) ? TASK_EVENT_INIT_STREAM_SCANHIST : */TASK_EVENT_INIT;
|
||||
int32_t ret = streamTaskHandleEvent(pTask->status.pSM, event);
|
||||
int32_t ret = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_INIT);
|
||||
if (ret != TSDB_CODE_SUCCESS) {
|
||||
stError("vgId:%d failed to handle event:%d", pMeta->vgId, event);
|
||||
stError("vgId:%d failed to handle event:%d", pMeta->vgId, TASK_EVENT_INIT);
|
||||
code = ret;
|
||||
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, pTaskId->streamId, pTaskId->taskId, pInfo->init, pInfo->start, false);
|
||||
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
|
||||
STaskId* pId = &pTask->hTaskInfo.id;
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, pId->streamId, pId->taskId, pInfo->init, pInfo->start, false);
|
||||
}
|
||||
}
|
||||
|
||||
streamMetaReleaseTask(pMeta, pTask);
|
||||
|
@ -1502,3 +1511,42 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) {
|
|||
taosArrayDestroy(pTaskList);
|
||||
return code;
|
||||
}
|
||||
|
||||
int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) {
|
||||
int32_t vgId = pMeta->vgId;
|
||||
stInfo("vgId:%d start to task:0x%x by checking downstream status", vgId, taskId);
|
||||
|
||||
SStreamTask* pTask = streamMetaAcquireTask(pMeta, streamId, taskId);
|
||||
if (pTask == NULL) {
|
||||
stError("vgId:%d failed to acquire task:0x%x during start tasks", pMeta->vgId, taskId);
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, taskId, 0, taosGetTimestampMs(), false);
|
||||
|
||||
streamMetaReleaseTask(pMeta, pTask);
|
||||
return TSDB_CODE_STREAM_TASK_IVLD_STATUS;
|
||||
}
|
||||
|
||||
// todo: may be we should find the related fill-history task and set it failed.
|
||||
|
||||
// fill-history task can only be launched by related stream tasks.
|
||||
STaskExecStatisInfo* pInfo = &pTask->execInfo;
|
||||
if (pTask->info.fillHistory == 1) {
|
||||
streamMetaReleaseTask(pMeta, pTask);
|
||||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
|
||||
ASSERT(pTask->status.downstreamReady == 0);
|
||||
|
||||
int32_t ret = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_INIT);
|
||||
if (ret != TSDB_CODE_SUCCESS) {
|
||||
stError("vgId:%d failed to handle event:%d", pMeta->vgId, TASK_EVENT_INIT);
|
||||
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, taskId, pInfo->init, pInfo->start, false);
|
||||
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
|
||||
STaskId* pId = &pTask->hTaskInfo.id;
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, pId->streamId, pId->taskId, pInfo->init, pInfo->start, false);
|
||||
}
|
||||
}
|
||||
|
||||
streamMetaReleaseTask(pMeta, pTask);
|
||||
return ret;
|
||||
}
|
|
@ -157,6 +157,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
|
|||
*numOfBlocks = 0;
|
||||
*blockSize = 0;
|
||||
|
||||
// todo remove it
|
||||
// no available token in bucket for sink task, let's wait for a little bit
|
||||
if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->outputInfo.pTokenBucket, pTask->id.idStr))) {
|
||||
stDebug("s-task:%s no available token in bucket for sink data, wait for 10ms", id);
|
||||
|
@ -174,6 +175,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
|
|||
if (qItem == NULL) {
|
||||
if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) {
|
||||
taosMsleep(WAIT_FOR_DURATION);
|
||||
// todo remove it
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -343,6 +345,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem)
|
|||
int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) {
|
||||
STaosQueue* pQueue = pTask->outputq.queue->pQueue;
|
||||
|
||||
#if 0
|
||||
// wait for the output queue is available for new data to dispatch
|
||||
while (streamQueueIsFull(pTask->outputq.queue)) {
|
||||
if (streamTaskShouldStop(pTask)) {
|
||||
|
@ -358,6 +361,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc
|
|||
|
||||
taosMsleep(OUTPUT_QUEUE_FULL_WAIT_DURATION);
|
||||
}
|
||||
#endif
|
||||
|
||||
int32_t code = taosWriteQitem(pQueue, pBlock);
|
||||
|
||||
|
@ -367,7 +371,14 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc
|
|||
stError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost",
|
||||
pTask->id.idStr, total + 1, size, tstrerror(code));
|
||||
} else {
|
||||
stDebug("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size);
|
||||
if (streamQueueIsFull(pTask->outputq.queue)) {
|
||||
stWarn(
|
||||
"s-task:%s outputQ is full(outputQ items:%d, size:%.2fMiB), set the output status BLOCKING, wait for 500ms "
|
||||
"after handle this batch of blocks",
|
||||
pTask->id.idStr, total, size);
|
||||
} else {
|
||||
stDebug("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size);
|
||||
}
|
||||
}
|
||||
|
||||
return TSDB_CODE_SUCCESS;
|
||||
|
|
|
@ -48,15 +48,13 @@ static void tryLaunchHistoryTask(void* param, void* tmrId);
|
|||
static void doProcessDownstreamReadyRsp(SStreamTask* pTask);
|
||||
|
||||
int32_t streamTaskSetReady(SStreamTask* pTask) {
|
||||
char* p = NULL;
|
||||
int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, &p);
|
||||
int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask);
|
||||
SStreamTaskState* p = streamTaskGetStatus(pTask);
|
||||
|
||||
if ((status == TASK_STATUS__SCAN_HISTORY/* || status == TASK_STATUS__STREAM_SCAN_HISTORY*/) &&
|
||||
pTask->info.taskLevel != TASK_LEVEL__SOURCE) {
|
||||
if ((p->state == TASK_STATUS__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, p);
|
||||
pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, p->name);
|
||||
}
|
||||
|
||||
ASSERT(pTask->status.downstreamReady == 0);
|
||||
|
@ -65,7 +63,7 @@ int32_t streamTaskSetReady(SStreamTask* pTask) {
|
|||
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, numOfDowns, el, p);
|
||||
pTask->id.idStr, numOfDowns, el, p->name);
|
||||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
|
||||
|
@ -93,12 +91,11 @@ static void doReExecScanhistory(void* param, void* tmrId) {
|
|||
SStreamTask* pTask = param;
|
||||
pTask->schedHistoryInfo.numOfTicks -= 1;
|
||||
|
||||
char* p = NULL;
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, &p);
|
||||
if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) {
|
||||
SStreamTaskState* p = streamTaskGetStatus(pTask);
|
||||
if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) {
|
||||
streamMetaReleaseTask(pTask->pMeta, pTask);
|
||||
int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1);
|
||||
stDebug("s-task:%s status:%s not start scan-history again, ref:%d", pTask->id.idStr, p, ref);
|
||||
stDebug("s-task:%s status:%s not start scan-history again, ref:%d", pTask->id.idStr, p->name, ref);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -155,10 +152,9 @@ static int32_t doStartScanHistoryTask(SStreamTask* pTask) {
|
|||
|
||||
int32_t streamTaskStartScanHistory(SStreamTask* pTask) {
|
||||
int32_t level = pTask->info.taskLevel;
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask)->state;
|
||||
|
||||
ASSERT(pTask->status.downstreamReady == 1 &&
|
||||
((status == TASK_STATUS__SCAN_HISTORY)/* || (status == TASK_STATUS__STREAM_SCAN_HISTORY)*/));
|
||||
ASSERT((pTask->status.downstreamReady == 1) && (status == TASK_STATUS__SCAN_HISTORY));
|
||||
|
||||
if (level == TASK_LEVEL__SOURCE) {
|
||||
return doStartScanHistoryTask(pTask);
|
||||
|
@ -315,7 +311,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_
|
|||
id, upstreamTaskId, vgId, stage, pInfo->stage);
|
||||
// record the checkpoint failure id and sent to mnode
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask)->state;
|
||||
if (status == TASK_STATUS__CK) {
|
||||
streamTaskSetCheckpointFailedId(pTask);
|
||||
}
|
||||
|
@ -325,7 +321,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_
|
|||
if (pInfo->stage != stage) {
|
||||
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask)->state;
|
||||
if (status == TASK_STATUS__CK) {
|
||||
streamTaskSetCheckpointFailedId(pTask);
|
||||
}
|
||||
|
@ -346,20 +342,20 @@ int32_t streamTaskOnNormalTaskReady(SStreamTask* pTask) {
|
|||
streamTaskSetReady(pTask);
|
||||
streamTaskSetRangeStreamCalc(pTask);
|
||||
|
||||
char* p = NULL;
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, &p);
|
||||
ASSERT(status == TASK_STATUS__READY);
|
||||
SStreamTaskState* p = streamTaskGetStatus(pTask);
|
||||
ASSERT(p->state == TASK_STATUS__READY);
|
||||
|
||||
int8_t schedStatus = pTask->status.schedStatus;
|
||||
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
|
||||
int64_t startVer = walReaderGetCurrentVer(pTask->exec.pWalReader);
|
||||
if (startVer == -1) {
|
||||
startVer = pTask->chkInfo.nextProcessVer;
|
||||
}
|
||||
|
||||
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, startVer);
|
||||
stDebug("s-task:%s status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, id, p->name, schedStatus,
|
||||
startVer);
|
||||
} else {
|
||||
stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p, pTask->status.schedStatus);
|
||||
stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p->name, schedStatus);
|
||||
}
|
||||
|
||||
return TSDB_CODE_SUCCESS;
|
||||
|
@ -372,15 +368,14 @@ int32_t streamTaskOnScanhistoryTaskReady(SStreamTask* pTask) {
|
|||
streamTaskSetReady(pTask);
|
||||
streamTaskSetRangeStreamCalc(pTask);
|
||||
|
||||
char* p = NULL;
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, &p);
|
||||
ASSERT(status == TASK_STATUS__SCAN_HISTORY/* || status == TASK_STATUS__STREAM_SCAN_HISTORY*/);
|
||||
SStreamTaskState* p = streamTaskGetStatus(pTask);
|
||||
ASSERT(p->state == TASK_STATUS__SCAN_HISTORY);
|
||||
|
||||
if (pTask->info.fillHistory == 1) {
|
||||
stDebug("s-task:%s fill-history task enters into scan-history data stage, status:%s", id, p);
|
||||
stDebug("s-task:%s fill-history task enters into scan-history data stage, status:%s", id, p->name);
|
||||
streamTaskStartScanHistory(pTask);
|
||||
} else {
|
||||
stDebug("s-task:%s scan wal data, status:%s", id, p);
|
||||
stDebug("s-task:%s scan wal data, status:%s", id, p->name);
|
||||
}
|
||||
|
||||
// NOTE: there will be an deadlock if launch fill history here.
|
||||
|
@ -393,12 +388,7 @@ int32_t streamTaskOnScanhistoryTaskReady(SStreamTask* pTask) {
|
|||
}
|
||||
|
||||
void doProcessDownstreamReadyRsp(SStreamTask* pTask) {
|
||||
EStreamTaskEvent event;
|
||||
if (pTask->info.fillHistory == 0) {
|
||||
event = /*HAS_RELATED_FILLHISTORY_TASK(pTask) ? TASK_EVENT_INIT_STREAM_SCANHIST : */TASK_EVENT_INIT;
|
||||
} else {
|
||||
event = TASK_EVENT_INIT_SCANHIST;
|
||||
}
|
||||
EStreamTaskEvent event = (pTask->info.fillHistory == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCANHIST;
|
||||
|
||||
streamTaskOnHandleEventSuccess(pTask->status.pSM, event);
|
||||
|
||||
|
@ -500,22 +490,14 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs
|
|||
addIntoNodeUpdateList(pTask, pRsp->downstreamNodeId);
|
||||
}
|
||||
|
||||
streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pTask->id.streamId, pTask->id.taskId, pTask->execInfo.init,
|
||||
taosGetTimestampMs(), false);
|
||||
int32_t startTs = pTask->execInfo.init;
|
||||
int64_t now = taosGetTimestampMs();
|
||||
streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pTask->id.streamId, pTask->id.taskId, startTs, now, false);
|
||||
|
||||
// automatically set the related fill-history task to be failed.
|
||||
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
|
||||
STaskId* pId = &pTask->hTaskInfo.id;
|
||||
int64_t current = taosGetTimestampMs();
|
||||
|
||||
SStreamTask* pHTask = streamMetaAcquireTask(pTask->pMeta, pId->streamId, pId->taskId);
|
||||
if (pHTask != NULL) {
|
||||
streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pId->streamId, pId->taskId, pHTask->execInfo.init, current,
|
||||
false);
|
||||
streamMetaReleaseTask(pTask->pMeta, pHTask);
|
||||
} else {
|
||||
streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pId->streamId, pId->taskId, 0, current, false);
|
||||
}
|
||||
streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pId->streamId, pId->taskId, startTs, now, false);
|
||||
}
|
||||
} else { // TASK_DOWNSTREAM_NOT_READY, let's retry in 100ms
|
||||
STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp);
|
||||
|
@ -562,6 +544,15 @@ int32_t streamSetParamForScanHistory(SStreamTask* pTask) {
|
|||
return qSetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor);
|
||||
}
|
||||
|
||||
int32_t streamResetParamForScanHistory(SStreamTask* pTask) {
|
||||
stDebug("s-task:%s reset operator option for scan-history data", pTask->id.idStr);
|
||||
if (pTask->exec.pExecutor != NULL) {
|
||||
return qResetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor);
|
||||
} else {
|
||||
return TSDB_CODE_SUCCESS;
|
||||
}
|
||||
}
|
||||
|
||||
int32_t streamRestoreParam(SStreamTask* pTask) {
|
||||
stDebug("s-task:%s restore operator param after scan-history", pTask->id.idStr);
|
||||
return qRestoreStreamOperatorOption(pTask->exec.pExecutor);
|
||||
|
@ -632,12 +623,11 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory
|
|||
ASSERT(taskLevel == TASK_LEVEL__AGG || taskLevel == TASK_LEVEL__SINK);
|
||||
|
||||
const char* id = pTask->id.idStr;
|
||||
char* p = NULL;
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, &p);
|
||||
SStreamTaskState* p = streamTaskGetStatus(pTask);
|
||||
|
||||
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", id, p,
|
||||
pReq->upstreamTaskId);
|
||||
if (p->state != TASK_STATUS__SCAN_HISTORY) {
|
||||
stError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", id,
|
||||
p->name, pReq->upstreamTaskId);
|
||||
|
||||
void* pBuf = NULL;
|
||||
int32_t len = 0;
|
||||
|
@ -690,7 +680,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory
|
|||
}
|
||||
|
||||
int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) {
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, NULL);
|
||||
ETaskStatus status = streamTaskGetStatus(pTask)->state;
|
||||
|
||||
// task restart now, not handle the scan-history finish rsp
|
||||
if (status == TASK_STATUS__UNINIT) {
|
||||
|
@ -749,8 +739,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) {
|
|||
ASSERT((*ppTask)->status.timerActive >= 1);
|
||||
|
||||
if (streamTaskShouldStop(*ppTask)) {
|
||||
char* p = NULL;
|
||||
streamTaskGetStatus((*ppTask), &p);
|
||||
char* p = streamTaskGetStatus(*ppTask)->name;
|
||||
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, p, (*ppTask)->hTaskInfo.retryTimes, ref);
|
||||
|
@ -789,10 +778,9 @@ 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))) {
|
||||
char* p = NULL;
|
||||
char* p = streamTaskGetStatus(pTask)->name;
|
||||
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);
|
||||
|
||||
|
@ -836,11 +824,23 @@ SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, in
|
|||
// an fill history task needs to be started.
|
||||
int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) {
|
||||
SStreamMeta* pMeta = pTask->pMeta;
|
||||
int32_t hTaskId = pTask->hTaskInfo.id.taskId;
|
||||
|
||||
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);
|
||||
int64_t streamId = pTask->hTaskInfo.id.streamId;
|
||||
int32_t hTaskId = pTask->hTaskInfo.id.taskId;
|
||||
ASSERT(hTaskId != 0);
|
||||
|
||||
SStreamTaskState* pStatus = streamTaskGetStatus(pTask);
|
||||
if (pStatus->state != TASK_STATUS__READY) {
|
||||
STaskExecStatisInfo* pInfo = &pTask->execInfo;
|
||||
stDebug("s-task:%s not launch related fill-history task:0x%" PRIx64 "-0x%x, status:%s", pTask->id.idStr,
|
||||
pTask->hTaskInfo.id.streamId, hTaskId, pStatus->name);
|
||||
|
||||
streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, hTaskId, pInfo->init, pInfo->start, false);
|
||||
return -1;// todo set the correct error code
|
||||
} else {
|
||||
stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr,
|
||||
pTask->hTaskInfo.id.streamId, hTaskId);
|
||||
}
|
||||
|
||||
// Set the execute conditions, including the query time window and the version range
|
||||
SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id));
|
||||
|
@ -886,7 +886,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) {
|
|||
}
|
||||
|
||||
int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) {
|
||||
if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__DROPPING) {
|
||||
if (streamTaskGetStatus(pTask)->state == TASK_STATUS__DROPPING) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
@ -1066,23 +1066,21 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) {
|
|||
}
|
||||
|
||||
void streamTaskResume(SStreamTask* pTask) {
|
||||
char* p = NULL;
|
||||
ETaskStatus status = streamTaskGetStatus(pTask, &p);
|
||||
SStreamTaskState prevState = *streamTaskGetStatus(pTask);
|
||||
SStreamMeta* pMeta = pTask->pMeta;
|
||||
|
||||
if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__HALT) {
|
||||
if (prevState.state == TASK_STATUS__PAUSE || prevState.state == TASK_STATUS__HALT) {
|
||||
streamTaskRestoreStatus(pTask);
|
||||
|
||||
char* pNew = NULL;
|
||||
streamTaskGetStatus(pTask, &pNew);
|
||||
if (status == TASK_STATUS__PAUSE) {
|
||||
char* pNew = streamTaskGetStatus(pTask)->name;
|
||||
if (prevState.state == 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);
|
||||
stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, prevState.name, num);
|
||||
} else {
|
||||
stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, p);
|
||||
stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, prevState.name);
|
||||
}
|
||||
} else {
|
||||
stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, p);
|
||||
stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, prevState.name);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -309,7 +309,9 @@ void tFreeStreamTask(SStreamTask* pTask) {
|
|||
ETaskStatus status1 = TASK_STATUS__UNINIT;
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
if (pTask->status.pSM != NULL) {
|
||||
status1 = streamTaskGetStatus(pTask, &p);
|
||||
SStreamTaskState* pStatus = streamTaskGetStatus(pTask);
|
||||
p = pStatus->name;
|
||||
status1 = pStatus->state;
|
||||
}
|
||||
taosThreadMutexUnlock(&pTask->lock);
|
||||
|
||||
|
@ -328,9 +330,9 @@ void tFreeStreamTask(SStreamTask* pTask) {
|
|||
taosMsleep(100);
|
||||
}
|
||||
|
||||
if (pTask->schedInfo.pTimer != NULL) {
|
||||
taosTmrStop(pTask->schedInfo.pTimer);
|
||||
pTask->schedInfo.pTimer = NULL;
|
||||
if (pTask->schedInfo.pDelayTimer != NULL) {
|
||||
taosTmrStop(pTask->schedInfo.pDelayTimer);
|
||||
pTask->schedInfo.pDelayTimer = NULL;
|
||||
}
|
||||
|
||||
if (pTask->hTaskInfo.pTimer != NULL) {
|
||||
|
|
|
@ -38,7 +38,6 @@ 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"},
|
||||
|
@ -65,8 +64,7 @@ static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus nex
|
|||
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);
|
||||
char* p = streamTaskGetStatus(pTask)->name;
|
||||
|
||||
stDebug("s-task:%s status:%s attach event:%s required status:%s, since not allowed to handle it", pTask->id.idStr, p,
|
||||
GET_EVT_NAME(pEvtInfo->event), StreamTaskStatusList[pEvtInfo->status].name);
|
||||
|
@ -110,12 +108,12 @@ int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask) {
|
|||
|
||||
// todo check rsp code for handle Event:TASK_EVENT_SCANHIST_DONE
|
||||
static bool isInvalidStateTransfer(ETaskStatus state, const EStreamTaskEvent event) {
|
||||
if (/*event == TASK_EVENT_INIT_STREAM_SCANHIST || */event == TASK_EVENT_INIT || event == TASK_EVENT_INIT_SCANHIST) {
|
||||
if (event == TASK_EVENT_INIT || event == TASK_EVENT_INIT_SCANHIST) {
|
||||
return (state != TASK_STATUS__UNINIT);
|
||||
}
|
||||
|
||||
if (event == TASK_EVENT_SCANHIST_DONE) {
|
||||
return (state != TASK_STATUS__SCAN_HISTORY/* && state != TASK_STATUS__STREAM_SCAN_HISTORY*/);
|
||||
return (state != TASK_STATUS__SCAN_HISTORY);
|
||||
}
|
||||
|
||||
if (event == TASK_EVENT_GEN_CHECKPOINT) {
|
||||
|
@ -275,7 +273,7 @@ static int32_t doHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event, STaskSt
|
|||
while (1) {
|
||||
// wait for the task to be here
|
||||
taosThreadMutexLock(&pTask->lock);
|
||||
ETaskStatus s = streamTaskGetStatus(pTask, NULL);
|
||||
ETaskStatus s = streamTaskGetStatus(pTask)->state;
|
||||
taosThreadMutexUnlock(&pTask->lock);
|
||||
|
||||
if ((s == pTrans->next.state) && (pSM->prev.evt == pTrans->event)) {// this event has been handled already
|
||||
|
@ -400,12 +398,8 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM, EStreamTaskEvent even
|
|||
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;
|
||||
SStreamTaskState* streamTaskGetStatus(const SStreamTask* pTask) {
|
||||
return &pTask->status.pSM->current; // copy one obj in case of multi-thread environment
|
||||
}
|
||||
|
||||
const char* streamTaskGetStatusStr(ETaskStatus status) {
|
||||
|
@ -482,14 +476,10 @@ void doInitStateTransferTable(void) {
|
|||
taosArrayPush(streamTaskSMTrans, &trans);
|
||||
trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__SCAN_HISTORY, TASK_EVENT_INIT_SCANHIST, streamTaskInitStatus, streamTaskOnScanhistoryTaskReady, false, false);
|
||||
taosArrayPush(streamTaskSMTrans, &trans);
|
||||
// trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__STREAM_SCAN_HISTORY, TASK_EVENT_INIT_STREAM_SCANHIST, streamTaskInitStatus, streamTaskOnScanhistoryTaskReady, false, false);
|
||||
// taosArrayPush(streamTaskSMTrans, &trans);
|
||||
|
||||
// scan-history related event
|
||||
trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, NULL, NULL, NULL, true);
|
||||
taosArrayPush(streamTaskSMTrans, &trans);
|
||||
// trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, NULL, 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);
|
||||
|
@ -499,8 +489,6 @@ void doInitStateTransferTable(void) {
|
|||
|
||||
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);
|
||||
|
@ -519,8 +507,6 @@ void doInitStateTransferTable(void) {
|
|||
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);
|
||||
|
@ -554,8 +540,6 @@ void doInitStateTransferTable(void) {
|
|||
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);
|
||||
|
@ -574,7 +558,5 @@ void doInitStateTransferTable(void) {
|
|||
taosArrayPush(streamTaskSMTrans, &trans);
|
||||
trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, streamTaskSendTransSuccessMsg, 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);
|
||||
}
|
||||
//clang-format on
|
|
@ -26,6 +26,7 @@
|
|||
#define FLUSH_NUM 4
|
||||
#define DEFAULT_MAX_STREAM_BUFFER_SIZE (128 * 1024 * 1024)
|
||||
#define MIN_NUM_OF_ROW_BUFF 10240
|
||||
#define MIN_NUM_OF_RECOVER_ROW_BUFF 128
|
||||
|
||||
#define TASK_KEY "streamFileState"
|
||||
#define STREAM_STATE_INFO_NAME "StreamStateCheckPoint"
|
||||
|
@ -660,7 +661,7 @@ int32_t recoverSesssion(SStreamFileState* pFileState, int64_t ckId) {
|
|||
if (pCur == NULL) {
|
||||
return -1;
|
||||
}
|
||||
int32_t recoverNum = TMIN(MIN_NUM_OF_ROW_BUFF, pFileState->maxRowCount);
|
||||
int32_t recoverNum = TMIN(MIN_NUM_OF_RECOVER_ROW_BUFF, pFileState->maxRowCount);
|
||||
while (code == TSDB_CODE_SUCCESS) {
|
||||
if (pFileState->curRowCount >= recoverNum) {
|
||||
break;
|
||||
|
@ -694,7 +695,7 @@ int32_t recoverSnapshot(SStreamFileState* pFileState, int64_t ckId) {
|
|||
if (pCur == NULL) {
|
||||
return -1;
|
||||
}
|
||||
int32_t recoverNum = TMIN(MIN_NUM_OF_ROW_BUFF, pFileState->maxRowCount);
|
||||
int32_t recoverNum = TMIN(MIN_NUM_OF_RECOVER_ROW_BUFF, pFileState->maxRowCount);
|
||||
while (code == TSDB_CODE_SUCCESS) {
|
||||
if (pFileState->curRowCount >= recoverNum) {
|
||||
break;
|
||||
|
|
|
@ -305,8 +305,8 @@ int32_t walFetchHead(SWalReader *pRead, int64_t ver) {
|
|||
}
|
||||
|
||||
int32_t walSkipFetchBody(SWalReader *pRead) {
|
||||
wDebug("vgId:%d, skip fetch body %" PRId64 ", first ver:%" PRId64 ", commit ver:%" PRId64 ", last ver:%" PRId64
|
||||
", applied ver:%" PRId64", 0x%"PRIx64,
|
||||
wDebug("vgId:%d, skip fetch body:%" PRId64 ", first:%" PRId64 ", commit:%" PRId64 ", last:%" PRId64
|
||||
", applied:%" PRId64 ", 0x%" PRIx64,
|
||||
pRead->pWal->cfg.vgId, pRead->pHead->head.version, pRead->pWal->vers.firstVer, pRead->pWal->vers.commitVer,
|
||||
pRead->pWal->vers.lastVer, pRead->pWal->vers.appliedVer, pRead->readerId);
|
||||
|
||||
|
@ -325,11 +325,11 @@ int32_t walFetchBody(SWalReader *pRead) {
|
|||
int64_t ver = pReadHead->version;
|
||||
int32_t vgId = pRead->pWal->cfg.vgId;
|
||||
int64_t id = pRead->readerId;
|
||||
SWalVer *pVer = &pRead->pWal->vers;
|
||||
|
||||
wDebug("vgId:%d, fetch body %" PRId64 ", first ver:%" PRId64 ", commit ver:%" PRId64 ", last ver:%" PRId64
|
||||
", applied ver:%" PRId64 ", 0x%" PRIx64,
|
||||
vgId, ver, pRead->pWal->vers.firstVer, pRead->pWal->vers.commitVer, pRead->pWal->vers.lastVer,
|
||||
pRead->pWal->vers.appliedVer, id);
|
||||
wDebug("vgId:%d, fetch body:%" PRId64 ", first:%" PRId64 ", commit:%" PRId64 ", last:%" PRId64 ", applied:%" PRId64
|
||||
", 0x%" PRIx64,
|
||||
vgId, ver, pVer->firstVer, pVer->commitVer, pVer->lastVer, pVer->appliedVer, id);
|
||||
|
||||
if (pRead->capacity < pReadHead->bodyLen) {
|
||||
SWalCkHead *ptr = (SWalCkHead *)taosMemoryRealloc(pRead->pHead, sizeof(SWalCkHead) + pReadHead->bodyLen);
|
||||
|
|
Loading…
Reference in New Issue