Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
taosdata
TDengine
提交
38b62ddd
T
TDengine
项目概览
taosdata
/
TDengine
大约 2 年 前同步成功
通知
1192
Star
22018
Fork
4786
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
1
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
T
TDengine
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
1
Issue
1
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
提交
38b62ddd
编写于
9月 04, 2022
作者:
C
Cary Xu
浏览文件
操作
浏览文件
下载
差异文件
other: merge 3.0
上级
bbd8eeaa
1a764d64
变更
47
展开全部
显示空白变更内容
内联
并排
Showing
47 changed file
with
2726 addition
and
1939 deletion
+2726
-1939
include/common/tcommon.h
include/common/tcommon.h
+0
-1
include/common/tmsg.h
include/common/tmsg.h
+1
-0
include/util/trbtree.h
include/util/trbtree.h
+78
-0
source/client/src/clientImpl.c
source/client/src/clientImpl.c
+6
-1
source/client/src/clientRawBlockWrite.c
source/client/src/clientRawBlockWrite.c
+1
-0
source/common/src/systable.c
source/common/src/systable.c
+1
-0
source/common/src/tdatablock.c
source/common/src/tdatablock.c
+3
-0
source/common/src/tglobal.c
source/common/src/tglobal.c
+0
-4
source/common/src/tmsg.c
source/common/src/tmsg.c
+2
-0
source/dnode/mnode/impl/inc/mndDef.h
source/dnode/mnode/impl/inc/mndDef.h
+1
-0
source/dnode/mnode/impl/src/mndDnode.c
source/dnode/mnode/impl/src/mndDnode.c
+4
-3
source/dnode/mnode/impl/src/mndVgroup.c
source/dnode/mnode/impl/src/mndVgroup.c
+3
-0
source/dnode/vnode/CMakeLists.txt
source/dnode/vnode/CMakeLists.txt
+4
-0
source/dnode/vnode/inc/vnode.h
source/dnode/vnode/inc/vnode.h
+1
-0
source/dnode/vnode/src/inc/tsdb.h
source/dnode/vnode/src/inc/tsdb.h
+70
-34
source/dnode/vnode/src/inc/vnodeInt.h
source/dnode/vnode/src/inc/vnodeInt.h
+1
-0
source/dnode/vnode/src/meta/metaQuery.c
source/dnode/vnode/src/meta/metaQuery.c
+32
-0
source/dnode/vnode/src/tsdb/tsdbCache.c
source/dnode/vnode/src/tsdb/tsdbCache.c
+49
-124
source/dnode/vnode/src/tsdb/tsdbCommit.c
source/dnode/vnode/src/tsdb/tsdbCommit.c
+774
-812
source/dnode/vnode/src/tsdb/tsdbCompact.c
source/dnode/vnode/src/tsdb/tsdbCompact.c
+27
-0
source/dnode/vnode/src/tsdb/tsdbCompress.c
source/dnode/vnode/src/tsdb/tsdbCompress.c
+64
-0
source/dnode/vnode/src/tsdb/tsdbDiskData.c
source/dnode/vnode/src/tsdb/tsdbDiskData.c
+84
-0
source/dnode/vnode/src/tsdb/tsdbFS.c
source/dnode/vnode/src/tsdb/tsdbFS.c
+188
-118
source/dnode/vnode/src/tsdb/tsdbFile.c
source/dnode/vnode/src/tsdb/tsdbFile.c
+45
-18
source/dnode/vnode/src/tsdb/tsdbMergeTree.c
source/dnode/vnode/src/tsdb/tsdbMergeTree.c
+378
-0
source/dnode/vnode/src/tsdb/tsdbRead.c
source/dnode/vnode/src/tsdb/tsdbRead.c
+264
-398
source/dnode/vnode/src/tsdb/tsdbReaderWriter.c
source/dnode/vnode/src/tsdb/tsdbReaderWriter.c
+123
-122
source/dnode/vnode/src/tsdb/tsdbRetention.c
source/dnode/vnode/src/tsdb/tsdbRetention.c
+1
-1
source/dnode/vnode/src/tsdb/tsdbSnapshot.c
source/dnode/vnode/src/tsdb/tsdbSnapshot.c
+58
-56
source/dnode/vnode/src/tsdb/tsdbUtil.c
source/dnode/vnode/src/tsdb/tsdbUtil.c
+101
-85
source/dnode/vnode/src/vnd/vnodeQuery.c
source/dnode/vnode/src/vnd/vnodeQuery.c
+1
-0
source/libs/executor/inc/executil.h
source/libs/executor/inc/executil.h
+1
-3
source/libs/executor/src/dataDeleter.c
source/libs/executor/src/dataDeleter.c
+3
-1
source/libs/executor/src/dataDispatcher.c
source/libs/executor/src/dataDispatcher.c
+12
-1
source/libs/executor/src/executil.c
source/libs/executor/src/executil.c
+0
-1
source/libs/executor/src/executor.c
source/libs/executor/src/executor.c
+0
-1
source/libs/executor/src/executorimpl.c
source/libs/executor/src/executorimpl.c
+0
-1
source/libs/executor/src/scanoperator.c
source/libs/executor/src/scanoperator.c
+1
-0
source/libs/executor/src/timewindowoperator.c
source/libs/executor/src/timewindowoperator.c
+8
-2
source/libs/executor/src/tsort.c
source/libs/executor/src/tsort.c
+4
-4
source/libs/monitor/src/monMsg.c
source/libs/monitor/src/monMsg.c
+2
-3
source/libs/transport/src/transComm.c
source/libs/transport/src/transComm.c
+4
-4
source/util/src/trbtree.c
source/util/src/trbtree.c
+274
-138
source/util/test/CMakeLists.txt
source/util/test/CMakeLists.txt
+8
-0
source/util/test/trbtreeTest.cpp
source/util/test/trbtreeTest.cpp
+40
-0
tests/script/tsim/parser/slimit_alter_tags.sim
tests/script/tsim/parser/slimit_alter_tags.sim
+1
-0
tests/system-test/fulltest.sh
tests/system-test/fulltest.sh
+3
-3
未找到文件。
include/common/tcommon.h
浏览文件 @
38b62ddd
...
@@ -184,7 +184,6 @@ typedef struct SQueryTableDataCond {
...
@@ -184,7 +184,6 @@ typedef struct SQueryTableDataCond {
STimeWindow
twindows
;
STimeWindow
twindows
;
int64_t
startVersion
;
int64_t
startVersion
;
int64_t
endVersion
;
int64_t
endVersion
;
int64_t
schemaVersion
;
}
SQueryTableDataCond
;
}
SQueryTableDataCond
;
int32_t
tEncodeDataBlock
(
void
**
buf
,
const
SSDataBlock
*
pBlock
);
int32_t
tEncodeDataBlock
(
void
**
buf
,
const
SSDataBlock
*
pBlock
);
...
...
include/common/tmsg.h
浏览文件 @
38b62ddd
...
@@ -1070,6 +1070,7 @@ typedef struct {
...
@@ -1070,6 +1070,7 @@ typedef struct {
typedef
struct
{
typedef
struct
{
int32_t
vgId
;
int32_t
vgId
;
int32_t
syncState
;
int32_t
syncState
;
int64_t
cacheUsage
;
int64_t
numOfTables
;
int64_t
numOfTables
;
int64_t
numOfTimeSeries
;
int64_t
numOfTimeSeries
;
int64_t
totalStorage
;
int64_t
totalStorage
;
...
...
include/util/trbtree.h
0 → 100644
浏览文件 @
38b62ddd
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef _TD_UTIL_RBTREE_H_
#define _TD_UTIL_RBTREE_H_
#include "os.h"
#ifdef __cplusplus
extern
"C"
{
#endif
typedef
struct
SRBTree
SRBTree
;
typedef
struct
SRBTreeNode
SRBTreeNode
;
typedef
struct
SRBTreeIter
SRBTreeIter
;
typedef
int32_t
(
*
tRBTreeCmprFn
)(
const
void
*
,
const
void
*
);
// SRBTree =============================================
#define tRBTreeMin(T) ((T)->min == ((T)->NIL) ? NULL : (T)->min)
#define tRBTreeMax(T) ((T)->max == ((T)->NIL) ? NULL : (T)->max)
void
tRBTreeCreate
(
SRBTree
*
pTree
,
tRBTreeCmprFn
cmprFn
);
SRBTreeNode
*
tRBTreePut
(
SRBTree
*
pTree
,
SRBTreeNode
*
z
);
void
tRBTreeDrop
(
SRBTree
*
pTree
,
SRBTreeNode
*
z
);
SRBTreeNode
*
tRBTreeDropByKey
(
SRBTree
*
pTree
,
void
*
pKey
);
SRBTreeNode
*
tRBTreeGet
(
SRBTree
*
pTree
,
void
*
pKey
);
// SRBTreeIter =============================================
#define tRBTreeIterCreate(tree, ascend) \
(SRBTreeIter) { .asc = (ascend), .pTree = (tree), .pNode = (ascend) ? (tree)->min : (tree)->max }
SRBTreeNode
*
tRBTreeIterNext
(
SRBTreeIter
*
pIter
);
// STRUCT =============================================
typedef
enum
{
RED
,
BLACK
}
ECOLOR
;
struct
SRBTreeNode
{
ECOLOR
color
;
SRBTreeNode
*
parent
;
SRBTreeNode
*
left
;
SRBTreeNode
*
right
;
};
#define RBTREE_NODE_PAYLOAD(N) ((const void *)&(N)[1])
struct
SRBTree
{
tRBTreeCmprFn
cmprFn
;
int64_t
n
;
SRBTreeNode
*
root
;
SRBTreeNode
*
min
;
SRBTreeNode
*
max
;
SRBTreeNode
*
NIL
;
SRBTreeNode
NILNODE
;
};
struct
SRBTreeIter
{
int8_t
asc
;
SRBTree
*
pTree
;
SRBTreeNode
*
pNode
;
};
#ifdef __cplusplus
}
#endif
#endif
/*_TD_UTIL_RBTREE_H_*/
\ No newline at end of file
source/client/src/clientImpl.c
浏览文件 @
38b62ddd
...
@@ -1672,7 +1672,12 @@ static int32_t doConvertJson(SReqResultInfo* pResultInfo, int32_t numOfCols, int
...
@@ -1672,7 +1672,12 @@ static int32_t doConvertJson(SReqResultInfo* pResultInfo, int32_t numOfCols, int
break
;
break
;
}
}
}
}
if
(
!
needConvert
)
return
TSDB_CODE_SUCCESS
;
if
(
!
needConvert
)
{
return
TSDB_CODE_SUCCESS
;
}
tscDebug
(
"start to convert form json format string"
);
char
*
p
=
(
char
*
)
pResultInfo
->
pData
;
char
*
p
=
(
char
*
)
pResultInfo
->
pData
;
int32_t
dataLen
=
estimateJsonLen
(
pResultInfo
,
numOfCols
,
numOfRows
);
int32_t
dataLen
=
estimateJsonLen
(
pResultInfo
,
numOfCols
,
numOfRows
);
...
...
source/client/src/clientRawBlockWrite.c
浏览文件 @
38b62ddd
...
@@ -765,6 +765,7 @@ static int32_t taosCreateTable(TAOS* taos, void* meta, int32_t metaLen) {
...
@@ -765,6 +765,7 @@ static int32_t taosCreateTable(TAOS* taos, void* meta, int32_t metaLen) {
}
}
taosArrayPush
(
pRequest
->
tableList
,
&
pName
);
taosArrayPush
(
pRequest
->
tableList
,
&
pName
);
pCreateReq
->
flags
|=
TD_CREATE_IF_NOT_EXISTS
;
// change tag cid to new cid
// change tag cid to new cid
if
(
pCreateReq
->
type
==
TSDB_CHILD_TABLE
)
{
if
(
pCreateReq
->
type
==
TSDB_CHILD_TABLE
)
{
STableMeta
*
pTableMeta
=
NULL
;
STableMeta
*
pTableMeta
=
NULL
;
...
...
source/common/src/systable.c
浏览文件 @
38b62ddd
...
@@ -207,6 +207,7 @@ static const SSysDbTableSchema vgroupsSchema[] = {
...
@@ -207,6 +207,7 @@ static const SSysDbTableSchema vgroupsSchema[] = {
{.
name
=
"v3_dnode"
,
.
bytes
=
4
,
.
type
=
TSDB_DATA_TYPE_INT
,
.
sysInfo
=
true
},
{.
name
=
"v3_dnode"
,
.
bytes
=
4
,
.
type
=
TSDB_DATA_TYPE_INT
,
.
sysInfo
=
true
},
{.
name
=
"v3_status"
,
.
bytes
=
10
+
VARSTR_HEADER_SIZE
,
.
type
=
TSDB_DATA_TYPE_VARCHAR
,
.
sysInfo
=
true
},
{.
name
=
"v3_status"
,
.
bytes
=
10
+
VARSTR_HEADER_SIZE
,
.
type
=
TSDB_DATA_TYPE_VARCHAR
,
.
sysInfo
=
true
},
{.
name
=
"status"
,
.
bytes
=
12
+
VARSTR_HEADER_SIZE
,
.
type
=
TSDB_DATA_TYPE_VARCHAR
,
.
sysInfo
=
true
},
{.
name
=
"status"
,
.
bytes
=
12
+
VARSTR_HEADER_SIZE
,
.
type
=
TSDB_DATA_TYPE_VARCHAR
,
.
sysInfo
=
true
},
{.
name
=
"cacheload"
,
.
bytes
=
4
,
.
type
=
TSDB_DATA_TYPE_INT
,
.
sysInfo
=
true
},
{.
name
=
"nfiles"
,
.
bytes
=
4
,
.
type
=
TSDB_DATA_TYPE_INT
,
.
sysInfo
=
true
},
{.
name
=
"nfiles"
,
.
bytes
=
4
,
.
type
=
TSDB_DATA_TYPE_INT
,
.
sysInfo
=
true
},
{.
name
=
"file_size"
,
.
bytes
=
4
,
.
type
=
TSDB_DATA_TYPE_INT
,
.
sysInfo
=
true
},
{.
name
=
"file_size"
,
.
bytes
=
4
,
.
type
=
TSDB_DATA_TYPE_INT
,
.
sysInfo
=
true
},
{.
name
=
"tsma"
,
.
bytes
=
1
,
.
type
=
TSDB_DATA_TYPE_TINYINT
,
.
sysInfo
=
true
},
{.
name
=
"tsma"
,
.
bytes
=
1
,
.
type
=
TSDB_DATA_TYPE_TINYINT
,
.
sysInfo
=
true
},
...
...
source/common/src/tdatablock.c
浏览文件 @
38b62ddd
...
@@ -2120,6 +2120,7 @@ void blockEncode(const SSDataBlock* pBlock, char* data, int32_t* dataLen, int32_
...
@@ -2120,6 +2120,7 @@ void blockEncode(const SSDataBlock* pBlock, char* data, int32_t* dataLen, int32_
int32_t
*
rows
=
(
int32_t
*
)
data
;
int32_t
*
rows
=
(
int32_t
*
)
data
;
*
rows
=
pBlock
->
info
.
rows
;
*
rows
=
pBlock
->
info
.
rows
;
data
+=
sizeof
(
int32_t
);
data
+=
sizeof
(
int32_t
);
ASSERT
(
*
rows
>
0
);
int32_t
*
cols
=
(
int32_t
*
)
data
;
int32_t
*
cols
=
(
int32_t
*
)
data
;
*
cols
=
numOfCols
;
*
cols
=
numOfCols
;
...
@@ -2183,6 +2184,8 @@ void blockEncode(const SSDataBlock* pBlock, char* data, int32_t* dataLen, int32_
...
@@ -2183,6 +2184,8 @@ void blockEncode(const SSDataBlock* pBlock, char* data, int32_t* dataLen, int32_
*
actualLen
=
*
dataLen
;
*
actualLen
=
*
dataLen
;
*
groupId
=
pBlock
->
info
.
groupId
;
*
groupId
=
pBlock
->
info
.
groupId
;
ASSERT
(
*
dataLen
>
0
);
uDebug
(
"build data block, actualLen:%d, rows:%d, cols:%d"
,
*
dataLen
,
*
rows
,
*
cols
);
}
}
const
char
*
blockDecode
(
SSDataBlock
*
pBlock
,
const
char
*
pData
)
{
const
char
*
blockDecode
(
SSDataBlock
*
pBlock
,
const
char
*
pData
)
{
...
...
source/common/src/tglobal.c
浏览文件 @
38b62ddd
...
@@ -129,10 +129,6 @@ int32_t tsMinIntervalTime = 1;
...
@@ -129,10 +129,6 @@ int32_t tsMinIntervalTime = 1;
int32_t
tsQueryBufferSize
=
-
1
;
int32_t
tsQueryBufferSize
=
-
1
;
int64_t
tsQueryBufferSizeBytes
=
-
1
;
int64_t
tsQueryBufferSizeBytes
=
-
1
;
// tsdb config
// For backward compatibility
bool
tsdbForceKeepFile
=
false
;
int32_t
tsDiskCfgNum
=
0
;
int32_t
tsDiskCfgNum
=
0
;
SDiskCfg
tsDiskCfg
[
TFS_MAX_DISKS
]
=
{
0
};
SDiskCfg
tsDiskCfg
[
TFS_MAX_DISKS
]
=
{
0
};
...
...
source/common/src/tmsg.c
浏览文件 @
38b62ddd
...
@@ -994,6 +994,7 @@ int32_t tSerializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) {
...
@@ -994,6 +994,7 @@ int32_t tSerializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) {
SVnodeLoad
*
pload
=
taosArrayGet
(
pReq
->
pVloads
,
i
);
SVnodeLoad
*
pload
=
taosArrayGet
(
pReq
->
pVloads
,
i
);
if
(
tEncodeI32
(
&
encoder
,
pload
->
vgId
)
<
0
)
return
-
1
;
if
(
tEncodeI32
(
&
encoder
,
pload
->
vgId
)
<
0
)
return
-
1
;
if
(
tEncodeI32
(
&
encoder
,
pload
->
syncState
)
<
0
)
return
-
1
;
if
(
tEncodeI32
(
&
encoder
,
pload
->
syncState
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pload
->
cacheUsage
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pload
->
numOfTables
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pload
->
numOfTables
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pload
->
numOfTimeSeries
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pload
->
numOfTimeSeries
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pload
->
totalStorage
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pload
->
totalStorage
)
<
0
)
return
-
1
;
...
@@ -1063,6 +1064,7 @@ int32_t tDeserializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) {
...
@@ -1063,6 +1064,7 @@ int32_t tDeserializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) {
SVnodeLoad
vload
=
{
0
};
SVnodeLoad
vload
=
{
0
};
if
(
tDecodeI32
(
&
decoder
,
&
vload
.
vgId
)
<
0
)
return
-
1
;
if
(
tDecodeI32
(
&
decoder
,
&
vload
.
vgId
)
<
0
)
return
-
1
;
if
(
tDecodeI32
(
&
decoder
,
&
vload
.
syncState
)
<
0
)
return
-
1
;
if
(
tDecodeI32
(
&
decoder
,
&
vload
.
syncState
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
vload
.
cacheUsage
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
vload
.
numOfTables
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
vload
.
numOfTables
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
vload
.
numOfTimeSeries
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
vload
.
numOfTimeSeries
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
vload
.
totalStorage
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
vload
.
totalStorage
)
<
0
)
return
-
1
;
...
...
source/dnode/mnode/impl/inc/mndDef.h
浏览文件 @
38b62ddd
...
@@ -343,6 +343,7 @@ typedef struct {
...
@@ -343,6 +343,7 @@ typedef struct {
uint32_t
hashEnd
;
uint32_t
hashEnd
;
char
dbName
[
TSDB_DB_FNAME_LEN
];
char
dbName
[
TSDB_DB_FNAME_LEN
];
int64_t
dbUid
;
int64_t
dbUid
;
int64_t
cacheUsage
;
int64_t
numOfTables
;
int64_t
numOfTables
;
int64_t
numOfTimeSeries
;
int64_t
numOfTimeSeries
;
int64_t
totalStorage
;
int64_t
totalStorage
;
...
...
source/dnode/mnode/impl/src/mndDnode.c
浏览文件 @
38b62ddd
...
@@ -347,6 +347,7 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) {
...
@@ -347,6 +347,7 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) {
SVgObj
*
pVgroup
=
mndAcquireVgroup
(
pMnode
,
pVload
->
vgId
);
SVgObj
*
pVgroup
=
mndAcquireVgroup
(
pMnode
,
pVload
->
vgId
);
if
(
pVgroup
!=
NULL
)
{
if
(
pVgroup
!=
NULL
)
{
if
(
pVload
->
syncState
==
TAOS_SYNC_STATE_LEADER
)
{
if
(
pVload
->
syncState
==
TAOS_SYNC_STATE_LEADER
)
{
pVgroup
->
cacheUsage
=
pVload
->
cacheUsage
;
pVgroup
->
numOfTables
=
pVload
->
numOfTables
;
pVgroup
->
numOfTables
=
pVload
->
numOfTables
;
pVgroup
->
numOfTimeSeries
=
pVload
->
numOfTimeSeries
;
pVgroup
->
numOfTimeSeries
=
pVload
->
numOfTimeSeries
;
pVgroup
->
totalStorage
=
pVload
->
totalStorage
;
pVgroup
->
totalStorage
=
pVload
->
totalStorage
;
...
...
source/dnode/mnode/impl/src/mndVgroup.c
浏览文件 @
38b62ddd
...
@@ -696,6 +696,9 @@ static int32_t mndRetrieveVgroups(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *p
...
@@ -696,6 +696,9 @@ static int32_t mndRetrieveVgroups(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *p
pColInfo
=
taosArrayGet
(
pBlock
->
pDataBlock
,
cols
++
);
pColInfo
=
taosArrayGet
(
pBlock
->
pDataBlock
,
cols
++
);
colDataAppendNULL
(
pColInfo
,
numOfRows
);
colDataAppendNULL
(
pColInfo
,
numOfRows
);
pColInfo
=
taosArrayGet
(
pBlock
->
pDataBlock
,
cols
++
);
colDataAppend
(
pColInfo
,
numOfRows
,
(
const
char
*
)
&
pVgroup
->
cacheUsage
,
false
);
pColInfo
=
taosArrayGet
(
pBlock
->
pDataBlock
,
cols
++
);
pColInfo
=
taosArrayGet
(
pBlock
->
pDataBlock
,
cols
++
);
colDataAppendNULL
(
pColInfo
,
numOfRows
);
colDataAppendNULL
(
pColInfo
,
numOfRows
);
...
...
source/dnode/vnode/CMakeLists.txt
浏览文件 @
38b62ddd
...
@@ -50,6 +50,10 @@ target_sources(
...
@@ -50,6 +50,10 @@ target_sources(
"src/tsdb/tsdbSnapshot.c"
"src/tsdb/tsdbSnapshot.c"
"src/tsdb/tsdbCacheRead.c"
"src/tsdb/tsdbCacheRead.c"
"src/tsdb/tsdbRetention.c"
"src/tsdb/tsdbRetention.c"
"src/tsdb/tsdbDiskData.c"
"src/tsdb/tsdbCompress.c"
"src/tsdb/tsdbCompact.c"
"src/tsdb/tsdbMergeTree.c"
# tq
# tq
"src/tq/tq.c"
"src/tq/tq.c"
...
...
source/dnode/vnode/inc/vnode.h
浏览文件 @
38b62ddd
...
@@ -155,6 +155,7 @@ int32_t tsdbGetTableSchema(SVnode *pVnode, int64_t uid, STSchema **pSchema, int6
...
@@ -155,6 +155,7 @@ int32_t tsdbGetTableSchema(SVnode *pVnode, int64_t uid, STSchema **pSchema, int6
void
tsdbCacheSetCapacity
(
SVnode
*
pVnode
,
size_t
capacity
);
void
tsdbCacheSetCapacity
(
SVnode
*
pVnode
,
size_t
capacity
);
size_t
tsdbCacheGetCapacity
(
SVnode
*
pVnode
);
size_t
tsdbCacheGetCapacity
(
SVnode
*
pVnode
);
size_t
tsdbCacheGetUsage
(
SVnode
*
pVnode
);
// tq
// tq
typedef
struct
SMetaTableInfo
{
typedef
struct
SMetaTableInfo
{
...
...
source/dnode/vnode/src/inc/tsdb.h
浏览文件 @
38b62ddd
...
@@ -42,15 +42,15 @@ typedef struct SMemTable SMemTable;
...
@@ -42,15 +42,15 @@ typedef struct SMemTable SMemTable;
typedef
struct
STbDataIter
STbDataIter
;
typedef
struct
STbDataIter
STbDataIter
;
typedef
struct
SMapData
SMapData
;
typedef
struct
SMapData
SMapData
;
typedef
struct
SBlockIdx
SBlockIdx
;
typedef
struct
SBlockIdx
SBlockIdx
;
typedef
struct
S
Block
SBloc
k
;
typedef
struct
S
DataBlk
SDataBl
k
;
typedef
struct
S
BlockL
SBlockL
;
typedef
struct
S
SstBlk
SSstBlk
;
typedef
struct
SColData
SColData
;
typedef
struct
SColData
SColData
;
typedef
struct
SDiskDataHdr
SDiskDataHdr
;
typedef
struct
SDiskDataHdr
SDiskDataHdr
;
typedef
struct
SBlockData
SBlockData
;
typedef
struct
SBlockData
SBlockData
;
typedef
struct
SDelFile
SDelFile
;
typedef
struct
SDelFile
SDelFile
;
typedef
struct
SHeadFile
SHeadFile
;
typedef
struct
SHeadFile
SHeadFile
;
typedef
struct
SDataFile
SDataFile
;
typedef
struct
SDataFile
SDataFile
;
typedef
struct
S
LastFile
SLa
stFile
;
typedef
struct
S
SstFile
SS
stFile
;
typedef
struct
SSmaFile
SSmaFile
;
typedef
struct
SSmaFile
SSmaFile
;
typedef
struct
SDFileSet
SDFileSet
;
typedef
struct
SDFileSet
SDFileSet
;
typedef
struct
SDataFWriter
SDataFWriter
;
typedef
struct
SDataFWriter
SDataFWriter
;
...
@@ -64,9 +64,12 @@ typedef struct STsdbReadSnap STsdbReadSnap;
...
@@ -64,9 +64,12 @@ typedef struct STsdbReadSnap STsdbReadSnap;
typedef
struct
SBlockInfo
SBlockInfo
;
typedef
struct
SBlockInfo
SBlockInfo
;
typedef
struct
SSmaInfo
SSmaInfo
;
typedef
struct
SSmaInfo
SSmaInfo
;
typedef
struct
SBlockCol
SBlockCol
;
typedef
struct
SBlockCol
SBlockCol
;
typedef
struct
SVersionRange
SVersionRange
;
#define TSDB_FILE_DLMT ((uint32_t)0xF00AFA0F)
#define TSDB_FILE_DLMT ((uint32_t)0xF00AFA0F)
#define TSDB_MAX_SUBBLOCKS 8
#define TSDB_MAX_SUBBLOCKS 8
#define TSDB_MAX_LAST_FILE 16
#define TSDB_DEFAULT_LAST_FILE 8
#define TSDB_FHDR_SIZE 512
#define TSDB_FHDR_SIZE 512
#define HAS_NONE ((int8_t)0x1)
#define HAS_NONE ((int8_t)0x1)
...
@@ -111,15 +114,15 @@ int32_t tTABLEIDCmprFn(const void *p1, const void *p2);
...
@@ -111,15 +114,15 @@ int32_t tTABLEIDCmprFn(const void *p1, const void *p2);
int32_t
tPutBlockCol
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tPutBlockCol
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tGetBlockCol
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tGetBlockCol
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tBlockColCmprFn
(
const
void
*
p1
,
const
void
*
p2
);
int32_t
tBlockColCmprFn
(
const
void
*
p1
,
const
void
*
p2
);
// S
Bloc
k
// S
DataBl
k
void
t
BlockReset
(
SBloc
k
*
pBlock
);
void
t
DataBlkReset
(
SDataBl
k
*
pBlock
);
int32_t
tPut
Bloc
k
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tPut
DataBl
k
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tGet
Bloc
k
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tGet
DataBl
k
(
uint8_t
*
p
,
void
*
ph
);
int32_t
t
Bloc
kCmprFn
(
const
void
*
p1
,
const
void
*
p2
);
int32_t
t
DataBl
kCmprFn
(
const
void
*
p1
,
const
void
*
p2
);
bool
t
BlockHasSma
(
SBlock
*
pBloc
k
);
bool
t
DataBlkHasSma
(
SDataBlk
*
pDataBl
k
);
// S
BlockL
// S
SstBlk
int32_t
tPut
BlockL
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tPut
SstBlk
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tGet
BlockL
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tGet
SstBlk
(
uint8_t
*
p
,
void
*
ph
);
// SBlockIdx
// SBlockIdx
int32_t
tPutBlockIdx
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tPutBlockIdx
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tGetBlockIdx
(
uint8_t
*
p
,
void
*
ph
);
int32_t
tGetBlockIdx
(
uint8_t
*
p
,
void
*
ph
);
...
@@ -170,6 +173,7 @@ int32_t tGetDelData(uint8_t *p, void *ph);
...
@@ -170,6 +173,7 @@ int32_t tGetDelData(uint8_t *p, void *ph);
void
tMapDataReset
(
SMapData
*
pMapData
);
void
tMapDataReset
(
SMapData
*
pMapData
);
void
tMapDataClear
(
SMapData
*
pMapData
);
void
tMapDataClear
(
SMapData
*
pMapData
);
int32_t
tMapDataPutItem
(
SMapData
*
pMapData
,
void
*
pItem
,
int32_t
(
*
tPutItemFn
)(
uint8_t
*
,
void
*
));
int32_t
tMapDataPutItem
(
SMapData
*
pMapData
,
void
*
pItem
,
int32_t
(
*
tPutItemFn
)(
uint8_t
*
,
void
*
));
int32_t
tMapDataCopy
(
SMapData
*
pFrom
,
SMapData
*
pTo
);
void
tMapDataGetItemByIdx
(
SMapData
*
pMapData
,
int32_t
idx
,
void
*
pItem
,
int32_t
(
*
tGetItemFn
)(
uint8_t
*
,
void
*
));
void
tMapDataGetItemByIdx
(
SMapData
*
pMapData
,
int32_t
idx
,
void
*
pItem
,
int32_t
(
*
tGetItemFn
)(
uint8_t
*
,
void
*
));
int32_t
tMapDataSearch
(
SMapData
*
pMapData
,
void
*
pSearchItem
,
int32_t
(
*
tGetItemFn
)(
uint8_t
*
,
void
*
),
int32_t
tMapDataSearch
(
SMapData
*
pMapData
,
void
*
pSearchItem
,
int32_t
(
*
tGetItemFn
)(
uint8_t
*
,
void
*
),
int32_t
(
*
tItemCmprFn
)(
const
void
*
,
const
void
*
),
void
*
pItem
);
int32_t
(
*
tItemCmprFn
)(
const
void
*
,
const
void
*
),
void
*
pItem
);
...
@@ -215,7 +219,7 @@ bool tsdbDelFileIsSame(SDelFile *pDelFile1, SDelFile *pDelFile2);
...
@@ -215,7 +219,7 @@ bool tsdbDelFileIsSame(SDelFile *pDelFile1, SDelFile *pDelFile2);
int32_t
tsdbDFileRollback
(
STsdb
*
pTsdb
,
SDFileSet
*
pSet
,
EDataFileT
ftype
);
int32_t
tsdbDFileRollback
(
STsdb
*
pTsdb
,
SDFileSet
*
pSet
,
EDataFileT
ftype
);
int32_t
tPutHeadFile
(
uint8_t
*
p
,
SHeadFile
*
pHeadFile
);
int32_t
tPutHeadFile
(
uint8_t
*
p
,
SHeadFile
*
pHeadFile
);
int32_t
tPutDataFile
(
uint8_t
*
p
,
SDataFile
*
pDataFile
);
int32_t
tPutDataFile
(
uint8_t
*
p
,
SDataFile
*
pDataFile
);
int32_t
tPut
LastFile
(
uint8_t
*
p
,
SLastFile
*
pLa
stFile
);
int32_t
tPut
SstFile
(
uint8_t
*
p
,
SSstFile
*
pS
stFile
);
int32_t
tPutSmaFile
(
uint8_t
*
p
,
SSmaFile
*
pSmaFile
);
int32_t
tPutSmaFile
(
uint8_t
*
p
,
SSmaFile
*
pSmaFile
);
int32_t
tPutDelFile
(
uint8_t
*
p
,
SDelFile
*
pDelFile
);
int32_t
tPutDelFile
(
uint8_t
*
p
,
SDelFile
*
pDelFile
);
int32_t
tGetDelFile
(
uint8_t
*
p
,
SDelFile
*
pDelFile
);
int32_t
tGetDelFile
(
uint8_t
*
p
,
SDelFile
*
pDelFile
);
...
@@ -224,7 +228,7 @@ int32_t tGetDFileSet(uint8_t *p, SDFileSet *pSet);
...
@@ -224,7 +228,7 @@ int32_t tGetDFileSet(uint8_t *p, SDFileSet *pSet);
void
tsdbHeadFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SHeadFile
*
pHeadF
,
char
fname
[]);
void
tsdbHeadFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SHeadFile
*
pHeadF
,
char
fname
[]);
void
tsdbDataFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SDataFile
*
pDataF
,
char
fname
[]);
void
tsdbDataFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SDataFile
*
pDataF
,
char
fname
[]);
void
tsdb
LastFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SLastFile
*
pLa
stF
,
char
fname
[]);
void
tsdb
SstFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SSstFile
*
pS
stF
,
char
fname
[]);
void
tsdbSmaFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SSmaFile
*
pSmaF
,
char
fname
[]);
void
tsdbSmaFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SSmaFile
*
pSmaF
,
char
fname
[]);
// SDelFile
// SDelFile
void
tsdbDelFileName
(
STsdb
*
pTsdb
,
SDelFile
*
pFile
,
char
fname
[]);
void
tsdbDelFileName
(
STsdb
*
pTsdb
,
SDelFile
*
pFile
,
char
fname
[]);
...
@@ -250,7 +254,7 @@ int32_t tsdbDataFWriterClose(SDataFWriter **ppWriter, int8_t sync);
...
@@ -250,7 +254,7 @@ int32_t tsdbDataFWriterClose(SDataFWriter **ppWriter, int8_t sync);
int32_t
tsdbUpdateDFileSetHeader
(
SDataFWriter
*
pWriter
);
int32_t
tsdbUpdateDFileSetHeader
(
SDataFWriter
*
pWriter
);
int32_t
tsdbWriteBlockIdx
(
SDataFWriter
*
pWriter
,
SArray
*
aBlockIdx
);
int32_t
tsdbWriteBlockIdx
(
SDataFWriter
*
pWriter
,
SArray
*
aBlockIdx
);
int32_t
tsdbWriteBlock
(
SDataFWriter
*
pWriter
,
SMapData
*
pMapData
,
SBlockIdx
*
pBlockIdx
);
int32_t
tsdbWriteBlock
(
SDataFWriter
*
pWriter
,
SMapData
*
pMapData
,
SBlockIdx
*
pBlockIdx
);
int32_t
tsdbWrite
BlockL
(
SDataFWriter
*
pWriter
,
SArray
*
aBlockL
);
int32_t
tsdbWrite
SstBlk
(
SDataFWriter
*
pWriter
,
SArray
*
aSstBlk
);
int32_t
tsdbWriteBlockData
(
SDataFWriter
*
pWriter
,
SBlockData
*
pBlockData
,
SBlockInfo
*
pBlkInfo
,
SSmaInfo
*
pSmaInfo
,
int32_t
tsdbWriteBlockData
(
SDataFWriter
*
pWriter
,
SBlockData
*
pBlockData
,
SBlockInfo
*
pBlkInfo
,
SSmaInfo
*
pSmaInfo
,
int8_t
cmprAlg
,
int8_t
toLast
);
int8_t
cmprAlg
,
int8_t
toLast
);
...
@@ -260,10 +264,10 @@ int32_t tsdbDataFReaderOpen(SDataFReader **ppReader, STsdb *pTsdb, SDFileSet *pS
...
@@ -260,10 +264,10 @@ int32_t tsdbDataFReaderOpen(SDataFReader **ppReader, STsdb *pTsdb, SDFileSet *pS
int32_t
tsdbDataFReaderClose
(
SDataFReader
**
ppReader
);
int32_t
tsdbDataFReaderClose
(
SDataFReader
**
ppReader
);
int32_t
tsdbReadBlockIdx
(
SDataFReader
*
pReader
,
SArray
*
aBlockIdx
);
int32_t
tsdbReadBlockIdx
(
SDataFReader
*
pReader
,
SArray
*
aBlockIdx
);
int32_t
tsdbReadBlock
(
SDataFReader
*
pReader
,
SBlockIdx
*
pBlockIdx
,
SMapData
*
pMapData
);
int32_t
tsdbReadBlock
(
SDataFReader
*
pReader
,
SBlockIdx
*
pBlockIdx
,
SMapData
*
pMapData
);
int32_t
tsdbRead
BlockL
(
SDataFReader
*
pReader
,
SArray
*
aBlockL
);
int32_t
tsdbRead
SstBlk
(
SDataFReader
*
pReader
,
int32_t
iSst
,
SArray
*
aSstBlk
);
int32_t
tsdbReadBlockSma
(
SDataFReader
*
pReader
,
S
Bloc
k
*
pBlock
,
SArray
*
aColumnDataAgg
);
int32_t
tsdbReadBlockSma
(
SDataFReader
*
pReader
,
S
DataBl
k
*
pBlock
,
SArray
*
aColumnDataAgg
);
int32_t
tsdbReadDataBlock
(
SDataFReader
*
pReader
,
S
Bloc
k
*
pBlock
,
SBlockData
*
pBlockData
);
int32_t
tsdbReadDataBlock
(
SDataFReader
*
pReader
,
S
DataBl
k
*
pBlock
,
SBlockData
*
pBlockData
);
int32_t
tsdbRead
LastBlock
(
SDataFReader
*
pReader
,
SBlockL
*
pBlockL
,
SBlockData
*
pBlockData
);
int32_t
tsdbRead
SstBlock
(
SDataFReader
*
pReader
,
int32_t
iSst
,
SSstBlk
*
pSstBlk
,
SBlockData
*
pBlockData
);
// SDelFWriter
// SDelFWriter
int32_t
tsdbDelFWriterOpen
(
SDelFWriter
**
ppWriter
,
SDelFile
*
pFile
,
STsdb
*
pTsdb
);
int32_t
tsdbDelFWriterOpen
(
SDelFWriter
**
ppWriter
,
SDelFile
*
pFile
,
STsdb
*
pTsdb
);
int32_t
tsdbDelFWriterClose
(
SDelFWriter
**
ppWriter
,
int8_t
sync
);
int32_t
tsdbDelFWriterClose
(
SDelFWriter
**
ppWriter
,
int8_t
sync
);
...
@@ -278,6 +282,8 @@ int32_t tsdbReadDelIdx(SDelFReader *pReader, SArray *aDelIdx);
...
@@ -278,6 +282,8 @@ int32_t tsdbReadDelIdx(SDelFReader *pReader, SArray *aDelIdx);
// tsdbRead.c ==============================================================================================
// tsdbRead.c ==============================================================================================
int32_t
tsdbTakeReadSnap
(
STsdb
*
pTsdb
,
STsdbReadSnap
**
ppSnap
);
int32_t
tsdbTakeReadSnap
(
STsdb
*
pTsdb
,
STsdbReadSnap
**
ppSnap
);
void
tsdbUntakeReadSnap
(
STsdb
*
pTsdb
,
STsdbReadSnap
*
pSnap
);
void
tsdbUntakeReadSnap
(
STsdb
*
pTsdb
,
STsdbReadSnap
*
pSnap
);
// tsdbMerge.c ==============================================================================================
int32_t
tsdbMerge
(
STsdb
*
pTsdb
);
#define TSDB_CACHE_NO(c) ((c).cacheLast == 0)
#define TSDB_CACHE_NO(c) ((c).cacheLast == 0)
#define TSDB_CACHE_LAST_ROW(c) (((c).cacheLast & 1) > 0)
#define TSDB_CACHE_LAST_ROW(c) (((c).cacheLast & 1) > 0)
...
@@ -324,6 +330,11 @@ struct TSDBKEY {
...
@@ -324,6 +330,11 @@ struct TSDBKEY {
TSKEY
ts
;
TSKEY
ts
;
};
};
struct
SVersionRange
{
uint64_t
minVer
;
uint64_t
maxVer
;
};
typedef
struct
SMemSkipListNode
SMemSkipListNode
;
typedef
struct
SMemSkipListNode
SMemSkipListNode
;
struct
SMemSkipListNode
{
struct
SMemSkipListNode
{
int8_t
level
;
int8_t
level
;
...
@@ -416,7 +427,7 @@ struct SSmaInfo {
...
@@ -416,7 +427,7 @@ struct SSmaInfo {
int32_t
size
;
int32_t
size
;
};
};
struct
S
Bloc
k
{
struct
S
DataBl
k
{
TSDBKEY
minKey
;
TSDBKEY
minKey
;
TSDBKEY
maxKey
;
TSDBKEY
maxKey
;
int64_t
minVer
;
int64_t
minVer
;
...
@@ -428,7 +439,7 @@ struct SBlock {
...
@@ -428,7 +439,7 @@ struct SBlock {
SSmaInfo
smaInfo
;
SSmaInfo
smaInfo
;
};
};
struct
S
BlockL
{
struct
S
SstBlk
{
int64_t
suid
;
int64_t
suid
;
int64_t
minUid
;
int64_t
minUid
;
int64_t
maxUid
;
int64_t
maxUid
;
...
@@ -467,12 +478,6 @@ struct SBlockData {
...
@@ -467,12 +478,6 @@ struct SBlockData {
SArray
*
aColData
;
// SArray<SColData>
SArray
*
aColData
;
// SArray<SColData>
};
};
// ================== TSDB global config
extern
bool
tsdbForceKeepFile
;
#define TSDB_FS_ITER_FORWARD TSDB_ORDER_ASC
#define TSDB_FS_ITER_BACKWARD TSDB_ORDER_DESC
struct
TABLEID
{
struct
TABLEID
{
tb_uid_t
suid
;
tb_uid_t
suid
;
tb_uid_t
uid
;
tb_uid_t
uid
;
...
@@ -536,7 +541,7 @@ struct SDataFile {
...
@@ -536,7 +541,7 @@ struct SDataFile {
int64_t
size
;
int64_t
size
;
};
};
struct
S
La
stFile
{
struct
S
S
stFile
{
volatile
int32_t
nRef
;
volatile
int32_t
nRef
;
int64_t
commitID
;
int64_t
commitID
;
...
@@ -556,8 +561,9 @@ struct SDFileSet {
...
@@ -556,8 +561,9 @@ struct SDFileSet {
int32_t
fid
;
int32_t
fid
;
SHeadFile
*
pHeadF
;
SHeadFile
*
pHeadF
;
SDataFile
*
pDataF
;
SDataFile
*
pDataF
;
SLastFile
*
pLastF
;
SSmaFile
*
pSmaF
;
SSmaFile
*
pSmaF
;
uint8_t
nSstF
;
SSstFile
*
aSstF
[
TSDB_MAX_LAST_FILE
];
};
};
struct
SRowIter
{
struct
SRowIter
{
...
@@ -586,13 +592,13 @@ struct SDataFWriter {
...
@@ -586,13 +592,13 @@ struct SDataFWriter {
TdFilePtr
pHeadFD
;
TdFilePtr
pHeadFD
;
TdFilePtr
pDataFD
;
TdFilePtr
pDataFD
;
TdFilePtr
pLastFD
;
TdFilePtr
pSmaFD
;
TdFilePtr
pSmaFD
;
TdFilePtr
pLastFD
;
SHeadFile
fHead
;
SHeadFile
fHead
;
SDataFile
fData
;
SDataFile
fData
;
SLastFile
fLast
;
SSmaFile
fSma
;
SSmaFile
fSma
;
SSstFile
fSst
[
TSDB_MAX_LAST_FILE
];
uint8_t
*
aBuf
[
4
];
uint8_t
*
aBuf
[
4
];
};
};
...
@@ -603,6 +609,36 @@ struct STsdbReadSnap {
...
@@ -603,6 +609,36 @@ struct STsdbReadSnap {
STsdbFS
fs
;
STsdbFS
fs
;
};
};
struct
SDataFReader
{
STsdb
*
pTsdb
;
SDFileSet
*
pSet
;
TdFilePtr
pHeadFD
;
TdFilePtr
pDataFD
;
TdFilePtr
pSmaFD
;
TdFilePtr
aLastFD
[
TSDB_MAX_LAST_FILE
];
uint8_t
*
aBuf
[
3
];
};
typedef
struct
{
int64_t
suid
;
int64_t
uid
;
TSDBROW
row
;
}
SRowInfo
;
typedef
struct
SMergeTree
{
int8_t
backward
;
SRBTree
rbt
;
SArray
*
pIterList
;
struct
SLDataIter
*
pIter
;
}
SMergeTree
;
int32_t
tMergeTreeOpen
(
SMergeTree
*
pMTree
,
int8_t
backward
,
SDataFReader
*
pFReader
,
uint64_t
uid
,
STimeWindow
*
pTimeWindow
,
SVersionRange
*
pVerRange
);
void
tMergeTreeAddIter
(
SMergeTree
*
pMTree
,
struct
SLDataIter
*
pIter
);
bool
tMergeTreeNext
(
SMergeTree
*
pMTree
);
TSDBROW
tMergeTreeGetRow
(
SMergeTree
*
pMTree
);
void
tMergeTreeClose
(
SMergeTree
*
pMTree
);
// ========== inline functions ==========
// ========== inline functions ==========
static
FORCE_INLINE
int32_t
tsdbKeyCmprFn
(
const
void
*
p1
,
const
void
*
p2
)
{
static
FORCE_INLINE
int32_t
tsdbKeyCmprFn
(
const
void
*
p1
,
const
void
*
p2
)
{
TSDBKEY
*
pKey1
=
(
TSDBKEY
*
)
p1
;
TSDBKEY
*
pKey1
=
(
TSDBKEY
*
)
p1
;
...
...
source/dnode/vnode/src/inc/vnodeInt.h
浏览文件 @
38b62ddd
...
@@ -36,6 +36,7 @@
...
@@ -36,6 +36,7 @@
#include "tlosertree.h"
#include "tlosertree.h"
#include "tlrucache.h"
#include "tlrucache.h"
#include "tmsgcb.h"
#include "tmsgcb.h"
#include "trbtree.h"
#include "tref.h"
#include "tref.h"
#include "tskiplist.h"
#include "tskiplist.h"
#include "tstream.h"
#include "tstream.h"
...
...
source/dnode/vnode/src/meta/metaQuery.c
浏览文件 @
38b62ddd
...
@@ -291,6 +291,38 @@ _query:
...
@@ -291,6 +291,38 @@ _query:
tDecoderClear
(
&
dc
);
tDecoderClear
(
&
dc
);
goto
_exit
;
goto
_exit
;
}
}
{
// Traverse to find the previous qualified data
TBC
*
pCur
;
tdbTbcOpen
(
pMeta
->
pTbDb
,
&
pCur
,
NULL
);
STbDbKey
key
=
{.
version
=
sver
,
.
uid
=
INT64_MAX
};
int
c
=
0
;
tdbTbcMoveTo
(
pCur
,
&
key
,
sizeof
(
key
),
&
c
);
if
(
c
<
0
){
tdbTbcMoveToPrev
(
pCur
);
}
void
*
pKey
=
NULL
;
void
*
pVal
=
NULL
;
int
vLen
=
0
,
kLen
=
0
;
while
(
1
){
int32_t
ret
=
tdbTbcPrev
(
pCur
,
&
pKey
,
&
kLen
,
&
pVal
,
&
vLen
);
if
(
ret
<
0
)
break
;
STbDbKey
*
tmp
=
(
STbDbKey
*
)
pKey
;
if
(
tmp
->
uid
!=
uid
){
continue
;
}
SDecoder
dcNew
=
{
0
};
SMetaEntry
meNew
=
{
0
};
tDecoderInit
(
&
dcNew
,
pVal
,
vLen
);
metaDecodeEntry
(
&
dcNew
,
&
meNew
);
pSchema
=
tCloneSSchemaWrapper
(
&
meNew
.
stbEntry
.
schemaRow
);
tDecoderClear
(
&
dcNew
);
tdbTbcClose
(
pCur
);
goto
_exit
;
}
tdbTbcClose
(
pCur
);
}
}
else
if
(
me
.
type
==
TSDB_CHILD_TABLE
)
{
}
else
if
(
me
.
type
==
TSDB_CHILD_TABLE
)
{
uid
=
me
.
ctbEntry
.
suid
;
uid
=
me
.
ctbEntry
.
suid
;
tDecoderClear
(
&
dc
);
tDecoderClear
(
&
dc
);
...
...
source/dnode/vnode/src/tsdb/tsdbCache.c
浏览文件 @
38b62ddd
...
@@ -420,29 +420,14 @@ typedef enum {
...
@@ -420,29 +420,14 @@ typedef enum {
typedef
struct
{
typedef
struct
{
SFSLASTNEXTROWSTATES
state
;
// [input]
SFSLASTNEXTROWSTATES
state
;
// [input]
STsdb
*
pTsdb
;
// [input]
STsdb
*
pTsdb
;
// [input]
SBlockIdx
*
pBlockIdxExp
;
// [input]
STSchema
*
pTSchema
;
// [input]
tb_uid_t
suid
;
tb_uid_t
uid
;
tb_uid_t
uid
;
int32_t
nFileSet
;
int32_t
nFileSet
;
int32_t
iFileSet
;
int32_t
iFileSet
;
SArray
*
aDFileSet
;
SArray
*
aDFileSet
;
SDataFReader
*
pDataFReader
;
SDataFReader
*
pDataFReader
;
SArray
*
aBlockL
;
SBlockL
*
pBlockL
;
SBlockData
*
pBlockDataL
;
SBlockData
blockDataL
;
int32_t
nRow
;
int32_t
iRow
;
TSDBROW
row
;
TSDBROW
row
;
/*
SArray *aBlockIdx;
SMergeTree
mergeTree
;
SBlockIdx *pBlockIdx;
SMapData blockMap;
int32_t nBlock;
int32_t iBlock;
SBlock block;
*/
}
SFSLastNextRowIter
;
}
SFSLastNextRowIter
;
static
int32_t
getNextRowFromFSLast
(
void
*
iter
,
TSDBROW
**
ppRow
)
{
static
int32_t
getNextRowFromFSLast
(
void
*
iter
,
TSDBROW
**
ppRow
)
{
...
@@ -451,22 +436,16 @@ static int32_t getNextRowFromFSLast(void *iter, TSDBROW **ppRow) {
...
@@ -451,22 +436,16 @@ static int32_t getNextRowFromFSLast(void *iter, TSDBROW **ppRow) {
switch
(
state
->
state
)
{
switch
(
state
->
state
)
{
case
SFSLASTNEXTROW_FS
:
case
SFSLASTNEXTROW_FS
:
// state->aDFileSet = state->pTsdb->pFS->cState->aDFileSet;
state
->
nFileSet
=
taosArrayGetSize
(
state
->
aDFileSet
);
state
->
nFileSet
=
taosArrayGetSize
(
state
->
aDFileSet
);
state
->
iFileSet
=
state
->
nFileSet
;
state
->
iFileSet
=
state
->
nFileSet
;
state
->
pBlockDataL
=
NULL
;
case
SFSLASTNEXTROW_FILESET
:
{
case
SFSLASTNEXTROW_FILESET
:
{
SDFileSet
*
pFileSet
=
NULL
;
SDFileSet
*
pFileSet
=
NULL
;
_next_fileset:
_next_fileset:
if
(
--
state
->
iFileSet
>=
0
)
{
if
(
--
state
->
iFileSet
>=
0
)
{
pFileSet
=
(
SDFileSet
*
)
taosArrayGet
(
state
->
aDFileSet
,
state
->
iFileSet
);
pFileSet
=
(
SDFileSet
*
)
taosArrayGet
(
state
->
aDFileSet
,
state
->
iFileSet
);
}
else
{
}
else
{
if
(
state
->
pBlockDataL
)
{
// tMergeTreeClose(&state->mergeTree);
tBlockDataDestroy
(
state
->
pBlockDataL
,
1
);
state
->
pBlockDataL
=
NULL
;
}
*
ppRow
=
NULL
;
*
ppRow
=
NULL
;
return
code
;
return
code
;
...
@@ -475,68 +454,24 @@ static int32_t getNextRowFromFSLast(void *iter, TSDBROW **ppRow) {
...
@@ -475,68 +454,24 @@ static int32_t getNextRowFromFSLast(void *iter, TSDBROW **ppRow) {
code
=
tsdbDataFReaderOpen
(
&
state
->
pDataFReader
,
state
->
pTsdb
,
pFileSet
);
code
=
tsdbDataFReaderOpen
(
&
state
->
pDataFReader
,
state
->
pTsdb
,
pFileSet
);
if
(
code
)
goto
_err
;
if
(
code
)
goto
_err
;
if
(
!
state
->
aBlockL
)
{
tMergeTreeOpen
(
&
state
->
mergeTree
,
1
,
state
->
pDataFReader
,
state
->
uid
,
state
->
aBlockL
=
taosArrayInit
(
0
,
sizeof
(
SBlockL
));
&
(
STimeWindow
){.
skey
=
TSKEY_MIN
,
.
ekey
=
TSKEY_MAX
},
}
else
{
&
(
SVersionRange
){.
minVer
=
0
,
.
maxVer
=
UINT64_MAX
});
taosArrayClear
(
state
->
aBlockL
);
bool
hasVal
=
tMergeTreeNext
(
&
state
->
mergeTree
);
}
if
(
!
hasVal
)
{
state
->
state
=
SFSLASTNEXTROW_FILESET
;
code
=
tsdbReadBlockL
(
state
->
pDataFReader
,
state
->
aBlockL
);
// tMergeTreeClose(&state->mergeTree);
if
(
code
)
goto
_err
;
// SBlockL *pBlockL = (SBlockL *)taosArrayGet(state->aBlockL, state->iBlockL);
state
->
pBlockL
=
taosArraySearch
(
state
->
aBlockL
,
state
->
pBlockIdxExp
,
tCmprBlockL
,
TD_EQ
);
if
(
!
state
->
pBlockL
)
{
goto
_next_fileset
;
goto
_next_fileset
;
}
}
int64_t
suid
=
state
->
pBlockL
->
suid
;
int64_t
uid
=
state
->
pBlockL
->
maxUid
;
if
(
!
state
->
pBlockDataL
)
{
state
->
pBlockDataL
=
&
state
->
blockDataL
;
tBlockDataCreate
(
state
->
pBlockDataL
);
}
code
=
tBlockDataInit
(
state
->
pBlockDataL
,
suid
,
suid
?
0
:
uid
,
state
->
pTSchema
);
if
(
code
)
goto
_err
;
}
case
SFSLASTNEXTROW_BLOCKDATA
:
code
=
tsdbReadLastBlock
(
state
->
pDataFReader
,
state
->
pBlockL
,
state
->
pBlockDataL
);
if
(
code
)
goto
_err
;
state
->
nRow
=
state
->
blockDataL
.
nRow
;
state
->
iRow
=
state
->
nRow
-
1
;
if
(
!
state
->
pBlockDataL
->
uid
)
{
while
(
state
->
pBlockIdxExp
->
uid
!=
state
->
pBlockDataL
->
aUid
[
state
->
iRow
])
{
--
state
->
iRow
;
}
}
state
->
state
=
SFSLASTNEXTROW_BLOCKROW
;
state
->
state
=
SFSLASTNEXTROW_BLOCKROW
;
case
SFSLASTNEXTROW_BLOCKROW
:
if
(
state
->
pBlockDataL
->
uid
)
{
if
(
state
->
iRow
>=
0
)
{
state
->
row
=
tsdbRowFromBlockData
(
state
->
pBlockDataL
,
state
->
iRow
);
*
ppRow
=
&
state
->
row
;
if
(
--
state
->
iRow
<
0
)
{
state
->
state
=
SFSLASTNEXTROW_FILESET
;
}
}
}
case
SFSLASTNEXTROW_BLOCKROW
:
}
else
{
state
->
row
=
tMergeTreeGetRow
(
&
state
->
mergeTree
);
if
(
state
->
iRow
>=
0
&&
state
->
pBlockIdxExp
->
uid
==
state
->
pBlockDataL
->
aUid
[
state
->
iRow
])
{
state
->
row
=
tsdbRowFromBlockData
(
state
->
pBlockDataL
,
state
->
iRow
);
*
ppRow
=
&
state
->
row
;
*
ppRow
=
&
state
->
row
;
bool
hasVal
=
tMergeTreeNext
(
&
state
->
mergeTree
);
if
(
--
state
->
iRow
<
0
||
state
->
pBlockIdxExp
->
uid
!=
state
->
pBlockDataL
->
aUid
[
state
->
iRow
]
)
{
if
(
!
hasVal
)
{
state
->
state
=
SFSLASTNEXTROW_FILESET
;
state
->
state
=
SFSLASTNEXTROW_FILESET
;
}
}
}
}
return
code
;
return
code
;
default:
default:
ASSERT
(
0
);
ASSERT
(
0
);
...
@@ -548,15 +483,6 @@ _err:
...
@@ -548,15 +483,6 @@ _err:
tsdbDataFReaderClose
(
&
state
->
pDataFReader
);
tsdbDataFReaderClose
(
&
state
->
pDataFReader
);
state
->
pDataFReader
=
NULL
;
state
->
pDataFReader
=
NULL
;
}
}
if
(
state
->
aBlockL
)
{
taosArrayDestroy
(
state
->
aBlockL
);
state
->
aBlockL
=
NULL
;
}
if
(
state
->
pBlockDataL
)
{
tBlockDataDestroy
(
state
->
pBlockDataL
,
1
);
state
->
pBlockDataL
=
NULL
;
}
*
ppRow
=
NULL
;
*
ppRow
=
NULL
;
return
code
;
return
code
;
...
@@ -574,14 +500,6 @@ int32_t clearNextRowFromFSLast(void *iter) {
...
@@ -574,14 +500,6 @@ int32_t clearNextRowFromFSLast(void *iter) {
tsdbDataFReaderClose
(
&
state
->
pDataFReader
);
tsdbDataFReaderClose
(
&
state
->
pDataFReader
);
state
->
pDataFReader
=
NULL
;
state
->
pDataFReader
=
NULL
;
}
}
if
(
state
->
aBlockL
)
{
taosArrayDestroy
(
state
->
aBlockL
);
state
->
aBlockL
=
NULL
;
}
if
(
state
->
pBlockDataL
)
{
tBlockDataDestroy
(
state
->
pBlockDataL
,
1
);
state
->
pBlockDataL
=
NULL
;
}
return
code
;
return
code
;
}
}
...
@@ -609,7 +527,7 @@ typedef struct SFSNextRowIter {
...
@@ -609,7 +527,7 @@ typedef struct SFSNextRowIter {
SMapData
blockMap
;
SMapData
blockMap
;
int32_t
nBlock
;
int32_t
nBlock
;
int32_t
iBlock
;
int32_t
iBlock
;
S
Block
block
;
S
DataBlk
block
;
SBlockData
blockData
;
SBlockData
blockData
;
SBlockData
*
pBlockData
;
SBlockData
*
pBlockData
;
int32_t
nRow
;
int32_t
nRow
;
...
@@ -684,13 +602,13 @@ static int32_t getNextRowFromFS(void *iter, TSDBROW **ppRow) {
...
@@ -684,13 +602,13 @@ static int32_t getNextRowFromFS(void *iter, TSDBROW **ppRow) {
}
}
case
SFSNEXTROW_BLOCKDATA
:
case
SFSNEXTROW_BLOCKDATA
:
if
(
state
->
iBlock
>=
0
)
{
if
(
state
->
iBlock
>=
0
)
{
S
Bloc
k
block
=
{
0
};
S
DataBl
k
block
=
{
0
};
t
Bloc
kReset
(
&
block
);
t
DataBl
kReset
(
&
block
);
// tBlockDataReset(&state->blockData);
// tBlockDataReset(&state->blockData);
tBlockDataReset
(
state
->
pBlockData
);
tBlockDataReset
(
state
->
pBlockData
);
tMapDataGetItemByIdx
(
&
state
->
blockMap
,
state
->
iBlock
,
&
block
,
tGet
Bloc
k
);
tMapDataGetItemByIdx
(
&
state
->
blockMap
,
state
->
iBlock
,
&
block
,
tGet
DataBl
k
);
/* code = tsdbReadBlockData(state->pDataFReader, &state->blockIdx, &block, &state->blockData, NULL, NULL); */
/* code = tsdbReadBlockData(state->pDataFReader, &state->blockIdx, &block, &state->blockData, NULL, NULL); */
tBlockDataReset
(
state
->
pBlockData
);
tBlockDataReset
(
state
->
pBlockData
);
code
=
tBlockDataInit
(
state
->
pBlockData
,
state
->
suid
,
state
->
uid
,
state
->
pTSchema
);
code
=
tBlockDataInit
(
state
->
pBlockData
,
state
->
suid
,
state
->
uid
,
state
->
pTSchema
);
...
@@ -972,9 +890,6 @@ static int32_t nextRowIterOpen(CacheNextRowIter *pIter, tb_uid_t uid, STsdb *pTs
...
@@ -972,9 +890,6 @@ static int32_t nextRowIterOpen(CacheNextRowIter *pIter, tb_uid_t uid, STsdb *pTs
pIter
->
fsLastState
.
state
=
(
SFSLASTNEXTROWSTATES
)
SFSNEXTROW_FS
;
pIter
->
fsLastState
.
state
=
(
SFSLASTNEXTROWSTATES
)
SFSNEXTROW_FS
;
pIter
->
fsLastState
.
pTsdb
=
pTsdb
;
pIter
->
fsLastState
.
pTsdb
=
pTsdb
;
pIter
->
fsLastState
.
aDFileSet
=
pIter
->
pReadSnap
->
fs
.
aDFileSet
;
pIter
->
fsLastState
.
aDFileSet
=
pIter
->
pReadSnap
->
fs
.
aDFileSet
;
pIter
->
fsLastState
.
pBlockIdxExp
=
&
pIter
->
idx
;
pIter
->
fsLastState
.
pTSchema
=
pTSchema
;
pIter
->
fsLastState
.
suid
=
suid
;
pIter
->
fsLastState
.
uid
=
uid
;
pIter
->
fsLastState
.
uid
=
uid
;
pIter
->
fsState
.
state
=
SFSNEXTROW_FS
;
pIter
->
fsState
.
state
=
SFSNEXTROW_FS
;
...
@@ -1372,8 +1287,11 @@ int32_t tsdbCacheGetLastH(SLRUCache *pCache, tb_uid_t uid, STsdb *pTsdb, LRUHand
...
@@ -1372,8 +1287,11 @@ int32_t tsdbCacheGetLastH(SLRUCache *pCache, tb_uid_t uid, STsdb *pTsdb, LRUHand
// getTableCacheKeyS(uid, "l", key, &keyLen);
// getTableCacheKeyS(uid, "l", key, &keyLen);
getTableCacheKey
(
uid
,
1
,
key
,
&
keyLen
);
getTableCacheKey
(
uid
,
1
,
key
,
&
keyLen
);
LRUHandle
*
h
=
taosLRUCacheLookup
(
pCache
,
key
,
keyLen
);
LRUHandle
*
h
=
taosLRUCacheLookup
(
pCache
,
key
,
keyLen
);
if
(
h
)
{
if
(
!
h
)
{
}
else
{
taosThreadMutexLock
(
&
pTsdb
->
lruMutex
);
h
=
taosLRUCacheLookup
(
pCache
,
key
,
keyLen
);
if
(
!
h
)
{
SArray
*
pLastArray
=
NULL
;
SArray
*
pLastArray
=
NULL
;
code
=
mergeLast
(
uid
,
pTsdb
,
&
pLastArray
);
code
=
mergeLast
(
uid
,
pTsdb
,
&
pLastArray
);
// if table's empty or error, return code of -1
// if table's empty or error, return code of -1
...
@@ -1384,13 +1302,18 @@ int32_t tsdbCacheGetLastH(SLRUCache *pCache, tb_uid_t uid, STsdb *pTsdb, LRUHand
...
@@ -1384,13 +1302,18 @@ int32_t tsdbCacheGetLastH(SLRUCache *pCache, tb_uid_t uid, STsdb *pTsdb, LRUHand
}
}
_taos_lru_deleter_t
deleter
=
deleteTableCacheLast
;
_taos_lru_deleter_t
deleter
=
deleteTableCacheLast
;
LRUStatus
status
=
LRUStatus
status
=
taosLRUCacheInsert
(
pCache
,
key
,
keyLen
,
pLastArray
,
pLastArray
->
capacity
,
deleter
,
NULL
,
taosLRUCacheInsert
(
pCache
,
key
,
keyLen
,
pLastArray
,
pLastArray
->
capacity
,
deleter
,
NULL
,
TAOS_LRU_PRIORITY_LOW
);
TAOS_LRU_PRIORITY_LOW
);
if
(
status
!=
TAOS_LRU_STATUS_OK
)
{
if
(
status
!=
TAOS_LRU_STATUS_OK
)
{
code
=
-
1
;
code
=
-
1
;
}
}
taosThreadMutexUnlock
(
&
pTsdb
->
lruMutex
);
h
=
taosLRUCacheLookup
(
pCache
,
key
,
keyLen
);
h
=
taosLRUCacheLookup
(
pCache
,
key
,
keyLen
);
}
else
{
taosThreadMutexUnlock
(
&
pTsdb
->
lruMutex
);
}
}
}
*
handle
=
h
;
*
handle
=
h
;
...
@@ -1411,3 +1334,5 @@ void tsdbCacheSetCapacity(SVnode *pVnode, size_t capacity) {
...
@@ -1411,3 +1334,5 @@ void tsdbCacheSetCapacity(SVnode *pVnode, size_t capacity) {
}
}
size_t
tsdbCacheGetCapacity
(
SVnode
*
pVnode
)
{
return
taosLRUCacheGetCapacity
(
pVnode
->
pTsdb
->
lruCache
);
}
size_t
tsdbCacheGetCapacity
(
SVnode
*
pVnode
)
{
return
taosLRUCacheGetCapacity
(
pVnode
->
pTsdb
->
lruCache
);
}
size_t
tsdbCacheGetUsage
(
SVnode
*
pVnode
)
{
return
taosLRUCacheGetUsage
(
pVnode
->
pTsdb
->
lruCache
);
}
source/dnode/vnode/src/tsdb/tsdbCommit.c
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
source/dnode/vnode/src/tsdb/tsdbCompact.c
0 → 100644
浏览文件 @
38b62ddd
/*
* 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"
typedef
struct
{
STsdb
*
pTsdb
;
STsdbFS
fs
;
}
STsdbCompactor
;
int32_t
tsdbCompact
(
STsdb
*
pTsdb
)
{
int32_t
code
=
0
;
// TODO
return
code
;
}
source/dnode/vnode/src/tsdb/tsdbCompress.c
0 → 100644
浏览文件 @
38b62ddd
/*
* 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"
// Integer =====================================================
typedef
struct
{
int8_t
rawCopy
;
int64_t
prevVal
;
int32_t
nVal
;
int32_t
nBuf
;
uint8_t
*
pBuf
;
}
SIntCompressor
;
#define I64_SAFE_ADD(a, b) (((a) >= 0 && (b) <= INT64_MAX - (b)) || ((a) < 0 && (b) >= INT64_MIN - (a)))
#define SIMPLE8B_MAX ((uint64_t)1152921504606846974LL)
static
int32_t
tsdbCmprI64
(
SIntCompressor
*
pCompressor
,
int64_t
val
)
{
int32_t
code
=
0
;
// raw copy
if
(
pCompressor
->
rawCopy
)
{
memcpy
(
pCompressor
->
pBuf
+
pCompressor
->
nBuf
,
&
val
,
sizeof
(
val
));
pCompressor
->
nBuf
+=
sizeof
(
val
);
pCompressor
->
nVal
++
;
goto
_exit
;
}
if
(
!
I64_SAFE_ADD
(
val
,
pCompressor
->
prevVal
))
{
pCompressor
->
rawCopy
=
1
;
// TODO: decompress and copy
pCompressor
->
nVal
++
;
goto
_exit
;
}
int64_t
diff
=
val
-
pCompressor
->
prevVal
;
uint8_t
zigzag
=
ZIGZAGE
(
int64_t
,
diff
);
if
(
zigzag
>=
SIMPLE8B_MAX
)
{
pCompressor
->
rawCopy
=
1
;
// TODO: decompress and copy
pCompressor
->
nVal
++
;
goto
_exit
;
}
_exit:
return
code
;
}
// Timestamp =====================================================
// Float =====================================================
\ No newline at end of file
source/dnode/vnode/src/tsdb/tsdbDiskData.c
0 → 100644
浏览文件 @
38b62ddd
/*
* 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"
typedef
struct
SDiskColBuilder
SDiskColBuilder
;
struct
SDiskColBuilder
{
uint8_t
flags
;
uint8_t
*
pBitMap
;
int32_t
*
aOffset
;
int32_t
nData
;
uint8_t
*
pData
;
};
int32_t
tDiskColAddVal
(
SDiskColBuilder
*
pBuilder
,
SColVal
*
pColVal
)
{
int32_t
code
=
0
;
// TODO
return
code
;
}
// ================================================================
typedef
struct
SDiskDataBuilder
SDiskDataBuilder
;
struct
SDiskDataBuilder
{
SDiskDataHdr
hdr
;
SArray
*
aBlockCol
;
// SArray<SBlockCol>
};
int32_t
tDiskDataBuilderCreate
(
SDiskDataBuilder
**
ppBuilder
)
{
int32_t
code
=
0
;
// TODO
return
code
;
}
void
tDiskDataBuilderDestroy
(
SDiskDataBuilder
*
pBuilder
)
{
// TODO
}
void
tDiskDataBuilderInit
(
SDiskDataBuilder
*
pBuilder
,
int64_t
suid
,
int64_t
uid
,
STSchema
*
pTSchema
,
int8_t
cmprAlg
)
{
pBuilder
->
hdr
=
(
SDiskDataHdr
){.
delimiter
=
TSDB_FILE_DLMT
,
//
.
fmtVer
=
0
,
.
suid
=
suid
,
.
uid
=
uid
,
.
cmprAlg
=
cmprAlg
};
}
void
tDiskDataBuilderReset
(
SDiskDataBuilder
*
pBuilder
)
{
// TODO
}
int32_t
tDiskDataBuilderAddRow
(
SDiskDataBuilder
*
pBuilder
,
TSDBROW
*
pRow
,
STSchema
*
pTSchema
,
int64_t
uid
)
{
int32_t
code
=
0
;
// uid (todo)
// version (todo)
// TSKEY (todo)
SRowIter
iter
=
{
0
};
tRowIterInit
(
&
iter
,
pRow
,
pTSchema
);
for
(
int32_t
iDiskCol
=
0
;
iDiskCol
<
0
;
iDiskCol
++
)
{
}
return
code
;
}
int32_t
tDiskDataBuilderGet
(
SDiskDataBuilder
*
pBuilder
,
uint8_t
**
ppData
)
{
int32_t
code
=
0
;
// TODO
return
code
;
}
\ No newline at end of file
source/dnode/vnode/src/tsdb/tsdbFS.c
浏览文件 @
38b62ddd
...
@@ -110,7 +110,7 @@ _err:
...
@@ -110,7 +110,7 @@ _err:
// taosRemoveFile(fname);
// taosRemoveFile(fname);
// }
// }
// //
la
st
// //
s
st
// if (isSameDisk && pFrom->pLastF->commitID == pTo->pLastF->commitID) {
// if (isSameDisk && pFrom->pLastF->commitID == pTo->pLastF->commitID) {
// if (pFrom->pLastF->size > pTo->pLastF->size) {
// if (pFrom->pLastF->size > pTo->pLastF->size) {
// code = tsdbDFileRollback(pFS->pTsdb, pTo, TSDB_LAST_FILE);
// code = tsdbDFileRollback(pFS->pTsdb, pTo, TSDB_LAST_FILE);
...
@@ -140,7 +140,7 @@ _err:
...
@@ -140,7 +140,7 @@ _err:
// tsdbDataFileName(pFS->pTsdb, pFrom->diskId, pFrom->fid, pFrom->pDataF, fname);
// tsdbDataFileName(pFS->pTsdb, pFrom->diskId, pFrom->fid, pFrom->pDataF, fname);
// taosRemoveFile(fname);
// taosRemoveFile(fname);
// //
la
st
// //
s
st
// tsdbLastFileName(pFS->pTsdb, pFrom->diskId, pFrom->fid, pFrom->pLastF, fname);
// tsdbLastFileName(pFS->pTsdb, pFrom->diskId, pFrom->fid, pFrom->pLastF, fname);
// taosRemoveFile(fname);
// taosRemoveFile(fname);
...
@@ -254,8 +254,10 @@ void tsdbFSDestroy(STsdbFS *pFS) {
...
@@ -254,8 +254,10 @@ void tsdbFSDestroy(STsdbFS *pFS) {
SDFileSet
*
pSet
=
(
SDFileSet
*
)
taosArrayGet
(
pFS
->
aDFileSet
,
iSet
);
SDFileSet
*
pSet
=
(
SDFileSet
*
)
taosArrayGet
(
pFS
->
aDFileSet
,
iSet
);
taosMemoryFree
(
pSet
->
pHeadF
);
taosMemoryFree
(
pSet
->
pHeadF
);
taosMemoryFree
(
pSet
->
pDataF
);
taosMemoryFree
(
pSet
->
pDataF
);
taosMemoryFree
(
pSet
->
pLastF
);
taosMemoryFree
(
pSet
->
pSmaF
);
taosMemoryFree
(
pSet
->
pSmaF
);
for
(
int32_t
iSst
=
0
;
iSst
<
pSet
->
nSstF
;
iSst
++
)
{
taosMemoryFree
(
pSet
->
aSstF
[
iSst
]);
}
}
}
taosArrayDestroy
(
pFS
->
aDFileSet
);
taosArrayDestroy
(
pFS
->
aDFileSet
);
...
@@ -309,29 +311,31 @@ static int32_t tsdbScanAndTryFixFS(STsdb *pTsdb) {
...
@@ -309,29 +311,31 @@ static int32_t tsdbScanAndTryFixFS(STsdb *pTsdb) {
if
(
code
)
goto
_err
;
if
(
code
)
goto
_err
;
}
}
//
last
===========
//
sma ==
===========
tsdb
LastFileName
(
pTsdb
,
pSet
->
diskId
,
pSet
->
fid
,
pSet
->
pLast
F
,
fname
);
tsdb
SmaFileName
(
pTsdb
,
pSet
->
diskId
,
pSet
->
fid
,
pSet
->
pSma
F
,
fname
);
if
(
taosStatFile
(
fname
,
&
size
,
NULL
))
{
if
(
taosStatFile
(
fname
,
&
size
,
NULL
))
{
code
=
TAOS_SYSTEM_ERROR
(
errno
);
code
=
TAOS_SYSTEM_ERROR
(
errno
);
goto
_err
;
goto
_err
;
}
}
if
(
size
!=
pSet
->
pLast
F
->
size
)
{
if
(
size
<
pSet
->
pSma
F
->
size
)
{
code
=
TSDB_CODE_FILE_CORRUPTED
;
code
=
TSDB_CODE_FILE_CORRUPTED
;
goto
_err
;
goto
_err
;
}
else
if
(
size
>
pSet
->
pSmaF
->
size
)
{
code
=
tsdbDFileRollback
(
pTsdb
,
pSet
,
TSDB_SMA_FILE
);
if
(
code
)
goto
_err
;
}
}
// sma =============
// sst ===========
tsdbSmaFileName
(
pTsdb
,
pSet
->
diskId
,
pSet
->
fid
,
pSet
->
pSmaF
,
fname
);
for
(
int32_t
iSst
=
0
;
iSst
<
pSet
->
nSstF
;
iSst
++
)
{
tsdbSstFileName
(
pTsdb
,
pSet
->
diskId
,
pSet
->
fid
,
pSet
->
aSstF
[
iSst
],
fname
);
if
(
taosStatFile
(
fname
,
&
size
,
NULL
))
{
if
(
taosStatFile
(
fname
,
&
size
,
NULL
))
{
code
=
TAOS_SYSTEM_ERROR
(
errno
);
code
=
TAOS_SYSTEM_ERROR
(
errno
);
goto
_err
;
goto
_err
;
}
}
if
(
size
<
pSet
->
pSmaF
->
size
)
{
if
(
size
!=
pSet
->
aSstF
[
iSst
]
->
size
)
{
code
=
TSDB_CODE_FILE_CORRUPTED
;
code
=
TSDB_CODE_FILE_CORRUPTED
;
goto
_err
;
goto
_err
;
}
else
if
(
size
>
pSet
->
pSmaF
->
size
)
{
}
code
=
tsdbDFileRollback
(
pTsdb
,
pSet
,
TSDB_SMA_FILE
);
if
(
code
)
goto
_err
;
}
}
}
}
...
@@ -382,41 +386,15 @@ static int32_t tsdbRecoverFS(STsdb *pTsdb, uint8_t *pData, int64_t nData) {
...
@@ -382,41 +386,15 @@ static int32_t tsdbRecoverFS(STsdb *pTsdb, uint8_t *pData, int64_t nData) {
taosArrayClear
(
pTsdb
->
fs
.
aDFileSet
);
taosArrayClear
(
pTsdb
->
fs
.
aDFileSet
);
n
+=
tGetU32v
(
pData
+
n
,
&
nSet
);
n
+=
tGetU32v
(
pData
+
n
,
&
nSet
);
for
(
uint32_t
iSet
=
0
;
iSet
<
nSet
;
iSet
++
)
{
for
(
uint32_t
iSet
=
0
;
iSet
<
nSet
;
iSet
++
)
{
SDFileSet
fSet
;
SDFileSet
fSet
=
{
0
}
;
// head
int32_t
nt
=
tGetDFileSet
(
pData
+
n
,
&
fSet
);
fSet
.
pHeadF
=
(
SHeadFile
*
)
taosMemoryCalloc
(
1
,
sizeof
(
SHeadFile
));
if
(
nt
<
0
)
{
if
(
fSet
.
pHeadF
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
goto
_err
;
}
}
fSet
.
pHeadF
->
nRef
=
1
;
// data
n
+=
nt
;
fSet
.
pDataF
=
(
SDataFile
*
)
taosMemoryCalloc
(
1
,
sizeof
(
SDataFile
));
if
(
fSet
.
pDataF
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
fSet
.
pDataF
->
nRef
=
1
;
// last
fSet
.
pLastF
=
(
SLastFile
*
)
taosMemoryCalloc
(
1
,
sizeof
(
SLastFile
));
if
(
fSet
.
pLastF
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
fSet
.
pLastF
->
nRef
=
1
;
// sma
fSet
.
pSmaF
=
(
SSmaFile
*
)
taosMemoryCalloc
(
1
,
sizeof
(
SSmaFile
));
if
(
fSet
.
pSmaF
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
fSet
.
pSmaF
->
nRef
=
1
;
n
+=
tGetDFileSet
(
pData
+
n
,
&
fSet
);
if
(
taosArrayPush
(
pTsdb
->
fs
.
aDFileSet
,
&
fSet
)
==
NULL
)
{
if
(
taosArrayPush
(
pTsdb
->
fs
.
aDFileSet
,
&
fSet
)
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
...
@@ -532,13 +510,15 @@ int32_t tsdbFSClose(STsdb *pTsdb) {
...
@@ -532,13 +510,15 @@ int32_t tsdbFSClose(STsdb *pTsdb) {
ASSERT
(
pSet
->
pDataF
->
nRef
==
1
);
ASSERT
(
pSet
->
pDataF
->
nRef
==
1
);
taosMemoryFree
(
pSet
->
pDataF
);
taosMemoryFree
(
pSet
->
pDataF
);
// last
ASSERT
(
pSet
->
pLastF
->
nRef
==
1
);
taosMemoryFree
(
pSet
->
pLastF
);
// sma
// sma
ASSERT
(
pSet
->
pSmaF
->
nRef
==
1
);
ASSERT
(
pSet
->
pSmaF
->
nRef
==
1
);
taosMemoryFree
(
pSet
->
pSmaF
);
taosMemoryFree
(
pSet
->
pSmaF
);
// sst
for
(
int32_t
iSst
=
0
;
iSst
<
pSet
->
nSstF
;
iSst
++
)
{
ASSERT
(
pSet
->
aSstF
[
iSst
]
->
nRef
==
1
);
taosMemoryFree
(
pSet
->
aSstF
[
iSst
]);
}
}
}
taosArrayDestroy
(
pTsdb
->
fs
.
aDFileSet
);
taosArrayDestroy
(
pTsdb
->
fs
.
aDFileSet
);
...
@@ -586,21 +566,23 @@ int32_t tsdbFSCopy(STsdb *pTsdb, STsdbFS *pFS) {
...
@@ -586,21 +566,23 @@ int32_t tsdbFSCopy(STsdb *pTsdb, STsdbFS *pFS) {
}
}
*
fSet
.
pDataF
=
*
pSet
->
pDataF
;
*
fSet
.
pDataF
=
*
pSet
->
pDataF
;
//
dat
a
//
sm
a
fSet
.
p
LastF
=
(
SLastFile
*
)
taosMemoryMalloc
(
sizeof
(
SLast
File
));
fSet
.
p
SmaF
=
(
SSmaFile
*
)
taosMemoryMalloc
(
sizeof
(
SSma
File
));
if
(
fSet
.
p
Last
F
==
NULL
)
{
if
(
fSet
.
p
Sma
F
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_exit
;
goto
_exit
;
}
}
*
fSet
.
p
LastF
=
*
pSet
->
pLast
F
;
*
fSet
.
p
SmaF
=
*
pSet
->
pSma
F
;
// last
// sst
fSet
.
pSmaF
=
(
SSmaFile
*
)
taosMemoryMalloc
(
sizeof
(
SSmaFile
));
for
(
fSet
.
nSstF
=
0
;
fSet
.
nSstF
<
pSet
->
nSstF
;
fSet
.
nSstF
++
)
{
if
(
fSet
.
pSmaF
==
NULL
)
{
fSet
.
aSstF
[
fSet
.
nSstF
]
=
(
SSstFile
*
)
taosMemoryMalloc
(
sizeof
(
SSstFile
));
if
(
fSet
.
aSstF
[
fSet
.
nSstF
]
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_exit
;
goto
_exit
;
}
}
*
fSet
.
pSmaF
=
*
pSet
->
pSmaF
;
*
fSet
.
aSstF
[
fSet
.
nSstF
]
=
*
pSet
->
aSstF
[
fSet
.
nSstF
];
}
if
(
taosArrayPush
(
pFS
->
aDFileSet
,
&
fSet
)
==
NULL
)
{
if
(
taosArrayPush
(
pFS
->
aDFileSet
,
&
fSet
)
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
...
@@ -651,14 +633,38 @@ int32_t tsdbFSUpsertFSet(STsdbFS *pFS, SDFileSet *pSet) {
...
@@ -651,14 +633,38 @@ int32_t tsdbFSUpsertFSet(STsdbFS *pFS, SDFileSet *pSet) {
if
(
c
==
0
)
{
if
(
c
==
0
)
{
*
pDFileSet
->
pHeadF
=
*
pSet
->
pHeadF
;
*
pDFileSet
->
pHeadF
=
*
pSet
->
pHeadF
;
*
pDFileSet
->
pDataF
=
*
pSet
->
pDataF
;
*
pDFileSet
->
pDataF
=
*
pSet
->
pDataF
;
*
pDFileSet
->
pLastF
=
*
pSet
->
pLastF
;
*
pDFileSet
->
pSmaF
=
*
pSet
->
pSmaF
;
*
pDFileSet
->
pSmaF
=
*
pSet
->
pSmaF
;
// sst
if
(
pSet
->
nSstF
>
pDFileSet
->
nSstF
)
{
ASSERT
(
pSet
->
nSstF
==
pDFileSet
->
nSstF
+
1
);
pDFileSet
->
aSstF
[
pDFileSet
->
nSstF
]
=
(
SSstFile
*
)
taosMemoryMalloc
(
sizeof
(
SSstFile
));
if
(
pDFileSet
->
aSstF
[
pDFileSet
->
nSstF
]
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_exit
;
goto
_exit
;
}
}
*
pDFileSet
->
aSstF
[
pDFileSet
->
nSstF
]
=
*
pSet
->
aSstF
[
pSet
->
nSstF
-
1
];
pDFileSet
->
nSstF
++
;
}
else
if
(
pSet
->
nSstF
<
pDFileSet
->
nSstF
)
{
ASSERT
(
pSet
->
nSstF
==
1
);
for
(
int32_t
iSst
=
1
;
iSst
<
pDFileSet
->
nSstF
;
iSst
++
)
{
taosMemoryFree
(
pDFileSet
->
aSstF
[
iSst
]);
}
}
SDFileSet
fSet
=
{.
diskId
=
pSet
->
diskId
,
.
fid
=
pSet
->
fid
};
*
pDFileSet
->
aSstF
[
0
]
=
*
pSet
->
aSstF
[
0
];
pDFileSet
->
nSstF
=
1
;
}
else
{
for
(
int32_t
iSst
=
0
;
iSst
<
pSet
->
nSstF
;
iSst
++
)
{
*
pDFileSet
->
aSstF
[
iSst
]
=
*
pSet
->
aSstF
[
iSst
];
}
}
goto
_exit
;
}
}
ASSERT
(
pSet
->
nSstF
==
1
);
SDFileSet
fSet
=
{.
diskId
=
pSet
->
diskId
,
.
fid
=
pSet
->
fid
,
.
nSstF
=
1
};
// head
// head
fSet
.
pHeadF
=
(
SHeadFile
*
)
taosMemoryMalloc
(
sizeof
(
SHeadFile
));
fSet
.
pHeadF
=
(
SHeadFile
*
)
taosMemoryMalloc
(
sizeof
(
SHeadFile
));
...
@@ -676,21 +682,21 @@ int32_t tsdbFSUpsertFSet(STsdbFS *pFS, SDFileSet *pSet) {
...
@@ -676,21 +682,21 @@ int32_t tsdbFSUpsertFSet(STsdbFS *pFS, SDFileSet *pSet) {
}
}
*
fSet
.
pDataF
=
*
pSet
->
pDataF
;
*
fSet
.
pDataF
=
*
pSet
->
pDataF
;
//
dat
a
//
sm
a
fSet
.
p
LastF
=
(
SLastFile
*
)
taosMemoryMalloc
(
sizeof
(
SLast
File
));
fSet
.
p
SmaF
=
(
SSmaFile
*
)
taosMemoryMalloc
(
sizeof
(
SSma
File
));
if
(
fSet
.
p
Last
F
==
NULL
)
{
if
(
fSet
.
p
Sma
F
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_exit
;
goto
_exit
;
}
}
*
fSet
.
p
LastF
=
*
pSet
->
pLast
F
;
*
fSet
.
p
SmaF
=
*
pSet
->
pSma
F
;
//
la
st
//
s
st
fSet
.
pSmaF
=
(
SSmaFile
*
)
taosMemoryMalloc
(
sizeof
(
SSma
File
));
fSet
.
aSstF
[
0
]
=
(
SSstFile
*
)
taosMemoryMalloc
(
sizeof
(
SSst
File
));
if
(
fSet
.
pSmaF
==
NULL
)
{
if
(
fSet
.
aSstF
[
0
]
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_exit
;
goto
_exit
;
}
}
*
fSet
.
pSmaF
=
*
pSet
->
pSmaF
;
*
fSet
.
aSstF
[
0
]
=
*
pSet
->
aSstF
[
0
]
;
if
(
taosArrayInsert
(
pFS
->
aDFileSet
,
idx
,
&
fSet
)
==
NULL
)
{
if
(
taosArrayInsert
(
pFS
->
aDFileSet
,
idx
,
&
fSet
)
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
...
@@ -836,27 +842,6 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
...
@@ -836,27 +842,6 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
pSetOld
->
pDataF
->
size
=
pSetNew
->
pDataF
->
size
;
pSetOld
->
pDataF
->
size
=
pSetNew
->
pDataF
->
size
;
}
}
// last
fSet
.
pLastF
=
pSetOld
->
pLastF
;
if
((
!
sameDisk
)
||
(
pSetOld
->
pLastF
->
commitID
!=
pSetNew
->
pLastF
->
commitID
))
{
pSetOld
->
pLastF
=
(
SLastFile
*
)
taosMemoryMalloc
(
sizeof
(
SLastFile
));
if
(
pSetOld
->
pLastF
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
*
pSetOld
->
pLastF
=
*
pSetNew
->
pLastF
;
pSetOld
->
pLastF
->
nRef
=
1
;
nRef
=
atomic_sub_fetch_32
(
&
fSet
.
pLastF
->
nRef
,
1
);
if
(
nRef
==
0
)
{
tsdbLastFileName
(
pTsdb
,
pSetOld
->
diskId
,
pSetOld
->
fid
,
fSet
.
pLastF
,
fname
);
taosRemoveFile
(
fname
);
taosMemoryFree
(
fSet
.
pLastF
);
}
}
else
{
ASSERT
(
pSetOld
->
pLastF
->
size
==
pSetNew
->
pLastF
->
size
);
}
// sma
// sma
fSet
.
pSmaF
=
pSetOld
->
pSmaF
;
fSet
.
pSmaF
=
pSetOld
->
pSmaF
;
if
((
!
sameDisk
)
||
(
pSetOld
->
pSmaF
->
commitID
!=
pSetNew
->
pSmaF
->
commitID
))
{
if
((
!
sameDisk
)
||
(
pSetOld
->
pSmaF
->
commitID
!=
pSetNew
->
pSmaF
->
commitID
))
{
...
@@ -879,6 +864,84 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
...
@@ -879,6 +864,84 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
pSetOld
->
pSmaF
->
size
=
pSetNew
->
pSmaF
->
size
;
pSetOld
->
pSmaF
->
size
=
pSetNew
->
pSmaF
->
size
;
}
}
// sst
if
(
sameDisk
)
{
if
(
pSetNew
->
nSstF
>
pSetOld
->
nSstF
)
{
ASSERT
(
pSetNew
->
nSstF
=
pSetOld
->
nSstF
+
1
);
pSetOld
->
aSstF
[
pSetOld
->
nSstF
]
=
(
SSstFile
*
)
taosMemoryMalloc
(
sizeof
(
SSstFile
));
if
(
pSetOld
->
aSstF
[
pSetOld
->
nSstF
]
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
*
pSetOld
->
aSstF
[
pSetOld
->
nSstF
]
=
*
pSetNew
->
aSstF
[
pSetOld
->
nSstF
];
pSetOld
->
aSstF
[
pSetOld
->
nSstF
]
->
nRef
=
1
;
pSetOld
->
nSstF
++
;
}
else
if
(
pSetNew
->
nSstF
<
pSetOld
->
nSstF
)
{
ASSERT
(
pSetNew
->
nSstF
==
1
);
for
(
int32_t
iSst
=
0
;
iSst
<
pSetOld
->
nSstF
;
iSst
++
)
{
SSstFile
*
pSstFile
=
pSetOld
->
aSstF
[
iSst
];
nRef
=
atomic_sub_fetch_32
(
&
pSstFile
->
nRef
,
1
);
if
(
nRef
==
0
)
{
tsdbSstFileName
(
pTsdb
,
pSetOld
->
diskId
,
pSetOld
->
fid
,
pSstFile
,
fname
);
taosRemoveFile
(
fname
);
taosMemoryFree
(
pSstFile
);
}
pSetOld
->
aSstF
[
iSst
]
=
NULL
;
}
pSetOld
->
nSstF
=
1
;
pSetOld
->
aSstF
[
0
]
=
(
SSstFile
*
)
taosMemoryMalloc
(
sizeof
(
SSstFile
));
if
(
pSetOld
->
aSstF
[
0
]
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
*
pSetOld
->
aSstF
[
0
]
=
*
pSetNew
->
aSstF
[
0
];
pSetOld
->
aSstF
[
0
]
->
nRef
=
1
;
}
else
{
for
(
int32_t
iSst
=
0
;
iSst
<
pSetOld
->
nSstF
;
iSst
++
)
{
if
(
pSetOld
->
aSstF
[
iSst
]
->
commitID
!=
pSetNew
->
aSstF
[
iSst
]
->
commitID
)
{
SSstFile
*
pSstFile
=
pSetOld
->
aSstF
[
iSst
];
nRef
=
atomic_sub_fetch_32
(
&
pSstFile
->
nRef
,
1
);
if
(
nRef
==
0
)
{
tsdbSstFileName
(
pTsdb
,
pSetOld
->
diskId
,
pSetOld
->
fid
,
pSstFile
,
fname
);
taosRemoveFile
(
fname
);
taosMemoryFree
(
pSstFile
);
}
pSetOld
->
aSstF
[
iSst
]
=
(
SSstFile
*
)
taosMemoryMalloc
(
sizeof
(
SSstFile
));
if
(
pSetOld
->
aSstF
[
iSst
]
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
*
pSetOld
->
aSstF
[
iSst
]
=
*
pSetNew
->
aSstF
[
iSst
];
pSetOld
->
aSstF
[
iSst
]
->
nRef
=
1
;
}
else
{
ASSERT
(
pSetOld
->
aSstF
[
iSst
]
->
size
==
pSetOld
->
aSstF
[
iSst
]
->
size
);
ASSERT
(
pSetOld
->
aSstF
[
iSst
]
->
offset
==
pSetOld
->
aSstF
[
iSst
]
->
offset
);
}
}
}
}
else
{
ASSERT
(
pSetOld
->
nSstF
==
pSetNew
->
nSstF
);
for
(
int32_t
iSst
=
0
;
iSst
<
pSetOld
->
nSstF
;
iSst
++
)
{
SSstFile
*
pSstFile
=
pSetOld
->
aSstF
[
iSst
];
nRef
=
atomic_sub_fetch_32
(
&
pSstFile
->
nRef
,
1
);
if
(
nRef
==
0
)
{
tsdbSstFileName
(
pTsdb
,
pSetOld
->
diskId
,
pSetOld
->
fid
,
pSstFile
,
fname
);
taosRemoveFile
(
fname
);
taosMemoryFree
(
pSstFile
);
}
pSetOld
->
aSstF
[
iSst
]
=
(
SSstFile
*
)
taosMemoryMalloc
(
sizeof
(
SSstFile
));
if
(
pSetOld
->
aSstF
[
iSst
]
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
*
pSetOld
->
aSstF
[
iSst
]
=
*
pSetNew
->
aSstF
[
iSst
];
pSetOld
->
aSstF
[
iSst
]
->
nRef
=
1
;
}
}
if
(
!
sameDisk
)
{
if
(
!
sameDisk
)
{
pSetOld
->
diskId
=
pSetNew
->
diskId
;
pSetOld
->
diskId
=
pSetNew
->
diskId
;
}
}
...
@@ -902,26 +965,27 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
...
@@ -902,26 +965,27 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
taosMemoryFree
(
pSetOld
->
pDataF
);
taosMemoryFree
(
pSetOld
->
pDataF
);
}
}
nRef
=
atomic_sub_fetch_32
(
&
pSetOld
->
p
Last
F
->
nRef
,
1
);
nRef
=
atomic_sub_fetch_32
(
&
pSetOld
->
p
Sma
F
->
nRef
,
1
);
if
(
nRef
==
0
)
{
if
(
nRef
==
0
)
{
tsdb
LastFileName
(
pTsdb
,
pSetOld
->
diskId
,
pSetOld
->
fid
,
pSetOld
->
pLast
F
,
fname
);
tsdb
SmaFileName
(
pTsdb
,
pSetOld
->
diskId
,
pSetOld
->
fid
,
pSetOld
->
pSma
F
,
fname
);
taosRemoveFile
(
fname
);
taosRemoveFile
(
fname
);
taosMemoryFree
(
pSetOld
->
p
Last
F
);
taosMemoryFree
(
pSetOld
->
p
Sma
F
);
}
}
nRef
=
atomic_sub_fetch_32
(
&
pSetOld
->
pSmaF
->
nRef
,
1
);
for
(
int8_t
iSst
=
0
;
iSst
<
pSetOld
->
nSstF
;
iSst
++
)
{
nRef
=
atomic_sub_fetch_32
(
&
pSetOld
->
aSstF
[
iSst
]
->
nRef
,
1
);
if
(
nRef
==
0
)
{
if
(
nRef
==
0
)
{
tsdbSmaFileName
(
pTsdb
,
pSetOld
->
diskId
,
pSetOld
->
fid
,
pSetOld
->
pSmaF
,
fname
);
tsdbSstFileName
(
pTsdb
,
pSetOld
->
diskId
,
pSetOld
->
fid
,
pSetOld
->
aSstF
[
iSst
]
,
fname
);
taosRemoveFile
(
fname
);
taosRemoveFile
(
fname
);
taosMemoryFree
(
pSetOld
->
pSmaF
);
taosMemoryFree
(
pSetOld
->
aSstF
[
iSst
]);
}
}
}
taosArrayRemove
(
pTsdb
->
fs
.
aDFileSet
,
iOld
);
taosArrayRemove
(
pTsdb
->
fs
.
aDFileSet
,
iOld
);
continue
;
continue
;
_add_new:
_add_new:
fSet
.
diskId
=
pSetNew
->
diskId
;
fSet
=
(
SDFileSet
){.
diskId
=
pSetNew
->
diskId
,
.
fid
=
pSetNew
->
fid
,
.
nSstF
=
1
};
fSet
.
fid
=
pSetNew
->
fid
;
// head
// head
fSet
.
pHeadF
=
(
SHeadFile
*
)
taosMemoryMalloc
(
sizeof
(
SHeadFile
));
fSet
.
pHeadF
=
(
SHeadFile
*
)
taosMemoryMalloc
(
sizeof
(
SHeadFile
));
...
@@ -941,15 +1005,6 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
...
@@ -941,15 +1005,6 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
*
fSet
.
pDataF
=
*
pSetNew
->
pDataF
;
*
fSet
.
pDataF
=
*
pSetNew
->
pDataF
;
fSet
.
pDataF
->
nRef
=
1
;
fSet
.
pDataF
->
nRef
=
1
;
// last
fSet
.
pLastF
=
(
SLastFile
*
)
taosMemoryMalloc
(
sizeof
(
SLastFile
));
if
(
fSet
.
pLastF
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
*
fSet
.
pLastF
=
*
pSetNew
->
pLastF
;
fSet
.
pLastF
->
nRef
=
1
;
// sma
// sma
fSet
.
pSmaF
=
(
SSmaFile
*
)
taosMemoryMalloc
(
sizeof
(
SSmaFile
));
fSet
.
pSmaF
=
(
SSmaFile
*
)
taosMemoryMalloc
(
sizeof
(
SSmaFile
));
if
(
fSet
.
pSmaF
==
NULL
)
{
if
(
fSet
.
pSmaF
==
NULL
)
{
...
@@ -959,6 +1014,16 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
...
@@ -959,6 +1014,16 @@ int32_t tsdbFSCommit2(STsdb *pTsdb, STsdbFS *pFSNew) {
*
fSet
.
pSmaF
=
*
pSetNew
->
pSmaF
;
*
fSet
.
pSmaF
=
*
pSetNew
->
pSmaF
;
fSet
.
pSmaF
->
nRef
=
1
;
fSet
.
pSmaF
->
nRef
=
1
;
// sst
ASSERT
(
pSetNew
->
nSstF
==
1
);
fSet
.
aSstF
[
0
]
=
(
SSstFile
*
)
taosMemoryMalloc
(
sizeof
(
SSstFile
));
if
(
fSet
.
aSstF
[
0
]
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
}
*
fSet
.
aSstF
[
0
]
=
*
pSetNew
->
aSstF
[
0
];
fSet
.
aSstF
[
0
]
->
nRef
=
1
;
if
(
taosArrayInsert
(
pTsdb
->
fs
.
aDFileSet
,
iOld
,
&
fSet
)
==
NULL
)
{
if
(
taosArrayInsert
(
pTsdb
->
fs
.
aDFileSet
,
iOld
,
&
fSet
)
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_err
;
goto
_err
;
...
@@ -1002,11 +1067,13 @@ int32_t tsdbFSRef(STsdb *pTsdb, STsdbFS *pFS) {
...
@@ -1002,11 +1067,13 @@ int32_t tsdbFSRef(STsdb *pTsdb, STsdbFS *pFS) {
nRef
=
atomic_fetch_add_32
(
&
pSet
->
pDataF
->
nRef
,
1
);
nRef
=
atomic_fetch_add_32
(
&
pSet
->
pDataF
->
nRef
,
1
);
ASSERT
(
nRef
>
0
);
ASSERT
(
nRef
>
0
);
nRef
=
atomic_fetch_add_32
(
&
pSet
->
p
Last
F
->
nRef
,
1
);
nRef
=
atomic_fetch_add_32
(
&
pSet
->
p
Sma
F
->
nRef
,
1
);
ASSERT
(
nRef
>
0
);
ASSERT
(
nRef
>
0
);
nRef
=
atomic_fetch_add_32
(
&
pSet
->
pSmaF
->
nRef
,
1
);
for
(
int32_t
iSst
=
0
;
iSst
<
pSet
->
nSstF
;
iSst
++
)
{
nRef
=
atomic_fetch_add_32
(
&
pSet
->
aSstF
[
iSst
]
->
nRef
,
1
);
ASSERT
(
nRef
>
0
);
ASSERT
(
nRef
>
0
);
}
if
(
taosArrayPush
(
pFS
->
aDFileSet
,
&
fSet
)
==
NULL
)
{
if
(
taosArrayPush
(
pFS
->
aDFileSet
,
&
fSet
)
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
code
=
TSDB_CODE_OUT_OF_MEMORY
;
...
@@ -1053,22 +1120,25 @@ void tsdbFSUnref(STsdb *pTsdb, STsdbFS *pFS) {
...
@@ -1053,22 +1120,25 @@ void tsdbFSUnref(STsdb *pTsdb, STsdbFS *pFS) {
taosMemoryFree
(
pSet
->
pDataF
);
taosMemoryFree
(
pSet
->
pDataF
);
}
}
//
last
//
sma
nRef
=
atomic_sub_fetch_32
(
&
pSet
->
p
Last
F
->
nRef
,
1
);
nRef
=
atomic_sub_fetch_32
(
&
pSet
->
p
Sma
F
->
nRef
,
1
);
ASSERT
(
nRef
>=
0
);
ASSERT
(
nRef
>=
0
);
if
(
nRef
==
0
)
{
if
(
nRef
==
0
)
{
tsdb
LastFileName
(
pTsdb
,
pSet
->
diskId
,
pSet
->
fid
,
pSet
->
pLast
F
,
fname
);
tsdb
SmaFileName
(
pTsdb
,
pSet
->
diskId
,
pSet
->
fid
,
pSet
->
pSma
F
,
fname
);
taosRemoveFile
(
fname
);
taosRemoveFile
(
fname
);
taosMemoryFree
(
pSet
->
p
Last
F
);
taosMemoryFree
(
pSet
->
p
Sma
F
);
}
}
// sma
// sst
nRef
=
atomic_sub_fetch_32
(
&
pSet
->
pSmaF
->
nRef
,
1
);
for
(
int32_t
iSst
=
0
;
iSst
<
pSet
->
nSstF
;
iSst
++
)
{
nRef
=
atomic_sub_fetch_32
(
&
pSet
->
aSstF
[
iSst
]
->
nRef
,
1
);
ASSERT
(
nRef
>=
0
);
ASSERT
(
nRef
>=
0
);
if
(
nRef
==
0
)
{
if
(
nRef
==
0
)
{
tsdbSmaFileName
(
pTsdb
,
pSet
->
diskId
,
pSet
->
fid
,
pSet
->
pSmaF
,
fname
);
tsdbSstFileName
(
pTsdb
,
pSet
->
diskId
,
pSet
->
fid
,
pSet
->
aSstF
[
iSst
]
,
fname
);
taosRemoveFile
(
fname
);
taosRemoveFile
(
fname
);
taosMemoryFree
(
pSet
->
pSmaF
);
taosMemoryFree
(
pSet
->
aSstF
[
iSst
]);
/* code */
}
}
}
}
}
...
...
source/dnode/vnode/src/tsdb/tsdbFile.c
浏览文件 @
38b62ddd
...
@@ -53,22 +53,22 @@ static int32_t tGetDataFile(uint8_t *p, SDataFile *pDataFile) {
...
@@ -53,22 +53,22 @@ static int32_t tGetDataFile(uint8_t *p, SDataFile *pDataFile) {
return
n
;
return
n
;
}
}
int32_t
tPut
LastFile
(
uint8_t
*
p
,
SLastFile
*
pLa
stFile
)
{
int32_t
tPut
SstFile
(
uint8_t
*
p
,
SSstFile
*
pS
stFile
)
{
int32_t
n
=
0
;
int32_t
n
=
0
;
n
+=
tPutI64v
(
p
?
p
+
n
:
p
,
p
La
stFile
->
commitID
);
n
+=
tPutI64v
(
p
?
p
+
n
:
p
,
p
S
stFile
->
commitID
);
n
+=
tPutI64v
(
p
?
p
+
n
:
p
,
p
La
stFile
->
size
);
n
+=
tPutI64v
(
p
?
p
+
n
:
p
,
p
S
stFile
->
size
);
n
+=
tPutI64v
(
p
?
p
+
n
:
p
,
p
La
stFile
->
offset
);
n
+=
tPutI64v
(
p
?
p
+
n
:
p
,
p
S
stFile
->
offset
);
return
n
;
return
n
;
}
}
static
int32_t
tGet
LastFile
(
uint8_t
*
p
,
SLastFile
*
pLa
stFile
)
{
static
int32_t
tGet
SstFile
(
uint8_t
*
p
,
SSstFile
*
pS
stFile
)
{
int32_t
n
=
0
;
int32_t
n
=
0
;
n
+=
tGetI64v
(
p
+
n
,
&
p
La
stFile
->
commitID
);
n
+=
tGetI64v
(
p
+
n
,
&
p
S
stFile
->
commitID
);
n
+=
tGetI64v
(
p
+
n
,
&
p
La
stFile
->
size
);
n
+=
tGetI64v
(
p
+
n
,
&
p
S
stFile
->
size
);
n
+=
tGetI64v
(
p
+
n
,
&
p
La
stFile
->
offset
);
n
+=
tGetI64v
(
p
+
n
,
&
p
S
stFile
->
offset
);
return
n
;
return
n
;
}
}
...
@@ -102,9 +102,9 @@ void tsdbDataFileName(STsdb *pTsdb, SDiskID did, int32_t fid, SDataFile *pDataF,
...
@@ -102,9 +102,9 @@ void tsdbDataFileName(STsdb *pTsdb, SDiskID did, int32_t fid, SDataFile *pDataF,
TD_DIRSEP
,
pTsdb
->
path
,
TD_DIRSEP
,
TD_VID
(
pTsdb
->
pVnode
),
fid
,
pDataF
->
commitID
,
".data"
);
TD_DIRSEP
,
pTsdb
->
path
,
TD_DIRSEP
,
TD_VID
(
pTsdb
->
pVnode
),
fid
,
pDataF
->
commitID
,
".data"
);
}
}
void
tsdb
LastFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SLastFile
*
pLa
stF
,
char
fname
[])
{
void
tsdb
SstFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SSstFile
*
pS
stF
,
char
fname
[])
{
snprintf
(
fname
,
TSDB_FILENAME_LEN
-
1
,
"%s%s%s%sv%df%dver%"
PRId64
"%s"
,
tfsGetDiskPath
(
pTsdb
->
pVnode
->
pTfs
,
did
),
snprintf
(
fname
,
TSDB_FILENAME_LEN
-
1
,
"%s%s%s%sv%df%dver%"
PRId64
"%s"
,
tfsGetDiskPath
(
pTsdb
->
pVnode
->
pTfs
,
did
),
TD_DIRSEP
,
pTsdb
->
path
,
TD_DIRSEP
,
TD_VID
(
pTsdb
->
pVnode
),
fid
,
p
LastF
->
commitID
,
".la
st"
);
TD_DIRSEP
,
pTsdb
->
path
,
TD_DIRSEP
,
TD_VID
(
pTsdb
->
pVnode
),
fid
,
p
SstF
->
commitID
,
".s
st"
);
}
}
void
tsdbSmaFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SSmaFile
*
pSmaF
,
char
fname
[])
{
void
tsdbSmaFileName
(
STsdb
*
pTsdb
,
SDiskID
did
,
int32_t
fid
,
SSmaFile
*
pSmaF
,
char
fname
[])
{
...
@@ -194,9 +194,11 @@ int32_t tPutDFileSet(uint8_t *p, SDFileSet *pSet) {
...
@@ -194,9 +194,11 @@ int32_t tPutDFileSet(uint8_t *p, SDFileSet *pSet) {
n
+=
tPutDataFile
(
p
?
p
+
n
:
p
,
pSet
->
pDataF
);
n
+=
tPutDataFile
(
p
?
p
+
n
:
p
,
pSet
->
pDataF
);
n
+=
tPutSmaFile
(
p
?
p
+
n
:
p
,
pSet
->
pSmaF
);
n
+=
tPutSmaFile
(
p
?
p
+
n
:
p
,
pSet
->
pSmaF
);
// last
// sst
n
+=
tPutU8
(
p
?
p
+
n
:
p
,
1
);
// for future compatibility
n
+=
tPutU8
(
p
?
p
+
n
:
p
,
pSet
->
nSstF
);
n
+=
tPutLastFile
(
p
?
p
+
n
:
p
,
pSet
->
pLastF
);
for
(
int32_t
iSst
=
0
;
iSst
<
pSet
->
nSstF
;
iSst
++
)
{
n
+=
tPutSstFile
(
p
?
p
+
n
:
p
,
pSet
->
aSstF
[
iSst
]);
}
return
n
;
return
n
;
}
}
...
@@ -208,15 +210,40 @@ int32_t tGetDFileSet(uint8_t *p, SDFileSet *pSet) {
...
@@ -208,15 +210,40 @@ int32_t tGetDFileSet(uint8_t *p, SDFileSet *pSet) {
n
+=
tGetI32v
(
p
+
n
,
&
pSet
->
diskId
.
id
);
n
+=
tGetI32v
(
p
+
n
,
&
pSet
->
diskId
.
id
);
n
+=
tGetI32v
(
p
+
n
,
&
pSet
->
fid
);
n
+=
tGetI32v
(
p
+
n
,
&
pSet
->
fid
);
// data
// head
pSet
->
pHeadF
=
(
SHeadFile
*
)
taosMemoryCalloc
(
1
,
sizeof
(
SHeadFile
));
if
(
pSet
->
pHeadF
==
NULL
)
{
return
-
1
;
}
pSet
->
pHeadF
->
nRef
=
1
;
n
+=
tGetHeadFile
(
p
+
n
,
pSet
->
pHeadF
);
n
+=
tGetHeadFile
(
p
+
n
,
pSet
->
pHeadF
);
// data
pSet
->
pDataF
=
(
SDataFile
*
)
taosMemoryCalloc
(
1
,
sizeof
(
SDataFile
));
if
(
pSet
->
pDataF
==
NULL
)
{
return
-
1
;
}
pSet
->
pDataF
->
nRef
=
1
;
n
+=
tGetDataFile
(
p
+
n
,
pSet
->
pDataF
);
n
+=
tGetDataFile
(
p
+
n
,
pSet
->
pDataF
);
// sma
pSet
->
pSmaF
=
(
SSmaFile
*
)
taosMemoryCalloc
(
1
,
sizeof
(
SSmaFile
));
if
(
pSet
->
pSmaF
==
NULL
)
{
return
-
1
;
}
pSet
->
pSmaF
->
nRef
=
1
;
n
+=
tGetSmaFile
(
p
+
n
,
pSet
->
pSmaF
);
n
+=
tGetSmaFile
(
p
+
n
,
pSet
->
pSmaF
);
// last
// sst
uint8_t
nLast
;
n
+=
tGetU8
(
p
+
n
,
&
pSet
->
nSstF
);
n
+=
tGetU8
(
p
+
n
,
&
nLast
);
for
(
int32_t
iSst
=
0
;
iSst
<
pSet
->
nSstF
;
iSst
++
)
{
n
+=
tGetLastFile
(
p
+
n
,
pSet
->
pLastF
);
pSet
->
aSstF
[
iSst
]
=
(
SSstFile
*
)
taosMemoryCalloc
(
1
,
sizeof
(
SSstFile
));
if
(
pSet
->
aSstF
[
iSst
]
==
NULL
)
{
return
-
1
;
}
pSet
->
aSstF
[
iSst
]
->
nRef
=
1
;
n
+=
tGetSstFile
(
p
+
n
,
pSet
->
aSstF
[
iSst
]);
}
return
n
;
return
n
;
}
}
...
...
source/dnode/vnode/src/tsdb/tsdbMergeTree.c
0 → 100644
浏览文件 @
38b62ddd
/*
* 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"
// SLDataIter =================================================
typedef
struct
SLDataIter
{
SRBTreeNode
node
;
SSstBlk
*
pSstBlk
;
SDataFReader
*
pReader
;
int32_t
iSst
;
int8_t
backward
;
SArray
*
aSstBlk
;
int32_t
iSstBlk
;
SBlockData
bData
[
2
];
int32_t
loadIndex
;
int32_t
iRow
;
SRowInfo
rInfo
;
uint64_t
uid
;
STimeWindow
timeWindow
;
SVersionRange
verRange
;
}
SLDataIter
;
static
SBlockData
*
getCurrentBlock
(
SLDataIter
*
pIter
)
{
return
&
pIter
->
bData
[
pIter
->
loadIndex
];
}
static
SBlockData
*
getNextBlock
(
SLDataIter
*
pIter
)
{
pIter
->
loadIndex
^=
1
;
return
getCurrentBlock
(
pIter
);
}
int32_t
tLDataIterOpen
(
struct
SLDataIter
**
pIter
,
SDataFReader
*
pReader
,
int32_t
iSst
,
int8_t
backward
,
uint64_t
uid
,
STimeWindow
*
pTimeWindow
,
SVersionRange
*
pRange
)
{
int32_t
code
=
0
;
*
pIter
=
taosMemoryCalloc
(
1
,
sizeof
(
SLDataIter
));
if
(
*
pIter
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_exit
;
}
(
*
pIter
)
->
uid
=
uid
;
(
*
pIter
)
->
timeWindow
=
*
pTimeWindow
;
(
*
pIter
)
->
verRange
=
*
pRange
;
(
*
pIter
)
->
pReader
=
pReader
;
(
*
pIter
)
->
iSst
=
iSst
;
(
*
pIter
)
->
backward
=
backward
;
(
*
pIter
)
->
aSstBlk
=
taosArrayInit
(
0
,
sizeof
(
SSstBlk
));
if
((
*
pIter
)
->
aSstBlk
==
NULL
)
{
code
=
TSDB_CODE_OUT_OF_MEMORY
;
goto
_exit
;
}
code
=
tBlockDataCreate
(
&
(
*
pIter
)
->
bData
[
0
]);
if
(
code
)
{
goto
_exit
;
}
code
=
tBlockDataCreate
(
&
(
*
pIter
)
->
bData
[
1
]);
if
(
code
)
{
goto
_exit
;
}
code
=
tsdbReadSstBlk
(
pReader
,
iSst
,
(
*
pIter
)
->
aSstBlk
);
if
(
code
)
{
goto
_exit
;
}
size_t
size
=
taosArrayGetSize
((
*
pIter
)
->
aSstBlk
);
// find the start block
int32_t
index
=
-
1
;
if
(
!
backward
)
{
// asc
for
(
int32_t
i
=
0
;
i
<
size
;
++
i
)
{
SSstBlk
*
p
=
taosArrayGet
((
*
pIter
)
->
aSstBlk
,
i
);
if
(
p
->
minUid
<=
uid
&&
p
->
maxUid
>=
uid
)
{
index
=
i
;
break
;
}
}
}
else
{
// desc
for
(
int32_t
i
=
size
-
1
;
i
>=
0
;
--
i
)
{
SSstBlk
*
p
=
taosArrayGet
((
*
pIter
)
->
aSstBlk
,
i
);
if
(
p
->
minUid
<=
uid
&&
p
->
maxUid
>=
uid
)
{
index
=
i
;
break
;
}
}
}
(
*
pIter
)
->
iSstBlk
=
index
;
if
(
index
!=
-
1
)
{
(
*
pIter
)
->
pSstBlk
=
taosArrayGet
((
*
pIter
)
->
aSstBlk
,
(
*
pIter
)
->
iSstBlk
);
}
_exit:
return
code
;
}
void
tLDataIterClose
(
SLDataIter
*
pIter
)
{
tBlockDataDestroy
(
&
pIter
->
bData
[
0
],
1
);
tBlockDataDestroy
(
&
pIter
->
bData
[
1
],
1
);
taosArrayDestroy
(
pIter
->
aSstBlk
);
taosMemoryFree
(
pIter
);
}
extern
int32_t
tsdbReadSstBlockEx
(
SDataFReader
*
pReader
,
int32_t
iSst
,
SSstBlk
*
pSstBlk
,
SBlockData
*
pBlockData
);
void
tLDataIterNextBlock
(
SLDataIter
*
pIter
)
{
int32_t
step
=
pIter
->
backward
?
-
1
:
1
;
pIter
->
iSstBlk
+=
step
;
int32_t
index
=
-
1
;
size_t
size
=
taosArrayGetSize
(
pIter
->
aSstBlk
);
for
(
int32_t
i
=
pIter
->
iSstBlk
;
i
<
size
&&
i
>=
0
;
i
+=
step
)
{
SSstBlk
*
p
=
taosArrayGet
(
pIter
->
aSstBlk
,
i
);
if
((
!
pIter
->
backward
)
&&
p
->
minUid
>
pIter
->
uid
)
{
break
;
}
if
(
pIter
->
backward
&&
p
->
maxUid
<
pIter
->
uid
)
{
break
;
}
if
(
p
->
minUid
<=
pIter
->
uid
&&
p
->
maxUid
>=
pIter
->
uid
)
{
index
=
i
;
break
;
}
}
if
(
index
==
-
1
)
{
pIter
->
pSstBlk
=
NULL
;
}
else
{
pIter
->
pSstBlk
=
(
SSstBlk
*
)
taosArrayGet
(
pIter
->
aSstBlk
,
pIter
->
iSstBlk
);
}
}
static
void
findNextValidRow
(
SLDataIter
*
pIter
)
{
int32_t
step
=
pIter
->
backward
?
-
1
:
1
;
bool
hasVal
=
false
;
int32_t
i
=
pIter
->
iRow
;
SBlockData
*
pBlockData
=
getCurrentBlock
(
pIter
);
for
(;
i
<
pBlockData
->
nRow
&&
i
>=
0
;
i
+=
step
)
{
if
(
pBlockData
->
aUid
!=
NULL
)
{
if
(
!
pIter
->
backward
)
{
if
(
pBlockData
->
aUid
[
i
]
<
pIter
->
uid
)
{
continue
;
}
else
if
(
pBlockData
->
aUid
[
i
]
>
pIter
->
uid
)
{
break
;
}
}
else
{
if
(
pBlockData
->
aUid
[
i
]
>
pIter
->
uid
)
{
continue
;
}
else
if
(
pBlockData
->
aUid
[
i
]
<
pIter
->
uid
)
{
break
;
}
}
}
int64_t
ts
=
pBlockData
->
aTSKEY
[
i
];
if
(
!
pIter
->
backward
)
{
// asc
if
(
ts
>
pIter
->
timeWindow
.
ekey
)
{
// no more data
break
;
}
else
if
(
ts
<
pIter
->
timeWindow
.
skey
)
{
continue
;
}
}
else
{
if
(
ts
<
pIter
->
timeWindow
.
skey
)
{
break
;
}
else
if
(
ts
>
pIter
->
timeWindow
.
ekey
)
{
continue
;
}
}
int64_t
ver
=
pBlockData
->
aVersion
[
i
];
if
(
ver
<
pIter
->
verRange
.
minVer
)
{
continue
;
}
// todo opt handle desc case
if
(
ver
>
pIter
->
verRange
.
maxVer
)
{
continue
;
}
// todo handle delete soon
#if 0
TSDBKEY k = {.ts = ts, .version = ver};
if (hasBeenDropped(pBlockScanInfo->delSkyline, &pBlockScanInfo->lastBlockDelIndex, &k, pLastBlockReader->order)) {
continue;
}
#endif
hasVal
=
true
;
break
;
}
pIter
->
iRow
=
(
hasVal
)
?
i
:
-
1
;
}
bool
tLDataIterNextRow
(
SLDataIter
*
pIter
)
{
int32_t
code
=
0
;
int32_t
step
=
pIter
->
backward
?
-
1
:
1
;
// no qualified last file block in current file, no need to fetch row
if
(
pIter
->
pSstBlk
==
NULL
)
{
return
false
;
}
int32_t
iBlockL
=
pIter
->
iSstBlk
;
SBlockData
*
pBlockData
=
getCurrentBlock
(
pIter
);
if
(
pBlockData
->
nRow
==
0
&&
pIter
->
pSstBlk
!=
NULL
)
{
// current block not loaded yet
pBlockData
=
getNextBlock
(
pIter
);
code
=
tsdbReadSstBlockEx
(
pIter
->
pReader
,
pIter
->
iSst
,
pIter
->
pSstBlk
,
pBlockData
);
if
(
code
!=
TSDB_CODE_SUCCESS
)
{
goto
_exit
;
}
pIter
->
iRow
=
(
pIter
->
backward
)
?
pBlockData
->
nRow
:
-
1
;
}
pIter
->
iRow
+=
step
;
while
(
1
)
{
findNextValidRow
(
pIter
);
if
(
pIter
->
iRow
>=
pBlockData
->
nRow
||
pIter
->
iRow
<
0
)
{
tLDataIterNextBlock
(
pIter
);
if
(
pIter
->
pSstBlk
==
NULL
)
{
// no more data
goto
_exit
;
}
}
else
{
break
;
}
if
(
iBlockL
!=
pIter
->
iSstBlk
)
{
pBlockData
=
getNextBlock
(
pIter
);
code
=
tsdbReadSstBlockEx
(
pIter
->
pReader
,
pIter
->
iSst
,
pIter
->
pSstBlk
,
pBlockData
);
if
(
code
)
{
goto
_exit
;
}
pIter
->
iRow
=
pIter
->
backward
?
(
pBlockData
->
nRow
-
1
)
:
0
;
}
}
pIter
->
rInfo
.
suid
=
pBlockData
->
suid
;
pIter
->
rInfo
.
uid
=
pBlockData
->
uid
;
pIter
->
rInfo
.
row
=
tsdbRowFromBlockData
(
pBlockData
,
pIter
->
iRow
);
_exit:
if
(
code
!=
TSDB_CODE_SUCCESS
)
{
terrno
=
code
;
}
return
(
code
==
TSDB_CODE_SUCCESS
)
&&
(
pIter
->
pSstBlk
!=
NULL
);
}
SRowInfo
*
tLDataIterGet
(
SLDataIter
*
pIter
)
{
return
&
pIter
->
rInfo
;
}
// SMergeTree =================================================
static
FORCE_INLINE
int32_t
tLDataIterCmprFn
(
const
void
*
p1
,
const
void
*
p2
)
{
SLDataIter
*
pIter1
=
(
SLDataIter
*
)(((
uint8_t
*
)
p1
)
-
sizeof
(
SRBTreeNode
));
SLDataIter
*
pIter2
=
(
SLDataIter
*
)(((
uint8_t
*
)
p2
)
-
sizeof
(
SRBTreeNode
));
TSDBKEY
key1
=
TSDBROW_KEY
(
&
pIter1
->
rInfo
.
row
);
TSDBKEY
key2
=
TSDBROW_KEY
(
&
pIter2
->
rInfo
.
row
);
if
(
key1
.
ts
<
key2
.
ts
)
{
return
-
1
;
}
else
if
(
key1
.
ts
>
key2
.
ts
)
{
return
1
;
}
else
{
if
(
key1
.
version
<
key2
.
version
)
{
return
-
1
;
}
else
if
(
key1
.
version
>
key2
.
version
)
{
return
1
;
}
else
{
return
0
;
}
}
}
int32_t
tMergeTreeOpen
(
SMergeTree
*
pMTree
,
int8_t
backward
,
SDataFReader
*
pFReader
,
uint64_t
uid
,
STimeWindow
*
pTimeWindow
,
SVersionRange
*
pVerRange
)
{
pMTree
->
backward
=
backward
;
pMTree
->
pIter
=
NULL
;
pMTree
->
pIterList
=
taosArrayInit
(
4
,
POINTER_BYTES
);
if
(
pMTree
->
pIterList
==
NULL
)
{
return
TSDB_CODE_OUT_OF_MEMORY
;
}
tRBTreeCreate
(
&
pMTree
->
rbt
,
tLDataIterCmprFn
);
int32_t
code
=
TSDB_CODE_OUT_OF_MEMORY
;
struct
SLDataIter
*
pIterList
[
TSDB_DEFAULT_LAST_FILE
]
=
{
0
};
for
(
int32_t
i
=
0
;
i
<
pFReader
->
pSet
->
nSstF
;
++
i
)
{
// open all last file
code
=
tLDataIterOpen
(
&
pIterList
[
i
],
pFReader
,
i
,
pMTree
->
backward
,
uid
,
pTimeWindow
,
pVerRange
);
if
(
code
!=
TSDB_CODE_SUCCESS
)
{
goto
_end
;
}
bool
hasVal
=
tLDataIterNextRow
(
pIterList
[
i
]);
if
(
hasVal
)
{
taosArrayPush
(
pMTree
->
pIterList
,
&
pIterList
[
i
]);
tMergeTreeAddIter
(
pMTree
,
pIterList
[
i
]);
}
else
{
tLDataIterClose
(
pIterList
[
i
]);
}
}
return
code
;
_end:
tMergeTreeClose
(
pMTree
);
return
code
;
}
void
tMergeTreeAddIter
(
SMergeTree
*
pMTree
,
SLDataIter
*
pIter
)
{
tRBTreePut
(
&
pMTree
->
rbt
,
(
SRBTreeNode
*
)
pIter
);
}
bool
tMergeTreeNext
(
SMergeTree
*
pMTree
)
{
int32_t
code
=
TSDB_CODE_SUCCESS
;
if
(
pMTree
->
pIter
)
{
SLDataIter
*
pIter
=
pMTree
->
pIter
;
bool
hasVal
=
tLDataIterNextRow
(
pIter
);
if
(
!
hasVal
)
{
pMTree
->
pIter
=
NULL
;
}
// compare with min in RB Tree
pIter
=
(
SLDataIter
*
)
tRBTreeMin
(
&
pMTree
->
rbt
);
if
(
pMTree
->
pIter
&&
pIter
)
{
int32_t
c
=
pMTree
->
rbt
.
cmprFn
(
RBTREE_NODE_PAYLOAD
(
&
pMTree
->
pIter
->
node
),
RBTREE_NODE_PAYLOAD
(
&
pIter
->
node
));
if
(
c
>
0
)
{
tRBTreePut
(
&
pMTree
->
rbt
,
(
SRBTreeNode
*
)
pMTree
->
pIter
);
pMTree
->
pIter
=
NULL
;
}
else
{
ASSERT
(
c
);
}
}
}
if
(
pMTree
->
pIter
==
NULL
)
{
pMTree
->
pIter
=
(
SLDataIter
*
)
tRBTreeMin
(
&
pMTree
->
rbt
);
if
(
pMTree
->
pIter
)
{
tRBTreeDrop
(
&
pMTree
->
rbt
,
(
SRBTreeNode
*
)
pMTree
->
pIter
);
}
}
return
pMTree
->
pIter
!=
NULL
;
}
TSDBROW
tMergeTreeGetRow
(
SMergeTree
*
pMTree
)
{
return
pMTree
->
pIter
->
rInfo
.
row
;
}
void
tMergeTreeClose
(
SMergeTree
*
pMTree
)
{
size_t
size
=
taosArrayGetSize
(
pMTree
->
pIterList
);
for
(
int32_t
i
=
0
;
i
<
size
;
++
i
)
{
SLDataIter
*
pIter
=
taosArrayGetP
(
pMTree
->
pIterList
,
i
);
tLDataIterClose
(
pIter
);
}
pMTree
->
pIterList
=
taosArrayDestroy
(
pMTree
->
pIterList
);
pMTree
->
pIter
=
NULL
;
}
source/dnode/vnode/src/tsdb/tsdbRead.c
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
source/dnode/vnode/src/tsdb/tsdbReaderWriter.c
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
source/dnode/vnode/src/tsdb/tsdbRetention.c
浏览文件 @
38b62ddd
...
@@ -60,7 +60,7 @@ int32_t tsdbDoRetention(STsdb *pTsdb, int64_t now) {
...
@@ -60,7 +60,7 @@ int32_t tsdbDoRetention(STsdb *pTsdb, int64_t now) {
if
(
expLevel
<
0
)
{
if
(
expLevel
<
0
)
{
taosMemoryFree
(
pSet
->
pHeadF
);
taosMemoryFree
(
pSet
->
pHeadF
);
taosMemoryFree
(
pSet
->
pDataF
);
taosMemoryFree
(
pSet
->
pDataF
);
taosMemoryFree
(
pSet
->
pLastF
);
taosMemoryFree
(
pSet
->
aSstF
[
0
]
);
taosMemoryFree
(
pSet
->
pSmaF
);
taosMemoryFree
(
pSet
->
pSmaF
);
taosArrayRemove
(
fs
.
aDFileSet
,
iSet
);
taosArrayRemove
(
fs
.
aDFileSet
,
iSet
);
iSet
--
;
iSet
--
;
...
...
source/dnode/vnode/src/tsdb/tsdbSnapshot.c
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
source/dnode/vnode/src/tsdb/tsdbUtil.c
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
source/dnode/vnode/src/vnd/vnodeQuery.c
浏览文件 @
38b62ddd
...
@@ -368,6 +368,7 @@ _exit:
...
@@ -368,6 +368,7 @@ _exit:
int32_t
vnodeGetLoad
(
SVnode
*
pVnode
,
SVnodeLoad
*
pLoad
)
{
int32_t
vnodeGetLoad
(
SVnode
*
pVnode
,
SVnodeLoad
*
pLoad
)
{
pLoad
->
vgId
=
TD_VID
(
pVnode
);
pLoad
->
vgId
=
TD_VID
(
pVnode
);
pLoad
->
syncState
=
syncGetMyRole
(
pVnode
->
sync
);
pLoad
->
syncState
=
syncGetMyRole
(
pVnode
->
sync
);
pLoad
->
cacheUsage
=
tsdbCacheGetUsage
(
pVnode
);
pLoad
->
numOfTables
=
metaGetTbNum
(
pVnode
->
pMeta
);
pLoad
->
numOfTables
=
metaGetTbNum
(
pVnode
->
pMeta
);
pLoad
->
numOfTimeSeries
=
metaGetTimeSeriesNum
(
pVnode
->
pMeta
);
pLoad
->
numOfTimeSeries
=
metaGetTimeSeriesNum
(
pVnode
->
pMeta
);
pLoad
->
totalStorage
=
(
int64_t
)
3
*
1073741824
;
pLoad
->
totalStorage
=
(
int64_t
)
3
*
1073741824
;
...
...
source/libs/executor/inc/executil.h
浏览文件 @
38b62ddd
...
@@ -87,8 +87,6 @@ struct SqlFunctionCtx;
...
@@ -87,8 +87,6 @@ struct SqlFunctionCtx;
size_t
getResultRowSize
(
struct
SqlFunctionCtx
*
pCtx
,
int32_t
numOfOutput
);
size_t
getResultRowSize
(
struct
SqlFunctionCtx
*
pCtx
,
int32_t
numOfOutput
);
void
initResultRowInfo
(
SResultRowInfo
*
pResultRowInfo
);
void
initResultRowInfo
(
SResultRowInfo
*
pResultRowInfo
);
void
initResultRow
(
SResultRow
*
pResultRow
);
void
closeResultRow
(
SResultRow
*
pResultRow
);
void
closeResultRow
(
SResultRow
*
pResultRow
);
struct
SResultRowEntryInfo
*
getResultEntryInfo
(
const
SResultRow
*
pRow
,
int32_t
index
,
const
int32_t
*
offset
);
struct
SResultRowEntryInfo
*
getResultEntryInfo
(
const
SResultRow
*
pRow
,
int32_t
index
,
const
int32_t
*
offset
);
...
...
source/libs/executor/src/dataDeleter.c
浏览文件 @
38b62ddd
...
@@ -168,7 +168,9 @@ static void getDataLength(SDataSinkHandle* pHandle, int64_t* pLen, bool* pQueryE
...
@@ -168,7 +168,9 @@ static void getDataLength(SDataSinkHandle* pHandle, int64_t* pLen, bool* pQueryE
taosReadQitem
(
pDeleter
->
pDataBlocks
,
(
void
**
)
&
pBuf
);
taosReadQitem
(
pDeleter
->
pDataBlocks
,
(
void
**
)
&
pBuf
);
memcpy
(
&
pDeleter
->
nextOutput
,
pBuf
,
sizeof
(
SDataDeleterBuf
));
memcpy
(
&
pDeleter
->
nextOutput
,
pBuf
,
sizeof
(
SDataDeleterBuf
));
taosFreeQitem
(
pBuf
);
taosFreeQitem
(
pBuf
);
*
pLen
=
((
SDataCacheEntry
*
)(
pDeleter
->
nextOutput
.
pData
))
->
dataLen
;
SDataCacheEntry
*
pEntry
=
(
SDataCacheEntry
*
)
pDeleter
->
nextOutput
.
pData
;
*
pLen
=
pEntry
->
dataLen
;
*
pQueryEnd
=
pDeleter
->
queryEnd
;
*
pQueryEnd
=
pDeleter
->
queryEnd
;
qDebug
(
"got data len %"
PRId64
", row num %d in sink"
,
*
pLen
,
((
SDataCacheEntry
*
)(
pDeleter
->
nextOutput
.
pData
))
->
numOfRows
);
qDebug
(
"got data len %"
PRId64
", row num %d in sink"
,
*
pLen
,
((
SDataCacheEntry
*
)(
pDeleter
->
nextOutput
.
pData
))
->
numOfRows
);
}
}
...
...
source/libs/executor/src/dataDispatcher.c
浏览文件 @
38b62ddd
...
@@ -93,6 +93,8 @@ static void toDataCacheEntry(SDataDispatchHandle* pHandle, const SInputData* pIn
...
@@ -93,6 +93,8 @@ static void toDataCacheEntry(SDataDispatchHandle* pHandle, const SInputData* pIn
pBuf
->
useSize
=
sizeof
(
SDataCacheEntry
);
pBuf
->
useSize
=
sizeof
(
SDataCacheEntry
);
blockEncode
(
pInput
->
pData
,
pEntry
->
data
,
&
pEntry
->
dataLen
,
numOfCols
,
pEntry
->
compressed
);
blockEncode
(
pInput
->
pData
,
pEntry
->
data
,
&
pEntry
->
dataLen
,
numOfCols
,
pEntry
->
compressed
);
ASSERT
(
pEntry
->
numOfRows
==
*
(
int32_t
*
)(
pEntry
->
data
+
8
));
ASSERT
(
pEntry
->
numOfCols
==
*
(
int32_t
*
)(
pEntry
->
data
+
8
+
4
));
pBuf
->
useSize
+=
pEntry
->
dataLen
;
pBuf
->
useSize
+=
pEntry
->
dataLen
;
...
@@ -170,7 +172,13 @@ static void getDataLength(SDataSinkHandle* pHandle, int64_t* pLen, bool* pQueryE
...
@@ -170,7 +172,13 @@ static void getDataLength(SDataSinkHandle* pHandle, int64_t* pLen, bool* pQueryE
taosReadQitem
(
pDispatcher
->
pDataBlocks
,
(
void
**
)
&
pBuf
);
taosReadQitem
(
pDispatcher
->
pDataBlocks
,
(
void
**
)
&
pBuf
);
memcpy
(
&
pDispatcher
->
nextOutput
,
pBuf
,
sizeof
(
SDataDispatchBuf
));
memcpy
(
&
pDispatcher
->
nextOutput
,
pBuf
,
sizeof
(
SDataDispatchBuf
));
taosFreeQitem
(
pBuf
);
taosFreeQitem
(
pBuf
);
*
pLen
=
((
SDataCacheEntry
*
)(
pDispatcher
->
nextOutput
.
pData
))
->
dataLen
;
SDataCacheEntry
*
pEntry
=
(
SDataCacheEntry
*
)
pDispatcher
->
nextOutput
.
pData
;
*
pLen
=
pEntry
->
dataLen
;
ASSERT
(
pEntry
->
numOfRows
==
*
(
int32_t
*
)(
pEntry
->
data
+
8
));
ASSERT
(
pEntry
->
numOfCols
==
*
(
int32_t
*
)(
pEntry
->
data
+
8
+
4
));
*
pQueryEnd
=
pDispatcher
->
queryEnd
;
*
pQueryEnd
=
pDispatcher
->
queryEnd
;
qDebug
(
"got data len %"
PRId64
", row num %d in sink"
,
*
pLen
,
((
SDataCacheEntry
*
)(
pDispatcher
->
nextOutput
.
pData
))
->
numOfRows
);
qDebug
(
"got data len %"
PRId64
", row num %d in sink"
,
*
pLen
,
((
SDataCacheEntry
*
)(
pDispatcher
->
nextOutput
.
pData
))
->
numOfRows
);
}
}
...
@@ -191,6 +199,9 @@ static int32_t getDataBlock(SDataSinkHandle* pHandle, SOutputData* pOutput) {
...
@@ -191,6 +199,9 @@ static int32_t getDataBlock(SDataSinkHandle* pHandle, SOutputData* pOutput) {
pOutput
->
numOfCols
=
pEntry
->
numOfCols
;
pOutput
->
numOfCols
=
pEntry
->
numOfCols
;
pOutput
->
compressed
=
pEntry
->
compressed
;
pOutput
->
compressed
=
pEntry
->
compressed
;
ASSERT
(
pEntry
->
numOfRows
==
*
(
int32_t
*
)(
pEntry
->
data
+
8
));
ASSERT
(
pEntry
->
numOfCols
==
*
(
int32_t
*
)(
pEntry
->
data
+
8
+
4
));
atomic_sub_fetch_64
(
&
pDispatcher
->
cachedSize
,
pEntry
->
dataLen
);
atomic_sub_fetch_64
(
&
pDispatcher
->
cachedSize
,
pEntry
->
dataLen
);
atomic_sub_fetch_64
(
&
gDataSinkStat
.
cachedSize
,
pEntry
->
dataLen
);
atomic_sub_fetch_64
(
&
gDataSinkStat
.
cachedSize
,
pEntry
->
dataLen
);
...
...
source/libs/executor/src/executil.c
浏览文件 @
38b62ddd
...
@@ -1302,7 +1302,6 @@ int32_t initQueryTableDataCond(SQueryTableDataCond* pCond, const STableScanPhysi
...
@@ -1302,7 +1302,6 @@ int32_t initQueryTableDataCond(SQueryTableDataCond* pCond, const STableScanPhysi
pCond
->
type
=
TIMEWINDOW_RANGE_CONTAINED
;
pCond
->
type
=
TIMEWINDOW_RANGE_CONTAINED
;
pCond
->
startVersion
=
-
1
;
pCond
->
startVersion
=
-
1
;
pCond
->
endVersion
=
-
1
;
pCond
->
endVersion
=
-
1
;
pCond
->
schemaVersion
=
-
1
;
// pCond->type = pTableScanNode->scanFlag;
// pCond->type = pTableScanNode->scanFlag;
int32_t
j
=
0
;
int32_t
j
=
0
;
...
...
source/libs/executor/src/executor.c
浏览文件 @
38b62ddd
...
@@ -731,7 +731,6 @@ int32_t initQueryTableDataCondForTmq(SQueryTableDataCond* pCond, SSnapContext* s
...
@@ -731,7 +731,6 @@ int32_t initQueryTableDataCondForTmq(SQueryTableDataCond* pCond, SSnapContext* s
pCond
->
type
=
TIMEWINDOW_RANGE_CONTAINED
;
pCond
->
type
=
TIMEWINDOW_RANGE_CONTAINED
;
pCond
->
startVersion
=
-
1
;
pCond
->
startVersion
=
-
1
;
pCond
->
endVersion
=
sContext
->
snapVersion
;
pCond
->
endVersion
=
sContext
->
snapVersion
;
pCond
->
schemaVersion
=
sContext
->
snapVersion
;
for
(
int32_t
i
=
0
;
i
<
pCond
->
numOfCols
;
++
i
)
{
for
(
int32_t
i
=
0
;
i
<
pCond
->
numOfCols
;
++
i
)
{
pCond
->
colList
[
i
].
type
=
mtInfo
.
schema
->
pSchema
[
i
].
type
;
pCond
->
colList
[
i
].
type
=
mtInfo
.
schema
->
pSchema
[
i
].
type
;
...
...
source/libs/executor/src/executorimpl.c
浏览文件 @
38b62ddd
...
@@ -3731,7 +3731,6 @@ static int32_t initTableblockDistQueryCond(uint64_t uid, SQueryTableDataCond* pC
...
@@ -3731,7 +3731,6 @@ static int32_t initTableblockDistQueryCond(uint64_t uid, SQueryTableDataCond* pC
pCond
->
type
=
TIMEWINDOW_RANGE_CONTAINED
;
pCond
->
type
=
TIMEWINDOW_RANGE_CONTAINED
;
pCond
->
startVersion
=
-
1
;
pCond
->
startVersion
=
-
1
;
pCond
->
endVersion
=
-
1
;
pCond
->
endVersion
=
-
1
;
pCond
->
schemaVersion
=
-
1
;
return
TSDB_CODE_SUCCESS
;
return
TSDB_CODE_SUCCESS
;
}
}
...
...
source/libs/executor/src/scanoperator.c
浏览文件 @
38b62ddd
...
@@ -695,6 +695,7 @@ static void destroyTableScanOperatorInfo(void* param) {
...
@@ -695,6 +695,7 @@ static void destroyTableScanOperatorInfo(void* param) {
cleanupQueryTableDataCond
(
&
pTableScanInfo
->
cond
);
cleanupQueryTableDataCond
(
&
pTableScanInfo
->
cond
);
tsdbReaderClose
(
pTableScanInfo
->
dataReader
);
tsdbReaderClose
(
pTableScanInfo
->
dataReader
);
pTableScanInfo
->
dataReader
=
NULL
;
if
(
pTableScanInfo
->
pColMatchInfo
!=
NULL
)
{
if
(
pTableScanInfo
->
pColMatchInfo
!=
NULL
)
{
taosArrayDestroy
(
pTableScanInfo
->
pColMatchInfo
);
taosArrayDestroy
(
pTableScanInfo
->
pColMatchInfo
);
...
...
source/libs/executor/src/timewindowoperator.c
浏览文件 @
38b62ddd
...
@@ -622,7 +622,8 @@ static void doInterpUnclosedTimeWindow(SOperatorInfo* pOperatorInfo, int32_t num
...
@@ -622,7 +622,8 @@ static void doInterpUnclosedTimeWindow(SOperatorInfo* pOperatorInfo, int32_t num
if
(
pr
->
closed
)
{
if
(
pr
->
closed
)
{
ASSERT
(
isResultRowInterpolated
(
pr
,
RESULT_ROW_START_INTERP
)
&&
ASSERT
(
isResultRowInterpolated
(
pr
,
RESULT_ROW_START_INTERP
)
&&
isResultRowInterpolated
(
pr
,
RESULT_ROW_END_INTERP
));
isResultRowInterpolated
(
pr
,
RESULT_ROW_END_INTERP
));
tdListPopHead
(
pResultRowInfo
->
openWindow
);
SListNode
*
pNode
=
tdListPopHead
(
pResultRowInfo
->
openWindow
);
taosMemoryFree
(
pNode
);
continue
;
continue
;
}
}
...
@@ -651,7 +652,8 @@ static void doInterpUnclosedTimeWindow(SOperatorInfo* pOperatorInfo, int32_t num
...
@@ -651,7 +652,8 @@ static void doInterpUnclosedTimeWindow(SOperatorInfo* pOperatorInfo, int32_t num
if
(
isResultRowInterpolated
(
pResult
,
RESULT_ROW_END_INTERP
))
{
if
(
isResultRowInterpolated
(
pResult
,
RESULT_ROW_END_INTERP
))
{
closeResultRow
(
pr
);
closeResultRow
(
pr
);
tdListPopHead
(
pResultRowInfo
->
openWindow
);
SListNode
*
pNode
=
tdListPopHead
(
pResultRowInfo
->
openWindow
);
taosMemoryFree
(
pNode
);
}
else
{
// the remains are can not be closed yet.
}
else
{
// the remains are can not be closed yet.
break
;
break
;
}
}
...
@@ -1731,6 +1733,10 @@ void destroyIntervalOperatorInfo(void* param) {
...
@@ -1731,6 +1733,10 @@ void destroyIntervalOperatorInfo(void* param) {
SIntervalAggOperatorInfo
*
pInfo
=
(
SIntervalAggOperatorInfo
*
)
param
;
SIntervalAggOperatorInfo
*
pInfo
=
(
SIntervalAggOperatorInfo
*
)
param
;
cleanupBasicInfo
(
&
pInfo
->
binfo
);
cleanupBasicInfo
(
&
pInfo
->
binfo
);
cleanupAggSup
(
&
pInfo
->
aggSup
);
cleanupAggSup
(
&
pInfo
->
aggSup
);
cleanupExprSupp
(
&
pInfo
->
scalarSupp
);
tdListFree
(
pInfo
->
binfo
.
resultRowInfo
.
openWindow
);
pInfo
->
pRecycledPages
=
taosArrayDestroy
(
pInfo
->
pRecycledPages
);
pInfo
->
pRecycledPages
=
taosArrayDestroy
(
pInfo
->
pRecycledPages
);
pInfo
->
pInterpCols
=
taosArrayDestroy
(
pInfo
->
pInterpCols
);
pInfo
->
pInterpCols
=
taosArrayDestroy
(
pInfo
->
pInterpCols
);
taosArrayDestroyEx
(
pInfo
->
pPrevValues
,
freeItem
);
taosArrayDestroyEx
(
pInfo
->
pPrevValues
,
freeItem
);
...
...
source/libs/executor/src/tsort.c
浏览文件 @
38b62ddd
...
@@ -227,9 +227,9 @@ static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int
...
@@ -227,9 +227,9 @@ static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int
continue
;
continue
;
}
}
SPageInfo
*
pPgInfo
=
*
(
SPageInfo
**
)
taosArrayGet
(
pSource
->
pageIdList
,
pSource
->
pageIndex
);
int32_t
*
pPgId
=
taosArrayGet
(
pSource
->
pageIdList
,
pSource
->
pageIndex
);
void
*
pPage
=
getBufPage
(
pHandle
->
pBuf
,
getPageId
(
pPgInfo
)
);
void
*
pPage
=
getBufPage
(
pHandle
->
pBuf
,
*
pPgId
);
code
=
blockDataFromBuf
(
pSource
->
src
.
pBlock
,
pPage
);
code
=
blockDataFromBuf
(
pSource
->
src
.
pBlock
,
pPage
);
if
(
code
!=
TSDB_CODE_SUCCESS
)
{
if
(
code
!=
TSDB_CODE_SUCCESS
)
{
return
code
;
return
code
;
...
@@ -302,9 +302,9 @@ static int32_t adjustMergeTreeForNextTuple(SSortSource *pSource, SMultiwayMergeT
...
@@ -302,9 +302,9 @@ static int32_t adjustMergeTreeForNextTuple(SSortSource *pSource, SMultiwayMergeT
pSource
->
pageIndex
=
-
1
;
pSource
->
pageIndex
=
-
1
;
pSource
->
src
.
pBlock
=
blockDataDestroy
(
pSource
->
src
.
pBlock
);
pSource
->
src
.
pBlock
=
blockDataDestroy
(
pSource
->
src
.
pBlock
);
}
else
{
}
else
{
SPageInfo
*
pPgInfo
=
*
(
SPageInfo
**
)
taosArrayGet
(
pSource
->
pageIdList
,
pSource
->
pageIndex
);
int32_t
*
pPgId
=
taosArrayGet
(
pSource
->
pageIdList
,
pSource
->
pageIndex
);
void
*
pPage
=
getBufPage
(
pHandle
->
pBuf
,
getPageId
(
pPgInfo
)
);
void
*
pPage
=
getBufPage
(
pHandle
->
pBuf
,
*
pPgId
);
int32_t
code
=
blockDataFromBuf
(
pSource
->
src
.
pBlock
,
pPage
);
int32_t
code
=
blockDataFromBuf
(
pSource
->
src
.
pBlock
,
pPage
);
if
(
code
!=
TSDB_CODE_SUCCESS
)
{
if
(
code
!=
TSDB_CODE_SUCCESS
)
{
return
code
;
return
code
;
...
...
source/libs/monitor/src/monMsg.c
浏览文件 @
38b62ddd
...
@@ -510,6 +510,7 @@ int32_t tSerializeSMonVloadInfo(void *buf, int32_t bufLen, SMonVloadInfo *pInfo)
...
@@ -510,6 +510,7 @@ int32_t tSerializeSMonVloadInfo(void *buf, int32_t bufLen, SMonVloadInfo *pInfo)
SVnodeLoad
*
pLoad
=
taosArrayGet
(
pInfo
->
pVloads
,
i
);
SVnodeLoad
*
pLoad
=
taosArrayGet
(
pInfo
->
pVloads
,
i
);
if
(
tEncodeI32
(
&
encoder
,
pLoad
->
vgId
)
<
0
)
return
-
1
;
if
(
tEncodeI32
(
&
encoder
,
pLoad
->
vgId
)
<
0
)
return
-
1
;
if
(
tEncodeI32
(
&
encoder
,
pLoad
->
syncState
)
<
0
)
return
-
1
;
if
(
tEncodeI32
(
&
encoder
,
pLoad
->
syncState
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pLoad
->
cacheUsage
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pLoad
->
numOfTables
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pLoad
->
numOfTables
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pLoad
->
numOfTimeSeries
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pLoad
->
numOfTimeSeries
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pLoad
->
totalStorage
)
<
0
)
return
-
1
;
if
(
tEncodeI64
(
&
encoder
,
pLoad
->
totalStorage
)
<
0
)
return
-
1
;
...
@@ -544,6 +545,7 @@ int32_t tDeserializeSMonVloadInfo(void *buf, int32_t bufLen, SMonVloadInfo *pInf
...
@@ -544,6 +545,7 @@ int32_t tDeserializeSMonVloadInfo(void *buf, int32_t bufLen, SMonVloadInfo *pInf
SVnodeLoad
load
=
{
0
};
SVnodeLoad
load
=
{
0
};
if
(
tDecodeI32
(
&
decoder
,
&
load
.
vgId
)
<
0
)
return
-
1
;
if
(
tDecodeI32
(
&
decoder
,
&
load
.
vgId
)
<
0
)
return
-
1
;
if
(
tDecodeI32
(
&
decoder
,
&
load
.
syncState
)
<
0
)
return
-
1
;
if
(
tDecodeI32
(
&
decoder
,
&
load
.
syncState
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
load
.
cacheUsage
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
load
.
numOfTables
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
load
.
numOfTables
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
load
.
numOfTimeSeries
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
load
.
numOfTimeSeries
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
load
.
totalStorage
)
<
0
)
return
-
1
;
if
(
tDecodeI64
(
&
decoder
,
&
load
.
totalStorage
)
<
0
)
return
-
1
;
...
@@ -594,7 +596,6 @@ int32_t tDeserializeSMonMloadInfo(void *buf, int32_t bufLen, SMonMloadInfo *pInf
...
@@ -594,7 +596,6 @@ int32_t tDeserializeSMonMloadInfo(void *buf, int32_t bufLen, SMonMloadInfo *pInf
return
0
;
return
0
;
}
}
int32_t
tSerializeSQnodeLoad
(
void
*
buf
,
int32_t
bufLen
,
SQnodeLoad
*
pInfo
)
{
int32_t
tSerializeSQnodeLoad
(
void
*
buf
,
int32_t
bufLen
,
SQnodeLoad
*
pInfo
)
{
SEncoder
encoder
=
{
0
};
SEncoder
encoder
=
{
0
};
tEncoderInit
(
&
encoder
,
buf
,
bufLen
);
tEncoderInit
(
&
encoder
,
buf
,
bufLen
);
...
@@ -639,5 +640,3 @@ int32_t tDeserializeSQnodeLoad(void *buf, int32_t bufLen, SQnodeLoad *pInfo) {
...
@@ -639,5 +640,3 @@ int32_t tDeserializeSQnodeLoad(void *buf, int32_t bufLen, SQnodeLoad *pInfo) {
tDecoderClear
(
&
decoder
);
tDecoderClear
(
&
decoder
);
return
0
;
return
0
;
}
}
source/libs/transport/src/transComm.c
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
source/util/src/trbtree.c
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
source/util/test/CMakeLists.txt
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
source/util/test/trbtreeTest.cpp
0 → 100644
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
tests/script/tsim/parser/slimit_alter_tags.sim
浏览文件 @
38b62ddd
...
@@ -128,6 +128,7 @@ if $rows != 5 then
...
@@ -128,6 +128,7 @@ if $rows != 5 then
return -1
return -1
endi
endi
if $data00 != $rowNum then
if $data00 != $rowNum then
print expect $rowNum , actual: $data00
return -1
return -1
endi
endi
if $data10 != $rowNum then
if $data10 != $rowNum then
...
...
tests/system-test/fulltest.sh
浏览文件 @
38b62ddd
此差异已折叠。
点击以展开。
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录