提交 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 "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 "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
此差异已折叠。
......@@ -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.
先完成此消息的编辑!
想要评论请 注册