提交 23ac69db 编写于 作者: H Haojun Liao

[td-11818] merge 3.0

......@@ -51,7 +51,7 @@ TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_MQ_QUERY, "mq-query" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_MQ_CONNECT, "mq-connect" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_MQ_DISCONNECT, "mq-disconnect" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_MQ_SET_CUR, "mq-set-cur" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_RSP_READY, "rsp-ready" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_RES_READY, "res-ready" )
// message from client to mnode
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_CONNECT, "connect" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_CREATE_ACCT, "create-acct" )
......@@ -78,8 +78,7 @@ TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_CREATE_STB, "create-stb" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_ALTER_STB, "alter-stb" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_DROP_STB, "drop-stb" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_VGROUP_LIST, "vgroup-list" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_KILL_QUERY, "kill-query" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_KILL_STREAM, "kill-stream" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_KILL_QUERY, "kill-query" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_KILL_CONN, "kill-conn" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_HEARTBEAT, "heartbeat" )
TAOS_DEFINE_MESSAGE_TYPE( TSDB_MSG_TYPE_SHOW, "show" )
......@@ -964,18 +963,6 @@ typedef struct {
char subSqlInfo[TSDB_SHOW_SUBQUERY_LEN]; // include subqueries' index, Obj IDs and states(C-complete/I-imcomplete)
} SQueryDesc;
typedef struct {
char sql[TSDB_SHOW_SQL_LEN];
char dstTable[TSDB_TABLE_NAME_LEN];
int32_t streamId;
int64_t num; // number of computing/cycles
int64_t useconds;
int64_t ctime;
int64_t stime;
int64_t slidingTime;
int64_t interval;
} SStreamDesc;
typedef struct {
int32_t connId;
int32_t pid;
......@@ -1112,6 +1099,7 @@ typedef struct {
} SUpdateTagValRsp;
typedef struct SSchedulerQueryMsg {
uint64_t schedulerId;
uint64_t queryId;
uint64_t taskId;
uint32_t contentLen;
......@@ -1119,15 +1107,39 @@ typedef struct SSchedulerQueryMsg {
} SSchedulerQueryMsg;
typedef struct SSchedulerReadyMsg {
uint64_t schedulerId;
uint64_t queryId;
uint64_t taskId;
} SSchedulerReadyMsg;
typedef struct SSchedulerFetchMsg {
uint64_t schedulerId;
uint64_t queryId;
uint64_t taskId;
} SSchedulerFetchMsg;
typedef struct SSchedulerStatusMsg {
uint64_t schedulerId;
} SSchedulerStatusMsg;
typedef struct STaskStatus {
uint64_t queryId;
uint64_t taskId;
int8_t status;
} STaskStatus;
typedef struct SSchedulerStatusRsp {
uint32_t num;
STaskStatus status[];
} SSchedulerStatusRsp;
typedef struct SSchedulerCancelMsg {
uint64_t schedulerId;
uint64_t queryId;
uint64_t taskId;
} SSchedulerCancelMsg;
#pragma pack(pop)
......
......@@ -19,7 +19,7 @@
#ifdef __cplusplus
extern "C" {
#endif
#include "trpc.h"
typedef struct {
uint64_t numOfStartTask;
......@@ -32,48 +32,6 @@ typedef struct {
uint64_t numOfErrors;
} SQnodeStat;
/* start Task msg */
typedef struct {
uint32_t schedulerIp;
uint16_t schedulerPort;
int64_t taskId;
int64_t queryId;
uint32_t srcIp;
uint16_t srcPort;
} SQnodeStartTaskMsg;
/* stop Task msg */
typedef struct {
int64_t taskId;
} SQnodeStopTaskMsg;
/* start/stop Task msg response */
typedef struct {
int64_t taskId;
int32_t code;
} SQnodeTaskRespMsg;
/* Task status msg */
typedef struct {
int64_t taskId;
int32_t status;
int64_t queryId;
} SQnodeTaskStatusMsg;
/* Qnode/Scheduler heartbeat msg */
typedef struct {
int32_t status;
int32_t load;
} SQnodeHeartbeatMsg;
/* Qnode sent/received msg */
typedef struct {
int8_t msgType;
int32_t msgLen;
char msg[];
} SQnodeMsg;
/**
* Start one Qnode in Dnode.
......
......@@ -44,41 +44,46 @@ typedef enum { QUERY_TERM = 0, QUERY_PREFIX = 1, QUERY_SUFFIX = 2, QUERY_REGEX =
* @param: oper
*
*/
SIndexMultiTermQuery *indexMultiTermQueryCreate(EIndexOperatorType oper);
void indexMultiTermQueryDestroy(SIndexMultiTermQuery *pQuery);
int indexMultiTermQueryAdd(SIndexMultiTermQuery *pQuery, SIndexTerm *term, EIndexQueryType type);
SIndexMultiTermQuery* indexMultiTermQueryCreate(EIndexOperatorType oper);
void indexMultiTermQueryDestroy(SIndexMultiTermQuery* pQuery);
int indexMultiTermQueryAdd(SIndexMultiTermQuery* pQuery, SIndexTerm* term, EIndexQueryType type);
/*
* @param:
* @param:
*/
int indexOpen(SIndexOpts *opt, const char *path, SIndex **index);
void indexClose(SIndex *index);
int indexPut(SIndex *index, SIndexMultiTerm *terms, uint64_t uid);
int indexDelete(SIndex *index, SIndexMultiTermQuery *query);
int indexSearch(SIndex *index, SIndexMultiTermQuery *query, SArray *result);
int indexRebuild(SIndex *index, SIndexOpts *opt);
int indexOpen(SIndexOpts* opt, const char* path, SIndex** index);
void indexClose(SIndex* index);
int indexPut(SIndex* index, SIndexMultiTerm* terms, uint64_t uid);
int indexDelete(SIndex* index, SIndexMultiTermQuery* query);
int indexSearch(SIndex* index, SIndexMultiTermQuery* query, SArray* result);
int indexRebuild(SIndex* index, SIndexOpts* opt);
/*
* @param
* @param
*/
SIndexMultiTerm *indexMultiTermCreate();
int indexMultiTermAdd(SIndexMultiTerm *terms, SIndexTerm *term);
void indexMultiTermDestroy(SIndexMultiTerm *terms);
SIndexMultiTerm* indexMultiTermCreate();
int indexMultiTermAdd(SIndexMultiTerm* terms, SIndexTerm* term);
void indexMultiTermDestroy(SIndexMultiTerm* terms);
/*
* @param:
* @param:
*/
SIndexOpts *indexOptsCreate();
void indexOptsDestroy(SIndexOpts *opts);
SIndexOpts* indexOptsCreate();
void indexOptsDestroy(SIndexOpts* opts);
/*
* @param:
* @param:
*/
SIndexTerm *indexTermCreate(int64_t suid, SIndexOperOnColumn operType, uint8_t colType, const char *colName,
int32_t nColName, const char *colVal, int32_t nColVal);
void indexTermDestroy(SIndexTerm *p);
SIndexTerm* indexTermCreate(int64_t suid,
SIndexOperOnColumn operType,
uint8_t colType,
const char* colName,
int32_t nColName,
const char* colVal,
int32_t nColVal);
void indexTermDestroy(SIndexTerm* p);
#ifdef __cplusplus
}
......
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_PARSENODES_H_
#define _TD_PARSENODES_H_
#ifdef __cplusplus
extern "C" {
#endif
#include "catalog.h"
#include "common.h"
#include "function.h"
#include "tmsgtype.h"
#include "tname.h"
#include "tvariant.h"
/*
* The first field of a node of any type is guaranteed to be the int16_t.
* Hence the type of any node can be gotten by casting it to SQueryNode.
*/
typedef struct SQueryNode {
int16_t type;
} SQueryNode;
#define nodeType(nodeptr) (((const SQueryNode*)(nodeptr))->type)
typedef struct SField {
char name[TSDB_COL_NAME_LEN];
uint8_t type;
int16_t bytes;
} SField;
typedef struct SParseBasicCtx {
const char *db;
int32_t acctId;
uint64_t requestId;
} SParseBasicCtx;
typedef struct SFieldInfo {
int16_t numOfOutput; // number of column in result
SField *final;
SArray *internalField; // SArray<SInternalField>
} SFieldInfo;
typedef struct SCond {
uint64_t uid;
int32_t len; // length of tag query condition data
char * cond;
} SCond;
typedef struct SJoinNode {
uint64_t uid;
int16_t tagColId;
SArray* tsJoin;
SArray* tagJoin;
} SJoinNode;
typedef struct SJoinInfo {
bool hasJoin;
SJoinNode *joinTables[TSDB_MAX_JOIN_TABLE_NUM];
} SJoinInfo;
typedef struct STagCond {
int16_t relType; // relation between tbname list and query condition, including : TK_AND or TK_OR
SCond tbnameCond; // tbname query condition, only support tbname query condition on one table
SJoinInfo joinInfo; // join condition, only support two tables join currently
SArray *pCond; // for different table, the query condition must be seperated
} STagCond;
typedef struct STableMetaInfo {
STableMeta *pTableMeta; // table meta, cached in client side and acquired by name
SVgroupsInfo *vgroupList;
SName name;
char aliasName[TSDB_TABLE_NAME_LEN]; // alias name of table specified in query sql
SArray *tagColList; // SArray<SColumn*>, involved tag columns
} STableMetaInfo;
typedef struct SColumnIndex {
int16_t tableIndex;
int16_t columnIndex;
int16_t type; // normal column/tag/ user input constant column
} SColumnIndex;
// select statement
typedef struct SQueryStmtInfo {
int16_t command; // the command may be different for each subclause, so keep it seperately.
uint32_t type; // query/insert type
STimeWindow window; // the whole query time window
SInterval interval; // tumble time window
SSessionWindow sessionWindow; // session time window
SStateWindow stateWindow; // state window query
SGroupbyExpr groupbyExpr; // groupby tags info
SArray * colList; // SArray<SColumn*>
SFieldInfo fieldsInfo;
SArray** exprList; // SArray<SExprInfo*>
SLimit limit;
SLimit slimit;
STagCond tagCond;
SArray * colCond;
SArray * order;
int16_t numOfTables;
int16_t curTableIdx;
STableMetaInfo **pTableMetaInfo;
struct STSBuf *tsBuf;
int16_t fillType; // final result fill type
int64_t * fillVal; // default value for fill
int32_t numOfFillVal; // fill value size
char * msg; // pointer to the pCmd->payload to keep error message temporarily
int64_t clauseLimit; // limit for current sub clause
int64_t prjOffset; // offset value in the original sql expression, only applied at client side
int64_t vgroupLimit; // table limit in case of super table projection query + global order + limit
int32_t udColumnId; // current user-defined constant output field column id, monotonically decreases from TSDB_UD_COLUMN_INDEX
int32_t bufLen;
char* buf;
SArray *pUdfInfo;
struct SQueryStmtInfo *sibling; // sibling
struct SQueryStmtInfo *pDownstream;
SMultiFunctionsDesc info;
SArray *pUpstream; // SArray<struct SQueryStmtInfo>
int32_t havingFieldNum;
int32_t exprListLevelIndex;
} SQueryStmtInfo;
typedef enum {
PAYLOAD_TYPE_KV = 0,
PAYLOAD_TYPE_RAW = 1,
} EPayloadType;
typedef struct SVgDataBlocks {
SVgroupInfo vg;
int32_t numOfTables; // number of tables in current submit block
uint32_t size;
char *pData; // SMsgDesc + SSubmitMsg + SSubmitBlk + ...
} SVgDataBlocks;
typedef struct SInsertStmtInfo {
int16_t nodeType;
SArray* pDataBlocks; // data block for each vgroup, SArray<SVgDataBlocks*>.
int8_t schemaAttache; // denote if submit block is built with table schema or not
uint8_t payloadType; // EPayloadType. 0: K-V payload for non-prepare insert, 1: rawPayload for prepare insert
uint32_t insertType; // insert data from [file|sql statement| bound statement]
const char* sql; // current sql statement position
} SInsertStmtInfo;
#ifdef __cplusplus
}
#endif
#endif /*_TD_PARSENODES_H_*/
......@@ -20,115 +20,7 @@
extern "C" {
#endif
#include "catalog.h"
#include "common.h"
#include "tname.h"
#include "tvariant.h"
#include "function.h"
typedef struct SField {
char name[TSDB_COL_NAME_LEN];
uint8_t type;
int32_t bytes;
} SField;
typedef struct SFieldInfo {
int16_t numOfOutput; // number of column in result
SField *final;
SArray *internalField; // SArray<SInternalField>
} SFieldInfo;
// TODO merged with SParseContext
typedef struct SParseCtx {
char *db;
int32_t acctId;
uint64_t requestId;
} SParseCtx;
typedef struct SCond {
uint64_t uid;
int32_t len; // length of tag query condition data
char * cond;
} SCond;
typedef struct SJoinNode {
uint64_t uid;
int16_t tagColId;
SArray* tsJoin;
SArray* tagJoin;
} SJoinNode;
typedef struct SJoinInfo {
bool hasJoin;
SJoinNode *joinTables[TSDB_MAX_JOIN_TABLE_NUM];
} SJoinInfo;
typedef struct STagCond {
int16_t relType; // relation between tbname list and query condition, including : TK_AND or TK_OR
SCond tbnameCond; // tbname query condition, only support tbname query condition on one table
SJoinInfo joinInfo; // join condition, only support two tables join currently
SArray *pCond; // for different table, the query condition must be seperated
} STagCond;
typedef struct STableMetaInfo {
STableMeta *pTableMeta; // table meta, cached in client side and acquired by name
SVgroupsInfo *vgroupList;
SName name;
char aliasName[TSDB_TABLE_NAME_LEN]; // alias name of table specified in query sql
SArray *tagColList; // SArray<SColumn*>, involved tag columns
} STableMetaInfo;
typedef struct SQueryStmtInfo {
int16_t command; // the command may be different for each subclause, so keep it seperately.
uint32_t type; // query/insert type
STimeWindow window; // the whole query time window
SInterval interval; // tumble time window
SSessionWindow sessionWindow; // session time window
SStateWindow stateWindow; // state window query
SGroupbyExpr groupbyExpr; // groupby tags info
SArray * colList; // SArray<SColumn*>
SFieldInfo fieldsInfo;
SArray** exprList; // SArray<SExprInfo*>
SLimit limit;
SLimit slimit;
STagCond tagCond;
SArray * colCond;
SArray * order;
int16_t numOfTables;
int16_t curTableIdx;
STableMetaInfo **pTableMetaInfo;
struct STSBuf *tsBuf;
int16_t fillType; // final result fill type
int64_t * fillVal; // default value for fill
int32_t numOfFillVal; // fill value size
char * msg; // pointer to the pCmd->payload to keep error message temporarily
int64_t clauseLimit; // limit for current sub clause
int64_t prjOffset; // offset value in the original sql expression, only applied at client side
int64_t vgroupLimit; // table limit in case of super table projection query + global order + limit
int32_t udColumnId; // current user-defined constant output field column id, monotonically decreases from TSDB_UD_COLUMN_INDEX
int32_t bufLen;
char* buf;
SArray *pUdfInfo;
struct SQueryStmtInfo *sibling; // sibling
struct SQueryStmtInfo *pDownstream;
SMultiFunctionsDesc info;
SArray *pUpstream; // SArray<struct SQueryStmtInfo>
int32_t havingFieldNum;
int32_t exprListLevelIndex;
} SQueryStmtInfo;
typedef struct SColumnIndex {
int16_t tableIndex;
int16_t columnIndex;
int16_t type; // normal column/tag/ user input constant column
} SColumnIndex;
struct SInsertStmtInfo;
#include "parsenodes.h"
/**
* True will be returned if the input sql string is insert, false otherwise.
......@@ -139,17 +31,16 @@ struct SInsertStmtInfo;
bool qIsInsertSql(const char* pStr, size_t length);
typedef struct SParseContext {
const char* pAcctId;
const char* pDbname;
void *pRpc;
const char* pClusterId;
const SEpSet* pEpSet;
int64_t id; // query id, generated by uuid generator
int8_t schemaAttached; // denote if submit block is built with table schema or not
const char* pSql; // sql string
size_t sqlLen; // length of the sql string
char* pMsg; // extended error message if exists to help avoid the problem in sql statement.
int32_t msgLen; // max length of the msg
SParseBasicCtx ctx;
void *pRpc;
struct SCatalog *pCatalog;
const SEpSet *pEpSet;
int64_t id; // query id, generated by uuid generator
int8_t schemaAttached; // denote if submit block is built with table schema or not
const char *pSql; // sql string
size_t sqlLen; // length of the sql string
char *pMsg; // extended error message if exists to help avoid the problem in sql statement.
int32_t msgLen; // max length of the msg
} SParseContext;
/**
......@@ -160,27 +51,7 @@ typedef struct SParseContext {
* @param msg extended error message if exists.
* @return error code
*/
int32_t qParseQuerySql(const char* pStr, size_t length, SParseCtx* pParseCtx, int32_t* type, void** pOutput, int32_t* outputLen, char* msg, int32_t msgLen);
typedef enum {
PAYLOAD_TYPE_KV = 0,
PAYLOAD_TYPE_RAW = 1,
} EPayloadType;
typedef struct SVgDataBlocks {
int64_t vgId; // virtual group id
int32_t numOfTables; // number of tables in current submit block
uint32_t size;
char *pData; // SMsgDesc + SSubmitMsg + SSubmitBlk + ...
} SVgDataBlocks;
typedef struct SInsertStmtInfo {
SArray* pDataBlocks; // data block for each vgroup, SArray<SVgDataBlocks*>.
int8_t schemaAttache; // denote if submit block is built with table schema or not
uint8_t payloadType; // EPayloadType. 0: K-V payload for non-prepare insert, 1: rawPayload for prepare insert
uint32_t insertType; // insert data from [file|sql statement| bound statement]
const char* sql; // current sql statement position
} SInsertStmtInfo;
int32_t qParseQuerySql(const char* pStr, size_t length, SParseBasicCtx* pParseCtx, int32_t* type, void** pOutput, int32_t* outputLen, char* msg, int32_t msgLen);
/**
* Parse the insert sql statement.
......@@ -217,9 +88,9 @@ typedef struct SSourceParam {
SExprInfo* createExprInfo(STableMetaInfo* pTableMetaInfo, const char* funcName, SSourceParam* pSource, SSchema* pResSchema, int16_t interSize);
int32_t copyExprInfoList(SArray* dst, const SArray* src, uint64_t uid, bool deepcopy);
int32_t copyAllExprInfo(SArray* dst, const SArray* src, bool deepcopy);
int32_t getExprFunctionLevel(SQueryStmtInfo* pQueryInfo);
int32_t getExprFunctionLevel(const SQueryStmtInfo* pQueryInfo);
STableMetaInfo* getMetaInfo(SQueryStmtInfo* pQueryInfo, int32_t tableIndex);
STableMetaInfo* getMetaInfo(const SQueryStmtInfo* pQueryInfo, int32_t tableIndex);
SSchema *getOneColumnSchema(const STableMeta* pTableMeta, int32_t colIndex);
SSchema createSchema(uint8_t type, int16_t bytes, int16_t colId, const char* name);
......
......@@ -25,6 +25,7 @@ extern "C" {
#define QUERY_TYPE_MERGE 1
#define QUERY_TYPE_PARTIAL 2
#define QUERY_TYPE_SCAN 3
#define QUERY_TYPE_MODIFY 4
enum OPERATOR_TYPE_E {
OP_Unknown,
......@@ -58,18 +59,17 @@ typedef struct SQueryNodeBasicInfo {
typedef struct SDataSink {
SQueryNodeBasicInfo info;
SDataBlockSchema schema;
} SDataSink;
typedef struct SDataDispatcher {
SDataSink sink;
// todo
} SDataDispatcher;
typedef struct SDataInserter {
SDataSink sink;
uint64_t uid; // unique id of the table
// todo data field
int32_t numOfTables;
uint32_t size;
char *pData;
} SDataInserter;
typedef struct SPhyNode {
......@@ -119,12 +119,13 @@ typedef struct SSubplanId {
typedef struct SSubplan {
SSubplanId id; // unique id of the subplan
int32_t type; // QUERY_TYPE_MERGE|QUERY_TYPE_PARTIAL|QUERY_TYPE_SCAN
int32_t level; // the execution level of current subplan, starting from 0.
SEpSet execEpSet; // for the scan sub plan, the optional execution node
SArray *pChildern; // the datasource subplan,from which to fetch the result
SArray *pParents; // the data destination subplan, get data from current subplan
SPhyNode *pNode; // physical plan of current subplan
int32_t type; // QUERY_TYPE_MERGE|QUERY_TYPE_PARTIAL|QUERY_TYPE_SCAN
int32_t level; // the execution level of current subplan, starting from 0.
SEpSet execEpSet; // for the scan sub plan, the optional execution node
SArray *pChildern; // the datasource subplan,from which to fetch the result
SArray *pParents; // the data destination subplan, get data from current subplan
SPhyNode *pNode; // physical plan of current subplan
SDataSink *pDataSink; // data of the subplan flow into the datasink
} SSubplan;
typedef struct SQueryDag {
......@@ -133,10 +134,12 @@ typedef struct SQueryDag {
SArray *pSubplans; // Element is SArray*, and nested element is SSubplan. The execution level of subplan, starting from 0.
} SQueryDag;
struct SQueryNode;
/**
* Create the physical plan for the query, according to the AST.
*/
int32_t qCreateQueryDag(const struct SQueryStmtInfo* pQueryInfo, struct SEpSet* pQnode, struct SQueryDag** pDag);
int32_t qCreateQueryDag(const struct SQueryNode* pQueryInfo, struct SEpSet* pQnode, struct SQueryDag** pDag);
// Set datasource of this subplan, multiple calls may be made to a subplan.
// @subplan subplan to be schedule
......@@ -144,7 +147,7 @@ int32_t qCreateQueryDag(const struct SQueryStmtInfo* pQueryInfo, struct SEpSet*
// @ep one execution location of this group of datasource subplans
int32_t qSetSubplanExecutionNode(SSubplan* subplan, uint64_t templateId, SEpAddr* ep);
int32_t qExplainQuery(const struct SQueryStmtInfo* pQueryInfo, struct SEpSet* pQnode, char** str);
int32_t qExplainQuery(const struct SQueryNode* pQueryInfo, struct SEpSet* pQnode, char** str);
/**
* Convert to subplan to string for the scheduler to send to the executor
......
......@@ -24,6 +24,15 @@ extern "C" {
#include "thash.h"
#include "tlog.h"
enum {
JOB_TASK_STATUS_NOT_START = 1,
JOB_TASK_STATUS_EXECUTING,
JOB_TASK_STATUS_SUCCEED,
JOB_TASK_STATUS_FAILED,
JOB_TASK_STATUS_CANCELLING,
JOB_TASK_STATUS_CANCELLED
};
typedef struct STableComInfo {
uint8_t numOfTags; // the number of tags in schema
uint8_t precision; // the number of precision
......
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_QWORKER_H_
#define _TD_QWORKER_H_
#ifdef __cplusplus
extern "C" {
#endif
#include "trpc.h"
typedef struct SQWorkerCfg {
uint32_t maxSchedulerNum;
uint32_t maxResCacheNum;
uint32_t maxSchTaskNum;
} SQWorkerCfg;
typedef struct {
uint64_t numOfStartTask;
uint64_t numOfStopTask;
uint64_t numOfRecvedFetch;
uint64_t numOfSentHb;
uint64_t numOfSentFetch;
uint64_t numOfTaskInQueue;
uint64_t numOfFetchInQueue;
uint64_t numOfErrors;
} SQWorkerStat;
int32_t qWorkerInit(SQWorkerCfg *cfg, void **qWorkerMgmt);
int32_t qWorkerProcessQueryMsg(void *qWorkerMgmt, SSchedulerQueryMsg *msg, SRpcMsg *rsp);
int32_t qWorkerProcessReadyMsg(void *qWorkerMgmt, SSchedulerReadyMsg *msg, SRpcMsg *rsp);
int32_t qWorkerProcessStatusMsg(void *qWorkerMgmt, SSchedulerStatusMsg *msg, SRpcMsg *rsp);
int32_t qWorkerProcessFetchMsg(void *qWorkerMgmt, SSchedulerFetchMsg *msg, SRpcMsg *rsp);
int32_t qWorkerProcessCancelMsg(void *qWorkerMgmt, SSchedulerCancelMsg *msg, SRpcMsg *rsp);
void qWorkerDestroy(void **qWorkerMgmt);
#ifdef __cplusplus
}
#endif
#endif /*_TD_QWORKER_H_*/
......@@ -25,6 +25,7 @@ extern "C" {
typedef struct SSchedulerCfg {
int32_t clusterType;
int32_t maxJobNum;
} SSchedulerCfg;
typedef struct SQueryProfileSummary {
......
......@@ -193,11 +193,10 @@ void taosHashCancelIterate(SHashObj *pHashObj, void *p);
/**
* Get the corresponding key information for a given data in hash table
* @param pHashObj
* @param data
* @return
*/
int32_t taosHashGetKey(SHashObj *pHashObj, void *data, void** key, size_t* keyLen);
int32_t taosHashGetKey(void *data, void** key, size_t* keyLen);
#ifdef __cplusplus
}
......
......@@ -152,7 +152,7 @@ TAOS_RES *taos_query_l(TAOS *taos, const char *sql, int sqlLen) {
void* output = NULL;
int32_t outputLen = 0;
SParseCtx c = {.requestId = pRequest->requestId, .acctId = pTscObj->acctId, .db = getConnectionDB(pTscObj)};
SParseBasicCtx c = {.requestId = pRequest->requestId, .acctId = pTscObj->acctId, .db = getConnectionDB(pTscObj)};
code = qParseQuerySql(pRequest->sqlstr, sqlLen, &c, &type, &output, &outputLen, pRequest->msgBuf, ERROR_MSG_BUF_DEFAULT_SIZE);
if (type == TSDB_SQL_CREATE_USER || type == TSDB_SQL_SHOW || type == TSDB_SQL_DROP_USER ||
type == TSDB_SQL_DROP_ACCT || type == TSDB_SQL_CREATE_DB || type == TSDB_SQL_CREATE_ACCT ||
......@@ -167,8 +167,6 @@ TAOS_RES *taos_query_l(TAOS *taos, const char *sql, int sqlLen) {
sendMsgToServer(pTscObj->pTransporter, &pTscObj->pAppInfo->mgmtEp.epSet, &body, &transporterId);
tsem_wait(&pRequest->body.rspSem);
destroyRequestMsgBody(&body);
} else {
assert(0);
......
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "os.h"
#include "taosmsg.h"
#include "query.h"
#include "clientInt.h"
#include "clientLog.h"
#include "tcache.h"
#include "tconfig.h"
#include "tglobal.h"
#include "tnote.h"
#include "tref.h"
#include "trpc.h"
#include "ttime.h"
#include "ttimezone.h"
#define TSC_VAR_NOT_RELEASE 1
#define TSC_VAR_RELEASED 0
SAppInfo appInfo;
int32_t tscReqRef = -1;
int32_t tscConnRef = -1;
static pthread_once_t tscinit = PTHREAD_ONCE_INIT;
volatile int32_t tscInitRes = 0;
static void registerRequest(SRequestObj* pRequest) {
STscObj *pTscObj = (STscObj *)taosAcquireRef(tscConnRef, pRequest->pTscObj->id);
assert(pTscObj != NULL);
// connection has been released already, abort creating request.
pRequest->self = taosAddRef(tscReqRef, pRequest);
int32_t num = atomic_add_fetch_32(&pTscObj->numOfReqs, 1);
if (pTscObj->pAppInfo) {
SInstanceActivity *pActivity = &pTscObj->pAppInfo->summary;
int32_t total = atomic_add_fetch_32(&pActivity->totalRequests, 1);
int32_t currentInst = atomic_add_fetch_32(&pActivity->currentRequests, 1);
tscDebug("0x%" PRIx64 " new Request from connObj:0x%" PRIx64 ", current:%d, app current:%d, total:%d", pRequest->self,
pRequest->pTscObj->id, num, currentInst, total);
}
}
static void deregisterRequest(SRequestObj* pRequest) {
assert(pRequest != NULL);
STscObj* pTscObj = pRequest->pTscObj;
SInstanceActivity* pActivity = &pTscObj->pAppInfo->summary;
int32_t currentInst = atomic_sub_fetch_32(&pActivity->currentRequests, 1);
int32_t num = atomic_sub_fetch_32(&pTscObj->numOfReqs, 1);
tscDebug("0x%"PRIx64" free Request from connObj: 0x%"PRIx64", current:%d, app current:%d", pRequest->self, pTscObj->id, num, currentInst);
taosReleaseRef(tscConnRef, pTscObj->id);
}
static void tscInitLogFile() {
taosReadGlobalLogCfg();
if (mkdir(tsLogDir, 0755) != 0 && errno != EEXIST) {
printf("failed to create log dir:%s\n", tsLogDir);
}
const char *defaultLogFileNamePrefix = "taoslog";
const int32_t maxLogFileNum = 10;
char temp[128] = {0};
sprintf(temp, "%s/%s", tsLogDir, defaultLogFileNamePrefix);
if (taosInitLog(temp, tsNumOfLogLines, maxLogFileNum) < 0) {
printf("failed to open log file in directory:%s\n", tsLogDir);
}
}
void closeTransporter(STscObj* pTscObj) {
if (pTscObj == NULL || pTscObj->pTransporter == NULL) {
return;
}
tscDebug("free transporter:%p in connObj: 0x%"PRIx64, pTscObj->pTransporter, pTscObj->id);
rpcClose(pTscObj->pTransporter);
pTscObj->pTransporter = NULL;
}
// TODO refactor
void* openTransporter(const char *user, const char *auth, int32_t numOfThread) {
SRpcInit rpcInit;
memset(&rpcInit, 0, sizeof(rpcInit));
rpcInit.localPort = 0;
rpcInit.label = "TSC";
rpcInit.numOfThreads = numOfThread;
rpcInit.cfp = processMsgFromServer;
rpcInit.sessions = tsMaxConnections;
rpcInit.connType = TAOS_CONN_CLIENT;
rpcInit.user = (char *)user;
rpcInit.idleTime = tsShellActivityTimer * 1000;
rpcInit.ckey = "key";
// rpcInit.spi = 1;
rpcInit.secret = (char *)auth;
void* pDnodeConn = rpcOpen(&rpcInit);
if (pDnodeConn == NULL) {
tscError("failed to init connection to server");
return NULL;
}
return pDnodeConn;
}
void destroyTscObj(void *pObj) {
STscObj *pTscObj = pObj;
atomic_sub_fetch_64(&pTscObj->pAppInfo->numOfConns, 1);
tscDebug("connObj 0x%"PRIx64" destroyed, totalConn:%"PRId64, pTscObj->id, pTscObj->pAppInfo->numOfConns);
pthread_mutex_destroy(&pTscObj->mutex);
tfree(pTscObj);
}
void* createTscObj(const char* user, const char* auth, const char *ip, uint32_t port, SAppInstInfo* pAppInfo) {
STscObj *pObj = (STscObj *)calloc(1, sizeof(STscObj));
if (NULL == pObj) {
terrno = TSDB_CODE_TSC_OUT_OF_MEMORY;
return NULL;
}
pObj->pAppInfo = pAppInfo;
if (pAppInfo != NULL) {
pObj->pTransporter = pAppInfo->pTransporter;
}
tstrncpy(pObj->user, user, sizeof(pObj->user));
memcpy(pObj->pass, auth, TSDB_PASSWORD_LEN);
pthread_mutex_init(&pObj->mutex, NULL);
pObj->id = taosAddRef(tscConnRef, pObj);
tscDebug("connObj created, 0x%"PRIx64, pObj->id);
return pObj;
}
void* createRequest(STscObj* pObj, __taos_async_fn_t fp, void* param, int32_t type) {
assert(pObj != NULL);
SRequestObj *pRequest = (SRequestObj *)calloc(1, sizeof(SRequestObj));
if (NULL == pRequest) {
terrno = TSDB_CODE_TSC_OUT_OF_MEMORY;
return NULL;
}
// TODO generated request uuid
pRequest->requestId = 0;
pRequest->metric.start = taosGetTimestampMs();
pRequest->type = type;
pRequest->pTscObj = pObj;
pRequest->body.fp = fp;
// pRequest->body.requestMsg. = param;
pRequest->msgBuf = calloc(1, ERROR_MSG_BUF_DEFAULT_SIZE);
tsem_init(&pRequest->body.rspSem, 0, 0);
registerRequest(pRequest);
return pRequest;
}
static void doDestroyRequest(void* p) {
assert(p != NULL);
SRequestObj* pRequest = (SRequestObj*)p;
assert(RID_VALID(pRequest->self));
tfree(pRequest->msgBuf);
tfree(pRequest->sqlstr);
tfree(pRequest->pInfo);
tfree(pRequest->body.resInfo.pRspMsg);
deregisterRequest(pRequest);
tfree(pRequest);
}
void destroyRequest(SRequestObj* pRequest) {
if (pRequest == NULL) {
return;
}
taosReleaseRef(tscReqRef, pRequest->self);
}
void taos_init_imp(void) {
// In the APIs of other program language, taos_cleanup is not available yet.
// So, to make sure taos_cleanup will be invoked to clean up the allocated resource to suppress the valgrind warning.
atexit(taos_cleanup);
errno = TSDB_CODE_SUCCESS;
srand(taosGetTimestampSec());
deltaToUtcInitOnce();
taosInitGlobalCfg();
taosReadCfgFromFile();
tscInitLogFile();
if (taosCheckAndPrintCfg()) {
tscInitRes = -1;
return;
}
taosInitNotes();
initMsgHandleFp();
rpcInit();
tscDebug("starting to initialize TAOS driver, local ep: %s", tsLocalEp);
taosSetCoreDump(true);
initTaskQueue();
tscConnRef = taosOpenRef(200, destroyTscObj);
tscReqRef = taosOpenRef(40960, doDestroyRequest);
taosGetAppName(appInfo.appName, NULL);
appInfo.pid = taosGetPId();
appInfo.startTime = taosGetTimestampMs();
appInfo.pInstMap = taosHashInit(4, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK);
tscDebug("client is initialized successfully");
}
int taos_init() {
pthread_once(&tscinit, taos_init_imp);
return tscInitRes;
}
int taos_options_imp(TSDB_OPTION option, const char *str) {
SGlobalCfg *cfg = NULL;
switch (option) {
case TSDB_OPTION_CONFIGDIR:
cfg = taosGetConfigOption("configDir");
assert(cfg != NULL);
if (cfg->cfgStatus <= TAOS_CFG_CSTATUS_OPTION) {
tstrncpy(configDir, str, TSDB_FILENAME_LEN);
cfg->cfgStatus = TAOS_CFG_CSTATUS_OPTION;
tscInfo("set config file directory:%s", str);
} else {
tscWarn("config option:%s, input value:%s, is configured by %s, use %s", cfg->option, str, tsCfgStatusStr[cfg->cfgStatus], (char *)cfg->ptr);
}
break;
case TSDB_OPTION_SHELL_ACTIVITY_TIMER:
cfg = taosGetConfigOption("shellActivityTimer");
assert(cfg != NULL);
if (cfg->cfgStatus <= TAOS_CFG_CSTATUS_OPTION) {
tsShellActivityTimer = atoi(str);
if (tsShellActivityTimer < 1) tsShellActivityTimer = 1;
if (tsShellActivityTimer > 3600) tsShellActivityTimer = 3600;
cfg->cfgStatus = TAOS_CFG_CSTATUS_OPTION;
tscInfo("set shellActivityTimer:%d", tsShellActivityTimer);
} else {
tscWarn("config option:%s, input value:%s, is configured by %s, use %d", cfg->option, str, tsCfgStatusStr[cfg->cfgStatus], *(int32_t *)cfg->ptr);
}
break;
case TSDB_OPTION_LOCALE: { // set locale
cfg = taosGetConfigOption("locale");
assert(cfg != NULL);
size_t len = strlen(str);
if (len == 0 || len > TSDB_LOCALE_LEN) {
tscInfo("Invalid locale:%s, use default", str);
return -1;
}
if (cfg->cfgStatus <= TAOS_CFG_CSTATUS_OPTION) {
char sep = '.';
if (strlen(tsLocale) == 0) { // locale does not set yet
char* defaultLocale = setlocale(LC_CTYPE, "");
// The locale of the current OS does not be set correctly, so the default locale cannot be acquired.
// The launch of current system will abort soon.
if (defaultLocale == NULL) {
tscError("failed to get default locale, please set the correct locale in current OS");
return -1;
}
tstrncpy(tsLocale, defaultLocale, TSDB_LOCALE_LEN);
}
// set the user specified locale
char *locale = setlocale(LC_CTYPE, str);
if (locale != NULL) { // failed to set the user specified locale
tscInfo("locale set, prev locale:%s, new locale:%s", tsLocale, locale);
cfg->cfgStatus = TAOS_CFG_CSTATUS_OPTION;
} else { // set the user specified locale failed, use default LC_CTYPE as current locale
locale = setlocale(LC_CTYPE, tsLocale);
tscInfo("failed to set locale:%s, current locale:%s", str, tsLocale);
}
tstrncpy(tsLocale, locale, TSDB_LOCALE_LEN);
char *charset = strrchr(tsLocale, sep);
if (charset != NULL) {
charset += 1;
charset = taosCharsetReplace(charset);
if (taosValidateEncodec(charset)) {
if (strlen(tsCharset) == 0) {
tscInfo("charset set:%s", charset);
} else {
tscInfo("charset changed from %s to %s", tsCharset, charset);
}
tstrncpy(tsCharset, charset, TSDB_LOCALE_LEN);
cfg->cfgStatus = TAOS_CFG_CSTATUS_OPTION;
} else {
tscInfo("charset:%s is not valid in locale, charset remains:%s", charset, tsCharset);
}
free(charset);
} else { // it may be windows system
tscInfo("charset remains:%s", tsCharset);
}
} else {
tscWarn("config option:%s, input value:%s, is configured by %s, use %s", cfg->option, str, tsCfgStatusStr[cfg->cfgStatus], (char *)cfg->ptr);
}
break;
}
case TSDB_OPTION_CHARSET: {
/* set charset will override the value of charset, assigned during system locale changed */
cfg = taosGetConfigOption("charset");
assert(cfg != NULL);
size_t len = strlen(str);
if (len == 0 || len > TSDB_LOCALE_LEN) {
tscInfo("failed to set charset:%s", str);
return -1;
}
if (cfg->cfgStatus <= TAOS_CFG_CSTATUS_OPTION) {
if (taosValidateEncodec(str)) {
if (strlen(tsCharset) == 0) {
tscInfo("charset is set:%s", str);
} else {
tscInfo("charset changed from %s to %s", tsCharset, str);
}
tstrncpy(tsCharset, str, TSDB_LOCALE_LEN);
cfg->cfgStatus = TAOS_CFG_CSTATUS_OPTION;
} else {
tscInfo("charset:%s not valid", str);
}
} else {
tscWarn("config option:%s, input value:%s, is configured by %s, use %s", cfg->option, str, tsCfgStatusStr[cfg->cfgStatus], (char *)cfg->ptr);
}
break;
}
case TSDB_OPTION_TIMEZONE:
cfg = taosGetConfigOption("timezone");
assert(cfg != NULL);
if (cfg->cfgStatus <= TAOS_CFG_CSTATUS_OPTION) {
tstrncpy(tsTimezone, str, TSDB_TIMEZONE_LEN);
tsSetTimeZone();
cfg->cfgStatus = TAOS_CFG_CSTATUS_OPTION;
tscDebug("timezone set:%s, input:%s by taos_options", tsTimezone, str);
} else {
tscWarn("config option:%s, input value:%s, is configured by %s, use %s", cfg->option, str, tsCfgStatusStr[cfg->cfgStatus], (char *)cfg->ptr);
}
break;
default:
// TODO return the correct error code to client in the format for taos_errstr()
tscError("Invalid option %d", option);
return -1;
}
return 0;
}
#if 0
#include "cJSON.h"
static setConfRet taos_set_config_imp(const char *config){
setConfRet ret = {SET_CONF_RET_SUCC, {0}};
static bool setConfFlag = false;
if (setConfFlag) {
ret.retCode = SET_CONF_RET_ERR_ONLY_ONCE;
strcpy(ret.retMsg, "configuration can only set once");
return ret;
}
taosInitGlobalCfg();
cJSON *root = cJSON_Parse(config);
if (root == NULL){
ret.retCode = SET_CONF_RET_ERR_JSON_PARSE;
strcpy(ret.retMsg, "parse json error");
return ret;
}
int size = cJSON_GetArraySize(root);
if(!cJSON_IsObject(root) || size == 0) {
ret.retCode = SET_CONF_RET_ERR_JSON_INVALID;
strcpy(ret.retMsg, "json content is invalid, must be not empty object");
return ret;
}
if(size >= 1000) {
ret.retCode = SET_CONF_RET_ERR_TOO_LONG;
strcpy(ret.retMsg, "json object size is too long");
return ret;
}
for(int i = 0; i < size; i++){
cJSON *item = cJSON_GetArrayItem(root, i);
if(!item) {
ret.retCode = SET_CONF_RET_ERR_INNER;
strcpy(ret.retMsg, "inner error");
return ret;
}
if(!taosReadConfigOption(item->string, item->valuestring, NULL, NULL, TAOS_CFG_CSTATUS_OPTION, TSDB_CFG_CTYPE_B_CLIENT)){
ret.retCode = SET_CONF_RET_ERR_PART;
if (strlen(ret.retMsg) == 0){
snprintf(ret.retMsg, RET_MSG_LENGTH, "part error|%s", item->string);
}else{
int tmp = RET_MSG_LENGTH - 1 - (int)strlen(ret.retMsg);
size_t leftSize = tmp >= 0 ? tmp : 0;
strncat(ret.retMsg, "|", leftSize);
tmp = RET_MSG_LENGTH - 1 - (int)strlen(ret.retMsg);
leftSize = tmp >= 0 ? tmp : 0;
strncat(ret.retMsg, item->string, leftSize);
}
}
}
cJSON_Delete(root);
setConfFlag = true;
return ret;
}
setConfRet taos_set_config(const char *config){
pthread_mutex_lock(&setConfMutex);
setConfRet ret = taos_set_config_imp(config);
pthread_mutex_unlock(&setConfMutex);
return ret;
}
#endif
\ No newline at end of file
......@@ -73,7 +73,6 @@ static void dndInitMsgFp(STransMgmt *pMgmt) {
pMgmt->msgFp[TSDB_MSG_TYPE_DROP_STB] = dndProcessMnodeWriteMsg;
pMgmt->msgFp[TSDB_MSG_TYPE_VGROUP_LIST] = dndProcessMnodeReadMsg;
pMgmt->msgFp[TSDB_MSG_TYPE_KILL_QUERY] = dndProcessMnodeWriteMsg;
pMgmt->msgFp[TSDB_MSG_TYPE_KILL_STREAM] = dndProcessMnodeWriteMsg;
pMgmt->msgFp[TSDB_MSG_TYPE_KILL_CONN] = dndProcessMnodeWriteMsg;
pMgmt->msgFp[TSDB_MSG_TYPE_HEARTBEAT] = dndProcessMnodeReadMsg;
pMgmt->msgFp[TSDB_MSG_TYPE_SHOW] = dndProcessMnodeReadMsg;
......
......@@ -8,8 +8,8 @@ add_subdirectory(db)
add_subdirectory(dnode)
# add_subdirectory(func)
# add_subdirectory(mnode)
# add_subdirectory(profile)
# add_subdirectory(show)
add_subdirectory(profile)
add_subdirectory(show)
add_subdirectory(stb)
# add_subdirectory(sync)
# add_subdirectory(telem)
......@@ -17,4 +17,4 @@ add_subdirectory(stb)
add_subdirectory(user)
add_subdirectory(vgroup)
# add_subdirectory(common)
add_subdirectory(sut)
add_executable(dnode_test_acct "")
target_sources(dnode_test_acct
PRIVATE
"acct.cpp"
"../sut/deploy.cpp"
)
aux_source_directory(. ACCT_SRC)
add_executable(dnode_test_acct ${ACCT_SRC})
target_link_libraries(
dnode_test_acct
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(dnode_test_acct
PUBLIC
"${CMAKE_SOURCE_DIR}/include/server/dnode/mgmt"
"${CMAKE_CURRENT_SOURCE_DIR}/../../inc"
"${CMAKE_CURRENT_SOURCE_DIR}/../sut"
PUBLIC sut
)
add_test(
......
......@@ -9,103 +9,59 @@
*
*/
#include "deploy.h"
#include "base.h"
class DndTestAcct : public ::testing::Test {
protected:
static SServer* CreateServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
SServer* pServer = createServer(path, fqdn, port, firstEp);
ASSERT(pServer);
return pServer;
}
static void SetUpTestSuite() {
initLog("/tmp/tdlog");
const char* fqdn = "localhost";
const char* firstEp = "localhost:9012";
pServer = CreateServer("/tmp/dnode_test_user", fqdn, 9012, firstEp);
pClient = createClient("root", "taosdata", fqdn, 9012);
taosMsleep(300);
}
static void TearDownTestSuite() {
stopServer(pServer);
dropClient(pClient);
pServer = NULL;
pClient = NULL;
}
static SServer* pServer;
static SClient* pClient;
static int32_t connId;
static void SetUpTestSuite() { test.Init("/tmp/dnode_test_acct", 9012); }
static void TearDownTestSuite() { test.Cleanup(); }
static Testbase test;
public:
void SetUp() override {}
void TearDown() override {}
};
SServer* DndTestAcct::pServer;
SClient* DndTestAcct::pClient;
int32_t DndTestAcct::connId;
Testbase DndTestAcct::test;
TEST_F(DndTestAcct, 01_CreateAcct) {
ASSERT_NE(pClient, nullptr);
SCreateAcctMsg* pReq = (SCreateAcctMsg*)rpcMallocCont(sizeof(SCreateAcctMsg));
int32_t contLen = sizeof(SCreateAcctMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCreateAcctMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_ACCT;
SCreateAcctMsg* pReq = (SCreateAcctMsg*)rpcMallocCont(contLen);
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_ACCT, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, TSDB_CODE_MND_MSG_NOT_PROCESSED);
}
TEST_F(DndTestAcct, 02_AlterAcct) {
ASSERT_NE(pClient, nullptr);
int32_t contLen = sizeof(SCreateAcctMsg);
SAlterAcctMsg* pReq = (SAlterAcctMsg*)rpcMallocCont(sizeof(SAlterAcctMsg));
SAlterAcctMsg* pReq = (SAlterAcctMsg*)rpcMallocCont(contLen);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SAlterAcctMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_ALTER_ACCT;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_ALTER_ACCT, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, TSDB_CODE_MND_MSG_NOT_PROCESSED);
}
TEST_F(DndTestAcct, 03_DropAcct) {
ASSERT_NE(pClient, nullptr);
SDropAcctMsg* pReq = (SDropAcctMsg*)rpcMallocCont(sizeof(SDropAcctMsg));
int32_t contLen = sizeof(SDropAcctMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SDropAcctMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_DROP_ACCT;
SDropAcctMsg* pReq = (SDropAcctMsg*)rpcMallocCont(contLen);
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_DROP_ACCT, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, TSDB_CODE_MND_MSG_NOT_PROCESSED);
}
TEST_F(DndTestAcct, 04_ShowAcct) {
ASSERT_NE(pClient, nullptr);
int32_t contLen = sizeof(SShowMsg);
SShowMsg* pReq = (SShowMsg*)rpcMallocCont(sizeof(SShowMsg));
SShowMsg* pReq = (SShowMsg*)rpcMallocCont(contLen);
pReq->type = TSDB_MGMT_TABLE_ACCT;
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SShowMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_SHOW;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_SHOW, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, TSDB_CODE_MND_INVALID_MSG_TYPE);
}
\ No newline at end of file
add_executable(dnode_test_cluster "")
target_sources(dnode_test_cluster
PRIVATE
"cluster.cpp"
"../sut/deploy.cpp"
)
aux_source_directory(. CLUSTER_SRC)
add_executable(dnode_test_cluster ${CLUSTER_SRC})
target_link_libraries(
dnode_test_cluster
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(dnode_test_cluster
PUBLIC
"${CMAKE_SOURCE_DIR}/include/server/dnode/mgmt"
"${CMAKE_CURRENT_SOURCE_DIR}/../../inc"
"${CMAKE_CURRENT_SOURCE_DIR}/../sut"
PUBLIC sut
)
add_test(
......
......@@ -9,162 +9,33 @@
*
*/
#include "deploy.h"
#include "base.h"
class DndTestCluster : public ::testing::Test {
protected:
static SServer* CreateServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
SServer* pServer = createServer(path, fqdn, port, firstEp);
ASSERT(pServer);
return pServer;
}
static void SetUpTestSuite() { test.Init("/tmp/dnode_test_cluster", 9030); }
static void TearDownTestSuite() { test.Cleanup(); }
static void SetUpTestSuite() {
initLog("/tmp/tdlog");
const char* fqdn = "localhost";
const char* firstEp = "localhost:9030";
pServer = CreateServer("/tmp/dnode_test_cluster", fqdn, 9030, firstEp);
pClient = createClient("root", "taosdata", fqdn, 9030);
taosMsleep(1100);
}
static void TearDownTestSuite() {
stopServer(pServer);
dropClient(pClient);
pServer = NULL;
pClient = NULL;
}
static SServer* pServer;
static SClient* pClient;
static int32_t connId;
static Testbase test;
public:
void SetUp() override {}
void TearDown() override {}
void SendTheCheckShowMetaMsg(int8_t showType, const char* showName, int32_t columns, const char* db) {
SShowMsg* pShow = (SShowMsg*)rpcMallocCont(sizeof(SShowMsg));
pShow->type = showType;
if (db != NULL) {
strcpy(pShow->db, db);
}
SRpcMsg showRpcMsg = {0};
showRpcMsg.pCont = pShow;
showRpcMsg.contLen = sizeof(SShowMsg);
showRpcMsg.msgType = TSDB_MSG_TYPE_SHOW;
sendMsg(pClient, &showRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
SShowRsp* pShowRsp = (SShowRsp*)pClient->pRsp->pCont;
ASSERT_NE(pShowRsp, nullptr);
pShowRsp->showId = htonl(pShowRsp->showId);
pMeta = &pShowRsp->tableMeta;
pMeta->numOfTags = htonl(pMeta->numOfTags);
pMeta->numOfColumns = htonl(pMeta->numOfColumns);
pMeta->sversion = htonl(pMeta->sversion);
pMeta->tversion = htonl(pMeta->tversion);
pMeta->tuid = htobe64(pMeta->tuid);
pMeta->suid = htobe64(pMeta->suid);
showId = pShowRsp->showId;
EXPECT_NE(pShowRsp->showId, 0);
EXPECT_STREQ(pMeta->tbFname, showName);
EXPECT_EQ(pMeta->numOfTags, 0);
EXPECT_EQ(pMeta->numOfColumns, columns);
EXPECT_EQ(pMeta->precision, 0);
EXPECT_EQ(pMeta->tableType, 0);
EXPECT_EQ(pMeta->update, 0);
EXPECT_EQ(pMeta->sversion, 0);
EXPECT_EQ(pMeta->tversion, 0);
EXPECT_EQ(pMeta->tuid, 0);
EXPECT_EQ(pMeta->suid, 0);
}
void CheckSchema(int32_t index, int8_t type, int32_t bytes, const char* name) {
SSchema* pSchema = &pMeta->pSchema[index];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, type);
EXPECT_EQ(pSchema->bytes, bytes);
EXPECT_STREQ(pSchema->name, name);
}
void SendThenCheckShowRetrieveMsg(int32_t rows) {
SRetrieveTableMsg* pRetrieve = (SRetrieveTableMsg*)rpcMallocCont(sizeof(SRetrieveTableMsg));
pRetrieve->showId = htonl(showId);
pRetrieve->free = 0;
SRpcMsg retrieveRpcMsg = {0};
retrieveRpcMsg.pCont = pRetrieve;
retrieveRpcMsg.contLen = sizeof(SRetrieveTableMsg);
retrieveRpcMsg.msgType = TSDB_MSG_TYPE_SHOW_RETRIEVE;
sendMsg(pClient, &retrieveRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
pRetrieveRsp = (SRetrieveTableRsp*)pClient->pRsp->pCont;
ASSERT_NE(pRetrieveRsp, nullptr);
pRetrieveRsp->numOfRows = htonl(pRetrieveRsp->numOfRows);
pRetrieveRsp->useconds = htobe64(pRetrieveRsp->useconds);
pRetrieveRsp->compLen = htonl(pRetrieveRsp->compLen);
EXPECT_EQ(pRetrieveRsp->numOfRows, rows);
EXPECT_EQ(pRetrieveRsp->useconds, 0);
// EXPECT_EQ(pRetrieveRsp->completed, completed);
EXPECT_EQ(pRetrieveRsp->precision, TSDB_TIME_PRECISION_MILLI);
EXPECT_EQ(pRetrieveRsp->compressed, 0);
EXPECT_EQ(pRetrieveRsp->compLen, 0);
pData = pRetrieveRsp->data;
pos = 0;
}
void CheckInt32() {
int32_t data = *((int32_t*)(pData + pos));
pos += sizeof(int32_t);
EXPECT_GT(data, 0);
}
void CheckTimestamp() {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
EXPECT_GT(data, 0);
}
void CheckBinary(int32_t len) {
pos += sizeof(VarDataLenT);
char* data = (char*)(pData + pos);
pos += len;
}
int32_t showId;
STableMetaMsg* pMeta;
SRetrieveTableRsp* pRetrieveRsp;
char* pData;
int32_t pos;
};
SServer* DndTestCluster::pServer;
SClient* DndTestCluster::pClient;
int32_t DndTestCluster::connId;
Testbase DndTestCluster::test;
TEST_F(DndTestCluster, 01_ShowCluster) {
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_CLUSTER, "show cluster", 3, NULL);
CheckSchema(0, TSDB_DATA_TYPE_INT, 4, "id");
CheckSchema(1, TSDB_DATA_TYPE_BINARY, TSDB_CLUSTER_ID_LEN + VARSTR_HEADER_SIZE, "name");
CheckSchema(2, TSDB_DATA_TYPE_TIMESTAMP, 8, "create_time");
test.SendShowMetaMsg(TSDB_MGMT_TABLE_CLUSTER, "");
CHECK_META( "show cluster", 3);
CHECK_SCHEMA(0, TSDB_DATA_TYPE_INT, 4, "id");
CHECK_SCHEMA(1, TSDB_DATA_TYPE_BINARY, TSDB_CLUSTER_ID_LEN + VARSTR_HEADER_SIZE, "name");
CHECK_SCHEMA(2, TSDB_DATA_TYPE_TIMESTAMP, 8, "create_time");
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 1);
SendThenCheckShowRetrieveMsg(1);
CheckInt32();
CheckBinary(TSDB_CLUSTER_ID_LEN);
IgnoreInt32();
IgnoreBinary(TSDB_CLUSTER_ID_LEN);
CheckTimestamp();
}
\ No newline at end of file
add_executable(dnode_test_db "")
target_sources(dnode_test_db
PRIVATE
"db.cpp"
"../sut/deploy.cpp"
)
aux_source_directory(. DB_SRC)
add_executable(dnode_test_db ${DB_SRC})
target_link_libraries(
dnode_test_db
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(dnode_test_db
PUBLIC
"${CMAKE_SOURCE_DIR}/include/server/dnode/mgmt"
"${CMAKE_CURRENT_SOURCE_DIR}/../../inc"
"${CMAKE_CURRENT_SOURCE_DIR}/../sut"
PUBLIC sut
)
add_test(
......
......@@ -9,199 +9,52 @@
*
*/
#include "deploy.h"
#include "base.h"
class DndTestDb : public ::testing::Test {
protected:
static SServer* CreateServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
SServer* pServer = createServer(path, fqdn, port, firstEp);
ASSERT(pServer);
return pServer;
}
static void SetUpTestSuite() {
initLog("/tmp/tdlog");
const char* fqdn = "localhost";
const char* firstEp = "localhost:9040";
pServer = CreateServer("/tmp/dnode_test_db", fqdn, 9040, firstEp);
pClient = createClient("root", "taosdata", fqdn, 9040);
taosMsleep(1100);
}
static void TearDownTestSuite() {
stopServer(pServer);
dropClient(pClient);
pServer = NULL;
pClient = NULL;
}
static void SetUpTestSuite() { test.Init("/tmp/dnode_test_db", 9040); }
static void TearDownTestSuite() { test.Cleanup(); }
static SServer* pServer;
static SClient* pClient;
static int32_t connId;
static Testbase test;
public:
void SetUp() override {}
void TearDown() override {}
void SendTheCheckShowMetaMsg(int8_t showType, const char* showName, int32_t columns, const char* db) {
SShowMsg* pShow = (SShowMsg*)rpcMallocCont(sizeof(SShowMsg));
pShow->type = showType;
if (db != NULL) {
strcpy(pShow->db, db);
}
SRpcMsg showRpcMsg = {0};
showRpcMsg.pCont = pShow;
showRpcMsg.contLen = sizeof(SShowMsg);
showRpcMsg.msgType = TSDB_MSG_TYPE_SHOW;
sendMsg(pClient, &showRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
SShowRsp* pShowRsp = (SShowRsp*)pClient->pRsp->pCont;
ASSERT_NE(pShowRsp, nullptr);
pShowRsp->showId = htonl(pShowRsp->showId);
pMeta = &pShowRsp->tableMeta;
pMeta->numOfTags = htonl(pMeta->numOfTags);
pMeta->numOfColumns = htonl(pMeta->numOfColumns);
pMeta->sversion = htonl(pMeta->sversion);
pMeta->tversion = htonl(pMeta->tversion);
pMeta->tuid = htobe64(pMeta->tuid);
pMeta->suid = htobe64(pMeta->suid);
showId = pShowRsp->showId;
EXPECT_NE(pShowRsp->showId, 0);
EXPECT_STREQ(pMeta->tbFname, showName);
EXPECT_EQ(pMeta->numOfTags, 0);
EXPECT_EQ(pMeta->numOfColumns, columns);
EXPECT_EQ(pMeta->precision, 0);
EXPECT_EQ(pMeta->tableType, 0);
EXPECT_EQ(pMeta->update, 0);
EXPECT_EQ(pMeta->sversion, 0);
EXPECT_EQ(pMeta->tversion, 0);
EXPECT_EQ(pMeta->tuid, 0);
EXPECT_EQ(pMeta->suid, 0);
}
void CheckSchema(int32_t index, int8_t type, int32_t bytes, const char* name) {
SSchema* pSchema = &pMeta->pSchema[index];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, type);
EXPECT_EQ(pSchema->bytes, bytes);
EXPECT_STREQ(pSchema->name, name);
}
void SendThenCheckShowRetrieveMsg(int32_t rows) {
SRetrieveTableMsg* pRetrieve = (SRetrieveTableMsg*)rpcMallocCont(sizeof(SRetrieveTableMsg));
pRetrieve->showId = htonl(showId);
pRetrieve->free = 0;
SRpcMsg retrieveRpcMsg = {0};
retrieveRpcMsg.pCont = pRetrieve;
retrieveRpcMsg.contLen = sizeof(SRetrieveTableMsg);
retrieveRpcMsg.msgType = TSDB_MSG_TYPE_SHOW_RETRIEVE;
sendMsg(pClient, &retrieveRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
pRetrieveRsp = (SRetrieveTableRsp*)pClient->pRsp->pCont;
ASSERT_NE(pRetrieveRsp, nullptr);
pRetrieveRsp->numOfRows = htonl(pRetrieveRsp->numOfRows);
pRetrieveRsp->useconds = htobe64(pRetrieveRsp->useconds);
pRetrieveRsp->compLen = htonl(pRetrieveRsp->compLen);
EXPECT_EQ(pRetrieveRsp->numOfRows, rows);
EXPECT_EQ(pRetrieveRsp->useconds, 0);
// EXPECT_EQ(pRetrieveRsp->completed, completed);
EXPECT_EQ(pRetrieveRsp->precision, TSDB_TIME_PRECISION_MILLI);
EXPECT_EQ(pRetrieveRsp->compressed, 0);
EXPECT_EQ(pRetrieveRsp->compLen, 0);
pData = pRetrieveRsp->data;
pos = 0;
}
void CheckInt8(int8_t val) {
int8_t data = *((int8_t*)(pData + pos));
pos += sizeof(int8_t);
EXPECT_EQ(data, val);
}
void CheckInt16(int16_t val) {
int16_t data = *((int16_t*)(pData + pos));
pos += sizeof(int16_t);
EXPECT_EQ(data, val);
}
void CheckInt32(int32_t val) {
int32_t data = *((int32_t*)(pData + pos));
pos += sizeof(int32_t);
EXPECT_EQ(data, val);
}
void CheckInt64(int64_t val) {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
EXPECT_EQ(data, val);
}
void CheckTimestamp() {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
EXPECT_GT(data, 0);
}
void CheckBinary(const char* val, int32_t len) {
pos += sizeof(VarDataLenT);
char* data = (char*)(pData + pos);
pos += len;
EXPECT_STREQ(data, val);
}
int32_t showId;
STableMetaMsg* pMeta;
SRetrieveTableRsp* pRetrieveRsp;
char* pData;
int32_t pos;
};
SServer* DndTestDb::pServer;
SClient* DndTestDb::pClient;
int32_t DndTestDb::connId;
Testbase DndTestDb::test;
TEST_F(DndTestDb, 01_ShowDb) {
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DB, "show databases", 17, NULL);
CheckSchema(0, TSDB_DATA_TYPE_BINARY, TSDB_DB_NAME_LEN - 1 + VARSTR_HEADER_SIZE, "name");
CheckSchema(1, TSDB_DATA_TYPE_TIMESTAMP, 8, "create_time");
CheckSchema(2, TSDB_DATA_TYPE_SMALLINT, 2, "vgroups");
CheckSchema(3, TSDB_DATA_TYPE_SMALLINT, 2, "replica");
CheckSchema(4, TSDB_DATA_TYPE_SMALLINT, 2, "quorum");
CheckSchema(5, TSDB_DATA_TYPE_SMALLINT, 2, "days");
CheckSchema(6, TSDB_DATA_TYPE_BINARY, 24 + VARSTR_HEADER_SIZE, "keep0,keep1,keep2");
CheckSchema(7, TSDB_DATA_TYPE_INT, 4, "cache");
CheckSchema(8, TSDB_DATA_TYPE_INT, 4, "blocks");
CheckSchema(9, TSDB_DATA_TYPE_INT, 4, "minrows");
CheckSchema(10, TSDB_DATA_TYPE_INT, 4, "maxrows");
CheckSchema(11, TSDB_DATA_TYPE_TINYINT, 1, "wallevel");
CheckSchema(12, TSDB_DATA_TYPE_INT, 4, "fsync");
CheckSchema(13, TSDB_DATA_TYPE_TINYINT, 1, "comp");
CheckSchema(14, TSDB_DATA_TYPE_TINYINT, 1, "cachelast");
CheckSchema(15, TSDB_DATA_TYPE_BINARY, 3 + VARSTR_HEADER_SIZE, "precision");
CheckSchema(16, TSDB_DATA_TYPE_TINYINT, 1, "update");
SendThenCheckShowRetrieveMsg(0);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DB, "");
CHECK_META("show databases", 17);
CHECK_SCHEMA(0, TSDB_DATA_TYPE_BINARY, TSDB_DB_NAME_LEN - 1 + VARSTR_HEADER_SIZE, "name");
CHECK_SCHEMA(1, TSDB_DATA_TYPE_TIMESTAMP, 8, "create_time");
CHECK_SCHEMA(2, TSDB_DATA_TYPE_SMALLINT, 2, "vgroups");
CHECK_SCHEMA(3, TSDB_DATA_TYPE_SMALLINT, 2, "replica");
CHECK_SCHEMA(4, TSDB_DATA_TYPE_SMALLINT, 2, "quorum");
CHECK_SCHEMA(5, TSDB_DATA_TYPE_SMALLINT, 2, "days");
CHECK_SCHEMA(6, TSDB_DATA_TYPE_BINARY, 24 + VARSTR_HEADER_SIZE, "keep0,keep1,keep2");
CHECK_SCHEMA(7, TSDB_DATA_TYPE_INT, 4, "cache");
CHECK_SCHEMA(8, TSDB_DATA_TYPE_INT, 4, "blocks");
CHECK_SCHEMA(9, TSDB_DATA_TYPE_INT, 4, "minrows");
CHECK_SCHEMA(10, TSDB_DATA_TYPE_INT, 4, "maxrows");
CHECK_SCHEMA(11, TSDB_DATA_TYPE_TINYINT, 1, "wallevel");
CHECK_SCHEMA(12, TSDB_DATA_TYPE_INT, 4, "fsync");
CHECK_SCHEMA(13, TSDB_DATA_TYPE_TINYINT, 1, "comp");
CHECK_SCHEMA(14, TSDB_DATA_TYPE_TINYINT, 1, "cachelast");
CHECK_SCHEMA(15, TSDB_DATA_TYPE_BINARY, 3 + VARSTR_HEADER_SIZE, "precision");
CHECK_SCHEMA(16, TSDB_DATA_TYPE_TINYINT, 1, "update");
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 0);
}
TEST_F(DndTestDb, 02_Create_Alter_Drop_Db) {
{
SCreateDbMsg* pReq = (SCreateDbMsg*)rpcMallocCont(sizeof(SCreateDbMsg));
int32_t contLen = sizeof(SCreateDbMsg);
SCreateDbMsg* pReq = (SCreateDbMsg*)rpcMallocCont(contLen);
strcpy(pReq->db, "1.d1");
pReq->numOfVgroups = htonl(2);
pReq->cacheBlockSize = htonl(16);
......@@ -223,20 +76,16 @@ TEST_F(DndTestDb, 02_Create_Alter_Drop_Db) {
pReq->cacheLastRow = 0;
pReq->ignoreExist = 1;
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCreateDbMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_DB;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_DB, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
// taosMsleep(1000000);
}
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DB, "show databases", 17, NULL);
SendThenCheckShowRetrieveMsg(1);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DB, "");
CHECK_META("show databases", 17);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 1);
CheckBinary("d1", TSDB_DB_NAME_LEN - 1);
CheckTimestamp();
CheckInt16(2); // vgroups
......@@ -255,12 +104,15 @@ TEST_F(DndTestDb, 02_Create_Alter_Drop_Db) {
CheckBinary("ms", 3); // precision
CheckInt8(0); // update
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_VGROUP, "show vgroups", 4, "1.d1");
CheckSchema(0, TSDB_DATA_TYPE_INT, 4, "vgId");
CheckSchema(1, TSDB_DATA_TYPE_INT, 4, "tables");
CheckSchema(2, TSDB_DATA_TYPE_SMALLINT, 2, "v1_dnode");
CheckSchema(3, TSDB_DATA_TYPE_BINARY, 9 + VARSTR_HEADER_SIZE, "v1_status");
SendThenCheckShowRetrieveMsg(2);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_VGROUP, "1.d1");
CHECK_META("show vgroups", 4);
CHECK_SCHEMA(0, TSDB_DATA_TYPE_INT, 4, "vgId");
CHECK_SCHEMA(1, TSDB_DATA_TYPE_INT, 4, "tables");
CHECK_SCHEMA(2, TSDB_DATA_TYPE_SMALLINT, 2, "v1_dnode");
CHECK_SCHEMA(3, TSDB_DATA_TYPE_BINARY, 9 + VARSTR_HEADER_SIZE, "v1_status");
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 2);
CheckInt32(1);
CheckInt32(2);
CheckInt32(0);
......@@ -271,7 +123,9 @@ TEST_F(DndTestDb, 02_Create_Alter_Drop_Db) {
CheckBinary("master", 9);
{
SAlterDbMsg* pReq = (SAlterDbMsg*)rpcMallocCont(sizeof(SAlterDbMsg));
int32_t contLen = sizeof(SAlterDbMsg);
SAlterDbMsg* pReq = (SAlterDbMsg*)rpcMallocCont(contLen);
strcpy(pReq->db, "1.d1");
pReq->totalBlocks = htonl(12);
pReq->daysToKeep0 = htonl(300);
......@@ -282,19 +136,14 @@ TEST_F(DndTestDb, 02_Create_Alter_Drop_Db) {
pReq->quorum = 2;
pReq->cacheLastRow = 1;
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SAlterDbMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_ALTER_DB;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_ALTER_DB, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DB, "show databases", 17, NULL);
SendThenCheckShowRetrieveMsg(1);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DB, "");
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 1);
CheckBinary("d1", TSDB_DB_NAME_LEN - 1);
CheckTimestamp();
CheckInt16(2); // vgroups
......@@ -314,19 +163,14 @@ TEST_F(DndTestDb, 02_Create_Alter_Drop_Db) {
CheckInt8(0); // update
// restart
stopServer(pServer);
pServer = NULL;
uInfo("start all server");
test.Restart();
const char* fqdn = "localhost";
const char* firstEp = "localhost:9040";
pServer = startServer("/tmp/dnode_test_db", fqdn, 9040, firstEp);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DB, "");
CHECK_META("show databases", 17);
uInfo("all server is running");
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 1);
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DB, "show databases", 17, NULL);
SendThenCheckShowRetrieveMsg(1);
CheckBinary("d1", TSDB_DB_NAME_LEN - 1);
CheckTimestamp();
CheckInt16(2); // vgroups
......@@ -346,27 +190,28 @@ TEST_F(DndTestDb, 02_Create_Alter_Drop_Db) {
CheckInt8(0); // update
{
SDropDbMsg* pReq = (SDropDbMsg*)rpcMallocCont(sizeof(SDropDbMsg));
strcpy(pReq->db, "1.d1");
int32_t contLen = sizeof(SDropDbMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SDropDbMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_DROP_DB;
SDropDbMsg* pReq = (SDropDbMsg*)rpcMallocCont(contLen);
strcpy(pReq->db, "1.d1");
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_DROP_DB, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DB, "show databases", 17, NULL);
SendThenCheckShowRetrieveMsg(0);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DB, "");
CHECK_META("show databases", 17);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 0);
}
TEST_F(DndTestDb, 03_Create_Use_Restart_Use_Db) {
{
SCreateDbMsg* pReq = (SCreateDbMsg*)rpcMallocCont(sizeof(SCreateDbMsg));
int32_t contLen = sizeof(SCreateDbMsg);
SCreateDbMsg* pReq = (SCreateDbMsg*)rpcMallocCont(contLen);
strcpy(pReq->db, "1.d2");
pReq->numOfVgroups = htonl(2);
pReq->cacheBlockSize = htonl(16);
......@@ -388,33 +233,26 @@ TEST_F(DndTestDb, 03_Create_Use_Restart_Use_Db) {
pReq->cacheLastRow = 0;
pReq->ignoreExist = 1;
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCreateDbMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_DB;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_DB, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DB, "show databases", 17, NULL);
SendThenCheckShowRetrieveMsg(1);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DB, "");
CHECK_META("show databases", 17);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 1);
CheckBinary("d2", TSDB_DB_NAME_LEN - 1);
{
SUseDbMsg* pReq = (SUseDbMsg*)rpcMallocCont(sizeof(SUseDbMsg));
int32_t contLen = sizeof(SUseDbMsg);
SUseDbMsg* pReq = (SUseDbMsg*)rpcMallocCont(contLen);
strcpy(pReq->db, "1.d2");
pReq->vgVersion = htonl(-1);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SUseDbMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_USE_DB;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_USE_DB, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
......
add_executable(dnode_test_dnode "")
target_sources(dnode_test_dnode
PRIVATE
"dnode.cpp"
"../sut/deploy.cpp"
)
aux_source_directory(. DTEST_SRC)
add_executable(dnode_test_dnode ${DTEST_SRC})
target_link_libraries(
dnode_test_dnode
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(dnode_test_dnode
PUBLIC
"${CMAKE_SOURCE_DIR}/include/server/dnode/mgmt"
"${CMAKE_CURRENT_SOURCE_DIR}/../../inc"
"${CMAKE_CURRENT_SOURCE_DIR}/../sut"
PUBLIC sut
)
add_test(
......
......@@ -9,189 +9,62 @@
*
*/
#include "deploy.h"
#include "base.h"
class DndTestDnode : public ::testing::Test {
public:
static SServer* CreateServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
SServer* pServer = createServer(path, fqdn, port, firstEp);
ASSERT(pServer);
return pServer;
}
void SetUp() override {}
void TearDown() override {}
public:
static void SetUpTestSuite() {
initLog("/tmp/tdlog");
test.Init("/tmp/dnode_test_dnode1", 9041);
const char* fqdn = "localhost";
const char* firstEp = "localhost:9041";
pServer1 = CreateServer("/tmp/dnode_test_dnode1", fqdn, 9041, firstEp);
pServer2 = CreateServer("/tmp/dnode_test_dnode2", fqdn, 9042, firstEp);
pServer3 = CreateServer("/tmp/dnode_test_dnode3", fqdn, 9043, firstEp);
pServer4 = CreateServer("/tmp/dnode_test_dnode4", fqdn, 9044, firstEp);
pServer5 = CreateServer("/tmp/dnode_test_dnode5", fqdn, 9045, firstEp);
pClient = createClient("root", "taosdata", fqdn, 9041);
server2.Start("/tmp/dnode_test_dnode2", fqdn, 9042, firstEp);
server3.Start("/tmp/dnode_test_dnode3", fqdn, 9043, firstEp);
server4.Start("/tmp/dnode_test_dnode4", fqdn, 9044, firstEp);
server5.Start("/tmp/dnode_test_dnode5", fqdn, 9045, firstEp);
taosMsleep(300);
}
static void TearDownTestSuite() {
stopServer(pServer1);
stopServer(pServer2);
stopServer(pServer3);
stopServer(pServer4);
stopServer(pServer5);
dropClient(pClient);
pServer1 = NULL;
pServer2 = NULL;
pServer3 = NULL;
pServer4 = NULL;
pServer5 = NULL;
pClient = NULL;
}
static SServer* pServer1;
static SServer* pServer2;
static SServer* pServer3;
static SServer* pServer4;
static SServer* pServer5;
static SClient* pClient;
public:
void SetUp() override {}
void TearDown() override {}
void SendTheCheckShowMetaMsg(int8_t showType, const char* showName, int32_t columns) {
SShowMsg* pShow = (SShowMsg*)rpcMallocCont(sizeof(SShowMsg));
pShow->type = showType;
strcpy(pShow->db, "");
SRpcMsg showRpcMsg = {0};
showRpcMsg.pCont = pShow;
showRpcMsg.contLen = sizeof(SShowMsg);
showRpcMsg.msgType = TSDB_MSG_TYPE_SHOW;
sendMsg(pClient, &showRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
SShowRsp* pShowRsp = (SShowRsp*)pClient->pRsp->pCont;
ASSERT_NE(pShowRsp, nullptr);
pShowRsp->showId = htonl(pShowRsp->showId);
pMeta = &pShowRsp->tableMeta;
pMeta->numOfTags = htonl(pMeta->numOfTags);
pMeta->numOfColumns = htonl(pMeta->numOfColumns);
pMeta->sversion = htonl(pMeta->sversion);
pMeta->tversion = htonl(pMeta->tversion);
pMeta->tuid = htobe64(pMeta->tuid);
pMeta->suid = htobe64(pMeta->suid);
showId = pShowRsp->showId;
EXPECT_NE(pShowRsp->showId, 0);
EXPECT_STREQ(pMeta->tbFname, showName);
EXPECT_EQ(pMeta->numOfTags, 0);
EXPECT_EQ(pMeta->numOfColumns, columns);
EXPECT_EQ(pMeta->precision, 0);
EXPECT_EQ(pMeta->tableType, 0);
EXPECT_EQ(pMeta->update, 0);
EXPECT_EQ(pMeta->sversion, 0);
EXPECT_EQ(pMeta->tversion, 0);
EXPECT_EQ(pMeta->tuid, 0);
EXPECT_EQ(pMeta->suid, 0);
}
void CheckSchema(int32_t index, int8_t type, int32_t bytes, const char* name) {
SSchema* pSchema = &pMeta->pSchema[index];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, type);
EXPECT_EQ(pSchema->bytes, bytes);
EXPECT_STREQ(pSchema->name, name);
}
void SendThenCheckShowRetrieveMsg(int32_t rows) {
SRetrieveTableMsg* pRetrieve = (SRetrieveTableMsg*)rpcMallocCont(sizeof(SRetrieveTableMsg));
pRetrieve->showId = htonl(showId);
pRetrieve->free = 0;
SRpcMsg retrieveRpcMsg = {0};
retrieveRpcMsg.pCont = pRetrieve;
retrieveRpcMsg.contLen = sizeof(SRetrieveTableMsg);
retrieveRpcMsg.msgType = TSDB_MSG_TYPE_SHOW_RETRIEVE;
sendMsg(pClient, &retrieveRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
pRetrieveRsp = (SRetrieveTableRsp*)pClient->pRsp->pCont;
ASSERT_NE(pRetrieveRsp, nullptr);
pRetrieveRsp->numOfRows = htonl(pRetrieveRsp->numOfRows);
pRetrieveRsp->useconds = htobe64(pRetrieveRsp->useconds);
pRetrieveRsp->compLen = htonl(pRetrieveRsp->compLen);
EXPECT_EQ(pRetrieveRsp->numOfRows, rows);
EXPECT_EQ(pRetrieveRsp->useconds, 0);
// EXPECT_EQ(pRetrieveRsp->completed, completed);
EXPECT_EQ(pRetrieveRsp->precision, TSDB_TIME_PRECISION_MILLI);
EXPECT_EQ(pRetrieveRsp->compressed, 0);
EXPECT_EQ(pRetrieveRsp->compLen, 0);
pData = pRetrieveRsp->data;
pos = 0;
server2.Stop();
server3.Stop();
server4.Stop();
server5.Stop();
test.Cleanup();
}
void CheckInt16(int16_t val) {
int16_t data = *((int16_t*)(pData + pos));
pos += sizeof(int16_t);
EXPECT_EQ(data, val);
}
void CheckInt64(int64_t val) {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
EXPECT_EQ(data, val);
}
static Testbase test;
static TestServer server2;
static TestServer server3;
static TestServer server4;
static TestServer server5;
};
void CheckTimestamp() {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
EXPECT_GT(data, 0);
}
Testbase DndTestDnode::test;
TestServer DndTestDnode::server2;
TestServer DndTestDnode::server3;
TestServer DndTestDnode::server4;
TestServer DndTestDnode::server5;
void CheckBinary(const char* val, int32_t len) {
pos += sizeof(VarDataLenT);
char* data = (char*)(pData + pos);
pos += len;
EXPECT_STREQ(data, val);
}
TEST_F(DndTestDnode, 01_ShowDnode) {
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "");
CHECK_META("show dnodes", 7);
int32_t showId;
STableMetaMsg* pMeta;
SRetrieveTableRsp* pRetrieveRsp;
char* pData;
int32_t pos;
};
CHECK_SCHEMA(0, TSDB_DATA_TYPE_SMALLINT, 2, "id");
CHECK_SCHEMA(1, TSDB_DATA_TYPE_BINARY, TSDB_EP_LEN + VARSTR_HEADER_SIZE, "endpoint");
CHECK_SCHEMA(2, TSDB_DATA_TYPE_SMALLINT, 2, "vnodes");
CHECK_SCHEMA(3, TSDB_DATA_TYPE_SMALLINT, 2, "max_vnodes");
CHECK_SCHEMA(4, TSDB_DATA_TYPE_BINARY, 10 + VARSTR_HEADER_SIZE, "status");
CHECK_SCHEMA(5, TSDB_DATA_TYPE_TIMESTAMP, 8, "create_time");
CHECK_SCHEMA(6, TSDB_DATA_TYPE_BINARY, 24 + VARSTR_HEADER_SIZE, "offline_reason");
SServer* DndTestDnode::pServer1;
SServer* DndTestDnode::pServer2;
SServer* DndTestDnode::pServer3;
SServer* DndTestDnode::pServer4;
SServer* DndTestDnode::pServer5;
SClient* DndTestDnode::pClient;
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 1);
TEST_F(DndTestDnode, 01_ShowDnode) {
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "show dnodes", 7);
CheckSchema(0, TSDB_DATA_TYPE_SMALLINT, 2, "id");
CheckSchema(1, TSDB_DATA_TYPE_BINARY, TSDB_EP_LEN + VARSTR_HEADER_SIZE, "endpoint");
CheckSchema(2, TSDB_DATA_TYPE_SMALLINT, 2, "vnodes");
CheckSchema(3, TSDB_DATA_TYPE_SMALLINT, 2, "max_vnodes");
CheckSchema(4, TSDB_DATA_TYPE_BINARY, 10 + VARSTR_HEADER_SIZE, "status");
CheckSchema(5, TSDB_DATA_TYPE_TIMESTAMP, 8, "create_time");
CheckSchema(6, TSDB_DATA_TYPE_BINARY, 24 + VARSTR_HEADER_SIZE, "offline_reason");
SendThenCheckShowRetrieveMsg(1);
CheckInt16(1);
CheckBinary("localhost:9041", TSDB_EP_LEN);
CheckInt16(0);
......@@ -202,40 +75,36 @@ TEST_F(DndTestDnode, 01_ShowDnode) {
}
TEST_F(DndTestDnode, 02_ConfigDnode) {
SCfgDnodeMsg* pReq = (SCfgDnodeMsg*)rpcMallocCont(sizeof(SCfgDnodeMsg));
int32_t contLen = sizeof(SCfgDnodeMsg);
SCfgDnodeMsg* pReq = (SCfgDnodeMsg*)rpcMallocCont(contLen);
pReq->dnodeId = htonl(1);
strcpy(pReq->config, "ddebugflag 131");
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCfgDnodeMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CONFIG_DNODE;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CONFIG_DNODE, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
TEST_F(DndTestDnode, 03_Create_Drop_Restart_Dnode) {
{
SCreateDnodeMsg* pReq = (SCreateDnodeMsg*)rpcMallocCont(sizeof(SCreateDnodeMsg));
strcpy(pReq->ep, "localhost:9042");
int32_t contLen = sizeof(SCreateDnodeMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCreateDnodeMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_DNODE;
SCreateDnodeMsg* pReq = (SCreateDnodeMsg*)rpcMallocCont(contLen);
strcpy(pReq->ep, "localhost:9042");
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_DNODE, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
taosMsleep(1300);
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "show dnodes", 7);
SendThenCheckShowRetrieveMsg(2);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "");
CHECK_META("show dnodes", 7);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 2);
CheckInt16(1);
CheckInt16(2);
CheckBinary("localhost:9041", TSDB_EP_LEN);
......@@ -252,22 +121,21 @@ TEST_F(DndTestDnode, 03_Create_Drop_Restart_Dnode) {
CheckBinary("", 24);
{
SDropDnodeMsg* pReq = (SDropDnodeMsg*)rpcMallocCont(sizeof(SDropDnodeMsg));
pReq->dnodeId = htonl(2);
int32_t contLen = sizeof(SDropDnodeMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SDropDnodeMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_DROP_DNODE;
SDropDnodeMsg* pReq = (SDropDnodeMsg*)rpcMallocCont(contLen);
pReq->dnodeId = htonl(2);
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_DROP_DNODE, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "show dnodes", 7);
SendThenCheckShowRetrieveMsg(1);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "");
CHECK_META("show dnodes", 7);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 1);
CheckInt16(1);
CheckBinary("localhost:9041", TSDB_EP_LEN);
CheckInt16(0);
......@@ -277,53 +145,44 @@ TEST_F(DndTestDnode, 03_Create_Drop_Restart_Dnode) {
CheckBinary("", 24);
{
SCreateDnodeMsg* pReq = (SCreateDnodeMsg*)rpcMallocCont(sizeof(SCreateDnodeMsg));
strcpy(pReq->ep, "localhost:9043");
int32_t contLen = sizeof(SCreateDnodeMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCreateDnodeMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_DNODE;
SCreateDnodeMsg* pReq = (SCreateDnodeMsg*)rpcMallocCont(contLen);
strcpy(pReq->ep, "localhost:9043");
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_DNODE, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
{
SCreateDnodeMsg* pReq = (SCreateDnodeMsg*)rpcMallocCont(sizeof(SCreateDnodeMsg));
strcpy(pReq->ep, "localhost:9044");
int32_t contLen = sizeof(SCreateDnodeMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCreateDnodeMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_DNODE;
SCreateDnodeMsg* pReq = (SCreateDnodeMsg*)rpcMallocCont(contLen);
strcpy(pReq->ep, "localhost:9044");
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_DNODE, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
{
SCreateDnodeMsg* pReq = (SCreateDnodeMsg*)rpcMallocCont(sizeof(SCreateDnodeMsg));
strcpy(pReq->ep, "localhost:9045");
int32_t contLen = sizeof(SCreateDnodeMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCreateDnodeMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_DNODE;
SCreateDnodeMsg* pReq = (SCreateDnodeMsg*)rpcMallocCont(contLen);
strcpy(pReq->ep, "localhost:9045");
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_DNODE, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
taosMsleep(1300);
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "show dnodes", 7);
SendThenCheckShowRetrieveMsg(4);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "");
CHECK_META("show dnodes", 7);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 4);
CheckInt16(1);
CheckInt16(3);
CheckInt16(4);
......@@ -355,31 +214,18 @@ TEST_F(DndTestDnode, 03_Create_Drop_Restart_Dnode) {
// restart
uInfo("stop all server");
stopServer(pServer1);
stopServer(pServer2);
stopServer(pServer3);
stopServer(pServer4);
stopServer(pServer5);
pServer1 = NULL;
pServer2 = NULL;
pServer3 = NULL;
pServer4 = NULL;
pServer5 = NULL;
uInfo("start all server");
const char* fqdn = "localhost";
const char* firstEp = "localhost:9041";
pServer1 = startServer("/tmp/dnode_test_dnode1", fqdn, 9041, firstEp);
pServer3 = startServer("/tmp/dnode_test_dnode3", fqdn, 9043, firstEp);
pServer4 = startServer("/tmp/dnode_test_dnode4", fqdn, 9044, firstEp);
pServer5 = startServer("/tmp/dnode_test_dnode5", fqdn, 9045, firstEp);
uInfo("all server is running");
test.Restart();
server2.Restart();
server3.Restart();
server4.Restart();
server5.Restart();
taosMsleep(1300);
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "show dnodes", 7);
SendThenCheckShowRetrieveMsg(4);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_DNODE, "");
CHECK_META("show dnodes", 7);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 4);
CheckInt16(1);
CheckInt16(3);
CheckInt16(4);
......
add_executable(dndTestProfile "")
target_sources(dndTestProfile
PRIVATE
"profile.cpp"
"../sut/deploy.cpp"
)
aux_source_directory(. PROFILE_SRC)
add_executable(dnode_test_profile ${PROFILE_SRC})
target_link_libraries(
dndTestProfile
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(dndTestProfile
PUBLIC
"${CMAKE_SOURCE_DIR}/include/server/dnode/mgmt"
"${CMAKE_CURRENT_SOURCE_DIR}/../../inc"
"${CMAKE_CURRENT_SOURCE_DIR}/../sut"
dnode_test_profile
PUBLIC sut
)
add_test(
NAME dndTestProfile
COMMAND dndTestProfile
NAME dnode_test_profile
COMMAND dnode_test_profile
)
add_executable(dnode_test_show "")
target_sources(dnode_test_show
PRIVATE
"show.cpp"
"../sut/deploy.cpp"
)
aux_source_directory(. SHOW_SRC)
add_executable(dnode_test_show ${SHOW_SRC})
target_link_libraries(
dnode_test_show
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(dnode_test_show
PUBLIC
"${CMAKE_SOURCE_DIR}/include/server/dnode/mgmt"
"${CMAKE_CURRENT_SOURCE_DIR}/../../inc"
"${CMAKE_CURRENT_SOURCE_DIR}/../sut"
PUBLIC sut
)
add_test(
......
......@@ -9,226 +9,79 @@
*
*/
#include "deploy.h"
#include "base.h"
class DndTestShow : public ::testing::Test {
protected:
static void SetUpTestSuite() { test.Init("/tmp/dnode_test_show", 9091); }
static void TearDownTestSuite() { test.Cleanup(); }
static Testbase test;
public:
void SetUp() override {}
void TearDown() override {}
static void SetUpTestSuite() {
const char* user = "root";
const char* pass = "taosdata";
const char* path = "/tmp/dndTestShow";
const char* fqdn = "localhost";
uint16_t port = 9523;
pServer = createServer(path, fqdn, port);
ASSERT(pServer);
pClient = createClient(user, pass, fqdn, port);
}
static void TearDownTestSuite() {
stopServer(pServer);
dropClient(pClient);
}
static SServer* pServer;
static SClient* pClient;
static int32_t connId;
};
SServer* DndTestShow::pServer;
SClient* DndTestShow::pClient;
int32_t DndTestShow::connId;
Testbase DndTestShow::test;
TEST_F(DndTestShow, SShowMsg_01) {
ASSERT_NE(pClient, nullptr);
TEST_F(DndTestShow, 01_ShowMsg_InvalidMsgMax) {
int32_t contLen = sizeof(SShowMsg);
SConnectMsg* pReq = (SConnectMsg*)rpcMallocCont(sizeof(SConnectMsg));
pReq->pid = htonl(1234);
strcpy(pReq->app, "dndTestShow");
SShowMsg* pReq = (SShowMsg*)rpcMallocCont(contLen);
pReq->type = TSDB_MGMT_TABLE_MAX;
strcpy(pReq->db, "");
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SConnectMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CONNECT;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_SHOW, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
SConnectRsp* pRsp = (SConnectRsp*)pMsg->pCont;
ASSERT_NE(pRsp, nullptr);
pRsp->connId = htonl(pRsp->connId);
EXPECT_EQ(pRsp->connId, 1);
connId = pRsp->connId;
ASSERT_EQ(pMsg->code, TSDB_CODE_MND_INVALID_MSG_TYPE);
}
TEST_F(DndTestShow, SShowMsg_02) {
ASSERT_NE(pClient, nullptr);
TEST_F(DndTestShow, 02_ShowMsg_InvalidMsgStart) {
int32_t contLen = sizeof(SShowMsg);
SShowMsg* pReq = (SShowMsg*)rpcMallocCont(sizeof(SShowMsg));
pReq->type = TSDB_MGMT_TABLE_MAX;
pReq->type = TSDB_MGMT_TABLE_START;
strcpy(pReq->db, "");
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SShowMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_SHOW;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_SHOW, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, TSDB_CODE_MND_INVALID_MSG_TYPE);
}
TEST_F(DndTestShow, SShowMsg_03) {
ASSERT_NE(pClient, nullptr);
TEST_F(DndTestShow, 02_ShowMsg_Conn) {
int32_t contLen = sizeof(SConnectMsg);
SShowMsg* pReq = (SShowMsg*)rpcMallocCont(sizeof(SShowMsg));
pReq->type = TSDB_MGMT_TABLE_START;
SConnectMsg* pReq = (SConnectMsg*)rpcMallocCont(contLen);
pReq->pid = htonl(1234);
strcpy(pReq->app, "dnode_test_show");
strcpy(pReq->db, "");
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SShowMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_SHOW;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CONNECT, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, TSDB_CODE_MND_INVALID_MSG_TYPE);
}
TEST_F(DndTestShow, SShowMsg_04) {
ASSERT_NE(pClient, nullptr);
int32_t showId = 0;
{
SShowMsg* pReq = (SShowMsg*)rpcMallocCont(sizeof(SShowMsg));
pReq->type = TSDB_MGMT_TABLE_CONNS;
strcpy(pReq->db, "");
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SShowMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_SHOW;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
ASSERT_NE(pMsg, nullptr);
SShowRsp* pRsp = (SShowRsp*)pMsg->pCont;
ASSERT_NE(pRsp, nullptr);
pRsp->showId = htonl(pRsp->showId);
STableMetaMsg* pMeta = &pRsp->tableMeta;
pMeta->contLen = htonl(pMeta->contLen);
pMeta->numOfColumns = htons(pMeta->numOfColumns);
pMeta->sversion = htons(pMeta->sversion);
pMeta->tversion = htons(pMeta->tversion);
pMeta->tid = htonl(pMeta->tid);
pMeta->uid = htobe64(pMeta->uid);
pMeta->suid = htobe64(pMeta->suid);
showId = pRsp->showId;
EXPECT_NE(pRsp->showId, 0);
EXPECT_EQ(pMeta->contLen, 0);
EXPECT_STREQ(pMeta->tbFname, "");
EXPECT_EQ(pMeta->numOfTags, 0);
EXPECT_EQ(pMeta->precision, 0);
EXPECT_EQ(pMeta->tableType, 0);
EXPECT_EQ(pMeta->numOfColumns, 7);
EXPECT_EQ(pMeta->sversion, 0);
EXPECT_EQ(pMeta->tversion, 0);
EXPECT_EQ(pMeta->tid, 0);
EXPECT_EQ(pMeta->uid, 0);
EXPECT_STREQ(pMeta->sTableName, "");
EXPECT_EQ(pMeta->suid, 0);
SSchema* pSchema = NULL;
pSchema = &pMeta->pSchema[0];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, TSDB_DATA_TYPE_INT);
EXPECT_EQ(pSchema->bytes, 4);
EXPECT_STREQ(pSchema->name, "connId");
pSchema = &pMeta->pSchema[1];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, TSDB_DATA_TYPE_BINARY);
EXPECT_EQ(pSchema->bytes, TSDB_USER_LEN + VARSTR_HEADER_SIZE);
EXPECT_STREQ(pSchema->name, "user");
pSchema = &pMeta->pSchema[2];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, TSDB_DATA_TYPE_BINARY);
EXPECT_EQ(pSchema->bytes, TSDB_USER_LEN + VARSTR_HEADER_SIZE);
EXPECT_STREQ(pSchema->name, "program");
pSchema = &pMeta->pSchema[3];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, TSDB_DATA_TYPE_INT);
EXPECT_EQ(pSchema->bytes, 4);
EXPECT_STREQ(pSchema->name, "pid");
pSchema = &pMeta->pSchema[4];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, TSDB_DATA_TYPE_BINARY);
EXPECT_EQ(pSchema->bytes, TSDB_IPv4ADDR_LEN + 6 + VARSTR_HEADER_SIZE);
EXPECT_STREQ(pSchema->name, "ip:port");
pSchema = &pMeta->pSchema[5];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, TSDB_DATA_TYPE_TIMESTAMP);
EXPECT_EQ(pSchema->bytes, 8);
EXPECT_STREQ(pSchema->name, "login_time");
pSchema = &pMeta->pSchema[6];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, TSDB_DATA_TYPE_TIMESTAMP);
EXPECT_EQ(pSchema->bytes, 8);
EXPECT_STREQ(pSchema->name, "last_access");
}
{
SRetrieveTableMsg* pReq = (SRetrieveTableMsg*)rpcMallocCont(sizeof(SRetrieveTableMsg));
pReq->showId = htonl(showId);
pReq->free = 0;
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SRetrieveTableMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_SHOW_RETRIEVE;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
SRetrieveTableRsp* pRsp = (SRetrieveTableRsp*)pMsg->pCont;
ASSERT_NE(pRsp, nullptr);
pRsp->numOfRows = htonl(pRsp->numOfRows);
pRsp->offset = htobe64(pRsp->offset);
pRsp->useconds = htobe64(pRsp->useconds);
pRsp->compLen = htonl(pRsp->compLen);
EXPECT_EQ(pRsp->numOfRows, 1);
EXPECT_EQ(pRsp->offset, 0);
EXPECT_EQ(pRsp->useconds, 0);
EXPECT_EQ(pRsp->completed, 1);
EXPECT_EQ(pRsp->precision, TSDB_TIME_PRECISION_MILLI);
EXPECT_EQ(pRsp->compressed, 0);
EXPECT_EQ(pRsp->reserved, 0);
EXPECT_EQ(pRsp->compLen, 0);
}
ASSERT_EQ(pMsg->code, 0);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_CONNS, "");
STableMetaMsg* pMeta = test.GetShowMeta();
EXPECT_STREQ(pMeta->tbFname, "show connections");
EXPECT_EQ(pMeta->numOfTags, 0);
EXPECT_EQ(pMeta->numOfColumns, 7);
EXPECT_EQ(pMeta->precision, 0);
EXPECT_EQ(pMeta->tableType, 0);
EXPECT_EQ(pMeta->update, 0);
EXPECT_EQ(pMeta->sversion, 0);
EXPECT_EQ(pMeta->tversion, 0);
EXPECT_EQ(pMeta->tuid, 0);
EXPECT_EQ(pMeta->suid, 0);
test.SendShowRetrieveMsg();
SRetrieveTableRsp* pRetrieveRsp = test.GetRetrieveRsp();
EXPECT_EQ(pRetrieveRsp->numOfRows, 1);
EXPECT_EQ(pRetrieveRsp->useconds, 0);
EXPECT_EQ(pRetrieveRsp->completed, 1);
EXPECT_EQ(pRetrieveRsp->precision, TSDB_TIME_PRECISION_MILLI);
EXPECT_EQ(pRetrieveRsp->compressed, 0);
EXPECT_EQ(pRetrieveRsp->compLen, 0);
}
add_executable(dnode_test_stb "")
target_sources(dnode_test_stb
PRIVATE
"stb.cpp"
"../sut/deploy.cpp"
)
aux_source_directory(. STB_SRC)
add_executable(dnode_test_stb ${STB_SRC})
target_link_libraries(
dnode_test_stb
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(dnode_test_stb
PUBLIC
"${CMAKE_SOURCE_DIR}/include/server/dnode/mgmt"
"${CMAKE_CURRENT_SOURCE_DIR}/../../inc"
"${CMAKE_CURRENT_SOURCE_DIR}/../sut"
PUBLIC sut
)
add_test(
......
......@@ -9,176 +9,27 @@
*
*/
#include "deploy.h"
#include "base.h"
class DndTestStb : public ::testing::Test {
protected:
static SServer* CreateServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
SServer* pServer = createServer(path, fqdn, port, firstEp);
ASSERT(pServer);
return pServer;
}
static void SetUpTestSuite() {
initLog("/tmp/tdlog");
static void SetUpTestSuite() { test.Init("/tmp/dnode_test_stb", 9101); }
static void TearDownTestSuite() { test.Cleanup(); }
const char* fqdn = "localhost";
const char* firstEp = "localhost:9101";
pServer = CreateServer("/tmp/dnode_test_stb", fqdn, 9101, firstEp);
pClient = createClient("root", "taosdata", fqdn, 9101);
taosMsleep(1100);
}
static void TearDownTestSuite() {
stopServer(pServer);
dropClient(pClient);
pServer = NULL;
pClient = NULL;
}
static SServer* pServer;
static SClient* pClient;
static int32_t connId;
static Testbase test;
public:
void SetUp() override {}
void TearDown() override {}
void SendTheCheckShowMetaMsg(int8_t showType, const char* showName, int32_t columns, const char* db) {
SShowMsg* pShow = (SShowMsg*)rpcMallocCont(sizeof(SShowMsg));
pShow->type = showType;
if (db != NULL) {
strcpy(pShow->db, db);
}
SRpcMsg showRpcMsg = {0};
showRpcMsg.pCont = pShow;
showRpcMsg.contLen = sizeof(SShowMsg);
showRpcMsg.msgType = TSDB_MSG_TYPE_SHOW;
sendMsg(pClient, &showRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
SShowRsp* pShowRsp = (SShowRsp*)pClient->pRsp->pCont;
ASSERT_NE(pShowRsp, nullptr);
pShowRsp->showId = htonl(pShowRsp->showId);
pMeta = &pShowRsp->tableMeta;
pMeta->numOfTags = htonl(pMeta->numOfTags);
pMeta->numOfColumns = htonl(pMeta->numOfColumns);
pMeta->sversion = htonl(pMeta->sversion);
pMeta->tversion = htonl(pMeta->tversion);
pMeta->tuid = htobe64(pMeta->tuid);
pMeta->suid = htobe64(pMeta->suid);
showId = pShowRsp->showId;
EXPECT_NE(pShowRsp->showId, 0);
EXPECT_STREQ(pMeta->tbFname, showName);
EXPECT_EQ(pMeta->numOfTags, 0);
EXPECT_EQ(pMeta->numOfColumns, columns);
EXPECT_EQ(pMeta->precision, 0);
EXPECT_EQ(pMeta->tableType, 0);
EXPECT_EQ(pMeta->update, 0);
EXPECT_EQ(pMeta->sversion, 0);
EXPECT_EQ(pMeta->tversion, 0);
EXPECT_EQ(pMeta->tuid, 0);
EXPECT_EQ(pMeta->suid, 0);
}
void CheckSchema(int32_t index, int8_t type, int32_t bytes, const char* name) {
SSchema* pSchema = &pMeta->pSchema[index];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, type);
EXPECT_EQ(pSchema->bytes, bytes);
EXPECT_STREQ(pSchema->name, name);
}
void SendThenCheckShowRetrieveMsg(int32_t rows) {
SRetrieveTableMsg* pRetrieve = (SRetrieveTableMsg*)rpcMallocCont(sizeof(SRetrieveTableMsg));
pRetrieve->showId = htonl(showId);
pRetrieve->free = 0;
SRpcMsg retrieveRpcMsg = {0};
retrieveRpcMsg.pCont = pRetrieve;
retrieveRpcMsg.contLen = sizeof(SRetrieveTableMsg);
retrieveRpcMsg.msgType = TSDB_MSG_TYPE_SHOW_RETRIEVE;
sendMsg(pClient, &retrieveRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
pRetrieveRsp = (SRetrieveTableRsp*)pClient->pRsp->pCont;
ASSERT_NE(pRetrieveRsp, nullptr);
pRetrieveRsp->numOfRows = htonl(pRetrieveRsp->numOfRows);
pRetrieveRsp->useconds = htobe64(pRetrieveRsp->useconds);
pRetrieveRsp->compLen = htonl(pRetrieveRsp->compLen);
EXPECT_EQ(pRetrieveRsp->numOfRows, rows);
EXPECT_EQ(pRetrieveRsp->useconds, 0);
// EXPECT_EQ(pRetrieveRsp->completed, completed);
EXPECT_EQ(pRetrieveRsp->precision, TSDB_TIME_PRECISION_MILLI);
EXPECT_EQ(pRetrieveRsp->compressed, 0);
EXPECT_EQ(pRetrieveRsp->compLen, 0);
pData = pRetrieveRsp->data;
pos = 0;
}
void CheckInt8(int8_t val) {
int8_t data = *((int8_t*)(pData + pos));
pos += sizeof(int8_t);
EXPECT_EQ(data, val);
}
void CheckInt16(int16_t val) {
int16_t data = *((int16_t*)(pData + pos));
pos += sizeof(int16_t);
EXPECT_EQ(data, val);
}
void CheckInt32(int32_t val) {
int32_t data = *((int32_t*)(pData + pos));
pos += sizeof(int32_t);
EXPECT_EQ(data, val);
}
void CheckInt64(int64_t val) {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
EXPECT_EQ(data, val);
}
void CheckTimestamp() {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
EXPECT_GT(data, 0);
}
void CheckBinary(const char* val, int32_t len) {
pos += sizeof(VarDataLenT);
char* data = (char*)(pData + pos);
pos += len;
EXPECT_STREQ(data, val);
}
int32_t showId;
STableMetaMsg* pMeta;
SRetrieveTableRsp* pRetrieveRsp;
char* pData;
int32_t pos;
};
SServer* DndTestStb::pServer;
SClient* DndTestStb::pClient;
int32_t DndTestStb::connId;
Testbase DndTestStb::test;
TEST_F(DndTestStb, 01_Create_Show_Meta_Drop_Restart_Stb) {
{
SCreateDbMsg* pReq = (SCreateDbMsg*)rpcMallocCont(sizeof(SCreateDbMsg));
int32_t contLen = sizeof(SCreateDbMsg);
SCreateDbMsg* pReq = (SCreateDbMsg*)rpcMallocCont(contLen);
strcpy(pReq->db, "1.d1");
pReq->numOfVgroups = htonl(2);
pReq->cacheBlockSize = htonl(16);
......@@ -200,13 +51,7 @@ TEST_F(DndTestStb, 01_Create_Show_Meta_Drop_Restart_Stb) {
pReq->cacheLastRow = 0;
pReq->ignoreExist = 1;
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCreateDbMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_DB;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_DB, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
......@@ -214,9 +59,9 @@ TEST_F(DndTestStb, 01_Create_Show_Meta_Drop_Restart_Stb) {
{
int32_t cols = 2;
int32_t tags = 3;
int32_t size = (tags + cols) * sizeof(SSchema) + sizeof(SCreateStbMsg);
int32_t contLen = (tags + cols) * sizeof(SSchema) + sizeof(SCreateStbMsg);
SCreateStbMsg* pReq = (SCreateStbMsg*)rpcMallocCont(size);
SCreateStbMsg* pReq = (SCreateStbMsg*)rpcMallocCont(contLen);
strcpy(pReq->name, "1.d1.stb");
pReq->numOfTags = htonl(tags);
pReq->numOfColumns = htonl(cols);
......@@ -261,24 +106,21 @@ TEST_F(DndTestStb, 01_Create_Show_Meta_Drop_Restart_Stb) {
strcpy(pSchema->name, "tag3");
}
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = size;
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_STB;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_STB, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_STB, "show stables", 4, "1.d1");
CheckSchema(0, TSDB_DATA_TYPE_BINARY, TSDB_TABLE_NAME_LEN + VARSTR_HEADER_SIZE, "name");
CheckSchema(1, TSDB_DATA_TYPE_TIMESTAMP, 8, "create_time");
CheckSchema(2, TSDB_DATA_TYPE_INT, 4, "columns");
CheckSchema(3, TSDB_DATA_TYPE_INT, 4, "tags");
test.SendShowMetaMsg(TSDB_MGMT_TABLE_STB, "1.d1");
CHECK_META("show stables", 4);
CHECK_SCHEMA(0, TSDB_DATA_TYPE_BINARY, TSDB_TABLE_NAME_LEN + VARSTR_HEADER_SIZE, "name");
CHECK_SCHEMA(1, TSDB_DATA_TYPE_TIMESTAMP, 8, "create_time");
CHECK_SCHEMA(2, TSDB_DATA_TYPE_INT, 4, "columns");
CHECK_SCHEMA(3, TSDB_DATA_TYPE_INT, 4, "tags");
SendThenCheckShowRetrieveMsg(1);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 1);
CheckBinary("stb", TSDB_TABLE_NAME_LEN);
CheckTimestamp();
CheckInt32(2);
......@@ -286,16 +128,12 @@ TEST_F(DndTestStb, 01_Create_Show_Meta_Drop_Restart_Stb) {
// ----- meta ------
{
STableInfoMsg* pReq = (STableInfoMsg*)rpcMallocCont(sizeof(STableInfoMsg));
strcpy(pReq->tableFname, "1.d1.stb");
int32_t contLen = sizeof(STableInfoMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(STableInfoMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_TABLE_META;
STableInfoMsg* pReq = (STableInfoMsg*)rpcMallocCont(contLen);
strcpy(pReq->tableFname, "1.d1.stb");
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_TABLE_META, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
......@@ -336,39 +174,31 @@ TEST_F(DndTestStb, 01_Create_Show_Meta_Drop_Restart_Stb) {
}
// restart
stopServer(pServer);
pServer = NULL;
test.Restart();
uInfo("start all server");
test.SendShowMetaMsg(TSDB_MGMT_TABLE_STB, "1.d1");
CHECK_META("show stables", 4);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 1);
const char* fqdn = "localhost";
const char* firstEp = "localhost:9101";
pServer = startServer("/tmp/dnode_test_stb", fqdn, 9101, firstEp);
uInfo("all server is running");
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_STB, "show stables", 4, "1.d1");
SendThenCheckShowRetrieveMsg(1);
CheckBinary("stb", TSDB_TABLE_NAME_LEN);
CheckTimestamp();
CheckInt32(2);
CheckInt32(3);
{
SDropStbMsg* pReq = (SDropStbMsg*)rpcMallocCont(sizeof(SDropStbMsg));
strcpy(pReq->name, "1.d1.stb");
int32_t contLen = sizeof(SDropStbMsg);
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SDropStbMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_DROP_STB;
SDropStbMsg* pReq = (SDropStbMsg*)rpcMallocCont(contLen);
strcpy(pReq->name, "1.d1.stb");
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_DROP_STB, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
SendTheCheckShowMetaMsg(TSDB_MGMT_TABLE_STB, "show stables", 4, "1.d1");
SendThenCheckShowRetrieveMsg(0);
test.SendShowMetaMsg(TSDB_MGMT_TABLE_STB, "1.d1");
CHECK_META("show stables", 4);
test.SendShowRetrieveMsg();
EXPECT_EQ(test.GetShowRows(), 0);
}
aux_source_directory(src SUT_SRC)
add_library(sut STATIC ${SUT_SRC})
target_link_libraries(
sut
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(
sut
PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/inc"
)
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "deploy.h"
void initLog(const char* path) {
dDebugFlag = 207;
vDebugFlag = 0;
mDebugFlag = 207;
cDebugFlag = 0;
jniDebugFlag = 0;
tmrDebugFlag = 0;
uDebugFlag = 143;
rpcDebugFlag = 0;
odbcDebugFlag = 0;
qDebugFlag = 0;
wDebugFlag = 0;
sDebugFlag = 0;
tsdbDebugFlag = 0;
cqDebugFlag = 0;
tscEmbeddedInUtil = 1;
taosRemoveDir(path);
taosMkDir(path);
char temp[PATH_MAX];
snprintf(temp, PATH_MAX, "%s/taosdlog", path);
if (taosInitLog(temp, tsNumOfLogLines, 1) != 0) {
printf("failed to init log file\n");
}
}
void* runServer(void* param) {
SServer* pServer = (SServer*)param;
while (1) {
taosMsleep(100);
pthread_testcancel();
}
}
void initOption(SDnodeOpt* pOption, const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
pOption->sver = 1;
pOption->numOfCores = 1;
pOption->numOfSupportMnodes = 1;
pOption->numOfSupportVnodes = 1;
pOption->numOfSupportQnodes = 1;
pOption->statusInterval = 1;
pOption->numOfThreadsPerCore = 1;
pOption->ratioOfQueryCores = 1;
pOption->maxShellConns = 1000;
pOption->shellActivityTimer = 30;
pOption->serverPort = port;
strcpy(pOption->dataDir, path);
snprintf(pOption->localEp, TSDB_EP_LEN, "%s:%u", fqdn, port);
snprintf(pOption->localFqdn, TSDB_FQDN_LEN, "%s", fqdn);
snprintf(pOption->firstEp, TSDB_EP_LEN, "%s", firstEp);
}
SServer* startServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
taosMkDir(path);
SDnodeOpt option = {0};
initOption(&option, path, fqdn, port, firstEp);
SDnode* pDnode = dndInit(&option);
ASSERT(pDnode);
SServer* pServer = (SServer*)calloc(1, sizeof(SServer));
ASSERT(pServer);
pServer->pDnode = pDnode;
pServer->threadId = taosCreateThread(runServer, pServer);
ASSERT(pServer->threadId);
return pServer;
}
SServer* createServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
taosRemoveDir(path);
return startServer(path, fqdn, port, firstEp);
}
void stopServer(SServer* pServer) {
if (pServer == NULL) return;
if (pServer->threadId != NULL) {
taosDestoryThread(pServer->threadId);
}
if (pServer->pDnode != NULL) {
dndCleanup(pServer->pDnode);
pServer->pDnode = NULL;
}
}
void processClientRsp(void* parent, SRpcMsg* pMsg, SEpSet* pEpSet) {
SClient* pClient = (SClient*)parent;
pClient->pRsp = pMsg;
uInfo("response:%s from dnode, pCont:%p contLen:%d code:0x%X", taosMsg[pMsg->msgType], pMsg->pCont, pMsg->contLen,
pMsg->code);
tsem_post(&pClient->sem);
}
SClient* createClient(const char* user, const char* pass, const char* fqdn, uint16_t port) {
SClient* pClient = (SClient*)calloc(1, sizeof(SClient));
ASSERT(pClient);
char secretEncrypt[TSDB_PASSWORD_LEN] = {0};
taosEncryptPass((uint8_t*)pass, strlen(pass), secretEncrypt);
SRpcInit rpcInit;
memset(&rpcInit, 0, sizeof(rpcInit));
rpcInit.label = (char*)"DND-C";
rpcInit.numOfThreads = 1;
rpcInit.cfp = processClientRsp;
rpcInit.sessions = 1024;
rpcInit.connType = TAOS_CONN_CLIENT;
rpcInit.idleTime = 30 * 1000;
rpcInit.user = (char*)user;
rpcInit.ckey = (char*)"key";
rpcInit.parent = pClient;
rpcInit.secret = (char*)secretEncrypt;
rpcInit.parent = pClient;
// rpcInit.spi = 1;
pClient->clientRpc = rpcOpen(&rpcInit);
ASSERT(pClient->clientRpc);
tsem_init(&pClient->sem, 0, 0);
strcpy(pClient->fqdn, fqdn);
pClient->port = port;
return pClient;
}
void dropClient(SClient* pClient) {
tsem_destroy(&pClient->sem);
rpcClose(pClient->clientRpc);
}
void sendMsg(SClient* pClient, SRpcMsg* pMsg) {
SEpSet epSet = {0};
epSet.inUse = 0;
epSet.numOfEps = 1;
epSet.port[0] = pClient->port;
memcpy(epSet.fqdn[0], pClient->fqdn, TSDB_FQDN_LEN);
rpcSendRequest(pClient->clientRpc, &epSet, pMsg, NULL);
tsem_wait(&pClient->sem);
}
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_TEST_BASE_H_
#define _TD_TEST_BASE_H_
#include <gtest/gtest.h>
#include "os.h"
#include "dnode.h"
#include "taosmsg.h"
#include "tconfig.h"
#include "tdataformat.h"
#include "tglobal.h"
#include "tnote.h"
#include "trpc.h"
#include "tthread.h"
#include "ulog.h"
#include "client.h"
#include "server.h"
class Testbase {
public:
void Init(const char* path, int16_t port);
void Cleanup();
void Restart();
SRpcMsg* SendMsg(int8_t msgType, void* pCont, int32_t contLen);
private:
void InitLog(const char* path);
private:
TestServer server;
TestClient client;
int32_t connId;
public:
void SendShowMetaMsg(int8_t showType, const char* db);
void SendShowRetrieveMsg();
STableMetaMsg* GetShowMeta();
SRetrieveTableRsp* GetRetrieveRsp();
int32_t GetMetaNum();
const char* GetMetaTbName();
int32_t GetMetaColId(int32_t index);
int8_t GetMetaType(int32_t index);
int32_t GetMetaBytes(int32_t index);
const char* GetMetaName(int32_t index);
const char* GetShowName();
int32_t GetShowRows();
int8_t GetShowInt8();
int16_t GetShowInt16();
int32_t GetShowInt32();
int64_t GetShowInt64();
int64_t GetShowTimestamp();
const char* GetShowBinary(int32_t len);
private:
int32_t showId;
STableMetaMsg* pMeta;
SRetrieveTableRsp* pRetrieveRsp;
char* pData;
int32_t pos;
};
#define CHECK_META(tbName, numOfColumns) \
{ \
EXPECT_EQ(test.GetMetaNum(), numOfColumns); \
EXPECT_STREQ(test.GetMetaTbName(), tbName); \
}
#define CHECK_SCHEMA(colId, type, bytes, colName) \
{ \
EXPECT_EQ(test.GetMetaType(colId), type); \
EXPECT_EQ(test.GetMetaBytes(colId), bytes); \
EXPECT_STREQ(test.GetMetaName(colId), colName); \
}
#define CheckBinary(val, len) \
{ EXPECT_STREQ(test.GetShowBinary(len), val); }
#define CheckInt8(val) \
{ EXPECT_EQ(test.GetShowInt8(), val); }
#define CheckInt16(val) \
{ EXPECT_EQ(test.GetShowInt16(), val); }
#define CheckInt32(val) \
{ EXPECT_EQ(test.GetShowInt32(), val); }
#define CheckInt64(val) \
{ EXPECT_EQ(test.GetShowInt64(), val); }
#define CheckTimestamp() \
{ EXPECT_GT(test.GetShowTimestamp(), 0); }
#define IgnoreBinary(len) \
{ test.GetShowBinary(len); }
#define IgnoreInt8() \
{ test.GetShowInt8(); }
#define IgnoreInt16() \
{ test.GetShowInt16(); }
#define IgnoreInt32() \
{ test.GetShowInt32(); }
#define IgnoreInt64() \
{ test.GetShowInt64(); }
#define IgnoreTimestamp() \
{ test.GetShowTimestamp(); }
#endif /* _TD_TEST_BASE_H_ */
\ No newline at end of file
......@@ -13,37 +13,24 @@
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include <gtest/gtest.h>
#include "os.h"
#ifndef _TD_TEST_CLIENT_H_
#define _TD_TEST_CLIENT_H_
#include "dnode.h"
#include "taosmsg.h"
#include "tconfig.h"
#include "tglobal.h"
#include "tnote.h"
#include "trpc.h"
#include "tthread.h"
#include "ulog.h"
#include "tdataformat.h"
class TestClient {
public:
bool Init(const char* user, const char* pass, const char* fqdn, uint16_t port);
void Cleanup();
typedef struct {
SDnode* pDnode;
pthread_t* threadId;
} SServer;
SRpcMsg* SendMsg(SRpcMsg* pMsg);
void SetRpcRsp(SRpcMsg* pRsp);
tsem_t* GetSem();
typedef struct {
private:
char fqdn[TSDB_FQDN_LEN];
uint16_t port;
void* clientRpc;
SRpcMsg* pRsp;
tsem_t sem;
} SClient;
void initLog(const char* path);
SServer* createServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp);
SServer* startServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp);
void stopServer(SServer* pServer);
SClient* createClient(const char* user, const char* pass, const char* fqdn, uint16_t port);
void dropClient(SClient* pClient);
void sendMsg(SClient* pClient, SRpcMsg* pMsg);
};
#endif /* _TD_TEST_CLIENT_H_ */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_TEST_SERVER_H_
#define _TD_TEST_SERVER_H_
class TestServer {
public:
bool Start(const char* path, const char* fqdn, uint16_t port, const char* firstEp);
void Stop();
void Restart();
private:
SDnodeOpt BuildOption(const char* path, const char* fqdn, uint16_t port, const char* firstEp);
bool DoStart();
private:
SDnode* pDnode;
pthread_t* threadId;
char path[PATH_MAX];
char fqdn[TSDB_FQDN_LEN];
char firstEp[TSDB_EP_LEN];
uint16_t port;
};
#endif /* _TD_TEST_SERVER_H_ */
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "base.h"
void Testbase::InitLog(const char* path) {
dDebugFlag = 207;
vDebugFlag = 0;
mDebugFlag = 207;
cDebugFlag = 0;
jniDebugFlag = 0;
tmrDebugFlag = 0;
uDebugFlag = 143;
rpcDebugFlag = 0;
odbcDebugFlag = 0;
qDebugFlag = 0;
wDebugFlag = 0;
sDebugFlag = 0;
tsdbDebugFlag = 0;
cqDebugFlag = 0;
tscEmbeddedInUtil = 1;
taosRemoveDir(path);
taosMkDir(path);
char temp[PATH_MAX];
snprintf(temp, PATH_MAX, "%s/taosdlog", path);
if (taosInitLog(temp, tsNumOfLogLines, 1) != 0) {
printf("failed to init log file\n");
}
}
void Testbase::Init(const char* path, int16_t port) {
char fqdn[] = "localhost";
char firstEp[TSDB_EP_LEN] = {0};
snprintf(firstEp, TSDB_EP_LEN, "%s:%u", fqdn, port);
InitLog("/tmp/tdlog");
server.Start(path, fqdn, port, firstEp);
client.Init("root", "taosdata", fqdn, port);
taosMsleep(1100);
}
void Testbase::Cleanup() {
server.Stop();
client.Cleanup();
}
void Testbase::Restart() { server.Restart(); }
SRpcMsg* Testbase::SendMsg(int8_t msgType, void* pCont, int32_t contLen) {
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pCont;
rpcMsg.contLen = contLen;
rpcMsg.msgType = msgType;
return client.SendMsg(&rpcMsg);
}
void Testbase::SendShowMetaMsg(int8_t showType, const char* db) {
int32_t contLen = sizeof(SShowMsg);
SShowMsg* pShow = (SShowMsg*)rpcMallocCont(contLen);
pShow->type = showType;
strcpy(pShow->db, db);
SRpcMsg* pMsg = SendMsg(TSDB_MSG_TYPE_SHOW, pShow, contLen);
SShowRsp* pShowRsp = (SShowRsp*)pMsg->pCont;
ASSERT(pShowRsp != nullptr);
pShowRsp->showId = htonl(pShowRsp->showId);
pMeta = &pShowRsp->tableMeta;
pMeta->numOfTags = htonl(pMeta->numOfTags);
pMeta->numOfColumns = htonl(pMeta->numOfColumns);
pMeta->sversion = htonl(pMeta->sversion);
pMeta->tversion = htonl(pMeta->tversion);
pMeta->tuid = htobe64(pMeta->tuid);
pMeta->suid = htobe64(pMeta->suid);
showId = pShowRsp->showId;
}
int32_t Testbase::GetMetaColId(int32_t index) {
SSchema* pSchema = &pMeta->pSchema[index];
pSchema->colId = htonl(pSchema->colId);
return pSchema->colId;
}
int8_t Testbase::GetMetaType(int32_t index) {
SSchema* pSchema = &pMeta->pSchema[index];
return pSchema->type;
}
int32_t Testbase::GetMetaBytes(int32_t index) {
SSchema* pSchema = &pMeta->pSchema[index];
pSchema->bytes = htonl(pSchema->bytes);
return pSchema->bytes;
}
const char* Testbase::GetMetaName(int32_t index) {
SSchema* pSchema = &pMeta->pSchema[index];
return pSchema->name;
}
int32_t Testbase::GetMetaNum() { return pMeta->numOfColumns; }
const char* Testbase::GetMetaTbName() { return pMeta->tbFname; }
void Testbase::SendShowRetrieveMsg() {
int32_t contLen = sizeof(SRetrieveTableMsg);
SRetrieveTableMsg* pRetrieve = (SRetrieveTableMsg*)rpcMallocCont(contLen);
pRetrieve->showId = htonl(showId);
pRetrieve->free = 0;
SRpcMsg* pMsg = SendMsg(TSDB_MSG_TYPE_SHOW_RETRIEVE, pRetrieve, contLen);
pRetrieveRsp = (SRetrieveTableRsp*)pMsg->pCont;
pRetrieveRsp->numOfRows = htonl(pRetrieveRsp->numOfRows);
pRetrieveRsp->useconds = htobe64(pRetrieveRsp->useconds);
pRetrieveRsp->compLen = htonl(pRetrieveRsp->compLen);
pData = pRetrieveRsp->data;
pos = 0;
}
const char* Testbase::GetShowName() { return pMeta->tbFname; }
int8_t Testbase::GetShowInt8() {
int8_t data = *((int8_t*)(pData + pos));
pos += sizeof(int8_t);
return data;
}
int16_t Testbase::GetShowInt16() {
int16_t data = *((int16_t*)(pData + pos));
pos += sizeof(int16_t);
return data;
}
int32_t Testbase::GetShowInt32() {
int32_t data = *((int32_t*)(pData + pos));
pos += sizeof(int32_t);
return data;
}
int64_t Testbase::GetShowInt64() {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
return data;
}
int64_t Testbase::GetShowTimestamp() {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
return data;
}
const char* Testbase::GetShowBinary(int32_t len) {
pos += sizeof(VarDataLenT);
char* data = (char*)(pData + pos);
pos += len;
return data;
}
int32_t Testbase::GetShowRows() { return pRetrieveRsp->numOfRows; }
STableMetaMsg* Testbase::GetShowMeta() { return pMeta; }
SRetrieveTableRsp* Testbase::GetRetrieveRsp() { return pRetrieveRsp; }
\ No newline at end of file
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "base.h"
static void processClientRsp(void* parent, SRpcMsg* pMsg, SEpSet* pEpSet) {
TestClient* client = (TestClient*)parent;
client->SetRpcRsp(pMsg);
uInfo("response:%s from dnode, code:0x%x", taosMsg[pMsg->msgType], pMsg->code);
tsem_post(client->GetSem());
}
void TestClient::SetRpcRsp(SRpcMsg* pRsp) { this->pRsp = pRsp; };
tsem_t* TestClient::GetSem() { return &sem; }
bool TestClient::Init(const char* user, const char* pass, const char* fqdn, uint16_t port) {
char secretEncrypt[TSDB_PASSWORD_LEN] = {0};
taosEncryptPass((uint8_t*)pass, strlen(pass), secretEncrypt);
SRpcInit rpcInit;
memset(&rpcInit, 0, sizeof(rpcInit));
rpcInit.label = (char*)"DND-C";
rpcInit.numOfThreads = 1;
rpcInit.cfp = processClientRsp;
rpcInit.sessions = 1024;
rpcInit.connType = TAOS_CONN_CLIENT;
rpcInit.idleTime = 30 * 1000;
rpcInit.user = (char*)user;
rpcInit.ckey = (char*)"key";
rpcInit.parent = this;
rpcInit.secret = (char*)secretEncrypt;
// rpcInit.spi = 1;
clientRpc = rpcOpen(&rpcInit);
ASSERT(clientRpc);
tsem_init(&sem, 0, 0);
strcpy(this->fqdn, fqdn);
this->port = port;
return true;
}
void TestClient::Cleanup() {
tsem_destroy(&sem);
rpcClose(clientRpc);
}
SRpcMsg* TestClient::SendMsg(SRpcMsg* pMsg) {
SEpSet epSet = {0};
epSet.inUse = 0;
epSet.numOfEps = 1;
epSet.port[0] = port;
memcpy(epSet.fqdn[0], fqdn, TSDB_FQDN_LEN);
rpcSendRequest(clientRpc, &epSet, pMsg, NULL);
tsem_wait(&sem);
return pRsp;
}
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "base.h"
void* serverLoop(void* param) {
while (1) {
taosMsleep(100);
pthread_testcancel();
}
}
SDnodeOpt TestServer::BuildOption(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
SDnodeOpt option = {0};
option.sver = 1;
option.numOfCores = 1;
option.numOfSupportMnodes = 1;
option.numOfSupportVnodes = 1;
option.numOfSupportQnodes = 1;
option.statusInterval = 1;
option.numOfThreadsPerCore = 1;
option.ratioOfQueryCores = 1;
option.maxShellConns = 1000;
option.shellActivityTimer = 30;
option.serverPort = port;
strcpy(option.dataDir, path);
snprintf(option.localEp, TSDB_EP_LEN, "%s:%u", fqdn, port);
snprintf(option.localFqdn, TSDB_FQDN_LEN, "%s", fqdn);
snprintf(option.firstEp, TSDB_EP_LEN, "%s", firstEp);
return option;
}
bool TestServer::DoStart() {
SDnodeOpt option = BuildOption(path, fqdn, port, firstEp);
taosMkDir(path);
pDnode = dndInit(&option);
if (pDnode != NULL) {
return false;
}
threadId = taosCreateThread(serverLoop, NULL);
if (threadId != NULL) {
return false;
}
return true;
}
void TestServer::Restart() {
uInfo("start all server");
Stop();
DoStart();
uInfo("all server is running");
}
bool TestServer::Start(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
strcpy(this->path, path);
strcpy(this->fqdn, fqdn);
this->port = port;
strcpy(this->firstEp, firstEp);
taosRemoveDir(path);
return DoStart();
}
void TestServer::Stop() {
if (threadId != NULL) {
taosDestoryThread(threadId);
threadId = NULL;
}
if (pDnode != NULL) {
dndCleanup(pDnode);
pDnode = NULL;
}
}
add_executable(dnode_test_user "")
target_sources(dnode_test_user
PRIVATE
"user.cpp"
"../sut/deploy.cpp"
)
aux_source_directory(. USER_SRC)
add_executable(dnode_test_user ${USER_SRC})
target_link_libraries(
dnode_test_user
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(dnode_test_user
PUBLIC
"${CMAKE_SOURCE_DIR}/include/server/dnode/mgmt"
"${CMAKE_CURRENT_SOURCE_DIR}/../../inc"
"${CMAKE_CURRENT_SOURCE_DIR}/../sut"
PUBLIC sut
)
add_test(
......
add_executable(dnode_test_vgroup "")
target_sources(dnode_test_vgroup
PRIVATE
"vgroup.cpp"
"../sut/deploy.cpp"
)
aux_source_directory(. VGROUP_SRC)
add_executable(dnode_test_vgroup ${VGROUP_SRC})
target_link_libraries(
dnode_test_vgroup
PUBLIC dnode
PUBLIC util
PUBLIC os
PUBLIC gtest_main
)
target_include_directories(dnode_test_vgroup
PUBLIC
"${CMAKE_SOURCE_DIR}/include/server/dnode/mgmt"
"${CMAKE_CURRENT_SOURCE_DIR}/../../inc"
"${CMAKE_CURRENT_SOURCE_DIR}/../sut"
PUBLIC sut
)
add_test(
......
......@@ -9,177 +9,28 @@
*
*/
#include "deploy.h"
#include "base.h"
class DndTestVgroup : public ::testing::Test {
protected:
static SServer* CreateServer(const char* path, const char* fqdn, uint16_t port, const char* firstEp) {
SServer* pServer = createServer(path, fqdn, port, firstEp);
ASSERT(pServer);
return pServer;
}
static void SetUpTestSuite() {
initLog("/tmp/tdlog");
const char* fqdn = "localhost";
const char* firstEp = "localhost:9150";
pServer = CreateServer("/tmp/dnode_test_vgroup", fqdn, 9150, firstEp);
pClient = createClient("root", "taosdata", fqdn, 9150);
taosMsleep(1100);
}
static void TearDownTestSuite() {
stopServer(pServer);
dropClient(pClient);
pServer = NULL;
pClient = NULL;
}
static void SetUpTestSuite() { test.Init("/tmp/dnode_test_vgroup", 9150); }
static void TearDownTestSuite() { test.Cleanup(); }
static SServer* pServer;
static SClient* pClient;
static int32_t connId;
static Testbase test;
public:
void SetUp() override {}
void TearDown() override {}
void SendTheCheckShowMetaMsg(int8_t showType, const char* showName, int32_t columns, const char* db) {
SShowMsg* pShow = (SShowMsg*)rpcMallocCont(sizeof(SShowMsg));
pShow->type = showType;
if (db != NULL) {
strcpy(pShow->db, db);
}
SRpcMsg showRpcMsg = {0};
showRpcMsg.pCont = pShow;
showRpcMsg.contLen = sizeof(SShowMsg);
showRpcMsg.msgType = TSDB_MSG_TYPE_SHOW;
sendMsg(pClient, &showRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
SShowRsp* pShowRsp = (SShowRsp*)pClient->pRsp->pCont;
ASSERT_NE(pShowRsp, nullptr);
pShowRsp->showId = htonl(pShowRsp->showId);
pMeta = &pShowRsp->tableMeta;
pMeta->numOfTags = htonl(pMeta->numOfTags);
pMeta->numOfColumns = htonl(pMeta->numOfColumns);
pMeta->sversion = htonl(pMeta->sversion);
pMeta->tversion = htonl(pMeta->tversion);
pMeta->tuid = htobe64(pMeta->tuid);
pMeta->suid = htobe64(pMeta->suid);
showId = pShowRsp->showId;
EXPECT_NE(pShowRsp->showId, 0);
EXPECT_STREQ(pMeta->tbFname, showName);
EXPECT_EQ(pMeta->numOfTags, 0);
EXPECT_EQ(pMeta->numOfColumns, columns);
EXPECT_EQ(pMeta->precision, 0);
EXPECT_EQ(pMeta->tableType, 0);
EXPECT_EQ(pMeta->update, 0);
EXPECT_EQ(pMeta->sversion, 0);
EXPECT_EQ(pMeta->tversion, 0);
EXPECT_EQ(pMeta->tuid, 0);
EXPECT_EQ(pMeta->suid, 0);
}
void CheckSchema(int32_t index, int8_t type, int32_t bytes, const char* name) {
SSchema* pSchema = &pMeta->pSchema[index];
pSchema->bytes = htonl(pSchema->bytes);
EXPECT_EQ(pSchema->colId, 0);
EXPECT_EQ(pSchema->type, type);
EXPECT_EQ(pSchema->bytes, bytes);
EXPECT_STREQ(pSchema->name, name);
}
void SendThenCheckShowRetrieveMsg(int32_t rows) {
SRetrieveTableMsg* pRetrieve = (SRetrieveTableMsg*)rpcMallocCont(sizeof(SRetrieveTableMsg));
pRetrieve->showId = htonl(showId);
pRetrieve->free = 0;
SRpcMsg retrieveRpcMsg = {0};
retrieveRpcMsg.pCont = pRetrieve;
retrieveRpcMsg.contLen = sizeof(SRetrieveTableMsg);
retrieveRpcMsg.msgType = TSDB_MSG_TYPE_SHOW_RETRIEVE;
sendMsg(pClient, &retrieveRpcMsg);
ASSERT_NE(pClient->pRsp, nullptr);
ASSERT_EQ(pClient->pRsp->code, 0);
ASSERT_NE(pClient->pRsp->pCont, nullptr);
pRetrieveRsp = (SRetrieveTableRsp*)pClient->pRsp->pCont;
ASSERT_NE(pRetrieveRsp, nullptr);
pRetrieveRsp->numOfRows = htonl(pRetrieveRsp->numOfRows);
pRetrieveRsp->useconds = htobe64(pRetrieveRsp->useconds);
pRetrieveRsp->compLen = htonl(pRetrieveRsp->compLen);
EXPECT_EQ(pRetrieveRsp->numOfRows, rows);
EXPECT_EQ(pRetrieveRsp->useconds, 0);
// EXPECT_EQ(pRetrieveRsp->completed, completed);
EXPECT_EQ(pRetrieveRsp->precision, TSDB_TIME_PRECISION_MILLI);
EXPECT_EQ(pRetrieveRsp->compressed, 0);
EXPECT_EQ(pRetrieveRsp->compLen, 0);
pData = pRetrieveRsp->data;
pos = 0;
}
void CheckInt8(int8_t val) {
int8_t data = *((int8_t*)(pData + pos));
pos += sizeof(int8_t);
EXPECT_EQ(data, val);
}
void CheckInt16(int16_t val) {
int16_t data = *((int16_t*)(pData + pos));
pos += sizeof(int16_t);
EXPECT_EQ(data, val);
}
void CheckInt32(int32_t val) {
int32_t data = *((int32_t*)(pData + pos));
pos += sizeof(int32_t);
EXPECT_EQ(data, val);
}
void CheckInt64(int64_t val) {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
EXPECT_EQ(data, val);
}
void CheckTimestamp() {
int64_t data = *((int64_t*)(pData + pos));
pos += sizeof(int64_t);
EXPECT_GT(data, 0);
}
void CheckBinary(const char* val, int32_t len) {
pos += sizeof(VarDataLenT);
char* data = (char*)(pData + pos);
pos += len;
EXPECT_STREQ(data, val);
}
int32_t showId;
STableMetaMsg* pMeta;
SRetrieveTableRsp* pRetrieveRsp;
char* pData;
int32_t pos;
};
SServer* DndTestVgroup::pServer;
SClient* DndTestVgroup::pClient;
int32_t DndTestVgroup::connId;
Testbase DndTestVgroup::test;
TEST_F(DndTestVgroup, 01_Create_Restart_Drop_Vnode) {
{
for (int i = 0; i < 3; ++i) {
SCreateVnodeMsg* pReq = (SCreateVnodeMsg*)rpcMallocCont(sizeof(SCreateVnodeMsg));
int32_t contLen = sizeof(SCreateVnodeMsg);
SCreateVnodeMsg* pReq = (SCreateVnodeMsg*)rpcMallocCont(contLen);
pReq->vgId = htonl(2);
pReq->dnodeId = htonl(1);
strcpy(pReq->db, "1.d1");
......@@ -209,13 +60,7 @@ TEST_F(DndTestVgroup, 01_Create_Restart_Drop_Vnode) {
pReplica->port = htons(9150);
}
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SCreateVnodeMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_VNODE_IN;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_CREATE_VNODE_IN, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
......@@ -223,7 +68,9 @@ TEST_F(DndTestVgroup, 01_Create_Restart_Drop_Vnode) {
{
for (int i = 0; i < 3; ++i) {
SAlterVnodeMsg* pReq = (SAlterVnodeMsg*)rpcMallocCont(sizeof(SAlterVnodeMsg));
int32_t contLen = sizeof(SAlterVnodeMsg);
SAlterVnodeMsg* pReq = (SAlterVnodeMsg*)rpcMallocCont(contLen);
pReq->vgId = htonl(2);
pReq->dnodeId = htonl(1);
strcpy(pReq->db, "1.d1");
......@@ -253,13 +100,7 @@ TEST_F(DndTestVgroup, 01_Create_Restart_Drop_Vnode) {
pReplica->port = htons(9150);
}
SRpcMsg rpcMsg = {0};
rpcMsg.pCont = pReq;
rpcMsg.contLen = sizeof(SAlterVnodeMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_ALTER_VNODE_IN;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_ALTER_VNODE_IN, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
......@@ -267,7 +108,9 @@ TEST_F(DndTestVgroup, 01_Create_Restart_Drop_Vnode) {
{
for (int i = 0; i < 3; ++i) {
SDropVnodeMsg* pReq = (SDropVnodeMsg*)rpcMallocCont(sizeof(SDropVnodeMsg));
int32_t contLen = sizeof(SDropVnodeMsg);
SDropVnodeMsg* pReq = (SDropVnodeMsg*)rpcMallocCont(contLen);
pReq->vgId = htonl(2);
pReq->dnodeId = htonl(1);
strcpy(pReq->db, "1.d1");
......@@ -278,8 +121,7 @@ TEST_F(DndTestVgroup, 01_Create_Restart_Drop_Vnode) {
rpcMsg.contLen = sizeof(SDropVnodeMsg);
rpcMsg.msgType = TSDB_MSG_TYPE_DROP_VNODE_IN;
sendMsg(pClient, &rpcMsg);
SRpcMsg* pMsg = pClient->pRsp;
SRpcMsg* pMsg = test.SendMsg(TSDB_MSG_TYPE_DROP_VNODE_IN, pReq, contLen);
ASSERT_NE(pMsg, nullptr);
ASSERT_EQ(pMsg->code, 0);
}
......
......@@ -138,7 +138,7 @@ static int32_t mndProcessShowMsg(SMnodeMsg *pMnodeMsg) {
ShowMetaFp metaFp = pMgmt->metaFps[type];
if (metaFp == NULL) {
terrno = TSDB_CODE_MND_INVALID_MSG_TYPE;
mError("failed to process show-meta msg:%s since no message handle", mndShowStr(type));
mError("failed to process show-meta msg:%s since %s", mndShowStr(type), terrstr());
return -1;
}
......
......@@ -11,3 +11,4 @@ add_subdirectory(executor)
add_subdirectory(planner)
add_subdirectory(function)
add_subdirectory(qcom)
add_subdirectory(qworker)
......@@ -1175,7 +1175,7 @@ int32_t filterAddGroupUnitFromNode(SFilterInfo *info, tExprNode* tree, SArray *g
void* key = NULL;
len = 0;
taosHashGetKey((SHashObj *)data, p, &key, &len);
taosHashGetKey(p, &key, &len);
void *fdata = NULL;
if (IS_VAR_DATA_TYPE(type)) {
......
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
......@@ -9,6 +9,6 @@ SCreateAcctMsg* buildAcctManipulationMsg(SSqlInfo* pInfo, int32_t* outputLen, in
SDropUserMsg* buildDropUserMsg(SSqlInfo* pInfo, int32_t* outputLen, int64_t id, char* msgBuf, int32_t msgLen);
SShowMsg* buildShowMsg(SShowInfo* pShowInfo, int64_t id, char* msgBuf, int32_t msgLen);
SCreateDbMsg* buildCreateDbMsg(SCreateDbInfo* pCreateDbInfo, char* msgBuf, int32_t msgLen);
SCreateStbMsg* buildCreateTableMsg(SCreateTableSql* pCreateTableSql, int32_t* len, SParseCtx* pParseCtx, SMsgBuf* pMsgBuf);
SCreateStbMsg* buildCreateTableMsg(SCreateTableSql* pCreateTableSql, int32_t* len, SParseBasicCtx* pParseCtx, SMsgBuf* pMsgBuf);
#endif // TDENGINE_ASTTOMSG_H
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册