提交 34b17dc0 编写于 作者: H Hongze Cheng

TD-353

上级 3eecf3db
......@@ -188,6 +188,8 @@ TAOS_DEFINE_ERROR(TSDB_CODE_TDB_FILE_CORRUPTED, 0, 0x0608, "tsdb file
TAOS_DEFINE_ERROR(TSDB_CODE_TDB_OUT_OF_MEMORY, 0, 0x0609, "tsdb out of memory")
TAOS_DEFINE_ERROR(TSDB_CODE_TDB_TAG_VER_OUT_OF_DATE, 0, 0x060A, "tsdb tag version is out of date")
TAOS_DEFINE_ERROR(TSDB_CODE_TDB_TIMESTAMP_OUT_OF_RANGE, 0, 0x060B, "tsdb timestamp is out of range")
TAOS_DEFINE_ERROR(TSDB_CODE_TDB_SUBMIT_MSG_MSSED_UP, 0, 0x060C, "tsdb submit message is messed up")
TAOS_DEFINE_ERROR(TSDB_CODE_TDB_INVALID_ACTION, 0, 0x060D, "tsdb invalid action")
// query
TAOS_DEFINE_ERROR(TSDB_CODE_QRY_INVALID_QHANDLE, 0, 0x0700, "query invalid handle")
......
......@@ -15,13 +15,16 @@
#ifndef _TD_TSDB_MAIN_H_
#define _TD_TSDB_MAIN_H_
#include "hash.h"
#include "tcoding.h"
#include "tglobal.h"
#include "tkvstore.h"
#include "tlist.h"
#include "tlog.h"
#include "tref.h"
#include "tsdb.h"
#include "tskiplist.h"
#include "tutil.h"
#include "tlog.h"
#include "tcoding.h"
#ifdef __cplusplus
extern "C" {
......@@ -34,223 +37,106 @@ extern int tsdbDebugFlag;
#define tsdbTrace(...) { if (tsdbDebugFlag & DEBUG_TRACE) { taosPrintLog("TDB ", tsdbDebugFlag, __VA_ARGS__); }}
#define tsdbPrint(...) { taosPrintLog("TDB ", 255, __VA_ARGS__); }
// ------------------------------ TSDB META FILE INTERFACES ------------------------------
#define TSDB_META_FILE_NAME "meta"
#define TSDB_META_HASH_FRACTION 1.1
typedef int (*iterFunc)(void *, void *cont, int contLen);
typedef void (*afterFunc)(void *);
typedef struct {
int fd; // File descriptor
int nDel; // number of deletions
int tombSize; // deleted size
int64_t size; // Total file size
void * map; // Map from uid ==> position
iterFunc iFunc;
afterFunc aFunc;
void * appH;
} SMetaFile;
SMetaFile *tsdbInitMetaFile(char *rootDir, int32_t maxTables, iterFunc iFunc, afterFunc aFunc, void *appH);
int32_t tsdbInsertMetaRecord(SMetaFile *mfh, uint64_t uid, void *cont, int32_t contLen);
int32_t tsdbDeleteMetaRecord(SMetaFile *mfh, uint64_t uid);
int32_t tsdbUpdateMetaRecord(SMetaFile *mfh, uint64_t uid, void *cont, int32_t contLen);
void tsdbCloseMetaFile(SMetaFile *mfh);
// ------------------------------ TSDB META INTERFACES ------------------------------
#define IS_CREATE_STABLE(pCfg) ((pCfg)->tagValues != NULL)
typedef struct {
TSKEY keyFirst;
TSKEY keyLast;
int32_t numOfRows;
void * pData;
} SMemTable;
// ---------- TSDB TABLE DEFINITION
#define TSDB_MAX_TABLE_SCHEMAS 16
#define TSDB_FILE_HEAD_SIZE 512
#define TSDB_FILE_DELIMITER 0xF00AFA0F
#define
// Definitions
// ------------------ tsdbMeta.c
typedef struct STable {
int8_t type;
STableId tableId;
uint64_t superUid; // Super table UID
int16_t numOfSchemas;
STSchema ** schema;
STSchema * tagSchema;
SKVRow tagVal;
SMemTable * mem;
SMemTable * imem;
void * pIndex; // For TSDB_SUPER_TABLE, it is the skiplist index
void * eventHandler; // TODO
void * streamHandler; // TODO
TSKEY lastKey; // lastkey inserted in this table, initialized as 0, TODO: make a structure
struct STable *next; // TODO: remove the next
struct STable *prev;
tstr * name; // NOTE: there a flexible string here
char * sql;
void * cqhandle;
ETableType type;
tstr* name; // NOTE: there a flexible string here
STableId tableId;
STable* pSuper; // super table pointer
uint8_t numOfSchemas;
STSchema schema[TSDB_MAX_TABLE_SCHEMAS];
STSchema* tagSchema;
SKVRow tagVal;
void* pIndex; // For TSDB_SUPER_TABLE, it is the skiplist index
void* eventHandler; // TODO
void* streamHandler; // TODO
TSKEY lastKey; // lastkey inserted in this table, initialized as 0, TODO: make a structure
char* sql;
void* cqhandle;
} STable;
#define TSDB_GET_TABLE_LAST_KEY(tb) ((tb)->lastKey)
void tsdbEncodeTable(STable *pTable, char *buf, int *contLen);
STable *tsdbDecodeTable(void *cont, int contLen);
void tsdbFreeEncode(void *cont);
// ---------- TSDB META HANDLE DEFINITION
typedef struct {
int32_t maxTables; // Max number of tables
int32_t nTables; // Tables created
STable **tables; // table array
STable *superList; // super table list TODO: change it to list container
void *map; // table map of (uid ===> table)
SMetaFile *mfh; // meta file handle
int maxRowBytes;
int maxCols;
void * pRepo;
pthread_rwlock_t rwLock;
int32_t nTables;
STable** tables;
SList* superList;
SHashObj* uidMap;
SKVStore* pStore;
int maxRowBytes;
int maxCols;
} STsdbMeta;
// element put in skiplist for each table
typedef struct STableIndexElem {
STsdbMeta* pMeta;
STable* pTable;
} STableIndexElem;
STsdbMeta *tsdbInitMeta(char *rootDir, int32_t maxTables, void *pRepo);
int32_t tsdbFreeMeta(STsdbMeta *pMeta);
STSchema * tsdbGetTableTagSchema(STsdbMeta *pMeta, STable *pTable);
// ---- Operation on STable
#define TSDB_TABLE_ID(pTable) ((pTable)->tableId)
#define TSDB_TABLE_UID(pTable) ((pTable)->uid)
#define TSDB_TABLE_NAME(pTable) ((pTable)->tableName)
#define TSDB_TABLE_TYPE(pTable) ((pTable)->type)
#define TSDB_TABLE_SUPER_TABLE_UID(pTable) ((pTable)->stableUid)
#define TSDB_TABLE_IS_SUPER_TABLE(pTable) (TSDB_TABLE_TYPE(pTable) == TSDB_SUPER_TABLE)
#define TSDB_TABLE_TAG_VALUE(pTable) ((pTable)->pTagVal)
#define TSDB_TABLE_CACHE_DATA(pTable) ((pTable)->content.pData)
#define TSDB_SUPER_TABLE_INDEX(pTable) ((pTable)->content.pIndex)
// ---- Operation on SMetaHandle
#define TSDB_NUM_OF_TABLES(pHandle) ((pHandle)->numOfTables)
#define TSDB_NUM_OF_SUPER_TABLES(pHandle) ((pHandle)->numOfSuperTables)
#define TSDB_TABLE_OF_ID(pHandle, id) ((pHandle)->pTables)[id]
#define TSDB_GET_TABLE_OF_NAME(pHandle, name) /* TODO */
STsdbMeta *tsdbGetMeta(TSDB_REPO_T *pRepo);
STable *tsdbIsValidTableToInsert(STsdbMeta *pMeta, STableId tableId);
// int32_t tsdbInsertRowToTableImpl(SSkipListNode *pNode, STable *pTable);
STable *tsdbGetTableByUid(STsdbMeta *pMeta, uint64_t uid);
char *getTSTupleKey(const void * data);
// ------------------ tsdbBuffer.c
typedef struct {
int blockId;
int offset;
int remain;
int padding;
char data[];
} STsdbCacheBlock;
int64_t blockId;
int offset;
int remain;
char data[];
} STsdbBufBlock;
typedef struct {
int64_t index;
int numOfCacheBlocks;
SList * memPool;
} STsdbBufferPool;
pthread_cond_t poolNotEmpty;
int bufBlockSize;
int tBufBlocks;
int nBufBlocks;
int64_t index;
SList* bufBlockList;
} STsdbBufPool;
// ------------------ tsdbMemTable.c
typedef struct {
TSKEY keyFirst;
TSKEY keyLast;
int64_t numOfRows;
SList * list;
} SCacheMem;
uint64_t uid;
TSKEY keyFirst;
TSKEY keyLast;
int64_t numOfRows;
SSkipList* pData;
} STableData;
typedef struct {
int cacheBlockSize;
int totalCacheBlocks;
STsdbBufferPool pool;
STsdbCacheBlock *curBlock;
SCacheMem * mem;
SCacheMem * imem;
TSDB_REPO_T * pRepo;
} STsdbCache;
STsdbCache *tsdbInitCache(int cacheBlockSize, int totalBlocks, TSDB_REPO_T *pRepo);
void tsdbFreeCache(STsdbCache *pCache);
void * tsdbAllocFromCache(STsdbCache *pCache, int bytes, TSKEY key);
// ------------------------------ TSDB FILE INTERFACES ------------------------------
#define TSDB_FILE_HEAD_SIZE 512
#define TSDB_FILE_DELIMITER 0xF00AFA0F
#define tsdbGetKeyFileId(key, daysPerFile, precision) ((key) / tsMsPerDay[(precision)] / (daysPerFile))
#define tsdbGetMaxNumOfFiles(keep, daysPerFile) ((keep) / (daysPerFile) + 3)
typedef enum {
TSDB_FILE_TYPE_HEAD = 0, // .head file type
TSDB_FILE_TYPE_DATA, // .data file type
TSDB_FILE_TYPE_LAST, // .last file type
TSDB_FILE_TYPE_MAX
} TSDB_FILE_TYPE;
#define IS_VALID_TSDB_FILE_TYPE(type) ((type) >= TSDB_FILE_TYPE_HEAD && (type) < TSDB_FILE_TYPE_MAX)
T_REF_DECLARE();
TSKEY keyFirst;
TSKEY keyLast;
int64_t numOfRows;
STableData** tData;
SList* actList;
SList* bufBlockList;
} SMemTable;
extern const char *tsdbFileSuffix[];
// ------------------ tsdbFile.c
typedef enum { TSDB_FILE_TYPE_HEAD = 0, TSDB_FILE_TYPE_DATA, TSDB_FILE_TYPE_LAST, TSDB_FILE_TYPE_MAX } TSDB_FILE_TYPE;
typedef struct {
uint32_t offset;
uint32_t len;
uint64_t size; // total size of the file
uint64_t tombSize; // unused file size
uint32_t totalBlocks;
uint32_t totalSubBlocks;
} STsdbFileInfo;
void *tsdbEncodeSFileInfo(void *buf, const STsdbFileInfo *pInfo);
void *tsdbDecodeSFileInfo(void *buf, STsdbFileInfo *pInfo);
typedef struct {
int fd;
char fname[128];
STsdbFileInfo info;
char* fname;
int fd;
uint64_t size;
uint64_t tombSize;
uint64_t totalBlocks;
uint64_t totalSubBlocks;
} SFile;
#define TSDB_IS_FILE_OPENED(f) ((f)->fd != -1)
typedef struct {
int32_t fileId;
SFile files[TSDB_FILE_TYPE_MAX];
int fileId;
SFile headF;
SFile dataF;
SFile lastF;
} SFileGroup;
// TSDB file handle
typedef struct {
int maxFGroups;
int numOfFGroups;
SFileGroup *fGroup;
int maxFGroups;
int nFGroups;
SFileGroup* pFGroup;
} STsdbFileH;
#define TSDB_MIN_FILE_ID(fh) (fh)->fGroup[0].fileId
#define TSDB_MAX_FILE_ID(fh) (fh)->fGroup[(fh)->numOfFGroups - 1].fileId
STsdbFileH *tsdbInitFileH(char *dataDir, STsdbCfg *pCfg);
void tsdbCloseFileH(STsdbFileH *pFileH);
int tsdbCreateFile(char *dataDir, int fileId, const char *suffix, SFile *pFile);
SFileGroup *tsdbCreateFGroup(STsdbFileH *pFileH, char *dataDir, int fid, int maxTables);
int tsdbOpenFile(SFile *pFile, int oflag);
int tsdbCloseFile(SFile *pFile);
SFileGroup *tsdbOpenFilesForCommit(STsdbFileH *pFileH, int fid);
int tsdbRemoveFileGroup(STsdbFileH *pFile, int fid);
int tsdbGetFileName(char *dataDir, int fileId, const char *suffix, char *fname);
#define TSDB_FGROUP_ITER_FORWARD TSDB_ORDER_ASC
#define TSDB_FGROUP_ITER_BACKWARD TSDB_ORDER_DESC
typedef struct {
int numOfFGroups;
SFileGroup *base;
......@@ -258,49 +144,30 @@ typedef struct {
int direction;
} SFileGroupIter;
void tsdbInitFileGroupIter(STsdbFileH *pFileH, SFileGroupIter *pIter, int direction);
void tsdbSeekFileGroupIter(SFileGroupIter *pIter, int fid);
SFileGroup *tsdbGetFileGroupNext(SFileGroupIter *pIter);
// ------------------ tsdbRWHelper.c
typedef struct {
uint32_t len;
uint32_t offset;
uint32_t padding; // For padding purpose
uint32_t padding;
uint32_t hasLast : 2;
uint32_t numOfBlocks : 30;
uint64_t uid;
TSKEY maxKey;
} SCompIdx; /* sizeof(SCompIdx) = 28 */
void *tsdbEncodeSCompIdx(void *buf, SCompIdx *pIdx);
void *tsdbDecodeSCompIdx(void *buf, SCompIdx *pIdx);
/**
* if numOfSubBlocks == 0, then the SCompBlock is a sub-block
* if numOfSubBlocks >= 1, then the SCompBlock is a super-block
* - if numOfSubBlocks == 1, then the SCompBlock refers to the data block, and offset/len refer to
* the data block offset and length
* - if numOfSubBlocks > 1, then the offset/len refer to the offset of the first sub-block in the
* binary
*/
} SCompIdx;
typedef struct {
int64_t last : 1; // If the block in data file or last file
int64_t offset : 63; // Offset of data block or sub-block index depending on numOfSubBlocks
int32_t algorithm : 8; // Compression algorithm
int32_t numOfRows : 24; // Number of total points
int32_t sversion; // Schema version
int32_t len; // Data block length or nothing
int16_t numOfSubBlocks; // Number of sub-blocks;
int64_t last : 1;
int64_t offset : 63;
int32_t algorithm : 8;
int32_t numOfRows : 24;
int32_t sversion;
int32_t len;
int16_t numOfSubBlocks;
int16_t numOfCols;
TSKEY keyFirst;
TSKEY keyLast;
} SCompBlock;
// Maximum number of sub-blocks a super-block can have
#define TSDB_MAX_SUBBLOCKS 8
#define IS_SUPER_BLOCK(pBlock) ((pBlock)->numOfSubBlocks >= 1)
#define IS_SUB_BLOCK(pBlock) ((pBlock)->numOfSubBlocks == 0)
typedef struct {
int32_t delimiter; // For recovery usage
int32_t checksum; // TODO: decide if checksum logic in this file or make it one API
......@@ -308,21 +175,9 @@ typedef struct {
SCompBlock blocks[];
} SCompInfo;
#define TSDB_COMPBLOCK_AT(pCompInfo, idx) ((pCompInfo)->blocks + (idx))
#define TSDB_COMPBLOCK_GET_START_AND_SIZE(pCompInfo, pCompBlock, size) \
do { \
if (pCompBlock->numOfSubBlocks > 1) { \
pCompBlock = pCompInfo->blocks + pCompBlock->offset; \
size = pCompBlock->numOfSubBlocks; \
} else { \
size = 1; \
} \
} while (0)
// TODO: take pre-calculation into account
typedef struct {
int16_t colId; // Column ID
int16_t len; // Column length // TODO: int16_t is not enough
int16_t colId;
int16_t len;
int32_t type : 8;
int32_t offset : 24;
int64_t sum;
......@@ -334,7 +189,6 @@ typedef struct {
char padding[2];
} SCompCol;
// TODO: Take recover into account
typedef struct {
int32_t delimiter; // For recovery usage
int32_t numOfCols; // For recovery usage
......@@ -342,72 +196,6 @@ typedef struct {
SCompCol cols[];
} SCompData;
STsdbFileH *tsdbGetFile(TSDB_REPO_T *pRepo);
int tsdbCopyBlockDataInFile(SFile *pOutFile, SFile *pInFile, SCompInfo *pCompInfo, int idx, int isLast,
SDataCols *pCols);
SFileGroup *tsdbSearchFGroup(STsdbFileH *pFileH, int fid);
void tsdbGetKeyRangeOfFileId(int32_t daysPerFile, int8_t precision, int32_t fileId, TSKEY *minKey, TSKEY *maxKey);
// TSDB repository definition
typedef struct STsdbRepo {
char *rootDir;
// TSDB configuration
STsdbCfg config;
STsdbAppH appH;
STsdbStat stat;
// The meter meta handle of this TSDB repository
STsdbMeta *tsdbMeta;
// The cache Handle
STsdbCache *tsdbCache;
// The TSDB file handle
STsdbFileH *tsdbFileH;
// Disk tier handle for multi-tier storage
void *diskTier;
pthread_mutex_t mutex;
int commit;
pthread_t commitThread;
// A limiter to monitor the resources used by tsdb
void *limiter;
int8_t state;
} STsdbRepo;
typedef struct {
int32_t totalLen;
int32_t len;
SDataRow row;
} SSubmitBlkIter;
int tsdbInitSubmitBlkIter(SSubmitBlk *pBlock, SSubmitBlkIter *pIter);
SDataRow tsdbGetSubmitBlkNext(SSubmitBlkIter *pIter);
#define TSDB_SUBMIT_MSG_HEAD_SIZE sizeof(SSubmitMsg)
// SSubmitMsg Iterator
typedef struct {
int32_t totalLen;
int32_t len;
SSubmitBlk *pBlock;
} SSubmitMsgIter;
int tsdbInitSubmitMsgIter(SSubmitMsg *pMsg, SSubmitMsgIter *pIter);
SSubmitBlk *tsdbGetSubmitMsgNext(SSubmitMsgIter *pIter);
int32_t tsdbTriggerCommit(TSDB_REPO_T *repo);
int32_t tsdbLockRepo(TSDB_REPO_T *repo);
int32_t tsdbUnLockRepo(TSDB_REPO_T *repo);
typedef enum { TSDB_WRITE_HELPER, TSDB_READ_HELPER } tsdb_rw_helper_t;
typedef struct {
......@@ -464,6 +252,87 @@ typedef struct {
void *compBuffer; // Buffer for temperary compress/decompress purpose
} SRWHelper;
// ------------------ tsdbMain.c
typedef struct {
int8_t state;
char* rootDir;
STsdbCfg config;
STsdbAppH appH;
STsdbStat stat;
STsdbMeta* tsdbMeta;
STsdbBufPool* pPool;
SMemTable* mem;
SMemTable* imem;
STsdbFileH* tsdbFileH;
pthread_mutex_t mutex;
int commit;
pthread_t commitThread;
} STsdbRepo;
// Operations
// ------------------ tsdbMeta.c
#define TABLE_TYPE(t) (t)->type
#define TABLE_NAME(t) (t)->name
#define TABLE_CHAR_NAME(t) TABLE_NAME(t)->data
#define TALBE_UID(t) (t)->tableId.uid
#define TABLE_TID(t) (t)->tableId.tid
#define TABLE_SUID(t) (t)->superUid
#define TABLE_LASTKEY(t) (t)->lastKey
STsdbMeta* tsdbNewMeta(STsdbCfg* pCfg);
void tsdbFreeMeta(STsdbMeta* pMeta);
// ------------------ tsdbBuffer.c
STsdbBufPool* tsdbNewBufPool();
void tsdbFreeBufPool(STsdbBufPool* pBufPool);
int tsdbOpenBufPool(STsdbRepo* pRepo);
int tsdbOpenBufPool(STsdbRepo* pRepo);
SListNode* tsdbAllocBufBlockFromPool(STsdbRepo* pRepo);
// ------------------ tsdbMemTable.c
// ------------------ tsdbFile.c
#define TSDB_KEY_FILEID(key, daysPerFile, precision) ((key) / tsMsPerDay[(precision)] / (daysPerFile))
#define TSDB_MAX_FILE(keep, daysPerFile) ((keep) / (daysPerFile) + 3)
#define TSDB_MIN_FILE_ID(fh) (fh)->pFGroup[0].fileId
#define TSDB_MAX_FILE_ID(fh) (fh)->pFGroup[(fh)->nFGroups - 1].fileId
#define TSDB_FGROUP_ITER_FORWARD TSDB_ORDER_ASC
#define TSDB_FGROUP_ITER_BACKWARD TSDB_ORDER_DESC
STsdbFileH* tsdbNewFileH(STsdbCfg* pCfg);
void tsdbFreeFileH(STsdbFileH* pFileH);
// ------------------ tsdbRWHelper.c
#define TSDB_MAX_SUBBLOCKS 8
#define IS_SUB_BLOCK(pBlock) ((pBlock)->numOfSubBlocks == 0)
// ------------------ tsdbMain.c
#define REPO_ID(r) (r)->config.tsdbId
char* tsdbGetMetaFileName(char* rootDir);
int tsdbLockRepo(STsdbRepo* pRepo);
int tsdbUnlockRepo(STsdbRepo* pRepo);
void* tsdbCommitData(void* arg);
#if 0
// TSDB repository definition
typedef struct {
int32_t totalLen;
int32_t len;
SDataRow row;
} SSubmitBlkIter;
// SSubmitMsg Iterator
typedef struct {
int32_t totalLen;
int32_t len;
SSubmitBlk *pBlock;
} SSubmitMsgIter;
// --------- Helper state
#define TSDB_HELPER_CLEAR_STATE 0x0 // Clear state
#define TSDB_HELPER_FILE_SET_AND_OPEN 0x1 // File is set
......@@ -518,6 +387,7 @@ STSchema *tsdbGetTableSchema(STsdbMeta *pMeta, STable *pTable);
#define DEFAULT_TAG_INDEX_COLUMN 0 // skip list built based on the first column of tags
int compFGroupKey(const void *key, const void *fgroup);
#endif
#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/>.
*/
#include "tsdb.h"
#include "tsdbMain.h"
#define POOL_IS_EMPTY(b) (listNEles((b)->bufBlockList) == 0)
static STsdbBufBlock *tsdbNewBufBlock(int bufBlockSize);
static void tsdbFreeBufBlock(STsdbBufBlock *pBufBlock);
// ---------------- INTERNAL FUNCTIONS ----------------
STsdbBufPool *tsdbNewBufPool() {
STsdbBufPool *pBufPool = (STsdbBufPool *)calloc(1, sizeof(*pBufPool));
if (pBufPool == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
int code = pthread_cond_init(&(pBufPool->poolNotEmpty), NULL);
if (code != 0) {
terrno = TAOS_SYSTEM_ERROR(code);
goto _err;
}
pBufPool->bufBlockList = tdListNew(sizeof(STsdbBufBlock *));
if (pBufPool->bufBlockList == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
return pBufPool;
_err:
tsdbFreeBufPool(pBufPool);
return NULL;
}
void tsdbFreeBufPool(STsdbBufPool *pBufPool) {
if (pBufPool) {
if (pBufPool->bufBlockList) {
ASSERT(listNEles(pBufPool->bufBlockList) == 0);
tdListFree(pBufPool->bufBlockList);
}
pthread_cond_destroy(&pBufPool->poolNotEmpty);
free(pBufPool);
}
}
int tsdbOpenBufPool(STsdbRepo *pRepo) {
STsdbCfg * pCfg = &(pRepo->config);
STsdbBufPool *pPool = pRepo->pPool;
ASSERT(pPool != NULL);
pPool->bufBlockSize = pCfg->cacheBlockSize;
pPool->tBufBlocks = pCfg->totalBlocks;
pPool->nBufBlocks = 0;
pPool->index = 0;
for (int i = 0; i < pCfg->totalBlocks; i++) {
STsdbBufBlock *pBufBlock = tsdbNewBufBlock(pCfg->cacheBlockSize);
if (pBufBlock == NULL) goto _err;
if (tdListAppend(pPool->bufBlockList, (void *)(&pBufBlock)) < 0) {
tsdbFreeBufBlock(pBufBlock);
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pPool->nBufBlocks++;
}
tsdbTrace("vgId:%d buffer pool is opened! bufBlockSize:%d tBufBlocks:%d nBufBlocks:%d", REPO_ID(pRepo),
pBufPool->bufBlockSize, pBufPool->tBufBlocks, pBufPool->nBufBlocks);
return 0;
_err:
tsdbCloseBufPool(pRepo);
return -1;
}
void tsdbCloseBufPool(STsdbRepo *pRepo) {
if (pRepo == NULL) return;
STsdbBufPool * pBufPool = pRepo->pPool;
STsdbBufBlock *pBufBlock = NULL;
if (pBufPool) {
SListNode *pNode = NULL;
while ((pNode = tdListPopHead(pBufPool->bufBlockList)) != NULL) {
tdListNodeGetData(pBufPool->bufBlockList, pNode, (void *)(&pBufBlock));
tsdbFreeBufBlock(pBufBlock);
}
}
tsdbTrace("vgId:%d buffer pool is closed", REPO_ID(pRepo));
}
SListNode *tsdbAllocBufBlockFromPool(STsdbRepo *pRepo) {
ASSERT(pRepo != NULL && pRepo->pool != NULL);
// ASSERT pRepo is locked
STsdbCfg * pCfg = &pRepo->config;
STsdbBufPool *pBufPool = pRepo->pool;
while (POOL_IS_EMPTY(pBufPool)) {
pthread_cond_wait(&(pBufPool->poolNotEmpty), &(pRepo->mutex));
}
ASSERT(!POOL_IS_EMPTY(pBufPool));
SListNode * pNode = tdListPopHead(pBufPool->bufBlockList);
STsdbBufBlock *pBufBlock = NULL;
tdListNodeGetData(pBufPool->bufBlockList, pNode, (void *)(&pBufBlock));
pBufBlock->blockId = pBufPool->index++;
pBufBlock->offset = 0;
pBufBlock->remain = pBufPool->bufBlockSize;
tsdbTrace("vgId:%d buffer block is allocated, blockId:%" PRId64, REPO_ID(pRepo), pBufBlock->blockId);
return pNode;
}
// ---------------- LOCAL FUNCTIONS ----------------
static STsdbBufBlock *tsdbNewBufBlock(int bufBlockSize) {
STsdbBufBlock *pBufBlock = (STsdbBufBlock *)malloc(sizeof(*pBufBlock) + bufBlockSize);
if (pBufBlock == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pBufBlock->blockId = 0;
pBufBlock->offset = 0;
pBufBlock->remain = bufBlockSize;
return pBufBlock;
_err:
tsdbFreeBufBlock(pBufBlock);
return NULL;
}
static void tsdbFreeBufBlock(STsdbBufBlock *pBufBlock) { tfree(pBufBlock); }
\ 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 <stdlib.h>
#include "tsdb.h"
#include "tsdbMain.h"
static int tsdbAllocBlockFromPool(STsdbCache *pCache);
static void tsdbFreeBlockList(SList *list);
static void tsdbFreeCacheMem(SCacheMem *mem);
static int tsdbAddCacheBlockToPool(STsdbCache *pCache);
STsdbCache *tsdbInitCache(int cacheBlockSize, int totalBlocks, TSDB_REPO_T *pRepo) {
STsdbCache *pCache = (STsdbCache *)calloc(1, sizeof(STsdbCache));
if (pCache == NULL) return NULL;
if (cacheBlockSize < 0) cacheBlockSize = TSDB_DEFAULT_CACHE_BLOCK_SIZE;
cacheBlockSize *= (1024 * 1024);
if (totalBlocks <= 1) totalBlocks = TSDB_DEFAULT_TOTAL_BLOCKS;
pCache->cacheBlockSize = cacheBlockSize;
pCache->totalCacheBlocks = totalBlocks;
pCache->pRepo = pRepo;
STsdbBufferPool *pPool = &(pCache->pool);
pPool->index = 0;
pPool->memPool = tdListNew(sizeof(STsdbCacheBlock *));
if (pPool->memPool == NULL) goto _err;
for (int i = 0; i < totalBlocks; i++) {
if (tsdbAddCacheBlockToPool(pCache) < 0) goto _err;
}
pCache->mem = NULL;
pCache->imem = NULL;
return pCache;
_err:
tsdbFreeCache(pCache);
return NULL;
}
void tsdbFreeCache(STsdbCache *pCache) {
tsdbFreeCacheMem(pCache->imem);
tsdbFreeCacheMem(pCache->mem);
tsdbFreeBlockList(pCache->pool.memPool);
free(pCache);
}
void *tsdbAllocFromCache(STsdbCache *pCache, int bytes, TSKEY key) {
if (pCache == NULL) return NULL;
if (bytes > pCache->cacheBlockSize) return NULL;
if (pCache->curBlock == NULL || pCache->curBlock->remain < bytes) {
if (pCache->curBlock !=NULL && listNEles(pCache->mem->list) >= pCache->totalCacheBlocks/2) {
tsdbTriggerCommit(pCache->pRepo);
}
while (tsdbAllocBlockFromPool(pCache) < 0) {
// TODO: deal with the error
// printf("Failed to allocate from cache pool\n");
}
}
void *ptr = (void *)(pCache->curBlock->data + pCache->curBlock->offset);
pCache->curBlock->offset += bytes;
pCache->curBlock->remain -= bytes;
memset(ptr, 0, bytes);
if (key < pCache->mem->keyFirst) pCache->mem->keyFirst = key;
if (key > pCache->mem->keyLast) pCache->mem->keyLast = key;
pCache->mem->numOfRows++;
return ptr;
}
static void tsdbFreeBlockList(SList *list) {
SListNode * node = NULL;
STsdbCacheBlock *pBlock = NULL;
while ((node = tdListPopHead(list)) != NULL) {
tdListNodeGetData(list, node, (void *)(&pBlock));
free(pBlock);
listNodeFree(node);
}
tdListFree(list);
}
static void tsdbFreeCacheMem(SCacheMem *mem) {
if (mem == NULL) return;
SList *list = mem->list;
tsdbFreeBlockList(list);
free(mem);
}
static int tsdbAllocBlockFromPool(STsdbCache *pCache) {
STsdbBufferPool *pPool = &(pCache->pool);
tsdbLockRepo(pCache->pRepo);
if (listNEles(pPool->memPool) == 0) {
tsdbUnLockRepo(pCache->pRepo);
return -1;
}
SListNode *node = tdListPopHead(pPool->memPool);
STsdbCacheBlock *pBlock = NULL;
tdListNodeGetData(pPool->memPool, node, (void *)(&pBlock));
pBlock->blockId = pPool->index++;
pBlock->offset = 0;
pBlock->remain = pCache->cacheBlockSize;
if (pCache->mem == NULL) { // Create a new one
pCache->mem = (SCacheMem *)malloc(sizeof(SCacheMem));
if (pCache->mem == NULL) return -1;
pCache->mem->keyFirst = INT64_MAX;
pCache->mem->keyLast = 0;
pCache->mem->numOfRows = 0;
pCache->mem->list = tdListNew(sizeof(STsdbCacheBlock *));
}
tdListAppendNode(pCache->mem->list, node);
pCache->curBlock = pBlock;
tsdbUnLockRepo(pCache->pRepo);
return 0;
}
int tsdbAlterCacheTotalBlocks(STsdbRepo *pRepo, int totalBlocks) {
STsdbCache *pCache = pRepo->tsdbCache;
int oldNumOfBlocks = pCache->totalCacheBlocks;
tsdbLockRepo((TSDB_REPO_T *)pRepo);
ASSERT(pCache->totalCacheBlocks != totalBlocks);
if (pCache->totalCacheBlocks < totalBlocks) {
ASSERT(pCache->totalCacheBlocks == pCache->pool.numOfCacheBlocks);
int blocksToAdd = pCache->totalCacheBlocks - totalBlocks;
pCache->totalCacheBlocks = totalBlocks;
for (int i = 0; i < blocksToAdd; i++) {
if (tsdbAddCacheBlockToPool(pCache) < 0) {
tsdbUnLockRepo((TSDB_REPO_T *)pRepo);
tsdbError("tsdbId:%d, failed to add cache block to cache pool", pRepo->config.tsdbId);
return -1;
}
}
} else {
pCache->totalCacheBlocks = totalBlocks;
tsdbAdjustCacheBlocks(pCache);
}
pRepo->config.totalBlocks = totalBlocks;
tsdbUnLockRepo((TSDB_REPO_T *)pRepo);
tsdbTrace("vgId:%d, tsdb total cache blocks changed from %d to %d", pRepo->config.tsdbId, oldNumOfBlocks, totalBlocks);
return 0;
}
static int tsdbAddCacheBlockToPool(STsdbCache *pCache) {
STsdbBufferPool *pPool = &pCache->pool;
STsdbCacheBlock *pBlock = malloc(sizeof(STsdbCacheBlock) + pCache->cacheBlockSize);
if (pBlock == NULL) return -1;
pBlock->offset = 0;
pBlock->remain = pCache->cacheBlockSize;
tdListAppend(pPool->memPool, (void *)(&pBlock));
pPool->numOfCacheBlocks++;
return 0;
}
static int tsdbRemoveCacheBlockFromPool(STsdbCache *pCache) {
STsdbBufferPool *pPool = &pCache->pool;
STsdbCacheBlock *pBlock = NULL;
ASSERT(pCache->totalCacheBlocks >= 0);
SListNode *node = tdListPopHead(pPool->memPool);
if (node == NULL) return -1;
tdListNodeGetData(pPool->memPool, node, &pBlock);
free(pBlock);
listNodeFree(node);
pPool->numOfCacheBlocks--;
return 0;
}
void tsdbAdjustCacheBlocks(STsdbCache *pCache) {
while (pCache->totalCacheBlocks < pCache->pool.numOfCacheBlocks) {
if (tsdbRemoveCacheBlockFromPool(pCache) < 0) break;
}
}
\ No newline at end of file
......@@ -35,8 +35,35 @@ const char *tsdbFileSuffix[] = {
".last" // TSDB_FILE_TYPE_LAST
};
static int compFGroup(const void *arg1, const void *arg2);
static int tsdbOpenFGroup(STsdbFileH *pFileH, char *dataDir, int fid);
// ---------------- INTERNAL FUNCTIONS ----------------
STsdbFileH* tsdbNewFileH(STsdbCfg* pCfg) {
STsdbFileH *pFileH = (STsdbFileH *)calloc(1, sizeof(*pFileH));
if (pFileH == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pFileH->maxFGroups = pCfg->keep / pCfg->daysPerFile + 3;
pFileH->pFGroup = (SFileGroup *)calloc(pFileH->maxFGroups, sizeof(SFileGroup));
if (pFileH->pFGroup == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
return pFileH;
_err:
tsdbFreeFileH(pFileH);
return NULL;
}
void tsdbFreeFileH(STsdbFileH* pFileH) {
if (pFileH) {
tfree(pFileH->pFGroup);
free(pFileH);
}
}
STsdbFileH *tsdbInitFileH(char *dataDir, STsdbCfg *pCfg) {
STsdbFileH *pFileH = (STsdbFileH *)calloc(1, sizeof(STsdbFileH));
......@@ -80,41 +107,6 @@ void tsdbCloseFileH(STsdbFileH *pFileH) {
}
}
static int tsdbInitFile(char *dataDir, int fid, const char *suffix, SFile *pFile) {
uint32_t version;
char buf[512] = "\0";
tsdbGetFileName(dataDir, fid, suffix, pFile->fname);
if (access(pFile->fname, F_OK|R_OK|W_OK) < 0) return -1;
pFile->fd = -1;
if (tsdbOpenFile(pFile, O_RDONLY) < 0) return -1;
if (tread(pFile->fd, buf, TSDB_FILE_HEAD_SIZE) < TSDB_FILE_HEAD_SIZE) return -1;
if (!taosCheckChecksumWhole((uint8_t *)buf, TSDB_FILE_HEAD_SIZE)) return -1;
void *pBuf = buf;
pBuf = taosDecodeFixedU32(pBuf, &version);
pBuf = tsdbDecodeSFileInfo(pBuf, &(pFile->info));
tsdbCloseFile(pFile);
return 0;
}
static int tsdbOpenFGroup(STsdbFileH *pFileH, char *dataDir, int fid) {
if (tsdbSearchFGroup(pFileH, fid) != NULL) return 0;
SFileGroup fGroup = {0};
fGroup.fileId = fid;
for (int type = TSDB_FILE_TYPE_HEAD; type < TSDB_FILE_TYPE_MAX; type++) {
if (tsdbInitFile(dataDir, fid, tsdbFileSuffix[type], &fGroup.files[type]) < 0) return -1;
}
pFileH->fGroup[pFileH->numOfFGroups++] = fGroup;
qsort((void *)(pFileH->fGroup), pFileH->numOfFGroups, sizeof(SFileGroup), compFGroup);
return 0;
}
/**
* Create the file group if the file group not exists.
*
......@@ -228,28 +220,6 @@ SFileGroup *tsdbGetFileGroupNext(SFileGroupIter *pIter) {
return ret;
}
// int tsdbLoadDataBlock(SFile *pFile, SCompBlock *pStartBlock, int numOfBlocks, SDataCols *pCols, SCompData *pCompData) {
// SCompBlock *pBlock = pStartBlock;
// for (int i = 0; i < numOfBlocks; i++) {
// if (tsdbLoadCompCols(pFile, pBlock, (void *)pCompData) < 0) return -1;
// pCols->numOfRows += (pCompData->cols[0].len / 8);
// for (int iCol = 0; iCol < pBlock->numOfCols; iCol++) {
// SCompCol *pCompCol = &(pCompData->cols[iCol]);
// // pCols->numOfRows += pBlock->numOfRows;
// int k = 0;
// for (; k < pCols->numOfCols; k++) {
// if (pCompCol->colId == pCols->cols[k].colId) break;
// }
// if (tsdbLoadColData(pFile, pCompCol, pBlock->offset,
// (void *)((char *)(pCols->cols[k].pData) + pCols->cols[k].len)) < 0)
// return -1;
// }
// pStartBlock++;
// }
// return 0;
// }
int tsdbCopyBlockDataInFile(SFile *pOutFile, SFile *pInFile, SCompInfo *pCompInfo, int idx, int isLast, SDataCols *pCols) {
SCompBlock *pSuperBlock = TSDB_COMPBLOCK_AT(pCompInfo, idx);
SCompBlock *pStartBlock = NULL;
......@@ -294,10 +264,6 @@ int compFGroupKey(const void *key, const void *fgroup) {
}
}
static int compFGroup(const void *arg1, const void *arg2) {
return ((SFileGroup *)arg1)->fileId - ((SFileGroup *)arg2)->fileId;
}
int tsdbGetFileName(char *dataDir, int fileId, const char *suffix, char *fname) {
if (dataDir == NULL || fname == NULL) return -1;
......@@ -371,4 +337,44 @@ SFileGroup *tsdbSearchFGroup(STsdbFileH *pFileH, int fid) {
void *ptr = bsearch((void *)&fid, (void *)(pFileH->fGroup), pFileH->numOfFGroups, sizeof(SFileGroup), compFGroupKey);
if (ptr == NULL) return NULL;
return (SFileGroup *)ptr;
}
// ---------------- LOCAL FUNCTIONS ----------------
static int tsdbInitFile(char *dataDir, int fid, const char *suffix, SFile *pFile) {
uint32_t version;
char buf[512] = "\0";
tsdbGetFileName(dataDir, fid, suffix, pFile->fname);
if (access(pFile->fname, F_OK|R_OK|W_OK) < 0) return -1;
pFile->fd = -1;
if (tsdbOpenFile(pFile, O_RDONLY) < 0) return -1;
if (tread(pFile->fd, buf, TSDB_FILE_HEAD_SIZE) < TSDB_FILE_HEAD_SIZE) return -1;
if (!taosCheckChecksumWhole((uint8_t *)buf, TSDB_FILE_HEAD_SIZE)) return -1;
void *pBuf = buf;
pBuf = taosDecodeFixedU32(pBuf, &version);
pBuf = tsdbDecodeSFileInfo(pBuf, &(pFile->info));
tsdbCloseFile(pFile);
return 0;
}
static int tsdbOpenFGroup(STsdbFileH *pFileH, char *dataDir, int fid) {
if (tsdbSearchFGroup(pFileH, fid) != NULL) return 0;
SFileGroup fGroup = {0};
fGroup.fileId = fid;
for (int type = TSDB_FILE_TYPE_HEAD; type < TSDB_FILE_TYPE_MAX; type++) {
if (tsdbInitFile(dataDir, fid, tsdbFileSuffix[type], &fGroup.files[type]) < 0) return -1;
}
pFileH->fGroup[pFileH->numOfFGroups++] = fGroup;
qsort((void *)(pFileH->fGroup), pFileH->numOfFGroups, sizeof(SFileGroup), compFGroup);
return 0;
}
static int compFGroup(const void *arg1, const void *arg2) {
return ((SFileGroup *)arg1)->fileId - ((SFileGroup *)arg2)->fileId;
}
\ 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 "tsdbMain.h"
#include "os.h"
#include "taosdef.h"
#include "tulog.h"
#include "talgo.h"
#include "tsdb.h"
#include "tsdbMain.h"
#include "tscompression.h"
#include "taosdef.h"
#include "tchecksum.h"
#include "tscompression.h"
#include "tsdb.h"
#include "ttime.h"
#include <sys/stat.h>
#include "tulog.h"
#define IS_VALID_PRECISION(precision) (((precision) >= TSDB_TIME_PRECISION_MILLI) && ((precision) <= TSDB_TIME_PRECISION_NANO))
#define TSDB_DEFAULT_COMPRESSION TWO_STAGE_COMP
#define IS_VALID_COMPRESSION(compression) (((compression) >= NO_COMPRESSION) && ((compression) <= TWO_STAGE_COMP))
#define TSDB_MIN_ID 0
#define TSDB_MAX_ID INT_MAX
#include <pthread.h>
#include <sys/stat.h>
#define TSDB_CFG_FILE_NAME "config"
#define TSDB_DATA_DIR_NAME "data"
#define TSDB_DEFAULT_FILE_BLOCK_ROW_OPTION 0.7
#define TSDB_MAX_LAST_FILE_SIZE (1024 * 1024 * 10) // 10M
enum { TSDB_REPO_STATE_ACTIVE, TSDB_REPO_STATE_CLOSED, TSDB_REPO_STATE_CONFIGURING };
static int32_t tsdbCheckAndSetDefaultCfg(STsdbCfg *pCfg);
static int32_t tsdbSetRepoEnv(STsdbRepo *pRepo);
static int32_t tsdbDestroyRepoEnv(STsdbRepo *pRepo);
// static int tsdbOpenMetaFile(char *tsdbDir);
static int32_t tsdbInsertDataToTable(TSDB_REPO_T *repo, SSubmitBlk *pBlock, TSKEY now, int * affectedrows);
static int32_t tsdbRestoreCfg(STsdbRepo *pRepo, STsdbCfg *pCfg);
static int32_t tsdbGetDataDirName(STsdbRepo *pRepo, char *fname);
static void * tsdbCommitData(void *arg);
static int tsdbCommitToFile(STsdbRepo *pRepo, int fid, SSkipListIterator **iters, SRWHelper *pHelper,
SDataCols *pDataCols);
static TSKEY tsdbNextIterKey(SSkipListIterator *pIter);
static int tsdbHasDataToCommit(SSkipListIterator **iters, int nIters, TSKEY minKey, TSKEY maxKey);
static void tsdbAlterCompression(STsdbRepo *pRepo, int8_t compression);
static void tsdbAlterKeep(STsdbRepo *pRepo, int32_t keep);
static void tsdbAlterMaxTables(STsdbRepo *pRepo, int32_t maxTables);
static int32_t tsdbSaveConfig(STsdbRepo *pRepo);
#define TSDB_GET_TABLE_BY_ID(pRepo, sid) (((STSDBRepo *)pRepo)->pTableList)[sid]
#define TSDB_GET_TABLE_BY_NAME(pRepo, name)
#define TSDB_IS_REPO_ACTIVE(pRepo) ((pRepo)->state == TSDB_REPO_STATE_ACTIVE)
#define TSDB_IS_REPO_CLOSED(pRepo) ((pRepo)->state == TSDB_REPO_STATE_CLOSED)
#define TSDB_META_FILE_NAME "meta"
#define TSDB_META_FILE_INDEX 10000000
/**
* Set the default TSDB configuration
*/
void tsdbSetDefaultCfg(STsdbCfg *pCfg) {
if (pCfg == NULL) return;
pCfg->precision = -1;
pCfg->tsdbId = 0;
pCfg->maxTables = -1;
pCfg->daysPerFile = -1;
pCfg->minRowsPerFileBlock = -1;
pCfg->maxRowsPerFileBlock = -1;
pCfg->keep = -1;
pCfg->compression = TWO_STAGE_COMP;
// Function declaration
int32_t tsdbCreateRepo(char *rootDir, STsdbCfg *pCfg) {
if (mkdir(rootDir, 0755) < 0) {
tsdbError("vgId:%d failed to create rootDir %s since %s", pCfg->tsdbId, rootDir, strerror(errno));
terrno = TAOS_SYSTEM_ERROR(errno);
return -1;
}
if (tsdbCheckAndSetDefaultCfg(pCfg) < 0) return -1;
if (tsdbSetRepoEnv(rootDir, pCfg) < 0) return -1;
tsdbTrace(
"vgId%d tsdb env create succeed! cacheBlockSize %d totalBlocks %d maxTables %d daysPerFile %d keep "
"%d minRowsPerFileBlock %d maxRowsPerFileBlock %d precision %d compression %d",
pCfg->tsdbId, pCfg->cacheBlockSize, pCfg->totalBlocks, pCfg->maxTables, pCfg->daysPerFile, pCfg->keep,
pCfg->minRowsPerFileBlock, pCfg->maxRowsPerFileBlock, pCfg->precision, pCfg->compression);
return 0;
}
int32_t tsdbDropRepo(char *rootDir) {
return tsdbUnsetRepoEnv(rootDir);
}
TSDB_REPO_T *tsdbOpenRepo(char *rootDir, STsdbAppH *pAppH) {
STsdbCfg config = {0};
STsdbRepo *pRepo = NULL;
if (tsdbLoadConfig(rootDir, &config) < 0) {
tsdbError("failed to open repo in rootDir %s since %s", rootDir, tstrerror(terrno));
return NULL;
}
pRepo = tsdbNewRepo(rootDir, pAppH, &config);
if (pRepo == NULL) {
tsdbError("failed to open repo in rootDir %s since %s", rootDir, tstrerror(terrno));
return NULL;
}
if (tsdbOpenMeta(pRepo) < 0) {
tsdbError("vgId:%d failed to open meta since %s", REPO_ID(pRepo), tstrerror(terrno));
goto _err;
}
if (tsdbOpenBufPool(pRepo) < 0) {
tsdbError("vgId:%d failed to open buffer pool since %s", REPO_ID(pRepo), tstrerror(terrno));
goto _err;
}
if (tsdbOpenFileH(pRepo) < 0) {
tsdbError("vgId:%d failed to open file handle since %s", REPO_ID(pRepo), tstrerror(terrno));
goto _err;
}
// // Restore key from file
// if (tsdbRestoreInfo(pRepo) < 0) {
// tsdbFreeCache(pRepo->tsdbCache);
// tsdbFreeMeta(pRepo->tsdbMeta);
// tsdbCloseFileH(pRepo->tsdbFileH);
// free(pRepo->rootDir);
// free(pRepo);
// return NULL;
// }
// pRepo->state = TSDB_REPO_STATE_ACTIVE;
tsdbTrace("vgId:%d open tsdb repository succeed!", REPO_ID(pRepo));
return (TSDB_REPO_T *)pRepo;
_err:
tsdbCloseRepo(pRepo, false);
tsdbFreeRepo(pRepo);
return NULL;
}
int32_t tsdbCloseRepo(TSDB_REPO_T *repo, int toCommit) {
// TODO
// STsdbRepo *pRepo = (STsdbRepo *)repo;
// if (pRepo == NULL) return 0;
// int id = pRepo->config.tsdbId;
// pRepo->state = TSDB_REPO_STATE_CLOSED;
// tsdbLockRepo(repo);
// if (pRepo->commit) {
// tsdbUnLockRepo(repo);
// return -1;
// }
// pRepo->commit = 1;
// // Loop to move pData to iData
// for (int i = 1; i < pRepo->config.maxTables; i++) {
// STable *pTable = pRepo->tsdbMeta->tables[i];
// if (pTable != NULL && pTable->mem != NULL) {
// pTable->imem = pTable->mem;
// pTable->mem = NULL;
// }
// }
// // TODO: Loop to move mem to imem
// pRepo->tsdbCache->imem = pRepo->tsdbCache->mem;
// pRepo->tsdbCache->mem = NULL;
// pRepo->tsdbCache->curBlock = NULL;
// tsdbUnLockRepo(repo);
// if (pRepo->appH.notifyStatus) pRepo->appH.notifyStatus(pRepo->appH.appH, TSDB_STATUS_COMMIT_START);
// if (toCommit) tsdbCommitData((void *)repo);
// tsdbCloseFileH(pRepo->tsdbFileH);
// tsdbFreeMeta(pRepo->tsdbMeta);
// tsdbFreeCache(pRepo->tsdbCache);
// tfree(pRepo->rootDir);
// tfree(pRepo);
// tsdbTrace("vgId:%d repository is closed!", id);
return 0;
}
int32_t tsdbInsertData(TSDB_REPO_T *repo, SSubmitMsg *pMsg, SShellSubmitRspMsg *pRsp) {
STsdbRepo * pRepo = (STsdbRepo *)repo;
SSubmitMsgIter msgIter;
if (tsdbInitSubmitMsgIter(pMsg, &msgIter) < 0) {
tsdbError("vgId:%d submit message is messed up", REPO_ID(pRepo));
return terrno;
}
SSubmitBlk *pBlock = NULL;
int32_t code = TSDB_CODE_SUCCESS;
int32_t affectedrows = 0;
TSKEY now = taosGetTimestamp(pRepo->config.precision);
while ((pBlock = tsdbGetSubmitMsgNext(&msgIter)) != NULL) {
if ((code = tsdbInsertDataToTable(pRepo, pBlock, now, &affectedrows)) != TSDB_CODE_SUCCESS) {
return code;
}
}
pRsp->affectedRows = htonl(affectedrows);
return code;
}
uint32_t tsdbGetFileInfo(TSDB_REPO_T *repo, char *name, uint32_t *index, uint32_t eindex, int32_t *size) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
// STsdbMeta *pMeta = pRepo->tsdbMeta;
STsdbFileH *pFileH = pRepo->tsdbFileH;
uint32_t magic = 0;
char fname[256] = "\0";
struct stat fState;
tsdbTrace("vgId:%d name:%s index:%d eindex:%d", pRepo->config.tsdbId, name, *index, eindex);
ASSERT(*index <= eindex);
char *sdup = strdup(pRepo->rootDir);
char *prefix = dirname(sdup);
if (name[0] == 0) { // get the file from index or after, but not larger than eindex
int fid = (*index) / 3;
if (pFileH->numOfFGroups == 0 || fid > pFileH->fGroup[pFileH->numOfFGroups - 1].fileId) {
if (*index <= TSDB_META_FILE_INDEX && TSDB_META_FILE_INDEX <= eindex) {
tsdbGetMetaFileName(pRepo->rootDir, fname);
*index = TSDB_META_FILE_INDEX;
} else {
tfree(sdup);
return 0;
}
} else {
SFileGroup *pFGroup =
taosbsearch(&fid, pFileH->fGroup, pFileH->numOfFGroups, sizeof(SFileGroup), compFGroupKey, TD_GE);
if (pFGroup->fileId == fid) {
strcpy(fname, pFGroup->files[(*index) % 3].fname);
} else {
if (pFGroup->fileId * 3 + 2 < eindex) {
strcpy(fname, pFGroup->files[0].fname);
*index = pFGroup->fileId * 3;
} else {
tfree(sdup);
return 0;
}
}
}
strcpy(name, fname + strlen(prefix));
} else { // get the named file at the specified index. If not there, return 0
if (*index == TSDB_META_FILE_INDEX) { // get meta file
tsdbGetMetaFileName(pRepo->rootDir, fname);
} else {
int fid = (*index) / 3;
SFileGroup *pFGroup = tsdbSearchFGroup(pFileH, fid);
if (pFGroup == NULL) { // not found
tfree(sdup);
return 0;
}
SFile *pFile = &pFGroup->files[(*index) % 3];
strcpy(fname, pFile->fname);
}
}
if (stat(fname, &fState) < 0) {
tfree(sdup);
return 0;
}
tfree(sdup);
*size = fState.st_size;
magic = *size;
return magic;
}
int tsdbUpdateTagValue(TSDB_REPO_T *repo, SUpdateTableTagValMsg *pMsg) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
STsdbMeta *pMeta = pRepo->tsdbMeta;
int16_t tversion = htons(pMsg->tversion);
STable *pTable = tsdbGetTableByUid(pMeta, htobe64(pMsg->uid));
if (pTable == NULL) return TSDB_CODE_TDB_INVALID_TABLE_ID;
if (pTable->tableId.tid != htonl(pMsg->tid)) return TSDB_CODE_TDB_INVALID_TABLE_ID;
if (pTable->type != TSDB_CHILD_TABLE) {
tsdbError("vgId:%d failed to update tag value of table %s since its type is %d", pRepo->config.tsdbId,
varDataVal(pTable->name), pTable->type);
return TSDB_CODE_TDB_INVALID_TABLE_TYPE;
}
if (schemaVersion(tsdbGetTableTagSchema(pMeta, pTable)) < tversion) {
tsdbTrace("vgId:%d server tag version %d is older than client tag version %d, try to config", pRepo->config.tsdbId,
schemaVersion(tsdbGetTableTagSchema(pMeta, pTable)), tversion);
void *msg = (*pRepo->appH.configFunc)(pRepo->config.tsdbId, htonl(pMsg->tid));
if (msg == NULL) {
return terrno;
}
// Deal with error her
STableCfg *pTableCfg = tsdbCreateTableCfgFromMsg(msg);
STable * super = tsdbGetTableByUid(pMeta, pTableCfg->superUid);
ASSERT(super != NULL);
int32_t code = tsdbUpdateTable(pMeta, super, pTableCfg);
if (code != TSDB_CODE_SUCCESS) {
return code;
}
tsdbClearTableCfg(pTableCfg);
rpcFreeCont(msg);
}
STSchema *pTagSchema = tsdbGetTableTagSchema(pMeta, pTable);
if (schemaVersion(pTagSchema) > tversion) {
tsdbError(
"vgId:%d failed to update tag value of table %s since version out of date, client tag version:%d server tag "
"version:%d",
pRepo->config.tsdbId, varDataVal(pTable->name), tversion, schemaVersion(pTable->tagSchema));
return TSDB_CODE_TDB_TAG_VER_OUT_OF_DATE;
}
if (schemaColAt(pTagSchema, DEFAULT_TAG_INDEX_COLUMN)->colId == htons(pMsg->colId)) {
tsdbRemoveTableFromIndex(pMeta, pTable);
}
// TODO: remove table from index if it is the first column of tag
tdSetKVRowDataOfCol(&pTable->tagVal, htons(pMsg->colId), htons(pMsg->type), pMsg->data);
if (schemaColAt(pTagSchema, DEFAULT_TAG_INDEX_COLUMN)->colId == htons(pMsg->colId)) {
tsdbAddTableIntoIndex(pMeta, pTable);
}
return TSDB_CODE_SUCCESS;
}
void tsdbStartStream(TSDB_REPO_T *repo) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
STsdbMeta *pMeta = pRepo->tsdbMeta;
for (int i = 0; i < pRepo->config.maxTables; i++) {
STable *pTable = pMeta->tables[i];
if (pTable && pTable->type == TSDB_STREAM_TABLE) {
pTable->cqhandle = (*pRepo->appH.cqCreateFunc)(pRepo->appH.cqH, pTable->tableId.uid, pTable->tableId.tid,
pTable->sql, tsdbGetTableSchema(pMeta, pTable));
}
}
}
STsdbCfg *tsdbGetCfg(const TSDB_REPO_T *repo) {
ASSERT(repo != NULL);
return &((STsdbRepo *)repo)->config;
}
// ----------------- INTERNAL FUNCTIONS -----------------
char *tsdbGetMetaFileName(char *rootDir) {
int tlen = strlen(rootDir) + strlen(TSDB_META_FILE_NAME) + 2;
char *fname = calloc(1, tlen);
if (fname == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
return NULL;
}
snprintf(fname, tlen, "%s/%s", rootDir, TSDB_META_FILE_NAME);
return fname;
}
int tsdbLockRepo(STsdbRepo *pRepo) {
int code = pthread_mutex_lock(&pRepo->mutex);
if (code != 0) {
tsdbError("vgId:%d failed to lock tsdb since %s", REPO_ID(pRepo), strerror(errno));
terrno = TAOS_SYSTEM_ERROR(code);
return -1;
}
return 0;
}
int tsdbUnlockRepo(STsdbRepo *pRepo) {
int code = pthread_mutex_unlock(&pRepo->mutex);
if (code != 0) {
tsdbError("vgId:%d failed to unlock tsdb since %s", REPO_ID(pRepo), strerror(errno));
terrno = TAOS_SYSTEM_ERROR(code);
return -1;
}
return 0;
}
/**
* Create a configuration for TSDB default
* @return a pointer to a configuration. the configuration object
* must call tsdbFreeCfg to free memory after usage
*/
STsdbCfg *tsdbCreateDefaultCfg() {
STsdbCfg *pCfg = (STsdbCfg *)malloc(sizeof(STsdbCfg));
if (pCfg == NULL) return NULL;
void *tsdbCommitData(void *arg) {
STsdbRepo *pRepo = (STsdbRepo *)arg;
STsdbMeta *pMeta = pRepo->tsdbMeta;
ASSERT(pRepo->imem != NULL);
ASSERT(pRepo->commit == 1);
tsdbPrint("vgId:%d start to commit! keyFirst " PRId64 " keyLast " PRId64 " numOfRows " PRId64, REPO_ID(pRepo),
pRepo->imem->keyFirst, pRepo->imem->keyLast, pRepo->imem->numOfRows);
// STsdbMeta * pMeta = pRepo->tsdbMeta;
// STsdbCache *pCache = pRepo->tsdbCache;
// STsdbCfg * pCfg = &(pRepo->config);
// SDataCols * pDataCols = NULL;
// SRWHelper whelper = {{0}};
// if (pCache->imem == NULL) return NULL;
tsdbPrint("vgId:%d, starting to commit....", pRepo->config.tsdbId);
// Create the iterator to read from cache
SSkipListIterator **iters = tsdbCreateTableIters(pRepo);
if (iters == NULL) {
tsdbError("vgId:%d failed to create table iterators since %s", REPO_ID(pRepo), tstrerror(terrno));
// TODO: deal with the error here
return NULL;
}
if (tsdbInitWriteHelper(&whelper, pRepo) < 0) {
tsdbError("vgId:%d failed to init write helper since %s", REPO_ID(pRepo), tstrerror(terrno));
// TODO
goto _exit;
}
if ((pDataCols = tdNewDataCols(pMeta->maxRowBytes, pMeta->maxCols, pCfg->maxRowsPerFileBlock)) == NULL) {
tsdbError("vgId:%d failed to init data cols with maxRowBytes %d maxCols %d since %s", REPO_ID(pRepo),
pMeta->maxRowBytes, pMeta->maxCols, tstrerror(terrno));
// TODO
goto _exit;
}
int sfid = tsdbGetKeyFileId(pCache->imem->keyFirst, pCfg->daysPerFile, pCfg->precision);
int efid = tsdbGetKeyFileId(pCache->imem->keyLast, pCfg->daysPerFile, pCfg->precision);
// Loop to commit to each file
for (int fid = sfid; fid <= efid; fid++) {
if (tsdbCommitToFile(pRepo, fid, iters, &whelper, pDataCols) < 0) {
ASSERT(false);
goto _exit;
}
}
// Do retention actions
tsdbFitRetention(pRepo);
if (pRepo->appH.notifyStatus) pRepo->appH.notifyStatus(pRepo->appH.appH, TSDB_STATUS_COMMIT_OVER);
_exit:
tdFreeDataCols(pDataCols);
tsdbDestroyTableIters(iters, pCfg->maxTables);
tsdbDestroyHelper(&whelper);
tsdbLockRepo(arg);
tdListMove(pCache->imem->list, pCache->pool.memPool);
tsdbAdjustCacheBlocks(pCache);
tdListFree(pCache->imem->list);
free(pCache->imem);
pCache->imem = NULL;
pRepo->commit = 0;
for (int i = 1; i < pCfg->maxTables; i++) {
STable *pTable = pMeta->tables[i];
if (pTable && pTable->imem) {
tsdbFreeMemTable(pTable->imem);
pTable->imem = NULL;
}
}
tsdbUnLockRepo(arg);
tsdbPrint("vgId:%d, commit over....", pRepo->config.tsdbId);
return NULL;
}
// ----------------- LOCAL FUNCTIONS -----------------
static int32_t tsdbCheckAndSetDefaultCfg(STsdbCfg *pCfg) {
// Check precision
if (pCfg->precision == -1) {
pCfg->precision = TSDB_DEFAULT_PRECISION;
} else {
if (!IS_VALID_PRECISION(pCfg->precision)) {
tsdbError("vgId:%d invalid precision configuration %d", pCfg->tsdbId, pCfg->precision);
goto _err;
}
}
// Check compression
if (pCfg->compression == -1) {
pCfg->compression = TSDB_DEFAULT_COMPRESSION;
} else {
if (!IS_VALID_COMPRESSION(pCfg->compression)) {
tsdbError("vgId:%d invalid compression configuration %d", pCfg->tsdbId, pCfg->precision);
goto _err;
}
}
// Check tsdbId
if (pCfg->tsdbId < 0) {
tsdbError("vgId:%d invalid vgroup ID", pCfg->tsdbId);
goto _err;
}
// Check maxTables
if (pCfg->maxTables == -1) {
pCfg->maxTables = TSDB_DEFAULT_TABLES;
} else {
if (pCfg->maxTables < TSDB_MIN_TABLES || pCfg->maxTables > TSDB_MAX_TABLES) {
tsdbError("vgId:%d invalid maxTables configuration! maxTables %d TSDB_MIN_TABLES %d TSDB_MAX_TABLES %d",
pCfg->tsdbId, pCfg->maxTables, TSDB_MIN_TABLES, TSDB_MAX_TABLES);
goto _err;
}
}
// Check daysPerFile
if (pCfg->daysPerFile == -1) {
pCfg->daysPerFile = TSDB_DEFAULT_DAYS_PER_FILE;
} else {
if (pCfg->daysPerFile < TSDB_MIN_DAYS_PER_FILE || pCfg->daysPerFile > TSDB_MAX_DAYS_PER_FILE) {
tsdbError(
"vgId:%d invalid daysPerFile configuration! daysPerFile %d TSDB_MIN_DAYS_PER_FILE %d TSDB_MAX_DAYS_PER_FILE "
"%d",
pCfg->tsdbId, pCfg->daysPerFile, TSDB_MIN_DAYS_PER_FILE, TSDB_MAX_DAYS_PER_FILE);
goto _err;
}
}
// Check minRowsPerFileBlock and maxRowsPerFileBlock
if (pCfg->minRowsPerFileBlock == -1) {
pCfg->minRowsPerFileBlock = TSDB_DEFAULT_MIN_ROW_FBLOCK;
} else {
if (pCfg->minRowsPerFileBlock < TSDB_MIN_MIN_ROW_FBLOCK || pCfg->minRowsPerFileBlock > TSDB_MAX_MIN_ROW_FBLOCK) {
tsdbError(
"vgId:%d invalid minRowsPerFileBlock configuration! minRowsPerFileBlock %d TSDB_MIN_MIN_ROW_FBLOCK %d "
"TSDB_MAX_MIN_ROW_FBLOCK %d",
pCfg->tsdbId, pCfg->minRowsPerFileBlock, TSDB_MIN_MIN_ROW_FBLOCK, TSDB_MAX_MIN_ROW_FBLOCK);
goto _err;
}
}
if (pCfg->maxRowsPerFileBlock == -1) {
pCfg->maxRowsPerFileBlock = TSDB_DEFAULT_MAX_ROW_FBLOCK;
} else {
if (pCfg->maxRowsPerFileBlock < TSDB_MIN_MAX_ROW_FBLOCK || pCfg->maxRowsPerFileBlock > TSDB_MAX_MAX_ROW_FBLOCK) {
tsdbError(
"vgId:%d invalid maxRowsPerFileBlock configuration! maxRowsPerFileBlock %d TSDB_MIN_MAX_ROW_FBLOCK %d "
"TSDB_MAX_MAX_ROW_FBLOCK %d",
pCfg->tsdbId, pCfg->maxRowsPerFileBlock, TSDB_MIN_MIN_ROW_FBLOCK, TSDB_MAX_MIN_ROW_FBLOCK);
goto _err;
}
}
if (pCfg->minRowsPerFileBlock > pCfg->maxRowsPerFileBlock) {
tsdbError("vgId:%d invalid configuration! minRowsPerFileBlock %d maxRowsPerFileBlock %d" pCfg->tsdbId,
pCfg->minRowsPerFileBlock, pCfg->maxRowsPerFileBlock);
goto _err;
}
// Check keep
if (pCfg->keep == -1) {
pCfg->keep = TSDB_DEFAULT_KEEP;
} else {
if (pCfg->keep < TSDB_MIN_KEEP || pCfg->keep > TSDB_MAX_KEEP) {
tsdbError(
"vgId:%d invalid keep configuration! keep %d TSDB_MIN_KEEP %d "
"TSDB_MAX_KEEP %d",
pCfg->tsdbId, pCfg->keep, TSDB_MIN_KEEP, TSDB_MAX_KEEP);
goto _err;
}
}
return 0;
_err:
terrno = TSDB_CODE_TDB_INVALID_CONFIG;
return -1;
}
static int32_t tsdbSetRepoEnv(char *rootDir, STsdbCfg *pCfg) {
if (tsdbSaveConfig(rootDir, pCfg) < 0) {
tsdbError("vgId:%d failed to set TSDB environment since %s", pCfg->tsdbId, tstrerror(terrno));
return -1;
}
char *dirName = tsdbGetDataDirName(rootDir);
if (dirName == NULL) return -1;
if (mkdir(dirName, 0755) < 0) {
tsdbError("vgId:%d failed to create directory %s since %s", pCfg->tsdbId, dirName, strerror(errno));
errno = TAOS_SYSTEM_ERROR(errno);
free(dirName);
return -1;
}
free(dirName);
char *fname = tsdbGetMetaFileName(rootDir);
if (fname == NULL) return -1;
if (tdCreateKVStore(fname) < 0) {
tsdbError("vgId:%d failed to open KV store since %s", pCfg->tsdbId, tstrerror(terrno));
free(fname);
return -1;
}
free(fname);
return 0;
}
static int32_t tsdbUnsetRepoEnv(char *rootDir) {
taosRemoveDir(rootDir);
tsdbTrace("repository %s is removed", rootDir);
return 0;
}
static int32_t tsdbSaveConfig(char *rootDir, STsdbCfg *pCfg) {
int fd = -1;
char *fname = NULL;
fname = tsdbGetCfgFname(rootDir);
if (fname == NULL) {
tsdbError("vgId:%d failed to save configuration since %s", pCfg->tsdbId, tstrerror(terrno));
goto _err;
}
fd = open(fname, O_WRONLY | O_CREAT, 0755);
if (fd < 0) {
tsdbError("vgId:%d failed to open file %s since %s", pCfg->tsdbId, fname, strerror(errno));
terrno = TAOS_SYSTEM_ERROR(errno);
goto _err
}
if (twrite(fd, (void *)pCfg, sizeof(STsdbCfg)) < sizeof(STsdbCfg)) {
tsdbError("vgId:%d failed to write %d bytes to file %s since %s", pCfg->tsdbId, sizeof(STsdbCfg), fname,
strerror(errno));
terrno = TAOS_SYSTEM_ERROR(errno);
goto _err;
}
if (fsync(fd) < 0) {
tsdbError("vgId:%d failed to fsync file %s since %s", pCfg->tsdbId, fname, strerror(errno));
terrno = TAOS_SYSTEM_ERROR(errno);
goto _err;
}
free(fname);
close(fd);
return 0;
_err:
tfree(fname);
if (fd > 0) close(fd);
return -1;
}
static int tsdbLoadConfig(char *rootDir, STsdbCfg *pCfg) {
char *fname = NULL;
int fd = -1;
fname = tsdbGetCfgFname(rootDir);
if (fname == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
fd = open(fname, O_RDONLY);
if (fd < 0) {
tsdbError("failed to open file %s since %s", fname, strerror(errno));
terrno = TAOS_SYSTEM_ERROR(errno);
goto _err;
}
if (tread(fd, (void *)pCfg, sizeof(*pCfg)) < sizeof(*pCfg)) {
tsdbError("failed to read %d bytes from file %s since %s", sizeof(*pCfg), fname, strerror(errno));
terrno = TAOS_SYSTEM_ERROR(errno);
goto _err;
}
tfree(fname);
close(fd);
return 0;
_err:
tfree(fname);
if (fd > 0) close(fd);
return -1;
}
static char *tsdbGetCfgFname(char *rootDir) {
int tlen = strlen(rootDir) + strlen(TSDB_CFG_FILE_NAME) + 2;
char *fname = calloc(1, tlen);
if (fname == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
return NULL;
}
snprintf(fname, tlen, "%s/%s", rootDir, TSDB_CFG_FILE_NAME);
return fname;
}
static char *tsdbGetDataDirName(char *rootDir) {
int tlen = strlen(rootDir) + strlen(TSDB_DATA_DIR_NAME) + 2;
char *fname = calloc(1, tlen);
if (fname == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
return NULL;
}
snprintf(fname, tlen, "%s/%s", rootDir, TSDB_DATA_DIR_NAME);
return fname;
}
static STsdbRepo *tsdbNewRepo(char *rootDir, STsdbAppH *pAppH, STsdbCfg *pCfg) {
STsdbRepo *pRepo = (STsdbRepo *)calloc(1, sizeof(STsdbRepo));
if (pRepo == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
int code = pthread_mutex_init(&pRepo->mutex, NULL);
if (code != 0) {
terrno = TAOS_SYSTEM_ERROR(code);
goto _err;
}
pRepo->rootDir = strdup(rootDir);
if (pRepo->rootDir == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pRepo->config = *pCfg;
pRepo->appH = *pAppH;
pRepo->tsdbMeta = tsdbNewMeta(pCfg);
if (pRepo->tsdbMeta == NULL) {
tsdbError("vgId:%d failed to create meta since %s", REPO_ID(pRepo), tstrerror(terrno));
goto _err;
}
pRepo->pPool = tsdbNewBufPool(pCfg);
if (pRepo->pPool == NULL) {
tsdbError("vgId:%d failed to create buffer pool since %s", REPO_ID(pRepo), tstrerror(terrno));
goto _err;
}
pRepo->tsdbFileH = tsdbNewFileH(pRepo);
if (pRepo->tsdbFileH == NULL) {
tsdbError("vgId:%d failed to create file handle since %s", REPO_ID(pRepo), tstrerror(terrno));
goto _err;
}
return pRepo;
_err:
tsdbFreeRepo(pRepo);
return NULL;
}
static void tsdbFreeRepo(STsdbRepo *pRepo) {
if (pRepo) {
tsdbFreeFileH(pRepo->tsdbFileH);
tsdbFreeBufPool(pRepo->pPool);
tsdbFreeMeta(pRepo->tsdbMeta);
tsdbFreeMemTable(pRepo->mem);
tsdbFreeMemTable(pRepo->imem);
tfree(pRepo->rootDir);
pthread_mutex_destroy(&pRepo->mutex);
free(pRepo);
}
}
static int tsdbInitSubmitMsgIter(SSubmitMsg *pMsg, SSubmitMsgIter *pIter) {
if (pMsg == NULL) {
terrno = TSDB_CODE_TDB_SUBMIT_MSG_MSSED_UP;
return -1;
}
pMsg->length = htonl(pMsg->length);
pMsg->numOfBlocks = htonl(pMsg->numOfBlocks);
pMsg->compressed = htonl(pMsg->compressed);
pIter->totalLen = pMsg->length;
pIter->len = TSDB_SUBMIT_MSG_HEAD_SIZE;
if (pMsg->length <= TSDB_SUBMIT_MSG_HEAD_SIZE) {
terrno = TSDB_CODE_TDB_SUBMIT_MSG_MSSED_UP;
return -1;
} else {
pIter->pBlock = pMsg->blocks;
}
return 0;
}
static int32_t tsdbInsertDataToTable(STsdbRepo *pRepo, SSubmitBlk *pBlock, TSKEY now, int32_t *affectedrows) {
STsdbMeta *pMeta = pRepo->tsdbMeta;
int64_t points = 0;
STable *pTable == tsdbGetTableByUid(pMeta, pBlock->uid);
if (pTable == NULL || TABLE_TID(pTable)) {
tsdbError("vgId:%d failed to get table to insert data, uid " PRIu64 " tid %d", REPO_ID(pRepo), pBlock->uid,
pBlock->tid);
terrno = TSDB_CODE_TDB_INVALID_TABLE_ID;
return -1;
}
if (TABLE_TYPE(pTable) == TSDB_SUPER_TABLE) {
tsdbError("vgId:%d invalid action trying to insert a super table %s", REPO_ID(pRepo), TABLE_CHAR_NAME(pTable));
terrno = TSDB_CODE_TDB_INVALID_ACTION;
return -1;
}
// // Check schema version
// int32_t tversion = pBlock->sversion;
// STSchema * pSchema = tsdbGetTableSchema(pMeta, pTable);
// ASSERT(pSchema != NULL);
// int16_t nversion = schemaVersion(pSchema);
// if (tversion > nversion) {
// tsdbTrace("vgId:%d table:%s tid:%d server schema version %d is older than clien version %d, try to config.",
// pRepo->config.tsdbId, varDataVal(pTable->name), pTable->tableId.tid, nversion, tversion);
// void *msg = (*pRepo->appH.configFunc)(pRepo->config.tsdbId, pTable->tableId.tid);
// if (msg == NULL) {
// return terrno;
// }
// // Deal with error her
// STableCfg *pTableCfg = tsdbCreateTableCfgFromMsg(msg);
// STable *pTableUpdate = NULL;
// if (pTable->type == TSDB_CHILD_TABLE) {
// pTableUpdate = tsdbGetTableByUid(pMeta, pTableCfg->superUid);
// } else {
// pTableUpdate = pTable;
// }
// int32_t code = tsdbUpdateTable(pMeta, pTableUpdate, pTableCfg);
// if (code != TSDB_CODE_SUCCESS) {
// return code;
// }
// tsdbClearTableCfg(pTableCfg);
// rpcFreeCont(msg);
// } else {
// if (tsdbGetTableSchemaByVersion(pMeta, pTable, tversion) == NULL) {
// tsdbError("vgId:%d table:%s tid:%d invalid schema version %d from client", pRepo->config.tsdbId,
// varDataVal(pTable->name), pTable->tableId.tid, tversion);
// return TSDB_CODE_TDB_TABLE_SCHEMA_VERSION;
// }
// }
SSubmitBlkIter blkIter = {0};
SDataRow row = NULL;
TSKEY minKey = now - tsMsPerDay[pRepo->config.precision] * pRepo->config.keep;
TSKEY maxKey = now + tsMsPerDay[pRepo->config.precision] * pRepo->config.daysPerFile;
tsdbInitSubmitBlkIter(pBlock, &blkIter);
while ((row = tsdbGetSubmitBlkNext(&blkIter)) != NULL) {
if (dataRowKey(row) < minKey || dataRowKey(row) > maxKey) {
tsdbError("vgId:%d table %s tid %d uid %ld timestamp is out of range! now " PRId64 " maxKey " PRId64
" minKey " PRId64,
REPO_ID(pRepo), TABLE_CHAR_NAME(pTable), TABLE_TID(pTable), TALBE_UID(pTable), now, minKey, maxKey);
return TSDB_CODE_TDB_TIMESTAMP_OUT_OF_RANGE;
}
if (tdInsertRowToTable(pRepo, row, pTable) < 0) {
return -1;
}
(*affectedrows)++;
points++;
}
atomic_fetch_add_64(&(pRepo->stat.pointsWritten), points * (pSchema->numOfCols));
atomic_fetch_add_64(&(pRepo->stat.totalStorage), points * pSchema->vlen);
return 0;
}
static SSubmitBlk *tsdbGetSubmitMsgNext(SSubmitMsgIter *pIter) {
SSubmitBlk *pBlock = pIter->pBlock;
if (pBlock == NULL) return NULL;
pBlock->len = htonl(pBlock->len);
pBlock->numOfRows = htons(pBlock->numOfRows);
pBlock->uid = htobe64(pBlock->uid);
pBlock->tid = htonl(pBlock->tid);
pBlock->sversion = htonl(pBlock->sversion);
pBlock->padding = htonl(pBlock->padding);
pIter->len = pIter->len + sizeof(SSubmitBlk) + pBlock->len;
if (pIter->len >= pIter->totalLen) {
pIter->pBlock = NULL;
} else {
pIter->pBlock = (SSubmitBlk *)((char *)pBlock + pBlock->len + sizeof(SSubmitBlk));
}
return pBlock;
}
static SDataRow tsdbGetSubmitBlkNext(SSubmitBlkIter *pIter) {
SDataRow row = pIter->row;
if (row == NULL) return NULL;
pIter->len += dataRowLen(row);
if (pIter->len >= pIter->totalLen) {
pIter->row = NULL;
} else {
pIter->row = (char *)row + dataRowLen(row);
}
return row;
}
static int32_t tsdbInsertRowToTable(STsdbRepo *pRepo, SDataRow row, STable *pTable) {
// TODO
int32_t level = 0;
int32_t headSize = 0;
if (pTable->mem == NULL) {
pTable->mem = (SMemTable *)calloc(1, sizeof(SMemTable));
if (pTable->mem == NULL) return -1;
pTable->mem->pData =
tSkipListCreate(5, TSDB_DATA_TYPE_TIMESTAMP, TYPE_BYTES[TSDB_DATA_TYPE_TIMESTAMP], 0, 0, 0, getTSTupleKey);
pTable->mem->keyFirst = INT64_MAX;
pTable->mem->keyLast = 0;
}
tSkipListNewNodeInfo(pTable->mem->pData, &level, &headSize);
TSKEY key = dataRowKey(row);
// printf("insert:%lld, size:%d\n", key, pTable->mem->numOfRows);
// Copy row into the memory
SSkipListNode *pNode = tsdbAllocFromCache(pRepo->tsdbCache, headSize + dataRowLen(row), key);
if (pNode == NULL) {
// TODO: deal with allocate failure
}
pNode->level = level;
dataRowCpy(SL_GET_NODE_DATA(pNode), row);
// Insert the skiplist node into the data
if (pTable->mem == NULL) {
pTable->mem = (SMemTable *)calloc(1, sizeof(SMemTable));
if (pTable->mem == NULL) return -1;
pTable->mem->pData =
tSkipListCreate(5, TSDB_DATA_TYPE_TIMESTAMP, TYPE_BYTES[TSDB_DATA_TYPE_TIMESTAMP], 0, 0, 0, getTSTupleKey);
pTable->mem->keyFirst = INT64_MAX;
pTable->mem->keyLast = 0;
}
tSkipListPut(pTable->mem->pData, pNode);
if (key > pTable->mem->keyLast) pTable->mem->keyLast = key;
if (key < pTable->mem->keyFirst) pTable->mem->keyFirst = key;
if (key > pTable->lastKey) pTable->lastKey = key;
pTable->mem->numOfRows = tSkipListGetSize(pTable->mem->pData);
tsdbTrace("vgId:%d, tid:%d, uid:%" PRId64 ", table:%s a row is inserted to table! key:%" PRId64, pRepo->config.tsdbId,
pTable->tableId.tid, pTable->tableId.uid, varDataVal(pTable->name), dataRowKey(row));
return 0;
}
static SSkipListIterator **tsdbCreateTableIters(STsdbRepo *pRepo) {
STsdbCfg *pCfg = &(pRepo->config);
SSkipListIterator **iters = (SSkipListIterator **)calloc(pCfg->maxTables, sizeof(SSkipListIterator *));
if (iters == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
return NULL;
}
for (int tid = 1; tid < maxTables; tid++) {
STable *pTable = pMeta->tables[tid];
if (pTable == NULL || pTable->imem == NULL || pTable->imem->numOfRows == 0) continue;
iters[tid] = tSkipListCreateIter(pTable->imem->pData);
if (iters[tid] == NULL) goto _err;
tsdbSetDefaultCfg(pCfg);
if (!tSkipListIterNext(iters[tid])) goto _err;
}
return pCfg;
}
return iters;
void tsdbFreeCfg(STsdbCfg *pCfg) {
if (pCfg != NULL) free(pCfg);
_err:
tsdbDestroyTableIters(iters, maxTables);
return NULL;
}
STsdbCfg *tsdbGetCfg(const TSDB_REPO_T *repo) {
assert(repo != NULL);
return &((STsdbRepo*)repo)->config;
}
static int tsdbCommitToFile(STsdbRepo *pRepo, int fid, SSkipListIterator **iters, SRWHelper *pHelper,
SDataCols *pDataCols) {
char dataDir[128] = {0};
STsdbMeta * pMeta = pRepo->tsdbMeta;
STsdbFileH *pFileH = pRepo->tsdbFileH;
STsdbCfg * pCfg = &pRepo->config;
SFileGroup *pGroup = NULL;
/**
* Create a new TSDB repository
* @param rootDir the TSDB repository root directory
* @param pCfg the TSDB repository configuration, upper layer need to free the pointer
* @param limiter the limitation tracker will implement in the future, make it void now
*
* @return a TSDB repository handle on success, NULL for failure
*/
int32_t tsdbCreateRepo(char *rootDir, STsdbCfg *pCfg) {
if (mkdir(rootDir, 0755) != 0) {
tsdbError("vgId:%d, failed to create rootDir! rootDir:%s, reason:%s", pCfg->tsdbId, rootDir, strerror(errno));
if (errno == EACCES) {
return TSDB_CODE_TDB_NO_DISK_PERMISSIONS;
} else if (errno == ENOSPC) {
return TSDB_CODE_TDB_NO_DISKSPACE;
} else if (errno == EEXIST) {
} else {
return TSDB_CODE_TDB_INIT_FAILED;
}
}
TSKEY minKey = 0, maxKey = 0;
tsdbGetKeyRangeOfFileId(pCfg->daysPerFile, pCfg->precision, fid, &minKey, &maxKey);
if (access(rootDir, F_OK | R_OK | W_OK) == -1) return -1;
// Check if there are data to commit to this file
int hasDataToCommit = tsdbHasDataToCommit(iters, pCfg->maxTables, minKey, maxKey);
if (!hasDataToCommit) return 0; // No data to commit, just return
if (tsdbCheckAndSetDefaultCfg(pCfg) < 0) {
return -1;
// Create and open files for commit
tsdbGetDataDirName(pRepo, dataDir);
if ((pGroup = tsdbCreateFGroup(pFileH, dataDir, fid, pCfg->maxTables)) == NULL) {
tsdbError("vgId:%d, failed to create file group %d", pRepo->config.tsdbId, fid);
goto _err;
}
STsdbRepo *pRepo = (STsdbRepo *)malloc(sizeof(STsdbRepo));
if (pRepo == NULL) {
return -1;
// Open files for write/read
if (tsdbSetAndOpenHelperFile(pHelper, pGroup) < 0) {
tsdbError("vgId:%d, failed to set helper file", pRepo->config.tsdbId);
goto _err;
}
pRepo->rootDir = strdup(rootDir);
pRepo->config = *pCfg;
// pRepo->limiter = limiter;
// Loop to commit data in each table
for (int tid = 1; tid < pCfg->maxTables; tid++) {
STable *pTable = pMeta->tables[tid];
if (pTable == NULL) continue;
// Create the environment files and directories
int32_t code = tsdbSetRepoEnv(pRepo);
free(pRepo->rootDir);
free(pRepo);
return code;
}
SSkipListIterator *pIter = iters[tid];
/**
* Close and free all resources taken by the repository
* @param repo the TSDB repository handle. The interface will free the handle too, so upper
* layer do NOT need to free the repo handle again.
*
* @return 0 for success, -1 for failure and the error number is set
*/
int32_t tsdbDropRepo(TSDB_REPO_T *repo) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
int id = pRepo->config.tsdbId;
// Set the helper and the buffer dataCols object to help to write this table
tsdbSetHelperTable(pHelper, pTable, pRepo);
tdInitDataCols(pDataCols, tsdbGetTableSchema(pMeta, pTable));
// Loop to write the data in the cache to files. If no data to write, just break the loop
int maxRowsToRead = pCfg->maxRowsPerFileBlock * 4 / 5;
int nLoop = 0;
while (true) {
int rowsRead = tsdbReadRowsFromCache(pMeta, pTable, pIter, maxKey, maxRowsToRead, pDataCols);
assert(rowsRead >= 0);
if (pDataCols->numOfRows == 0) break;
nLoop++;
ASSERT(dataColsKeyFirst(pDataCols) >= minKey && dataColsKeyFirst(pDataCols) <= maxKey);
ASSERT(dataColsKeyLast(pDataCols) >= minKey && dataColsKeyLast(pDataCols) <= maxKey);
pRepo->state = TSDB_REPO_STATE_CLOSED;
int rowsWritten = tsdbWriteDataBlock(pHelper, pDataCols);
ASSERT(rowsWritten != 0);
if (rowsWritten < 0) goto _err;
ASSERT(rowsWritten <= pDataCols->numOfRows);
tdPopDataColsPoints(pDataCols, rowsWritten);
maxRowsToRead = pCfg->maxRowsPerFileBlock * 4 / 5 - pDataCols->numOfRows;
}
// Free the metaHandle
tsdbFreeMeta(pRepo->tsdbMeta);
ASSERT(pDataCols->numOfRows == 0);
// Free the cache
tsdbFreeCache(pRepo->tsdbCache);
// Move the last block to the new .l file if neccessary
if (tsdbMoveLastBlockIfNeccessary(pHelper) < 0) {
tsdbError("vgId:%d, failed to move last block", pRepo->config.tsdbId);
goto _err;
}
// Destroy the repository info
tsdbDestroyRepoEnv(pRepo);
// Write the SCompBlock part
if (tsdbWriteCompInfo(pHelper) < 0) {
tsdbError("vgId:%d, failed to write compInfo part", pRepo->config.tsdbId);
goto _err;
}
}
free(pRepo->rootDir);
free(pRepo);
if (tsdbWriteCompIdx(pHelper) < 0) {
tsdbError("vgId:%d, failed to write compIdx part", pRepo->config.tsdbId);
goto _err;
}
tsdbTrace("vgId:%d, tsdb repository is dropped!", id);
tsdbCloseHelperFile(pHelper, 0);
// TODO: make it atomic with some methods
pGroup->files[TSDB_FILE_TYPE_HEAD] = pHelper->files.headF;
pGroup->files[TSDB_FILE_TYPE_DATA] = pHelper->files.dataF;
pGroup->files[TSDB_FILE_TYPE_LAST] = pHelper->files.lastF;
return 0;
_err:
ASSERT(false);
tsdbCloseHelperFile(pHelper, 1);
return -1;
}
#if 0
**
* Set the default TSDB configuration
*/
static int tsdbRestoreInfo(STsdbRepo *pRepo) {
STsdbMeta * pMeta = pRepo->tsdbMeta;
STsdbFileH *pFileH = pRepo->tsdbFileH;
......@@ -186,124 +1068,6 @@ _err:
return -1;
}
/**
* Open an existing TSDB storage repository
* @param tsdbDir the existing TSDB root directory
*
* @return a TSDB repository handle on success, NULL for failure and the error number is set
*/
TSDB_REPO_T *tsdbOpenRepo(char *rootDir, STsdbAppH *pAppH) {
char dataDir[128] = "\0";
if (access(rootDir, F_OK | W_OK | R_OK) < 0) {
return NULL;
}
STsdbRepo *pRepo = (STsdbRepo *)calloc(1, sizeof(STsdbRepo));
if (pRepo == NULL) {
return NULL;
}
pRepo->rootDir = strdup(rootDir);
tsdbRestoreCfg(pRepo, &(pRepo->config));
if (pAppH) pRepo->appH = *pAppH;
pRepo->tsdbMeta = tsdbInitMeta(rootDir, pRepo->config.maxTables, pRepo);
if (pRepo->tsdbMeta == NULL) {
free(pRepo->rootDir);
free(pRepo);
return NULL;
}
pRepo->tsdbCache = tsdbInitCache(pRepo->config.cacheBlockSize, pRepo->config.totalBlocks, (TSDB_REPO_T *)pRepo);
if (pRepo->tsdbCache == NULL) {
tsdbFreeMeta(pRepo->tsdbMeta);
free(pRepo->rootDir);
free(pRepo);
return NULL;
}
tsdbGetDataDirName(pRepo, dataDir);
pRepo->tsdbFileH = tsdbInitFileH(dataDir, &(pRepo->config));
if (pRepo->tsdbFileH == NULL) {
tsdbFreeCache(pRepo->tsdbCache);
tsdbFreeMeta(pRepo->tsdbMeta);
free(pRepo->rootDir);
free(pRepo);
return NULL;
}
// Restore key from file
if (tsdbRestoreInfo(pRepo) < 0) {
tsdbFreeCache(pRepo->tsdbCache);
tsdbFreeMeta(pRepo->tsdbMeta);
tsdbCloseFileH(pRepo->tsdbFileH);
free(pRepo->rootDir);
free(pRepo);
return NULL;
}
pRepo->state = TSDB_REPO_STATE_ACTIVE;
tsdbTrace("vgId:%d, open tsdb repository successfully!", pRepo->config.tsdbId);
return (TSDB_REPO_T *)pRepo;
}
// static int32_t tsdbFlushCache(STsdbRepo *pRepo) {
// // TODO
// return 0;
// }
/**
* Close a TSDB repository. Only free memory resources, and keep the files.
* @param repo the opened TSDB repository handle. The interface will free the handle too, so upper
* layer do NOT need to free the repo handle again.
*
* @return 0 for success, -1 for failure and the error number is set
*/
int32_t tsdbCloseRepo(TSDB_REPO_T *repo, int toCommit) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
if (pRepo == NULL) return 0;
int id = pRepo->config.tsdbId;
pRepo->state = TSDB_REPO_STATE_CLOSED;
tsdbLockRepo(repo);
if (pRepo->commit) {
tsdbUnLockRepo(repo);
return -1;
}
pRepo->commit = 1;
// Loop to move pData to iData
for (int i = 1; i < pRepo->config.maxTables; i++) {
STable *pTable = pRepo->tsdbMeta->tables[i];
if (pTable != NULL && pTable->mem != NULL) {
pTable->imem = pTable->mem;
pTable->mem = NULL;
}
}
// TODO: Loop to move mem to imem
pRepo->tsdbCache->imem = pRepo->tsdbCache->mem;
pRepo->tsdbCache->mem = NULL;
pRepo->tsdbCache->curBlock = NULL;
tsdbUnLockRepo(repo);
if (pRepo->appH.notifyStatus) pRepo->appH.notifyStatus(pRepo->appH.appH, TSDB_STATUS_COMMIT_START);
if (toCommit) tsdbCommitData((void *)repo);
tsdbCloseFileH(pRepo->tsdbFileH);
tsdbFreeMeta(pRepo->tsdbMeta);
tsdbFreeCache(pRepo->tsdbCache);
tfree(pRepo->rootDir);
tfree(pRepo);
tsdbTrace("vgId:%d, repository is closed!", id);
return 0;
}
/**
* Change the configuration of a repository
* @param pCfg the repository configuration, the upper layer should free the pointer
......@@ -346,50 +1110,6 @@ int32_t tsdbConfigRepo(TSDB_REPO_T *repo, STsdbCfg *pCfg) {
return TSDB_CODE_SUCCESS;
}
int32_t tsdbTriggerCommit(TSDB_REPO_T *repo) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
if (pRepo->appH.notifyStatus) pRepo->appH.notifyStatus(pRepo->appH.appH, TSDB_STATUS_COMMIT_START);
tsdbLockRepo(repo);
if (pRepo->commit) {
tsdbUnLockRepo(repo);
return -1;
}
pRepo->commit = 1;
// Loop to move pData to iData
for (int i = 1; i < pRepo->config.maxTables; i++) {
STable *pTable = pRepo->tsdbMeta->tables[i];
if (pTable != NULL && pTable->mem != NULL) {
pTable->imem = pTable->mem;
pTable->mem = NULL;
}
}
// TODO: Loop to move mem to imem
pRepo->tsdbCache->imem = pRepo->tsdbCache->mem;
pRepo->tsdbCache->mem = NULL;
pRepo->tsdbCache->curBlock = NULL;
tsdbUnLockRepo(repo);
// TODO: here should set as detached or use join for memory leak
pthread_attr_t thattr;
pthread_attr_init(&thattr);
pthread_attr_setdetachstate(&thattr, PTHREAD_CREATE_DETACHED);
pthread_create(&(pRepo->commitThread), &thattr, tsdbCommitData, (void *)repo);
tsdbTrace("vgId:%d, start to commit!", pRepo->config.tsdbId);
return 0;
}
int32_t tsdbLockRepo(TSDB_REPO_T *repo) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
return pthread_mutex_lock(&(pRepo->mutex));
}
int32_t tsdbUnLockRepo(TSDB_REPO_T *repo) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
return pthread_mutex_unlock(&(pRepo->mutex));
}
/**
* Get the TSDB repository information, including some statistics
......@@ -404,107 +1124,23 @@ STsdbRepoInfo *tsdbGetStatus(TSDB_REPO_T *pRepo) {
return NULL;
}
int tsdbUpdateTagValue(TSDB_REPO_T *repo, SUpdateTableTagValMsg *pMsg) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
STsdbMeta *pMeta = pRepo->tsdbMeta;
int16_t tversion = htons(pMsg->tversion);
STable *pTable = tsdbGetTableByUid(pMeta, htobe64(pMsg->uid));
if (pTable == NULL) return TSDB_CODE_TDB_INVALID_TABLE_ID;
if (pTable->tableId.tid != htonl(pMsg->tid)) return TSDB_CODE_TDB_INVALID_TABLE_ID;
if (pTable->type != TSDB_CHILD_TABLE) {
tsdbError("vgId:%d failed to update tag value of table %s since its type is %d", pRepo->config.tsdbId,
varDataVal(pTable->name), pTable->type);
return TSDB_CODE_TDB_INVALID_TABLE_TYPE;
}
if (schemaVersion(tsdbGetTableTagSchema(pMeta, pTable)) < tversion) {
tsdbTrace("vgId:%d server tag version %d is older than client tag version %d, try to config", pRepo->config.tsdbId,
schemaVersion(tsdbGetTableTagSchema(pMeta, pTable)), tversion);
void *msg = (*pRepo->appH.configFunc)(pRepo->config.tsdbId, htonl(pMsg->tid));
if (msg == NULL) {
return terrno;
}
// Deal with error her
STableCfg *pTableCfg = tsdbCreateTableCfgFromMsg(msg);
STable *super = tsdbGetTableByUid(pMeta, pTableCfg->superUid);
ASSERT(super != NULL);
int32_t code = tsdbUpdateTable(pMeta, super, pTableCfg);
if (code != TSDB_CODE_SUCCESS) {
return code;
}
tsdbClearTableCfg(pTableCfg);
rpcFreeCont(msg);
}
STSchema *pTagSchema = tsdbGetTableTagSchema(pMeta, pTable);
if (schemaVersion(pTagSchema) > tversion) {
tsdbError(
"vgId:%d failed to update tag value of table %s since version out of date, client tag version:%d server tag "
"version:%d",
pRepo->config.tsdbId, varDataVal(pTable->name), tversion, schemaVersion(pTable->tagSchema));
return TSDB_CODE_TDB_TAG_VER_OUT_OF_DATE;
}
if (schemaColAt(pTagSchema, DEFAULT_TAG_INDEX_COLUMN)->colId == htons(pMsg->colId)) {
tsdbRemoveTableFromIndex(pMeta, pTable);
}
// TODO: remove table from index if it is the first column of tag
tdSetKVRowDataOfCol(&pTable->tagVal, htons(pMsg->colId), htons(pMsg->type), pMsg->data);
if (schemaColAt(pTagSchema, DEFAULT_TAG_INDEX_COLUMN)->colId == htons(pMsg->colId)) {
tsdbAddTableIntoIndex(pMeta, pTable);
}
return TSDB_CODE_SUCCESS;
}
TSKEY tsdbGetTableLastKey(TSDB_REPO_T *repo, uint64_t uid) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
STable *pTable = tsdbGetTableByUid(pRepo->tsdbMeta, uid);
if (pTable == NULL) return -1;
return TSDB_GET_TABLE_LAST_KEY(pTable);
}
void tsdbStartStream(TSDB_REPO_T *repo) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
STsdbMeta *pMeta = pRepo->tsdbMeta;
for (int i = 0; i < pRepo->config.maxTables; i++) {
STable *pTable = pMeta->tables[i];
if (pTable && pTable->type == TSDB_STREAM_TABLE) {
pTable->cqhandle = (*pRepo->appH.cqCreateFunc)(pRepo->appH.cqH, pTable->tableId.uid, pTable->tableId.tid, pTable->sql, tsdbGetTableSchema(pMeta, pTable));
}
}
STable *pTable = tsdbGetTableByUid(pRepo->tsdbMeta, uid);
if (pTable == NULL) return -1;
return TSDB_GET_TABLE_LAST_KEY(pTable);
}
STableInfo *tsdbGetTableInfo(TSDB_REPO_T *pRepo, STableId tableId) {
// TODO
return NULL;
}
// TODO: need to return the number of data inserted
int32_t tsdbInsertData(TSDB_REPO_T *repo, SSubmitMsg *pMsg, SShellSubmitRspMsg * pRsp) {
SSubmitMsgIter msgIter;
STsdbRepo *pRepo = (STsdbRepo *)repo;
tsdbInitSubmitMsgIter(pMsg, &msgIter);
SSubmitBlk *pBlock = NULL;
int32_t code = TSDB_CODE_SUCCESS;
int32_t affectedrows = 0;
TSKEY now = taosGetTimestamp(pRepo->config.precision);
while ((pBlock = tsdbGetSubmitMsgNext(&msgIter)) != NULL) {
if ((code = tsdbInsertDataToTable(repo, pBlock, now, &affectedrows)) != TSDB_CODE_SUCCESS) {
return code;
}
}
pRsp->affectedRows = htonl(affectedrows);
return code;
}
void tsdbClearTableCfg(STableCfg *config) {
if (config) {
......@@ -526,59 +1162,8 @@ int tsdbInitSubmitBlkIter(SSubmitBlk *pBlock, SSubmitBlkIter *pIter) {
return 0;
}
SDataRow tsdbGetSubmitBlkNext(SSubmitBlkIter *pIter) {
SDataRow row = pIter->row;
if (row == NULL) return NULL;
pIter->len += dataRowLen(row);
if (pIter->len >= pIter->totalLen) {
pIter->row = NULL;
} else {
pIter->row = (char *)row + dataRowLen(row);
}
return row;
}
int tsdbInitSubmitMsgIter(SSubmitMsg *pMsg, SSubmitMsgIter *pIter) {
if (pMsg == NULL || pIter == NULL) return -1;
pMsg->length = htonl(pMsg->length);
pMsg->numOfBlocks = htonl(pMsg->numOfBlocks);
pMsg->compressed = htonl(pMsg->compressed);
pIter->totalLen = pMsg->length;
pIter->len = TSDB_SUBMIT_MSG_HEAD_SIZE;
if (pMsg->length <= TSDB_SUBMIT_MSG_HEAD_SIZE) {
pIter->pBlock = NULL;
} else {
pIter->pBlock = pMsg->blocks;
}
return 0;
}
SSubmitBlk *tsdbGetSubmitMsgNext(SSubmitMsgIter *pIter) {
SSubmitBlk *pBlock = pIter->pBlock;
if (pBlock == NULL) return NULL;
pBlock->len = htonl(pBlock->len);
pBlock->numOfRows = htons(pBlock->numOfRows);
pBlock->uid = htobe64(pBlock->uid);
pBlock->tid = htonl(pBlock->tid);
pBlock->sversion = htonl(pBlock->sversion);
pBlock->padding = htonl(pBlock->padding);
pIter->len = pIter->len + sizeof(SSubmitBlk) + pBlock->len;
if (pIter->len >= pIter->totalLen) {
pIter->pBlock = NULL;
} else {
pIter->pBlock = (SSubmitBlk *)((char *)pBlock + pBlock->len + sizeof(SSubmitBlk));
}
return pBlock;
}
STsdbMeta* tsdbGetMeta(TSDB_REPO_T* pRepo) {
STsdbRepo *tsdb = (STsdbRepo *)pRepo;
......@@ -591,120 +1176,8 @@ STsdbFileH* tsdbGetFile(TSDB_REPO_T* pRepo) {
}
// Check the configuration and set default options
static int32_t tsdbCheckAndSetDefaultCfg(STsdbCfg *pCfg) {
// Check precision
if (pCfg->precision == -1) {
pCfg->precision = TSDB_DEFAULT_PRECISION;
} else {
if (!IS_VALID_PRECISION(pCfg->precision)) {
tsdbError("vgId:%d, invalid precision configuration! precision:%d", pCfg->tsdbId, pCfg->precision);
return -1;
}
}
// Check compression
if (pCfg->compression == -1) {
pCfg->compression = TSDB_DEFAULT_COMPRESSION;
} else {
if (!IS_VALID_COMPRESSION(pCfg->compression)) {
tsdbError("vgId:%d: invalid compression configuration! compression:%d", pCfg->tsdbId, pCfg->precision);
return -1;
}
}
// Check tsdbId
if (pCfg->tsdbId < 0) return -1;
// Check maxTables
if (pCfg->maxTables == -1) {
pCfg->maxTables = TSDB_DEFAULT_TABLES;
} else {
if (pCfg->maxTables < TSDB_MIN_TABLES || pCfg->maxTables > TSDB_MAX_TABLES) {
tsdbError("vgId:%d: invalid maxTables configuration! maxTables:%d TSDB_MIN_TABLES:%d TSDB_MAX_TABLES:%d",
pCfg->tsdbId, pCfg->maxTables, TSDB_MIN_TABLES, TSDB_MAX_TABLES);
return -1;
}
}
// Check daysPerFile
if (pCfg->daysPerFile == -1) {
pCfg->daysPerFile = TSDB_DEFAULT_DAYS_PER_FILE;
} else {
if (pCfg->daysPerFile < TSDB_MIN_DAYS_PER_FILE || pCfg->daysPerFile > TSDB_MAX_DAYS_PER_FILE) {
tsdbError(
"vgId:%d, invalid daysPerFile configuration! daysPerFile:%d TSDB_MIN_DAYS_PER_FILE:%d TSDB_MAX_DAYS_PER_FILE:"
"%d",
pCfg->tsdbId, pCfg->daysPerFile, TSDB_MIN_DAYS_PER_FILE, TSDB_MAX_DAYS_PER_FILE);
return -1;
}
}
// Check minRowsPerFileBlock and maxRowsPerFileBlock
if (pCfg->minRowsPerFileBlock == -1) {
pCfg->minRowsPerFileBlock = TSDB_DEFAULT_MIN_ROW_FBLOCK;
} else {
if (pCfg->minRowsPerFileBlock < TSDB_MIN_MIN_ROW_FBLOCK || pCfg->minRowsPerFileBlock > TSDB_MAX_MIN_ROW_FBLOCK) {
tsdbError(
"vgId:%d, invalid minRowsPerFileBlock configuration! minRowsPerFileBlock:%d TSDB_MIN_MIN_ROW_FBLOCK:%d "
"TSDB_MAX_MIN_ROW_FBLOCK:%d",
pCfg->tsdbId, pCfg->minRowsPerFileBlock, TSDB_MIN_MIN_ROW_FBLOCK, TSDB_MAX_MIN_ROW_FBLOCK);
return -1;
}
}
if (pCfg->maxRowsPerFileBlock == -1) {
pCfg->maxRowsPerFileBlock = TSDB_DEFAULT_MAX_ROW_FBLOCK;
} else {
if (pCfg->maxRowsPerFileBlock < TSDB_MIN_MAX_ROW_FBLOCK || pCfg->maxRowsPerFileBlock > TSDB_MAX_MAX_ROW_FBLOCK) {
tsdbError(
"vgId:%d, invalid maxRowsPerFileBlock configuration! maxRowsPerFileBlock:%d TSDB_MIN_MAX_ROW_FBLOCK:%d "
"TSDB_MAX_MAX_ROW_FBLOCK:%d",
pCfg->tsdbId, pCfg->maxRowsPerFileBlock, TSDB_MIN_MIN_ROW_FBLOCK, TSDB_MAX_MIN_ROW_FBLOCK);
return -1;
}
}
if (pCfg->minRowsPerFileBlock > pCfg->maxRowsPerFileBlock) return -1;
// Check keep
if (pCfg->keep == -1) {
pCfg->keep = TSDB_DEFAULT_KEEP;
} else {
if (pCfg->keep < TSDB_MIN_KEEP || pCfg->keep > TSDB_MAX_KEEP) {
tsdbError(
"vgId:%d, invalid keep configuration! keep:%d TSDB_MIN_KEEP:%d "
"TSDB_MAX_KEEP:%d",
pCfg->tsdbId, pCfg->keep, TSDB_MIN_KEEP, TSDB_MAX_KEEP);
return -1;
}
}
return 0;
}
static int32_t tsdbGetCfgFname(STsdbRepo *pRepo, char *fname) {
if (pRepo == NULL) return -1;
sprintf(fname, "%s/%s", pRepo->rootDir, TSDB_CFG_FILE_NAME);
return 0;
}
static int32_t tsdbSaveConfig(STsdbRepo *pRepo) {
char fname[128] = "\0"; // TODO: get rid of the literal 128
if (tsdbGetCfgFname(pRepo, fname) < 0) return -1;
int fd = open(fname, O_WRONLY | O_CREAT, 0755);
if (fd < 0) {
return -1;
}
if (write(fd, (void *)(&(pRepo->config)), sizeof(STsdbCfg)) < 0) {
return -1;
}
close(fd);
return 0;
}
static int32_t tsdbRestoreCfg(STsdbRepo *pRepo, STsdbCfg *pCfg) {
char fname[128] = "\0";
......@@ -726,187 +1199,36 @@ static int32_t tsdbRestoreCfg(STsdbRepo *pRepo, STsdbCfg *pCfg) {
return 0;
}
static int32_t tsdbGetDataDirName(STsdbRepo *pRepo, char *fname) {
if (pRepo == NULL || pRepo->rootDir == NULL) return -1;
sprintf(fname, "%s/%s", pRepo->rootDir, TSDB_DATA_DIR_NAME);
return 0;
}
static int32_t tsdbSetRepoEnv(STsdbRepo *pRepo) {
STsdbCfg *pCfg = &pRepo->config;
if (tsdbSaveConfig(pRepo) < 0) return -1;
char dirName[128] = "\0";
if (tsdbGetDataDirName(pRepo, dirName) < 0) return -1;
if (mkdir(dirName, 0755) < 0) {
tsdbError("vgId:%d, failed to create repository directory! reason:%s", pRepo->config.tsdbId, strerror(errno));
return -1;
}
tsdbTrace(
"vgId:%d, set up tsdb environment succeed! cacheBlockSize:%d, totalBlocks:%d, maxTables:%d, daysPerFile:%d, keep:"
"%d, minRowsPerFileBlock:%d, maxRowsPerFileBlock:%d, precision:%d, compression:%d",
pRepo->config.tsdbId, pCfg->cacheBlockSize, pCfg->totalBlocks, pCfg->maxTables, pCfg->daysPerFile, pCfg->keep,
pCfg->minRowsPerFileBlock, pCfg->maxRowsPerFileBlock, pCfg->precision, pCfg->compression);
return 0;
}
static int32_t tsdbDestroyRepoEnv(STsdbRepo *pRepo) {
char fname[260];
if (pRepo == NULL) return 0;
char *dirName = calloc(1, strlen(pRepo->rootDir) + strlen("tsdb") + 2);
if (dirName == NULL) {
return -1;
}
sprintf(dirName, "%s/%s", pRepo->rootDir, "tsdb");
DIR *dir = opendir(dirName);
if (dir == NULL) return -1;
struct dirent *dp;
while ((dp = readdir(dir)) != NULL) {
if ((strcmp(dp->d_name, ".") == 0) || (strcmp(dp->d_name, "..") == 0)) continue;
sprintf(fname, "%s/%s", pRepo->rootDir, dp->d_name);
remove(fname);
}
closedir(dir);
rmdir(dirName);
return 0;
}
// static int tsdbOpenMetaFile(char *tsdbDir) {
// // TODO
// return 0;
// }
static int32_t tdInsertRowToTable(STsdbRepo *pRepo, SDataRow row, STable *pTable) {
// TODO
int32_t level = 0;
int32_t headSize = 0;
if (pTable->mem == NULL) {
pTable->mem = (SMemTable *)calloc(1, sizeof(SMemTable));
if (pTable->mem == NULL) return -1;
pTable->mem->pData = tSkipListCreate(5, TSDB_DATA_TYPE_TIMESTAMP, TYPE_BYTES[TSDB_DATA_TYPE_TIMESTAMP], 0, 0, 0, getTSTupleKey);
pTable->mem->keyFirst = INT64_MAX;
pTable->mem->keyLast = 0;
}
tSkipListNewNodeInfo(pTable->mem->pData, &level, &headSize);
TSKEY key = dataRowKey(row);
// printf("insert:%lld, size:%d\n", key, pTable->mem->numOfRows);
// Copy row into the memory
SSkipListNode *pNode = tsdbAllocFromCache(pRepo->tsdbCache, headSize + dataRowLen(row), key);
if (pNode == NULL) {
// TODO: deal with allocate failure
}
pNode->level = level;
dataRowCpy(SL_GET_NODE_DATA(pNode), row);
// Insert the skiplist node into the data
if (pTable->mem == NULL) {
pTable->mem = (SMemTable *)calloc(1, sizeof(SMemTable));
if (pTable->mem == NULL) return -1;
pTable->mem->pData = tSkipListCreate(5, TSDB_DATA_TYPE_TIMESTAMP, TYPE_BYTES[TSDB_DATA_TYPE_TIMESTAMP], 0, 0, 0, getTSTupleKey);
pTable->mem->keyFirst = INT64_MAX;
pTable->mem->keyLast = 0;
}
tSkipListPut(pTable->mem->pData, pNode);
if (key > pTable->mem->keyLast) pTable->mem->keyLast = key;
if (key < pTable->mem->keyFirst) pTable->mem->keyFirst = key;
if (key > pTable->lastKey) pTable->lastKey = key;
pTable->mem->numOfRows = tSkipListGetSize(pTable->mem->pData);
tsdbTrace("vgId:%d, tid:%d, uid:%" PRId64 ", table:%s a row is inserted to table! key:%" PRId64, pRepo->config.tsdbId,
pTable->tableId.tid, pTable->tableId.uid, varDataVal(pTable->name), dataRowKey(row));
return 0;
}
static int32_t tsdbInsertDataToTable(TSDB_REPO_T *repo, SSubmitBlk *pBlock, TSKEY now, int32_t *affectedrows) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
STsdbMeta *pMeta = pRepo->tsdbMeta;
int64_t points = 0;
STableId tableId = {.uid = pBlock->uid, .tid = pBlock->tid};
STable *pTable = tsdbIsValidTableToInsert(pRepo->tsdbMeta, tableId);
if (pTable == NULL) {
tsdbError("vgId:%d, failed to get table for insert, uid:" PRIu64 ", tid:%d", pRepo->config.tsdbId, pBlock->uid,
pBlock->tid);
return TSDB_CODE_TDB_INVALID_TABLE_ID;
}
// Check schema version
int32_t tversion = pBlock->sversion;
STSchema * pSchema = tsdbGetTableSchema(pMeta, pTable);
ASSERT(pSchema != NULL);
int16_t nversion = schemaVersion(pSchema);
if (tversion > nversion) {
tsdbTrace("vgId:%d table:%s tid:%d server schema version %d is older than clien version %d, try to config.",
pRepo->config.tsdbId, varDataVal(pTable->name), pTable->tableId.tid, nversion, tversion);
void *msg = (*pRepo->appH.configFunc)(pRepo->config.tsdbId, pTable->tableId.tid);
if (msg == NULL) {
return terrno;
}
// Deal with error her
STableCfg *pTableCfg = tsdbCreateTableCfgFromMsg(msg);
STable *pTableUpdate = NULL;
if (pTable->type == TSDB_CHILD_TABLE) {
pTableUpdate = tsdbGetTableByUid(pMeta, pTableCfg->superUid);
} else {
pTableUpdate = pTable;
}
int32_t code = tsdbUpdateTable(pMeta, pTableUpdate, pTableCfg);
if (code != TSDB_CODE_SUCCESS) {
return code;
}
tsdbClearTableCfg(pTableCfg);
rpcFreeCont(msg);
} else {
if (tsdbGetTableSchemaByVersion(pMeta, pTable, tversion) == NULL) {
tsdbError("vgId:%d table:%s tid:%d invalid schema version %d from client", pRepo->config.tsdbId,
varDataVal(pTable->name), pTable->tableId.tid, tversion);
return TSDB_CODE_TDB_TABLE_SCHEMA_VERSION;
}
char *dirName = calloc(1, strlen(pRepo->rootDir) + strlen("tsdb") + 2);
if (dirName == NULL) {
return -1;
}
SSubmitBlkIter blkIter = {0};
SDataRow row = NULL;
TSKEY minKey = now - tsMsPerDay[pRepo->config.precision] * pRepo->config.keep;
TSKEY maxKey = now + tsMsPerDay[pRepo->config.precision] * pRepo->config.daysPerFile;
sprintf(dirName, "%s/%s", pRepo->rootDir, "tsdb");
tsdbInitSubmitBlkIter(pBlock, &blkIter);
while ((row = tsdbGetSubmitBlkNext(&blkIter)) != NULL) {
if (dataRowKey(row) < minKey || dataRowKey(row) > maxKey) {
tsdbError("vgId:%d, table:%s, tid:%d, talbe uid:%ld timestamp is out of range. now:" PRId64 ", maxKey:" PRId64
", minKey:" PRId64,
pRepo->config.tsdbId, varDataVal(pTable->name), pTable->tableId.tid, pTable->tableId.uid, now, minKey, maxKey);
return TSDB_CODE_TDB_TIMESTAMP_OUT_OF_RANGE;
}
DIR *dir = opendir(dirName);
if (dir == NULL) return -1;
if (tdInsertRowToTable(pRepo, row, pTable) < 0) {
return -1;
}
(*affectedrows)++;
points++;
struct dirent *dp;
while ((dp = readdir(dir)) != NULL) {
if ((strcmp(dp->d_name, ".") == 0) || (strcmp(dp->d_name, "..") == 0)) continue;
sprintf(fname, "%s/%s", pRepo->rootDir, dp->d_name);
remove(fname);
}
atomic_fetch_add_64(&(pRepo->stat.pointsWritten), points * (pSchema->numOfCols));
atomic_fetch_add_64(&(pRepo->stat.totalStorage), points * pSchema->vlen);
return TSDB_CODE_SUCCESS;
closedir(dir);
rmdir(dirName);
return 0;
}
static int tsdbReadRowsFromCache(STsdbMeta *pMeta, STable *pTable, SSkipListIterator *pIter, TSKEY maxKey, int maxRowsToRead, SDataCols *pCols) {
ASSERT(maxRowsToRead > 0);
if (pIter == NULL) return 0;
......@@ -949,26 +1271,6 @@ static void tsdbDestroyTableIters(SSkipListIterator **iters, int maxTables) {
free(iters);
}
static SSkipListIterator **tsdbCreateTableIters(STsdbMeta *pMeta, int maxTables) {
SSkipListIterator **iters = (SSkipListIterator **)calloc(maxTables, sizeof(SSkipListIterator *));
if (iters == NULL) return NULL;
for (int tid = 1; tid < maxTables; tid++) {
STable *pTable = pMeta->tables[tid];
if (pTable == NULL || pTable->imem == NULL || pTable->imem->numOfRows == 0) continue;
iters[tid] = tSkipListCreateIter(pTable->imem->pData);
if (iters[tid] == NULL) goto _err;
if (!tSkipListIterNext(iters[tid])) goto _err;
}
return iters;
_err:
tsdbDestroyTableIters(iters, maxTables);
return NULL;
}
static void tsdbFreeMemTable(SMemTable *pMemTable) {
if (pMemTable) {
......@@ -978,159 +1280,7 @@ static void tsdbFreeMemTable(SMemTable *pMemTable) {
}
// Commit to file
static void *tsdbCommitData(void *arg) {
STsdbRepo * pRepo = (STsdbRepo *)arg;
STsdbMeta * pMeta = pRepo->tsdbMeta;
STsdbCache *pCache = pRepo->tsdbCache;
STsdbCfg * pCfg = &(pRepo->config);
SDataCols * pDataCols = NULL;
SRWHelper whelper = {{0}};
if (pCache->imem == NULL) return NULL;
tsdbPrint("vgId:%d, starting to commit....", pRepo->config.tsdbId);
// Create the iterator to read from cache
SSkipListIterator **iters = tsdbCreateTableIters(pMeta, pCfg->maxTables);
if (iters == NULL) {
ASSERT(0);
return NULL;
}
if (tsdbInitWriteHelper(&whelper, pRepo) < 0) goto _exit;
if ((pDataCols = tdNewDataCols(pMeta->maxRowBytes, pMeta->maxCols, pCfg->maxRowsPerFileBlock)) == NULL) goto _exit;
int sfid = tsdbGetKeyFileId(pCache->imem->keyFirst, pCfg->daysPerFile, pCfg->precision);
int efid = tsdbGetKeyFileId(pCache->imem->keyLast, pCfg->daysPerFile, pCfg->precision);
// Loop to commit to each file
for (int fid = sfid; fid <= efid; fid++) {
if (tsdbCommitToFile(pRepo, fid, iters, &whelper, pDataCols) < 0) {
ASSERT(false);
goto _exit;
}
}
// Do retention actions
tsdbFitRetention(pRepo);
if (pRepo->appH.notifyStatus) pRepo->appH.notifyStatus(pRepo->appH.appH, TSDB_STATUS_COMMIT_OVER);
_exit:
tdFreeDataCols(pDataCols);
tsdbDestroyTableIters(iters, pCfg->maxTables);
tsdbDestroyHelper(&whelper);
tsdbLockRepo(arg);
tdListMove(pCache->imem->list, pCache->pool.memPool);
tsdbAdjustCacheBlocks(pCache);
tdListFree(pCache->imem->list);
free(pCache->imem);
pCache->imem = NULL;
pRepo->commit = 0;
for (int i = 1; i < pCfg->maxTables; i++) {
STable *pTable = pMeta->tables[i];
if (pTable && pTable->imem) {
tsdbFreeMemTable(pTable->imem);
pTable->imem = NULL;
}
}
tsdbUnLockRepo(arg);
tsdbPrint("vgId:%d, commit over....", pRepo->config.tsdbId);
return NULL;
}
static int tsdbCommitToFile(STsdbRepo *pRepo, int fid, SSkipListIterator **iters, SRWHelper *pHelper, SDataCols *pDataCols) {
char dataDir[128] = {0};
STsdbMeta * pMeta = pRepo->tsdbMeta;
STsdbFileH *pFileH = pRepo->tsdbFileH;
STsdbCfg * pCfg = &pRepo->config;
SFileGroup *pGroup = NULL;
TSKEY minKey = 0, maxKey = 0;
tsdbGetKeyRangeOfFileId(pCfg->daysPerFile, pCfg->precision, fid, &minKey, &maxKey);
// Check if there are data to commit to this file
int hasDataToCommit = tsdbHasDataToCommit(iters, pCfg->maxTables, minKey, maxKey);
if (!hasDataToCommit) return 0; // No data to commit, just return
// Create and open files for commit
tsdbGetDataDirName(pRepo, dataDir);
if ((pGroup = tsdbCreateFGroup(pFileH, dataDir, fid, pCfg->maxTables)) == NULL) {
tsdbError("vgId:%d, failed to create file group %d", pRepo->config.tsdbId, fid);
goto _err;
}
// Open files for write/read
if (tsdbSetAndOpenHelperFile(pHelper, pGroup) < 0) {
tsdbError("vgId:%d, failed to set helper file", pRepo->config.tsdbId);
goto _err;
}
// Loop to commit data in each table
for (int tid = 1; tid < pCfg->maxTables; tid++) {
STable * pTable = pMeta->tables[tid];
if (pTable == NULL) continue;
SSkipListIterator *pIter = iters[tid];
// Set the helper and the buffer dataCols object to help to write this table
tsdbSetHelperTable(pHelper, pTable, pRepo);
tdInitDataCols(pDataCols, tsdbGetTableSchema(pMeta, pTable));
// Loop to write the data in the cache to files. If no data to write, just break the loop
int maxRowsToRead = pCfg->maxRowsPerFileBlock * 4 / 5;
int nLoop = 0;
while (true) {
int rowsRead = tsdbReadRowsFromCache(pMeta, pTable, pIter, maxKey, maxRowsToRead, pDataCols);
assert(rowsRead >= 0);
if (pDataCols->numOfRows == 0) break;
nLoop++;
ASSERT(dataColsKeyFirst(pDataCols) >= minKey && dataColsKeyFirst(pDataCols) <= maxKey);
ASSERT(dataColsKeyLast(pDataCols) >= minKey && dataColsKeyLast(pDataCols) <= maxKey);
int rowsWritten = tsdbWriteDataBlock(pHelper, pDataCols);
ASSERT(rowsWritten != 0);
if (rowsWritten < 0) goto _err;
ASSERT(rowsWritten <= pDataCols->numOfRows);
tdPopDataColsPoints(pDataCols, rowsWritten);
maxRowsToRead = pCfg->maxRowsPerFileBlock * 4 / 5 - pDataCols->numOfRows;
}
ASSERT(pDataCols->numOfRows == 0);
// Move the last block to the new .l file if neccessary
if (tsdbMoveLastBlockIfNeccessary(pHelper) < 0) {
tsdbError("vgId:%d, failed to move last block", pRepo->config.tsdbId);
goto _err;
}
// Write the SCompBlock part
if (tsdbWriteCompInfo(pHelper) < 0) {
tsdbError("vgId:%d, failed to write compInfo part", pRepo->config.tsdbId);
goto _err;
}
}
if (tsdbWriteCompIdx(pHelper) < 0) {
tsdbError("vgId:%d, failed to write compIdx part", pRepo->config.tsdbId);
goto _err;
}
tsdbCloseHelperFile(pHelper, 0);
// TODO: make it atomic with some methods
pGroup->files[TSDB_FILE_TYPE_HEAD] = pHelper->files.headF;
pGroup->files[TSDB_FILE_TYPE_DATA] = pHelper->files.dataF;
pGroup->files[TSDB_FILE_TYPE_LAST] = pHelper->files.lastF;
return 0;
_err:
ASSERT(false);
tsdbCloseHelperFile(pHelper, 1);
return -1;
}
/**
* Return the next iterator key.
......@@ -1199,76 +1349,6 @@ static void tsdbAlterMaxTables(STsdbRepo *pRepo, int32_t maxTables) {
tsdbTrace("vgId:%d, tsdb maxTables is changed from %d to %d!", pRepo->config.tsdbId, oldMaxTables, maxTables);
}
#define TSDB_META_FILE_INDEX 10000000
uint32_t tsdbGetFileInfo(TSDB_REPO_T *repo, char *name, uint32_t *index, uint32_t eindex, int32_t *size) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
// STsdbMeta *pMeta = pRepo->tsdbMeta;
STsdbFileH *pFileH = pRepo->tsdbFileH;
uint32_t magic = 0;
char fname[256] = "\0";
struct stat fState;
tsdbTrace("vgId:%d name:%s index:%d eindex:%d", pRepo->config.tsdbId, name, *index, eindex);
ASSERT(*index <= eindex);
char *sdup = strdup(pRepo->rootDir);
char *prefix = dirname(sdup);
if (name[0] == 0) { // get the file from index or after, but not larger than eindex
int fid = (*index) / 3;
if (pFileH->numOfFGroups == 0 || fid > pFileH->fGroup[pFileH->numOfFGroups - 1].fileId) {
if (*index <= TSDB_META_FILE_INDEX && TSDB_META_FILE_INDEX <= eindex) {
tsdbGetMetaFileName(pRepo->rootDir, fname);
*index = TSDB_META_FILE_INDEX;
} else {
tfree(sdup);
return 0;
}
} else {
SFileGroup *pFGroup =
taosbsearch(&fid, pFileH->fGroup, pFileH->numOfFGroups, sizeof(SFileGroup), compFGroupKey, TD_GE);
if (pFGroup->fileId == fid) {
strcpy(fname, pFGroup->files[(*index) % 3].fname);
} else {
if (pFGroup->fileId * 3 + 2 < eindex) {
strcpy(fname, pFGroup->files[0].fname);
*index = pFGroup->fileId * 3;
} else {
tfree(sdup);
return 0;
}
}
}
strcpy(name, fname + strlen(prefix));
} else { // get the named file at the specified index. If not there, return 0
if (*index == TSDB_META_FILE_INDEX) { // get meta file
tsdbGetMetaFileName(pRepo->rootDir, fname);
} else {
int fid = (*index) / 3;
SFileGroup *pFGroup = tsdbSearchFGroup(pFileH, fid);
if (pFGroup == NULL) { // not found
tfree(sdup);
return 0;
}
SFile *pFile = &pFGroup->files[(*index) % 3];
strcpy(fname, pFile->fname);
}
}
if (stat(fname, &fState) < 0) {
tfree(sdup);
return 0;
}
tfree(sdup);
*size = fState.st_size;
magic = *size;
return magic;
}
void tsdbReportStat(void *repo, int64_t *totalPoints, int64_t *totalStorage, int64_t *compStorage){
ASSERT(repo != NULL);
......@@ -1276,4 +1356,6 @@ void tsdbReportStat(void *repo, int64_t *totalPoints, int64_t *totalStorage, int
*totalPoints = pRepo->stat.pointsWritten;
*totalStorage = pRepo->stat.totalStorage;
*compStorage = pRepo->stat.compStorage;
}
\ No newline at end of file
}
#endif
\ 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 "tsdb.h"
#include "tsdbMain.h"
#define TSDB_DATA_SKIPLIST_LEVEL 5
// ---------------- INTERNAL FUNCTIONS ----------------
int tsdbInsertRowToMem(STsdbRepo *pRepo, SDataRow row, STable *pTable) {
STsdbCfg *pCfg = &pRepo->config;
int32_t level = 0;
int32_t headSize = 0;
TSKEY key = dataRowKey(row);
// TODO
tSkipListNewNodeInfo(pRepo->mem->tData[TABLE_TID(pTable)]->pData, &level, &headSize);
// TODO: for duplicate keys, you do not need to allocate memory here
SSkipListNode *pNode = tsdbAllocBytes(pRepo, headSize + dataRowLen(row));
if (pNode == NULL) {
tsdbError("vgId:%d failed to insert row with key %" PRId64 " to table %s since %s", REPO_ID(pRepo), key,
TABLE_CHAR_NAME(pTable), tstrerror(terrno));
return -1;
}
SMemTable *pMemTable = pRepo->mem;
ASSERT(pMemTable != NULL);
pNode->level = level;
dataRowCpy(SL_GET_NODE_DATA(pNode), row);
STableData *pTableData = pMemTable->tData[TABLE_TID(pTable)];
if (pTableData == NULL) {
pTableData = tsdbNewTableData(pCfg);
if (pTableData == NULL) {
tsdbError("vgId:%d failed to insert row with key %" PRId64 " to table %s since %s", REPO_ID(pRepo), key,
TABLE_CHAR_NAME(pTable), tstrerror(terrno));
return -1;
}
pRepo->mem->tData[TABLE_TID(pTable)] = pTableData;
}
ASSERT(pTableData != NULL);
if (pTableData->uid != TALBE_UID(pTable)) {
// TODO
}
if (tSkipListPut(pTableData->pData, pNode) == NULL) {
tsdbFreeBytes(pRepo, (void *)pNode, headSize + dataRowLen);
return 0;
}
if (pMemTable->keyFirst > key) pMemTable->keyFirst = key;
if (pMemTable->keyLast < key) pMemTable->keyLast = key;
pMemTable->numOfRows++;
if (pTableData->keyFirst > key) pTableData->keyFirst = key;
if (pTableData->keyLast < key) pTableData->keyLast = key;
pTableData->numOfRows++;
ASSERT(pTableData->numOfRows == tSkipListGetSize(pTableData->pData));
tsdbTrace("vgId:%d a row is inserted to table %s tid %d uid %" PRIu64 " key %" PRIu64, REPO_ID(pRepo),
TABLE_CHAR_NAME(pTable), TABLE_TID(pTable), TALBE_UID(pTable), key);
return 0;
}
int tsdbRefMemTable(STsdbRepo *pRepo, SMemTable *pMemTable) {
ASSERT(pMemTable != NULL);
T_REF_INC(pMemTable);
}
// Need to lock the repository
int tsdbUnRefMemTable(STsdbRepo *pRepo, SMemTable *pMemTable) {
ASSERT(pMemTable != NULL);
if (T_REF_DEC(pMemTable) == 0) {
STsdbCfg * pCfg = &pRepo->config;
STsdbBufPool *pBufPool = pRepo->pPool;
SListNode *pNode = NULL;
while ((pNode = tdListPopHead(pMemTable->bufBlockList)) != NULL) {
tdListAppendNode(pBufPool->bufBlockList, pNode);
if (pthread_cond_signal(&pBufPool->poolNotEmpty) != 0) {
// TODO
}
}
for (int i = 0; i < pCfg->maxTables; i++) {
if (pMemTable->tData[i] != NULL) {
tsdbFreeTableData(pMemTable->tData[i]);
}
}
tdListDiscard(pMemTable->actList);
tdListDiscard(pMemTable->bufBlockList);
tsdbFreeMemTable(pMemTable);
}
return 0;
}
// ---------------- LOCAL FUNCTIONS ----------------
static FORCE_INLINE STsdbBufBlock *tsdbGetCurrBufBlock(STsdbRepo *pRepo) {
if (pRepo->mem == NULL) return NULL;
SListNode *pNode = listTail(pRepo->mem);
if (pNode == NULL) return NULL;
STsdbBufBlock *pBufBlock = NULL;
tdListNodeGetData(pMemTable->bufBlockList, pNode, (void *)(&pBufBlock));
return pBufBlock;
}
static void *tsdbAllocBytes(STsdbRepo *pRepo, int bytes) {
STsdbCfg * pCfg = &pRepo->config;
STsdbBufBlock *pBufBlock = tsdbGetCurrBufBlock(pRepo);
if (pBufBlock != NULL && pBufBlock->remain < bytes) {
if (listNEles(pRepo->mem) >= pCfg->totalBlocks / 2) { // need to trigger commit
if (pRepo->imem != NULL) {
if (pRepo->commit) pthread_join(pRepo->commitThread, NULL);
ASSERT(pRepo->commit == 0);
SMemTable *pIMem = pRepo->imem;
if (tsdbLockRepo(pRepo) < 0) {
// TODO
return NULL;
}
pRepo->imem = pRepo->mem;
pRepo->mem = NULL;
pRepo->commit = 1;
if (pthread_create(&pRepo->commitThread, NULL, tsdbCommitData, (void *)pRepo) != 0) {
// TODO
tsdbUnlockRepo(pRepo);
return NULL;
}
if (tsdbUnlockRepo(pRepo) < 0) {
// TODO
return NULL;
}
tsdbUnRefMemTable(pRepo, pIMem);
}
}
}
if (pRepo->mem == NULL) {
SMemTable *pMemTable = tsdbNewMemTable(&pRepo->config);
if (pMemTable == NULL) return NULL;
if (tsdbLockRepo(pRepo) < 0) {
tsdbFreeMemTable(pMemTable);
return NULL;
}
SListNode *pNode = tsdbAllocBufBlockFromPool(pRepo);
tdListAppendNode(pMemTable->bufBlockList, pNode);
pRepo->mem = pMemTable;
if (tsdbUnlockRepo(pRepo) < 0) return NULL;
}
pBufBlock = tsdbGetCurrBufBlock(pRepo);
ASSERT(pBufBlock->remain >= bytes);
void *ptr = POINTER_SHIFT(pBufBlock->data, pBufBlock->offset);
pBufBlock->offset += bytes;
pBufBlock->remain -= bytes;
return ptr;
}
static void tsdbFreeBytes(STsdbRepo *pRepo, void *ptr, int bytes) {
STsdbBufBlock *pBufBlock = tsdbGetCurrBufBlock(pRepo);
ASSERT(pBufBlock != NULL);
pBufBlock->offset -= bytes;
pBufBlock->remain += bytes;
ASSERT(ptr == POINTER_SHIFT(pBufBlock->data, pBufBlock->offset));
}
static SMemTable* tsdbNewMemTable(STsdbCfg* pCfg) {
SMemTable *pMemTable = (SMemTable *)calloc(1, sizeof(*pMemTable));
if (pMemTable == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pMemTable->keyFirst = INT64_MAX;
pMemTable->keyLast = 0;
pMemTable->numOfRows = 0;
pMemTable->tData = (STableData**)calloc(pCfg->maxTables, sizeof(STableData*));
if (pMemTable->tData == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pMemTable->actList = tdListNew(0);
if (pMemTable->actList == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pMemTable->bufBlockList = tdListNew(sizeof(STsdbBufBlock*));
if (pMemTable->bufBlockList == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
T_REF_INC(pMemTable);
return pMemTable;
_err:
tsdbFreeMemTable(pMemTable);
return NULL;
}
static void tsdbFreeMemTable(SMemTable* pMemTable) {
if (pMemTable) {
ASSERT((pMemTable->bufBlockList == NULL) ? true : (listNEles(pMemTable->bufBlockList) == 0));
ASSERT((pMemTable->actList == NULL) ? true : (listNEles(pMemTable->actList) == 0));
tdListFree(pMemTable->bufBlockList);
tdListFree(pMemTable->actList);
tfree(pMemTable->tData);
free(pMemTable);
}
}
static STableData *tsdbNewTableData(STsdbCfg *pCfg, STable *pTable) {
STableData *pTableData = (STableData *)calloc(1, sizeof(*pTableData));
if (pTableData == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pTableData->uid = TALBE_UID(pTable);
pTableData->keyFirst = INT64_MAX;
pTableData->keyLast = 0;
pTableData->numOfRows = 0;
pTableData->pData = tSkipListCreate(TSDB_DATA_SKIPLIST_LEVEL, TSDB_DATA_TYPE_TIMESTAMP, TYPE_BYTES[TSDB_DATA_TYPE_TIMESTAMP], 0, 0, 0, );
if (pTableData->pData == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
// TODO
pTableData->pData->level = 1;
return pTableData;
_err:
tsdbFreeTableData(pTableData);
return NULL;
}
static void tsdbFreeTableData(STableData *pTableData) {
if (pTableData) {
tSkipListDestroy(pTableData->pData);
free(pTableData);
}
}
\ 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 <stdlib.h>
#include "tskiplist.h"
#include "tsdb.h"
......@@ -5,33 +19,337 @@
#include "hash.h"
#include "tsdbMain.h"
#define TSDB_SUPER_TABLE_SL_LEVEL 5 // TODO: may change here
// #define TSDB_META_FILE_NAME "META"
// ------------------ OUTER FUNCTIONS ------------------
int tsdbCreateTable(TSDB_REPO_T *repo, STableCfg *pCfg) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
STsdbMeta *pMeta = pRepo->tsdbMeta;
static int tsdbFreeTable(STable *pTable);
static int32_t tsdbCheckTableCfg(STableCfg *pCfg);
static int tsdbAddTableToMeta(STsdbMeta *pMeta, STable *pTable, bool addIdx);
static int tsdbRemoveTableFromMeta(STsdbMeta *pMeta, STable *pTable, bool rmFromIdx);
static int tsdbInitTableCfg(STableCfg *config, ETableType type, uint64_t uid, int32_t tid);
static int tsdbTableSetSuperUid(STableCfg *config, uint64_t uid);
static int tsdbTableSetSchema(STableCfg *config, STSchema *pSchema, bool dup);
static int tsdbTableSetTagSchema(STableCfg *config, STSchema *pSchema, bool dup);
static int tsdbTableSetTagValue(STableCfg *config, SKVRow row, bool dup);
static int tsdbTableSetName(STableCfg *config, char *name, bool dup);
static int tsdbTableSetSName(STableCfg *config, char *sname, bool dup);
static int tsdbTableSetStreamSql(STableCfg *config, char *sql, bool dup);
if (tsdbCheckTableCfg(pCfg) < 0) return -1;
/**
* Encode a TSDB table object as a binary content
* ASSUMPTIONS: VALID PARAMETERS
*
* @param pTable table object to encode
* @param contLen the encoded binary content length
*
* @return binary content for success
* NULL fro failure
*/
void tsdbEncodeTable(STable *pTable, char *buf, int *contLen) {
STable *pTable = tsdbGetTableByUid(pMeta, pCfg->tableId.uid);
if (pTable != NULL) {
tsdbError("vgId:%d table %s already exists, tid %d uid %" PRId64, pRepo->config.tsdbId, varDataVal(pTable->name),
pTable->tableId.tid, pTable->tableId.uid);
return TSDB_CODE_TDB_TABLE_ALREADY_EXIST;
}
STable *super = NULL;
int newSuper = 0;
if (pCfg->type == TSDB_CHILD_TABLE) {
super = tsdbGetTableByUid(pMeta, pCfg->superUid);
if (super == NULL) { // super table not exists, try to create it
newSuper = 1;
super = tsdbNewTable(pCfg, true);
if (super == NULL) return -1;
} else {
if (super->type != TSDB_SUPER_TABLE) return -1;
if (super->tableId.uid != pCfg->superUid) return -1;
tsdbUpdateTable(pMeta, super, pCfg);
}
}
STable *table = tsdbNewTable(pCfg, false);
if (table == NULL) {
if (newSuper) {
tsdbFreeTable(super);
return -1;
}
}
table->lastKey = TSKEY_INITIAL_VAL;
// Register to meta
if (newSuper) {
tsdbAddTableToMeta(pMeta, super, true);
tsdbTrace("vgId:%d, super table %s is created! uid:%" PRId64, pRepo->config.tsdbId, varDataVal(super->name),
super->tableId.uid);
}
tsdbAddTableToMeta(pMeta, table, true);
tsdbTrace("vgId:%d, table %s is created! tid:%d, uid:%" PRId64, pRepo->config.tsdbId, varDataVal(table->name),
table->tableId.tid, table->tableId.uid);
// Write to meta file
int bufLen = 0;
char *buf = malloc(1024*1024);
if (newSuper) {
tsdbEncodeTable(super, buf, &bufLen);
tsdbInsertMetaRecord(pMeta->mfh, super->tableId.uid, buf, bufLen);
}
tsdbEncodeTable(table, buf, &bufLen);
tsdbInsertMetaRecord(pMeta->mfh, table->tableId.uid, buf, bufLen);
tfree(buf);
return 0;
}
int tsdbDropTable(TSDB_REPO_T *repo, STableId tableId) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
if (pRepo == NULL) return -1;
STsdbMeta *pMeta = pRepo->tsdbMeta;
if (pMeta == NULL) return -1;
STable *pTable = tsdbGetTableByUid(pMeta, tableId.uid);
if (pTable == NULL) {
tsdbError("vgId:%d, failed to drop table since table not exists! tid:%d, uid:" PRId64, pRepo->config.tsdbId,
tableId.tid, tableId.uid);
return -1;
}
tsdbTrace("vgId:%d, table %s is dropped! tid:%d, uid:%" PRId64, pRepo->config.tsdbId, varDataVal(pTable->name),
tableId.tid, tableId.uid);
if (tsdbRemoveTableFromMeta(pMeta, pTable, true) < 0) return -1;
return 0;
}
void* tsdbGetTableTagVal(TSDB_REPO_T* repo, const STableId* id, int32_t colId, int16_t type, int16_t bytes) {
STsdbMeta* pMeta = tsdbGetMeta(repo);
STable* pTable = tsdbGetTableByUid(pMeta, id->uid);
STSchema *pSchema = tsdbGetTableTagSchema(pMeta, pTable);
STColumn *pCol = tdGetColOfID(pSchema, colId);
if (pCol == NULL) {
return NULL; // No matched tag volumn
}
char* val = tdGetKVRowValOfCol(pTable->tagVal, colId);
assert(type == pCol->type && bytes == pCol->bytes);
if (val != NULL && IS_VAR_DATA_TYPE(type)) {
assert(varDataLen(val) < pCol->bytes);
}
return val;
}
char *tsdbGetTableName(TSDB_REPO_T *repo, const STableId *id) {
STsdbMeta *pMeta = tsdbGetMeta(repo);
STable * pTable = tsdbGetTableByUid(pMeta, id->uid);
if (pTable == NULL) {
return NULL;
} else {
return (char *)pTable->name;
}
}
STableCfg *tsdbCreateTableCfgFromMsg(SMDCreateTableMsg *pMsg) {
if (pMsg == NULL) return NULL;
SSchema * pSchema = (SSchema *)pMsg->data;
int16_t numOfCols = htons(pMsg->numOfColumns);
int16_t numOfTags = htons(pMsg->numOfTags);
STSchemaBuilder schemaBuilder = {0};
STableCfg *pCfg = (STableCfg *)calloc(1, sizeof(STableCfg));
if (pCfg == NULL) return NULL;
if (tsdbInitTableCfg(pCfg, pMsg->tableType, htobe64(pMsg->uid), htonl(pMsg->sid)) < 0) goto _err;
if (tdInitTSchemaBuilder(&schemaBuilder, htonl(pMsg->sversion)) < 0) goto _err;
for (int i = 0; i < numOfCols; i++) {
tdAddColToSchema(&schemaBuilder, pSchema[i].type, htons(pSchema[i].colId), htons(pSchema[i].bytes));
}
if (tsdbTableSetSchema(pCfg, tdGetSchemaFromBuilder(&schemaBuilder), false) < 0) goto _err;
if (tsdbTableSetName(pCfg, pMsg->tableId, true) < 0) goto _err;
if (numOfTags > 0) {
// Decode tag schema
tdResetTSchemaBuilder(&schemaBuilder, htonl(pMsg->tversion));
for (int i = numOfCols; i < numOfCols + numOfTags; i++) {
tdAddColToSchema(&schemaBuilder, pSchema[i].type, htons(pSchema[i].colId), htons(pSchema[i].bytes));
}
if (tsdbTableSetTagSchema(pCfg, tdGetSchemaFromBuilder(&schemaBuilder), false) < 0) goto _err;
if (tsdbTableSetSName(pCfg, pMsg->superTableId, true) < 0) goto _err;
if (tsdbTableSetSuperUid(pCfg, htobe64(pMsg->superTableUid)) < 0) goto _err;
// Decode tag values
if (pMsg->tagDataLen) {
int accBytes = 0;
char *pTagData = pMsg->data + (numOfCols + numOfTags) * sizeof(SSchema);
SKVRowBuilder kvRowBuilder = {0};
if (tdInitKVRowBuilder(&kvRowBuilder) < 0) goto _err;
for (int i = numOfCols; i < numOfCols + numOfTags; i++) {
tdAddColToKVRow(&kvRowBuilder, htons(pSchema[i].colId), pSchema[i].type, pTagData + accBytes);
accBytes += htons(pSchema[i].bytes);
}
tsdbTableSetTagValue(pCfg, tdGetKVRowFromBuilder(&kvRowBuilder), false);
tdDestroyKVRowBuilder(&kvRowBuilder);
}
}
if (pMsg->tableType == TSDB_STREAM_TABLE) {
char *sql = pMsg->data + (numOfCols + numOfTags) * sizeof(SSchema);
tsdbTableSetStreamSql(pCfg, sql, true);
}
tdDestroyTSchemaBuilder(&schemaBuilder);
return pCfg;
_err:
tdDestroyTSchemaBuilder(&schemaBuilder);
tsdbClearTableCfg(pCfg);
tfree(pCfg);
return NULL;
}
// ------------------ INTERNAL FUNCTIONS ------------------
STsdbMeta *tsdbNewMeta(STsdbCfg *pCfg) {
STsdbMeta* pMeta = (STsdbMeta*)calloc(1, sizeof(*pMeta));
if (pMeta == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
int code = pthread_rwlock_init(&pMeta->rwLock, NULL);
if (code != 0) {
terrno = TAOS_SYSTEM_ERROR(code);
goto _err;
}
pMeta->tables = (STable **)calloc(pCfg->maxTables, sizeof(STable *));
if (pMeta->tables == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pMeta->superList = tdListNew(sizeof(STable *));
if (pMeta->superList == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
pMeta->uidMap = taosHashInit(pCfg->maxTables, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false);
if (pMeta->uidMap == NULL) {
terrno = TSDB_CODE_TDB_OUT_OF_MEMORY;
goto _err;
}
return pMeta;
_err;
tsdbFreeMeta(pMeta);
return NULL;
}
void tsdbFreeMeta(STsdbMeta *pMeta) {
if (pMeta) {
taosHashCleanup(pMeta->uidMap);
tdListFree(pMeta->superList);
tfree(pMeta->tables);
pthread_rwlock_destroy(&pMeta->rwLock);
free(pMeta);
}
}
int tsdbOpenMeta(STsdbRepo *pRepo) {
// TODO
}
int tsdbCloseMeta(STsdbRepo *pRepo) {
// TODO
}
STSchema *tsdbGetTableSchema(STsdbMeta *pMeta, STable *pTable) {
if (pTable->type == TSDB_NORMAL_TABLE || pTable->type == TSDB_SUPER_TABLE || pTable->type == TSDB_STREAM_TABLE) {
return pTable->schema[pTable->numOfSchemas - 1];
} else if (pTable->type == TSDB_CHILD_TABLE) {
STable *pSuper = tsdbGetTableByUid(pMeta, pTable->superUid);
if (pSuper == NULL) return NULL;
return pSuper->schema[pSuper->numOfSchemas-1];
} else {
return NULL;
}
}
STable *tsdbGetTableByUid(STsdbMeta *pMeta, uint64_t uid) {
void *ptr = taosHashGet(pMeta->uidMap, (char *)(&uid), sizeof(uid));
if (ptr == NULL) return NULL;
return *(STable **)ptr;
}
STSchema *tsdbGetTableSchemaByVersion(STsdbMeta *pMeta, STable *pTable, int16_t version) {
STable *pSearchTable = NULL;
if (pTable->type == TSDB_CHILD_TABLE) {
pSearchTable = tsdbGetTableByUid(pMeta, pTable->superUid);
} else {
pSearchTable = pTable;
}
ASSERT(pSearchTable != NULL);
void *ptr = taosbsearch(&version, pSearchTable->schema, pSearchTable->numOfSchemas, sizeof(STSchema *),
tsdbCompareSchemaVersion, TD_EQ);
if (ptr == NULL) return NULL;
return *(STSchema **)ptr;
}
STSchema * tsdbGetTableTagSchema(STsdbMeta *pMeta, STable *pTable) {
if (pTable->type == TSDB_SUPER_TABLE) {
return pTable->tagSchema;
} else if (pTable->type == TSDB_CHILD_TABLE) {
STable *pSuper = tsdbGetTableByUid(pMeta, pTable->superUid);
if (pSuper == NULL) return NULL;
return pSuper->tagSchema;
} else {
return NULL;
}
}
int tsdbUpdateTable(STsdbMeta *pMeta, STable *pTable, STableCfg *pCfg) {
ASSERT(pTable->type != TSDB_CHILD_TABLE);
bool isChanged = false;
if (pTable->type == TSDB_SUPER_TABLE) {
if (schemaVersion(pTable->tagSchema) < schemaVersion(pCfg->tagSchema)) {
int32_t code = tsdbUpdateTableTagSchema(pTable, pCfg->tagSchema);
if (code != TSDB_CODE_SUCCESS) return code;
}
isChanged = true;
}
STSchema *pTSchema = tsdbGetTableSchema(pMeta, pTable);
if (schemaVersion(pTSchema) < schemaVersion(pCfg->schema)) {
if (pTable->numOfSchemas < TSDB_MAX_TABLE_SCHEMAS) {
pTable->schema[pTable->numOfSchemas++] = tdDupSchema(pCfg->schema);
} else {
ASSERT(pTable->numOfSchemas == TSDB_MAX_TABLE_SCHEMAS);
STSchema *tSchema = tdDupSchema(pCfg->schema);
tdFreeSchema(pTable->schema[0]);
memmove(pTable->schema, pTable->schema + 1, sizeof(STSchema *) * (TSDB_MAX_TABLE_SCHEMAS - 1));
pTable->schema[pTable->numOfSchemas - 1] = tSchema;
}
isChanged = true;
}
if (isChanged) {
char *buf = malloc(1024 * 1024);
int bufLen = 0;
tsdbEncodeTable(pTable, buf, &bufLen);
tsdbInsertMetaRecord(pMeta->mfh, pTable->tableId.uid, buf, bufLen);
free(buf);
}
return TSDB_CODE_SUCCESS;
}
char *getTSTupleKey(const void * data) {
SDataRow row = (SDataRow)data;
return POINTER_SHIFT(row, TD_DATA_ROW_HEAD_SIZE);
}
// ------------------ LOCAL FUNCTIONS ------------------
static void tsdbEncodeTable(STable *pTable, char *buf, int *contLen) {
if (pTable == NULL) return;
void *ptr = buf;
......@@ -68,17 +386,7 @@ void tsdbEncodeTable(STable *pTable, char *buf, int *contLen) {
*contLen = (char *)ptr - buf;
}
/**
* Decode from an encoded binary
* ASSUMPTIONS: valid parameters
*
* @param cont binary object
* @param contLen binary length
*
* @return TSDB table object for success
* NULL for failure
*/
STable *tsdbDecodeTable(void *cont, int contLen) {
static STable *tsdbDecodeTable(void *cont, int contLen) {
STable *pTable = (STable *)calloc(1, sizeof(STable));
if (pTable == NULL) return NULL;
......@@ -127,128 +435,6 @@ STable *tsdbDecodeTable(void *cont, int contLen) {
return pTable;
}
void tsdbFreeEncode(void *cont) {
if (cont != NULL) free(cont);
}
static char* getTagIndexKey(const void* pData) {
STableIndexElem* elem = (STableIndexElem*) pData;
STSchema* pSchema = tsdbGetTableTagSchema(elem->pMeta, elem->pTable);
STColumn* pCol = &pSchema->columns[DEFAULT_TAG_INDEX_COLUMN];
void * res = tdGetKVRowValOfCol(elem->pTable->tagVal, pCol->colId);
return res;
}
int tsdbRestoreTable(void *pHandle, void *cont, int contLen) {
STsdbMeta *pMeta = (STsdbMeta *)pHandle;
STable *pTable = tsdbDecodeTable(cont, contLen);
if (pTable == NULL) return -1;
if (pTable->type == TSDB_SUPER_TABLE) {
STColumn* pColSchema = schemaColAt(pTable->tagSchema, 0);
pTable->pIndex = tSkipListCreate(TSDB_SUPER_TABLE_SL_LEVEL, pColSchema->type, pColSchema->bytes,
1, 0, 1, getTagIndexKey);
}
tsdbAddTableToMeta(pMeta, pTable, false);
return 0;
}
void tsdbOrgMeta(void *pHandle) {
STsdbMeta *pMeta = (STsdbMeta *)pHandle;
for (int i = 1; i < pMeta->maxTables; i++) {
STable *pTable = pMeta->tables[i];
if (pTable != NULL && pTable->type == TSDB_CHILD_TABLE) {
tsdbAddTableIntoIndex(pMeta, pTable);
}
}
}
/**
* Initialize the meta handle
* ASSUMPTIONS: VALID PARAMETER
*/
STsdbMeta *tsdbInitMeta(char *rootDir, int32_t maxTables, void *pRepo) {
STsdbMeta *pMeta = (STsdbMeta *)malloc(sizeof(STsdbMeta));
if (pMeta == NULL) return NULL;
pMeta->maxTables = maxTables;
pMeta->nTables = 0;
pMeta->superList = NULL;
pMeta->tables = (STable **)calloc(maxTables, sizeof(STable *));
pMeta->maxRowBytes = 0;
pMeta->maxCols = 0;
pMeta->pRepo = pRepo;
if (pMeta->tables == NULL) {
free(pMeta);
return NULL;
}
pMeta->map = taosHashInit(maxTables * TSDB_META_HASH_FRACTION, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false);
if (pMeta->map == NULL) {
free(pMeta->tables);
free(pMeta);
return NULL;
}
pMeta->mfh = tsdbInitMetaFile(rootDir, maxTables, tsdbRestoreTable, tsdbOrgMeta, pMeta);
if (pMeta->mfh == NULL) {
taosHashCleanup(pMeta->map);
free(pMeta->tables);
free(pMeta);
return NULL;
}
return pMeta;
}
int32_t tsdbFreeMeta(STsdbMeta *pMeta) {
STsdbRepo *pRepo = (STsdbRepo *)pMeta->pRepo;
if (pMeta == NULL) return 0;
tsdbCloseMetaFile(pMeta->mfh);
for (int i = 1; i < pMeta->maxTables; i++) {
if (pMeta->tables[i] != NULL) {
STable *pTable = pMeta->tables[i];
if (pTable->type == TSDB_STREAM_TABLE) (*pRepo->appH.cqDropFunc)(pTable->cqhandle);
tsdbFreeTable(pTable);
}
}
free(pMeta->tables);
STable *pTable = pMeta->superList;
while (pTable != NULL) {
STable *pTemp = pTable;
pTable = pTemp->next;
tsdbFreeTable(pTemp);
}
taosHashCleanup(pMeta->map);
free(pMeta);
return 0;
}
// Get the newest table schema
STSchema *tsdbGetTableSchema(STsdbMeta *pMeta, STable *pTable) {
if (pTable->type == TSDB_NORMAL_TABLE || pTable->type == TSDB_SUPER_TABLE || pTable->type == TSDB_STREAM_TABLE) {
return pTable->schema[pTable->numOfSchemas - 1];
} else if (pTable->type == TSDB_CHILD_TABLE) {
STable *pSuper = tsdbGetTableByUid(pMeta, pTable->superUid);
if (pSuper == NULL) return NULL;
return pSuper->schema[pSuper->numOfSchemas-1];
} else {
return NULL;
}
}
static int tsdbCompareSchemaVersion(const void *key1, const void *key2) {
if (*(int16_t *)key1 < (*(STSchema **)key2)->version) {
return -1;
......@@ -259,65 +445,43 @@ static int tsdbCompareSchemaVersion(const void *key1, const void *key2) {
}
}
STSchema *tsdbGetTableSchemaByVersion(STsdbMeta *pMeta, STable *pTable, int16_t version) {
STable *pSearchTable = NULL;
if (pTable->type == TSDB_CHILD_TABLE) {
pSearchTable = tsdbGetTableByUid(pMeta, pTable->superUid);
} else {
pSearchTable = pTable;
}
ASSERT(pSearchTable != NULL);
void *ptr = taosbsearch(&version, pSearchTable->schema, pSearchTable->numOfSchemas, sizeof(STSchema *),
tsdbCompareSchemaVersion, TD_EQ);
if (ptr == NULL) return NULL;
return *(STSchema **)ptr;
}
static int tsdbRestoreTable(void *pHandle, void *cont, int contLen) {
STsdbMeta *pMeta = (STsdbMeta *)pHandle;
STSchema * tsdbGetTableTagSchema(STsdbMeta *pMeta, STable *pTable) {
STable *pTable = tsdbDecodeTable(cont, contLen);
if (pTable == NULL) return -1;
if (pTable->type == TSDB_SUPER_TABLE) {
return pTable->tagSchema;
} else if (pTable->type == TSDB_CHILD_TABLE) {
STable *pSuper = tsdbGetTableByUid(pMeta, pTable->superUid);
if (pSuper == NULL) return NULL;
return pSuper->tagSchema;
} else {
return NULL;
STColumn* pColSchema = schemaColAt(pTable->tagSchema, 0);
pTable->pIndex = tSkipListCreate(TSDB_SUPER_TABLE_SL_LEVEL, pColSchema->type, pColSchema->bytes,
1, 0, 1, getTagIndexKey);
}
}
void* tsdbGetTableTagVal(TSDB_REPO_T* repo, const STableId* id, int32_t colId, int16_t type, int16_t bytes) {
STsdbMeta* pMeta = tsdbGetMeta(repo);
STable* pTable = tsdbGetTableByUid(pMeta, id->uid);
STSchema *pSchema = tsdbGetTableTagSchema(pMeta, pTable);
STColumn *pCol = tdGetColOfID(pSchema, colId);
if (pCol == NULL) {
return NULL; // No matched tag volumn
}
char* val = tdGetKVRowValOfCol(pTable->tagVal, colId);
assert(type == pCol->type && bytes == pCol->bytes);
if (val != NULL && IS_VAR_DATA_TYPE(type)) {
assert(varDataLen(val) < pCol->bytes);
}
return val;
tsdbAddTableToMeta(pMeta, pTable, false);
return 0;
}
char* tsdbGetTableName(TSDB_REPO_T *repo, const STableId* id) {
STsdbMeta* pMeta = tsdbGetMeta(repo);
STable* pTable = tsdbGetTableByUid(pMeta, id->uid);
if (pTable == NULL) {
return NULL;
} else {
return (char*) pTable->name;
static void tsdbOrgMeta(void *pHandle) {
STsdbMeta *pMeta = (STsdbMeta *)pHandle;
for (int i = 1; i < pMeta->maxTables; i++) {
STable *pTable = pMeta->tables[i];
if (pTable != NULL && pTable->type == TSDB_CHILD_TABLE) {
tsdbAddTableIntoIndex(pMeta, pTable);
}
}
}
static char *getTagIndexKey(const void *pData) {
STableIndexElem *elem = (STableIndexElem *)pData;
STSchema *pSchema = tsdbGetTableTagSchema(elem->pMeta, elem->pTable);
STColumn *pCol = &pSchema->columns[DEFAULT_TAG_INDEX_COLUMN];
void * res = tdGetKVRowValOfCol(elem->pTable->tagVal, pCol->colId);
return res;
}
static STable *tsdbNewTable(STableCfg *pCfg, bool isSuper) {
STable *pTable = NULL;
size_t tsize = 0;
......@@ -395,234 +559,6 @@ _err:
return NULL;
}
static int tsdbUpdateTableTagSchema(STable *pTable, STSchema *newSchema) {
ASSERT(pTable->type == TSDB_SUPER_TABLE);
ASSERT(schemaVersion(pTable->tagSchema) < schemaVersion(newSchema));
STSchema *pOldSchema = pTable->tagSchema;
STSchema *pNewSchema = tdDupSchema(newSchema);
if (pNewSchema == NULL) return TSDB_CODE_TDB_OUT_OF_MEMORY;
pTable->tagSchema = pNewSchema;
tdFreeSchema(pOldSchema);
return TSDB_CODE_SUCCESS;
}
int tsdbUpdateTable(STsdbMeta *pMeta, STable *pTable, STableCfg *pCfg) {
ASSERT(pTable->type != TSDB_CHILD_TABLE);
bool isChanged = false;
if (pTable->type == TSDB_SUPER_TABLE) {
if (schemaVersion(pTable->tagSchema) < schemaVersion(pCfg->tagSchema)) {
int32_t code = tsdbUpdateTableTagSchema(pTable, pCfg->tagSchema);
if (code != TSDB_CODE_SUCCESS) return code;
}
isChanged = true;
}
STSchema *pTSchema = tsdbGetTableSchema(pMeta, pTable);
if (schemaVersion(pTSchema) < schemaVersion(pCfg->schema)) {
if (pTable->numOfSchemas < TSDB_MAX_TABLE_SCHEMAS) {
pTable->schema[pTable->numOfSchemas++] = tdDupSchema(pCfg->schema);
} else {
ASSERT(pTable->numOfSchemas == TSDB_MAX_TABLE_SCHEMAS);
STSchema *tSchema = tdDupSchema(pCfg->schema);
tdFreeSchema(pTable->schema[0]);
memmove(pTable->schema, pTable->schema+1, sizeof(STSchema *) * (TSDB_MAX_TABLE_SCHEMAS - 1));
pTable->schema[pTable->numOfSchemas-1] = tSchema;
}
isChanged = true;
}
if (isChanged) {
char *buf = malloc(1024 * 1024);
int bufLen = 0;
tsdbEncodeTable(pTable, buf, &bufLen);
tsdbInsertMetaRecord(pMeta->mfh, pTable->tableId.uid, buf, bufLen);
free(buf);
}
return TSDB_CODE_SUCCESS;
}
int tsdbCreateTable(TSDB_REPO_T *repo, STableCfg *pCfg) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
STsdbMeta *pMeta = pRepo->tsdbMeta;
if (tsdbCheckTableCfg(pCfg) < 0) return -1;
STable *pTable = tsdbGetTableByUid(pMeta, pCfg->tableId.uid);
if (pTable != NULL) {
tsdbError("vgId:%d table %s already exists, tid %d uid %" PRId64, pRepo->config.tsdbId, varDataVal(pTable->name),
pTable->tableId.tid, pTable->tableId.uid);
return TSDB_CODE_TDB_TABLE_ALREADY_EXIST;
}
STable *super = NULL;
int newSuper = 0;
if (pCfg->type == TSDB_CHILD_TABLE) {
super = tsdbGetTableByUid(pMeta, pCfg->superUid);
if (super == NULL) { // super table not exists, try to create it
newSuper = 1;
super = tsdbNewTable(pCfg, true);
if (super == NULL) return -1;
} else {
if (super->type != TSDB_SUPER_TABLE) return -1;
if (super->tableId.uid != pCfg->superUid) return -1;
tsdbUpdateTable(pMeta, super, pCfg);
}
}
STable *table = tsdbNewTable(pCfg, false);
if (table == NULL) {
if (newSuper) {
tsdbFreeTable(super);
return -1;
}
}
table->lastKey = TSKEY_INITIAL_VAL;
// Register to meta
if (newSuper) {
tsdbAddTableToMeta(pMeta, super, true);
tsdbTrace("vgId:%d, super table %s is created! uid:%" PRId64, pRepo->config.tsdbId, varDataVal(super->name),
super->tableId.uid);
}
tsdbAddTableToMeta(pMeta, table, true);
tsdbTrace("vgId:%d, table %s is created! tid:%d, uid:%" PRId64, pRepo->config.tsdbId, varDataVal(table->name),
table->tableId.tid, table->tableId.uid);
// Write to meta file
int bufLen = 0;
char *buf = malloc(1024*1024);
if (newSuper) {
tsdbEncodeTable(super, buf, &bufLen);
tsdbInsertMetaRecord(pMeta->mfh, super->tableId.uid, buf, bufLen);
}
tsdbEncodeTable(table, buf, &bufLen);
tsdbInsertMetaRecord(pMeta->mfh, table->tableId.uid, buf, bufLen);
tfree(buf);
return 0;
}
/**
* Check if a table is valid to insert.
* @return NULL for invalid and the pointer to the table if valid
*/
STable *tsdbIsValidTableToInsert(STsdbMeta *pMeta, STableId tableId) {
STable *pTable = tsdbGetTableByUid(pMeta, tableId.uid);
if (pTable == NULL) {
return NULL;
}
if (TSDB_TABLE_IS_SUPER_TABLE(pTable)) return NULL;
if (pTable->tableId.tid != tableId.tid) return NULL;
return pTable;
}
STableCfg *tsdbCreateTableCfgFromMsg(SMDCreateTableMsg *pMsg) {
if (pMsg == NULL) return NULL;
SSchema * pSchema = (SSchema *)pMsg->data;
int16_t numOfCols = htons(pMsg->numOfColumns);
int16_t numOfTags = htons(pMsg->numOfTags);
STSchemaBuilder schemaBuilder = {0};
STableCfg *pCfg = (STableCfg *)calloc(1, sizeof(STableCfg));
if (pCfg == NULL) return NULL;
if (tsdbInitTableCfg(pCfg, pMsg->tableType, htobe64(pMsg->uid), htonl(pMsg->sid)) < 0) goto _err;
if (tdInitTSchemaBuilder(&schemaBuilder, htonl(pMsg->sversion)) < 0) goto _err;
for (int i = 0; i < numOfCols; i++) {
tdAddColToSchema(&schemaBuilder, pSchema[i].type, htons(pSchema[i].colId), htons(pSchema[i].bytes));
}
if (tsdbTableSetSchema(pCfg, tdGetSchemaFromBuilder(&schemaBuilder), false) < 0) goto _err;
if (tsdbTableSetName(pCfg, pMsg->tableId, true) < 0) goto _err;
if (numOfTags > 0) {
// Decode tag schema
tdResetTSchemaBuilder(&schemaBuilder, htonl(pMsg->tversion));
for (int i = numOfCols; i < numOfCols + numOfTags; i++) {
tdAddColToSchema(&schemaBuilder, pSchema[i].type, htons(pSchema[i].colId), htons(pSchema[i].bytes));
}
if (tsdbTableSetTagSchema(pCfg, tdGetSchemaFromBuilder(&schemaBuilder), false) < 0) goto _err;
if (tsdbTableSetSName(pCfg, pMsg->superTableId, true) < 0) goto _err;
if (tsdbTableSetSuperUid(pCfg, htobe64(pMsg->superTableUid)) < 0) goto _err;
// Decode tag values
if (pMsg->tagDataLen) {
int accBytes = 0;
char *pTagData = pMsg->data + (numOfCols + numOfTags) * sizeof(SSchema);
SKVRowBuilder kvRowBuilder = {0};
if (tdInitKVRowBuilder(&kvRowBuilder) < 0) goto _err;
for (int i = numOfCols; i < numOfCols + numOfTags; i++) {
tdAddColToKVRow(&kvRowBuilder, htons(pSchema[i].colId), pSchema[i].type, pTagData + accBytes);
accBytes += htons(pSchema[i].bytes);
}
tsdbTableSetTagValue(pCfg, tdGetKVRowFromBuilder(&kvRowBuilder), false);
tdDestroyKVRowBuilder(&kvRowBuilder);
}
}
if (pMsg->tableType == TSDB_STREAM_TABLE) {
char *sql = pMsg->data + (numOfCols + numOfTags) * sizeof(SSchema);
tsdbTableSetStreamSql(pCfg, sql, true);
}
tdDestroyTSchemaBuilder(&schemaBuilder);
return pCfg;
_err:
tdDestroyTSchemaBuilder(&schemaBuilder);
tsdbClearTableCfg(pCfg);
tfree(pCfg);
return NULL;
}
// int32_t tsdbDropTableImpl(STsdbMeta *pMeta, STableId tableId) {
int tsdbDropTable(TSDB_REPO_T *repo, STableId tableId) {
STsdbRepo *pRepo = (STsdbRepo *)repo;
if (pRepo == NULL) return -1;
STsdbMeta *pMeta = pRepo->tsdbMeta;
if (pMeta == NULL) return -1;
STable *pTable = tsdbGetTableByUid(pMeta, tableId.uid);
if (pTable == NULL) {
tsdbError("vgId:%d, failed to drop table since table not exists! tid:%d, uid:" PRId64, pRepo->config.tsdbId,
tableId.tid, tableId.uid);
return -1;
}
tsdbTrace("vgId:%d, table %s is dropped! tid:%d, uid:%" PRId64, pRepo->config.tsdbId, varDataVal(pTable->name),
tableId.tid, tableId.uid);
if (tsdbRemoveTableFromMeta(pMeta, pTable, true) < 0) return -1;
return 0;
}
// int32_t tsdbInsertRowToTableImpl(SSkipListNode *pNode, STable *pTable) {
// tSkipListPut(pTable->mem->pData, pNode);
// return 0;
// }
static void tsdbFreeMemTable(SMemTable *pMemTable) {
if (pMemTable) {
tSkipListDestroy(pMemTable->pData);
}
free(pMemTable);
}
static int tsdbFreeTable(STable *pTable) {
if (pTable == NULL) return 0;
......@@ -653,17 +589,16 @@ static int tsdbFreeTable(STable *pTable) {
return 0;
}
static int32_t tsdbCheckTableCfg(STableCfg *pCfg) {
// TODO
return 0;
}
STable *tsdbGetTableByUid(STsdbMeta *pMeta, uint64_t uid) {
void *ptr = taosHashGet(pMeta->map, (char *)(&uid), sizeof(uid));
if (ptr == NULL) return NULL;
static int tsdbUpdateTableTagSchema(STable *pTable, STSchema *newSchema) {
ASSERT(pTable->type == TSDB_SUPER_TABLE);
ASSERT(schemaVersion(pTable->tagSchema) < schemaVersion(newSchema));
STSchema *pOldSchema = pTable->tagSchema;
STSchema *pNewSchema = tdDupSchema(newSchema);
if (pNewSchema == NULL) return TSDB_CODE_TDB_OUT_OF_MEMORY;
pTable->tagSchema = pNewSchema;
tdFreeSchema(pOldSchema);
return *(STable **)ptr;
return TSDB_CODE_SUCCESS;
}
static int tsdbAddTableToMeta(STsdbMeta *pMeta, STable *pTable, bool addIdx) {
......@@ -746,7 +681,7 @@ static int tsdbRemoveTableFromMeta(STsdbMeta *pMeta, STable *pTable, bool rmFrom
return 0;
}
int tsdbAddTableIntoIndex(STsdbMeta *pMeta, STable *pTable) {
static int tsdbAddTableIntoIndex(STsdbMeta *pMeta, STable *pTable) {
assert(pTable->type == TSDB_CHILD_TABLE && pTable != NULL);
STable* pSTable = tsdbGetTableByUid(pMeta, pTable->superUid);
assert(pSTable != NULL);
......@@ -771,7 +706,7 @@ int tsdbAddTableIntoIndex(STsdbMeta *pMeta, STable *pTable) {
return 0;
}
int tsdbRemoveTableFromIndex(STsdbMeta *pMeta, STable *pTable) {
static int tsdbRemoveTableFromIndex(STsdbMeta *pMeta, STable *pTable) {
assert(pTable->type == TSDB_CHILD_TABLE && pTable != NULL);
STable* pSTable = tsdbGetTableByUid(pMeta, pTable->superUid);
......@@ -799,11 +734,13 @@ int tsdbRemoveTableFromIndex(STsdbMeta *pMeta, STable *pTable) {
return 0;
}
#if 0
#define TSDB_SUPER_TABLE_SL_LEVEL 5 // TODO: may change here
// #define TSDB_META_FILE_NAME "META"
char *getTSTupleKey(const void * data) {
SDataRow row = (SDataRow)data;
return POINTER_SHIFT(row, TD_DATA_ROW_HEAD_SIZE);
}
static int tsdbInitTableCfg(STableCfg *config, ETableType type, uint64_t uid, int32_t tid) {
if (config == NULL) return -1;
......@@ -914,3 +851,4 @@ static int tsdbTableSetStreamSql(STableCfg *config, char *sql, bool dup) {
return 0;
}
#endif
\ 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 "os.h"
#include "taosdef.h"
#include "hash.h"
#include "tsdbMain.h"
#define TSDB_META_FILE_VERSION_MAJOR 1
#define TSDB_META_FILE_VERSION_MINOR 0
#define TSDB_META_FILE_HEADER_SIZE 512
typedef struct {
int32_t offset;
int32_t size;
uint64_t uid;
} SRecordInfo;
// static int32_t tsdbGetMetaFileName(char *rootDir, char *fname);
// static int32_t tsdbCheckMetaHeader(int fd);
static int32_t tsdbWriteMetaHeader(int fd);
static int tsdbCreateMetaFile(char *fname);
static int tsdbRestoreFromMetaFile(char *fname, SMetaFile *mfh);
SMetaFile *tsdbInitMetaFile(char *rootDir, int32_t maxTables, iterFunc iFunc, afterFunc aFunc, void *appH) {
char fname[128] = "\0";
if (tsdbGetMetaFileName(rootDir, fname) < 0) return NULL;
SMetaFile *mfh = (SMetaFile *)calloc(1, sizeof(SMetaFile));
if (mfh == NULL) return NULL;
mfh->iFunc = iFunc;
mfh->aFunc = aFunc;
mfh->appH = appH;
mfh->nDel = 0;
mfh->tombSize = 0;
mfh->size = 0;
// OPEN MAP
mfh->map =
taosHashInit(maxTables * TSDB_META_HASH_FRACTION, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false);
if (mfh->map == NULL) {
free(mfh);
return NULL;
}
// OPEN FILE
if (access(fname, F_OK) < 0) { // file not exists
mfh->fd = tsdbCreateMetaFile(fname);
if (mfh->fd < 0) {
taosHashCleanup(mfh->map);
free(mfh);
return NULL;
}
mfh->size += TSDB_META_FILE_HEADER_SIZE;
} else { // file exists, recover from file
if (tsdbRestoreFromMetaFile(fname, mfh) < 0) {
taosHashCleanup(mfh->map);
free(mfh);
return NULL;
}
}
return mfh;
}
int32_t tsdbInsertMetaRecord(SMetaFile *mfh, uint64_t uid, void *cont, int32_t contLen) {
if (taosHashGet(mfh->map, (char *)(&uid), sizeof(uid)) != NULL) {
return -1;
}
SRecordInfo info;
info.offset = mfh->size;
info.size = contLen;
info.uid = uid;
mfh->size += (contLen + sizeof(SRecordInfo));
if (taosHashPut(mfh->map, (char *)(&uid), sizeof(uid), (void *)(&info), sizeof(SRecordInfo)) < 0) {
return -1;
}
// TODO: make below a function to implement
if (lseek(mfh->fd, info.offset, SEEK_SET) < 0) {
return -1;
}
if (write(mfh->fd, (void *)(&info), sizeof(SRecordInfo)) < 0) {
return -1;
}
if (write(mfh->fd, cont, contLen) < 0) {
return -1;
}
// fsync(mfh->fd);
mfh->tombSize++;
return 0;
}
int32_t tsdbDeleteMetaRecord(SMetaFile *mfh, uint64_t uid) {
char *ptr = taosHashGet(mfh->map, (char *)(&uid), sizeof(uid));
if (ptr == NULL) return -1;
SRecordInfo info = *(SRecordInfo *)ptr;
// Remove record from hash table
taosHashRemove(mfh->map, (char *)(&uid), sizeof(uid));
// Remove record from file
info.offset = -info.offset;
if (lseek(mfh->fd, -info.offset, SEEK_CUR) < 0) {
return -1;
}
if (write(mfh->fd, (void *)(&info), sizeof(SRecordInfo)) < 0) {
return -1;
}
// fsync(mfh->fd);
mfh->nDel++;
return 0;
}
int32_t tsdbUpdateMetaRecord(SMetaFile *mfh, uint64_t uid, void *cont, int32_t contLen) {
char *ptr = taosHashGet(mfh->map, (char *)(&uid), sizeof(uid));
if (ptr == NULL) return -1;
SRecordInfo info = *(SRecordInfo *)ptr;
// Update the hash table
if (taosHashPut(mfh->map, (char *)(&uid), sizeof(uid), (void *)(&info), sizeof(SRecordInfo)) < 0) {
return -1;
}
// Update record in file
if (info.size >= contLen) { // Just update it in place
info.size = contLen;
} else { // Just append to the end of file
info.offset = mfh->size;
info.size = contLen;
mfh->size += contLen;
}
if (lseek(mfh->fd, -info.offset, SEEK_CUR) < 0) {
return -1;
}
if (write(mfh->fd, (void *)(&info), sizeof(SRecordInfo)) < 0) {
return -1;
}
// fsync(mfh->fd);
return 0;
}
void tsdbCloseMetaFile(SMetaFile *mfh) {
if (mfh == NULL) return;
close(mfh->fd);
taosHashCleanup(mfh->map);
tfree(mfh);
}
int32_t tsdbGetMetaFileName(char *rootDir, char *fname) {
if (rootDir == NULL) return -1;
sprintf(fname, "%s/%s", rootDir, TSDB_META_FILE_NAME);
return 0;
}
// static int32_t tsdbCheckMetaHeader(int fd) {
// // TODO: write the meta file header check function
// return 0;
// }
static int32_t tsdbWriteMetaHeader(int fd) {
// TODO: write the meta file header to file
char head[TSDB_META_FILE_HEADER_SIZE] = "\0";
sprintf(head, "version: %d.%d", TSDB_META_FILE_VERSION_MAJOR, TSDB_META_FILE_VERSION_MINOR);
write(fd, (void *)head, TSDB_META_FILE_HEADER_SIZE);
return 0;
}
// static int32_t tsdbReadMetaHeader(int fd) {
// lseek(fd, TSDB_META_FILE_HEADER_SIZE, SEEK_SET);
// return 0;
// }
static int tsdbCreateMetaFile(char *fname) {
int fd = open(fname, O_RDWR | O_CREAT, 0755);
if (fd < 0) return -1;
if (tsdbWriteMetaHeader(fd) < 0) {
close(fd);
return -1;
}
return fd;
}
static int tsdbCheckMetaFileIntegrety(int fd) {
// TODO
return 0;
}
static int tsdbRestoreFromMetaFile(char *fname, SMetaFile *mfh) {
int fd = open(fname, O_RDWR);
if (fd < 0) return -1;
if (tsdbCheckMetaFileIntegrety(fd) < 0) {
// TODO: decide if to auto-recover the file
close(fd);
return -1;
}
if (lseek(fd, TSDB_META_FILE_HEADER_SIZE, SEEK_SET) < 0) {
// TODO: deal with the error
close(fd);
return -1;
}
mfh->size += TSDB_META_FILE_HEADER_SIZE;
mfh->fd = fd;
void *buf = NULL;
// int buf_size = 0;
SRecordInfo info;
while (1) {
if (read(mfh->fd, (void *)(&info), sizeof(SRecordInfo)) == 0) break;
if (info.offset < 0) {
mfh->size += (info.size + sizeof(SRecordInfo));
mfh->tombSize += (info.size + sizeof(SRecordInfo));
lseek(mfh->fd, info.size, SEEK_CUR);
mfh->size = mfh->size + sizeof(SRecordInfo) + info.size;
mfh->tombSize = mfh->tombSize + sizeof(SRecordInfo) + info.size;
} else {
if (taosHashPut(mfh->map, (char *)(&info.uid), sizeof(info.uid), (void *)(&info), sizeof(SRecordInfo)) < 0) {
if (buf) free(buf);
return -1;
}
buf = realloc(buf, info.size);
if (buf == NULL) return -1;
if (read(mfh->fd, buf, info.size) < 0) {
if (buf) free(buf);
return -1;
}
(*mfh->iFunc)(mfh->appH, buf, info.size);
mfh->size = mfh->size + sizeof(SRecordInfo) + info.size;
}
}
(*mfh->aFunc)(mfh->appH);
if (buf) free(buf);
return 0;
}
\ No newline at end of file
......@@ -14,137 +14,13 @@
*/
#include "os.h"
#include "tsdbMain.h"
#include "tchecksum.h"
#include "tscompression.h"
#include "talgo.h"
#include "tchecksum.h"
#include "tcoding.h"
#include "tscompression.h"
#include "tsdbMain.h"
// Local function definitions
// static int tsdbCheckHelperCfg(SHelperCfg *pCfg);
static int tsdbInitHelperFile(SRWHelper *pHelper);
// static void tsdbClearHelperFile(SHelperFile *pHFile);
static bool tsdbShouldCreateNewLast(SRWHelper *pHelper);
static int tsdbWriteBlockToFile(SRWHelper *pHelper, SFile *pFile, SDataCols *pDataCols, int rowsToWrite,
SCompBlock *pCompBlock, bool isLast, bool isSuperBlock);
static int compareKeyBlock(const void *arg1, const void *arg2);
static int tsdbMergeDataWithBlock(SRWHelper *pHelper, int blkIdx, SDataCols *pDataCols);
static int tsdbInsertSuperBlock(SRWHelper *pHelper, SCompBlock *pCompBlock, int blkIdx);
static int tsdbAddSubBlock(SRWHelper *pHelper, SCompBlock *pCompBlock, int blkIdx, int rowsAdded);
static int tsdbUpdateSuperBlock(SRWHelper *pHelper, SCompBlock *pCompBlock, int blkIdx);
static int tsdbGetRowsInRange(SDataCols *pDataCols, TSKEY minKey, TSKEY maxKey);
static void tsdbResetHelperBlock(SRWHelper *pHelper);
// ---------- Operations on Helper File part
static void tsdbResetHelperFileImpl(SRWHelper *pHelper) {
memset((void *)&pHelper->files, 0, sizeof(pHelper->files));
pHelper->files.fid = -1;
pHelper->files.headF.fd = -1;
pHelper->files.dataF.fd = -1;
pHelper->files.lastF.fd = -1;
pHelper->files.nHeadF.fd = -1;
pHelper->files.nLastF.fd = -1;
}
static int tsdbInitHelperFile(SRWHelper *pHelper) {
// pHelper->compIdxSize = sizeof(SCompIdx) * pHelper->config.maxTables + sizeof(TSCKSUM);
size_t tsize = sizeof(SCompIdx) * pHelper->config.maxTables + sizeof(TSCKSUM);
pHelper->pCompIdx = (SCompIdx *)tmalloc(tsize);
if (pHelper->pCompIdx == NULL) return -1;
tsdbResetHelperFileImpl(pHelper);
return 0;
}
static void tsdbDestroyHelperFile(SRWHelper *pHelper) {
tsdbCloseHelperFile(pHelper, false);
tzfree(pHelper->pCompIdx);
}
// ---------- Operations on Helper Table part
static void tsdbResetHelperTableImpl(SRWHelper *pHelper) {
memset((void *)&pHelper->tableInfo, 0, sizeof(SHelperTable));
pHelper->hasOldLastBlock = false;
}
static void tsdbResetHelperTable(SRWHelper *pHelper) {
tsdbResetHelperBlock(pHelper);
tsdbResetHelperTableImpl(pHelper);
helperClearState(pHelper, (TSDB_HELPER_TABLE_SET|TSDB_HELPER_INFO_LOAD));
}
static void tsdbInitHelperTable(SRWHelper *pHelper) {
tsdbResetHelperTableImpl(pHelper);
}
static void tsdbDestroyHelperTable(SRWHelper *pHelper) { tzfree((void *)pHelper->pCompInfo); }
// ---------- Operations on Helper Block part
static void tsdbResetHelperBlockImpl(SRWHelper *pHelper) {
tdResetDataCols(pHelper->pDataCols[0]);
tdResetDataCols(pHelper->pDataCols[1]);
}
static void tsdbResetHelperBlock(SRWHelper *pHelper) {
tsdbResetHelperBlockImpl(pHelper);
// helperClearState(pHelper, TSDB_HELPER_)
}
static int tsdbInitHelperBlock(SRWHelper *pHelper) {
pHelper->pDataCols[0] = tdNewDataCols(pHelper->config.maxRowSize, pHelper->config.maxCols, pHelper->config.maxRows);
pHelper->pDataCols[1] = tdNewDataCols(pHelper->config.maxRowSize, pHelper->config.maxCols, pHelper->config.maxRows);
if (pHelper->pDataCols[0] == NULL || pHelper->pDataCols[1] == NULL) return -1;
tsdbResetHelperBlockImpl(pHelper);
return 0;
}
static void tsdbDestroyHelperBlock(SRWHelper *pHelper) {
tzfree(pHelper->pCompData);
tdFreeDataCols(pHelper->pDataCols[0]);
tdFreeDataCols(pHelper->pDataCols[1]);
}
static int tsdbInitHelper(SRWHelper *pHelper, STsdbRepo *pRepo, tsdb_rw_helper_t type) {
if (pHelper == NULL || pRepo == NULL) return -1;
memset((void *)pHelper, 0, sizeof(*pHelper));
// Init global configuration
pHelper->config.type = type;
pHelper->config.maxTables = pRepo->config.maxTables;
pHelper->config.maxRowSize = pRepo->tsdbMeta->maxRowBytes;
pHelper->config.maxRows = pRepo->config.maxRowsPerFileBlock;
pHelper->config.maxCols = pRepo->tsdbMeta->maxCols;
pHelper->config.minRowsPerFileBlock = pRepo->config.minRowsPerFileBlock;
pHelper->config.maxRowsPerFileBlock = pRepo->config.maxRowsPerFileBlock;
pHelper->config.compress = pRepo->config.compression;
pHelper->state = TSDB_HELPER_CLEAR_STATE;
// Init file part
if (tsdbInitHelperFile(pHelper) < 0) goto _err;
// Init table part
tsdbInitHelperTable(pHelper);
// Init block part
if (tsdbInitHelperBlock(pHelper) < 0) goto _err;
pHelper->pBuffer =
tmalloc(sizeof(SCompData) + (sizeof(SCompCol) + sizeof(TSCKSUM) + COMP_OVERFLOW_BYTES) * pHelper->config.maxCols +
pHelper->config.maxRowSize * pHelper->config.maxRowsPerFileBlock + sizeof(TSCKSUM));
if (pHelper->pBuffer == NULL) goto _err;
return 0;
_err:
tsdbDestroyHelper(pHelper);
return -1;
}
// ------------------------------------------ OPERATIONS FOR OUTSIDE ------------------------------------------
// ---------------------- INTERNAL FUNCTIONS ----------------------
int tsdbInitReadHelper(SRWHelper *pHelper, STsdbRepo *pRepo) {
return tsdbInitHelper(pHelper, pRepo, TSDB_READ_HELPER);
}
......@@ -180,7 +56,6 @@ void tsdbResetHelper(SRWHelper *pHelper) {
}
}
// ------------ Operations for read/write purpose
int tsdbSetAndOpenHelperFile(SRWHelper *pHelper, SFileGroup *pGroup) {
ASSERT(pHelper != NULL && pGroup != NULL);
......@@ -231,7 +106,7 @@ int tsdbSetAndOpenHelperFile(SRWHelper *pHelper, SFileGroup *pGroup) {
return tsdbLoadCompIdx(pHelper, NULL);
_err:
_err:
return -1;
}
......@@ -264,7 +139,7 @@ int tsdbCloseHelperFile(SRWHelper *pHelper, bool hasError) {
pHelper->files.headF.info = pHelper->files.nHeadF.info;
}
}
if (pHelper->files.nLastF.fd > 0) {
if (!hasError) tsdbUpdateFileHeader(&(pHelper->files.nLastF), 0);
fsync(pHelper->files.nLastF.fd);
......@@ -306,7 +181,7 @@ void tsdbSetHelperTable(SRWHelper *pHelper, STable *pTable, STsdbRepo *pRepo) {
/**
* Write part of of points from pDataCols to file
*
*
* @return: number of points written to file successfully
* -1 for failure
*/
......@@ -342,8 +217,8 @@ int tsdbWriteDataBlock(SRWHelper *pHelper, SDataCols *pDataCols) {
if (tsdbInsertSuperBlock(pHelper, &compBlock, pIdx->numOfBlocks) < 0) goto _err;
} else { // (Has old data) AND ((has last block) OR (key overlap)), need to merge the block
SCompBlock *pCompBlock = taosbsearch((void *)(&keyFirst), (void *)(pHelper->pCompInfo->blocks),
pIdx->numOfBlocks, sizeof(SCompBlock), compareKeyBlock, TD_GE);
SCompBlock *pCompBlock = taosbsearch((void *)(&keyFirst), (void *)(pHelper->pCompInfo->blocks), pIdx->numOfBlocks,
sizeof(SCompBlock), compareKeyBlock, TD_GE);
int blkIdx = (pCompBlock == NULL) ? (pIdx->numOfBlocks - 1) : (pCompBlock - pHelper->pCompInfo->blocks);
......@@ -358,10 +233,11 @@ int tsdbWriteDataBlock(SRWHelper *pHelper, SDataCols *pDataCols) {
rowsToWrite = tsdbMergeDataWithBlock(pHelper, blkIdx, pDataCols);
if (rowsToWrite < 0) goto _err;
} else { // Save as a super block in the middle
rowsToWrite = tsdbGetRowsInRange(pDataCols, 0, pCompBlock->keyFirst-1);
} else { // Save as a super block in the middle
rowsToWrite = tsdbGetRowsInRange(pDataCols, 0, pCompBlock->keyFirst - 1);
ASSERT(rowsToWrite > 0);
if (tsdbWriteBlockToFile(pHelper, &(pHelper->files.dataF), pDataCols, rowsToWrite, &compBlock, false, true) < 0) goto _err;
if (tsdbWriteBlockToFile(pHelper, &(pHelper->files.dataF), pDataCols, rowsToWrite, &compBlock, false, true) < 0)
goto _err;
if (tsdbInsertSuperBlock(pHelper, &compBlock, blkIdx) < 0) goto _err;
}
}
......@@ -375,7 +251,7 @@ _err:
int tsdbMoveLastBlockIfNeccessary(SRWHelper *pHelper) {
ASSERT(TSDB_HELPER_TYPE(pHelper) == TSDB_WRITE_HELPER);
SCompIdx *pIdx = pHelper->pCompIdx + pHelper->tableInfo.tid;
SCompIdx * pIdx = pHelper->pCompIdx + pHelper->tableInfo.tid;
SCompBlock compBlock;
if ((pHelper->files.nLastF.fd > 0) && (pHelper->hasOldLastBlock)) {
if (tsdbLoadCompInfo(pHelper, NULL) < 0) return -1;
......@@ -450,7 +326,7 @@ int tsdbWriteCompIdx(SRWHelper *pHelper) {
if (pCompIdx->offset > 0) {
int drift = POINTER_DISTANCE(buf, pHelper->pBuffer);
if (tsizeof(pHelper->pBuffer) - drift < 128) {
pHelper->pBuffer = trealloc(pHelper->pBuffer, tsizeof(pHelper->pBuffer)*2);
pHelper->pBuffer = trealloc(pHelper->pBuffer, tsizeof(pHelper->pBuffer) * 2);
}
buf = POINTER_SHIFT(pHelper->pBuffer, drift);
buf = taosEncodeVariantU32(buf, i);
......@@ -472,7 +348,7 @@ int tsdbLoadCompIdx(SRWHelper *pHelper, void *target) {
if (!helperHasState(pHelper, TSDB_HELPER_IDX_LOAD)) {
// If not load from file, just load it in object
SFile *pFile = &(pHelper->files.headF);
int fd = pFile->fd;
int fd = pFile->fd;
memset(pHelper->pCompIdx, 0, tsizeof(pHelper->pCompIdx));
if (pFile->info.offset > 0) {
......@@ -480,8 +356,7 @@ int tsdbLoadCompIdx(SRWHelper *pHelper, void *target) {
if (lseek(fd, pFile->info.offset, SEEK_SET) < 0) return -1;
if ((pHelper->pBuffer = trealloc(pHelper->pBuffer, pFile->info.len)) == NULL) return -1;
if (tread(fd, (void *)(pHelper->pBuffer), pFile->info.len) < pFile->info.len)
return -1;
if (tread(fd, (void *)(pHelper->pBuffer), pFile->info.len) < pFile->info.len) return -1;
if (!taosCheckChecksumWhole((uint8_t *)(pHelper->pBuffer), pFile->info.len)) {
// TODO: File is broken, try to deal with it
return -1;
......@@ -502,7 +377,6 @@ int tsdbLoadCompIdx(SRWHelper *pHelper, void *target) {
ASSERT(((char *)ptr - (char *)pHelper->pBuffer) == (pFile->info.len - sizeof(TSCKSUM)));
if (lseek(fd, TSDB_FILE_HEAD_SIZE, SEEK_SET) < 0) return -1;
}
}
helperSetState(pHelper, TSDB_HELPER_IDX_LOAD);
......@@ -582,53 +456,12 @@ void tsdbGetDataStatis(SRWHelper *pHelper, SDataStatis *pStatis, int numOfCols)
}
}
static int comparColIdCompCol(const void *arg1, const void *arg2) {
return (*(int16_t *)arg1) - ((SCompCol *)arg2)->colId;
}
static int comparColIdDataCol(const void *arg1, const void *arg2) {
return (*(int16_t *)arg1) - ((SDataCol *)arg2)->colId;
}
static int tsdbLoadSingleColumnData(int fd, SCompBlock *pCompBlock, SCompCol *pCompCol, void *buf) {
size_t tsize = sizeof(SCompData) + sizeof(SCompCol) * pCompBlock->numOfCols;
if (lseek(fd, pCompBlock->offset + tsize + pCompCol->offset, SEEK_SET) < 0) return -1;
if (tread(fd, buf, pCompCol->len) < pCompCol->len) return -1;
return 0;
}
static int tsdbLoadSingleBlockDataCols(SRWHelper *pHelper, SCompBlock *pCompBlock, int16_t *colIds, int numOfColIds,
SDataCols *pDataCols) {
if (tsdbLoadCompData(pHelper, pCompBlock, NULL) < 0) return -1;
int fd = (pCompBlock->last) ? pHelper->files.lastF.fd : pHelper->files.dataF.fd;
void *ptr = NULL;
for (int i = 0; i < numOfColIds; i++) {
int16_t colId = colIds[i];
ptr = bsearch((void *)&colId, (void *)pHelper->pCompData->cols, pHelper->pCompData->numOfCols, sizeof(SCompCol), comparColIdCompCol);
if (ptr == NULL) continue;
SCompCol *pCompCol = (SCompCol *)ptr;
ptr = bsearch((void *)&colId, (void *)(pDataCols->cols), pDataCols->numOfCols, sizeof(SDataCol), comparColIdDataCol);
ASSERT(ptr != NULL);
SDataCol *pDataCol = (SDataCol *)ptr;
pDataCol->len = pCompCol->len;
if (tsdbLoadSingleColumnData(fd, pCompBlock, pCompCol, pDataCol->pData) < 0) return -1;
}
return 0;
}
// Load specific column data from file
int tsdbLoadBlockDataCols(SRWHelper *pHelper, SDataCols *pDataCols, int blkIdx, int16_t *colIds, int numOfColIds) {
SCompBlock *pCompBlock = pHelper->pCompInfo->blocks + blkIdx;
ASSERT(pCompBlock->numOfSubBlocks >= 1); // Must be super block
ASSERT(pCompBlock->numOfSubBlocks >= 1); // Must be super block
int numOfSubBlocks = pCompBlock->numOfSubBlocks;
int numOfSubBlocks = pCompBlock->numOfSubBlocks;
SCompBlock *pStartBlock =
(numOfSubBlocks == 1) ? pCompBlock : (SCompBlock *)((char *)pHelper->pCompInfo->blocks + pCompBlock->offset);
......@@ -642,95 +475,6 @@ int tsdbLoadBlockDataCols(SRWHelper *pHelper, SDataCols *pDataCols, int blkIdx,
return 0;
}
static int tsdbCheckAndDecodeColumnData(SDataCol *pDataCol, char *content, int32_t len, int8_t comp, int numOfRows,
int maxPoints, char *buffer, int bufferSize) {
// Verify by checksum
if (!taosCheckChecksumWhole((uint8_t *)content, len)) return -1;
// Decode the data
if (comp) {
// // Need to decompress
pDataCol->len = (*(tDataTypeDesc[pDataCol->type].decompFunc))(
content, len - sizeof(TSCKSUM), numOfRows, pDataCol->pData, pDataCol->spaceSize, comp, buffer, bufferSize);
if (pDataCol->type == TSDB_DATA_TYPE_BINARY || pDataCol->type == TSDB_DATA_TYPE_NCHAR) {
dataColSetOffset(pDataCol, numOfRows);
}
} else {
// No need to decompress, just memcpy it
pDataCol->len = len - sizeof(TSCKSUM);
memcpy(pDataCol->pData, content, pDataCol->len);
if (pDataCol->type == TSDB_DATA_TYPE_BINARY || pDataCol->type == TSDB_DATA_TYPE_NCHAR) {
dataColSetOffset(pDataCol, numOfRows);
}
}
return 0;
}
/**
* Interface to read the data of a sub-block OR the data of a super-block of which (numOfSubBlocks == 1)
*/
static int tsdbLoadBlockDataImpl(SRWHelper *pHelper, SCompBlock *pCompBlock, SDataCols *pDataCols) {
ASSERT(pCompBlock->numOfSubBlocks <= 1);
ASSERT(tsizeof(pHelper->pBuffer) >= pCompBlock->len);
SCompData *pCompData = (SCompData *)pHelper->pBuffer;
int fd = (pCompBlock->last) ? pHelper->files.lastF.fd : pHelper->files.dataF.fd;
if (lseek(fd, pCompBlock->offset, SEEK_SET) < 0) goto _err;
if (tread(fd, (void *)pCompData, pCompBlock->len) < pCompBlock->len) goto _err;
ASSERT(pCompData->numOfCols == pCompBlock->numOfCols);
int32_t tsize = sizeof(SCompData) + sizeof(SCompCol) * pCompBlock->numOfCols + sizeof(TSCKSUM);
if (!taosCheckChecksumWhole((uint8_t *)pCompData, tsize)) goto _err;
pDataCols->numOfRows = pCompBlock->numOfRows;
// Recover the data
int ccol = 0;
int dcol = 0;
while (dcol < pDataCols->numOfCols) {
SDataCol *pDataCol = &(pDataCols->cols[dcol]);
if (ccol >= pCompData->numOfCols) {
// Set current column as NULL and forward
dataColSetNEleNull(pDataCol, pCompBlock->numOfRows, pDataCols->maxPoints);
dcol++;
continue;
}
SCompCol *pCompCol = &(pCompData->cols[ccol]);
if (pCompCol->colId == pDataCol->colId) {
if (pCompBlock->algorithm == TWO_STAGE_COMP) {
int zsize = pDataCol->bytes * pCompBlock->numOfRows + COMP_OVERFLOW_BYTES;
if (pCompCol->type == TSDB_DATA_TYPE_BINARY || pCompCol->type == TSDB_DATA_TYPE_NCHAR) {
zsize += (sizeof(VarDataLenT) * pCompBlock->numOfRows);
}
pHelper->compBuffer = trealloc(pHelper->compBuffer, zsize);
if (pHelper->compBuffer == NULL) goto _err;
}
if (tsdbCheckAndDecodeColumnData(pDataCol, (char *)pCompData + tsize + pCompCol->offset, pCompCol->len,
pCompBlock->algorithm, pCompBlock->numOfRows, pDataCols->maxPoints,
pHelper->compBuffer, tsizeof(pHelper->compBuffer)) < 0)
goto _err;
dcol++;
ccol++;
} else if (pCompCol->colId < pDataCol->colId) {
ccol++;
} else {
// Set current column as NULL and forward
dataColSetNEleNull(pDataCol, pCompBlock->numOfRows, pDataCols->maxPoints);
dcol++;
}
}
return 0;
_err:
return -1;
}
// Load the whole block data
int tsdbLoadBlockData(SRWHelper *pHelper, SCompBlock *pCompBlock, SDataCols *target) {
// SCompBlock *pCompBlock = pHelper->pCompInfo->blocks + blkIdx;
......@@ -754,21 +498,89 @@ _err:
return -1;
}
static bool tsdbShouldCreateNewLast(SRWHelper *pHelper) {
ASSERT(pHelper->files.lastF.fd > 0);
struct stat st;
fstat(pHelper->files.lastF.fd, &st);
if (st.st_size > 32 * 1024 + TSDB_FILE_HEAD_SIZE) return true;
return false;
}
static int tsdbWriteBlockToFile(SRWHelper *pHelper, SFile *pFile, SDataCols *pDataCols, int rowsToWrite, SCompBlock *pCompBlock,
bool isLast, bool isSuperBlock) {
ASSERT(rowsToWrite > 0 && rowsToWrite <= pDataCols->numOfRows && rowsToWrite <= pHelper->config.maxRowsPerFileBlock);
ASSERT(isLast ? rowsToWrite < pHelper->config.minRowsPerFileBlock : true);
SCompData *pCompData = (SCompData *)(pHelper->pBuffer);
int64_t offset = 0;
void *tsdbEncodeSCompIdx(void *buf, SCompIdx *pIdx) {
buf = taosEncodeVariantU32(buf, pIdx->len);
buf = taosEncodeVariantU32(buf, pIdx->offset);
buf = taosEncodeFixedU8(buf, pIdx->hasLast);
buf = taosEncodeVariantU32(buf, pIdx->numOfBlocks);
buf = taosEncodeFixedU64(buf, pIdx->uid);
buf = taosEncodeFixedU64(buf, pIdx->maxKey);
return buf;
}
void *tsdbDecodeSCompIdx(void *buf, SCompIdx *pIdx) {
uint8_t hasLast = 0;
uint32_t numOfBlocks = 0;
uint64_t value = 0;
if ((buf = taosDecodeVariantU32(buf, &(pIdx->len))) == NULL) return NULL;
if ((buf = taosDecodeVariantU32(buf, &(pIdx->offset))) == NULL) return NULL;
if ((buf = taosDecodeFixedU8(buf, &(hasLast))) == NULL) return NULL;
pIdx->hasLast = hasLast;
if ((buf = taosDecodeVariantU32(buf, &(numOfBlocks))) == NULL) return NULL;
pIdx->numOfBlocks = numOfBlocks;
if ((buf = taosDecodeFixedU64(buf, &value)) == NULL) return NULL;
pIdx->uid = (int64_t)value;
if ((buf = taosDecodeFixedU64(buf, &value)) == NULL) return NULL;
pIdx->maxKey = (TSKEY)value;
return buf;
}
int tsdbUpdateFileHeader(SFile *pFile, uint32_t version) {
char buf[TSDB_FILE_HEAD_SIZE] = "\0";
void *pBuf = (void *)buf;
pBuf = taosEncodeFixedU32(pBuf, version);
pBuf = tsdbEncodeSFileInfo(pBuf, &(pFile->info));
taosCalcChecksumAppend(0, (uint8_t *)buf, TSDB_FILE_HEAD_SIZE);
if (lseek(pFile->fd, 0, SEEK_SET) < 0) return -1;
if (twrite(pFile->fd, (void *)buf, TSDB_FILE_HEAD_SIZE) < TSDB_FILE_HEAD_SIZE) return -1;
return 0;
}
void *tsdbEncodeSFileInfo(void *buf, const STsdbFileInfo *pInfo) {
buf = taosEncodeFixedU32(buf, pInfo->offset);
buf = taosEncodeFixedU32(buf, pInfo->len);
buf = taosEncodeFixedU64(buf, pInfo->size);
buf = taosEncodeFixedU64(buf, pInfo->tombSize);
buf = taosEncodeFixedU32(buf, pInfo->totalBlocks);
buf = taosEncodeFixedU32(buf, pInfo->totalSubBlocks);
return buf;
}
void *tsdbDecodeSFileInfo(void *buf, STsdbFileInfo *pInfo) {
buf = taosDecodeFixedU32(buf, &(pInfo->offset));
buf = taosDecodeFixedU32(buf, &(pInfo->len));
buf = taosDecodeFixedU64(buf, &(pInfo->size));
buf = taosDecodeFixedU64(buf, &(pInfo->tombSize));
buf = taosDecodeFixedU32(buf, &(pInfo->totalBlocks));
buf = taosDecodeFixedU32(buf, &(pInfo->totalSubBlocks));
return buf;
}
// ---------------------- INTERNAL FUNCTIONS ----------------------
static bool tsdbShouldCreateNewLast(SRWHelper *pHelper) {
ASSERT(pHelper->files.lastF.fd > 0);
struct stat st;
fstat(pHelper->files.lastF.fd, &st);
if (st.st_size > 32 * 1024 + TSDB_FILE_HEAD_SIZE) return true;
return false;
}
static int tsdbWriteBlockToFile(SRWHelper *pHelper, SFile *pFile, SDataCols *pDataCols, int rowsToWrite,
SCompBlock *pCompBlock, bool isLast, bool isSuperBlock) {
ASSERT(rowsToWrite > 0 && rowsToWrite <= pDataCols->numOfRows && rowsToWrite <= pHelper->config.maxRowsPerFileBlock);
ASSERT(isLast ? rowsToWrite < pHelper->config.minRowsPerFileBlock : true);
SCompData *pCompData = (SCompData *)(pHelper->pBuffer);
int64_t offset = 0;
offset = lseek(pFile->fd, 0, SEEK_END);
if (offset < 0) goto _err;
......@@ -822,8 +634,8 @@ static int tsdbWriteBlockToFile(SRWHelper *pHelper, SFile *pFile, SDataCols *pDa
}
pCompCol->len = (*(tDataTypeDesc[pDataCol->type].compFunc))(
(char *)pDataCol->pData, tlen, rowsToWrite, tptr, tsizeof(pHelper->pBuffer) - lsize,
pHelper->config.compress, pHelper->compBuffer, tsizeof(pHelper->compBuffer));
(char *)pDataCol->pData, tlen, rowsToWrite, tptr, tsizeof(pHelper->pBuffer) - lsize, pHelper->config.compress,
pHelper->compBuffer, tsizeof(pHelper->compBuffer));
} else {
pCompCol->len = tlen;
memcpy(tptr, pDataCol->pData, pCompCol->len);
......@@ -861,7 +673,7 @@ static int tsdbWriteBlockToFile(SRWHelper *pHelper, SFile *pFile, SDataCols *pDa
return 0;
_err:
_err:
return -1;
}
......@@ -878,11 +690,6 @@ static int compareKeyBlock(const void *arg1, const void *arg2) {
return 0;
}
// static FORCE_INLINE int compKeyFunc(const void *arg1, const void *arg2) {
// return ((*(TSKEY *)arg1) - (*(TSKEY *)arg2));
// }
// Merge the data with a block in file
static int tsdbMergeDataWithBlock(SRWHelper *pHelper, int blkIdx, SDataCols *pDataCols) {
// TODO: set pHelper->hasOldBlock
int rowsWritten = 0;
......@@ -899,8 +706,9 @@ static int tsdbMergeDataWithBlock(SRWHelper *pHelper, int blkIdx, SDataCols *pDa
ASSERT(keyFirst >= blockAtIdx(pHelper, blkIdx)->keyFirst);
// ASSERT(compareKeyBlock((void *)&keyFirst, (void *)pCompBlock) == 0);
if (keyFirst > blockAtIdx(pHelper, blkIdx)->keyLast) { // Merge with the last block by append
ASSERT(blockAtIdx(pHelper, blkIdx)->numOfRows < pHelper->config.minRowsPerFileBlock && blkIdx == pIdx->numOfBlocks-1);
if (keyFirst > blockAtIdx(pHelper, blkIdx)->keyLast) { // Merge with the last block by append
ASSERT(blockAtIdx(pHelper, blkIdx)->numOfRows < pHelper->config.minRowsPerFileBlock &&
blkIdx == pIdx->numOfBlocks - 1);
int defaultRowsToWrite = pHelper->config.maxRowsPerFileBlock * 4 / 5; // TODO: make a interface
rowsWritten = MIN((defaultRowsToWrite - blockAtIdx(pHelper, blkIdx)->numOfRows), pDataCols->numOfRows);
......@@ -918,15 +726,15 @@ static int tsdbMergeDataWithBlock(SRWHelper *pHelper, int blkIdx, SDataCols *pDa
if (tdMergeDataCols(pHelper->pDataCols[0], pDataCols, rowsWritten) < 0) goto _err;
// Write
SFile *pWFile = NULL;
bool isLast = false;
bool isLast = false;
if (pHelper->pDataCols[0]->numOfRows >= pHelper->config.minRowsPerFileBlock) {
pWFile = &(pHelper->files.dataF);
} else {
isLast = true;
pWFile = (pHelper->files.nLastF.fd > 0) ? &(pHelper->files.nLastF) : &(pHelper->files.lastF);
}
if (tsdbWriteBlockToFile(pHelper, pWFile, pHelper->pDataCols[0],
pHelper->pDataCols[0]->numOfRows, &compBlock, isLast, true) < 0)
if (tsdbWriteBlockToFile(pHelper, pWFile, pHelper->pDataCols[0], pHelper->pDataCols[0]->numOfRows, &compBlock,
isLast, true) < 0)
goto _err;
if (tsdbUpdateSuperBlock(pHelper, &compBlock, blkIdx) < 0) goto _err;
}
......@@ -940,7 +748,8 @@ static int tsdbMergeDataWithBlock(SRWHelper *pHelper, int blkIdx, SDataCols *pDa
TSKEY keyLimit = (blkIdx == pIdx->numOfBlocks - 1) ? INT64_MAX : blockAtIdx(pHelper, blkIdx + 1)->keyFirst - 1;
// rows1: number of rows must merge in this block
int rows1 = tsdbGetRowsInRange(pDataCols, blockAtIdx(pHelper, blkIdx)->keyFirst, blockAtIdx(pHelper, blkIdx)->keyLast);
int rows1 =
tsdbGetRowsInRange(pDataCols, blockAtIdx(pHelper, blkIdx)->keyFirst, blockAtIdx(pHelper, blkIdx)->keyLast);
// rows2: max number of rows the block can have more
int rows2 = pHelper->config.maxRowsPerFileBlock - blockAtIdx(pHelper, blkIdx)->numOfRows;
// rows3: number of rows between this block and the next block
......@@ -972,8 +781,8 @@ static int tsdbMergeDataWithBlock(SRWHelper *pHelper, int blkIdx, SDataCols *pDa
rowsWritten = rows3;
int iter1 = 0; // iter over pHelper->pDataCols[0]
int iter2 = 0; // iter over pDataCols
int iter1 = 0; // iter over pHelper->pDataCols[0]
int iter2 = 0; // iter over pDataCols
int round = 0;
// tdResetDataCols(pHelper->pDataCols[1]);
while (true) {
......@@ -997,7 +806,7 @@ static int tsdbMergeDataWithBlock(SRWHelper *pHelper, int blkIdx, SDataCols *pDa
return rowsWritten;
_err:
_err:
return -1;
}
......@@ -1012,7 +821,6 @@ static int compTSKEY(const void *key1, const void *key2) {
}
static int tsdbAdjustInfoSizeIfNeeded(SRWHelper *pHelper, size_t esize) {
if (tsizeof((void *)pHelper->pCompInfo) <= esize) {
size_t tsize = esize + sizeof(SCompBlock) * 16;
pHelper->pCompInfo = (SCompInfo *)trealloc(pHelper->pCompInfo, tsize);
......@@ -1075,7 +883,7 @@ static int tsdbAddSubBlock(SRWHelper *pHelper, SCompBlock *pCompBlock, int blkId
size_t spaceNeeded =
(pSCompBlock->numOfSubBlocks == 1) ? pIdx->len + sizeof(SCompBlock) * 2 : pIdx->len + sizeof(SCompBlock);
if (tsdbAdjustInfoSizeIfNeeded(pHelper, spaceNeeded) < 0) goto _err;
if (tsdbAdjustInfoSizeIfNeeded(pHelper, spaceNeeded) < 0) goto _err;
pSCompBlock = pHelper->pCompInfo->blocks + blkIdx;
......@@ -1092,7 +900,6 @@ static int tsdbAddSubBlock(SRWHelper *pHelper, SCompBlock *pCompBlock, int blkId
}
}
*(SCompBlock *)((char *)(pHelper->pCompInfo) + pSCompBlock->offset + pSCompBlock->len) = *pCompBlock;
pSCompBlock->numOfSubBlocks++;
......@@ -1112,7 +919,7 @@ static int tsdbAddSubBlock(SRWHelper *pHelper, SCompBlock *pCompBlock, int blkId
}
}
if (ptr == NULL) ptr = POINTER_SHIFT(pHelper->pCompInfo, pIdx->len-sizeof(TSCKSUM));
if (ptr == NULL) ptr = POINTER_SHIFT(pHelper->pCompInfo, pIdx->len - sizeof(TSCKSUM));
size_t tsize = pIdx->len - ((char *)ptr - (char *)(pHelper->pCompInfo));
if (tsize > 0) {
......@@ -1206,71 +1013,235 @@ static int tsdbGetRowsInRange(SDataCols *pDataCols, TSKEY minKey, TSKEY maxKey)
return ((TSKEY *)ptr2 - (TSKEY *)ptr1) + 1;
}
void *tsdbEncodeSCompIdx(void *buf, SCompIdx *pIdx) {
buf = taosEncodeVariantU32(buf, pIdx->len);
buf = taosEncodeVariantU32(buf, pIdx->offset);
buf = taosEncodeFixedU8(buf, pIdx->hasLast);
buf = taosEncodeVariantU32(buf, pIdx->numOfBlocks);
buf = taosEncodeFixedU64(buf, pIdx->uid);
buf = taosEncodeFixedU64(buf, pIdx->maxKey);
static void tsdbResetHelperFileImpl(SRWHelper *pHelper) {
memset((void *)&pHelper->files, 0, sizeof(pHelper->files));
pHelper->files.fid = -1;
pHelper->files.headF.fd = -1;
pHelper->files.dataF.fd = -1;
pHelper->files.lastF.fd = -1;
pHelper->files.nHeadF.fd = -1;
pHelper->files.nLastF.fd = -1;
}
return buf;
static int tsdbInitHelperFile(SRWHelper *pHelper) {
// pHelper->compIdxSize = sizeof(SCompIdx) * pHelper->config.maxTables + sizeof(TSCKSUM);
size_t tsize = sizeof(SCompIdx) * pHelper->config.maxTables + sizeof(TSCKSUM);
pHelper->pCompIdx = (SCompIdx *)tmalloc(tsize);
if (pHelper->pCompIdx == NULL) return -1;
tsdbResetHelperFileImpl(pHelper);
return 0;
}
void *tsdbDecodeSCompIdx(void *buf, SCompIdx *pIdx) {
uint8_t hasLast = 0;
uint32_t numOfBlocks = 0;
uint64_t value = 0;
static void tsdbDestroyHelperFile(SRWHelper *pHelper) {
tsdbCloseHelperFile(pHelper, false);
tzfree(pHelper->pCompIdx);
}
if ((buf = taosDecodeVariantU32(buf, &(pIdx->len))) == NULL) return NULL;
if ((buf = taosDecodeVariantU32(buf, &(pIdx->offset))) == NULL) return NULL;
if ((buf = taosDecodeFixedU8(buf, &(hasLast))) == NULL) return NULL;
pIdx->hasLast = hasLast;
if ((buf = taosDecodeVariantU32(buf, &(numOfBlocks))) == NULL) return NULL;
pIdx->numOfBlocks = numOfBlocks;
if ((buf = taosDecodeFixedU64(buf, &value)) == NULL) return NULL;
pIdx->uid = (int64_t)value;
if ((buf = taosDecodeFixedU64(buf, &value)) == NULL) return NULL;
pIdx->maxKey = (TSKEY)value;
// ---------- Operations on Helper Table part
static void tsdbResetHelperTableImpl(SRWHelper *pHelper) {
memset((void *)&pHelper->tableInfo, 0, sizeof(SHelperTable));
pHelper->hasOldLastBlock = false;
}
return buf;
static void tsdbResetHelperTable(SRWHelper *pHelper) {
tsdbResetHelperBlock(pHelper);
tsdbResetHelperTableImpl(pHelper);
helperClearState(pHelper, (TSDB_HELPER_TABLE_SET | TSDB_HELPER_INFO_LOAD));
}
int tsdbUpdateFileHeader(SFile *pFile, uint32_t version) {
char buf[TSDB_FILE_HEAD_SIZE] = "\0";
static void tsdbInitHelperTable(SRWHelper *pHelper) { tsdbResetHelperTableImpl(pHelper); }
void *pBuf = (void *)buf;
pBuf = taosEncodeFixedU32(pBuf, version);
pBuf = tsdbEncodeSFileInfo(pBuf, &(pFile->info));
static void tsdbDestroyHelperTable(SRWHelper *pHelper) { tzfree((void *)pHelper->pCompInfo); }
taosCalcChecksumAppend(0, (uint8_t *)buf, TSDB_FILE_HEAD_SIZE);
// ---------- Operations on Helper Block part
static void tsdbResetHelperBlockImpl(SRWHelper *pHelper) {
tdResetDataCols(pHelper->pDataCols[0]);
tdResetDataCols(pHelper->pDataCols[1]);
}
if (lseek(pFile->fd, 0, SEEK_SET) < 0) return -1;
if (twrite(pFile->fd, (void *)buf, TSDB_FILE_HEAD_SIZE) < TSDB_FILE_HEAD_SIZE) return -1;
static void tsdbResetHelperBlock(SRWHelper *pHelper) {
tsdbResetHelperBlockImpl(pHelper);
// helperClearState(pHelper, TSDB_HELPER_)
}
static int tsdbInitHelperBlock(SRWHelper *pHelper) {
pHelper->pDataCols[0] = tdNewDataCols(pHelper->config.maxRowSize, pHelper->config.maxCols, pHelper->config.maxRows);
pHelper->pDataCols[1] = tdNewDataCols(pHelper->config.maxRowSize, pHelper->config.maxCols, pHelper->config.maxRows);
if (pHelper->pDataCols[0] == NULL || pHelper->pDataCols[1] == NULL) return -1;
tsdbResetHelperBlockImpl(pHelper);
return 0;
}
static void tsdbDestroyHelperBlock(SRWHelper *pHelper) {
tzfree(pHelper->pCompData);
tdFreeDataCols(pHelper->pDataCols[0]);
tdFreeDataCols(pHelper->pDataCols[1]);
}
static int tsdbInitHelper(SRWHelper *pHelper, STsdbRepo *pRepo, tsdb_rw_helper_t type) {
if (pHelper == NULL || pRepo == NULL) return -1;
memset((void *)pHelper, 0, sizeof(*pHelper));
// Init global configuration
pHelper->config.type = type;
pHelper->config.maxTables = pRepo->config.maxTables;
pHelper->config.maxRowSize = pRepo->tsdbMeta->maxRowBytes;
pHelper->config.maxRows = pRepo->config.maxRowsPerFileBlock;
pHelper->config.maxCols = pRepo->tsdbMeta->maxCols;
pHelper->config.minRowsPerFileBlock = pRepo->config.minRowsPerFileBlock;
pHelper->config.maxRowsPerFileBlock = pRepo->config.maxRowsPerFileBlock;
pHelper->config.compress = pRepo->config.compression;
pHelper->state = TSDB_HELPER_CLEAR_STATE;
// Init file part
if (tsdbInitHelperFile(pHelper) < 0) goto _err;
// Init table part
tsdbInitHelperTable(pHelper);
// Init block part
if (tsdbInitHelperBlock(pHelper) < 0) goto _err;
pHelper->pBuffer =
tmalloc(sizeof(SCompData) + (sizeof(SCompCol) + sizeof(TSCKSUM) + COMP_OVERFLOW_BYTES) * pHelper->config.maxCols +
pHelper->config.maxRowSize * pHelper->config.maxRowsPerFileBlock + sizeof(TSCKSUM));
if (pHelper->pBuffer == NULL) goto _err;
return 0;
_err:
tsdbDestroyHelper(pHelper);
return -1;
}
static int comparColIdCompCol(const void *arg1, const void *arg2) {
return (*(int16_t *)arg1) - ((SCompCol *)arg2)->colId;
}
static int comparColIdDataCol(const void *arg1, const void *arg2) {
return (*(int16_t *)arg1) - ((SDataCol *)arg2)->colId;
}
void *tsdbEncodeSFileInfo(void *buf, const STsdbFileInfo *pInfo) {
buf = taosEncodeFixedU32(buf, pInfo->offset);
buf = taosEncodeFixedU32(buf, pInfo->len);
buf = taosEncodeFixedU64(buf, pInfo->size);
buf = taosEncodeFixedU64(buf, pInfo->tombSize);
buf = taosEncodeFixedU32(buf, pInfo->totalBlocks);
buf = taosEncodeFixedU32(buf, pInfo->totalSubBlocks);
static int tsdbLoadSingleColumnData(int fd, SCompBlock *pCompBlock, SCompCol *pCompCol, void *buf) {
size_t tsize = sizeof(SCompData) + sizeof(SCompCol) * pCompBlock->numOfCols;
if (lseek(fd, pCompBlock->offset + tsize + pCompCol->offset, SEEK_SET) < 0) return -1;
if (tread(fd, buf, pCompCol->len) < pCompCol->len) return -1;
return buf;
return 0;
}
void *tsdbDecodeSFileInfo(void *buf, STsdbFileInfo *pInfo) {
buf = taosDecodeFixedU32(buf, &(pInfo->offset));
buf = taosDecodeFixedU32(buf, &(pInfo->len));
buf = taosDecodeFixedU64(buf, &(pInfo->size));
buf = taosDecodeFixedU64(buf, &(pInfo->tombSize));
buf = taosDecodeFixedU32(buf, &(pInfo->totalBlocks));
buf = taosDecodeFixedU32(buf, &(pInfo->totalSubBlocks));
static int tsdbLoadSingleBlockDataCols(SRWHelper *pHelper, SCompBlock *pCompBlock, int16_t *colIds, int numOfColIds,
SDataCols *pDataCols) {
if (tsdbLoadCompData(pHelper, pCompBlock, NULL) < 0) return -1;
int fd = (pCompBlock->last) ? pHelper->files.lastF.fd : pHelper->files.dataF.fd;
return buf;
}
\ No newline at end of file
void *ptr = NULL;
for (int i = 0; i < numOfColIds; i++) {
int16_t colId = colIds[i];
ptr = bsearch((void *)&colId, (void *)pHelper->pCompData->cols, pHelper->pCompData->numOfCols, sizeof(SCompCol),
comparColIdCompCol);
if (ptr == NULL) continue;
SCompCol *pCompCol = (SCompCol *)ptr;
ptr =
bsearch((void *)&colId, (void *)(pDataCols->cols), pDataCols->numOfCols, sizeof(SDataCol), comparColIdDataCol);
ASSERT(ptr != NULL);
SDataCol *pDataCol = (SDataCol *)ptr;
pDataCol->len = pCompCol->len;
if (tsdbLoadSingleColumnData(fd, pCompBlock, pCompCol, pDataCol->pData) < 0) return -1;
}
return 0;
}
static int tsdbCheckAndDecodeColumnData(SDataCol *pDataCol, char *content, int32_t len, int8_t comp, int numOfRows,
int maxPoints, char *buffer, int bufferSize) {
// Verify by checksum
if (!taosCheckChecksumWhole((uint8_t *)content, len)) return -1;
// Decode the data
if (comp) {
// // Need to decompress
pDataCol->len = (*(tDataTypeDesc[pDataCol->type].decompFunc))(
content, len - sizeof(TSCKSUM), numOfRows, pDataCol->pData, pDataCol->spaceSize, comp, buffer, bufferSize);
if (pDataCol->type == TSDB_DATA_TYPE_BINARY || pDataCol->type == TSDB_DATA_TYPE_NCHAR) {
dataColSetOffset(pDataCol, numOfRows);
}
} else {
// No need to decompress, just memcpy it
pDataCol->len = len - sizeof(TSCKSUM);
memcpy(pDataCol->pData, content, pDataCol->len);
if (pDataCol->type == TSDB_DATA_TYPE_BINARY || pDataCol->type == TSDB_DATA_TYPE_NCHAR) {
dataColSetOffset(pDataCol, numOfRows);
}
}
return 0;
}
static int tsdbLoadBlockDataImpl(SRWHelper *pHelper, SCompBlock *pCompBlock, SDataCols *pDataCols) {
ASSERT(pCompBlock->numOfSubBlocks <= 1);
ASSERT(tsizeof(pHelper->pBuffer) >= pCompBlock->len);
SCompData *pCompData = (SCompData *)pHelper->pBuffer;
int fd = (pCompBlock->last) ? pHelper->files.lastF.fd : pHelper->files.dataF.fd;
if (lseek(fd, pCompBlock->offset, SEEK_SET) < 0) goto _err;
if (tread(fd, (void *)pCompData, pCompBlock->len) < pCompBlock->len) goto _err;
ASSERT(pCompData->numOfCols == pCompBlock->numOfCols);
int32_t tsize = sizeof(SCompData) + sizeof(SCompCol) * pCompBlock->numOfCols + sizeof(TSCKSUM);
if (!taosCheckChecksumWhole((uint8_t *)pCompData, tsize)) goto _err;
pDataCols->numOfRows = pCompBlock->numOfRows;
// Recover the data
int ccol = 0;
int dcol = 0;
while (dcol < pDataCols->numOfCols) {
SDataCol *pDataCol = &(pDataCols->cols[dcol]);
if (ccol >= pCompData->numOfCols) {
// Set current column as NULL and forward
dataColSetNEleNull(pDataCol, pCompBlock->numOfRows, pDataCols->maxPoints);
dcol++;
continue;
}
SCompCol *pCompCol = &(pCompData->cols[ccol]);
if (pCompCol->colId == pDataCol->colId) {
if (pCompBlock->algorithm == TWO_STAGE_COMP) {
int zsize = pDataCol->bytes * pCompBlock->numOfRows + COMP_OVERFLOW_BYTES;
if (pCompCol->type == TSDB_DATA_TYPE_BINARY || pCompCol->type == TSDB_DATA_TYPE_NCHAR) {
zsize += (sizeof(VarDataLenT) * pCompBlock->numOfRows);
}
pHelper->compBuffer = trealloc(pHelper->compBuffer, zsize);
if (pHelper->compBuffer == NULL) goto _err;
}
if (tsdbCheckAndDecodeColumnData(pDataCol, (char *)pCompData + tsize + pCompCol->offset, pCompCol->len,
pCompBlock->algorithm, pCompBlock->numOfRows, pDataCols->maxPoints,
pHelper->compBuffer, tsizeof(pHelper->compBuffer)) < 0)
goto _err;
dcol++;
ccol++;
} else if (pCompCol->colId < pDataCol->colId) {
ccol++;
} else {
// Set current column as NULL and forward
dataColSetNEleNull(pDataCol, pCompBlock->numOfRows, pDataCols->maxPoints);
dcol++;
}
}
return 0;
_err:
return -1;
}
......@@ -57,6 +57,7 @@ SListNode *tdListPopHead(SList *list);
SListNode *tdListPopTail(SList *list);
SListNode *tdListPopNode(SList *list, SListNode *node);
void tdListMove(SList *src, SList *dst);
void tdListDiscard(SList *list);
void tdListNodeGetData(SList *list, SListNode *node, void *target);
void tdListInitIter(SList *list, SListIter *pIter, TD_LIST_DIRECTION_T direction);
......
......@@ -148,6 +148,13 @@ void tdListMove(SList *src, SList *dst) {
}
}
void tdListDiscard(SList *list) {
if (list) {
list->head = list->tail = NULL;
list->numOfEles = 0;
}
}
void tdListNodeGetData(SList *list, SListNode *node, void *target) { memcpy(target, node->data, list->eleSize); }
void tdListInitIter(SList *list, SListIter *pIter, TD_LIST_DIRECTION_T direction) {
......
......@@ -219,6 +219,8 @@ void *tSkipListDestroy(SSkipList *pSkipList) {
void tSkipListNewNodeInfo(SSkipList *pSkipList, int32_t *level, int32_t *headSize) {
if (pSkipList == NULL) {
*level = 1;
*headSize = SL_NODE_HEADER_SIZE(*level);
return;
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册