Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
2dot5
ClickHouse
提交
d93b0d50
C
ClickHouse
项目概览
2dot5
/
ClickHouse
通知
3
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
DevOps
流水线
流水线任务
计划
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
C
ClickHouse
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
DevOps
DevOps
流水线
流水线任务
计划
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
流水线任务
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
提交
d93b0d50
编写于
12月 23, 2019
作者:
N
Nikita Mikhaylov
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
more
上级
be5729e1
变更
3
隐藏空白更改
内联
并排
Showing
3 changed file
with
88 addition
and
24 deletion
+88
-24
dbms/src/Dictionaries/CacheDictionary.cpp
dbms/src/Dictionaries/CacheDictionary.cpp
+2
-6
dbms/src/Dictionaries/CacheDictionary.h
dbms/src/Dictionaries/CacheDictionary.h
+9
-0
dbms/src/Dictionaries/CacheDictionary.inc.h
dbms/src/Dictionaries/CacheDictionary.inc.h
+77
-18
未找到文件。
dbms/src/Dictionaries/CacheDictionary.cpp
浏览文件 @
d93b0d50
...
...
@@ -12,7 +12,6 @@
#include <Common/typeid_cast.h>
#include <ext/range.h>
#include <ext/size.h>
#include <Interpreters/Context.h>
#include "CacheDictionary.inc.h"
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
...
...
@@ -292,9 +291,7 @@ void CacheDictionary::has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8>
size_t
cache_expired
=
0
,
cache_not_found
=
0
,
cache_hit
=
0
;
Context
*
context
=
current_thread
->
getThreadGroup
()
->
global_context
;
const
bool
allow_read_expired_keys_from_cache_dictionary
=
context
->
getSettingsRef
().
allow_read_expired_keys_from_cache_dictionary
;
const
bool
allow_read_expired_keys_from_cache_dictionary
=
getAllowReadExpiredKeysSetting
();
const
auto
rows
=
ext
::
size
(
ids
);
{
...
...
@@ -380,8 +377,7 @@ void CacheDictionary::has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8>
}
};
UInt64
timeout
{
10
};
const
bool
res
=
update_queue
.
tryPush
(
update_unit
,
timeout
);
const
bool
res
=
update_queue
.
tryPush
(
update_unit
,
update_queue_push_timeout_milliseconds
);
if
(
!
res
)
throw
std
::
runtime_error
(
"Too many updates"
);
...
...
dbms/src/Dictionaries/CacheDictionary.h
浏览文件 @
d93b0d50
...
...
@@ -16,6 +16,7 @@
#include <pcg_random.hpp>
#include <Common/ArenaWithFreeLists.h>
#include <Common/CurrentMetrics.h>
#include <Interpreters/Context.h>
#include <ext/bit_cast.h>
#include "DictionaryStructure.h"
#include "IDictionary.h"
...
...
@@ -313,6 +314,14 @@ private:
void
updateThreadFunction
();
std
::
atomic
<
bool
>
finished
{
false
};
bool
getAllowReadExpiredKeysSetting
()
const
{
Context
*
context
=
current_thread
->
getThreadGroup
()
->
global_context
;
return
context
->
getSettingsRef
().
allow_read_expired_keys_from_cache_dictionary
;
}
const
size_t
update_queue_push_timeout_milliseconds
=
10
;
void
waitForCurrentUpdateFinish
()
const
;
mutable
std
::
mutex
update_mutex
;
};
...
...
dbms/src/Dictionaries/CacheDictionary.inc.h
浏览文件 @
d93b0d50
...
...
@@ -162,10 +162,13 @@ void CacheDictionary::getItemsString(
out
->
getOffsets
().
resize_assume_reserved
(
0
);
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
std
::
unordered_map
<
Key
,
std
::
vector
<
size_t
>>
outdated_ids
;
std
::
unordered_map
<
Key
,
std
::
vector
<
size_t
>>
cache_expired_ids
;
std
::
unordered_map
<
Key
,
std
::
vector
<
size_t
>>
cache_not_found_ids
;
/// we are going to store every string separately
std
::
unordered_map
<
Key
,
String
>
map
;
const
bool
allow_read_expired_keys_from_cache_dictionary
=
getAllowReadExpiredKeysSetting
();
size_t
total_length
=
0
;
size_t
cache_expired
=
0
,
cache_not_found
=
0
,
cache_hit
=
0
;
{
...
...
@@ -177,25 +180,42 @@ void CacheDictionary::getItemsString(
const
auto
id
=
ids
[
row
];
const
auto
find_result
=
findCellIdx
(
id
,
now
);
auto
insert_value_routine
=
[
&
]()
{
const
auto
&
cell_idx
=
find_result
.
cell_idx
;
const
auto
&
cell
=
cells
[
cell_idx
];
const
auto
string_ref
=
cell
.
isDefault
()
?
get_default
(
row
)
:
attribute_array
[
cell_idx
];
if
(
!
cell
.
isDefault
())
map
[
id
]
=
String
{
string_ref
};
total_length
+=
string_ref
.
size
+
1
;
};
if
(
!
find_result
.
valid
)
{
outdated_ids
[
id
].
push_back
(
row
);
if
(
find_result
.
outdated
)
{
cache_expired_ids
[
id
].
push_back
(
row
);
++
cache_expired
;
if
(
allow_read_expired_keys_from_cache_dictionary
)
{
insert_value_routine
();
}
}
else
{
cache_not_found_ids
[
id
].
push_back
(
row
);
++
cache_not_found
;
}
}
else
{
++
cache_hit
;
const
auto
&
cell_idx
=
find_result
.
cell_idx
;
const
auto
&
cell
=
cells
[
cell_idx
];
const
auto
string_ref
=
cell
.
isDefault
()
?
get_default
(
row
)
:
attribute_array
[
cell_idx
];
if
(
!
cell
.
isDefault
())
map
[
id
]
=
String
{
string_ref
};
total_length
+=
string_ref
.
size
+
1
;
insert_value_routine
();
}
}
}
...
...
@@ -205,28 +225,67 @@ void CacheDictionary::getItemsString(
ProfileEvents
::
increment
(
ProfileEvents
::
DictCacheKeysHit
,
cache_hit
);
query_count
.
fetch_add
(
rows
,
std
::
memory_order_relaxed
);
hit_count
.
fetch_add
(
rows
-
outdated_ids
.
size
(),
std
::
memory_order_release
);
const
size_t
outdated_ids_count
=
cache_expired
+
cache_not_found
;
hit_count
.
fetch_add
(
rows
-
outdated_ids_count
,
std
::
memory_order_release
);
if
(
!
cache_expired_ids
.
empty
())
{
std
::
vector
<
Key
>
required_expired_ids
(
cache_not_found_ids
.
size
());
std
::
transform
(
std
::
begin
(
cache_expired_ids
),
std
::
end
(
cache_expired_ids
),
std
::
begin
(
required_expired_ids
),
[](
auto
&
pair
)
{
return
pair
.
first
;
});
if
(
allow_read_expired_keys_from_cache_dictionary
)
{
UpdateUnit
update_unit
{
required_expired_ids
,
[
&
](
const
auto
,
const
auto
)
{},
[
&
](
const
auto
,
const
auto
)
{}};
if
(
!
update_queue
.
tryPush
(
update_unit
,
update_queue_push_timeout_milliseconds
))
throw
std
::
runtime_error
(
"Too many updates"
);
}
else
{
UpdateUnit
update_unit
{
required_expired_ids
,
[
&
](
const
auto
id
,
const
auto
cell_idx
)
{
const
auto
attribute_value
=
attribute_array
[
cell_idx
];
map
[
id
]
=
String
{
attribute_value
};
total_length
+=
(
attribute_value
.
size
+
1
)
*
cache_not_found_ids
[
id
].
size
();
},
[
&
](
const
auto
id
,
const
auto
)
{
for
(
const
auto
row
:
cache_not_found_ids
[
id
])
total_length
+=
get_default
(
row
).
size
+
1
;
}};
if
(
!
update_queue
.
tryPush
(
update_unit
,
update_queue_push_timeout_milliseconds
))
throw
std
::
runtime_error
(
"Too many updates"
);
waitForCurrentUpdateFinish
();
}
}
/// request new values
if
(
!
outdate
d_ids
.
empty
())
if
(
!
cache_not_foun
d_ids
.
empty
())
{
std
::
vector
<
Key
>
required_ids
(
outdate
d_ids
.
size
());
std
::
transform
(
std
::
begin
(
outdated_ids
),
std
::
end
(
outdate
d_ids
),
std
::
begin
(
required_ids
),
[](
auto
&
pair
)
{
return
pair
.
first
;
});
std
::
vector
<
Key
>
required_ids
(
cache_not_foun
d_ids
.
size
());
std
::
transform
(
std
::
begin
(
cache_not_found_ids
),
std
::
end
(
cache_not_foun
d_ids
),
std
::
begin
(
required_ids
),
[](
auto
&
pair
)
{
return
pair
.
first
;
});
update
(
UpdateUnit
update_unit
{
required_ids
,
[
&
](
const
auto
id
,
const
auto
cell_idx
)
{
const
auto
attribute_value
=
attribute_array
[
cell_idx
];
map
[
id
]
=
String
{
attribute_value
};
total_length
+=
(
attribute_value
.
size
+
1
)
*
outdate
d_ids
[
id
].
size
();
total_length
+=
(
attribute_value
.
size
+
1
)
*
cache_not_foun
d_ids
[
id
].
size
();
},
[
&
](
const
auto
id
,
const
auto
)
{
for
(
const
auto
row
:
outdate
d_ids
[
id
])
for
(
const
auto
row
:
cache_not_foun
d_ids
[
id
])
total_length
+=
get_default
(
row
).
size
+
1
;
});
}};
if
(
!
update_queue
.
tryPush
(
update_unit
,
update_queue_push_timeout_milliseconds
))
throw
std
::
runtime_error
(
"Too many updates"
);
}
out
->
getChars
().
reserve
(
total_length
);
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录