提交 2a7e0171 编写于 作者: H Haojun Liao

[td-13039] support interval query.

上级 c440e553
......@@ -589,7 +589,7 @@ SOperatorInfo* createTableSeqScanOperatorInfo(void* pTsdbQueryHandle, SQueryRunt
SOperatorInfo* createAggregateOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput);
SOperatorInfo* createProjectOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput);
SOperatorInfo* createLimitOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream);
SOperatorInfo* createTimeIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput);
SOperatorInfo* createIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput);
SOperatorInfo* createAllTimeIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput);
SOperatorInfo* createSWindowOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput);
SOperatorInfo* createFillOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput, bool multigroupResult);
......
......@@ -2166,7 +2166,7 @@ static int32_t setupQueryRuntimeEnv(SQueryRuntimeEnv *pRuntimeEnv, int32_t numOf
}
case OP_TimeWindow: {
pRuntimeEnv->proot =
createTimeIntervalOperatorInfo(pRuntimeEnv, pRuntimeEnv->proot, pQueryAttr->pExpr1, pQueryAttr->numOfOutput);
createIntervalOperatorInfo(pRuntimeEnv, pRuntimeEnv->proot, pQueryAttr->pExpr1, pQueryAttr->numOfOutput);
int32_t opType = pRuntimeEnv->proot->upstream[0]->operatorType;
if (opType != OP_DummyInput && opType != OP_Join) {
setTableScanFilterOperatorInfo(pRuntimeEnv->proot->upstream[0]->info, pRuntimeEnv->proot);
......@@ -6756,7 +6756,7 @@ SOperatorInfo* createLimitOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorI
return pOperator;
}
SOperatorInfo* createTimeIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput) {
SOperatorInfo* createIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput) {
STableIntervalOperatorInfo* pInfo = calloc(1, sizeof(STableIntervalOperatorInfo));
pInfo->pCtx = createSQLFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset);
......
......@@ -32,7 +32,7 @@ typedef struct SDiskbasedBuf SDiskbasedBuf;
#define DEFAULT_INTERN_BUF_PAGE_SIZE (1024L) // in bytes
typedef struct SFilePage {
int64_t num;
int32_t num;
char data[];
} SFilePage;
......
......@@ -127,6 +127,7 @@ int32_t colDataAppend(SColumnInfoData* pColumnInfoData, uint32_t currentRow, con
case TSDB_DATA_TYPE_USMALLINT: {*(int16_t*) p = *(int16_t*) pData;break;}
case TSDB_DATA_TYPE_INT:
case TSDB_DATA_TYPE_UINT: {*(int32_t*) p = *(int32_t*) pData;break;}
case TSDB_DATA_TYPE_TIMESTAMP:
case TSDB_DATA_TYPE_BIGINT:
case TSDB_DATA_TYPE_UBIGINT: {*(int64_t*) p = *(int64_t*) pData;break;}
default:
......
......@@ -68,6 +68,7 @@ typedef struct SResultRow {
} SResultRow;
typedef struct SResultRowInfo {
SResultRow *pCurResult; // current active result row info
SResultRow** pResult; // result list
// int16_t type:8; // data type for hash key
int32_t size; // number of result set
......
......@@ -453,7 +453,19 @@ typedef struct SAggSupporter {
SResultRowPool *pool; // The window result objects pool, all the resultRow Objects are allocated and managed by this object.
} SAggSupporter;
typedef struct SOptrBasicInfo STableIntervalOperatorInfo;
typedef struct STableIntervalOperatorInfo {
SOptrBasicInfo binfo;
SDiskbasedBuf *pResultBuf; // query result buffer based on blocked-wised disk file
SGroupResInfo groupResInfo;
SInterval interval;
STimeWindow win;
int32_t precision;
bool timeWindowInterpo;
char **pRow;
SAggSupporter aggSup;
STableQueryInfo *pCurrent;
int32_t order;
} STableIntervalOperatorInfo;
typedef struct SAggOperatorInfo {
SOptrBasicInfo binfo;
......@@ -606,8 +618,8 @@ SOperatorInfo* createMultiTableAggOperatorInfo(SOperatorInfo* downstream, SArray
SOperatorInfo* createProjectOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr,
int32_t numOfOutput);
SOperatorInfo* createLimitOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream);
SOperatorInfo* createTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr,
int32_t numOfOutput);
SOperatorInfo* createIntervalOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SExecTaskInfo* pTaskInfo);
SOperatorInfo* createAllTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream,
SExprInfo* pExpr, int32_t numOfOutput);
SOperatorInfo* createSWindowOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr,
......
......@@ -54,7 +54,6 @@ int32_t getOutputInterResultBufSize(STaskAttr* pQueryAttr) {
}
int32_t initResultRowInfo(SResultRowInfo *pResultRowInfo, int32_t size) {
// pResultRowInfo->type = type;
pResultRowInfo->size = 0;
pResultRowInfo->curPos = -1;
pResultRowInfo->capacity = size;
......
......@@ -44,8 +44,8 @@ typedef struct SLHashObj {
* +-----------+-------+--------+
*/
typedef struct SLHashNode {
int32_t keyLen;
int32_t dataLen;
uint16_t keyLen;
uint16_t dataLen;
} SLHashNode;
#define GET_LHASH_NODE_KEY(_n) (((char*)(_n)) + sizeof(SLHashNode))
......@@ -70,10 +70,10 @@ static int32_t doGetRelatedSplitBucketId(int32_t bucketId, int32_t bits) {
}
static void doCopyObject(char* p, const void* key, int32_t keyLen, const void* data, int32_t size) {
*(int32_t*) p = keyLen;
p += sizeof(int32_t);
*(int32_t*) p = size;
p += sizeof(int32_t);
*(uint16_t*) p = keyLen;
p += sizeof(uint16_t);
*(uint16_t*) p = size;
p += sizeof(uint16_t);
memcpy(p, key, keyLen);
p += keyLen;
......@@ -118,7 +118,7 @@ static int32_t doAddToBucket(SLHashObj* pHashObj, SLHashBucket* pBucket, int32_t
}
pBucket->size += 1;
printf("===> add to bucket:0x%x, num:%d, key:%d\n", index, pBucket->size, *(int*) key);
// printf("===> add to bucket:0x%x, num:%d, key:%d\n", index, pBucket->size, *(int*) key);
return TSDB_CODE_SUCCESS;
}
......@@ -154,6 +154,14 @@ static void doCompressBucketPages(SLHashObj *pHashObj, SLHashBucket* pBucket) {
int32_t* pageId = taosArrayGetLast(pBucket->pPageIdList);
SFilePage* pLast = getBufPage(pHashObj->pBuf, *pageId);
if (pLast->num <= sizeof(SFilePage)) {
// this is empty
dBufSetBufPageRecycled(pHashObj->pBuf, pLast);
releaseBufPage(pHashObj->pBuf, pFirst);
taosArrayRemove(pBucket->pPageIdList, numOfPages - 1);
return;
}
char* pStart = pLast->data;
int32_t nodeSize = GET_LHASH_NODE_LEN(pStart);
while (1) {
......@@ -162,21 +170,33 @@ static void doCompressBucketPages(SLHashObj *pHashObj, SLHashBucket* pBucket) {
SLHashNode* pNode = (SLHashNode*)pStart;
doCopyObject(p, GET_LHASH_NODE_KEY(pStart), pNode->keyLen, GET_LHASH_NODE_DATA(pStart), pNode->dataLen);
setBufPageDirty(pFirst, true);
setBufPageDirty(pLast, true);
ASSERT(pLast->num >= nodeSize + sizeof(SFilePage));
pFirst->num += nodeSize;
pLast->num -= nodeSize;
pStart += nodeSize;
if (pStart - pLast->data >= pLast->num) {
if (pLast->num <= sizeof(SFilePage)) {
// this is empty
dBufSetBufPageRecycled(pHashObj->pBuf, pLast);
releaseBufPage(pHashObj->pBuf, pFirst);
taosArrayRemove(pBucket->pPageIdList, numOfPages - 1);
break;
}
nodeSize = GET_LHASH_NODE_LEN(pStart);
} else { // move to the front of pLast page
memmove(pLast->data, pStart,(((char*)pLast) + pLast->num - pStart));
if (pStart != pLast->data) {
memmove(pLast->data, pStart, (((char*)pLast) + pLast->num - pStart));
setBufPageDirty(pLast, true);
}
releaseBufPage(pHashObj->pBuf, pLast);
releaseBufPage(pHashObj->pBuf, pFirst);
break;
}
}
......@@ -216,7 +236,7 @@ static int32_t doAddNewBucket(SLHashObj* pHashObj) {
taosArrayPush(pBucket->pPageIdList, &pageId);
pHashObj->numOfBuckets += 1;
printf("---------------add new bucket, id:0x%x, total:%d\n", pHashObj->numOfBuckets - 1, pHashObj->numOfBuckets);
// printf("---------------add new bucket, id:0x%x, total:%d\n", pHashObj->numOfBuckets - 1, pHashObj->numOfBuckets);
return TSDB_CODE_SUCCESS;
}
......@@ -281,7 +301,7 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data
if (v >= pHashObj->numOfBuckets) {
int32_t newBucketId = doGetAlternativeBucketId(v, pHashObj->bits, pHashObj->numOfBuckets);
printf("bucketId: 0x%x not exists, put it into 0x%x instead\n", v, newBucketId);
// printf("bucketId: 0x%x not exists, put it into 0x%x instead\n", v, newBucketId);
v = newBucketId;
}
......@@ -305,7 +325,7 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data
int32_t numOfBits = ceil(log(pHashObj->numOfBuckets) / log(2));
if (numOfBits > pHashObj->bits) {
printf("extend the bits from %d to %d, new bucket:%d\n", pHashObj->bits, numOfBits, newBucketId);
// printf("extend the bits from %d to %d, new bucket:%d\n", pHashObj->bits, numOfBits, newBucketId);
ASSERT(numOfBits == pHashObj->bits + 1);
pHashObj->bits = numOfBits;
}
......@@ -314,7 +334,7 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data
// load all data in this bucket and check if the data needs to relocated into the new bucket
SLHashBucket* pBucket = pHashObj->pBucket[splitBucketId];
printf("split %d items' bucket:0x%x to new bucket:0x%x\n", pBucket->size, splitBucketId, newBucketId);
// printf("split %d items' bucket:0x%x to new bucket:0x%x\n", pBucket->size, splitBucketId, newBucketId);
for (int32_t i = 0; i < taosArrayGetSize(pBucket->pPageIdList); ++i) {
int32_t pageId = *(int32_t*)taosArrayGet(pBucket->pPageIdList, i);
......@@ -331,14 +351,14 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data
if (v1 != splitBucketId) { // place it into the new bucket
ASSERT(v1 == newBucketId);
printf("move key:%d to 0x%x bucket, remain items:%d\n", *(int32_t*)k, v1, pBucket->size - 1);
// printf("move key:%d to 0x%x bucket, remain items:%d\n", *(int32_t*)k, v1, pBucket->size - 1);
SLHashBucket* pNewBucket = pHashObj->pBucket[newBucketId];
doAddToBucket(pHashObj, pNewBucket, newBucketId, (void*)GET_LHASH_NODE_KEY(pNode), pNode->keyLen,
GET_LHASH_NODE_KEY(pNode), pNode->dataLen);
doRemoveFromBucket(p, pNode, pBucket);
} else {
printf("check key:%d, located into: %d, skip it\n", *(int*) k, v1);
// printf("check key:%d, located into: %d, skip it\n", *(int*) k, v1);
int32_t nodeSize = GET_LHASH_NODE_LEN(pStart);
pStart += nodeSize;
......@@ -398,8 +418,8 @@ void tHashPrint(const SLHashObj* pHashObj, int32_t type) {
if (type == LINEAR_HASH_DATA) {
for (int32_t i = 0; i < pHashObj->numOfBuckets; ++i) {
printf("bucket: 0x%x, obj:%d, page:%d\n", i, pHashObj->pBucket[i]->size,
(int)taosArrayGetSize(pHashObj->pBucket[i]->pPageIdList));
// printf("bucket: 0x%x, obj:%d, page:%d\n", i, pHashObj->pBucket[i]->size,
// (int)taosArrayGetSize(pHashObj->pBucket[i]->pPageIdList));
}
} else {
dBufPrintStatis(pHashObj->pBuf);
......
......@@ -14,6 +14,7 @@
*/
#include <executorimpl.h>
#include <function.h>
#include <gtest/gtest.h>
#include <tglobal.h>
#include <iostream>
......@@ -47,6 +48,8 @@ typedef struct SDummyInputInfo {
int32_t startVal;
int32_t type;
int32_t numOfRowsPerPage;
int32_t numOfCols; // number of columns
int64_t tsStart;
SSDataBlock* pBlock;
} SDummyInputInfo;
......@@ -117,16 +120,96 @@ SSDataBlock* getDummyBlock(void* param, bool* newgroup) {
return pBlock;
}
SOperatorInfo* createDummyOperator(int32_t startVal, int32_t numOfBlocks, int32_t rowsPerPage, int32_t type) {
SSDataBlock* get2ColsDummyBlock(void* param, bool* newgroup) {
SOperatorInfo* pOperator = static_cast<SOperatorInfo*>(param);
SDummyInputInfo* pInfo = static_cast<SDummyInputInfo*>(pOperator->info);
if (pInfo->current >= pInfo->totalPages) {
return NULL;
}
if (pInfo->pBlock == NULL) {
pInfo->pBlock = static_cast<SSDataBlock*>(calloc(1, sizeof(SSDataBlock)));
pInfo->pBlock->pDataBlock = taosArrayInit(4, sizeof(SColumnInfoData));
SColumnInfoData colInfo = {0};
colInfo.info.type = TSDB_DATA_TYPE_TIMESTAMP;
colInfo.info.bytes = sizeof(int64_t);
colInfo.info.colId = 1;
colInfo.pData = static_cast<char*>(calloc(pInfo->numOfRowsPerPage, sizeof(int64_t)));
// colInfo.nullbitmap = static_cast<char*>(calloc(1, (pInfo->numOfRowsPerPage + 7) / 8));
taosArrayPush(pInfo->pBlock->pDataBlock, &colInfo);
SColumnInfoData colInfo1 = {0};
colInfo1.info.type = TSDB_DATA_TYPE_INT;
colInfo1.info.bytes = 4;
colInfo1.info.colId = 2;
colInfo1.pData = static_cast<char*>(calloc(pInfo->numOfRowsPerPage, sizeof(int32_t)));
colInfo1.nullbitmap = static_cast<char*>(calloc(1, (pInfo->numOfRowsPerPage + 7) / 8));
taosArrayPush(pInfo->pBlock->pDataBlock, &colInfo1);
} else {
blockDataClearup(pInfo->pBlock, false);
}
SSDataBlock* pBlock = pInfo->pBlock;
char buf[128] = {0};
char b1[128] = {0};
int64_t ts = 0;
int32_t v = 0;
for(int32_t i = 0; i < pInfo->numOfRowsPerPage; ++i) {
SColumnInfoData* pColInfo = static_cast<SColumnInfoData*>(TARRAY_GET_ELEM(pBlock->pDataBlock, 0));
ts = (++pInfo->tsStart);
colDataAppend(pColInfo, i, reinterpret_cast<const char*>(&ts), false);
SColumnInfoData* pColInfo1 = static_cast<SColumnInfoData*>(TARRAY_GET_ELEM(pBlock->pDataBlock, 1));
if (pInfo->type == data_desc) {
v = (--pInfo->startVal);
} else if (pInfo->type == data_asc) {
v = ++pInfo->startVal;
} else if (pInfo->type == data_rand) {
v = random();
}
colDataAppend(pColInfo1, i, reinterpret_cast<const char*>(&v), false);
// sprintf(buf, "this is %d row", i);
// STR_TO_VARSTR(b1, buf);
//
// SColumnInfoData* pColInfo2 = static_cast<SColumnInfoData*>(TARRAY_GET_ELEM(pBlock->pDataBlock, 1));
// colDataAppend(pColInfo2, i, b1, false);
}
pBlock->info.rows = pInfo->numOfRowsPerPage;
pBlock->info.numOfCols = 1;
pInfo->current += 1;
blockDataUpdateTsWindow(pBlock);
return pBlock;
}
SOperatorInfo* createDummyOperator(int32_t startVal, int32_t numOfBlocks, int32_t rowsPerPage, int32_t type, int32_t numOfCols) {
SOperatorInfo* pOperator = static_cast<SOperatorInfo*>(calloc(1, sizeof(SOperatorInfo)));
pOperator->name = "dummyInputOpertor4Test";
pOperator->exec = getDummyBlock;
if (numOfCols == 1) {
pOperator->exec = getDummyBlock;
} else {
pOperator->exec = get2ColsDummyBlock;
}
SDummyInputInfo *pInfo = (SDummyInputInfo*) calloc(1, sizeof(SDummyInputInfo));
pInfo->totalPages = numOfBlocks;
pInfo->startVal = startVal;
pInfo->numOfRowsPerPage = rowsPerPage;
pInfo->type = type;
pInfo->tsStart = 1620000000000;
pOperator->info = pInfo;
return pOperator;
......@@ -357,8 +440,6 @@ TEST(testCase, external_sort_Test) {
taosArrayDestroy(pOrderVal);
}
TEST(testCase, sorted_merge_Test) {
srand(time(NULL));
......@@ -432,4 +513,79 @@ TEST(testCase, sorted_merge_Test) {
}
#endif
TEST(testCase, time_interval_Operator_Test) {
srand(time(NULL));
SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder));
SOrder o = {0};
o.order = TSDB_ORDER_ASC;
o.col.info.colId = 1;
o.col.info.type = TSDB_DATA_TYPE_INT;
taosArrayPush(pOrderVal, &o);
SArray* pExprInfo = taosArrayInit(4, sizeof(SExprInfo));
SExprInfo *exp = static_cast<SExprInfo*>(calloc(1, sizeof(SExprInfo)));
exp->base.resSchema = createSchema(TSDB_DATA_TYPE_TIMESTAMP, sizeof(int64_t), 1, "ts");
exp->base.pColumns = static_cast<SColumn*>(calloc(1, sizeof(SColumn)));
exp->base.pColumns->flag = TSDB_COL_NORMAL;
exp->base.pColumns->info = (SColumnInfo) {.colId = 1, .type = TSDB_DATA_TYPE_TIMESTAMP, .bytes = 8};
exp->base.numOfCols = 1;
taosArrayPush(pExprInfo, &exp);
SExprInfo *exp1 = static_cast<SExprInfo*>(calloc(1, sizeof(SExprInfo)));
exp1->base.resSchema = createSchema(TSDB_DATA_TYPE_BIGINT, 8, 2, "res1");
exp1->base.pColumns = static_cast<SColumn*>(calloc(1, sizeof(SColumn)));
exp1->base.pColumns->flag = TSDB_COL_NORMAL;
exp1->base.pColumns->info = (SColumnInfo) {.colId = 1, .type = TSDB_DATA_TYPE_INT, .bytes = 4};
exp1->base.numOfCols = 1;
taosArrayPush(pExprInfo, &exp1);
SOperatorInfo* p = createDummyOperator(1, 1, 2000, data_asc, 2);
SExecTaskInfo ti = {0};
SOperatorInfo* pOperator = createIntervalOperatorInfo(p, pExprInfo, &ti);
bool newgroup = false;
SSDataBlock* pRes = NULL;
int32_t total = 1;
int64_t s1 = taosGetTimestampUs();
int32_t t = 1;
while(1) {
int64_t s = taosGetTimestampUs();
pRes = pOperator->exec(pOperator, &newgroup);
int64_t e = taosGetTimestampUs();
if (t++ == 1) {
printf("---------------elapsed:%ld\n", e - s);
}
if (pRes == NULL) {
break;
}
SColumnInfoData* pCol1 = static_cast<SColumnInfoData*>(taosArrayGet(pRes->pDataBlock, 0));
// SColumnInfoData* pCol2 = static_cast<SColumnInfoData*>(taosArrayGet(pRes->pDataBlock, 1));
for (int32_t i = 0; i < pRes->info.rows; ++i) {
// char* p = colDataGetData(pCol2, i);
printf("%d: %ld\n", total++, ((int64_t*)pCol1->pData)[i]);
// printf("%d: %d, %s\n", total++, ((int32_t*)pCol1->pData)[i], (char*)varDataVal(p));
}
}
int64_t s2 = taosGetTimestampUs();
printf("total:%ld\n", s2 - s1);
pOperator->cleanupFn(pOperator->info, 2);
tfree(exp);
tfree(exp1);
taosArrayDestroy(pExprInfo);
taosArrayDestroy(pOrderVal);
}
#pragma GCC diagnostic pop
......@@ -28,9 +28,9 @@ TEST(testCase, linear_hash_Tests) {
srand(time(NULL));
_hash_fn_t fn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT);
#if 1
SLHashObj* pHashObj = tHashInit(10, 128 + 8, fn, 8);
for(int32_t i = 0; i < 100; ++i) {
#if 0
SLHashObj* pHashObj = tHashInit(256, 4096, fn, 320);
for(int32_t i = 0; i < 5000000; ++i) {
int32_t code = tHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i));
assert(code == 0);
}
......@@ -46,13 +46,13 @@ TEST(testCase, linear_hash_Tests) {
// }
// }
tHashPrint(pHashObj, LINEAR_HASH_DATA);
tHashPrint(pHashObj, LINEAR_HASH_STATIS);
tHashCleanup(pHashObj);
#endif
#if 0
SHashObj* pHashObj = taosHashInit(1000, fn, false, HASH_NO_LOCK);
for(int32_t i = 0; i < 500000; ++i) {
for(int32_t i = 0; i < 1000000; ++i) {
taosHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i));
}
......
......@@ -4395,7 +4395,7 @@ SFunctionFpSet fpSet[1] = {
.addInput = count_function,
.finalize = doFinalizer,
.combine = count_func_merge,
}
},
};
SAggFunctionInfo aggFunc[35] = {{
......
......@@ -5,12 +5,6 @@
#include "tcompression.h"
#include "thash.h"
//enum {
// true = 0x1,
// BUF_PAGE_RELEASED = 0x2,
// true = 0x3,
//};
#define GET_DATA_PAYLOAD(_p) ((char *)(_p)->pData + POINTER_BYTES)
#define NO_IN_MEM_AVAILABLE_PAGES(_b) (listNEles((_b)->lruList) >= (_b)->inMemPages)
......@@ -20,7 +14,7 @@ typedef struct SPageDiskInfo {
} SPageDiskInfo, SFreeListItem;
struct SPageInfo {
SListNode* pn; // point to list node
SListNode* pn; // point to list node struct
void* pData;
int64_t offset;
int32_t pageId;
......@@ -38,7 +32,8 @@ struct SDiskbasedBuf {
char* path; // file path
int32_t pageSize; // current used page size
int32_t inMemPages; // numOfPages that are allocated in memory
SHashObj* groupSet; // id hash table
SList* freePgList; // free page list
SHashObj* groupSet; // id hash table, todo remove it
SHashObj* all;
SList* lruList;
void* emptyDummyIdList; // dummy id list
......@@ -110,6 +105,14 @@ static uint64_t allocatePositionInFile(SDiskbasedBuf* pBuf, size_t size) {
}
}
static void setPageNotInBuf(SPageInfo* pPageInfo) {
pPageInfo->pData = NULL;
}
static FORCE_INLINE size_t getAllocPageSize(int32_t pageSize) {
return pageSize + POINTER_BYTES + 2;
}
/**
* +--------------------------+-------------------+--------------+
* | PTR to SPageInfo (8bytes)| Payload (PageSize)| 2 Extra Bytes|
......@@ -189,17 +192,17 @@ static char* doFlushPageToDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) {
}
} else {// NOTE: the size may be -1, the this recycle page has not been flushed to disk yet.
size = pg->length;
if (size == -1) {
printf("----\n");
}
}
ASSERT(size > 0 || (pg->offset == -1 && pg->length == -1));
char* pDataBuf = pg->pData;
memset(pDataBuf, 0, pBuf->pageSize);
pg->pData = NULL; // this means the data is not in buffer
pg->length = size;
pg->dirty = false;
memset(pDataBuf, 0, getAllocPageSize(pBuf->pageSize));
pg->length = size; // on disk size
return pDataBuf;
}
......@@ -214,7 +217,11 @@ static char* flushPageToDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) {
}
}
return doFlushPageToDisk(pBuf, pg);
char* p = doFlushPageToDisk(pBuf, pg);
setPageNotInBuf(pg);
pg->dirty = false;
return p;
}
// load file block data in disk
......@@ -284,12 +291,23 @@ static SListNode* getEldestUnrefedPage(SDiskbasedBuf* pBuf) {
assert(pageInfo->pageId >= 0 && pageInfo->pn == pn);
if (!pageInfo->used) {
// printf("%d is chosen\n", pageInfo->pageId);
break;
} else {
printf("page %d is used, dirty:%d\n", pageInfo->pageId, pageInfo->dirty);
// printf("page %d is used, dirty:%d\n", pageInfo->pageId, pageInfo->dirty);
}
}
// int32_t pos = listNEles(pBuf->lruList);
// SListIter iter1 = {0};
// tdListInitIter(pBuf->lruList, &iter1, TD_LIST_BACKWARD);
// SListNode* pn1 = NULL;
// while((pn1 = tdListNext(&iter1)) != NULL) {
// SPageInfo* pageInfo = *(SPageInfo**) pn1->data;
// printf("page %d is used, dirty:%d, pos:%d\n", pageInfo->pageId, pageInfo->dirty, pos - 1);
// pos -= 1;
// }
return pn;
}
......@@ -333,10 +351,6 @@ static void lruListMoveToFront(SList *pList, SPageInfo* pi) {
tdListPrependNode(pList, pi->pn);
}
static FORCE_INLINE size_t getAllocPageSize(int32_t pageSize) {
return pageSize + POINTER_BYTES + 2;
}
static SPageInfo* getPageInfoFromPayload(void* page) {
int32_t offset = offsetof(SPageInfo, pData);
char* p = page - offset;
......@@ -348,41 +362,42 @@ static SPageInfo* getPageInfoFromPayload(void* page) {
int32_t createDiskbasedBuf(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir) {
*pBuf = calloc(1, sizeof(SDiskbasedBuf));
SDiskbasedBuf* pResBuf = *pBuf;
if (pResBuf == NULL) {
SDiskbasedBuf* pPBuf = *pBuf;
if (pPBuf == NULL) {
return TSDB_CODE_OUT_OF_MEMORY;
}
pResBuf->pageSize = pagesize;
pResBuf->numOfPages = 0; // all pages are in buffer in the first place
pResBuf->totalBufSize = 0;
pResBuf->inMemPages = inMemBufSize/pagesize; // maximum allowed pages, it is a soft limit.
pResBuf->allocateId = -1;
pResBuf->comp = true;
pResBuf->file = NULL;
pResBuf->qId = qId;
pResBuf->fileSize = 0;
pResBuf->pFree = taosArrayInit(4, sizeof(SFreeListItem));
pPBuf->pageSize = pagesize;
pPBuf->numOfPages = 0; // all pages are in buffer in the first place
pPBuf->totalBufSize = 0;
pPBuf->inMemPages = inMemBufSize/pagesize; // maximum allowed pages, it is a soft limit.
pPBuf->allocateId = -1;
pPBuf->comp = true;
pPBuf->file = NULL;
pPBuf->qId = qId;
pPBuf->fileSize = 0;
pPBuf->pFree = taosArrayInit(4, sizeof(SFreeListItem));
pPBuf->freePgList = tdListNew(POINTER_BYTES);
// at least more than 2 pages must be in memory
assert(inMemBufSize >= pagesize * 2);
pResBuf->lruList = tdListNew(POINTER_BYTES);
pPBuf->lruList = tdListNew(POINTER_BYTES);
// init id hash table
_hash_fn_t fn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT);
pResBuf->groupSet = taosHashInit(10, fn, true, false);
pResBuf->assistBuf = malloc(pResBuf->pageSize + 2); // EXTRA BYTES
pResBuf->all = taosHashInit(10, fn, true, false);
pPBuf->groupSet = taosHashInit(10, fn, true, false);
pPBuf->assistBuf = malloc(pPBuf->pageSize + 2); // EXTRA BYTES
pPBuf->all = taosHashInit(10, fn, true, false);
char path[PATH_MAX] = {0};
taosGetTmpfilePath(dir, "paged-buf", path);
pResBuf->path = strdup(path);
pPBuf->path = strdup(path);
pResBuf->emptyDummyIdList = taosArrayInit(1, sizeof(int32_t));
pPBuf->emptyDummyIdList = taosArrayInit(1, sizeof(int32_t));
// qDebug("QInfo:0x%"PRIx64" create resBuf for output, page size:%d, inmem buf pages:%d, file:%s", qId, pResBuf->pageSize,
// pResBuf->inMemPages, pResBuf->path);
// qDebug("QInfo:0x%"PRIx64" create resBuf for output, page size:%d, inmem buf pages:%d, file:%s", qId, pPBuf->pageSize,
// pPBuf->inMemPages, pPBuf->path);
return TSDB_CODE_SUCCESS;
}
......@@ -401,19 +416,29 @@ void* getNewBufPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) {
}
}
// register new id in this group
*pageId = (++pBuf->allocateId);
// register page id info
SPageInfo* pi = registerPage(pBuf, groupId, *pageId);
SPageInfo* pi = NULL;
if (listNEles(pBuf->freePgList) != 0) {
SListNode* pItem = tdListPopHead(pBuf->freePgList);
pi = *(SPageInfo**) pItem->data;
pi->used = true;
*pageId = pi->pageId;
tfree(pItem);
} else {// create a new pageinfo
// register new id in this group
*pageId = (++pBuf->allocateId);
// register page id info
pi = registerPage(pBuf, groupId, *pageId);
// add to hash map
taosHashPut(pBuf->all, pageId, sizeof(int32_t), &pi, POINTER_BYTES);
pBuf->totalBufSize += pBuf->pageSize;
}
// add to LRU list
assert(listNEles(pBuf->lruList) < pBuf->inMemPages && pBuf->inMemPages > 0);
lruListPushFront(pBuf->lruList, pi);
// add to hash map
taosHashPut(pBuf->all, pageId, sizeof(int32_t), &pi, POINTER_BYTES);
// allocate buf
if (availablePage == NULL) {
pi->pData = calloc(1, getAllocPageSize(pBuf->pageSize)); // add extract bytes in case of zipped buffer increased.
......@@ -421,11 +446,7 @@ void* getNewBufPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) {
pi->pData = availablePage;
}
pBuf->totalBufSize += pBuf->pageSize;
((void**)pi->pData)[0] = pi;
pi->used = true;
return (void *)(GET_DATA_PAYLOAD(pi));
}
......@@ -467,6 +488,7 @@ void* getBufPage(SDiskbasedBuf* pBuf, int32_t id) {
(*pi)->pData = availablePage;
}
// set the ptr to the new SPageInfo
((void**)((*pi)->pData))[0] = (*pi);
lruListPushFront(pBuf->lruList, *pi);
......@@ -551,6 +573,8 @@ void destroyDiskbasedBuf(SDiskbasedBuf* pBuf) {
}
tdListFree(pBuf->lruList);
tdListFree(pBuf->freePgList);
taosArrayDestroy(pBuf->emptyDummyIdList);
taosArrayDestroy(pBuf->pFree);
......@@ -596,14 +620,15 @@ void setBufPageCompressOnDisk(SDiskbasedBuf* pBuf, bool comp) {
void dBufSetBufPageRecycled(SDiskbasedBuf *pBuf, void* pPage) {
SPageInfo* ppi = getPageInfoFromPayload(pPage);
ppi->used = false;
ppi->used = false;
ppi->dirty = false;
// it is a in-memory page that has not been flushed to disk yet.
if (ppi->length != -1 && ppi->offset != -1) {
SFreeListItem item = {.length = ppi->length, .offset = ppi->offset};
taosArrayPush(pBuf->pFree, &item);
}
// add this pageinfo into the free page info list
SListNode* pNode = tdListPopNode(pBuf->lruList, ppi->pn);
tfree(ppi->pData);
tfree(pNode);
tdListAppend(pBuf->freePgList, &ppi);
}
void dBufSetPrintInfo(SDiskbasedBuf* pBuf) {
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册