提交 637a1c31 编写于 作者: X Xiaoyu Wang

Merge remote-tracking branch 'origin/3.0' into feature/3.0_wxy

...@@ -461,7 +461,7 @@ int32_t qKillQuery(qinfo_t qinfo) { ...@@ -461,7 +461,7 @@ int32_t qKillQuery(qinfo_t qinfo) {
} }
qDebug("QInfo:0x%"PRIx64" query killed", pQInfo->qId); qDebug("QInfo:0x%"PRIx64" query killed", pQInfo->qId);
setQueryKilled(pQInfo); setTaskKilled(pQInfo);
// Wait for the query executing thread being stopped/ // Wait for the query executing thread being stopped/
// Once the query is stopped, the owner of qHandle will be cleared immediately. // Once the query is stopped, the owner of qHandle will be cleared immediately.
...@@ -634,7 +634,7 @@ int32_t qKillQueryByQId(void* pMgmt, int64_t qId, int32_t waitMs, int32_t waitCo ...@@ -634,7 +634,7 @@ int32_t qKillQueryByQId(void* pMgmt, int64_t qId, int32_t waitMs, int32_t waitCo
return TSDB_CODE_QRY_INVALID_QHANDLE; return TSDB_CODE_QRY_INVALID_QHANDLE;
} }
qWarn("QId:0x%"PRIx64" be killed(no memory commit).", pQInfo->qId); qWarn("QId:0x%"PRIx64" be killed(no memory commit).", pQInfo->qId);
setQueryKilled(pQInfo); setTaskKilled(pQInfo);
// wait query stop // wait query stop
int32_t loop = 0; int32_t loop = 0;
......
...@@ -1241,6 +1241,12 @@ typedef struct { ...@@ -1241,6 +1241,12 @@ typedef struct {
char data[]; char data[];
} SVShowTablesFetchRsp; } SVShowTablesFetchRsp;
typedef struct SMqCMGetSubEpReq {
int64_t consumerId;
int32_t epoch;
char cgroup[TSDB_CONSUMER_GROUP_LEN];
} SMqCMGetSubEpReq;
#pragma pack(pop) #pragma pack(pop)
static FORCE_INLINE int32_t tEncodeSMsgHead(void** buf, const SMsgHead* pMsg) { static FORCE_INLINE int32_t tEncodeSMsgHead(void** buf, const SMsgHead* pMsg) {
...@@ -1562,7 +1568,7 @@ static FORCE_INLINE int32_t tEncodeSMqSetCVgReq(void** buf, const SMqSetCVgReq* ...@@ -1562,7 +1568,7 @@ static FORCE_INLINE int32_t tEncodeSMqSetCVgReq(void** buf, const SMqSetCVgReq*
tlen += taosEncodeString(buf, pReq->logicalPlan); tlen += taosEncodeString(buf, pReq->logicalPlan);
tlen += taosEncodeString(buf, pReq->physicalPlan); tlen += taosEncodeString(buf, pReq->physicalPlan);
tlen += taosEncodeFixedU32(buf, pReq->qmsgLen); tlen += taosEncodeFixedU32(buf, pReq->qmsgLen);
tlen += taosEncodeBinary(buf, pReq->qmsg, pReq->qmsgLen); tlen += taosEncodeString(buf, (char*)pReq->qmsg);
//tlen += tEncodeSSubQueryMsg(buf, &pReq->msg); //tlen += tEncodeSSubQueryMsg(buf, &pReq->msg);
return tlen; return tlen;
} }
...@@ -1577,7 +1583,7 @@ static FORCE_INLINE void* tDecodeSMqSetCVgReq(void* buf, SMqSetCVgReq* pReq) { ...@@ -1577,7 +1583,7 @@ static FORCE_INLINE void* tDecodeSMqSetCVgReq(void* buf, SMqSetCVgReq* pReq) {
buf = taosDecodeString(buf, &pReq->logicalPlan); buf = taosDecodeString(buf, &pReq->logicalPlan);
buf = taosDecodeString(buf, &pReq->physicalPlan); buf = taosDecodeString(buf, &pReq->physicalPlan);
buf = taosDecodeFixedU32(buf, &pReq->qmsgLen); buf = taosDecodeFixedU32(buf, &pReq->qmsgLen);
buf = taosDecodeBinary(buf, &pReq->qmsg, pReq->qmsgLen); buf = taosDecodeString(buf, (char**)&pReq->qmsg);
//buf = tDecodeSSubQueryMsg(buf, &pReq->msg); //buf = tDecodeSSubQueryMsg(buf, &pReq->msg);
return buf; return buf;
} }
...@@ -1639,6 +1645,92 @@ typedef struct SMqConsumeReq { ...@@ -1639,6 +1645,92 @@ typedef struct SMqConsumeReq {
char topic[TSDB_TOPIC_FNAME_LEN]; char topic[TSDB_TOPIC_FNAME_LEN];
} SMqConsumeReq; } SMqConsumeReq;
typedef struct SMqSubVgEp {
int32_t vgId;
SEpSet epSet;
} SMqSubVgEp;
typedef struct SMqSubTopicEp {
char topic[TSDB_TOPIC_FNAME_LEN];
SArray* vgs; // SArray<SMqSubVgEp>
} SMqSubTopicEp;
typedef struct SMqCMGetSubEpRsp {
int64_t consumerId;
char cgroup[TSDB_CONSUMER_GROUP_LEN];
SArray* topics; // SArray<SMqSubTopicEp>
} SMqCMGetSubEpRsp;
static FORCE_INLINE int32_t tEncodeSMqSubVgEp(void** buf, const SMqSubVgEp* pVgEp) {
int32_t tlen = 0;
tlen += taosEncodeFixedI16(buf, pVgEp->vgId);
tlen += taosEncodeSEpSet(buf, &pVgEp->epSet);
return tlen;
}
static FORCE_INLINE void* tDecodeSMqSubVgEp(void* buf, SMqSubVgEp* pVgEp) {
buf = taosDecodeFixedI32(buf, &pVgEp->vgId);
buf = taosDecodeSEpSet(buf, &pVgEp->epSet);
return buf;
}
static FORCE_INLINE int32_t tEncodeSMqSubTopicEp(void** buf, const SMqSubTopicEp* pTopicEp) {
int32_t tlen = 0;
tlen += taosEncodeString(buf, pTopicEp->topic);
int32_t sz = taosArrayGetSize(pTopicEp->vgs);
tlen += taosEncodeFixedI32(buf, sz);
for (int32_t i = 0; i < sz; i++) {
SMqSubVgEp* pVgEp = (SMqSubVgEp*)taosArrayGet(pTopicEp->vgs, i);
tlen += tEncodeSMqSubVgEp(buf, pVgEp);
}
return tlen;
}
static FORCE_INLINE void* tDecodeSMqSubTopicEp(void* buf, SMqSubTopicEp* pTopicEp) {
buf = taosDecodeStringTo(buf, pTopicEp->topic);
int32_t sz;
buf = taosDecodeFixedI32(buf, &sz);
pTopicEp->vgs = taosArrayInit(sz, sizeof(SMqSubVgEp));
if (pTopicEp->vgs == NULL) {
return NULL;
}
for (int32_t i = 0; i < sz; i++) {
SMqSubVgEp vgEp;
buf = tDecodeSMqSubVgEp(buf, &vgEp);
taosArrayPush(pTopicEp->vgs, &vgEp);
}
return buf;
}
static FORCE_INLINE int32_t tEncodeSMqCMGetSubEpRsp(void** buf, const SMqCMGetSubEpRsp* pRsp) {
int32_t tlen = 0;
tlen += taosEncodeFixedI64(buf, pRsp->consumerId);
tlen += taosEncodeString(buf, pRsp->cgroup);
int32_t sz = taosArrayGetSize(pRsp->topics);
tlen += taosEncodeFixedI32(buf, sz);
for (int32_t i = 0; i < sz; i++) {
SMqSubTopicEp* pVgEp = (SMqSubTopicEp*)taosArrayGet(pRsp->topics, i);
tlen += tEncodeSMqSubTopicEp(buf, pVgEp);
}
return tlen;
}
static FORCE_INLINE void* tDecodeSMqCMGetSubEpRsp(void* buf, SMqCMGetSubEpRsp* pRsp) {
buf = taosDecodeFixedI64(buf, &pRsp->consumerId);
buf = taosDecodeStringTo(buf, pRsp->cgroup);
int32_t sz;
buf = taosDecodeFixedI32(buf, &sz);
pRsp->topics = taosArrayInit(sz, sizeof(SMqSubTopicEp));
if (pRsp->topics == NULL) {
return NULL;
}
for (int32_t i = 0; i < sz; i++) {
SMqSubTopicEp topicEp;
buf = tDecodeSMqSubTopicEp(buf, &topicEp);
taosArrayPush(pRsp->topics, &topicEp);
}
return buf;
}
#ifdef __cplusplus #ifdef __cplusplus
} }
......
...@@ -140,6 +140,7 @@ enum { ...@@ -140,6 +140,7 @@ enum {
TD_DEF_MSG_TYPE(TDMT_MND_ALTER_TOPIC, "mnode-alter-topic", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_ALTER_TOPIC, "mnode-alter-topic", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_MND_DROP_TOPIC, "mnode-drop-topic", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_DROP_TOPIC, "mnode-drop-topic", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_MND_SUBSCRIBE, "mnode-subscribe", SCMSubscribeReq, SCMSubscribeRsp) TD_DEF_MSG_TYPE(TDMT_MND_SUBSCRIBE, "mnode-subscribe", SCMSubscribeReq, SCMSubscribeRsp)
TD_DEF_MSG_TYPE(TDMT_MND_GET_SUB_EP, "mnode-get-sub-ep", SMqCMGetSubEpReq, SMqCMGetSubEpRsp)
TD_DEF_MSG_TYPE(TDMT_MND_MQ_TIMER, "mnode-timer", SMqTmrMsg, SMqTmrMsg) TD_DEF_MSG_TYPE(TDMT_MND_MQ_TIMER, "mnode-timer", SMqTmrMsg, SMqTmrMsg)
// Requests handled by VNODE // Requests handled by VNODE
......
...@@ -45,7 +45,7 @@ int32_t qSetStreamInput(qTaskInfo_t tinfo, void* input); ...@@ -45,7 +45,7 @@ int32_t qSetStreamInput(qTaskInfo_t tinfo, void* input);
* @param qId * @param qId
* @return * @return
*/ */
int32_t qCreateExecTask(void* readHandle, int32_t vgId, struct SSubplan* pPlan, qTaskInfo_t* pTaskInfo, DataSinkHandle* handle); int32_t qCreateExecTask(void* readHandle, int32_t vgId, uint64_t taskId, struct SSubplan* pPlan, qTaskInfo_t* pTaskInfo, DataSinkHandle* handle);
/** /**
* The main task execution function, including query on both table and multiple tables, * The main task execution function, including query on both table and multiple tables,
......
...@@ -51,6 +51,7 @@ extern "C" { ...@@ -51,6 +51,7 @@ extern "C" {
#include <libgen.h> #include <libgen.h>
#include <sys/mman.h> #include <sys/mman.h>
#include <sys/prctl.h>
#include "osAtomic.h" #include "osAtomic.h"
#include "osDef.h" #include "osDef.h"
......
...@@ -181,7 +181,8 @@ extern "C" { ...@@ -181,7 +181,8 @@ extern "C" {
#endif #endif
#else #else
// Windows // Windows
#define setThreadName(name) // #define setThreadName(name)
#define setThreadName(name) do { prctl(PR_SET_NAME, (name)); } while (0)
#endif #endif
#if defined(_WIN32) #if defined(_WIN32)
......
...@@ -15,6 +15,7 @@ ...@@ -15,6 +15,7 @@
#ifndef _TD_UTIL_QUEUE_H #ifndef _TD_UTIL_QUEUE_H
#define _TD_UTIL_QUEUE_H #define _TD_UTIL_QUEUE_H
#include "os.h"
#ifdef __cplusplus #ifdef __cplusplus
extern "C" { extern "C" {
...@@ -40,13 +41,13 @@ shall be used to set up the protection. ...@@ -40,13 +41,13 @@ shall be used to set up the protection.
typedef struct STaosQueue STaosQueue; typedef struct STaosQueue STaosQueue;
typedef struct STaosQset STaosQset; typedef struct STaosQset STaosQset;
typedef struct STaosQall STaosQall; typedef struct STaosQall STaosQall;
typedef void (*FProcessItem)(void *ahandle, void *pItem); typedef void (*FItem)(void *ahandle, void *pItem);
typedef void (*FProcessItems)(void *ahandle, STaosQall *qall, int32_t numOfItems); typedef void (*FItems)(void *ahandle, STaosQall *qall, int32_t numOfItems);
STaosQueue *taosOpenQueue(); STaosQueue *taosOpenQueue();
void taosCloseQueue(STaosQueue *queue); void taosCloseQueue(STaosQueue *queue);
void taosSetQueueFp(STaosQueue *queue, FProcessItem itemFp, FProcessItems itemsFp); void taosSetQueueFp(STaosQueue *queue, FItem itemFp, FItems itemsFp);
void *taosAllocateQitem(int32_t size); void * taosAllocateQitem(int32_t size);
void taosFreeQitem(void *pItem); void taosFreeQitem(void *pItem);
int32_t taosWriteQitem(STaosQueue *queue, void *pItem); int32_t taosWriteQitem(STaosQueue *queue, void *pItem);
int32_t taosReadQitem(STaosQueue *queue, void **ppItem); int32_t taosReadQitem(STaosQueue *queue, void **ppItem);
...@@ -66,8 +67,11 @@ int32_t taosAddIntoQset(STaosQset *qset, STaosQueue *queue, void *ahandle); ...@@ -66,8 +67,11 @@ int32_t taosAddIntoQset(STaosQset *qset, STaosQueue *queue, void *ahandle);
void taosRemoveFromQset(STaosQset *qset, STaosQueue *queue); void taosRemoveFromQset(STaosQset *qset, STaosQueue *queue);
int32_t taosGetQueueNumber(STaosQset *qset); int32_t taosGetQueueNumber(STaosQset *qset);
int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, void **ahandle, FProcessItem *itemFp); int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, void **ahandle, FItem *itemFp);
int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, void **ahandle, FProcessItems *itemsFp); int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, void **ahandle, FItems *itemsFp);
int32_t taosReadQitemFromQsetByThread(STaosQset *qset, void **ppItem, void **ahandle, FItem *itemFp, int32_t threadId);
void taosResetQsetThread(STaosQset *qset, void *pItem);
int32_t taosGetQueueItemsNumber(STaosQueue *queue); int32_t taosGetQueueItemsNumber(STaosQueue *queue);
int32_t taosGetQsetItemsNumber(STaosQset *qset); int32_t taosGetQsetItemsNumber(STaosQset *qset);
......
...@@ -15,57 +15,61 @@ ...@@ -15,57 +15,61 @@
#ifndef _TD_UTIL_WORKER_H #ifndef _TD_UTIL_WORKER_H
#define _TD_UTIL_WORKER_H #define _TD_UTIL_WORKER_H
#include "tqueue.h" #include "tqueue.h"
#ifdef __cplusplus #ifdef __cplusplus
extern "C" { extern "C" {
#endif #endif
typedef struct SWorkerPool SWorkerPool; typedef struct SQWorkerPool SQWorkerPool;
typedef struct SMWorkerPool SMWorkerPool; typedef struct SWWorkerPool SWWorkerPool;
typedef struct SWorker { typedef struct SQWorker {
int32_t id; // worker ID int32_t id; // worker ID
pthread_t thread; // thread pthread_t thread; // thread
SWorkerPool *pool; SQWorkerPool *pool;
} SWorker; } SQWorker, SFWorker;
typedef struct SWorkerPool { typedef struct SQWorkerPool {
int32_t max; // max number of workers int32_t max; // max number of workers
int32_t min; // min number of workers int32_t min; // min number of workers
int32_t num; // current number of workers int32_t num; // current number of workers
STaosQset *qset; STaosQset * qset;
const char *name; const char * name;
SWorker *workers; SQWorker * workers;
pthread_mutex_t mutex; pthread_mutex_t mutex;
} SWorkerPool; } SQWorkerPool, SFWorkerPool;
typedef struct SMWorker { typedef struct SWWorker {
int32_t id; // worker id int32_t id; // worker id
pthread_t thread; // thread pthread_t thread; // thread
STaosQall *qall; STaosQall * qall;
STaosQset *qset; // queue set STaosQset * qset; // queue set
SMWorkerPool *pool; SWWorkerPool *pool;
} SMWorker; } SWWorker;
typedef struct SMWorkerPool { typedef struct SWWorkerPool {
int32_t max; // max number of workers int32_t max; // max number of workers
int32_t nextId; // from 0 to max-1, cyclic int32_t nextId; // from 0 to max-1, cyclic
const char *name; const char * name;
SMWorker *workers; SWWorker * workers;
pthread_mutex_t mutex; pthread_mutex_t mutex;
} SMWorkerPool; } SWWorkerPool;
int32_t tQWorkerInit(SQWorkerPool *pool);
void tQWorkerCleanup(SQWorkerPool *pool);
STaosQueue *tQWorkerAllocQueue(SQWorkerPool *pool, void *ahandle, FItem fp);
void tQWorkerFreeQueue(SQWorkerPool *pool, STaosQueue *queue);
int32_t tWorkerInit(SWorkerPool *pool); int32_t tFWorkerInit(SFWorkerPool *pool);
void tWorkerCleanup(SWorkerPool *pool); void tFWorkerCleanup(SFWorkerPool *pool);
STaosQueue *tWorkerAllocQueue(SWorkerPool *pool, void *ahandle, FProcessItem fp); STaosQueue *tFWorkerAllocQueue(SFWorkerPool *pool, void *ahandle, FItem fp);
void tWorkerFreeQueue(SWorkerPool *pool, STaosQueue *queue); void tFWorkerFreeQueue(SFWorkerPool *pool, STaosQueue *queue);
int32_t tMWorkerInit(SMWorkerPool *pool); int32_t tWWorkerInit(SWWorkerPool *pool);
void tMWorkerCleanup(SMWorkerPool *pool); void tWWorkerCleanup(SWWorkerPool *pool);
STaosQueue *tMWorkerAllocQueue(SMWorkerPool *pool, void *ahandle, FProcessItems fp); STaosQueue *tWWorkerAllocQueue(SWWorkerPool *pool, void *ahandle, FItems fp);
void tMWorkerFreeQueue(SMWorkerPool *pool, STaosQueue *queue); void tWWorkerFreeQueue(SWWorkerPool *pool, STaosQueue *queue);
#ifdef __cplusplus #ifdef __cplusplus
} }
......
...@@ -263,7 +263,7 @@ int32_t scheduleQuery(SRequestObj* pRequest, SQueryDag* pDag, SArray* pNodeList) ...@@ -263,7 +263,7 @@ int32_t scheduleQuery(SRequestObj* pRequest, SQueryDag* pDag, SArray* pNodeList)
typedef struct SMqClientVg { typedef struct SMqClientVg {
// statistics // statistics
int64_t consumeCnt; int64_t pollCnt;
// offset // offset
int64_t committedOffset; int64_t committedOffset;
int64_t currentOffset; int64_t currentOffset;
...@@ -345,6 +345,7 @@ tmq_t* taos_consumer_new(void* conn, tmq_conf_t* conf, char* errstr, int32_t err ...@@ -345,6 +345,7 @@ tmq_t* taos_consumer_new(void* conn, tmq_conf_t* conf, char* errstr, int32_t err
strcpy(pTmq->groupId, conf->groupId); strcpy(pTmq->groupId, conf->groupId);
pTmq->commit_cb = conf->commit_cb; pTmq->commit_cb = conf->commit_cb;
pTmq->consumerId = generateRequestId() & ((uint64_t)-1 >> 1); pTmq->consumerId = generateRequestId() & ((uint64_t)-1 >> 1);
pTmq->clientTopics = taosArrayInit(0, sizeof(SMqClientTopic));
return pTmq; return pTmq;
} }
...@@ -411,13 +412,12 @@ TAOS_RES* tmq_subscribe(tmq_t* tmq, tmq_list_t* topic_list) { ...@@ -411,13 +412,12 @@ TAOS_RES* tmq_subscribe(tmq_t* tmq, tmq_list_t* topic_list) {
tSerializeSCMSubscribeReq(&abuf, &req); tSerializeSCMSubscribeReq(&abuf, &req);
/*printf("formatted: %s\n", dagStr);*/ /*printf("formatted: %s\n", dagStr);*/
pRequest = createRequest(tmq->pTscObj, NULL, NULL, TSDB_SQL_SELECT); pRequest = createRequest(tmq->pTscObj, NULL, NULL, TDMT_MND_SUBSCRIBE);
if (pRequest == NULL) { if (pRequest == NULL) {
tscError("failed to malloc sqlObj"); tscError("failed to malloc sqlObj");
} }
pRequest->body.requestMsg = (SDataBuf){ .pData = buf, .len = tlen }; pRequest->body.requestMsg = (SDataBuf){ .pData = buf, .len = tlen };
pRequest->type = TDMT_MND_SUBSCRIBE;
SMsgSendInfo* body = buildMsgInfoImpl(pRequest); SMsgSendInfo* body = buildMsgInfoImpl(pRequest);
SEpSet epSet = getEpSet_s(&tmq->pTscObj->pAppInfo->mgmtEp); SEpSet epSet = getEpSet_s(&tmq->pTscObj->pAppInfo->mgmtEp);
...@@ -596,6 +596,37 @@ struct tmq_message_t { ...@@ -596,6 +596,37 @@ struct tmq_message_t {
SMqConsumeRsp rsp; SMqConsumeRsp rsp;
}; };
int32_t tmq_poll_cb_inner(void* param, const SDataBuf* pMsg, int32_t code) {
return 0;
}
int32_t tmq_ask_ep_cb(void* param, const SDataBuf* pMsg, int32_t code) {
tmq_t* tmq = (tmq_t*)param;
SMqCMGetSubEpRsp rsp;
tDecodeSMqCMGetSubEpRsp(pMsg->pData, &rsp);
int32_t sz = taosArrayGetSize(rsp.topics);
// TODO: lock
tmq->clientTopics = taosArrayInit(sz, sizeof(SMqClientTopic));
for (int32_t i = 0; i < sz; i++) {
SMqClientTopic topic = {0};
SMqSubTopicEp* pTopicEp = taosArrayGet(rsp.topics, i);
topic.topicName = strdup(pTopicEp->topic);
int32_t vgSz = taosArrayGetSize(pTopicEp->vgs);
topic.vgs = taosArrayInit(vgSz, sizeof(SMqClientVg));
for (int32_t j = 0; j < vgSz; j++) {
SMqSubVgEp* pVgEp = taosArrayGet(pTopicEp->vgs, j);
SMqClientVg clientVg = {
.vgId = pVgEp->vgId,
.epSet = pVgEp->epSet
};
taosArrayPush(topic.vgs, &clientVg);
}
taosArrayPush(tmq->clientTopics, &topic);
}
// unlock
return 0;
}
tmq_message_t* tmq_consume_poll(tmq_t* tmq, int64_t blocking_time) { tmq_message_t* tmq_consume_poll(tmq_t* tmq, int64_t blocking_time) {
if (tmq->clientTopics == NULL || taosArrayGetSize(tmq->clientTopics) == 0) { if (tmq->clientTopics == NULL || taosArrayGetSize(tmq->clientTopics) == 0) {
return NULL; return NULL;
...@@ -605,9 +636,38 @@ tmq_message_t* tmq_consume_poll(tmq_t* tmq, int64_t blocking_time) { ...@@ -605,9 +636,38 @@ tmq_message_t* tmq_consume_poll(tmq_t* tmq, int64_t blocking_time) {
req.reqType = 1; req.reqType = 1;
req.blockingTime = blocking_time; req.blockingTime = blocking_time;
req.consumerId = tmq->consumerId; req.consumerId = tmq->consumerId;
tmq_message_t* tmq_message = NULL;
strcpy(req.cgroup, tmq->groupId); strcpy(req.cgroup, tmq->groupId);
SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, tmq->nextTopicIdx); if (taosArrayGetSize(tmq->clientTopics) == 0) {
int32_t tlen = sizeof(SMqCMGetSubEpReq);
SMqCMGetSubEpReq* buf = malloc(tlen);
if (buf == NULL) {
tscError("failed to malloc get subscribe ep buf");
}
buf->consumerId = htobe64(buf->consumerId);
pRequest = createRequest(tmq->pTscObj, NULL, NULL, TDMT_MND_GET_SUB_EP);
if (pRequest == NULL) {
tscError("failed to malloc subscribe ep request");
}
pRequest->body.requestMsg = (SDataBuf){ .pData = buf, .len = tlen };
SMsgSendInfo* sendInfo = buildMsgInfoImpl(pRequest);
sendInfo->requestObjRefId = 0;
sendInfo->param = tmq;
sendInfo->fp = tmq_ask_ep_cb;
SEpSet epSet = getEpSet_s(&tmq->pTscObj->pAppInfo->mgmtEp);
int64_t transporterId = 0;
asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &epSet, &transporterId, sendInfo);
tsem_wait(&pRequest->body.rspSem);
}
SMqClientTopic* pTopic = taosArrayGetP(tmq->clientTopics, tmq->nextTopicIdx);
tmq->nextTopicIdx = (tmq->nextTopicIdx + 1) % taosArrayGetSize(tmq->clientTopics); tmq->nextTopicIdx = (tmq->nextTopicIdx + 1) % taosArrayGetSize(tmq->clientTopics);
strcpy(req.topic, pTopic->topicName); strcpy(req.topic, pTopic->topicName);
int32_t nextVgIdx = pTopic->nextVgIdx; int32_t nextVgIdx = pTopic->nextVgIdx;
...@@ -618,14 +678,17 @@ tmq_message_t* tmq_consume_poll(tmq_t* tmq, int64_t blocking_time) { ...@@ -618,14 +678,17 @@ tmq_message_t* tmq_consume_poll(tmq_t* tmq, int64_t blocking_time) {
pRequest->body.requestMsg = (SDataBuf){ .pData = &req, .len = sizeof(SMqConsumeReq) }; pRequest->body.requestMsg = (SDataBuf){ .pData = &req, .len = sizeof(SMqConsumeReq) };
pRequest->type = TDMT_VND_CONSUME; pRequest->type = TDMT_VND_CONSUME;
SMsgSendInfo* body = buildMsgInfoImpl(pRequest); SMsgSendInfo* sendInfo = buildMsgInfoImpl(pRequest);
sendInfo->requestObjRefId = 0;
sendInfo->param = &tmq_message;
sendInfo->fp = tmq_poll_cb_inner;
int64_t transporterId = 0; int64_t transporterId = 0;
asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &pVg->epSet, &transporterId, body); asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &pVg->epSet, &transporterId, sendInfo);
tsem_wait(&pRequest->body.rspSem); tsem_wait(&pRequest->body.rspSem);
return (tmq_message_t*)pRequest->body.resInfo.pData; return tmq_message;
/*tsem_wait(&pRequest->body.rspSem);*/ /*tsem_wait(&pRequest->body.rspSem);*/
......
此差异已折叠。
...@@ -31,8 +31,8 @@ typedef struct { ...@@ -31,8 +31,8 @@ typedef struct {
SDnode *pDnode; SDnode *pDnode;
STaosQueue *queue; STaosQueue *queue;
union { union {
SWorkerPool pool; SQWorkerPool pool;
SMWorkerPool mpool; SWWorkerPool mpool;
}; };
} SDnodeWorker; } SDnodeWorker;
...@@ -109,10 +109,10 @@ typedef struct { ...@@ -109,10 +109,10 @@ typedef struct {
int32_t openVnodes; int32_t openVnodes;
int32_t totalVnodes; int32_t totalVnodes;
SRWLatch latch; SRWLatch latch;
SWorkerPool queryPool; SQWorkerPool queryPool;
SWorkerPool fetchPool; SFWorkerPool fetchPool;
SMWorkerPool syncPool; SWWorkerPool syncPool;
SMWorkerPool writePool; SWWorkerPool writePool;
} SVnodesMgmt; } SVnodesMgmt;
typedef struct { typedef struct {
......
...@@ -487,6 +487,8 @@ static void *dnodeThreadRoutine(void *param) { ...@@ -487,6 +487,8 @@ static void *dnodeThreadRoutine(void *param) {
SDnodeMgmt *pMgmt = &pDnode->dmgmt; SDnodeMgmt *pMgmt = &pDnode->dmgmt;
int32_t ms = pDnode->cfg.statusInterval * 1000; int32_t ms = pDnode->cfg.statusInterval * 1000;
setThreadName("dnode-hb");
while (true) { while (true) {
pthread_testcancel(); pthread_testcancel();
taosMsleep(ms); taosMsleep(ms);
......
...@@ -25,9 +25,9 @@ ...@@ -25,9 +25,9 @@
#include "dndMnode.h" #include "dndMnode.h"
#include "dndVnodes.h" #include "dndVnodes.h"
#define INTERNAL_USER "_internal" #define INTERNAL_USER "_dnd"
#define INTERNAL_CKEY "_key" #define INTERNAL_CKEY "_key"
#define INTERNAL_SECRET "_secret" #define INTERNAL_SECRET "_pwd"
static void dndInitMsgFp(STransMgmt *pMgmt) { static void dndInitMsgFp(STransMgmt *pMgmt) {
// Requests handled by DNODE // Requests handled by DNODE
...@@ -115,11 +115,12 @@ static void dndInitMsgFp(STransMgmt *pMgmt) { ...@@ -115,11 +115,12 @@ static void dndInitMsgFp(STransMgmt *pMgmt) {
pMgmt->msgFp[TMSG_INDEX(TDMT_MND_SUBSCRIBE)] = dndProcessMnodeWriteMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_MND_SUBSCRIBE)] = dndProcessMnodeWriteMsg;
/*pMgmt->msgFp[TMSG_INDEX(TDMT_VND_SUBSCRIBE_RSP)] = dndProcessMnodeWriteMsg;*/ /*pMgmt->msgFp[TMSG_INDEX(TDMT_VND_SUBSCRIBE_RSP)] = dndProcessMnodeWriteMsg;*/
pMgmt->msgFp[TMSG_INDEX(TDMT_VND_MQ_SET_CONN_RSP)] = dndProcessMnodeWriteMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_MQ_SET_CONN_RSP)] = dndProcessMnodeWriteMsg;
pMgmt->msgFp[TMSG_INDEX(TDMT_MND_GET_SUB_EP)] = dndProcessMnodeReadMsg;
// Requests handled by VNODE // Requests handled by VNODE
pMgmt->msgFp[TMSG_INDEX(TDMT_VND_SUBMIT)] = dndProcessVnodeWriteMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_SUBMIT)] = dndProcessVnodeWriteMsg;
pMgmt->msgFp[TMSG_INDEX(TDMT_VND_QUERY)] = dndProcessVnodeQueryMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_QUERY)] = dndProcessVnodeQueryMsg;
pMgmt->msgFp[TMSG_INDEX(TDMT_VND_QUERY_CONTINUE)] = dndProcessVnodeQueryMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_QUERY_CONTINUE)] = dndProcessVnodeQueryMsg;
pMgmt->msgFp[TMSG_INDEX(TDMT_VND_FETCH)] = dndProcessVnodeFetchMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_FETCH)] = dndProcessVnodeFetchMsg;
pMgmt->msgFp[TMSG_INDEX(TDMT_VND_ALTER_TABLE)] = dndProcessVnodeWriteMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_ALTER_TABLE)] = dndProcessVnodeWriteMsg;
pMgmt->msgFp[TMSG_INDEX(TDMT_VND_UPDATE_TAG_VAL)] = dndProcessVnodeWriteMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_UPDATE_TAG_VAL)] = dndProcessVnodeWriteMsg;
...@@ -157,17 +158,18 @@ static void dndProcessResponse(void *parent, SRpcMsg *pRsp, SEpSet *pEpSet) { ...@@ -157,17 +158,18 @@ static void dndProcessResponse(void *parent, SRpcMsg *pRsp, SEpSet *pEpSet) {
if (dndGetStat(pDnode) == DND_STAT_STOPPED) { if (dndGetStat(pDnode) == DND_STAT_STOPPED) {
if (pRsp == NULL || pRsp->pCont == NULL) return; if (pRsp == NULL || pRsp->pCont == NULL) return;
dTrace("RPC %p, rsp:%s is ignored since dnode is stopping", pRsp->handle, TMSG_INFO(msgType)); dTrace("RPC %p, rsp:%s ignored since dnode exiting, app:%p", pRsp->handle, TMSG_INFO(msgType), pRsp->ahandle);
rpcFreeCont(pRsp->pCont); rpcFreeCont(pRsp->pCont);
return; return;
} }
DndMsgFp fp = pMgmt->msgFp[TMSG_INDEX(msgType)]; DndMsgFp fp = pMgmt->msgFp[TMSG_INDEX(msgType)];
if (fp != NULL) { if (fp != NULL) {
dTrace("RPC %p, rsp:%s will be processed, code:0x%x", pRsp->handle, TMSG_INFO(msgType), pRsp->code & 0XFFFF); dTrace("RPC %p, rsp:%s will be processed, code:0x%x app:%p", pRsp->handle, TMSG_INFO(msgType), pRsp->code & 0XFFFF,
pRsp->ahandle);
(*fp)(pDnode, pRsp, pEpSet); (*fp)(pDnode, pRsp, pEpSet);
} else { } else {
dError("RPC %p, rsp:%s not processed", pRsp->handle, TMSG_INFO(msgType)); dError("RPC %p, rsp:%s not processed, app:%p", pRsp->handle, TMSG_INFO(msgType), pRsp->ahandle);
rpcFreeCont(pRsp->pCont); rpcFreeCont(pRsp->pCont);
} }
} }
...@@ -194,7 +196,7 @@ static int32_t dndInitClient(SDnode *pDnode) { ...@@ -194,7 +196,7 @@ static int32_t dndInitClient(SDnode *pDnode) {
pMgmt->clientRpc = rpcOpen(&rpcInit); pMgmt->clientRpc = rpcOpen(&rpcInit);
if (pMgmt->clientRpc == NULL) { if (pMgmt->clientRpc == NULL) {
dError("failed to init rpc client"); dError("failed to init dnode rpc client");
return -1; return -1;
} }
...@@ -217,40 +219,39 @@ static void dndProcessRequest(void *param, SRpcMsg *pReq, SEpSet *pEpSet) { ...@@ -217,40 +219,39 @@ static void dndProcessRequest(void *param, SRpcMsg *pReq, SEpSet *pEpSet) {
tmsg_t msgType = pReq->msgType; tmsg_t msgType = pReq->msgType;
if (msgType == TDMT_DND_NETWORK_TEST) { if (msgType == TDMT_DND_NETWORK_TEST) {
dTrace("RPC %p, network test req, app:%p will be processed, code:0x%x", pReq->handle, pReq->ahandle, pReq->code); dTrace("RPC %p, network test req will be processed, app:%p", pReq->handle, pReq->ahandle);
dndProcessStartupReq(pDnode, pReq); dndProcessStartupReq(pDnode, pReq);
return; return;
} }
if (dndGetStat(pDnode) == DND_STAT_STOPPED) { if (dndGetStat(pDnode) == DND_STAT_STOPPED) {
dError("RPC %p, req:%s app:%p is ignored since dnode exiting", pReq->handle, TMSG_INFO(msgType), pReq->ahandle); dError("RPC %p, req:%s ignored since dnode exiting, app:%p", pReq->handle, TMSG_INFO(msgType), pReq->ahandle);
SRpcMsg rspMsg = {.handle = pReq->handle, .code = TSDB_CODE_DND_OFFLINE}; SRpcMsg rspMsg = {.handle = pReq->handle, .code = TSDB_CODE_DND_OFFLINE, .ahandle = pReq->ahandle};
rpcSendResponse(&rspMsg); rpcSendResponse(&rspMsg);
rpcFreeCont(pReq->pCont); rpcFreeCont(pReq->pCont);
return; return;
} else if (dndGetStat(pDnode) != DND_STAT_RUNNING) { } else if (dndGetStat(pDnode) != DND_STAT_RUNNING) {
dError("RPC %p, req:%s app:%p is ignored since dnode not running", pReq->handle, TMSG_INFO(msgType), pReq->ahandle); dError("RPC %p, req:%s ignored since dnode not running, app:%p", pReq->handle, TMSG_INFO(msgType), pReq->ahandle);
SRpcMsg rspMsg = {.handle = pReq->handle, .code = TSDB_CODE_APP_NOT_READY}; SRpcMsg rspMsg = {.handle = pReq->handle, .code = TSDB_CODE_APP_NOT_READY, .ahandle = pReq->ahandle};
rpcSendResponse(&rspMsg); rpcSendResponse(&rspMsg);
rpcFreeCont(pReq->pCont); rpcFreeCont(pReq->pCont);
return; return;
} }
if (pReq->pCont == NULL) { if (pReq->pCont == NULL) {
dTrace("RPC %p, req:%s app:%p not processed since content is null", pReq->handle, TMSG_INFO(msgType), dTrace("RPC %p, req:%s not processed since its empty, app:%p", pReq->handle, TMSG_INFO(msgType), pReq->ahandle);
pReq->ahandle); SRpcMsg rspMsg = {.handle = pReq->handle, .code = TSDB_CODE_DND_INVALID_MSG_LEN, .ahandle = pReq->ahandle};
SRpcMsg rspMsg = {.handle = pReq->handle, .code = TSDB_CODE_DND_INVALID_MSG_LEN};
rpcSendResponse(&rspMsg); rpcSendResponse(&rspMsg);
return; return;
} }
DndMsgFp fp = pMgmt->msgFp[TMSG_INDEX(msgType)]; DndMsgFp fp = pMgmt->msgFp[TMSG_INDEX(msgType)];
if (fp != NULL) { if (fp != NULL) {
dTrace("RPC %p, req:%s app:%p will be processed", pReq->handle, TMSG_INFO(msgType), pReq->ahandle); dTrace("RPC %p, req:%s will be processed, app:%p", pReq->handle, TMSG_INFO(msgType), pReq->ahandle);
(*fp)(pDnode, pReq, pEpSet); (*fp)(pDnode, pReq, pEpSet);
} else { } else {
dError("RPC %p, req:%s app:%p is not processed since no handle", pReq->handle, TMSG_INFO(msgType), pReq->ahandle); dError("RPC %p, req:%s not processed since no handle, app:%p", pReq->handle, TMSG_INFO(msgType), pReq->ahandle);
SRpcMsg rspMsg = {.handle = pReq->handle, .code = TSDB_CODE_MSG_NOT_PROCESSED}; SRpcMsg rspMsg = {.handle = pReq->handle, .code = TSDB_CODE_MSG_NOT_PROCESSED, .ahandle = pReq->ahandle};
rpcSendResponse(&rspMsg); rpcSendResponse(&rspMsg);
rpcFreeCont(pReq->pCont); rpcFreeCont(pReq->pCont);
} }
...@@ -290,24 +291,24 @@ static int32_t dndRetrieveUserAuthInfo(void *parent, char *user, char *spi, char ...@@ -290,24 +291,24 @@ static int32_t dndRetrieveUserAuthInfo(void *parent, char *user, char *spi, char
SDnode *pDnode = parent; SDnode *pDnode = parent;
if (dndAuthInternalReq(parent, user, spi, encrypt, secret, ckey) == 0) { if (dndAuthInternalReq(parent, user, spi, encrypt, secret, ckey) == 0) {
dTrace("user:%s, get auth from internal mnode, spi:%d encrypt:%d", user, *spi, *encrypt); dTrace("user:%s, get auth from mnode, spi:%d encrypt:%d", user, *spi, *encrypt);
return 0; return 0;
} }
if (dndGetUserAuthFromMnode(pDnode, user, spi, encrypt, secret, ckey) == 0) { if (dndGetUserAuthFromMnode(pDnode, user, spi, encrypt, secret, ckey) == 0) {
dTrace("user:%s, get auth from internal mnode, spi:%d encrypt:%d", user, *spi, *encrypt); dTrace("user:%s, get auth from mnode, spi:%d encrypt:%d", user, *spi, *encrypt);
return 0; return 0;
} }
if (terrno != TSDB_CODE_APP_NOT_READY) { if (terrno != TSDB_CODE_APP_NOT_READY) {
dTrace("failed to get user auth from internal mnode since %s", terrstr()); dTrace("failed to get user auth from mnode since %s", terrstr());
return -1; return -1;
} }
SAuthReq *pReq = rpcMallocCont(sizeof(SAuthReq)); SAuthReq *pReq = rpcMallocCont(sizeof(SAuthReq));
tstrncpy(pReq->user, user, TSDB_USER_LEN); tstrncpy(pReq->user, user, TSDB_USER_LEN);
SRpcMsg rpcMsg = {.pCont = pReq, .contLen = sizeof(SAuthReq), .msgType = TDMT_MND_AUTH}; SRpcMsg rpcMsg = {.pCont = pReq, .contLen = sizeof(SAuthReq), .msgType = TDMT_MND_AUTH, .ahandle = (void *)9528};
SRpcMsg rpcRsp = {0}; SRpcMsg rpcRsp = {0};
dTrace("user:%s, send user auth req to other mnodes, spi:%d encrypt:%d", user, pReq->spi, pReq->encrypt); dTrace("user:%s, send user auth req to other mnodes, spi:%d encrypt:%d", user, pReq->spi, pReq->encrypt);
dndSendMsgToMnodeRecv(pDnode, &rpcMsg, &rpcRsp); dndSendMsgToMnodeRecv(pDnode, &rpcMsg, &rpcRsp);
...@@ -351,7 +352,7 @@ static int32_t dndInitServer(SDnode *pDnode) { ...@@ -351,7 +352,7 @@ static int32_t dndInitServer(SDnode *pDnode) {
pMgmt->serverRpc = rpcOpen(&rpcInit); pMgmt->serverRpc = rpcOpen(&rpcInit);
if (pMgmt->serverRpc == NULL) { if (pMgmt->serverRpc == NULL) {
dError("failed to init rpc server"); dError("failed to init dnode rpc server");
return -1; return -1;
} }
......
...@@ -253,7 +253,7 @@ static int32_t dndGetVnodesFromFile(SDnode *pDnode, SWrapperCfg **ppCfgs, int32_ ...@@ -253,7 +253,7 @@ static int32_t dndGetVnodesFromFile(SDnode *pDnode, SWrapperCfg **ppCfgs, int32_
} }
for (int32_t i = 0; i < vnodesNum; ++i) { for (int32_t i = 0; i < vnodesNum; ++i) {
cJSON *vnode = cJSON_GetArrayItem(vnodes, i); cJSON * vnode = cJSON_GetArrayItem(vnodes, i);
SWrapperCfg *pCfg = &pCfgs[i]; SWrapperCfg *pCfg = &pCfgs[i];
cJSON *vgId = cJSON_GetObjectItem(vnode, "vgId"); cJSON *vgId = cJSON_GetObjectItem(vnode, "vgId");
...@@ -382,7 +382,7 @@ static void *dnodeOpenVnodeFunc(void *param) { ...@@ -382,7 +382,7 @@ static void *dnodeOpenVnodeFunc(void *param) {
dndReportStartup(pDnode, "open-vnodes", stepDesc); dndReportStartup(pDnode, "open-vnodes", stepDesc);
SVnodeCfg cfg = {.pDnode = pDnode, .pTfs = pDnode->pTfs, .vgId = pCfg->vgId}; SVnodeCfg cfg = {.pDnode = pDnode, .pTfs = pDnode->pTfs, .vgId = pCfg->vgId};
SVnode *pImpl = vnodeOpen(pCfg->path, &cfg); SVnode * pImpl = vnodeOpen(pCfg->path, &cfg);
if (pImpl == NULL) { if (pImpl == NULL) {
dError("vgId:%d, failed to open vnode by thread:%d", pCfg->vgId, pThread->threadIndex); dError("vgId:%d, failed to open vnode by thread:%d", pCfg->vgId, pThread->threadIndex);
pThread->failed++; pThread->failed++;
...@@ -737,15 +737,9 @@ int32_t dndProcessCompactVnodeReq(SDnode *pDnode, SRpcMsg *pReq) { ...@@ -737,15 +737,9 @@ int32_t dndProcessCompactVnodeReq(SDnode *pDnode, SRpcMsg *pReq) {
return 0; return 0;
} }
static void dndProcessVnodeQueryQueue(SVnodeObj *pVnode, SRpcMsg *pMsg) { static void dndProcessVnodeQueryQueue(SVnodeObj *pVnode, SRpcMsg *pMsg) { vnodeProcessQueryMsg(pVnode->pImpl, pMsg); }
SRpcMsg *pRsp = NULL;
vnodeProcessQueryReq(pVnode->pImpl, pMsg, &pRsp);
}
static void dndProcessVnodeFetchQueue(SVnodeObj *pVnode, SRpcMsg *pMsg) { static void dndProcessVnodeFetchQueue(SVnodeObj *pVnode, SRpcMsg *pMsg) { vnodeProcessFetchMsg(pVnode->pImpl, pMsg); }
SRpcMsg *pRsp = NULL;
vnodeProcessFetchReq(pVnode->pImpl, pMsg, &pRsp);
}
static void dndProcessVnodeWriteQueue(SVnodeObj *pVnode, STaosQall *qall, int32_t numOfMsgs) { static void dndProcessVnodeWriteQueue(SVnodeObj *pVnode, STaosQall *qall, int32_t numOfMsgs) {
SArray *pArray = taosArrayInit(numOfMsgs, sizeof(SRpcMsg *)); SArray *pArray = taosArrayInit(numOfMsgs, sizeof(SRpcMsg *));
...@@ -916,27 +910,27 @@ static int32_t dndInitVnodeWorkers(SDnode *pDnode) { ...@@ -916,27 +910,27 @@ static int32_t dndInitVnodeWorkers(SDnode *pDnode) {
int32_t maxWriteThreads = TMAX(pDnode->env.numOfCores, 1); int32_t maxWriteThreads = TMAX(pDnode->env.numOfCores, 1);
int32_t maxSyncThreads = TMAX(pDnode->env.numOfCores / 2, 1); int32_t maxSyncThreads = TMAX(pDnode->env.numOfCores / 2, 1);
SWorkerPool *pPool = &pMgmt->queryPool; SQWorkerPool *pQPool = &pMgmt->queryPool;
pPool->name = "vnode-query"; pQPool->name = "vnode-query";
pPool->min = minQueryThreads; pQPool->min = minQueryThreads;
pPool->max = maxQueryThreads; pQPool->max = maxQueryThreads;
if (tWorkerInit(pPool) != 0) return -1; if (tQWorkerInit(pQPool) != 0) return -1;
pPool = &pMgmt->fetchPool; SFWorkerPool *pFPool = &pMgmt->fetchPool;
pPool->name = "vnode-fetch"; pFPool->name = "vnode-fetch";
pPool->min = minFetchThreads; pFPool->min = minFetchThreads;
pPool->max = maxFetchThreads; pFPool->max = maxFetchThreads;
if (tWorkerInit(pPool) != 0) return -1; if (tFWorkerInit(pFPool) != 0) return -1;
SMWorkerPool *pMPool = &pMgmt->writePool; SWWorkerPool *pWPool = &pMgmt->writePool;
pMPool->name = "vnode-write"; pWPool->name = "vnode-write";
pMPool->max = maxWriteThreads; pWPool->max = maxWriteThreads;
if (tMWorkerInit(pMPool) != 0) return -1; if (tWWorkerInit(pWPool) != 0) return -1;
pMPool = &pMgmt->syncPool; pWPool = &pMgmt->syncPool;
pMPool->name = "vnode-sync"; pWPool->name = "vnode-sync";
pMPool->max = maxSyncThreads; pWPool->max = maxSyncThreads;
if (tMWorkerInit(pMPool) != 0) return -1; if (tWWorkerInit(pWPool) != 0) return -1;
dDebug("vnode workers is initialized"); dDebug("vnode workers is initialized");
return 0; return 0;
...@@ -944,21 +938,21 @@ static int32_t dndInitVnodeWorkers(SDnode *pDnode) { ...@@ -944,21 +938,21 @@ static int32_t dndInitVnodeWorkers(SDnode *pDnode) {
static void dndCleanupVnodeWorkers(SDnode *pDnode) { static void dndCleanupVnodeWorkers(SDnode *pDnode) {
SVnodesMgmt *pMgmt = &pDnode->vmgmt; SVnodesMgmt *pMgmt = &pDnode->vmgmt;
tWorkerCleanup(&pMgmt->fetchPool); tFWorkerCleanup(&pMgmt->fetchPool);
tWorkerCleanup(&pMgmt->queryPool); tQWorkerCleanup(&pMgmt->queryPool);
tMWorkerCleanup(&pMgmt->writePool); tWWorkerCleanup(&pMgmt->writePool);
tMWorkerCleanup(&pMgmt->syncPool); tWWorkerCleanup(&pMgmt->syncPool);
dDebug("vnode workers is closed"); dDebug("vnode workers is closed");
} }
static int32_t dndAllocVnodeQueue(SDnode *pDnode, SVnodeObj *pVnode) { static int32_t dndAllocVnodeQueue(SDnode *pDnode, SVnodeObj *pVnode) {
SVnodesMgmt *pMgmt = &pDnode->vmgmt; SVnodesMgmt *pMgmt = &pDnode->vmgmt;
pVnode->pWriteQ = tMWorkerAllocQueue(&pMgmt->writePool, pVnode, (FProcessItems)dndProcessVnodeWriteQueue); pVnode->pWriteQ = tWWorkerAllocQueue(&pMgmt->writePool, pVnode, (FItems)dndProcessVnodeWriteQueue);
pVnode->pApplyQ = tMWorkerAllocQueue(&pMgmt->writePool, pVnode, (FProcessItems)dndProcessVnodeApplyQueue); pVnode->pApplyQ = tWWorkerAllocQueue(&pMgmt->writePool, pVnode, (FItems)dndProcessVnodeApplyQueue);
pVnode->pSyncQ = tMWorkerAllocQueue(&pMgmt->syncPool, pVnode, (FProcessItems)dndProcessVnodeSyncQueue); pVnode->pSyncQ = tWWorkerAllocQueue(&pMgmt->syncPool, pVnode, (FItems)dndProcessVnodeSyncQueue);
pVnode->pFetchQ = tWorkerAllocQueue(&pMgmt->fetchPool, pVnode, (FProcessItem)dndProcessVnodeFetchQueue); pVnode->pFetchQ = tFWorkerAllocQueue(&pMgmt->fetchPool, pVnode, (FItem)dndProcessVnodeFetchQueue);
pVnode->pQueryQ = tWorkerAllocQueue(&pMgmt->queryPool, pVnode, (FProcessItem)dndProcessVnodeQueryQueue); pVnode->pQueryQ = tQWorkerAllocQueue(&pMgmt->queryPool, pVnode, (FItem)dndProcessVnodeQueryQueue);
if (pVnode->pApplyQ == NULL || pVnode->pWriteQ == NULL || pVnode->pSyncQ == NULL || pVnode->pFetchQ == NULL || if (pVnode->pApplyQ == NULL || pVnode->pWriteQ == NULL || pVnode->pSyncQ == NULL || pVnode->pFetchQ == NULL ||
pVnode->pQueryQ == NULL) { pVnode->pQueryQ == NULL) {
...@@ -971,11 +965,11 @@ static int32_t dndAllocVnodeQueue(SDnode *pDnode, SVnodeObj *pVnode) { ...@@ -971,11 +965,11 @@ static int32_t dndAllocVnodeQueue(SDnode *pDnode, SVnodeObj *pVnode) {
static void dndFreeVnodeQueue(SDnode *pDnode, SVnodeObj *pVnode) { static void dndFreeVnodeQueue(SDnode *pDnode, SVnodeObj *pVnode) {
SVnodesMgmt *pMgmt = &pDnode->vmgmt; SVnodesMgmt *pMgmt = &pDnode->vmgmt;
tWorkerFreeQueue(&pMgmt->queryPool, pVnode->pQueryQ); tQWorkerFreeQueue(&pMgmt->queryPool, pVnode->pQueryQ);
tWorkerFreeQueue(&pMgmt->fetchPool, pVnode->pFetchQ); tFWorkerFreeQueue(&pMgmt->fetchPool, pVnode->pFetchQ);
tMWorkerFreeQueue(&pMgmt->writePool, pVnode->pWriteQ); tWWorkerFreeQueue(&pMgmt->writePool, pVnode->pWriteQ);
tMWorkerFreeQueue(&pMgmt->writePool, pVnode->pApplyQ); tWWorkerFreeQueue(&pMgmt->writePool, pVnode->pApplyQ);
tMWorkerFreeQueue(&pMgmt->syncPool, pVnode->pSyncQ); tWWorkerFreeQueue(&pMgmt->syncPool, pVnode->pSyncQ);
pVnode->pWriteQ = NULL; pVnode->pWriteQ = NULL;
pVnode->pApplyQ = NULL; pVnode->pApplyQ = NULL;
pVnode->pSyncQ = NULL; pVnode->pSyncQ = NULL;
......
...@@ -31,28 +31,28 @@ int32_t dndInitWorker(SDnode *pDnode, SDnodeWorker *pWorker, EWorkerType type, c ...@@ -31,28 +31,28 @@ int32_t dndInitWorker(SDnode *pDnode, SDnodeWorker *pWorker, EWorkerType type, c
pWorker->pDnode = pDnode; pWorker->pDnode = pDnode;
if (pWorker->type == DND_WORKER_SINGLE) { if (pWorker->type == DND_WORKER_SINGLE) {
SWorkerPool *pPool = &pWorker->pool; SQWorkerPool *pPool = &pWorker->pool;
pPool->name = name; pPool->name = name;
pPool->min = minNum; pPool->min = minNum;
pPool->max = maxNum; pPool->max = maxNum;
if (tWorkerInit(pPool) != 0) { if (tQWorkerInit(pPool) != 0) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1; return -1;
} }
pWorker->queue = tWorkerAllocQueue(pPool, pDnode, (FProcessItem)queueFp); pWorker->queue = tQWorkerAllocQueue(pPool, pDnode, (FItem)queueFp);
if (pWorker->queue == NULL) { if (pWorker->queue == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1; return -1;
} }
} else if (pWorker->type == DND_WORKER_MULTI) { } else if (pWorker->type == DND_WORKER_MULTI) {
SMWorkerPool *pPool = &pWorker->mpool; SWWorkerPool *pPool = &pWorker->mpool;
pPool->name = name; pPool->name = name;
pPool->max = maxNum; pPool->max = maxNum;
if (tMWorkerInit(pPool) != 0) { if (tWWorkerInit(pPool) != 0) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1; return -1;
} }
pWorker->queue = tMWorkerAllocQueue(pPool, pDnode, (FProcessItems)queueFp); pWorker->queue = tWWorkerAllocQueue(pPool, pDnode, (FItems)queueFp);
if (pWorker->queue == NULL) { if (pWorker->queue == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1; return -1;
...@@ -70,11 +70,11 @@ void dndCleanupWorker(SDnodeWorker *pWorker) { ...@@ -70,11 +70,11 @@ void dndCleanupWorker(SDnodeWorker *pWorker) {
} }
if (pWorker->type == DND_WORKER_SINGLE) { if (pWorker->type == DND_WORKER_SINGLE) {
tWorkerCleanup(&pWorker->pool); tQWorkerCleanup(&pWorker->pool);
tWorkerFreeQueue(&pWorker->pool, pWorker->queue); tQWorkerFreeQueue(&pWorker->pool, pWorker->queue);
} else if (pWorker->type == DND_WORKER_MULTI) { } else if (pWorker->type == DND_WORKER_MULTI) {
tMWorkerCleanup(&pWorker->mpool); tWWorkerCleanup(&pWorker->mpool);
tMWorkerFreeQueue(&pWorker->mpool, pWorker->queue); tWWorkerFreeQueue(&pWorker->mpool, pWorker->queue);
} else { } else {
} }
} }
......
...@@ -25,7 +25,7 @@ extern "C" { ...@@ -25,7 +25,7 @@ extern "C" {
int32_t mndInitConsumer(SMnode *pMnode); int32_t mndInitConsumer(SMnode *pMnode);
void mndCleanupConsumer(SMnode *pMnode); void mndCleanupConsumer(SMnode *pMnode);
SMqConsumerObj *mndAcquireConsumer(SMnode *pMnode, int32_t consumerId); SMqConsumerObj *mndAcquireConsumer(SMnode *pMnode, int64_t consumerId);
void mndReleaseConsumer(SMnode *pMnode, SMqConsumerObj *pConsumer); void mndReleaseConsumer(SMnode *pMnode, SMqConsumerObj *pConsumer);
SSdbRaw *mndConsumerActionEncode(SMqConsumerObj *pConsumer); SSdbRaw *mndConsumerActionEncode(SMqConsumerObj *pConsumer);
......
...@@ -30,24 +30,23 @@ ...@@ -30,24 +30,23 @@
#define MND_CONSUMER_VER_NUMBER 1 #define MND_CONSUMER_VER_NUMBER 1
#define MND_CONSUMER_RESERVE_SIZE 64 #define MND_CONSUMER_RESERVE_SIZE 64
static int32_t mndConsumerActionInsert(SSdb *pSdb, SMqConsumerObj *pConsumer); static int32_t mndConsumerActionInsert(SSdb *pSdb, SMqConsumerObj *pConsumer);
static int32_t mndConsumerActionDelete(SSdb *pSdb, SMqConsumerObj *pConsumer); static int32_t mndConsumerActionDelete(SSdb *pSdb, SMqConsumerObj *pConsumer);
static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pConsumer, SMqConsumerObj *pNewConsumer); static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pConsumer, SMqConsumerObj *pNewConsumer);
static int32_t mndProcessConsumerMetaMsg(SMnodeMsg *pMsg); static int32_t mndProcessConsumerMetaMsg(SMnodeMsg *pMsg);
static int32_t mndGetConsumerMeta(SMnodeMsg *pMsg, SShowObj *pShow, STableMetaRsp *pMeta); static int32_t mndGetConsumerMeta(SMnodeMsg *pMsg, SShowObj *pShow, STableMetaRsp *pMeta);
static int32_t mndRetrieveConsumer(SMnodeMsg *pMsg, SShowObj *pShow, char *data, int32_t rows); static int32_t mndRetrieveConsumer(SMnodeMsg *pMsg, SShowObj *pShow, char *data, int32_t rows);
static void mndCancelGetNextConsumer(SMnode *pMnode, void *pIter); static void mndCancelGetNextConsumer(SMnode *pMnode, void *pIter);
int32_t mndInitConsumer(SMnode *pMnode) { int32_t mndInitConsumer(SMnode *pMnode) {
SSdbTable table = {.sdbType = SDB_CONSUMER, SSdbTable table = {.sdbType = SDB_CONSUMER,
.keyType = SDB_KEY_BINARY, .keyType = SDB_KEY_INT64,
.encodeFp = (SdbEncodeFp)mndConsumerActionEncode, .encodeFp = (SdbEncodeFp)mndConsumerActionEncode,
.decodeFp = (SdbDecodeFp)mndConsumerActionDecode, .decodeFp = (SdbDecodeFp)mndConsumerActionDecode,
.insertFp = (SdbInsertFp)mndConsumerActionInsert, .insertFp = (SdbInsertFp)mndConsumerActionInsert,
.updateFp = (SdbUpdateFp)mndConsumerActionUpdate, .updateFp = (SdbUpdateFp)mndConsumerActionUpdate,
.deleteFp = (SdbDeleteFp)mndConsumerActionDelete}; .deleteFp = (SdbDeleteFp)mndConsumerActionDelete};
return sdbSetTable(pMnode->pSdb, table); return sdbSetTable(pMnode->pSdb, table);
} }
...@@ -61,10 +60,10 @@ SSdbRaw *mndConsumerActionEncode(SMqConsumerObj *pConsumer) { ...@@ -61,10 +60,10 @@ SSdbRaw *mndConsumerActionEncode(SMqConsumerObj *pConsumer) {
SSdbRaw *pRaw = sdbAllocRaw(SDB_CONSUMER, MND_CONSUMER_VER_NUMBER, size); SSdbRaw *pRaw = sdbAllocRaw(SDB_CONSUMER, MND_CONSUMER_VER_NUMBER, size);
if (pRaw == NULL) goto CM_ENCODE_OVER; if (pRaw == NULL) goto CM_ENCODE_OVER;
void* buf = malloc(tlen); void *buf = malloc(tlen);
if (buf == NULL) goto CM_ENCODE_OVER; if (buf == NULL) goto CM_ENCODE_OVER;
void* abuf = buf; void *abuf = buf;
tEncodeSMqConsumerObj(&abuf, pConsumer); tEncodeSMqConsumerObj(&abuf, pConsumer);
int32_t dataPos = 0; int32_t dataPos = 0;
...@@ -106,7 +105,7 @@ SSdbRow *mndConsumerActionDecode(SSdbRaw *pRaw) { ...@@ -106,7 +105,7 @@ SSdbRow *mndConsumerActionDecode(SSdbRaw *pRaw) {
int32_t dataPos = 0; int32_t dataPos = 0;
int32_t len; int32_t len;
SDB_GET_INT32(pRaw, dataPos, &len, CM_DECODE_OVER); SDB_GET_INT32(pRaw, dataPos, &len, CM_DECODE_OVER);
void* buf = malloc(len); void *buf = malloc(len);
if (buf == NULL) goto CM_DECODE_OVER; if (buf == NULL) goto CM_DECODE_OVER;
SDB_GET_BINARY(pRaw, dataPos, buf, len, CM_DECODE_OVER); SDB_GET_BINARY(pRaw, dataPos, buf, len, CM_DECODE_OVER);
SDB_GET_RESERVE(pRaw, dataPos, MND_CONSUMER_RESERVE_SIZE, CM_DECODE_OVER); SDB_GET_RESERVE(pRaw, dataPos, MND_CONSUMER_RESERVE_SIZE, CM_DECODE_OVER);
...@@ -147,7 +146,7 @@ static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pOldConsumer, ...@@ -147,7 +146,7 @@ static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pOldConsumer,
return 0; return 0;
} }
SMqConsumerObj *mndAcquireConsumer(SMnode *pMnode, int32_t consumerId) { SMqConsumerObj *mndAcquireConsumer(SMnode *pMnode, int64_t consumerId) {
SSdb *pSdb = pMnode->pSdb; SSdb *pSdb = pMnode->pSdb;
SMqConsumerObj *pConsumer = sdbAcquire(pSdb, SDB_CONSUMER, &consumerId); SMqConsumerObj *pConsumer = sdbAcquire(pSdb, SDB_CONSUMER, &consumerId);
if (pConsumer == NULL) { if (pConsumer == NULL) {
......
...@@ -30,6 +30,8 @@ ...@@ -30,6 +30,8 @@
#define MND_SUBSCRIBE_VER_NUMBER 1 #define MND_SUBSCRIBE_VER_NUMBER 1
#define MND_SUBSCRIBE_RESERVE_SIZE 64 #define MND_SUBSCRIBE_RESERVE_SIZE 64
static char *mndMakeSubscribeKey(char *cgroup, char *topicName);
static SSdbRaw *mndSubActionEncode(SMqSubscribeObj *); static SSdbRaw *mndSubActionEncode(SMqSubscribeObj *);
static SSdbRow *mndSubActionDecode(SSdbRaw *pRaw); static SSdbRow *mndSubActionDecode(SSdbRaw *pRaw);
static int32_t mndSubActionInsert(SSdb *pSdb, SMqSubscribeObj *); static int32_t mndSubActionInsert(SSdb *pSdb, SMqSubscribeObj *);
...@@ -41,9 +43,10 @@ static int32_t mndProcessSubscribeRsp(SMnodeMsg *pMsg); ...@@ -41,9 +43,10 @@ static int32_t mndProcessSubscribeRsp(SMnodeMsg *pMsg);
static int32_t mndProcessSubscribeInternalReq(SMnodeMsg *pMsg); static int32_t mndProcessSubscribeInternalReq(SMnodeMsg *pMsg);
static int32_t mndProcessSubscribeInternalRsp(SMnodeMsg *pMsg); static int32_t mndProcessSubscribeInternalRsp(SMnodeMsg *pMsg);
static int32_t mndProcessMqTimerMsg(SMnodeMsg *pMsg); static int32_t mndProcessMqTimerMsg(SMnodeMsg *pMsg);
static int32_t mndProcessGetSubEpReq(SMnodeMsg *pMsg);
static int mndBuildMqSetConsumerVgReq(SMnode *pMnode, STrans *pTrans, SMqConsumerObj *pConsumer, static int mndBuildMqSetConsumerVgReq(SMnode *pMnode, STrans *pTrans, SMqConsumerObj *pConsumer,
SMqConsumerTopic *pConsumerTopic, SMqTopicObj *pTopic); SMqConsumerTopic *pConsumerTopic, SMqTopicObj *pTopic, SMqConsumerEp *pSub);
int32_t mndInitSubscribe(SMnode *pMnode) { int32_t mndInitSubscribe(SMnode *pMnode) {
SSdbTable table = {.sdbType = SDB_SUBSCRIBE, SSdbTable table = {.sdbType = SDB_SUBSCRIBE,
...@@ -57,9 +60,60 @@ int32_t mndInitSubscribe(SMnode *pMnode) { ...@@ -57,9 +60,60 @@ int32_t mndInitSubscribe(SMnode *pMnode) {
mndSetMsgHandle(pMnode, TDMT_MND_SUBSCRIBE, mndProcessSubscribeReq); mndSetMsgHandle(pMnode, TDMT_MND_SUBSCRIBE, mndProcessSubscribeReq);
mndSetMsgHandle(pMnode, TDMT_VND_MQ_SET_CONN_RSP, mndProcessSubscribeInternalRsp); mndSetMsgHandle(pMnode, TDMT_VND_MQ_SET_CONN_RSP, mndProcessSubscribeInternalRsp);
mndSetMsgHandle(pMnode, TDMT_MND_MQ_TIMER, mndProcessMqTimerMsg); mndSetMsgHandle(pMnode, TDMT_MND_MQ_TIMER, mndProcessMqTimerMsg);
mndSetMsgHandle(pMnode, TDMT_MND_GET_SUB_EP, mndProcessGetSubEpReq);
return sdbSetTable(pMnode->pSdb, table); return sdbSetTable(pMnode->pSdb, table);
} }
static int32_t mndProcessGetSubEpReq(SMnodeMsg *pMsg) {
SMnode *pMnode = pMsg->pMnode;
SMqCMGetSubEpReq *pReq = (SMqCMGetSubEpReq *)pMsg->pCont;
SMqCMGetSubEpRsp rsp;
int64_t consumerId = be64toh(pReq->consumerId);
SMqConsumerObj *pConsumer = mndAcquireConsumer(pMsg->pMnode, consumerId);
if (pConsumer == NULL) {
/*terrno = */
return -1;
}
ASSERT(strcmp(pReq->cgroup, pConsumer->cgroup) == 0);
strcpy(rsp.cgroup, pReq->cgroup);
rsp.consumerId = consumerId;
SArray *pTopics = pConsumer->topics;
int32_t sz = taosArrayGetSize(pTopics);
rsp.topics = taosArrayInit(sz, sizeof(SMqSubTopicEp));
for (int32_t i = 0; i < sz; i++) {
SMqSubTopicEp topicEp;
SMqConsumerTopic *pConsumerTopic = taosArrayGet(pTopics, i);
strcpy(topicEp.topic, pConsumerTopic->name);
SMqSubscribeObj *pSub = mndAcquireSubscribe(pMnode, pConsumer->cgroup, pConsumerTopic->name);
int32_t assignedSz = taosArrayGetSize(pSub->assigned);
topicEp.vgs = taosArrayInit(assignedSz, sizeof(SMqSubVgEp));
for (int32_t j = 0; j < assignedSz; j++) {
SMqConsumerEp *pCEp = taosArrayGet(pSub->assigned, i);
if (pCEp->consumerId == consumerId) {
taosArrayPush(pSub->assigned, pCEp);
}
}
if (taosArrayGetSize(topicEp.vgs) != 0) {
taosArrayPush(rsp.topics, &topicEp);
}
}
int32_t tlen = tEncodeSMqCMGetSubEpRsp(NULL, &rsp);
void *buf = malloc(tlen);
if (buf == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1;
}
void *abuf = buf;
tEncodeSMqCMGetSubEpRsp(&abuf, &rsp);
//TODO: free rsp
pMsg->pCont = buf;
pMsg->contLen = tlen;
return 0;
}
static int32_t mndSplitSubscribeKey(char *key, char **topic, char **cgroup) { static int32_t mndSplitSubscribeKey(char *key, char **topic, char **cgroup) {
int i = 0; int i = 0;
while (key[i] != ':') { while (key[i] != ':') {
...@@ -97,7 +151,7 @@ static int32_t mndProcessMqTimerMsg(SMnodeMsg *pMsg) { ...@@ -97,7 +151,7 @@ static int32_t mndProcessMqTimerMsg(SMnodeMsg *pMsg) {
// build msg // build msg
SMqSetCVgReq* pReq = malloc(sizeof(SMqSetCVgReq) + pCEp->qmsgLen); SMqSetCVgReq *pReq = malloc(sizeof(SMqSetCVgReq));
if (pReq == NULL) { if (pReq == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1; return -1;
...@@ -108,7 +162,8 @@ static int32_t mndProcessMqTimerMsg(SMnodeMsg *pMsg) { ...@@ -108,7 +162,8 @@ static int32_t mndProcessMqTimerMsg(SMnodeMsg *pMsg) {
pReq->logicalPlan = strdup(pTopic->logicalPlan); pReq->logicalPlan = strdup(pTopic->logicalPlan);
pReq->physicalPlan = strdup(pTopic->physicalPlan); pReq->physicalPlan = strdup(pTopic->physicalPlan);
pReq->qmsgLen = pCEp->qmsgLen; pReq->qmsgLen = pCEp->qmsgLen;
memcpy(pReq->qmsg, pCEp->qmsg, pCEp->qmsgLen); /*memcpy(pReq->qmsg, pCEp->qmsg, pCEp->qmsgLen);*/
pReq->qmsg = strdup(pCEp->qmsg);
int32_t tlen = tEncodeSMqSetCVgReq(NULL, pReq); int32_t tlen = tEncodeSMqSetCVgReq(NULL, pReq);
void *reqStr = malloc(tlen); void *reqStr = malloc(tlen);
if (reqStr == NULL) { if (reqStr == NULL) {
...@@ -146,11 +201,11 @@ static int32_t mndProcessMqTimerMsg(SMnodeMsg *pMsg) { ...@@ -146,11 +201,11 @@ static int32_t mndProcessMqTimerMsg(SMnodeMsg *pMsg) {
} }
static int mndInitUnassignedVg(SMnode *pMnode, SMqTopicObj *pTopic, SArray *unassignedVg) { static int mndInitUnassignedVg(SMnode *pMnode, SMqTopicObj *pTopic, SArray *unassignedVg) {
//convert phyplan to dag // convert phyplan to dag
SQueryDag *pDag = qStringToDag(pTopic->physicalPlan); SQueryDag *pDag = qStringToDag(pTopic->physicalPlan);
SArray *pArray; SArray *pArray;
SArray* inner = taosArrayGet(pDag->pSubplans, 0); SArray *inner = taosArrayGet(pDag->pSubplans, 0);
SSubplan *plan = taosArrayGetP(inner, 0); SSubplan *plan = taosArrayGetP(inner, 0);
plan->execNode.inUse = 0; plan->execNode.inUse = 0;
strcpy(plan->execNode.epAddr[0].fqdn, "localhost"); strcpy(plan->execNode.epAddr[0].fqdn, "localhost");
plan->execNode.epAddr[0].port = 6030; plan->execNode.epAddr[0].port = 6030;
...@@ -161,21 +216,24 @@ static int mndInitUnassignedVg(SMnode *pMnode, SMqTopicObj *pTopic, SArray *unas ...@@ -161,21 +216,24 @@ static int mndInitUnassignedVg(SMnode *pMnode, SMqTopicObj *pTopic, SArray *unas
return -1; return -1;
} }
int32_t sz = taosArrayGetSize(pArray); int32_t sz = taosArrayGetSize(pArray);
//convert dag to msg // convert dag to msg
for (int32_t i = 0; i < sz; i++) { for (int32_t i = 0; i < sz; i++) {
SMqConsumerEp CEp; SMqConsumerEp CEp;
CEp.status = 0; CEp.status = 0;
CEp.lastConsumerHbTs = CEp.lastVgHbTs = -1; CEp.lastConsumerHbTs = CEp.lastVgHbTs = -1;
STaskInfo* pTaskInfo = taosArrayGet(pArray, i); STaskInfo *pTaskInfo = taosArrayGet(pArray, i);
tConvertQueryAddrToEpSet(&CEp.epSet, &pTaskInfo->addr); tConvertQueryAddrToEpSet(&CEp.epSet, &pTaskInfo->addr);
/*mDebug("subscribe convert ep %d %s %s %s %s %s\n", CEp.epSet.numOfEps, CEp.epSet.fqdn[0], CEp.epSet.fqdn[1], CEp.epSet.fqdn[2], CEp.epSet.fqdn[3], CEp.epSet.fqdn[4]);*/ /*mDebug("subscribe convert ep %d %s %s %s %s %s\n", CEp.epSet.numOfEps, CEp.epSet.fqdn[0], CEp.epSet.fqdn[1],
* CEp.epSet.fqdn[2], CEp.epSet.fqdn[3], CEp.epSet.fqdn[4]);*/
CEp.vgId = pTaskInfo->addr.nodeId; CEp.vgId = pTaskInfo->addr.nodeId;
CEp.qmsgLen = pTaskInfo->msg->contentLen; CEp.qmsg = strdup(pTaskInfo->msg->msg);
CEp.qmsg = malloc(CEp.qmsgLen); CEp.qmsgLen = strlen(CEp.qmsg) + 1;
if (CEp.qmsg == NULL) { printf("abc:\n%s\n", CEp.qmsg);
return -1; /*CEp.qmsg = malloc(CEp.qmsgLen);*/
} /*if (CEp.qmsg == NULL) {*/
memcpy(CEp.qmsg, pTaskInfo->msg->msg, pTaskInfo->msg->contentLen); /*return -1;*/
/*}*/
/*memcpy(CEp.qmsg, pTaskInfo->msg->msg, pTaskInfo->msg->contentLen);*/
taosArrayPush(unassignedVg, &CEp); taosArrayPush(unassignedVg, &CEp);
} }
...@@ -184,7 +242,7 @@ static int mndInitUnassignedVg(SMnode *pMnode, SMqTopicObj *pTopic, SArray *unas ...@@ -184,7 +242,7 @@ static int mndInitUnassignedVg(SMnode *pMnode, SMqTopicObj *pTopic, SArray *unas
} }
static int mndBuildMqSetConsumerVgReq(SMnode *pMnode, STrans *pTrans, SMqConsumerObj *pConsumer, static int mndBuildMqSetConsumerVgReq(SMnode *pMnode, STrans *pTrans, SMqConsumerObj *pConsumer,
SMqConsumerTopic *pConsumerTopic, SMqTopicObj *pTopic) { SMqConsumerTopic *pConsumerTopic, SMqTopicObj *pTopic, SMqConsumerEp *pCEp) {
int32_t sz = taosArrayGetSize(pConsumerTopic->pVgInfo); int32_t sz = taosArrayGetSize(pConsumerTopic->pVgInfo);
for (int32_t i = 0; i < sz; i++) { for (int32_t i = 0; i < sz; i++) {
int32_t vgId = *(int32_t *)taosArrayGet(pConsumerTopic->pVgInfo, i); int32_t vgId = *(int32_t *)taosArrayGet(pConsumerTopic->pVgInfo, i);
...@@ -199,6 +257,8 @@ static int mndBuildMqSetConsumerVgReq(SMnode *pMnode, STrans *pTrans, SMqConsume ...@@ -199,6 +257,8 @@ static int mndBuildMqSetConsumerVgReq(SMnode *pMnode, STrans *pTrans, SMqConsume
req.sql = pTopic->sql; req.sql = pTopic->sql;
req.logicalPlan = pTopic->logicalPlan; req.logicalPlan = pTopic->logicalPlan;
req.physicalPlan = pTopic->physicalPlan; req.physicalPlan = pTopic->physicalPlan;
req.qmsg = strdup(pCEp->qmsg);
req.qmsgLen = strlen(req.qmsg);
int32_t tlen = tEncodeSMqSetCVgReq(NULL, &req); int32_t tlen = tEncodeSMqSetCVgReq(NULL, &req);
void *buf = malloc(sizeof(SMsgHead) + tlen); void *buf = malloc(sizeof(SMsgHead) + tlen);
if (buf == NULL) { if (buf == NULL) {
...@@ -206,18 +266,18 @@ static int mndBuildMqSetConsumerVgReq(SMnode *pMnode, STrans *pTrans, SMqConsume ...@@ -206,18 +266,18 @@ static int mndBuildMqSetConsumerVgReq(SMnode *pMnode, STrans *pTrans, SMqConsume
return -1; return -1;
} }
SMsgHead* pMsgHead = (SMsgHead*)buf; SMsgHead *pMsgHead = (SMsgHead *)buf;
pMsgHead->contLen = htonl(sizeof(SMsgHead) + tlen); pMsgHead->contLen = htonl(sizeof(SMsgHead) + tlen);
pMsgHead->vgId = htonl(vgId); pMsgHead->vgId = htonl(vgId);
void* abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); void *abuf = POINTER_SHIFT(buf, sizeof(SMsgHead));
tEncodeSMqSetCVgReq(&abuf, &req); tEncodeSMqSetCVgReq(&abuf, &req);
STransAction action = {0}; STransAction action = {0};
action.epSet = mndGetVgroupEpset(pMnode, pVgObj); action.epSet = mndGetVgroupEpset(pMnode, pVgObj);
action.pCont = buf; action.pCont = buf;
action.contLen = tlen; action.contLen = sizeof(SMsgHead) + tlen;
action.msgType = TDMT_VND_MQ_SET_CONN; action.msgType = TDMT_VND_MQ_SET_CONN;
mndReleaseVgroup(pMnode, pVgObj); mndReleaseVgroup(pMnode, pVgObj);
...@@ -285,7 +345,7 @@ static SSdbRow *mndSubActionDecode(SSdbRaw *pRaw) { ...@@ -285,7 +345,7 @@ static SSdbRow *mndSubActionDecode(SSdbRaw *pRaw) {
int32_t dataPos = 0; int32_t dataPos = 0;
int32_t tlen; int32_t tlen;
SDB_GET_INT32(pRaw, dataPos, &tlen, SUB_DECODE_OVER); SDB_GET_INT32(pRaw, dataPos, &tlen, SUB_DECODE_OVER);
void *buf = malloc(tlen + 1); void *buf = malloc(tlen + 1);
if (buf == NULL) goto SUB_DECODE_OVER; if (buf == NULL) goto SUB_DECODE_OVER;
SDB_GET_BINARY(pRaw, dataPos, buf, tlen, SUB_DECODE_OVER); SDB_GET_BINARY(pRaw, dataPos, buf, tlen, SUB_DECODE_OVER);
SDB_GET_RESERVE(pRaw, dataPos, MND_SUBSCRIBE_RESERVE_SIZE, SUB_DECODE_OVER); SDB_GET_RESERVE(pRaw, dataPos, MND_SUBSCRIBE_RESERVE_SIZE, SUB_DECODE_OVER);
...@@ -493,11 +553,11 @@ static int32_t mndProcessSubscribeReq(SMnodeMsg *pMsg) { ...@@ -493,11 +553,11 @@ static int32_t mndProcessSubscribeReq(SMnodeMsg *pMsg) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1; return -1;
} }
char* key = mndMakeSubscribeKey(consumerGroup, newTopicName); char *key = mndMakeSubscribeKey(consumerGroup, newTopicName);
strcpy(pSub->key, key); strcpy(pSub->key, key);
// set unassigned vg // set unassigned vg
mndInitUnassignedVg(pMnode, pTopic, pSub->unassignedVg); mndInitUnassignedVg(pMnode, pTopic, pSub->unassignedVg);
//TODO: disable alter // TODO: disable alter
} }
taosArrayPush(pSub->availConsumer, &consumerId); taosArrayPush(pSub->availConsumer, &consumerId);
...@@ -506,12 +566,15 @@ static int32_t mndProcessSubscribeReq(SMnodeMsg *pMsg) { ...@@ -506,12 +566,15 @@ static int32_t mndProcessSubscribeReq(SMnodeMsg *pMsg) {
if (taosArrayGetSize(pConsumerTopic->pVgInfo) > 0) { if (taosArrayGetSize(pConsumerTopic->pVgInfo) > 0) {
ASSERT(taosArrayGetSize(pConsumerTopic->pVgInfo) == 1); ASSERT(taosArrayGetSize(pConsumerTopic->pVgInfo) == 1);
int32_t vgId = *(int32_t *)taosArrayGetLast(pConsumerTopic->pVgInfo); int32_t vgId = *(int32_t *)taosArrayGetLast(pConsumerTopic->pVgInfo);
// send setmsg to vnode SMqConsumerEp *pCEp = taosArrayGetLast(pSub->assigned);
if (mndBuildMqSetConsumerVgReq(pMnode, pTrans, pConsumer, pConsumerTopic, pTopic) < 0) { if (pCEp->vgId == vgId) {
// TODO if (mndBuildMqSetConsumerVgReq(pMnode, pTrans, pConsumer, pConsumerTopic, pTopic, pCEp) < 0) {
return -1; // TODO
return -1;
}
} }
// send setmsg to vnode
} }
SSdbRaw *pRaw = mndSubActionEncode(pSub); SSdbRaw *pRaw = mndSubActionEncode(pSub);
......
...@@ -318,8 +318,8 @@ int tqRegisterContext(STqGroup*, void* ahandle); ...@@ -318,8 +318,8 @@ int tqRegisterContext(STqGroup*, void* ahandle);
int tqSendLaunchQuery(STqMsgItem*, int64_t offset); int tqSendLaunchQuery(STqMsgItem*, int64_t offset);
#endif #endif
int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg** ppRsp); int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg);
int32_t tqProcessSetConnReq(STQ* pTq, char* msg, SRpcMsg** ppRsp); int32_t tqProcessSetConnReq(STQ* pTq, char* msg);
#ifdef __cplusplus #ifdef __cplusplus
} }
......
...@@ -104,8 +104,7 @@ typedef void* tsdbReadHandleT; ...@@ -104,8 +104,7 @@ typedef void* tsdbReadHandleT;
* @param qinfo query info handle from query processor * @param qinfo query info handle from query processor
* @return * @return
*/ */
tsdbReadHandleT *tsdbQueryTables(STsdb *tsdb, STsdbQueryCond *pCond, STableGroupInfo *tableInfoGroup, uint64_t qId, tsdbReadHandleT *tsdbQueryTables(STsdb *tsdb, STsdbQueryCond *pCond, STableGroupInfo *tableInfoGroup, uint64_t qId, uint64_t taskId);
void *pRef);
/** /**
* Get the last row of the given query time window for all the tables in STableGroupInfo object. * Get the last row of the given query time window for all the tables in STableGroupInfo object.
......
...@@ -159,20 +159,18 @@ int vnodeProcessSyncReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp); ...@@ -159,20 +159,18 @@ int vnodeProcessSyncReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp);
* *
* @param pVnode The vnode object. * @param pVnode The vnode object.
* @param pMsg The request message * @param pMsg The request message
* @param pRsp The response message
* @return int 0 for success, -1 for failure * @return int 0 for success, -1 for failure
*/ */
int vnodeProcessQueryReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp); int vnodeProcessQueryMsg(SVnode *pVnode, SRpcMsg *pMsg);
/** /**
* @brief Process a fetch message. * @brief Process a fetch message.
* *
* @param pVnode The vnode object. * @param pVnode The vnode object.
* @param pMsg The request message * @param pMsg The request message
* @param pRsp The response message
* @return int 0 for success, -1 for failure * @return int 0 for success, -1 for failure
*/ */
int vnodeProcessFetchReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp); int vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg);
/* ------------------------ SVnodeCfg ------------------------ */ /* ------------------------ SVnodeCfg ------------------------ */
/** /**
......
...@@ -667,8 +667,8 @@ int tqItemSSize() { ...@@ -667,8 +667,8 @@ int tqItemSSize() {
} }
#endif #endif
int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg** ppRsp) { int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg) {
SMqConsumeReq* pReq = pMsg->pCont; SMqConsumeReq* pReq = pMsg->pCont;
SRpcMsg rpcMsg; SRpcMsg rpcMsg;
int64_t reqId = pReq->reqId; int64_t reqId = pReq->reqId;
int64_t consumerId = pReq->consumerId; int64_t consumerId = pReq->consumerId;
...@@ -783,7 +783,7 @@ int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg** ppRsp) { ...@@ -783,7 +783,7 @@ int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg** ppRsp) {
return 0; return 0;
} }
int32_t tqProcessSetConnReq(STQ* pTq, char* msg, SRpcMsg** ppRsp) { int32_t tqProcessSetConnReq(STQ* pTq, char* msg) {
SMqSetCVgReq req; SMqSetCVgReq req;
tDecodeSMqSetCVgReq(msg, &req); tDecodeSMqSetCVgReq(msg, &req);
STqConsumerHandle* pConsumer = calloc(sizeof(STqConsumerHandle), 1); STqConsumerHandle* pConsumer = calloc(sizeof(STqConsumerHandle), 1);
...@@ -811,7 +811,7 @@ int32_t tqProcessSetConnReq(STQ* pTq, char* msg, SRpcMsg** ppRsp) { ...@@ -811,7 +811,7 @@ int32_t tqProcessSetConnReq(STQ* pTq, char* msg, SRpcMsg** ppRsp) {
for (int i = 0; i < TQ_BUFFER_SIZE; i++) { for (int i = 0; i < TQ_BUFFER_SIZE; i++) {
pTopic->buffer.output[i].status = 0; pTopic->buffer.output[i].status = 0;
STqReadHandle* pReadHandle = tqInitSubmitMsgScanner(pTq->pMeta); STqReadHandle* pReadHandle = tqInitSubmitMsgScanner(pTq->pMeta);
pTopic->buffer.output[i].task = qCreateStreamExecTaskInfo(&req.qmsg, pReadHandle); pTopic->buffer.output[i].task = qCreateStreamExecTaskInfo(req.qmsg, pReadHandle);
} }
taosArrayPush(pConsumer->topics, pTopic); taosArrayPush(pConsumer->topics, pTopic);
terrno = TSDB_CODE_SUCCESS; terrno = TSDB_CODE_SUCCESS;
......
...@@ -98,6 +98,8 @@ int32_t vnodePutReqToVQueryQ(SVnode* pVnode, struct SRpcMsg* pReq) { ...@@ -98,6 +98,8 @@ int32_t vnodePutReqToVQueryQ(SVnode* pVnode, struct SRpcMsg* pReq) {
/* ------------------------ STATIC METHODS ------------------------ */ /* ------------------------ STATIC METHODS ------------------------ */
static void* loop(void* arg) { static void* loop(void* arg) {
setThreadName("vnode-commit");
SVnodeTask* pTask; SVnodeTask* pTask;
for (;;) { for (;;) {
pthread_mutex_lock(&(vnodeMgr.mutex)); pthread_mutex_lock(&(vnodeMgr.mutex));
......
...@@ -17,14 +17,14 @@ ...@@ -17,14 +17,14 @@
#include "vnd.h" #include "vnd.h"
static int32_t vnodeGetTableList(SVnode *pVnode, SRpcMsg *pMsg); static int32_t vnodeGetTableList(SVnode *pVnode, SRpcMsg *pMsg);
static int vnodeGetTableMeta(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp); static int vnodeGetTableMeta(SVnode *pVnode, SRpcMsg *pMsg);
int vnodeQueryOpen(SVnode *pVnode) { int vnodeQueryOpen(SVnode *pVnode) {
return qWorkerInit(NODE_TYPE_VNODE, pVnode->vgId, NULL, (void **)&pVnode->pQuery, pVnode, return qWorkerInit(NODE_TYPE_VNODE, pVnode->vgId, NULL, (void **)&pVnode->pQuery, pVnode,
(putReqToQueryQFp)vnodePutReqToVQueryQ); (putReqToQueryQFp)vnodePutReqToVQueryQ);
} }
int vnodeProcessQueryReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { int vnodeProcessQueryMsg(SVnode *pVnode, SRpcMsg *pMsg) {
vTrace("message in query queue is processing"); vTrace("message in query queue is processing");
switch (pMsg->msgType) { switch (pMsg->msgType) {
...@@ -38,7 +38,7 @@ int vnodeProcessQueryReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { ...@@ -38,7 +38,7 @@ int vnodeProcessQueryReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) {
} }
} }
int vnodeProcessFetchReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { int vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg) {
vTrace("message in fetch queue is processing"); vTrace("message in fetch queue is processing");
switch (pMsg->msgType) { switch (pMsg->msgType) {
case TDMT_VND_FETCH: case TDMT_VND_FETCH:
...@@ -57,24 +57,24 @@ int vnodeProcessFetchReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { ...@@ -57,24 +57,24 @@ int vnodeProcessFetchReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) {
return vnodeGetTableList(pVnode, pMsg); return vnodeGetTableList(pVnode, pMsg);
// return qWorkerProcessShowFetchMsg(pVnode->pMeta, pVnode->pQuery, pMsg); // return qWorkerProcessShowFetchMsg(pVnode->pMeta, pVnode->pQuery, pMsg);
case TDMT_VND_TABLE_META: case TDMT_VND_TABLE_META:
return vnodeGetTableMeta(pVnode, pMsg, pRsp); return vnodeGetTableMeta(pVnode, pMsg);
case TDMT_VND_CONSUME: case TDMT_VND_CONSUME:
return tqProcessConsumeReq(pVnode->pTq, pMsg, pRsp); return tqProcessConsumeReq(pVnode->pTq, pMsg);
default: default:
vError("unknown msg type:%d in fetch queue", pMsg->msgType); vError("unknown msg type:%d in fetch queue", pMsg->msgType);
return TSDB_CODE_VND_APP_ERROR; return TSDB_CODE_VND_APP_ERROR;
} }
} }
static int vnodeGetTableMeta(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { static int vnodeGetTableMeta(SVnode *pVnode, SRpcMsg *pMsg) {
STableInfoReq * pReq = (STableInfoReq *)(pMsg->pCont); STableInfoReq * pReq = (STableInfoReq *)(pMsg->pCont);
STbCfg * pTbCfg = NULL; STbCfg * pTbCfg = NULL;
STbCfg * pStbCfg = NULL; STbCfg * pStbCfg = NULL;
tb_uid_t uid; tb_uid_t uid;
int32_t nCols; int32_t nCols;
int32_t nTagCols; int32_t nTagCols;
SSchemaWrapper *pSW; SSchemaWrapper *pSW = NULL;
STableMetaRsp * pTbMetaMsg = NULL; STableMetaRsp *pTbMetaMsg = NULL;
SSchema * pTagSchema; SSchema * pTagSchema;
SRpcMsg rpcMsg; SRpcMsg rpcMsg;
int msgLen = 0; int msgLen = 0;
...@@ -145,15 +145,22 @@ static int vnodeGetTableMeta(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { ...@@ -145,15 +145,22 @@ static int vnodeGetTableMeta(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) {
_exit: _exit:
free(pSW->pSchema); if (pSW != NULL) {
free(pSW); tfree(pSW->pSchema);
free(pTbCfg->name); tfree(pSW);
free(pTbCfg);
if (pTbCfg->type == META_SUPER_TABLE) {
free(pTbCfg->stbCfg.pTagSchema);
} else if (pTbCfg->type == META_SUPER_TABLE) {
kvRowFree(pTbCfg->ctbCfg.pTag);
} }
if (pTbCfg) {
tfree(pTbCfg->name);
if (pTbCfg->type == META_SUPER_TABLE) {
free(pTbCfg->stbCfg.pTagSchema);
} else if (pTbCfg->type == META_SUPER_TABLE) {
kvRowFree(pTbCfg->ctbCfg.pTag);
}
tfree(pTbCfg);
}
rpcMsg.handle = pMsg->handle; rpcMsg.handle = pMsg->handle;
rpcMsg.ahandle = pMsg->ahandle; rpcMsg.ahandle = pMsg->ahandle;
rpcMsg.pCont = pTbMetaMsg; rpcMsg.pCont = pTbMetaMsg;
......
...@@ -115,7 +115,7 @@ int vnodeApplyWMsg(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { ...@@ -115,7 +115,7 @@ int vnodeApplyWMsg(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) {
} }
break; break;
case TDMT_VND_MQ_SET_CONN: { case TDMT_VND_MQ_SET_CONN: {
if (tqProcessSetConnReq(pVnode->pTq, POINTER_SHIFT(ptr, sizeof(SMsgHead)), NULL) < 0) { if (tqProcessSetConnReq(pVnode->pTq, POINTER_SHIFT(ptr, sizeof(SMsgHead))) < 0) {
// TODO: handle error // TODO: handle error
} }
} break; } break;
......
...@@ -38,7 +38,7 @@ ...@@ -38,7 +38,7 @@
#define GET_RES_WINDOW_KEY_LEN(_l) ((_l) + sizeof(uint64_t)) #define GET_RES_WINDOW_KEY_LEN(_l) ((_l) + sizeof(uint64_t))
#define GET_RES_EXT_WINDOW_KEY_LEN(_l) ((_l) + sizeof(uint64_t) + POINTER_BYTES) #define GET_RES_EXT_WINDOW_KEY_LEN(_l) ((_l) + sizeof(uint64_t) + POINTER_BYTES)
#define GET_TASKID(_t) (((SExecTaskInfo*)(_t))->id.queryId) #define GET_TASKID(_t) (((SExecTaskInfo*)(_t))->id.idstr)
#define curTimeWindowIndex(_winres) ((_winres)->curIndex) #define curTimeWindowIndex(_winres) ((_winres)->curIndex)
......
...@@ -240,6 +240,7 @@ typedef struct STaskIdInfo { ...@@ -240,6 +240,7 @@ typedef struct STaskIdInfo {
uint64_t subplanId; uint64_t subplanId;
uint64_t templateId; uint64_t templateId;
uint64_t taskId; // this is a subplan id uint64_t taskId; // this is a subplan id
char *idstr;
} STaskIdInfo; } STaskIdInfo;
typedef struct SExecTaskInfo { typedef struct SExecTaskInfo {
...@@ -250,9 +251,8 @@ typedef struct SExecTaskInfo { ...@@ -250,9 +251,8 @@ typedef struct SExecTaskInfo {
STaskCostInfo cost; STaskCostInfo cost;
int64_t owner; // if it is in execution int64_t owner; // if it is in execution
int32_t code; int32_t code;
uint64_t totalRows; // total number of rows
STableGroupInfo tableqinfoGroupInfo; // this is a group array list, including SArray<STableQueryInfo*> structure STableGroupInfo tableqinfoGroupInfo; // this is a group array list, including SArray<STableQueryInfo*> structure
pthread_mutex_t lock; // used to synchronize the rsp/query threads
char *sql; // query sql string char *sql; // query sql string
jmp_buf env; // jmp_buf env; //
struct SOperatorInfo *pRoot; struct SOperatorInfo *pRoot;
...@@ -378,8 +378,10 @@ typedef struct SExchangeInfo { ...@@ -378,8 +378,10 @@ typedef struct SExchangeInfo {
SSDataBlock *pResult; SSDataBlock *pResult;
int32_t current; int32_t current;
uint64_t rowsOfCurrentSource; uint64_t rowsOfCurrentSource;
uint64_t bytes; // total load bytes from remote
uint64_t totalRows; uint64_t totalSize; // total load bytes from remote
uint64_t totalRows; // total number of rows
uint64_t totalElapsed;// total elapsed time
} SExchangeInfo; } SExchangeInfo;
typedef struct STableScanInfo { typedef struct STableScanInfo {
...@@ -622,8 +624,6 @@ int32_t createIndirectQueryFuncExprFromMsg(SQueryTableReq *pQueryMsg, int32_t nu ...@@ -622,8 +624,6 @@ int32_t createIndirectQueryFuncExprFromMsg(SQueryTableReq *pQueryMsg, int32_t nu
int32_t createQueryFilter(char *data, uint16_t len, SFilterInfo** pFilters); int32_t createQueryFilter(char *data, uint16_t len, SFilterInfo** pFilters);
SGroupbyExpr *createGroupbyExprFromMsg(SQueryTableReq *pQueryMsg, SColIndex *pColIndex, int32_t *code); SGroupbyExpr *createGroupbyExprFromMsg(SQueryTableReq *pQueryMsg, SColIndex *pColIndex, int32_t *code);
SQInfo *createQInfoImpl(SQueryTableReq *pQueryMsg, SGroupbyExpr *pGroupbyExpr, SExprInfo *pExprs,
SExprInfo *pSecExprs, STableGroupInfo *pTableGroupInfo, SColumnInfo* pTagCols, SFilterInfo* pFilters, int32_t vgId, char* sql, uint64_t qId, struct SUdfInfo* pUdfInfo);
int32_t initQInfo(STsBufInfo* pTsBufInfo, void* tsdb, void* sourceOptr, SQInfo* pQInfo, STaskParam* param, char* start, int32_t initQInfo(STsBufInfo* pTsBufInfo, void* tsdb, void* sourceOptr, SQInfo* pQInfo, STaskParam* param, char* start,
int32_t prevResultLen, void* merger); int32_t prevResultLen, void* merger);
...@@ -645,26 +645,24 @@ void setQueryStatus(STaskRuntimeEnv *pRuntimeEnv, int8_t status); ...@@ -645,26 +645,24 @@ void setQueryStatus(STaskRuntimeEnv *pRuntimeEnv, int8_t status);
bool onlyQueryTags(STaskAttr* pQueryAttr); bool onlyQueryTags(STaskAttr* pQueryAttr);
//void destroyUdfInfo(struct SUdfInfo* pUdfInfo); //void destroyUdfInfo(struct SUdfInfo* pUdfInfo);
bool isValidQInfo(void *param);
int32_t doDumpQueryResult(SQInfo *pQInfo, char *data, int8_t compressed, int32_t *compLen); int32_t doDumpQueryResult(SQInfo *pQInfo, char *data, int8_t compressed, int32_t *compLen);
size_t getResultSize(SQInfo *pQInfo, int64_t *numOfRows); size_t getResultSize(SQInfo *pQInfo, int64_t *numOfRows);
void setQueryKilled(SQInfo *pQInfo); void setTaskKilled(SExecTaskInfo *pTaskInfo);
void publishOperatorProfEvent(SOperatorInfo* operatorInfo, EQueryProfEventType eventType); void publishOperatorProfEvent(SOperatorInfo* operatorInfo, EQueryProfEventType eventType);
void publishQueryAbortEvent(SExecTaskInfo * pTaskInfo, int32_t code); void publishQueryAbortEvent(SExecTaskInfo * pTaskInfo, int32_t code);
void calculateOperatorProfResults(SQInfo* pQInfo); void calculateOperatorProfResults(SQInfo* pQInfo);
void queryCostStatis(SQInfo *pQInfo); void queryCostStatis(SExecTaskInfo *pTaskInfo);
void doDestroyTask(SQInfo *pQInfo); void doDestroyTask(SExecTaskInfo *pTaskInfo);
void freeQueryAttr(STaskAttr *pQuery); void freeQueryAttr(STaskAttr *pQuery);
int32_t getMaximumIdleDurationSec(); int32_t getMaximumIdleDurationSec();
void doInvokeUdf(struct SUdfInfo* pUdfInfo, SQLFunctionCtx *pCtx, int32_t idx, int32_t type); void doInvokeUdf(struct SUdfInfo* pUdfInfo, SQLFunctionCtx *pCtx, int32_t idx, int32_t type);
void setTaskStatus(SExecTaskInfo *pTaskInfo, int8_t status); void setTaskStatus(SExecTaskInfo *pTaskInfo, int8_t status);
int32_t createExecTaskInfoImpl(SSubplan* pPlan, SExecTaskInfo** pTaskInfo, void* readerHandle); int32_t createExecTaskInfoImpl(SSubplan* pPlan, SExecTaskInfo** pTaskInfo, void* readerHandle, uint64_t taskId);
#endif // TDENGINE_EXECUTORIMPL_H #endif // TDENGINE_EXECUTORIMPL_H
...@@ -121,11 +121,19 @@ static void toDataCacheEntry(const SDataDispatchHandle* pHandle, const SInputDat ...@@ -121,11 +121,19 @@ static void toDataCacheEntry(const SDataDispatchHandle* pHandle, const SInputDat
} }
static bool allocBuf(SDataDispatchHandle* pDispatcher, const SInputData* pInput, SDataDispatchBuf* pBuf) { static bool allocBuf(SDataDispatchHandle* pDispatcher, const SInputData* pInput, SDataDispatchBuf* pBuf) {
if (taosQueueSize(pDispatcher->pDataBlocks) >= pDispatcher->pManager->cfg.maxDataBlockNumPerQuery) { uint32_t capacity = pDispatcher->pManager->cfg.maxDataBlockNumPerQuery;
if (taosQueueSize(pDispatcher->pDataBlocks) > capacity) {
qError("SinkNode queue is full, no capacity, max:%d, current:%d, no capacity", capacity,
taosQueueSize(pDispatcher->pDataBlocks));
return false; return false;
} }
pBuf->allocSize = DATA_META_LENGTH(pInput->pTableRetrieveTsMap) + pDispatcher->schema.resultRowSize * pInput->pData->info.rows; pBuf->allocSize = DATA_META_LENGTH(pInput->pTableRetrieveTsMap) + pDispatcher->schema.resultRowSize * pInput->pData->info.rows;
pBuf->pData = malloc(pBuf->allocSize); pBuf->pData = malloc(pBuf->allocSize);
if (pBuf->pData == NULL) {
qError("SinkNode failed to malloc memory, size:%d, code:%d", pBuf->allocSize, TAOS_SYSTEM_ERROR(errno));
}
return NULL != pBuf->pData; return NULL != pBuf->pData;
} }
......
...@@ -50,11 +50,11 @@ int32_t qSetStreamInput(qTaskInfo_t tinfo, void* input) { ...@@ -50,11 +50,11 @@ int32_t qSetStreamInput(qTaskInfo_t tinfo, void* input) {
SExecTaskInfo* pTaskInfo = (SExecTaskInfo*) tinfo; SExecTaskInfo* pTaskInfo = (SExecTaskInfo*) tinfo;
int32_t code = doSetStreamBlock(pTaskInfo->pRoot, input, GET_TASKID(pTaskInfo)); int32_t code = doSetStreamBlock(pTaskInfo->pRoot, input, pTaskInfo->id.queryId);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
qError("failed to set the stream block data, reqId:0x%"PRIx64, GET_TASKID(pTaskInfo)); qError("%s failed to set the stream block data", GET_TASKID(pTaskInfo));
} else { } else {
qDebug("set the stream block successfully, reqId:0x%"PRIx64, GET_TASKID(pTaskInfo)); qDebug("%s set the stream block successfully", GET_TASKID(pTaskInfo));
} }
return code; return code;
...@@ -81,7 +81,7 @@ qTaskInfo_t qCreateStreamExecTaskInfo(void* msg, void* streamReadHandle) { ...@@ -81,7 +81,7 @@ qTaskInfo_t qCreateStreamExecTaskInfo(void* msg, void* streamReadHandle) {
} }
qTaskInfo_t pTaskInfo = NULL; qTaskInfo_t pTaskInfo = NULL;
code = qCreateExecTask(streamReadHandle, 0, plan, &pTaskInfo, NULL); code = qCreateExecTask(streamReadHandle, 0, 0, plan, &pTaskInfo, NULL);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
// TODO: destroy SSubplan & pTaskInfo // TODO: destroy SSubplan & pTaskInfo
terrno = code; terrno = code;
......
...@@ -69,11 +69,11 @@ void freeParam(STaskParam *param) { ...@@ -69,11 +69,11 @@ void freeParam(STaskParam *param) {
tfree(param->prevResult); tfree(param->prevResult);
} }
int32_t qCreateExecTask(void* readHandle, int32_t vgId, SSubplan* pSubplan, qTaskInfo_t* pTaskInfo, DataSinkHandle* handle) { int32_t qCreateExecTask(void* readHandle, int32_t vgId, uint64_t taskId, SSubplan* pSubplan, qTaskInfo_t* pTaskInfo, DataSinkHandle* handle) {
assert(readHandle != NULL && pSubplan != NULL); assert(readHandle != NULL && pSubplan != NULL);
SExecTaskInfo** pTask = (SExecTaskInfo**)pTaskInfo; SExecTaskInfo** pTask = (SExecTaskInfo**)pTaskInfo;
int32_t code = createExecTaskInfoImpl(pSubplan, pTask, readHandle); int32_t code = createExecTaskInfoImpl(pSubplan, pTask, readHandle, taskId);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
goto _error; goto _error;
} }
...@@ -83,8 +83,9 @@ int32_t qCreateExecTask(void* readHandle, int32_t vgId, SSubplan* pSubplan, qTas ...@@ -83,8 +83,9 @@ int32_t qCreateExecTask(void* readHandle, int32_t vgId, SSubplan* pSubplan, qTas
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
goto _error; goto _error;
} }
if (handle) {
code = dsCreateDataSinker(pSubplan->pDataSink, handle); code = dsCreateDataSinker(pSubplan->pDataSink, handle);
}
_error: _error:
// if failed to add ref for all tables in this query, abort current query // if failed to add ref for all tables in this query, abort current query
...@@ -140,16 +141,10 @@ int32_t qExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) { ...@@ -140,16 +141,10 @@ int32_t qExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) {
SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo;
int64_t threadId = taosGetSelfPthreadId(); int64_t threadId = taosGetSelfPthreadId();
// todo: remove it.
if (tinfo == NULL) {
return TSDB_CODE_SUCCESS;
}
*pRes = NULL; *pRes = NULL;
int64_t curOwner = 0; int64_t curOwner = 0;
if ((curOwner = atomic_val_compare_exchange_64(&pTaskInfo->owner, 0, threadId)) != 0) { if ((curOwner = atomic_val_compare_exchange_64(&pTaskInfo->owner, 0, threadId)) != 0) {
qError("QInfo:0x%" PRIx64 "-%p qhandle is now executed by thread:%p", GET_TASKID(pTaskInfo), pTaskInfo, qError("%s-%p execTask is now executed by thread:%p", GET_TASKID(pTaskInfo), pTaskInfo,
(void*)curOwner); (void*)curOwner);
pTaskInfo->code = TSDB_CODE_QRY_IN_EXEC; pTaskInfo->code = TSDB_CODE_QRY_IN_EXEC;
return pTaskInfo->code; return pTaskInfo->code;
...@@ -160,7 +155,7 @@ int32_t qExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) { ...@@ -160,7 +155,7 @@ int32_t qExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) {
} }
if (isTaskKilled(pTaskInfo)) { if (isTaskKilled(pTaskInfo)) {
qDebug("QInfo:0x%" PRIx64 " it is already killed, abort", GET_TASKID(pTaskInfo)); qDebug("%s already killed, abort", GET_TASKID(pTaskInfo));
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
...@@ -169,12 +164,12 @@ int32_t qExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) { ...@@ -169,12 +164,12 @@ int32_t qExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) {
if (ret != TSDB_CODE_SUCCESS) { if (ret != TSDB_CODE_SUCCESS) {
publishQueryAbortEvent(pTaskInfo, ret); publishQueryAbortEvent(pTaskInfo, ret);
pTaskInfo->code = ret; pTaskInfo->code = ret;
qDebug("QInfo:0x%" PRIx64 " query abort due to error/cancel occurs, code:%s", GET_TASKID(pTaskInfo), qDebug("%s task abort due to error/cancel occurs, code:%s", GET_TASKID(pTaskInfo),
tstrerror(pTaskInfo->code)); tstrerror(pTaskInfo->code));
return pTaskInfo->code; return pTaskInfo->code;
} }
qDebug("QInfo:0x%" PRIx64 " query task is launched", GET_TASKID(pTaskInfo)); qDebug("%s execTask is launched", GET_TASKID(pTaskInfo));
bool newgroup = false; bool newgroup = false;
publishOperatorProfEvent(pTaskInfo->pRoot, QUERY_PROF_BEFORE_OPERATOR_EXEC); publishOperatorProfEvent(pTaskInfo->pRoot, QUERY_PROF_BEFORE_OPERATOR_EXEC);
...@@ -183,66 +178,25 @@ int32_t qExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) { ...@@ -183,66 +178,25 @@ int32_t qExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) {
st = taosGetTimestampUs(); st = taosGetTimestampUs();
*pRes = pTaskInfo->pRoot->exec(pTaskInfo->pRoot, &newgroup); *pRes = pTaskInfo->pRoot->exec(pTaskInfo->pRoot, &newgroup);
pTaskInfo->cost.elapsedTime += (taosGetTimestampUs() - st); uint64_t el = (taosGetTimestampUs() - st);
pTaskInfo->cost.elapsedTime += el;
publishOperatorProfEvent(pTaskInfo->pRoot, QUERY_PROF_AFTER_OPERATOR_EXEC); publishOperatorProfEvent(pTaskInfo->pRoot, QUERY_PROF_AFTER_OPERATOR_EXEC);
if (NULL == *pRes) { if (NULL == *pRes) {
*useconds = pTaskInfo->cost.elapsedTime; *useconds = pTaskInfo->cost.elapsedTime;
} }
qDebug("QInfo:0x%" PRIx64 " query paused, %d rows returned, total:%" PRId64 " rows, in sinkNode:%d", int32_t current = (*pRes != NULL)? (*pRes)->info.rows:0;
GET_TASKID(pTaskInfo), 0, 0L, 0); pTaskInfo->totalRows += current;
qDebug("%s task suspended, %d rows returned, total:%" PRId64 " rows, in sinkNode:%d, elapsed:%.2f ms",
GET_TASKID(pTaskInfo), current, pTaskInfo->totalRows, 0, el/1000.0);
atomic_store_64(&pTaskInfo->owner, 0); atomic_store_64(&pTaskInfo->owner, 0);
return pTaskInfo->code; return pTaskInfo->code;
} }
int32_t qRetrieveQueryResultInfo(qTaskInfo_t qinfo, bool* buildRes, void* pRspContext) {
SQInfo *pQInfo = (SQInfo *)qinfo;
if (pQInfo == NULL || !isValidQInfo(pQInfo)) {
qError("QInfo invalid qhandle");
return TSDB_CODE_QRY_INVALID_QHANDLE;
}
*buildRes = false;
if (IS_QUERY_KILLED(pQInfo)) {
qDebug("QInfo:0x%"PRIx64" query is killed, code:0x%08x", pQInfo->qId, pQInfo->code);
return pQInfo->code;
}
int32_t code = TSDB_CODE_SUCCESS;
if (tsRetrieveBlockingModel) {
pQInfo->rspContext = pRspContext;
tsem_wait(&pQInfo->ready);
*buildRes = true;
code = pQInfo->code;
} else {
STaskRuntimeEnv* pRuntimeEnv = &pQInfo->runtimeEnv;
STaskAttr *pQueryAttr = pQInfo->runtimeEnv.pQueryAttr;
pthread_mutex_lock(&pQInfo->lock);
assert(pQInfo->rspContext == NULL);
if (pQInfo->dataReady == QUERY_RESULT_READY) {
*buildRes = true;
qDebug("QInfo:0x%"PRIx64" retrieve result info, rowsize:%d, rows:%d, code:%s", pQInfo->qId, pQueryAttr->resultRowSize,
GET_NUM_OF_RESULTS(pRuntimeEnv), tstrerror(pQInfo->code));
} else {
*buildRes = false;
qDebug("QInfo:0x%"PRIx64" retrieve req set query return result after paused", pQInfo->qId);
pQInfo->rspContext = pRspContext;
assert(pQInfo->rspContext != NULL);
}
code = pQInfo->code;
pthread_mutex_unlock(&pQInfo->lock);
}
return code;
}
void* qGetResultRetrieveMsg(qTaskInfo_t qinfo) { void* qGetResultRetrieveMsg(qTaskInfo_t qinfo) {
SQInfo* pQInfo = (SQInfo*) qinfo; SQInfo* pQInfo = (SQInfo*) qinfo;
assert(pQInfo != NULL); assert(pQInfo != NULL);
...@@ -251,18 +205,18 @@ void* qGetResultRetrieveMsg(qTaskInfo_t qinfo) { ...@@ -251,18 +205,18 @@ void* qGetResultRetrieveMsg(qTaskInfo_t qinfo) {
} }
int32_t qKillTask(qTaskInfo_t qinfo) { int32_t qKillTask(qTaskInfo_t qinfo) {
SQInfo *pQInfo = (SQInfo *)qinfo; SExecTaskInfo *pTaskInfo = (SExecTaskInfo *)qinfo;
if (pQInfo == NULL || !isValidQInfo(pQInfo)) { if (pTaskInfo == NULL) {
return TSDB_CODE_QRY_INVALID_QHANDLE; return TSDB_CODE_QRY_INVALID_QHANDLE;
} }
qDebug("QInfo:0x%"PRIx64" query killed", pQInfo->qId); qDebug("%s execTask killed", GET_TASKID(pTaskInfo));
setQueryKilled(pQInfo); setTaskKilled(pTaskInfo);
// Wait for the query executing thread being stopped/ // Wait for the query executing thread being stopped/
// Once the query is stopped, the owner of qHandle will be cleared immediately. // Once the query is stopped, the owner of qHandle will be cleared immediately.
while (pQInfo->owner != 0) { while (pTaskInfo->owner != 0) {
taosMsleep(100); taosMsleep(100);
} }
...@@ -270,14 +224,14 @@ int32_t qKillTask(qTaskInfo_t qinfo) { ...@@ -270,14 +224,14 @@ int32_t qKillTask(qTaskInfo_t qinfo) {
} }
int32_t qAsyncKillTask(qTaskInfo_t qinfo) { int32_t qAsyncKillTask(qTaskInfo_t qinfo) {
SQInfo *pQInfo = (SQInfo *)qinfo; SExecTaskInfo *pTaskInfo = (SExecTaskInfo *)qinfo;
if (pQInfo == NULL || !isValidQInfo(pQInfo)) { if (pTaskInfo == NULL) {
return TSDB_CODE_QRY_INVALID_QHANDLE; return TSDB_CODE_QRY_INVALID_QHANDLE;
} }
qDebug("QInfo:0x%"PRIx64" query async killed", pQInfo->qId); qDebug("%s execTask async killed", GET_TASKID(pTaskInfo));
setQueryKilled(pQInfo); setTaskKilled(pTaskInfo);
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
...@@ -292,145 +246,12 @@ int32_t qIsTaskCompleted(qTaskInfo_t qinfo) { ...@@ -292,145 +246,12 @@ int32_t qIsTaskCompleted(qTaskInfo_t qinfo) {
return isTaskKilled(pTaskInfo) || Q_STATUS_EQUAL(pTaskInfo->status, TASK_OVER); return isTaskKilled(pTaskInfo) || Q_STATUS_EQUAL(pTaskInfo->status, TASK_OVER);
} }
void qDestroyTask(qTaskInfo_t qHandle) { void qDestroyTask(qTaskInfo_t qTaskHandle) {
SQInfo* pQInfo = (SQInfo*) qHandle; SExecTaskInfo* pTaskInfo = (SExecTaskInfo*) qTaskHandle;
if (!isValidQInfo(pQInfo)) { qDebug("%s execTask completed, numOfRows:%"PRId64, GET_TASKID(pTaskInfo), pTaskInfo->totalRows);
return;
}
qDebug("QInfo:0x%"PRIx64" query completed", pQInfo->qId);
queryCostStatis(pQInfo); // print the query cost summary
doDestroyTask(pQInfo);
}
void* qOpenTaskMgmt(int32_t vgId) {
const int32_t refreshHandleInterval = 30; // every 30 seconds, refresh handle pool
char cacheName[128] = {0};
sprintf(cacheName, "qhandle_%d", vgId);
STaskMgmt* pTaskMgmt = calloc(1, sizeof(STaskMgmt));
if (pTaskMgmt == NULL) {
terrno = TSDB_CODE_QRY_OUT_OF_MEMORY;
return NULL;
}
pTaskMgmt->qinfoPool = taosCacheInit(TSDB_CACHE_PTR_KEY, refreshHandleInterval, true, freeqinfoFn, cacheName);
pTaskMgmt->closed = false;
pTaskMgmt->vgId = vgId;
pthread_mutex_init(&pTaskMgmt->lock, NULL);
qDebug("vgId:%d, open queryTaskMgmt success", vgId);
return pTaskMgmt;
}
void qTaskMgmtNotifyClosing(void* pQMgmt) {
if (pQMgmt == NULL) {
return;
}
STaskMgmt* pQueryMgmt = pQMgmt;
qInfo("vgId:%d, set querymgmt closed, wait for all queries cancelled", pQueryMgmt->vgId);
pthread_mutex_lock(&pQueryMgmt->lock);
pQueryMgmt->closed = true;
pthread_mutex_unlock(&pQueryMgmt->lock);
taosCacheRefresh(pQueryMgmt->qinfoPool, taskMgmtKillTaskFn, NULL);
}
void qQueryMgmtReOpen(void *pQMgmt) {
if (pQMgmt == NULL) {
return;
}
STaskMgmt *pQueryMgmt = pQMgmt;
qInfo("vgId:%d, set querymgmt reopen", pQueryMgmt->vgId);
pthread_mutex_lock(&pQueryMgmt->lock);
pQueryMgmt->closed = false;
pthread_mutex_unlock(&pQueryMgmt->lock);
}
void qCleanupTaskMgmt(void* pQMgmt) {
if (pQMgmt == NULL) {
return;
}
STaskMgmt* pQueryMgmt = pQMgmt;
int32_t vgId = pQueryMgmt->vgId;
assert(pQueryMgmt->closed);
SCacheObj* pqinfoPool = pQueryMgmt->qinfoPool;
pQueryMgmt->qinfoPool = NULL;
taosCacheCleanup(pqinfoPool);
pthread_mutex_destroy(&pQueryMgmt->lock);
tfree(pQueryMgmt);
qDebug("vgId:%d, queryMgmt cleanup completed", vgId);
}
void** qRegisterTask(void* pMgmt, uint64_t qId, void *qInfo) {
if (pMgmt == NULL) {
terrno = TSDB_CODE_VND_INVALID_VGROUP_ID;
return NULL;
}
STaskMgmt *pQueryMgmt = pMgmt;
if (pQueryMgmt->qinfoPool == NULL) {
qError("QInfo:0x%"PRIx64"-%p failed to add qhandle into qMgmt, since qMgmt is closed", qId, (void*)qInfo);
terrno = TSDB_CODE_VND_INVALID_VGROUP_ID;
return NULL;
}
pthread_mutex_lock(&pQueryMgmt->lock); queryCostStatis(pTaskInfo); // print the query cost summary
if (pQueryMgmt->closed) { doDestroyTask(pTaskInfo);
pthread_mutex_unlock(&pQueryMgmt->lock);
qError("QInfo:0x%"PRIx64"-%p failed to add qhandle into cache, since qMgmt is colsing", qId, (void*)qInfo);
terrno = TSDB_CODE_VND_INVALID_VGROUP_ID;
return NULL;
} else {
void** handle = taosCachePut(pQueryMgmt->qinfoPool, &qId, sizeof(qId), &qInfo, sizeof(TSDB_CACHE_PTR_TYPE),
(getMaximumIdleDurationSec()*1000));
pthread_mutex_unlock(&pQueryMgmt->lock);
return handle;
}
}
void** qAcquireTask(void* pMgmt, uint64_t _key) {
STaskMgmt *pQueryMgmt = pMgmt;
if (pQueryMgmt->closed) {
terrno = TSDB_CODE_VND_INVALID_VGROUP_ID;
return NULL;
}
if (pQueryMgmt->qinfoPool == NULL) {
terrno = TSDB_CODE_QRY_INVALID_QHANDLE;
return NULL;
}
void** handle = taosCacheAcquireByKey(pQueryMgmt->qinfoPool, &_key, sizeof(_key));
if (handle == NULL || *handle == NULL) {
terrno = TSDB_CODE_QRY_INVALID_QHANDLE;
return NULL;
} else {
return handle;
}
}
void** qReleaseTask(void* pMgmt, void* pQInfo, bool freeHandle) {
STaskMgmt *pQueryMgmt = pMgmt;
if (pQueryMgmt->qinfoPool == NULL) {
return NULL;
}
taosCacheRelease(pQueryMgmt->qinfoPool, pQInfo, freeHandle);
return 0;
} }
#if 0 #if 0
...@@ -444,8 +265,8 @@ int32_t qKillQueryByQId(void* pMgmt, int64_t qId, int32_t waitMs, int32_t waitCo ...@@ -444,8 +265,8 @@ int32_t qKillQueryByQId(void* pMgmt, int64_t qId, int32_t waitMs, int32_t waitCo
if (pQInfo == NULL || !isValidQInfo(pQInfo)) { if (pQInfo == NULL || !isValidQInfo(pQInfo)) {
return TSDB_CODE_QRY_INVALID_QHANDLE; return TSDB_CODE_QRY_INVALID_QHANDLE;
} }
qWarn("QId:0x%"PRIx64" be killed(no memory commit).", pQInfo->qId); qWarn("%s be killed(no memory commit).", pQInfo->qId);
setQueryKilled(pQInfo); setTaskKilled(pQInfo);
// wait query stop // wait query stop
int32_t loop = 0; int32_t loop = 0;
...@@ -461,4 +282,4 @@ int32_t qKillQueryByQId(void* pMgmt, int64_t qId, int32_t waitMs, int32_t waitCo ...@@ -461,4 +282,4 @@ int32_t qKillQueryByQId(void* pMgmt, int64_t qId, int32_t waitMs, int32_t waitCo
return error; return error;
} }
#endif #endif
\ No newline at end of file
...@@ -219,7 +219,7 @@ TEST(testCase, build_executor_tree_Test) { ...@@ -219,7 +219,7 @@ TEST(testCase, build_executor_tree_Test) {
SExecTaskInfo* pTaskInfo = nullptr; SExecTaskInfo* pTaskInfo = nullptr;
DataSinkHandle sinkHandle = nullptr; DataSinkHandle sinkHandle = nullptr;
int32_t code = qCreateExecTask((void*) 1, 2, NULL, (void**) &pTaskInfo, &sinkHandle); int32_t code = qCreateExecTask((void*) 1, 2, 1, NULL, (void**) &pTaskInfo, &sinkHandle);
} }
#pragma GCC diagnostic pop #pragma GCC diagnostic pop
\ No newline at end of file
...@@ -413,7 +413,6 @@ static void doDestroyQueryNode(SQueryPlanNode* pQueryNode) { ...@@ -413,7 +413,6 @@ static void doDestroyQueryNode(SQueryPlanNode* pQueryNode) {
tfree(pQueryTableInfo->tableName); tfree(pQueryTableInfo->tableName);
} }
printf("----------->Free:%p\n", pQueryNode->pExpr);
taosArrayDestroy(pQueryNode->pExpr); taosArrayDestroy(pQueryNode->pExpr);
tfree(pQueryNode->pExtInfo); tfree(pQueryNode->pExtInfo);
......
...@@ -1125,8 +1125,8 @@ int32_t subPlanToString(const SSubplan* subplan, char** str, int32_t* len) { ...@@ -1125,8 +1125,8 @@ int32_t subPlanToString(const SSubplan* subplan, char** str, int32_t* len) {
*str = cJSON_Print(json); *str = cJSON_Print(json);
cJSON_Delete(json); cJSON_Delete(json);
printf("====Physical plan:====\n"); // printf("====Physical plan:====\n");
printf("%s\n", *str); // printf("%s\n", *str);
*len = strlen(*str) + 1; *len = strlen(*str) + 1;
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
......
...@@ -173,17 +173,17 @@ typedef struct SQWorkerMgmt { ...@@ -173,17 +173,17 @@ typedef struct SQWorkerMgmt {
#define QW_SCH_ELOG(param, ...) qError("QW:%p SID:%"PRIx64" " param, mgmt, sId, __VA_ARGS__) #define QW_SCH_ELOG(param, ...) qError("QW:%p SID:%"PRIx64" " param, mgmt, sId, __VA_ARGS__)
#define QW_SCH_DLOG(param, ...) qDebug("QW:%p SID:%"PRIx64" " param, mgmt, sId, __VA_ARGS__) #define QW_SCH_DLOG(param, ...) qDebug("QW:%p SID:%"PRIx64" " param, mgmt, sId, __VA_ARGS__)
#define QW_TASK_ELOG(param, ...) qError("QW:%p QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, qId, tId, __VA_ARGS__) #define QW_TASK_ELOG(param, ...) qError("QW:%p QID:0x%"PRIx64",TID:0x%"PRIx64" " param, mgmt, qId, tId, __VA_ARGS__)
#define QW_TASK_WLOG(param, ...) qWarn("QW:%p QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, qId, tId, __VA_ARGS__) #define QW_TASK_WLOG(param, ...) qWarn("QW:%p QID:0x%"PRIx64",TID:0x%"PRIx64" " param, mgmt, qId, tId, __VA_ARGS__)
#define QW_TASK_DLOG(param, ...) qDebug("QW:%p QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, qId, tId, __VA_ARGS__) #define QW_TASK_DLOG(param, ...) qDebug("QW:%p QID:0x%"PRIx64",TID:0x%"PRIx64" " param, mgmt, qId, tId, __VA_ARGS__)
#define QW_TASK_ELOG_E(param) qError("QW:%p QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, qId, tId) #define QW_TASK_ELOG_E(param) qError("QW:%p QID:0x%"PRIx64",TID:0x%"PRIx64" " param, mgmt, qId, tId)
#define QW_TASK_WLOG_E(param) qWarn("QW:%p QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, qId, tId) #define QW_TASK_WLOG_E(param) qWarn("QW:%p QID:0x%"PRIx64",TID:0x%"PRIx64" " param, mgmt, qId, tId)
#define QW_TASK_DLOG_E(param) qDebug("QW:%p QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, qId, tId) #define QW_TASK_DLOG_E(param) qDebug("QW:%p QID:0x%"PRIx64",TID:0x%"PRIx64" " param, mgmt, qId, tId)
#define QW_SCH_TASK_ELOG(param, ...) qError("QW:%p SID:%"PRIx64",QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, sId, qId, tId, __VA_ARGS__) #define QW_SCH_TASK_ELOG(param, ...) qError("QW:%p SID:%"PRIx64",QID:0x%"PRIx64",TID:0x%"PRIx64" " param, mgmt, sId, qId, tId, __VA_ARGS__)
#define QW_SCH_TASK_WLOG(param, ...) qWarn("QW:%p SID:%"PRIx64",QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, sId, qId, tId, __VA_ARGS__) #define QW_SCH_TASK_WLOG(param, ...) qWarn("QW:%p SID:%"PRIx64",QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, sId, qId, tId, __VA_ARGS__)
#define QW_SCH_TASK_DLOG(param, ...) qDebug("QW:%p SID:%"PRIx64",QID:0x%"PRIx64",TID:%"PRIx64" " param, mgmt, sId, qId, tId, __VA_ARGS__) #define QW_SCH_TASK_DLOG(param, ...) qDebug("QW:%p SID:%"PRIx64",QID:0x%"PRIx64",TID:0x%"PRIx64" " param, mgmt, sId, qId, tId, __VA_ARGS__)
#define QW_LOCK_DEBUG(...) do { if (gQWDebug.lockDebug) { qDebug(__VA_ARGS__); } } while (0) #define QW_LOCK_DEBUG(...) do { if (gQWDebug.lockDebug) { qDebug(__VA_ARGS__); } } while (0)
......
...@@ -32,7 +32,7 @@ int32_t qwProcessDrop(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t ...@@ -32,7 +32,7 @@ int32_t qwProcessDrop(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t
int32_t qwBuildAndSendDropRsp(void *connection, int32_t code); int32_t qwBuildAndSendDropRsp(void *connection, int32_t code);
int32_t qwBuildAndSendCancelRsp(SRpcMsg *pMsg, int32_t code); int32_t qwBuildAndSendCancelRsp(SRpcMsg *pMsg, int32_t code);
int32_t qwBuildAndSendFetchRsp(void *connection, SRetrieveTableRsp *pRsp, int32_t dataLength, int32_t code); int32_t qwBuildAndSendFetchRsp(void *connection, SRetrieveTableRsp *pRsp, int32_t dataLength, int32_t code);
void qwBuildFetchRsp(void *msg, SOutputData *input, int32_t len); void qwBuildFetchRsp(void *msg, SOutputData *input, int32_t len, bool qComplete);
int32_t qwBuildAndSendCQueryMsg(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, void *connection); int32_t qwBuildAndSendCQueryMsg(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, void *connection);
int32_t qwBuildAndSendSchSinkMsg(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, void *connection); int32_t qwBuildAndSendSchSinkMsg(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, void *connection);
int32_t qwBuildAndSendReadyRsp(void *connection, int32_t code); int32_t qwBuildAndSendReadyRsp(void *connection, int32_t code);
......
...@@ -456,7 +456,7 @@ _return: ...@@ -456,7 +456,7 @@ _return:
QW_RET(code); QW_RET(code);
} }
int32_t qwExecTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) { int32_t qwExecTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx, bool *queryEnd) {
int32_t code = 0; int32_t code = 0;
bool qcontinue = true; bool qcontinue = true;
SSDataBlock* pRes = NULL; SSDataBlock* pRes = NULL;
...@@ -467,11 +467,11 @@ int32_t qwExecTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) { ...@@ -467,11 +467,11 @@ int32_t qwExecTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) {
DataSinkHandle sinkHandle = ctx->sinkHandle; DataSinkHandle sinkHandle = ctx->sinkHandle;
while (true) { while (true) {
QW_TASK_DLOG("start to execTask in executor, loopIdx:%d", i++); QW_TASK_DLOG("start to execTask, loopIdx:%d", i++);
code = qExecTask(*taskHandle, &pRes, &useconds); code = qExecTask(*taskHandle, &pRes, &useconds);
if (code) { if (code) {
QW_TASK_ELOG("qExecTask failed, code:%x", code); QW_TASK_ELOG("qExecTask failed, code:%s", tstrerror(code));
QW_ERR_JRET(code); QW_ERR_JRET(code);
} }
...@@ -485,6 +485,10 @@ int32_t qwExecTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) { ...@@ -485,6 +485,10 @@ int32_t qwExecTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) {
if (TASK_TYPE_TEMP == ctx->taskType) { if (TASK_TYPE_TEMP == ctx->taskType) {
qwFreeTaskHandle(QW_FPARAMS(), taskHandle); qwFreeTaskHandle(QW_FPARAMS(), taskHandle);
} }
if (queryEnd) {
*queryEnd = true;
}
break; break;
} }
...@@ -492,7 +496,7 @@ int32_t qwExecTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) { ...@@ -492,7 +496,7 @@ int32_t qwExecTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) {
SInputData inputData = {.pData = pRes, .pTableRetrieveTsMap = NULL}; SInputData inputData = {.pData = pRes, .pTableRetrieveTsMap = NULL};
code = dsPutDataBlock(sinkHandle, &inputData, &qcontinue); code = dsPutDataBlock(sinkHandle, &inputData, &qcontinue);
if (code) { if (code) {
QW_TASK_ELOG("dsPutDataBlock failed, code:%x", code); QW_TASK_ELOG("dsPutDataBlock failed, code:%s", tstrerror(code));
QW_ERR_JRET(code); QW_ERR_JRET(code);
} }
...@@ -587,12 +591,7 @@ int32_t qwGetResFromSink(QW_FPARAMS_DEF, SQWTaskCtx *ctx, int32_t *dataLen, void ...@@ -587,12 +591,7 @@ int32_t qwGetResFromSink(QW_FPARAMS_DEF, SQWTaskCtx *ctx, int32_t *dataLen, void
QW_ERR_RET(code); QW_ERR_RET(code);
} }
queryEnd = pOutput->queryEnd; if (DS_BUF_EMPTY == pOutput->bufStatus && pOutput->queryEnd) {
pOutput->queryEnd = false;
if (DS_BUF_EMPTY == pOutput->bufStatus && queryEnd) {
pOutput->queryEnd = true;
QW_SCH_TASK_DLOG("task all fetched, status:%d", JOB_TASK_STATUS_SUCCEED); QW_SCH_TASK_DLOG("task all fetched, status:%d", JOB_TASK_STATUS_SUCCEED);
QW_ERR_RET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_SUCCEED)); QW_ERR_RET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_SUCCEED));
} }
...@@ -974,7 +973,7 @@ int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, int8_t taskType) { ...@@ -974,7 +973,7 @@ int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, int8_t taskType) {
QW_ERR_JRET(code); QW_ERR_JRET(code);
} }
code = qCreateExecTask(qwMsg->node, 0, (struct SSubplan *)plan, &pTaskInfo, &sinkHandle); code = qCreateExecTask(qwMsg->node, 0, tId, (struct SSubplan *)plan, &pTaskInfo, &sinkHandle);
if (code) { if (code) {
QW_TASK_ELOG("qCreateExecTask failed, code:%s", tstrerror(code)); QW_TASK_ELOG("qCreateExecTask failed, code:%s", tstrerror(code));
QW_ERR_JRET(code); QW_ERR_JRET(code);
...@@ -996,7 +995,7 @@ int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, int8_t taskType) { ...@@ -996,7 +995,7 @@ int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, int8_t taskType) {
atomic_store_ptr(&ctx->sinkHandle, sinkHandle); atomic_store_ptr(&ctx->sinkHandle, sinkHandle);
if (pTaskInfo && sinkHandle) { if (pTaskInfo && sinkHandle) {
QW_ERR_JRET(qwExecTask(QW_FPARAMS(), ctx)); QW_ERR_JRET(qwExecTask(QW_FPARAMS(), ctx, NULL));
} }
_return: _return:
...@@ -1083,6 +1082,7 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { ...@@ -1083,6 +1082,7 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) {
SQWPhaseOutput output = {0}; SQWPhaseOutput output = {0};
void *rsp = NULL; void *rsp = NULL;
int32_t dataLen = 0; int32_t dataLen = 0;
bool queryEnd = false;
do { do {
QW_ERR_JRET(qwHandlePrePhaseEvents(QW_FPARAMS(), QW_PHASE_PRE_CQUERY, &input, &output)); QW_ERR_JRET(qwHandlePrePhaseEvents(QW_FPARAMS(), QW_PHASE_PRE_CQUERY, &input, &output));
...@@ -1102,7 +1102,7 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { ...@@ -1102,7 +1102,7 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) {
DataSinkHandle sinkHandle = ctx->sinkHandle; DataSinkHandle sinkHandle = ctx->sinkHandle;
QW_ERR_JRET(qwExecTask(QW_FPARAMS(), ctx)); QW_ERR_JRET(qwExecTask(QW_FPARAMS(), ctx, &queryEnd));
if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH)) { if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH)) {
SOutputData sOutput = {0}; SOutputData sOutput = {0};
...@@ -1114,13 +1114,10 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { ...@@ -1114,13 +1114,10 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) {
// RC WARNING // RC WARNING
atomic_store_8(&ctx->queryContinue, 1); atomic_store_8(&ctx->queryContinue, 1);
} }
if (sOutput.queryEnd) {
needStop = true;
}
if (rsp) { if (rsp) {
qwBuildFetchRsp(rsp, &sOutput, dataLen); bool qComplete = (DS_BUF_EMPTY == sOutput.bufStatus && sOutput.queryEnd);
qwBuildFetchRsp(rsp, &sOutput, dataLen, qComplete);
QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_FETCH); QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_FETCH);
...@@ -1131,6 +1128,10 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { ...@@ -1131,6 +1128,10 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) {
} }
} }
if (queryEnd) {
needStop = true;
}
_return: _return:
if (NULL == ctx) { if (NULL == ctx) {
...@@ -1196,7 +1197,8 @@ int32_t qwProcessFetch(QW_FPARAMS_DEF, SQWMsg *qwMsg) { ...@@ -1196,7 +1197,8 @@ int32_t qwProcessFetch(QW_FPARAMS_DEF, SQWMsg *qwMsg) {
if (NULL == rsp) { if (NULL == rsp) {
QW_SET_EVENT_RECEIVED(ctx, QW_EVENT_FETCH); QW_SET_EVENT_RECEIVED(ctx, QW_EVENT_FETCH);
} else { } else {
qwBuildFetchRsp(rsp, &sOutput, dataLen); bool qComplete = (DS_BUF_EMPTY == sOutput.bufStatus && sOutput.queryEnd);
qwBuildFetchRsp(rsp, &sOutput, dataLen, qComplete);
} }
if ((!sOutput.queryEnd) && (DS_BUF_LOW == sOutput.bufStatus || DS_BUF_EMPTY == sOutput.bufStatus)) { if ((!sOutput.queryEnd) && (DS_BUF_LOW == sOutput.bufStatus || DS_BUF_EMPTY == sOutput.bufStatus)) {
......
...@@ -26,11 +26,11 @@ int32_t qwMallocFetchRsp(int32_t length, SRetrieveTableRsp **rsp) { ...@@ -26,11 +26,11 @@ int32_t qwMallocFetchRsp(int32_t length, SRetrieveTableRsp **rsp) {
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
void qwBuildFetchRsp(void *msg, SOutputData *input, int32_t len) { void qwBuildFetchRsp(void *msg, SOutputData *input, int32_t len, bool qComplete) {
SRetrieveTableRsp *rsp = (SRetrieveTableRsp *)msg; SRetrieveTableRsp *rsp = (SRetrieveTableRsp *)msg;
rsp->useconds = htobe64(input->useconds); rsp->useconds = htobe64(input->useconds);
rsp->completed = input->queryEnd; rsp->completed = qComplete;
rsp->precision = input->precision; rsp->precision = input->precision;
rsp->compressed = input->compressed; rsp->compressed = input->compressed;
rsp->compLen = htonl(len); rsp->compLen = htonl(len);
...@@ -258,12 +258,12 @@ int32_t qwBuildAndSendCQueryMsg(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, ...@@ -258,12 +258,12 @@ int32_t qwBuildAndSendCQueryMsg(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId,
int32_t code = (*mgmt->putToQueueFp)(mgmt->nodeObj, &pNewMsg); int32_t code = (*mgmt->putToQueueFp)(mgmt->nodeObj, &pNewMsg);
if (TSDB_CODE_SUCCESS != code) { if (TSDB_CODE_SUCCESS != code) {
QW_SCH_TASK_ELOG("put query continue msg to queue failed, code:%x", code); QW_SCH_TASK_ELOG("put query continue msg to queue failed, vgId:%d, code:%s", mgmt->nodeId, tstrerror(code));
rpcFreeCont(req); rpcFreeCont(req);
QW_ERR_RET(code); QW_ERR_RET(code);
} }
QW_SCH_TASK_DLOG("put query continue msg to query queue, vgId:%d", mgmt->nodeId); QW_SCH_TASK_DLOG("put task continue exec msg to query queue, vgId:%d", mgmt->nodeId);
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
......
...@@ -1482,13 +1482,14 @@ int32_t schedulerConvertDagToTaskList(SQueryDag* pDag, SArray **pTasks) { ...@@ -1482,13 +1482,14 @@ int32_t schedulerConvertDagToTaskList(SQueryDag* pDag, SArray **pTasks) {
} }
int32_t msgSize = sizeof(SSubQueryMsg) + msgLen; int32_t msgSize = sizeof(SSubQueryMsg) + msgLen;
msg = calloc(1, msgSize);
if (NULL == msg) { if (NULL == msg) {
qError("calloc %d failed", msgSize); qError("calloc %d failed", msgSize);
SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY); SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
} }
SSubQueryMsg *pMsg = (SSubQueryMsg*) msg; SSubQueryMsg* pMsg = calloc(1, msgSize);
/*SSubQueryMsg *pMsg = (SSubQueryMsg*) msg;*/
memcpy(pMsg->msg, msg, msgLen);
pMsg->header.vgId = tInfo.addr.nodeId; pMsg->header.vgId = tInfo.addr.nodeId;
...@@ -1497,7 +1498,7 @@ int32_t schedulerConvertDagToTaskList(SQueryDag* pDag, SArray **pTasks) { ...@@ -1497,7 +1498,7 @@ int32_t schedulerConvertDagToTaskList(SQueryDag* pDag, SArray **pTasks) {
pMsg->taskId = schGenUUID(); pMsg->taskId = schGenUUID();
pMsg->taskType = TASK_TYPE_PERSISTENT; pMsg->taskType = TASK_TYPE_PERSISTENT;
pMsg->contentLen = msgLen; pMsg->contentLen = msgLen;
memcpy(pMsg->msg, msg, msgLen); /*memcpy(pMsg->msg, ((SSubQueryMsg*)msg)->msg, msgLen);*/
tInfo.msg = pMsg; tInfo.msg = pMsg;
......
...@@ -242,11 +242,14 @@ void *rpcOpen(const SRpcInit *pInit) { ...@@ -242,11 +242,14 @@ void *rpcOpen(const SRpcInit *pInit) {
pRpc = (SRpcInfo *)calloc(1, sizeof(SRpcInfo)); pRpc = (SRpcInfo *)calloc(1, sizeof(SRpcInfo));
if (pRpc == NULL) return NULL; if (pRpc == NULL) return NULL;
if (pInit->label) tstrncpy(pRpc->label, pInit->label, strlen(pInit->label)); if (pInit->label) tstrncpy(pRpc->label, pInit->label, tListLen(pInit->label));
pRpc->connType = pInit->connType; pRpc->connType = pInit->connType;
if (pRpc->connType == TAOS_CONN_CLIENT) { if (pRpc->connType == TAOS_CONN_CLIENT) {
pRpc->numOfThreads = pInit->numOfThreads; pRpc->numOfThreads = pInit->numOfThreads;
if (pRpc->numOfThreads >= 10) {
pRpc->numOfThreads = 10;
}
} else { } else {
pRpc->numOfThreads = pInit->numOfThreads > TSDB_MAX_RPC_THREADS ? TSDB_MAX_RPC_THREADS : pInit->numOfThreads; pRpc->numOfThreads = pInit->numOfThreads > TSDB_MAX_RPC_THREADS ? TSDB_MAX_RPC_THREADS : pInit->numOfThreads;
} }
...@@ -769,8 +772,8 @@ static SRpcConn *rpcAllocateServerConn(SRpcInfo *pRpc, SRecvInfo *pRecv) { ...@@ -769,8 +772,8 @@ static SRpcConn *rpcAllocateServerConn(SRpcInfo *pRpc, SRecvInfo *pRecv) {
} }
taosHashPut(pRpc->hash, hashstr, size, (char *)&pConn, POINTER_BYTES); taosHashPut(pRpc->hash, hashstr, size, (char *)&pConn, POINTER_BYTES);
tDebug("%s %p server connection is allocated, uid:0x%x sid:%d key:%s spi:%d", pRpc->label, pConn, pConn->linkUid, sid, tDebug("%s %p server connection is allocated, uid:0x%x sid:%d key:%s spi:%d", pRpc->label, pConn, pConn->linkUid,
hashstr, pConn->spi); sid, hashstr, pConn->spi);
} }
return pConn; return pConn;
...@@ -1357,9 +1360,14 @@ static void rpcSendMsgToPeer(SRpcConn *pConn, void *msg, int msgLen) { ...@@ -1357,9 +1360,14 @@ static void rpcSendMsgToPeer(SRpcConn *pConn, void *msg, int msgLen) {
tDebug("%s, %s is sent to %s:%hu, len:%d sig:0x%08x:0x%08x:%d", pConn->info, TMSG_INFO(pHead->msgType), tDebug("%s, %s is sent to %s:%hu, len:%d sig:0x%08x:0x%08x:%d", pConn->info, TMSG_INFO(pHead->msgType),
pConn->peerFqdn, pConn->peerPort, msgLen, pHead->sourceId, pHead->destId, pHead->tranId); pConn->peerFqdn, pConn->peerPort, msgLen, pHead->sourceId, pHead->destId, pHead->tranId);
} else { } else {
if (pHead->code == 0) pConn->secured = 1; // for success response, set link as secured if (pHead->code == 0) {
tDebug("%s, %s is sent to 0x%x:%hu, code:0x%x len:%d sig:0x%08x:0x%08x:%d", pConn->info, TMSG_INFO(pHead->msgType), pConn->secured = 1; // for success response, set link as secured
pConn->peerIp, pConn->peerPort, htonl(pHead->code), msgLen, pHead->sourceId, pHead->destId, pHead->tranId); }
char ipport[40] = {0};
taosIpPort2String(pConn->peerIp, pConn->peerPort, ipport);
tDebug("%s, %s is sent to %s, code:0x%x len:%d sig:0x%08x:0x%08x:%d", pConn->info, TMSG_INFO(pHead->msgType),
ipport, htonl(pHead->code), msgLen, pHead->sourceId, pHead->destId, pHead->tranId);
} }
// tTrace("connection type is: %d", pConn->connType); // tTrace("connection type is: %d", pConn->connType);
......
...@@ -105,24 +105,23 @@ static void* clientThread(void* arg); ...@@ -105,24 +105,23 @@ static void* clientThread(void* arg);
static void clientHandleResp(SCliConn* conn) { static void clientHandleResp(SCliConn* conn) {
STransConnCtx* pCtx = ((SCliMsg*)conn->data)->ctx; STransConnCtx* pCtx = ((SCliMsg*)conn->data)->ctx;
SRpcInfo* pRpc = pCtx->pTransInst; SRpcInfo* pRpc = pCtx->pTransInst;
SRpcMsg rpcMsg;
STransMsgHead* pHead = (STransMsgHead*)(conn->readBuf.buf); STransMsgHead* pHead = (STransMsgHead*)(conn->readBuf.buf);
pHead->code = htonl(pHead->code); pHead->code = htonl(pHead->code);
pHead->msgLen = htonl(pHead->msgLen); pHead->msgLen = htonl(pHead->msgLen);
SRpcMsg rpcMsg;
rpcMsg.contLen = transContLenFromMsg(pHead->msgLen); rpcMsg.contLen = transContLenFromMsg(pHead->msgLen);
rpcMsg.pCont = transContFromHead(pHead); rpcMsg.pCont = transContFromHead(pHead);
rpcMsg.code = pHead->code; rpcMsg.code = pHead->code;
rpcMsg.msgType = pHead->msgType; rpcMsg.msgType = pHead->msgType;
// rpcMsg.pCont = conn->readBuf.buf;
// rpcMsg.contLen = conn->readBuf.len;
rpcMsg.ahandle = pCtx->ahandle; rpcMsg.ahandle = pCtx->ahandle;
(pRpc->cfp)(NULL, &rpcMsg, NULL); (pRpc->cfp)(NULL, &rpcMsg, NULL);
conn->notifyCount += 1; conn->notifyCount += 1;
SCliThrdObj* pThrd = conn->hostThrd; SCliThrdObj* pThrd = conn->hostThrd;
tfree(conn->data);
addConnToPool(pThrd->pool, pCtx->ip, pCtx->port, conn); addConnToPool(pThrd->pool, pCtx->ip, pCtx->port, conn);
// start thread's timer of conn pool if not active // start thread's timer of conn pool if not active
...@@ -145,7 +144,7 @@ static void clientHandleExcept(SCliConn* pConn) { ...@@ -145,7 +144,7 @@ static void clientHandleExcept(SCliConn* pConn) {
rpcMsg.code = -1; rpcMsg.code = -1;
// SRpcInfo* pRpc = pMsg->ctx->pRpc; // SRpcInfo* pRpc = pMsg->ctx->pRpc;
(pRpc->cfp)(NULL, &rpcMsg, NULL); (pRpc->cfp)(NULL, &rpcMsg, NULL);
tfree(pConn->data);
pConn->notifyCount += 1; pConn->notifyCount += 1;
destroyTransConnCtx(pCtx); destroyTransConnCtx(pCtx);
clientConnDestroy(pConn, true); clientConnDestroy(pConn, true);
......
...@@ -13,35 +13,36 @@ ...@@ -13,35 +13,36 @@
* along with this program. If not, see <http://www.gnu.org/licenses/>. * along with this program. If not, see <http://www.gnu.org/licenses/>.
*/ */
#include "os.h" #define _DEFAULT_SOURCE
#include "taoserror.h"
#include "tqueue.h" #include "tqueue.h"
#include "taoserror.h"
#include "ulog.h" #include "ulog.h"
typedef struct STaosQnode STaosQnode; typedef struct STaosQnode STaosQnode;
typedef struct STaosQnode { typedef struct STaosQnode {
STaosQnode *next; STaosQnode *next;
STaosQueue *queue;
char item[]; char item[];
} STaosQnode; } STaosQnode;
typedef struct STaosQueue { typedef struct STaosQueue {
int32_t itemSize; int32_t itemSize;
int32_t numOfItems; int32_t numOfItems;
STaosQnode *head; int32_t threadId;
STaosQnode *tail; STaosQnode * head;
STaosQueue *next; // for queue set STaosQnode * tail;
STaosQset *qset; // for queue set STaosQueue * next; // for queue set
void *ahandle; // for queue set STaosQset * qset; // for queue set
FProcessItem itemFp; void * ahandle; // for queue set
FProcessItems itemsFp; FItem itemFp;
FItems itemsFp;
pthread_mutex_t mutex; pthread_mutex_t mutex;
} STaosQueue; } STaosQueue;
typedef struct STaosQset { typedef struct STaosQset {
STaosQueue *head; STaosQueue * head;
STaosQueue *current; STaosQueue * current;
pthread_mutex_t mutex; pthread_mutex_t mutex;
int32_t numOfQueues; int32_t numOfQueues;
int32_t numOfItems; int32_t numOfItems;
...@@ -56,19 +57,23 @@ typedef struct STaosQall { ...@@ -56,19 +57,23 @@ typedef struct STaosQall {
} STaosQall; } STaosQall;
STaosQueue *taosOpenQueue() { STaosQueue *taosOpenQueue() {
STaosQueue *queue = calloc(sizeof(STaosQueue), 1); STaosQueue *queue = calloc(1, sizeof(STaosQueue));
if (queue == NULL) { if (queue == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL; return NULL;
} }
pthread_mutex_init(&queue->mutex, NULL); if (pthread_mutex_init(&queue->mutex, NULL) != 0) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}
uTrace("queue:%p is opened", queue); queue->threadId = -1;
uDebug("queue:%p is opened", queue);
return queue; return queue;
} }
void taosSetQueueFp(STaosQueue *queue, FProcessItem itemFp, FProcessItems itemsFp) { void taosSetQueueFp(STaosQueue *queue, FItem itemFp, FItems itemsFp) {
if (queue == NULL) return; if (queue == NULL) return;
queue->itemFp = itemFp; queue->itemFp = itemFp;
queue->itemsFp = itemsFp; queue->itemsFp = itemsFp;
...@@ -77,7 +82,7 @@ void taosSetQueueFp(STaosQueue *queue, FProcessItem itemFp, FProcessItems itemsF ...@@ -77,7 +82,7 @@ void taosSetQueueFp(STaosQueue *queue, FProcessItem itemFp, FProcessItems itemsF
void taosCloseQueue(STaosQueue *queue) { void taosCloseQueue(STaosQueue *queue) {
if (queue == NULL) return; if (queue == NULL) return;
STaosQnode *pTemp; STaosQnode *pTemp;
STaosQset *qset; STaosQset * qset;
pthread_mutex_lock(&queue->mutex); pthread_mutex_lock(&queue->mutex);
STaosQnode *pNode = queue->head; STaosQnode *pNode = queue->head;
...@@ -85,7 +90,9 @@ void taosCloseQueue(STaosQueue *queue) { ...@@ -85,7 +90,9 @@ void taosCloseQueue(STaosQueue *queue) {
qset = queue->qset; qset = queue->qset;
pthread_mutex_unlock(&queue->mutex); pthread_mutex_unlock(&queue->mutex);
if (queue->qset) taosRemoveFromQset(qset, queue); if (queue->qset) {
taosRemoveFromQset(qset, queue);
}
while (pNode) { while (pNode) {
pTemp = pNode; pTemp = pNode;
...@@ -96,7 +103,7 @@ void taosCloseQueue(STaosQueue *queue) { ...@@ -96,7 +103,7 @@ void taosCloseQueue(STaosQueue *queue) {
pthread_mutex_destroy(&queue->mutex); pthread_mutex_destroy(&queue->mutex);
free(queue); free(queue);
uTrace("queue:%p is closed", queue); uDebug("queue:%p is closed", queue);
} }
bool taosQueueEmpty(STaosQueue *queue) { bool taosQueueEmpty(STaosQueue *queue) {
...@@ -120,19 +127,23 @@ int32_t taosQueueSize(STaosQueue *queue) { ...@@ -120,19 +127,23 @@ int32_t taosQueueSize(STaosQueue *queue) {
} }
void *taosAllocateQitem(int32_t size) { void *taosAllocateQitem(int32_t size) {
STaosQnode *pNode = (STaosQnode *)calloc(sizeof(STaosQnode) + size, 1); STaosQnode *pNode = calloc(1, sizeof(STaosQnode) + size);
if (pNode == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}
if (pNode == NULL) return NULL;
uTrace("item:%p, node:%p is allocated", pNode->item, pNode); uTrace("item:%p, node:%p is allocated", pNode->item, pNode);
return (void *)pNode->item; return (void *)pNode->item;
} }
void taosFreeQitem(void *param) { void taosFreeQitem(void *pItem) {
if (param == NULL) return; if (pItem == NULL) return;
char *temp = (char *)param; char *temp = pItem;
temp -= sizeof(STaosQnode); temp -= sizeof(STaosQnode);
uTrace("item:%p, node:%p is freed", param, temp); uTrace("item:%p, node:%p is freed", pItem, temp);
free(temp); free(temp);
} }
...@@ -175,7 +186,7 @@ int32_t taosReadQitem(STaosQueue *queue, void **ppItem) { ...@@ -175,7 +186,7 @@ int32_t taosReadQitem(STaosQueue *queue, void **ppItem) {
queue->numOfItems--; queue->numOfItems--;
if (queue->qset) atomic_sub_fetch_32(&queue->qset->numOfItems, 1); if (queue->qset) atomic_sub_fetch_32(&queue->qset->numOfItems, 1);
code = 1; code = 1;
uDebug("item:%p is read out from queue:%p, items:%d", *ppItem, queue, queue->numOfItems); uTrace("item:%p is read out from queue:%p, items:%d", *ppItem, queue, queue->numOfItems);
} }
pthread_mutex_unlock(&queue->mutex); pthread_mutex_unlock(&queue->mutex);
...@@ -183,7 +194,7 @@ int32_t taosReadQitem(STaosQueue *queue, void **ppItem) { ...@@ -183,7 +194,7 @@ int32_t taosReadQitem(STaosQueue *queue, void **ppItem) {
return code; return code;
} }
STaosQall *taosAllocateQall() { return calloc(sizeof(STaosQall), 1); } STaosQall *taosAllocateQall() { return calloc(1, sizeof(STaosQall)); }
void taosFreeQall(STaosQall *qall) { free(qall); } void taosFreeQall(STaosQall *qall) { free(qall); }
...@@ -238,7 +249,7 @@ int32_t taosGetQitem(STaosQall *qall, void **ppItem) { ...@@ -238,7 +249,7 @@ int32_t taosGetQitem(STaosQall *qall, void **ppItem) {
void taosResetQitems(STaosQall *qall) { qall->current = qall->start; } void taosResetQitems(STaosQall *qall) { qall->current = qall->start; }
STaosQset *taosOpenQset() { STaosQset *taosOpenQset() {
STaosQset *qset = (STaosQset *)calloc(sizeof(STaosQset), 1); STaosQset *qset = calloc(sizeof(STaosQset), 1);
if (qset == NULL) { if (qset == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY; terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL; return NULL;
...@@ -247,7 +258,7 @@ STaosQset *taosOpenQset() { ...@@ -247,7 +258,7 @@ STaosQset *taosOpenQset() {
pthread_mutex_init(&qset->mutex, NULL); pthread_mutex_init(&qset->mutex, NULL);
tsem_init(&qset->sem, 0, 0); tsem_init(&qset->sem, 0, 0);
uTrace("qset:%p is opened", qset); uDebug("qset:%p is opened", qset);
return qset; return qset;
} }
...@@ -268,7 +279,7 @@ void taosCloseQset(STaosQset *qset) { ...@@ -268,7 +279,7 @@ void taosCloseQset(STaosQset *qset) {
pthread_mutex_destroy(&qset->mutex); pthread_mutex_destroy(&qset->mutex);
tsem_destroy(&qset->sem); tsem_destroy(&qset->sem);
free(qset); free(qset);
uTrace("qset:%p is closed", qset); uDebug("qset:%p is closed", qset);
} }
// tsem_post 'qset->sem', so that reader threads waiting for it // tsem_post 'qset->sem', so that reader threads waiting for it
...@@ -338,12 +349,12 @@ void taosRemoveFromQset(STaosQset *qset, STaosQueue *queue) { ...@@ -338,12 +349,12 @@ void taosRemoveFromQset(STaosQset *qset, STaosQueue *queue) {
pthread_mutex_unlock(&qset->mutex); pthread_mutex_unlock(&qset->mutex);
uTrace("queue:%p is removed from qset:%p", queue, qset); uDebug("queue:%p is removed from qset:%p", queue, qset);
} }
int32_t taosGetQueueNumber(STaosQset *qset) { return qset->numOfQueues; } int32_t taosGetQueueNumber(STaosQset *qset) { return qset->numOfQueues; }
int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, void **ahandle, FProcessItem *itemFp) { int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, void **ahandle, FItem *itemFp) {
STaosQnode *pNode = NULL; STaosQnode *pNode = NULL;
int32_t code = 0; int32_t code = 0;
...@@ -365,6 +376,7 @@ int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, void **ahandle, FP ...@@ -365,6 +376,7 @@ int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, void **ahandle, FP
*ppItem = pNode->item; *ppItem = pNode->item;
if (ahandle) *ahandle = queue->ahandle; if (ahandle) *ahandle = queue->ahandle;
if (itemFp) *itemFp = queue->itemFp; if (itemFp) *itemFp = queue->itemFp;
queue->head = pNode->next; queue->head = pNode->next;
if (queue->head == NULL) queue->tail = NULL; if (queue->head == NULL) queue->tail = NULL;
queue->numOfItems--; queue->numOfItems--;
...@@ -382,7 +394,7 @@ int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, void **ahandle, FP ...@@ -382,7 +394,7 @@ int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, void **ahandle, FP
return code; return code;
} }
int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, void **ahandle, FProcessItems *itemsFp) { int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, void **ahandle, FItems *itemsFp) {
STaosQueue *queue; STaosQueue *queue;
int32_t code = 0; int32_t code = 0;
...@@ -411,7 +423,9 @@ int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, void **ahand ...@@ -411,7 +423,9 @@ int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, void **ahand
queue->tail = NULL; queue->tail = NULL;
queue->numOfItems = 0; queue->numOfItems = 0;
atomic_sub_fetch_32(&qset->numOfItems, qall->numOfItems); atomic_sub_fetch_32(&qset->numOfItems, qall->numOfItems);
for (int32_t j = 1; j < qall->numOfItems; ++j) tsem_wait(&qset->sem); for (int32_t j = 1; j < qall->numOfItems; ++j) {
tsem_wait(&qset->sem);
}
} }
pthread_mutex_unlock(&queue->mutex); pthread_mutex_unlock(&queue->mutex);
...@@ -423,6 +437,65 @@ int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, void **ahand ...@@ -423,6 +437,65 @@ int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, void **ahand
return code; return code;
} }
int32_t taosReadQitemFromQsetByThread(STaosQset *qset, void **ppItem, void **ahandle, FItem *itemFp, int32_t threadId) {
STaosQnode *pNode = NULL;
int32_t code = -1;
tsem_wait(&qset->sem);
pthread_mutex_lock(&qset->mutex);
for (int32_t i = 0; i < qset->numOfQueues; ++i) {
if (qset->current == NULL) qset->current = qset->head;
STaosQueue *queue = qset->current;
if (queue) qset->current = queue->next;
if (queue == NULL) break;
if (queue->head == NULL) continue;
if (queue->threadId != -1 && queue->threadId != threadId) {
code = 0;
continue;
}
pthread_mutex_lock(&queue->mutex);
if (queue->head) {
pNode = queue->head;
pNode->queue = queue;
queue->threadId = threadId;
*ppItem = pNode->item;
if (ahandle) *ahandle = queue->ahandle;
if (itemFp) *itemFp = queue->itemFp;
queue->head = pNode->next;
if (queue->head == NULL) queue->tail = NULL;
queue->numOfItems--;
atomic_sub_fetch_32(&qset->numOfItems, 1);
code = 1;
uTrace("item:%p is read out from queue:%p, items:%d", *ppItem, queue, queue->numOfItems);
}
pthread_mutex_unlock(&queue->mutex);
if (pNode) break;
}
pthread_mutex_unlock(&qset->mutex);
return code;
}
void taosResetQsetThread(STaosQset *qset, void *pItem) {
if (pItem == NULL) return;
STaosQnode *pNode = (STaosQnode *)((char *)pItem - sizeof(STaosQnode));
pthread_mutex_lock(&qset->mutex);
pNode->queue->threadId = -1;
for (int32_t i = 0; i < pNode->queue->numOfItems; ++i) {
tsem_post(&qset->sem);
}
pthread_mutex_unlock(&qset->mutex);
}
int32_t taosGetQueueItemsNumber(STaosQueue *queue) { int32_t taosGetQueueItemsNumber(STaosQueue *queue) {
if (!queue) return 0; if (!queue) return 0;
......
...@@ -14,38 +14,46 @@ ...@@ -14,38 +14,46 @@
*/ */
#define _DEFAULT_SOURCE #define _DEFAULT_SOURCE
#include "os.h"
#include "ulog.h"
#include "tqueue.h"
#include "tworker.h" #include "tworker.h"
#include "taoserror.h"
#include "ulog.h"
typedef void* (*ThreadFp)(void *param); typedef void *(*ThreadFp)(void *param);
int32_t tWorkerInit(SWorkerPool *pool) { int32_t tQWorkerInit(SQWorkerPool *pool) {
pool->qset = taosOpenQset(); pool->qset = taosOpenQset();
pool->workers = calloc(sizeof(SWorker), pool->max); pool->workers = calloc(sizeof(SQWorker), pool->max);
pthread_mutex_init(&pool->mutex, NULL); if (pool->workers == NULL) {
for (int i = 0; i < pool->max; ++i) { terrno = TSDB_CODE_OUT_OF_MEMORY;
SWorker *worker = pool->workers + i; return -1;
}
if (pthread_mutex_init(&pool->mutex, NULL)) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1;
}
for (int32_t i = 0; i < pool->max; ++i) {
SQWorker *worker = pool->workers + i;
worker->id = i; worker->id = i;
worker->pool = pool; worker->pool = pool;
} }
uInfo("worker:%s is initialized, min:%d max:%d", pool->name, pool->min, pool->max); uDebug("worker:%s is initialized, min:%d max:%d", pool->name, pool->min, pool->max);
return 0; return 0;
} }
void tWorkerCleanup(SWorkerPool *pool) { void tQWorkerCleanup(SQWorkerPool *pool) {
for (int i = 0; i < pool->max; ++i) { for (int32_t i = 0; i < pool->max; ++i) {
SWorker *worker = pool->workers + i; SQWorker *worker = pool->workers + i;
if (worker == NULL) continue; if (worker == NULL) continue;
if (taosCheckPthreadValid(worker->thread)) { if (taosCheckPthreadValid(worker->thread)) {
taosQsetThreadResume(pool->qset); taosQsetThreadResume(pool->qset);
} }
} }
for (int i = 0; i < pool->max; ++i) { for (int32_t i = 0; i < pool->max; ++i) {
SWorker *worker = pool->workers + i; SQWorker *worker = pool->workers + i;
if (worker == NULL) continue; if (worker == NULL) continue;
if (taosCheckPthreadValid(worker->thread)) { if (taosCheckPthreadValid(worker->thread)) {
pthread_join(worker->thread, NULL); pthread_join(worker->thread, NULL);
...@@ -56,15 +64,15 @@ void tWorkerCleanup(SWorkerPool *pool) { ...@@ -56,15 +64,15 @@ void tWorkerCleanup(SWorkerPool *pool) {
taosCloseQset(pool->qset); taosCloseQset(pool->qset);
pthread_mutex_destroy(&pool->mutex); pthread_mutex_destroy(&pool->mutex);
uInfo("worker:%s is closed", pool->name); uDebug("worker:%s is closed", pool->name);
} }
static void *tWorkerThreadFp(SWorker *worker) { static void *tQWorkerThreadFp(SQWorker *worker) {
SWorkerPool *pool = worker->pool; SQWorkerPool *pool = worker->pool;
FProcessItem fp = NULL; FItem fp = NULL;
void *msg = NULL; void * msg = NULL;
void *ahandle = NULL; void * ahandle = NULL;
int32_t code = 0; int32_t code = 0;
taosBlockSIGPIPE(); taosBlockSIGPIPE();
...@@ -77,7 +85,7 @@ static void *tWorkerThreadFp(SWorker *worker) { ...@@ -77,7 +85,7 @@ static void *tWorkerThreadFp(SWorker *worker) {
break; break;
} }
if (fp) { if (fp != NULL) {
(*fp)(ahandle, msg); (*fp)(ahandle, msg);
} }
} }
...@@ -85,11 +93,12 @@ static void *tWorkerThreadFp(SWorker *worker) { ...@@ -85,11 +93,12 @@ static void *tWorkerThreadFp(SWorker *worker) {
return NULL; return NULL;
} }
STaosQueue *tWorkerAllocQueue(SWorkerPool *pool, void *ahandle, FProcessItem fp) { STaosQueue *tWorkerAllocQueue(SQWorkerPool *pool, void *ahandle, FItem fp, ThreadFp threadFp) {
pthread_mutex_lock(&pool->mutex); pthread_mutex_lock(&pool->mutex);
STaosQueue *queue = taosOpenQueue(); STaosQueue *queue = taosOpenQueue();
if (queue == NULL) { if (queue == NULL) {
pthread_mutex_unlock(&pool->mutex); pthread_mutex_unlock(&pool->mutex);
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL; return NULL;
} }
...@@ -99,14 +108,18 @@ STaosQueue *tWorkerAllocQueue(SWorkerPool *pool, void *ahandle, FProcessItem fp) ...@@ -99,14 +108,18 @@ STaosQueue *tWorkerAllocQueue(SWorkerPool *pool, void *ahandle, FProcessItem fp)
// spawn a thread to process queue // spawn a thread to process queue
if (pool->num < pool->max) { if (pool->num < pool->max) {
do { do {
SWorker *worker = pool->workers + pool->num; SQWorker *worker = pool->workers + pool->num;
pthread_attr_t thAttr; pthread_attr_t thAttr;
pthread_attr_init(&thAttr); pthread_attr_init(&thAttr);
pthread_attr_setdetachstate(&thAttr, PTHREAD_CREATE_JOINABLE); pthread_attr_setdetachstate(&thAttr, PTHREAD_CREATE_JOINABLE);
if (pthread_create(&worker->thread, &thAttr, (ThreadFp)tWorkerThreadFp, worker) != 0) { if (pthread_create(&worker->thread, &thAttr, threadFp, worker) != 0) {
uError("worker:%s:%d failed to create thread to process since %s", pool->name, worker->id, strerror(errno)); uError("worker:%s:%d failed to create thread to process since %s", pool->name, worker->id, strerror(errno));
taosCloseQueue(queue);
terrno = TSDB_CODE_OUT_OF_MEMORY;
queue = NULL;
break;
} }
pthread_attr_destroy(&thAttr); pthread_attr_destroy(&thAttr);
...@@ -121,19 +134,73 @@ STaosQueue *tWorkerAllocQueue(SWorkerPool *pool, void *ahandle, FProcessItem fp) ...@@ -121,19 +134,73 @@ STaosQueue *tWorkerAllocQueue(SWorkerPool *pool, void *ahandle, FProcessItem fp)
return queue; return queue;
} }
void tWorkerFreeQueue(SWorkerPool *pool, STaosQueue *queue) { STaosQueue *tQWorkerAllocQueue(SQWorkerPool *pool, void *ahandle, FItem fp) {
return tWorkerAllocQueue(pool, ahandle, fp, (ThreadFp)tQWorkerThreadFp);
}
void tQWorkerFreeQueue(SQWorkerPool *pool, STaosQueue *queue) {
taosCloseQueue(queue); taosCloseQueue(queue);
uDebug("worker:%s, queue:%p is freed", pool->name, queue); uDebug("worker:%s, queue:%p is freed", pool->name, queue);
} }
int32_t tMWorkerInit(SMWorkerPool *pool) { int32_t tFWorkerInit(SFWorkerPool *pool) { return tQWorkerInit((SQWorkerPool *)pool); }
void tFWorkerCleanup(SFWorkerPool *pool) { tQWorkerCleanup(pool); }
static void *tFWorkerThreadFp(SQWorker *worker) {
SQWorkerPool *pool = worker->pool;
FItem fp = NULL;
void * msg = NULL;
void * ahandle = NULL;
int32_t code = 0;
taosBlockSIGPIPE();
setThreadName(pool->name);
uDebug("worker:%s:%d is running", pool->name, worker->id);
while (1) {
code = taosReadQitemFromQsetByThread(pool->qset, (void **)&msg, &ahandle, &fp, worker->id);
if (code < 0) {
uDebug("worker:%s:%d qset:%p, got no message and exiting", pool->name, worker->id, pool->qset);
break;
} else if (code == 0) {
// uTrace("worker:%s:%d qset:%p, got no message and continue", pool->name, worker->id, pool->qset);
continue;
}
if (fp != NULL) {
(*fp)(ahandle, msg);
}
taosResetQsetThread(pool->qset, msg);
}
return NULL;
}
STaosQueue *tFWorkerAllocQueue(SQWorkerPool *pool, void *ahandle, FItem fp) {
return tWorkerAllocQueue(pool, ahandle, fp, (ThreadFp)tFWorkerThreadFp);
}
void tFWorkerFreeQueue(SFWorkerPool *pool, STaosQueue *queue) { tQWorkerFreeQueue(pool, queue); }
int32_t tWWorkerInit(SWWorkerPool *pool) {
pool->nextId = 0; pool->nextId = 0;
pool->workers = calloc(sizeof(SMWorker), pool->max); pool->workers = calloc(sizeof(SWWorker), pool->max);
if (pool->workers == NULL) return -1; if (pool->workers == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1;
}
if (pthread_mutex_init(&pool->mutex, NULL) != 0) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1;
}
pthread_mutex_init(&pool->mutex, NULL);
for (int32_t i = 0; i < pool->max; ++i) { for (int32_t i = 0; i < pool->max; ++i) {
SMWorker *worker = pool->workers + i; SWWorker *worker = pool->workers + i;
worker->id = i; worker->id = i;
worker->qall = NULL; worker->qall = NULL;
worker->qset = NULL; worker->qset = NULL;
...@@ -144,16 +211,18 @@ int32_t tMWorkerInit(SMWorkerPool *pool) { ...@@ -144,16 +211,18 @@ int32_t tMWorkerInit(SMWorkerPool *pool) {
return 0; return 0;
} }
void tMWorkerCleanup(SMWorkerPool *pool) { void tWWorkerCleanup(SWWorkerPool *pool) {
for (int32_t i = 0; i < pool->max; ++i) { for (int32_t i = 0; i < pool->max; ++i) {
SMWorker *worker = pool->workers + i; SWWorker *worker = pool->workers + i;
if (taosCheckPthreadValid(worker->thread)) { if (taosCheckPthreadValid(worker->thread)) {
if (worker->qset) taosQsetThreadResume(worker->qset); if (worker->qset) {
taosQsetThreadResume(worker->qset);
}
} }
} }
for (int32_t i = 0; i < pool->max; ++i) { for (int32_t i = 0; i < pool->max; ++i) {
SMWorker *worker = pool->workers + i; SWWorker *worker = pool->workers + i;
if (taosCheckPthreadValid(worker->thread)) { if (taosCheckPthreadValid(worker->thread)) {
pthread_join(worker->thread, NULL); pthread_join(worker->thread, NULL);
taosFreeQall(worker->qall); taosFreeQall(worker->qall);
...@@ -167,12 +236,12 @@ void tMWorkerCleanup(SMWorkerPool *pool) { ...@@ -167,12 +236,12 @@ void tMWorkerCleanup(SMWorkerPool *pool) {
uInfo("worker:%s is closed", pool->name); uInfo("worker:%s is closed", pool->name);
} }
static void *tWriteWorkerThreadFp(SMWorker *worker) { static void *tWWorkerThreadFp(SWWorker *worker) {
SMWorkerPool *pool = worker->pool; SWWorkerPool *pool = worker->pool;
FProcessItems fp = NULL; FItems fp = NULL;
void *msg = NULL; void * msg = NULL;
void *ahandle = NULL; void * ahandle = NULL;
int32_t numOfMsgs = 0; int32_t numOfMsgs = 0;
int32_t qtype = 0; int32_t qtype = 0;
...@@ -187,7 +256,7 @@ static void *tWriteWorkerThreadFp(SMWorker *worker) { ...@@ -187,7 +256,7 @@ static void *tWriteWorkerThreadFp(SMWorker *worker) {
break; break;
} }
if (fp) { if (fp != NULL) {
(*fp)(ahandle, worker->qall, numOfMsgs); (*fp)(ahandle, worker->qall, numOfMsgs);
} }
} }
...@@ -195,13 +264,14 @@ static void *tWriteWorkerThreadFp(SMWorker *worker) { ...@@ -195,13 +264,14 @@ static void *tWriteWorkerThreadFp(SMWorker *worker) {
return NULL; return NULL;
} }
STaosQueue *tMWorkerAllocQueue(SMWorkerPool *pool, void *ahandle, FProcessItems fp) { STaosQueue *tWWorkerAllocQueue(SWWorkerPool *pool, void *ahandle, FItems fp) {
pthread_mutex_lock(&pool->mutex); pthread_mutex_lock(&pool->mutex);
SMWorker *worker = pool->workers + pool->nextId; SWWorker *worker = pool->workers + pool->nextId;
STaosQueue *queue = taosOpenQueue(); STaosQueue *queue = taosOpenQueue();
if (queue == NULL) { if (queue == NULL) {
pthread_mutex_unlock(&pool->mutex); pthread_mutex_unlock(&pool->mutex);
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL; return NULL;
} }
...@@ -221,17 +291,19 @@ STaosQueue *tMWorkerAllocQueue(SMWorkerPool *pool, void *ahandle, FProcessItems ...@@ -221,17 +291,19 @@ STaosQueue *tMWorkerAllocQueue(SMWorkerPool *pool, void *ahandle, FProcessItems
taosCloseQset(worker->qset); taosCloseQset(worker->qset);
taosCloseQueue(queue); taosCloseQueue(queue);
pthread_mutex_unlock(&pool->mutex); pthread_mutex_unlock(&pool->mutex);
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL; return NULL;
} }
pthread_attr_t thAttr; pthread_attr_t thAttr;
pthread_attr_init(&thAttr); pthread_attr_init(&thAttr);
pthread_attr_setdetachstate(&thAttr, PTHREAD_CREATE_JOINABLE); pthread_attr_setdetachstate(&thAttr, PTHREAD_CREATE_JOINABLE);
if (pthread_create(&worker->thread, &thAttr, (ThreadFp)tWriteWorkerThreadFp, worker) != 0) { if (pthread_create(&worker->thread, &thAttr, (ThreadFp)tWWorkerThreadFp, worker) != 0) {
uError("worker:%s:%d failed to create thread to process since %s", pool->name, worker->id, strerror(errno)); uError("worker:%s:%d failed to create thread to process since %s", pool->name, worker->id, strerror(errno));
taosFreeQall(worker->qall); taosFreeQall(worker->qall);
taosCloseQset(worker->qset); taosCloseQset(worker->qset);
taosCloseQueue(queue); taosCloseQueue(queue);
terrno = TSDB_CODE_OUT_OF_MEMORY;
queue = NULL; queue = NULL;
} else { } else {
uDebug("worker:%s:%d is launched, max:%d", pool->name, worker->id, pool->max); uDebug("worker:%s:%d is launched, max:%d", pool->name, worker->id, pool->max);
...@@ -250,7 +322,7 @@ STaosQueue *tMWorkerAllocQueue(SMWorkerPool *pool, void *ahandle, FProcessItems ...@@ -250,7 +322,7 @@ STaosQueue *tMWorkerAllocQueue(SMWorkerPool *pool, void *ahandle, FProcessItems
return queue; return queue;
} }
void tMWorkerFreeQueue(SMWorkerPool *pool, STaosQueue *queue) { void tWWorkerFreeQueue(SWWorkerPool *pool, STaosQueue *queue) {
taosCloseQueue(queue); taosCloseQueue(queue);
uDebug("worker:%s, queue:%p is freed", pool->name, queue); uDebug("worker:%s, queue:%p is freed", pool->name, queue);
} }
...@@ -13,12 +13,12 @@ IF (HEADER_GTEST_INCLUDE_DIR AND (LIB_GTEST_STATIC_DIR OR LIB_GTEST_SHARED_DIR)) ...@@ -13,12 +13,12 @@ IF (HEADER_GTEST_INCLUDE_DIR AND (LIB_GTEST_STATIC_DIR OR LIB_GTEST_SHARED_DIR))
LIST(REMOVE_ITEM SOURCE_LIST ${CMAKE_CURRENT_SOURCE_DIR}/trefTest.c) LIST(REMOVE_ITEM SOURCE_LIST ${CMAKE_CURRENT_SOURCE_DIR}/trefTest.c)
ADD_EXECUTABLE(utilTest ${SOURCE_LIST}) ADD_EXECUTABLE(utilTest ${SOURCE_LIST})
TARGET_LINK_LIBRARIES(utilTest util common os gtest pthread gcov) TARGET_LINK_LIBRARIES(utilTest util common os gtest pthread)
LIST(REMOVE_ITEM SOURCE_LIST ${CMAKE_CURRENT_SOURCE_DIR}/cacheTest.cpp) LIST(REMOVE_ITEM SOURCE_LIST ${CMAKE_CURRENT_SOURCE_DIR}/cacheTest.cpp)
LIST(APPEND SOURCE_LIST ${CMAKE_CURRENT_SOURCE_DIR}/hashTest.cpp) LIST(APPEND SOURCE_LIST ${CMAKE_CURRENT_SOURCE_DIR}/hashTest.cpp)
ADD_EXECUTABLE(hashTest ${SOURCE_LIST}) ADD_EXECUTABLE(hashTest ${SOURCE_LIST})
TARGET_LINK_LIBRARIES(hashTest util common os gtest pthread gcov) TARGET_LINK_LIBRARIES(hashTest util common os gtest pthread)
LIST(APPEND BIN_SRC ${CMAKE_CURRENT_SOURCE_DIR}/trefTest.c) LIST(APPEND BIN_SRC ${CMAKE_CURRENT_SOURCE_DIR}/trefTest.c)
ADD_EXECUTABLE(trefTest ${BIN_SRC}) ADD_EXECUTABLE(trefTest ${BIN_SRC})
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册