mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge remote-tracking branch 'origin/master' into HEAD
This commit is contained in:
commit
1d0b61dd0e
@ -13,3 +13,6 @@ ClickHouse® is an open-source column-oriented database management system that a
|
||||
* [Code Browser](https://clickhouse.tech/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation.
|
||||
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
|
||||
* You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person.
|
||||
|
||||
## Upcoming Events
|
||||
* [SF Bay Area ClickHouse Community Meetup (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/278144089/) on 16 June 2021.
|
||||
|
@ -552,6 +552,63 @@ create table query_metric_stats_denorm engine File(TSVWithNamesAndTypes,
|
||||
order by test, query_index, metric_name
|
||||
;
|
||||
" 2> >(tee -a analyze/errors.log 1>&2)
|
||||
|
||||
# Fetch historical query variability thresholds from the CI database
|
||||
clickhouse-local --query "
|
||||
left join file('analyze/report-thresholds.tsv', TSV,
|
||||
'test text, report_threshold float') thresholds
|
||||
on query_metric_stats.test = thresholds.test
|
||||
"
|
||||
|
||||
if [ -v CHPC_DATABASE_URL ]
|
||||
then
|
||||
set +x # Don't show password in the log
|
||||
client=(clickhouse-client
|
||||
# Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000
|
||||
# so I have to extract host and port with clickhouse-local. I tried to use
|
||||
# Poco URI parser to support this in the client, but it's broken and can't
|
||||
# parse host:port.
|
||||
$(clickhouse-local --query "with '${CHPC_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV")
|
||||
--secure
|
||||
--user "${CHPC_DATABASE_USER}"
|
||||
--password "${CHPC_DATABASE_PASSWORD}"
|
||||
--config "right/config/client_config.xml"
|
||||
--database perftest
|
||||
--date_time_input_format=best_effort)
|
||||
|
||||
|
||||
# Precision is going to be 1.5 times worse for PRs. How do I know it? I ran this:
|
||||
# SELECT quantilesExact(0., 0.1, 0.5, 0.75, 0.95, 1.)(p / m)
|
||||
# FROM
|
||||
# (
|
||||
# SELECT
|
||||
# quantileIf(0.95)(stat_threshold, pr_number = 0) AS m,
|
||||
# quantileIf(0.95)(stat_threshold, (pr_number != 0) AND (abs(diff) < stat_threshold)) AS p
|
||||
# FROM query_metrics_v2
|
||||
# WHERE (event_date > (today() - toIntervalMonth(1))) AND (metric = 'client_time')
|
||||
# GROUP BY
|
||||
# test,
|
||||
# query_index,
|
||||
# query_display_name
|
||||
# HAVING count(*) > 100
|
||||
# )
|
||||
# The file can be empty if the server is inaccessible, so we can't use TSVWithNamesAndTypes.
|
||||
"${client[@]}" --query "
|
||||
select test, query_index,
|
||||
quantileExact(0.99)(abs(diff)) max_diff,
|
||||
quantileExactIf(0.99)(stat_threshold, abs(diff) < stat_threshold) * 1.5 max_stat_threshold,
|
||||
query_display_name
|
||||
from query_metrics_v2
|
||||
where event_date > now() - interval 1 month
|
||||
and metric = 'client_time'
|
||||
and pr_number = 0
|
||||
group by test, query_index, query_display_name
|
||||
having count(*) > 100
|
||||
" > analyze/historical-thresholds.tsv
|
||||
else
|
||||
touch analyze/historical-thresholds.tsv
|
||||
fi
|
||||
|
||||
}
|
||||
|
||||
# Analyze results
|
||||
@ -596,6 +653,26 @@ create view query_metric_stats as
|
||||
diff float, stat_threshold float')
|
||||
;
|
||||
|
||||
create table report_thresholds engine File(TSVWithNamesAndTypes, 'report/thresholds.tsv')
|
||||
as select
|
||||
query_display_names.test test, query_display_names.query_index query_index,
|
||||
ceil(greatest(0.1, historical_thresholds.max_diff,
|
||||
test_thresholds.report_threshold), 2) changed_threshold,
|
||||
ceil(greatest(0.2, historical_thresholds.max_stat_threshold,
|
||||
test_thresholds.report_threshold + 0.1), 2) unstable_threshold,
|
||||
query_display_names.query_display_name query_display_name
|
||||
from query_display_names
|
||||
left join file('analyze/historical-thresholds.tsv', TSV,
|
||||
'test text, query_index int, max_diff float, max_stat_threshold float,
|
||||
query_display_name text') historical_thresholds
|
||||
on query_display_names.test = historical_thresholds.test
|
||||
and query_display_names.query_index = historical_thresholds.query_index
|
||||
and query_display_names.query_display_name = historical_thresholds.query_display_name
|
||||
left join file('analyze/report-thresholds.tsv', TSV,
|
||||
'test text, report_threshold float') test_thresholds
|
||||
on query_display_names.test = test_thresholds.test
|
||||
;
|
||||
|
||||
-- Main statistics for queries -- query time as reported in query log.
|
||||
create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv')
|
||||
as select
|
||||
@ -610,23 +687,23 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv')
|
||||
-- uncaught regressions, because for the default 7 runs we do for PRs,
|
||||
-- the randomization distribution has only 16 values, so the max quantile
|
||||
-- is actually 0.9375.
|
||||
abs(diff) > report_threshold and abs(diff) >= stat_threshold as changed_fail,
|
||||
abs(diff) > report_threshold - 0.05 and abs(diff) >= stat_threshold as changed_show,
|
||||
abs(diff) > changed_threshold and abs(diff) >= stat_threshold as changed_fail,
|
||||
abs(diff) > changed_threshold - 0.05 and abs(diff) >= stat_threshold as changed_show,
|
||||
|
||||
not changed_fail and stat_threshold > report_threshold + 0.10 as unstable_fail,
|
||||
not changed_show and stat_threshold > report_threshold - 0.05 as unstable_show,
|
||||
not changed_fail and stat_threshold > unstable_threshold as unstable_fail,
|
||||
not changed_show and stat_threshold > unstable_threshold - 0.05 as unstable_show,
|
||||
|
||||
left, right, diff, stat_threshold,
|
||||
if(report_threshold > 0, report_threshold, 0.10) as report_threshold,
|
||||
query_metric_stats.test test, query_metric_stats.query_index query_index,
|
||||
query_display_name
|
||||
query_display_names.query_display_name query_display_name
|
||||
from query_metric_stats
|
||||
left join file('analyze/report-thresholds.tsv', TSV,
|
||||
'test text, report_threshold float') thresholds
|
||||
on query_metric_stats.test = thresholds.test
|
||||
left join query_display_names
|
||||
on query_metric_stats.test = query_display_names.test
|
||||
and query_metric_stats.query_index = query_display_names.query_index
|
||||
left join report_thresholds
|
||||
on query_display_names.test = report_thresholds.test
|
||||
and query_display_names.query_index = report_thresholds.query_index
|
||||
and query_display_names.query_display_name = report_thresholds.query_display_name
|
||||
-- 'server_time' is rounded down to ms, which might be bad for very short queries.
|
||||
-- Use 'client_time' instead.
|
||||
where metric_name = 'client_time'
|
||||
@ -889,7 +966,6 @@ create table all_query_metrics_tsv engine File(TSV, 'report/all-query-metrics.ts
|
||||
order by test, query_index;
|
||||
" 2> >(tee -a report/errors.log 1>&2)
|
||||
|
||||
|
||||
# Prepare source data for metrics and flamegraphs for queries that were profiled
|
||||
# by perf.py.
|
||||
for version in {right,left}
|
||||
|
@ -453,7 +453,10 @@ if args.report == 'main':
|
||||
text += tableRow(r, attrs, anchor)
|
||||
|
||||
text += tableEnd()
|
||||
tables.append(text)
|
||||
|
||||
# Don't add an empty table.
|
||||
if very_unstable_queries:
|
||||
tables.append(text)
|
||||
|
||||
add_unstable_queries()
|
||||
|
||||
@ -552,13 +555,13 @@ if args.report == 'main':
|
||||
message_array.append(str(slower_queries) + ' slower')
|
||||
|
||||
if unstable_partial_queries:
|
||||
unstable_queries += unstable_partial_queries
|
||||
error_tests += unstable_partial_queries
|
||||
very_unstable_queries += unstable_partial_queries
|
||||
status = 'failure'
|
||||
|
||||
# Don't show mildly unstable queries, only the very unstable ones we
|
||||
# treat as errors.
|
||||
if very_unstable_queries:
|
||||
error_tests += very_unstable_queries
|
||||
status = 'failure'
|
||||
message_array.append(str(very_unstable_queries) + ' unstable')
|
||||
|
||||
|
@ -73,4 +73,4 @@ RUN set -x \
|
||||
VOLUME /var/lib/docker
|
||||
EXPOSE 2375
|
||||
ENTRYPOINT ["dockerd-entrypoint.sh"]
|
||||
CMD ["sh", "-c", "python3 regression.py --no-color -o classic --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS}; cat test.log | tfs report results --format json > results.json; /usr/local/bin/process_testflows_result.py || echo -e 'failure\tCannot parse results' > check_status.tsv"]
|
||||
CMD ["sh", "-c", "python3 regression.py --no-color -o new-fails --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS}; cat test.log | tfs report results --format json > results.json; /usr/local/bin/process_testflows_result.py || echo -e 'failure\tCannot parse results' > check_status.tsv; find * -type f | grep _instances | grep clickhouse-server | xargs -n1 tar -rvf clickhouse_logs.tar; gzip -9 clickhouse_logs.tar"]
|
||||
|
@ -338,7 +338,7 @@ or
|
||||
|
||||
``` sql
|
||||
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
|
||||
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict))
|
||||
PATH ./user_files/test_dict))
|
||||
```
|
||||
|
||||
### complex_key_ssd_cache {#complex-key-ssd-cache}
|
||||
|
@ -53,7 +53,7 @@ optional settings are available:
|
||||
or
|
||||
|
||||
``` sql
|
||||
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
|
||||
SETTINGS(format_csv_allow_single_quotes = 0)
|
||||
```
|
||||
|
||||
@ -88,7 +88,7 @@ Example of settings:
|
||||
or
|
||||
|
||||
``` sql
|
||||
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
|
||||
```
|
||||
|
||||
Setting fields:
|
||||
|
@ -199,7 +199,7 @@ Provides possibility to start background move data according to [TTL table expre
|
||||
Returns `Ok.` even if table doesn’t exist. Returns error when database doesn’t exist:
|
||||
|
||||
``` sql
|
||||
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
|
||||
SYSTEM START MOVES [[db.]merge_tree_family_table_name]
|
||||
```
|
||||
|
||||
## Managing ReplicatedMergeTree Tables {#query-language-system-replicated}
|
||||
|
@ -55,7 +55,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
|
||||
または
|
||||
|
||||
``` sql
|
||||
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
|
||||
SETTINGS(format_csv_allow_single_quotes = 0)
|
||||
```
|
||||
|
||||
@ -87,7 +87,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
|
||||
または
|
||||
|
||||
``` sql
|
||||
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
|
||||
```
|
||||
|
||||
フィールドの設定:
|
||||
|
@ -338,7 +338,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000))
|
||||
|
||||
``` sql
|
||||
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
|
||||
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict))
|
||||
PATH ./user_files/test_dict))
|
||||
```
|
||||
|
||||
### complex_key_ssd_cache {#complex-key-ssd-cache}
|
||||
|
@ -53,7 +53,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
|
||||
или
|
||||
|
||||
``` sql
|
||||
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
|
||||
SETTINGS(format_csv_allow_single_quotes = 0)
|
||||
```
|
||||
|
||||
@ -87,7 +87,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
|
||||
или
|
||||
|
||||
``` sql
|
||||
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
|
||||
```
|
||||
|
||||
Поля настройки:
|
||||
|
@ -55,7 +55,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
|
||||
或
|
||||
|
||||
``` sql
|
||||
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
|
||||
SETTINGS(format_csv_allow_single_quotes = 0)
|
||||
```
|
||||
|
||||
@ -87,7 +87,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
|
||||
或
|
||||
|
||||
``` sql
|
||||
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
|
||||
```
|
||||
|
||||
设置字段:
|
||||
|
@ -16,10 +16,13 @@ namespace ErrorCodes
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
struct Settings;
|
||||
|
||||
namespace
|
||||
{
|
||||
template <template <typename> class Data>
|
||||
AggregateFunctionPtr createAggregateFunctionSegmentLengthSum(const std::string & name, const DataTypes & arguments, const Array &)
|
||||
AggregateFunctionPtr
|
||||
createAggregateFunctionSegmentLengthSum(const std::string & name, const DataTypes & arguments, const Array &, const Settings *)
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception(
|
||||
|
@ -27,30 +27,45 @@ namespace CurrentMemoryTracker
|
||||
|
||||
using DB::current_thread;
|
||||
|
||||
void alloc(Int64 size)
|
||||
namespace
|
||||
{
|
||||
if (auto * memory_tracker = getMemoryTracker())
|
||||
void allocImpl(Int64 size, bool throw_if_memory_exceeded)
|
||||
{
|
||||
if (current_thread)
|
||||
if (auto * memory_tracker = getMemoryTracker())
|
||||
{
|
||||
current_thread->untracked_memory += size;
|
||||
if (current_thread->untracked_memory > current_thread->untracked_memory_limit)
|
||||
if (current_thread)
|
||||
{
|
||||
/// Zero untracked before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes
|
||||
/// more. It could be useful to enlarge Exception message in rethrow logic.
|
||||
Int64 tmp = current_thread->untracked_memory;
|
||||
current_thread->untracked_memory = 0;
|
||||
memory_tracker->alloc(tmp);
|
||||
current_thread->untracked_memory += size;
|
||||
if (current_thread->untracked_memory > current_thread->untracked_memory_limit)
|
||||
{
|
||||
/// Zero untracked before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes
|
||||
/// more. It could be useful to enlarge Exception message in rethrow logic.
|
||||
Int64 tmp = current_thread->untracked_memory;
|
||||
current_thread->untracked_memory = 0;
|
||||
memory_tracker->allocImpl(tmp, throw_if_memory_exceeded);
|
||||
}
|
||||
}
|
||||
/// total_memory_tracker only, ignore untracked_memory
|
||||
else
|
||||
{
|
||||
memory_tracker->allocImpl(size, throw_if_memory_exceeded);
|
||||
}
|
||||
}
|
||||
/// total_memory_tracker only, ignore untracked_memory
|
||||
else
|
||||
{
|
||||
memory_tracker->alloc(size);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void alloc(Int64 size)
|
||||
{
|
||||
bool throw_if_memory_exceeded = true;
|
||||
allocImpl(size, throw_if_memory_exceeded);
|
||||
}
|
||||
|
||||
void allocNoThrow(Int64 size)
|
||||
{
|
||||
bool throw_if_memory_exceeded = false;
|
||||
allocImpl(size, throw_if_memory_exceeded);
|
||||
}
|
||||
|
||||
void realloc(Int64 old_size, Int64 new_size)
|
||||
{
|
||||
Int64 addition = new_size - old_size;
|
||||
|
@ -6,6 +6,7 @@
|
||||
namespace CurrentMemoryTracker
|
||||
{
|
||||
void alloc(Int64 size);
|
||||
void allocNoThrow(Int64 size);
|
||||
void realloc(Int64 old_size, Int64 new_size);
|
||||
void free(Int64 size);
|
||||
}
|
||||
|
@ -128,7 +128,7 @@ void MemoryTracker::logMemoryUsage(Int64 current) const
|
||||
}
|
||||
|
||||
|
||||
void MemoryTracker::alloc(Int64 size)
|
||||
void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded)
|
||||
{
|
||||
if (size < 0)
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Negative size ({}) is passed to MemoryTracker. It is a bug.", size);
|
||||
@ -137,7 +137,7 @@ void MemoryTracker::alloc(Int64 size)
|
||||
{
|
||||
/// Since the BlockerInThread should respect the level, we should go to the next parent.
|
||||
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
||||
loaded_next->alloc(size);
|
||||
loaded_next->allocImpl(size, throw_if_memory_exceeded);
|
||||
return;
|
||||
}
|
||||
|
||||
@ -173,7 +173,7 @@ void MemoryTracker::alloc(Int64 size)
|
||||
}
|
||||
|
||||
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
|
||||
if (unlikely(_memory_tracker_always_throw_logical_error_on_allocation))
|
||||
if (unlikely(_memory_tracker_always_throw_logical_error_on_allocation) && throw_if_memory_exceeded)
|
||||
{
|
||||
_memory_tracker_always_throw_logical_error_on_allocation = false;
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Memory tracker: allocations not allowed.");
|
||||
@ -181,21 +181,28 @@ void MemoryTracker::alloc(Int64 size)
|
||||
#endif
|
||||
|
||||
std::bernoulli_distribution fault(fault_probability);
|
||||
if (unlikely(fault_probability && fault(thread_local_rng)) && memoryTrackerCanThrow(level, true))
|
||||
if (unlikely(fault_probability && fault(thread_local_rng)) && memoryTrackerCanThrow(level, true) && throw_if_memory_exceeded)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
|
||||
amount.fetch_sub(size, std::memory_order_relaxed);
|
||||
|
||||
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
||||
BlockerInThread untrack_lock(VariableContext::Global);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
|
||||
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
||||
amount.fetch_sub(size, std::memory_order_relaxed);
|
||||
throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
||||
"Memory tracker{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
|
||||
description ? " " : "", description ? description : "",
|
||||
formatReadableSizeWithBinarySuffix(will_be),
|
||||
size, formatReadableSizeWithBinarySuffix(current_hard_limit));
|
||||
throw DB::Exception(
|
||||
DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
||||
"Memory tracker{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
|
||||
description ? " " : "",
|
||||
description ? description : "",
|
||||
formatReadableSizeWithBinarySuffix(will_be),
|
||||
size,
|
||||
formatReadableSizeWithBinarySuffix(current_hard_limit));
|
||||
}
|
||||
|
||||
|
||||
if (unlikely(current_profiler_limit && will_be > current_profiler_limit))
|
||||
{
|
||||
BlockerInThread untrack_lock(VariableContext::Global);
|
||||
@ -210,36 +217,59 @@ void MemoryTracker::alloc(Int64 size)
|
||||
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), size);
|
||||
}
|
||||
|
||||
if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false))
|
||||
if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded)
|
||||
{
|
||||
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
||||
BlockerInThread untrack_lock(VariableContext::Global);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
|
||||
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
||||
amount.fetch_sub(size, std::memory_order_relaxed);
|
||||
throw DB::Exception(DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
||||
"Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
|
||||
description ? " " : "", description ? description : "",
|
||||
formatReadableSizeWithBinarySuffix(will_be),
|
||||
size, formatReadableSizeWithBinarySuffix(current_hard_limit));
|
||||
throw DB::Exception(
|
||||
DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
|
||||
"Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
|
||||
description ? " " : "",
|
||||
description ? description : "",
|
||||
formatReadableSizeWithBinarySuffix(will_be),
|
||||
size,
|
||||
formatReadableSizeWithBinarySuffix(current_hard_limit));
|
||||
}
|
||||
|
||||
updatePeak(will_be);
|
||||
if (throw_if_memory_exceeded)
|
||||
{
|
||||
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
|
||||
BlockerInThread untrack_lock(VariableContext::Global);
|
||||
bool log_memory_usage = true;
|
||||
updatePeak(will_be, log_memory_usage);
|
||||
}
|
||||
else
|
||||
{
|
||||
bool log_memory_usage = false;
|
||||
updatePeak(will_be, log_memory_usage);
|
||||
}
|
||||
|
||||
if (auto * loaded_next = parent.load(std::memory_order_relaxed))
|
||||
loaded_next->alloc(size);
|
||||
loaded_next->allocImpl(size, throw_if_memory_exceeded);
|
||||
}
|
||||
|
||||
void MemoryTracker::alloc(Int64 size)
|
||||
{
|
||||
bool throw_if_memory_exceeded = true;
|
||||
allocImpl(size, throw_if_memory_exceeded);
|
||||
}
|
||||
|
||||
void MemoryTracker::updatePeak(Int64 will_be)
|
||||
void MemoryTracker::allocNoThrow(Int64 size)
|
||||
{
|
||||
bool throw_if_memory_exceeded = false;
|
||||
allocImpl(size, throw_if_memory_exceeded);
|
||||
}
|
||||
|
||||
void MemoryTracker::updatePeak(Int64 will_be, bool log_memory_usage)
|
||||
{
|
||||
auto peak_old = peak.load(std::memory_order_relaxed);
|
||||
if (will_be > peak_old) /// Races doesn't matter. Could rewrite with CAS, but not worth.
|
||||
{
|
||||
peak.store(will_be, std::memory_order_relaxed);
|
||||
|
||||
if ((level == VariableContext::Process || level == VariableContext::Global)
|
||||
if (log_memory_usage && (level == VariableContext::Process || level == VariableContext::Global)
|
||||
&& will_be / log_peak_memory_usage_every > peak_old / log_peak_memory_usage_every)
|
||||
logMemoryUsage(will_be);
|
||||
}
|
||||
@ -317,7 +347,9 @@ void MemoryTracker::reset()
|
||||
void MemoryTracker::set(Int64 to)
|
||||
{
|
||||
amount.store(to, std::memory_order_relaxed);
|
||||
updatePeak(to);
|
||||
|
||||
bool log_memory_usage = true;
|
||||
updatePeak(to, log_memory_usage);
|
||||
}
|
||||
|
||||
|
||||
|
@ -58,7 +58,7 @@ private:
|
||||
/// This description will be used as prefix into log messages (if isn't nullptr)
|
||||
std::atomic<const char *> description_ptr = nullptr;
|
||||
|
||||
void updatePeak(Int64 will_be);
|
||||
void updatePeak(Int64 will_be, bool log_memory_usage);
|
||||
void logMemoryUsage(Int64 current) const;
|
||||
|
||||
public:
|
||||
@ -73,6 +73,10 @@ public:
|
||||
*/
|
||||
void alloc(Int64 size);
|
||||
|
||||
void allocNoThrow(Int64 size);
|
||||
|
||||
void allocImpl(Int64 size, bool throw_if_memory_exceeded);
|
||||
|
||||
void realloc(Int64 old_size, Int64 new_size)
|
||||
{
|
||||
Int64 addition = new_size - old_size;
|
||||
|
@ -113,12 +113,22 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
|
||||
if (shutdown)
|
||||
return on_error();
|
||||
|
||||
jobs.emplace(std::move(job), priority);
|
||||
++scheduled_jobs;
|
||||
/// We must not to allocate any memory after we emplaced a job in a queue.
|
||||
/// Because if an exception would be thrown, we won't notify a thread about job occurrence.
|
||||
|
||||
if (threads.size() < std::min(max_threads, scheduled_jobs))
|
||||
/// Check if there are enough threads to process job.
|
||||
if (threads.size() < std::min(max_threads, scheduled_jobs + 1))
|
||||
{
|
||||
threads.emplace_front();
|
||||
try
|
||||
{
|
||||
threads.emplace_front();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Most likely this is a std::bad_alloc exception
|
||||
return on_error();
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
threads.front() = Thread([this, it = threads.begin()] { worker(it); });
|
||||
@ -126,19 +136,15 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
|
||||
catch (...)
|
||||
{
|
||||
threads.pop_front();
|
||||
|
||||
/// Remove the job and return error to caller.
|
||||
/// Note that if we have allocated at least one thread, we may continue
|
||||
/// (one thread is enough to process all jobs).
|
||||
/// But this condition indicate an error nevertheless and better to refuse.
|
||||
|
||||
jobs.pop();
|
||||
--scheduled_jobs;
|
||||
return on_error();
|
||||
}
|
||||
}
|
||||
|
||||
jobs.emplace(std::move(job), priority);
|
||||
++scheduled_jobs;
|
||||
new_job_or_shutdown.notify_one();
|
||||
}
|
||||
new_job_or_shutdown.notify_one();
|
||||
|
||||
return ReturnType(true);
|
||||
}
|
||||
|
||||
@ -165,6 +171,10 @@ void ThreadPoolImpl<Thread>::wait()
|
||||
{
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
/// Signal here just in case.
|
||||
/// If threads are waiting on condition variables, but there are some jobs in the queue
|
||||
/// then it will prevent us from deadlock.
|
||||
new_job_or_shutdown.notify_all();
|
||||
job_finished.wait(lock, [this] { return scheduled_jobs == 0; });
|
||||
|
||||
if (first_exception)
|
||||
|
@ -227,7 +227,7 @@ struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest
|
||||
void readImpl(ReadBuffer & in) override;
|
||||
|
||||
ZooKeeperResponsePtr makeResponse() const override;
|
||||
bool isReadRequest() const override { return !has_watch; }
|
||||
bool isReadRequest() const override { return true; }
|
||||
|
||||
size_t bytesSize() const override { return ExistsRequest::bytesSize() + sizeof(xid) + sizeof(has_watch); }
|
||||
};
|
||||
@ -248,7 +248,7 @@ struct ZooKeeperGetRequest final : GetRequest, ZooKeeperRequest
|
||||
void readImpl(ReadBuffer & in) override;
|
||||
|
||||
ZooKeeperResponsePtr makeResponse() const override;
|
||||
bool isReadRequest() const override { return !has_watch; }
|
||||
bool isReadRequest() const override { return true; }
|
||||
|
||||
size_t bytesSize() const override { return GetRequest::bytesSize() + sizeof(xid) + sizeof(has_watch); }
|
||||
};
|
||||
@ -291,7 +291,7 @@ struct ZooKeeperListRequest : ListRequest, ZooKeeperRequest
|
||||
void writeImpl(WriteBuffer & out) const override;
|
||||
void readImpl(ReadBuffer & in) override;
|
||||
ZooKeeperResponsePtr makeResponse() const override;
|
||||
bool isReadRequest() const override { return !has_watch; }
|
||||
bool isReadRequest() const override { return true; }
|
||||
|
||||
size_t bytesSize() const override { return ListRequest::bytesSize() + sizeof(xid) + sizeof(has_watch); }
|
||||
};
|
||||
@ -325,7 +325,7 @@ struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest
|
||||
void readImpl(ReadBuffer & in) override;
|
||||
|
||||
ZooKeeperResponsePtr makeResponse() const override;
|
||||
bool isReadRequest() const override { return !has_watch; }
|
||||
bool isReadRequest() const override { return true; }
|
||||
|
||||
size_t bytesSize() const override { return CheckRequest::bytesSize() + sizeof(xid) + sizeof(has_watch); }
|
||||
};
|
||||
|
@ -20,6 +20,7 @@ namespace ErrorCodes
|
||||
extern const int SYSTEM_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int CANNOT_STATVFS;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
|
||||
@ -108,4 +109,23 @@ String getFilesystemName([[maybe_unused]] const String & mount_point)
|
||||
#endif
|
||||
}
|
||||
|
||||
bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path)
|
||||
{
|
||||
auto absolute_path = std::filesystem::weakly_canonical(path);
|
||||
auto absolute_prefix_path = std::filesystem::weakly_canonical(prefix_path);
|
||||
|
||||
auto [_, prefix_path_mismatch_it] = std::mismatch(absolute_path.begin(), absolute_path.end(), absolute_prefix_path.begin(), absolute_prefix_path.end());
|
||||
|
||||
bool path_starts_with_prefix_path = (prefix_path_mismatch_it == absolute_prefix_path.end());
|
||||
return path_starts_with_prefix_path;
|
||||
}
|
||||
|
||||
bool pathStartsWith(const String & path, const String & prefix_path)
|
||||
{
|
||||
auto filesystem_path = std::filesystem::path(path);
|
||||
auto filesystem_prefix_path = std::filesystem::path(prefix_path);
|
||||
|
||||
return pathStartsWith(filesystem_path, filesystem_prefix_path);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -29,4 +29,10 @@ String getFilesystemName([[maybe_unused]] const String & mount_point);
|
||||
|
||||
struct statvfs getStatVFS(const String & path);
|
||||
|
||||
/// Returns true if path starts with prefix path
|
||||
bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path);
|
||||
|
||||
/// Returns true if path starts with prefix path
|
||||
bool pathStartsWith(const String & path, const String & prefix_path);
|
||||
|
||||
}
|
||||
|
@ -41,9 +41,9 @@ struct InitializeJemallocZoneAllocatorForOSX
|
||||
namespace Memory
|
||||
{
|
||||
|
||||
inline ALWAYS_INLINE void trackMemory(std::size_t size)
|
||||
inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size)
|
||||
{
|
||||
std::size_t actual_size = size;
|
||||
size_t actual_size = size;
|
||||
|
||||
#if USE_JEMALLOC && JEMALLOC_VERSION_MAJOR >= 5
|
||||
/// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function
|
||||
@ -52,21 +52,13 @@ inline ALWAYS_INLINE void trackMemory(std::size_t size)
|
||||
actual_size = nallocx(size, 0);
|
||||
#endif
|
||||
|
||||
CurrentMemoryTracker::alloc(actual_size);
|
||||
return actual_size;
|
||||
}
|
||||
|
||||
inline ALWAYS_INLINE bool trackMemoryNoExcept(std::size_t size) noexcept
|
||||
inline ALWAYS_INLINE void trackMemory(std::size_t size)
|
||||
{
|
||||
try
|
||||
{
|
||||
trackMemory(size);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
std::size_t actual_size = getActualAllocationSize(size);
|
||||
CurrentMemoryTracker::allocNoThrow(actual_size);
|
||||
}
|
||||
|
||||
inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0) noexcept
|
||||
@ -98,27 +90,29 @@ inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t
|
||||
void * operator new(std::size_t size)
|
||||
{
|
||||
Memory::trackMemory(size);
|
||||
|
||||
return Memory::newImpl(size);
|
||||
}
|
||||
|
||||
void * operator new[](std::size_t size)
|
||||
{
|
||||
Memory::trackMemory(size);
|
||||
|
||||
return Memory::newImpl(size);
|
||||
}
|
||||
|
||||
void * operator new(std::size_t size, const std::nothrow_t &) noexcept
|
||||
{
|
||||
if (likely(Memory::trackMemoryNoExcept(size)))
|
||||
return Memory::newNoExept(size);
|
||||
return nullptr;
|
||||
Memory::trackMemory(size);
|
||||
|
||||
return Memory::newNoExept(size);
|
||||
}
|
||||
|
||||
void * operator new[](std::size_t size, const std::nothrow_t &) noexcept
|
||||
{
|
||||
if (likely(Memory::trackMemoryNoExcept(size)))
|
||||
return Memory::newNoExept(size);
|
||||
return nullptr;
|
||||
Memory::trackMemory(size);
|
||||
|
||||
return Memory::newNoExept(size);
|
||||
}
|
||||
|
||||
/// delete
|
||||
|
@ -18,7 +18,6 @@ PEERDIR(
|
||||
contrib/libs/openssl
|
||||
contrib/libs/poco/NetSSL_OpenSSL
|
||||
contrib/libs/re2
|
||||
contrib/libs/cxxsupp/libcxxabi-parts
|
||||
contrib/restricted/dragonbox
|
||||
)
|
||||
|
||||
|
@ -17,7 +17,6 @@ PEERDIR(
|
||||
contrib/libs/openssl
|
||||
contrib/libs/poco/NetSSL_OpenSSL
|
||||
contrib/libs/re2
|
||||
contrib/libs/cxxsupp/libcxxabi-parts
|
||||
contrib/restricted/dragonbox
|
||||
)
|
||||
|
||||
|
@ -19,7 +19,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory)
|
||||
[[maybe_unused]] Block & sample_block,
|
||||
ContextPtr /* context */,
|
||||
const std::string & /* default_database */,
|
||||
bool /*check_config*/) -> DictionarySourcePtr
|
||||
bool /*created_from_ddl*/) -> DictionarySourcePtr
|
||||
{
|
||||
#if USE_CASSANDRA
|
||||
setupCassandraDriverLibraryLogging(CASS_LOG_INFO);
|
||||
|
@ -211,7 +211,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
const std::string & default_database [[maybe_unused]],
|
||||
bool /* check_config */) -> DictionarySourcePtr
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr
|
||||
{
|
||||
bool secure = config.getBool(config_prefix + ".secure", false);
|
||||
auto context_copy = Context::createCopy(context);
|
||||
|
@ -17,22 +17,22 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
|
||||
}
|
||||
|
||||
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex)
|
||||
void DictionaryFactory::registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex)
|
||||
{
|
||||
if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second)
|
||||
auto it = registered_layouts.find(layout_type);
|
||||
if (it != registered_layouts.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "DictionaryFactory: the layout name '{}' is not unique", layout_type);
|
||||
|
||||
layout_complexity[layout_type] = is_complex;
|
||||
|
||||
RegisteredLayout layout { .layout_create_function = create_layout, .is_layout_complex = is_layout_complex };
|
||||
registered_layouts.emplace(layout_type, std::move(layout));
|
||||
}
|
||||
|
||||
|
||||
DictionaryPtr DictionaryFactory::create(
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context,
|
||||
bool check_source_config) const
|
||||
bool created_from_ddl) const
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
const auto & layout_prefix = config_prefix + ".layout";
|
||||
@ -45,7 +45,7 @@ DictionaryPtr DictionaryFactory::create(
|
||||
const DictionaryStructure dict_struct{config, config_prefix};
|
||||
|
||||
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(
|
||||
name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), check_source_config);
|
||||
name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), created_from_ddl);
|
||||
LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name);
|
||||
|
||||
if (context->hasQueryContext() && context->getSettingsRef().log_queries)
|
||||
@ -57,8 +57,8 @@ DictionaryPtr DictionaryFactory::create(
|
||||
const auto found = registered_layouts.find(layout_type);
|
||||
if (found != registered_layouts.end())
|
||||
{
|
||||
const auto & layout_creator = found->second;
|
||||
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
const auto & layout_creator = found->second.layout_create_function;
|
||||
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl);
|
||||
}
|
||||
}
|
||||
|
||||
@ -76,14 +76,16 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreat
|
||||
|
||||
bool DictionaryFactory::isComplex(const std::string & layout_type) const
|
||||
{
|
||||
auto found = layout_complexity.find(layout_type);
|
||||
auto it = registered_layouts.find(layout_type);
|
||||
|
||||
if (found != layout_complexity.end())
|
||||
return found->second;
|
||||
if (it == registered_layouts.end())
|
||||
{
|
||||
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
|
||||
"Unknown dictionary layout type: {}",
|
||||
layout_type);
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
|
||||
"Unknown dictionary layout type: {}",
|
||||
layout_type);
|
||||
return it->second.is_layout_complex;
|
||||
}
|
||||
|
||||
|
||||
|
@ -37,29 +37,36 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
ContextPtr context,
|
||||
bool check_source_config = false) const;
|
||||
bool created_from_ddl) const;
|
||||
|
||||
/// Create dictionary from DDL-query
|
||||
DictionaryPtr create(const std::string & name,
|
||||
const ASTCreateQuery & ast,
|
||||
ContextPtr context) const;
|
||||
|
||||
using Creator = std::function<DictionaryPtr(
|
||||
using LayoutCreateFunction = std::function<DictionaryPtr(
|
||||
const std::string & name,
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr)>;
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr context,
|
||||
bool created_from_ddl)>;
|
||||
|
||||
bool isComplex(const std::string & layout_type) const;
|
||||
|
||||
void registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex);
|
||||
void registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex);
|
||||
|
||||
private:
|
||||
using LayoutRegistry = std::unordered_map<std::string, Creator>;
|
||||
struct RegisteredLayout
|
||||
{
|
||||
LayoutCreateFunction layout_create_function;
|
||||
bool is_layout_complex;
|
||||
};
|
||||
|
||||
using LayoutRegistry = std::unordered_map<std::string, RegisteredLayout>;
|
||||
LayoutRegistry registered_layouts;
|
||||
using LayoutComplexity = std::unordered_map<std::string, bool>;
|
||||
LayoutComplexity layout_complexity;
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -305,7 +305,9 @@ namespace
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr)
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
bool /* created_from_ddl */)
|
||||
{
|
||||
const auto * layout_name = dictionary_key_type == DictionaryKeyType::simple ? "direct" : "complex_key_direct";
|
||||
|
||||
|
@ -20,6 +20,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static const UInt64 max_block_size = 8192;
|
||||
|
||||
namespace ErrorCodes
|
||||
@ -266,7 +267,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
bool created_from_ddl) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `executable` does not support attribute expressions");
|
||||
@ -274,7 +275,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
|
||||
/// Executable dictionaries may execute arbitrary commands.
|
||||
/// It's OK for dictionaries created by administrator from xml-file, but
|
||||
/// maybe dangerous for dictionaries created from DDL-queries.
|
||||
if (check_config)
|
||||
if (created_from_ddl)
|
||||
throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable dictionary source are not allowed to be created from DDL query");
|
||||
|
||||
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config);
|
||||
|
@ -275,7 +275,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
bool created_from_ddl) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `executable_pool` does not support attribute expressions");
|
||||
@ -283,7 +283,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
|
||||
/// Executable dictionaries may execute arbitrary commands.
|
||||
/// It's OK for dictionaries created by administrator from xml-file, but
|
||||
/// maybe dangerous for dictionaries created from DDL-queries.
|
||||
if (check_config)
|
||||
if (created_from_ddl)
|
||||
throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable pool dictionary source are not allowed to be created from DDL query");
|
||||
|
||||
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config);
|
||||
|
@ -1,13 +1,15 @@
|
||||
#include "FileDictionarySource.h"
|
||||
|
||||
#include <Poco/File.h>
|
||||
#include <filesystem>
|
||||
|
||||
#include <Poco/File.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <DataStreams/OwningBlockInputStream.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include "DictionarySourceFactory.h"
|
||||
#include "DictionaryStructure.h"
|
||||
#include "registerDictionaries.h"
|
||||
@ -26,28 +28,14 @@ namespace ErrorCodes
|
||||
|
||||
FileDictionarySource::FileDictionarySource(
|
||||
const std::string & filepath_, const std::string & format_,
|
||||
Block & sample_block_, ContextPtr context_, bool check_config)
|
||||
Block & sample_block_, ContextPtr context_, bool created_from_ddl)
|
||||
: filepath{filepath_}
|
||||
, format{format_}
|
||||
, sample_block{sample_block_}
|
||||
, context(context_)
|
||||
{
|
||||
if (check_config)
|
||||
{
|
||||
auto source_file_path = std::filesystem::path(filepath);
|
||||
auto source_file_absolute_path = std::filesystem::canonical(source_file_path);
|
||||
|
||||
String user_files_path_string_value = context->getUserFilesPath();
|
||||
auto user_files_path = std::filesystem::path(user_files_path_string_value);
|
||||
auto user_files_absolute_path = std::filesystem::canonical(user_files_path);
|
||||
|
||||
auto [_, user_files_absolute_path_mismatch_it] = std::mismatch(source_file_absolute_path.begin(), source_file_absolute_path.end(), user_files_absolute_path.begin(), user_files_absolute_path.end());
|
||||
|
||||
bool user_files_absolute_path_include_source_file_absolute_path = user_files_absolute_path_mismatch_it == user_files_absolute_path.end();
|
||||
|
||||
if (!user_files_absolute_path_include_source_file_absolute_path)
|
||||
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", filepath, user_files_path_string_value);
|
||||
}
|
||||
if (created_from_ddl && !pathStartsWith(filepath, context->getUserFilesPath()))
|
||||
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", filepath, context->getUserFilesPath());
|
||||
}
|
||||
|
||||
|
||||
@ -91,7 +79,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
bool created_from_ddl) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `file` does not support attribute expressions");
|
||||
@ -101,7 +89,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
|
||||
|
||||
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config);
|
||||
|
||||
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context_local_copy, check_config);
|
||||
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context_local_copy, created_from_ddl);
|
||||
};
|
||||
|
||||
factory.registerSource("file", create_table_source);
|
||||
|
@ -17,7 +17,7 @@ class FileDictionarySource final : public IDictionarySource
|
||||
{
|
||||
public:
|
||||
FileDictionarySource(const std::string & filepath_, const std::string & format_,
|
||||
Block & sample_block_, ContextPtr context_, bool check_config);
|
||||
Block & sample_block_, ContextPtr context_, bool created_from_ddl);
|
||||
|
||||
FileDictionarySource(const FileDictionarySource & other);
|
||||
|
||||
|
@ -502,10 +502,12 @@ BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_nam
|
||||
void registerDictionaryFlat(DictionaryFactory & factory)
|
||||
{
|
||||
auto create_layout = [=](const std::string & full_name,
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr) -> DictionaryPtr
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
bool /* created_from_ddl */) -> DictionaryPtr
|
||||
{
|
||||
if (dict_struct.key)
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout 'flat'");
|
||||
|
@ -22,6 +22,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
static const UInt64 max_block_size = 8192;
|
||||
|
||||
|
||||
@ -31,7 +32,7 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
bool check_config)
|
||||
bool created_from_ddl)
|
||||
: log(&Poco::Logger::get("HTTPDictionarySource"))
|
||||
, update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
@ -42,7 +43,7 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
, context(context_)
|
||||
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context))
|
||||
{
|
||||
if (check_config)
|
||||
if (created_from_ddl)
|
||||
context->getRemoteHostFilter().checkURL(Poco::URI(url));
|
||||
|
||||
const auto & credentials_prefix = config_prefix + ".credentials";
|
||||
@ -232,14 +233,14 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool check_config) -> DictionarySourcePtr {
|
||||
bool created_from_ddl) -> DictionarySourcePtr {
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `http` does not support attribute expressions");
|
||||
|
||||
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config);
|
||||
|
||||
return std::make_unique<HTTPDictionarySource>(
|
||||
dict_struct, config, config_prefix + ".http", sample_block, context_local_copy, check_config);
|
||||
dict_struct, config, config_prefix + ".http", sample_block, context_local_copy, created_from_ddl);
|
||||
};
|
||||
factory.registerSource("http", create_table_source);
|
||||
}
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
bool check_config);
|
||||
bool created_from_ddl);
|
||||
|
||||
HTTPDictionarySource(const HTTPDictionarySource & other);
|
||||
HTTPDictionarySource & operator=(const HTTPDictionarySource &) = delete;
|
||||
|
@ -751,13 +751,13 @@ void registerDictionaryHashed(DictionaryFactory & factory)
|
||||
using namespace std::placeholders;
|
||||
|
||||
factory.registerLayout("hashed",
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false);
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false);
|
||||
factory.registerLayout("sparse_hashed",
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false);
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false);
|
||||
factory.registerLayout("complex_key_hashed",
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true);
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true);
|
||||
factory.registerLayout("complex_key_sparse_hashed",
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true);
|
||||
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true);
|
||||
|
||||
}
|
||||
|
||||
|
@ -934,7 +934,9 @@ void registerDictionaryTrie(DictionaryFactory & factory)
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr) -> DictionaryPtr
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
bool /*created_from_ddl*/) -> DictionaryPtr
|
||||
{
|
||||
if (!dict_struct.key || dict_struct.key->size() != 1)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary of layout 'ip_trie' has to have one 'key'");
|
||||
|
@ -1,18 +1,18 @@
|
||||
#include "LibraryDictionarySource.h"
|
||||
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <ext/bit_cast.h>
|
||||
#include <ext/range.h>
|
||||
#include <ext/scope_guard.h>
|
||||
#include "DictionarySourceFactory.h"
|
||||
#include "DictionarySourceHelpers.h"
|
||||
#include "DictionaryStructure.h"
|
||||
#include "registerDictionaries.h"
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Dictionaries/DictionarySourceFactory.h>
|
||||
#include <Dictionaries/DictionarySourceHelpers.h>
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Dictionaries/registerDictionaries.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -32,7 +32,7 @@ LibraryDictionarySource::LibraryDictionarySource(
|
||||
const std::string & config_prefix_,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
bool check_config)
|
||||
bool created_from_ddl)
|
||||
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
||||
, dict_struct{dict_struct_}
|
||||
, config_prefix{config_prefix_}
|
||||
@ -41,13 +41,8 @@ LibraryDictionarySource::LibraryDictionarySource(
|
||||
, sample_block{sample_block_}
|
||||
, context(Context::createCopy(context_))
|
||||
{
|
||||
|
||||
if (check_config)
|
||||
{
|
||||
const String dictionaries_lib_path = context->getDictionariesLibPath();
|
||||
if (!startsWith(path, dictionaries_lib_path))
|
||||
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "LibraryDictionarySource: Library path {} is not inside {}", path, dictionaries_lib_path);
|
||||
}
|
||||
if (created_from_ddl && !pathStartsWith(path, context->getDictionariesLibPath()))
|
||||
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", path, context->getDictionariesLibPath());
|
||||
|
||||
if (!Poco::File(path).exists())
|
||||
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", Poco::File(path).path());
|
||||
@ -179,9 +174,9 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
bool created_from_ddl) -> DictionarySourcePtr
|
||||
{
|
||||
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);
|
||||
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, created_from_ddl);
|
||||
};
|
||||
|
||||
factory.registerSource("library", create_table_source);
|
||||
|
@ -40,7 +40,7 @@ public:
|
||||
const std::string & config_prefix_,
|
||||
Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
bool check_config);
|
||||
bool created_from_ddl);
|
||||
|
||||
LibraryDictionarySource(const LibraryDictionarySource & other);
|
||||
LibraryDictionarySource & operator=(const LibraryDictionarySource &) = delete;
|
||||
|
@ -15,7 +15,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
|
||||
Block & sample_block,
|
||||
ContextPtr,
|
||||
const std::string & /* default_database */,
|
||||
bool /* check_config */)
|
||||
bool /* created_from_ddl */)
|
||||
{
|
||||
const auto config_prefix = root_config_prefix + ".mongodb";
|
||||
return std::make_unique<MongoDBDictionarySource>(dict_struct,
|
||||
|
@ -26,7 +26,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
|
||||
[[maybe_unused]] Block & sample_block,
|
||||
[[maybe_unused]] ContextPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr {
|
||||
#if USE_MYSQL
|
||||
StreamSettings mysql_input_stream_settings(context->getSettingsRef()
|
||||
, config.getBool(config_prefix + ".mysql.close_connection", false) || config.getBool(config_prefix + ".mysql.share_connection", false)
|
||||
|
@ -166,7 +166,9 @@ DictionaryPtr createLayout(const std::string & ,
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr)
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
bool /*created_from_ddl*/)
|
||||
{
|
||||
const String database = config.getString(config_prefix + ".database", "");
|
||||
const String name = config.getString(config_prefix + ".name");
|
||||
|
@ -190,7 +190,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
|
||||
Block & sample_block,
|
||||
ContextPtr context,
|
||||
const std::string & /* default_database */,
|
||||
bool /* check_config */) -> DictionarySourcePtr
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr
|
||||
{
|
||||
#if USE_LIBPQXX
|
||||
const auto config_prefix = root_config_prefix + ".postgresql";
|
||||
|
@ -636,7 +636,9 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr) -> DictionaryPtr
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
bool /*created_from_ddl*/) -> DictionaryPtr
|
||||
{
|
||||
if (dict_struct.key)
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout 'range_hashed'");
|
||||
|
@ -14,7 +14,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory)
|
||||
Block & sample_block,
|
||||
ContextPtr /* context */,
|
||||
const std::string & /* default_database */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr {
|
||||
return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block);
|
||||
};
|
||||
factory.registerSource("redis", create_table_source);
|
||||
|
@ -53,7 +53,6 @@ struct SSDCacheDictionaryStorageConfiguration
|
||||
{
|
||||
const size_t strict_max_lifetime_seconds;
|
||||
const DictionaryLifetime lifetime;
|
||||
|
||||
const std::string file_path;
|
||||
const size_t max_partitions_count;
|
||||
const size_t block_size;
|
||||
|
@ -307,7 +307,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
|
||||
Block & /* sample_block */,
|
||||
ContextPtr /* context */,
|
||||
const std::string & /* default_database */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
bool /* created_from_ddl */) -> DictionarySourcePtr {
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
||||
"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.");
|
||||
// BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string"));
|
||||
|
@ -1,7 +1,9 @@
|
||||
#include "CacheDictionary.h"
|
||||
#include "CacheDictionaryStorage.h"
|
||||
#include "SSDCacheDictionaryStorage.h"
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -11,6 +13,7 @@ namespace ErrorCodes
|
||||
extern const int TOO_SMALL_BUFFER_SIZE;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
CacheDictionaryStorageConfiguration parseCacheStorageConfiguration(
|
||||
@ -84,19 +87,16 @@ SSDCacheDictionaryStorageConfiguration parseSSDCacheStorageConfiguration(
|
||||
"{}: write_buffer_size must be a multiple of block_size",
|
||||
full_name);
|
||||
|
||||
auto directory_path = config.getString(dictionary_configuration_prefix + "path");
|
||||
if (directory_path.empty())
|
||||
auto file_path = config.getString(dictionary_configuration_prefix + "path");
|
||||
if (file_path.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"{}: ssd cache dictionary cannot have empty path",
|
||||
full_name);
|
||||
|
||||
if (directory_path.at(0) != '/')
|
||||
directory_path = std::filesystem::path{config.getString("path")}.concat(directory_path).string();
|
||||
|
||||
SSDCacheDictionaryStorageConfiguration configuration{
|
||||
strict_max_lifetime_seconds,
|
||||
dict_lifetime,
|
||||
directory_path,
|
||||
file_path,
|
||||
max_partitions_count,
|
||||
block_size,
|
||||
file_size / block_size,
|
||||
@ -205,7 +205,9 @@ DictionaryPtr createSSDCacheDictionaryLayout(
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr)
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr context,
|
||||
bool created_from_ddl)
|
||||
{
|
||||
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionary");
|
||||
|
||||
@ -241,10 +243,14 @@ DictionaryPtr createSSDCacheDictionaryLayout(
|
||||
const bool allow_read_expired_keys = config.getBool(layout_prefix + ".cache.allow_read_expired_keys", false);
|
||||
|
||||
auto storage_configuration = parseSSDCacheStorageConfiguration(full_name, config, layout_prefix, dict_lifetime, dictionary_key_type);
|
||||
|
||||
if (created_from_ddl && !pathStartsWith(storage_configuration.file_path, context->getUserFilesPath()))
|
||||
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", storage_configuration.file_path, context->getUserFilesPath());
|
||||
|
||||
|
||||
auto storage = std::make_shared<SSDCacheDictionaryStorage<dictionary_key_type>>(storage_configuration);
|
||||
|
||||
auto update_queue_configuration
|
||||
= parseCacheDictionaryUpdateQueueConfiguration(full_name, config, layout_prefix, dictionary_key_type);
|
||||
auto update_queue_configuration = parseCacheDictionaryUpdateQueueConfiguration(full_name, config, layout_prefix, dictionary_key_type);
|
||||
|
||||
return std::make_unique<CacheDictionary<dictionary_key_type>>(
|
||||
dict_id, dict_struct, std::move(source_ptr), storage, update_queue_configuration, dict_lifetime, allow_read_expired_keys);
|
||||
@ -258,7 +264,9 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr) -> DictionaryPtr
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
bool /* created_from_ddl */) -> DictionaryPtr
|
||||
{
|
||||
return createCacheDictionaryLayout<DictionaryKeyType::simple>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
};
|
||||
@ -269,7 +277,9 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr) -> DictionaryPtr
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr /* context */,
|
||||
bool /* created_from_ddl */) -> DictionaryPtr
|
||||
{
|
||||
return createCacheDictionaryLayout<DictionaryKeyType::complex>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
};
|
||||
@ -282,9 +292,11 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr) -> DictionaryPtr
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr context,
|
||||
bool created_from_ddl) -> DictionaryPtr
|
||||
{
|
||||
return createSSDCacheDictionaryLayout<DictionaryKeyType::simple>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
return createSSDCacheDictionaryLayout<DictionaryKeyType::simple>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl);
|
||||
};
|
||||
|
||||
factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false);
|
||||
@ -293,8 +305,10 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr) -> DictionaryPtr {
|
||||
return createSSDCacheDictionaryLayout<DictionaryKeyType::complex>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
DictionarySourcePtr source_ptr,
|
||||
ContextPtr context,
|
||||
bool created_from_ddl) -> DictionaryPtr {
|
||||
return createSSDCacheDictionaryLayout<DictionaryKeyType::complex>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl);
|
||||
};
|
||||
|
||||
factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true);
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <common/types.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/IFunction.h>
|
||||
@ -18,7 +19,6 @@ namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
@ -26,43 +26,6 @@ namespace ErrorCodes
|
||||
* CustomWeek Transformations.
|
||||
*/
|
||||
|
||||
static inline UInt32 dateIsNotSupported(const char * name)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
/// This factor transformation will say that the function is monotone everywhere.
|
||||
struct ZeroTransform
|
||||
{
|
||||
static inline UInt16 execute(UInt16, UInt8, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(UInt32, UInt8, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(Int64, UInt8, const DateLUTImpl &) { return 0; }
|
||||
};
|
||||
|
||||
struct ToWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toWeek";
|
||||
|
||||
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct ToYearWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toYearWeek";
|
||||
@ -110,10 +73,34 @@ struct ToStartOfWeekImpl
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct Transformer
|
||||
struct ToWeekImpl
|
||||
{
|
||||
explicit Transformer(Transform transform_)
|
||||
static constexpr auto name = "toWeek";
|
||||
|
||||
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct WeekTransformer
|
||||
{
|
||||
explicit WeekTransformer(Transform transform_)
|
||||
: transform(std::move(transform_))
|
||||
{}
|
||||
|
||||
@ -139,7 +126,7 @@ struct CustomWeekTransformImpl
|
||||
template <typename Transform>
|
||||
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/, Transform transform = {})
|
||||
{
|
||||
const auto op = Transformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
|
||||
const auto op = WeekTransformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
|
||||
|
||||
UInt8 week_mode = DEFAULT_WEEK_MODE;
|
||||
if (arguments.size() > 1)
|
||||
|
@ -10,6 +10,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
@ -143,15 +144,15 @@ public:
|
||||
|
||||
if (checkAndGetDataType<DataTypeDate>(&type))
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic
|
||||
: is_not_monotonic;
|
||||
}
|
||||
else
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic
|
||||
: is_not_monotonic;
|
||||
}
|
||||
|
@ -2523,7 +2523,7 @@ private:
|
||||
if (!from_type)
|
||||
{
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"CAST AS Array can only be perforamed between same-dimensional Array or String types");
|
||||
"CAST AS Array can only be performed between same-dimensional Array or String types");
|
||||
}
|
||||
|
||||
DataTypePtr from_nested_type = from_type->getNestedType();
|
||||
@ -2533,7 +2533,7 @@ private:
|
||||
|
||||
if (from_type->getNumberOfDimensions() != to_type.getNumberOfDimensions() && !from_empty_array)
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
"CAST AS Array can only be perforamed between same-dimensional array types");
|
||||
"CAST AS Array can only be performed between same-dimensional array types");
|
||||
|
||||
const DataTypePtr & to_nested_type = to_type.getNestedType();
|
||||
|
||||
|
@ -38,8 +38,8 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
|
||||
{
|
||||
/// For dictionaries from databases (created with DDL queries) we have to perform
|
||||
/// additional checks, so we identify them here.
|
||||
bool dictionary_from_database = !repository_name.empty();
|
||||
return DictionaryFactory::instance().create(name, config, key_in_config, getContext(), dictionary_from_database);
|
||||
bool created_from_ddl = !repository_name.empty();
|
||||
return DictionaryFactory::instance().create(name, config, key_in_config, getContext(), created_from_ddl);
|
||||
}
|
||||
|
||||
ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const
|
||||
|
@ -1,57 +1,57 @@
|
||||
#include <iostream>
|
||||
|
||||
// #include <llvm/IR/IRBuilder.h>
|
||||
#include <llvm/IR/IRBuilder.h>
|
||||
|
||||
// #include <Interpreters/JIT/CHJIT.h>
|
||||
#include <Interpreters/JIT/CHJIT.h>
|
||||
|
||||
// void test_function()
|
||||
// {
|
||||
// std::cerr << "Test function" << std::endl;
|
||||
// }
|
||||
void test_function()
|
||||
{
|
||||
std::cerr << "Test function" << std::endl;
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
(void)(argc);
|
||||
(void)(argv);
|
||||
|
||||
// auto jit = DB::CHJIT();
|
||||
auto jit = DB::CHJIT();
|
||||
|
||||
// jit.registerExternalSymbol("test_function", reinterpret_cast<void *>(&test_function));
|
||||
jit.registerExternalSymbol("test_function", reinterpret_cast<void *>(&test_function));
|
||||
|
||||
// auto compiled_module_info = jit.compileModule([](llvm::Module & module)
|
||||
// {
|
||||
// auto & context = module.getContext();
|
||||
// llvm::IRBuilder<> b (context);
|
||||
auto compiled_module_info = jit.compileModule([](llvm::Module & module)
|
||||
{
|
||||
auto & context = module.getContext();
|
||||
llvm::IRBuilder<> b (context);
|
||||
|
||||
// auto * func_declaration_type = llvm::FunctionType::get(b.getVoidTy(), { }, /*isVarArg=*/false);
|
||||
// auto * func_declaration = llvm::Function::Create(func_declaration_type, llvm::Function::ExternalLinkage, "test_function", module);
|
||||
auto * func_declaration_type = llvm::FunctionType::get(b.getVoidTy(), { }, /*isVarArg=*/false);
|
||||
auto * func_declaration = llvm::Function::Create(func_declaration_type, llvm::Function::ExternalLinkage, "test_function", module);
|
||||
|
||||
// auto * value_type = b.getInt64Ty();
|
||||
// auto * pointer_type = value_type->getPointerTo();
|
||||
auto * value_type = b.getInt64Ty();
|
||||
auto * pointer_type = value_type->getPointerTo();
|
||||
|
||||
// auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { pointer_type }, /*isVarArg=*/false);
|
||||
// auto * function = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, "test_name", module);
|
||||
// auto * entry = llvm::BasicBlock::Create(context, "entry", function);
|
||||
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { pointer_type }, /*isVarArg=*/false);
|
||||
auto * function = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, "test_name", module);
|
||||
auto * entry = llvm::BasicBlock::Create(context, "entry", function);
|
||||
|
||||
// auto * argument = function->args().begin();
|
||||
// b.SetInsertPoint(entry);
|
||||
auto * argument = function->args().begin();
|
||||
b.SetInsertPoint(entry);
|
||||
|
||||
// b.CreateCall(func_declaration);
|
||||
b.CreateCall(func_declaration);
|
||||
|
||||
// auto * load_argument = b.CreateLoad(value_type, argument);
|
||||
// auto * value = b.CreateAdd(load_argument, load_argument);
|
||||
// b.CreateRet(value);
|
||||
// });
|
||||
auto * load_argument = b.CreateLoad(value_type, argument);
|
||||
auto * value = b.CreateAdd(load_argument, load_argument);
|
||||
b.CreateRet(value);
|
||||
});
|
||||
|
||||
// for (const auto & compiled_function_name : compiled_module_info.compiled_functions)
|
||||
// {
|
||||
// std::cerr << compiled_function_name << std::endl;
|
||||
// }
|
||||
for (const auto & compiled_function_name : compiled_module_info.compiled_functions)
|
||||
{
|
||||
std::cerr << compiled_function_name << std::endl;
|
||||
}
|
||||
|
||||
// int64_t value = 5;
|
||||
// auto * test_name_function = reinterpret_cast<int64_t (*)(int64_t *)>(jit.findCompiledFunction(compiled_module_info, "test_name"));
|
||||
// auto result = test_name_function(&value);
|
||||
// std::cerr << "Result " << result << std::endl;
|
||||
int64_t value = 5;
|
||||
auto * test_name_function = reinterpret_cast<int64_t (*)(int64_t *)>(jit.findCompiledFunction(compiled_module_info, "test_name"));
|
||||
auto result = test_name_function(&value);
|
||||
std::cerr << "Result " << result << std::endl;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
@ -846,34 +846,48 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
/// Parse numbers (including decimals), strings and arrays of them.
|
||||
|
||||
const char * data_begin = pos->begin;
|
||||
const char * data_end = pos->end;
|
||||
bool is_string_literal = pos->type == TokenType::StringLiteral;
|
||||
if (pos->type == TokenType::Number || is_string_literal)
|
||||
{
|
||||
++pos;
|
||||
}
|
||||
else if (pos->type == TokenType::OpeningSquareBracket)
|
||||
else if (isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket>(pos->type))
|
||||
{
|
||||
TokenType last_token = TokenType::OpeningSquareBracket;
|
||||
std::vector<TokenType> stack;
|
||||
while (pos.isValid())
|
||||
{
|
||||
if (pos->type == TokenType::OpeningSquareBracket)
|
||||
if (isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket>(pos->type))
|
||||
{
|
||||
if (!isOneOf<TokenType::OpeningSquareBracket, TokenType::Comma>(last_token))
|
||||
stack.push_back(pos->type);
|
||||
if (!isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket, TokenType::Comma>(last_token))
|
||||
return false;
|
||||
}
|
||||
else if (pos->type == TokenType::ClosingSquareBracket)
|
||||
{
|
||||
if (last_token == TokenType::Comma)
|
||||
if (isOneOf<TokenType::Comma, TokenType::OpeningRoundBracket>(last_token))
|
||||
return false;
|
||||
if (stack.empty() || stack.back() != TokenType::OpeningSquareBracket)
|
||||
return false;
|
||||
stack.pop_back();
|
||||
}
|
||||
else if (pos->type == TokenType::ClosingRoundBracket)
|
||||
{
|
||||
if (isOneOf<TokenType::Comma, TokenType::OpeningSquareBracket>(last_token))
|
||||
return false;
|
||||
if (stack.empty() || stack.back() != TokenType::OpeningRoundBracket)
|
||||
return false;
|
||||
stack.pop_back();
|
||||
}
|
||||
else if (pos->type == TokenType::Comma)
|
||||
{
|
||||
if (isOneOf<TokenType::OpeningSquareBracket, TokenType::Comma>(last_token))
|
||||
if (isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket, TokenType::Comma>(last_token))
|
||||
return false;
|
||||
}
|
||||
else if (isOneOf<TokenType::Number, TokenType::StringLiteral>(pos->type))
|
||||
{
|
||||
if (!isOneOf<TokenType::OpeningSquareBracket, TokenType::Comma>(last_token))
|
||||
if (!isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket, TokenType::Comma>(last_token))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
@ -881,14 +895,18 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
break;
|
||||
}
|
||||
|
||||
/// Update data_end on every iteration to avoid appearances of extra trailing
|
||||
/// whitespaces into data. Whitespaces are skipped at operator '++' of Pos.
|
||||
data_end = pos->end;
|
||||
last_token = pos->type;
|
||||
++pos;
|
||||
}
|
||||
|
||||
if (!stack.empty())
|
||||
return false;
|
||||
}
|
||||
|
||||
ASTPtr type_ast;
|
||||
const char * data_end = pos->begin;
|
||||
|
||||
if (ParserToken(TokenType::DoubleColon).ignore(pos, expected)
|
||||
&& ParserDataType().parse(pos, type_ast, expected))
|
||||
{
|
||||
|
@ -3,7 +3,9 @@
|
||||
#include <Processors/IProcessor.h>
|
||||
#include <Processors/QueryPlan/QueryIdHolder.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/Sources/SourceWithProgress.h>
|
||||
#include <Access/EnabledQuota.h>
|
||||
#include <DataStreams/SizeLimits.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -23,6 +23,30 @@ SourceWithProgress::SourceWithProgress(Block header, bool enable_auto_progress)
|
||||
{
|
||||
}
|
||||
|
||||
void SourceWithProgress::setProcessListElement(QueryStatus * elem)
|
||||
{
|
||||
process_list_elem = elem;
|
||||
|
||||
/// Update total_rows_approx as soon as possible.
|
||||
///
|
||||
/// It is important to do this, since you will not get correct
|
||||
/// total_rows_approx until the query will start reading all parts (in case
|
||||
/// of query needs to read from multiple parts), and this is especially a
|
||||
/// problem in case of max_threads=1.
|
||||
///
|
||||
/// NOTE: This can be done only if progress callback already set, since
|
||||
/// otherwise total_rows_approx will lost.
|
||||
if (total_rows_approx != 0 && progress_callback)
|
||||
{
|
||||
Progress total_rows_progress = {0, 0, total_rows_approx};
|
||||
|
||||
progress_callback(total_rows_progress);
|
||||
process_list_elem->updateProgressIn(total_rows_progress);
|
||||
|
||||
total_rows_approx = 0;
|
||||
}
|
||||
}
|
||||
|
||||
void SourceWithProgress::work()
|
||||
{
|
||||
if (!limits.speed_limits.checkTimeLimit(total_stopwatch.elapsed(), limits.timeout_overflow_mode))
|
||||
|
@ -51,7 +51,7 @@ public:
|
||||
void setLimits(const StreamLocalLimits & limits_) final { limits = limits_; }
|
||||
void setLeafLimits(const SizeLimits & leaf_limits_) final {leaf_limits = leaf_limits_; }
|
||||
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) final { quota = quota_; }
|
||||
void setProcessListElement(QueryStatus * elem) final { process_list_elem = elem; }
|
||||
void setProcessListElement(QueryStatus * elem) final;
|
||||
void setProgressCallback(const ProgressCallback & callback) final { progress_callback = callback; }
|
||||
void addTotalRowsApprox(size_t value) final { total_rows_approx += value; }
|
||||
|
||||
|
@ -343,6 +343,9 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge(
|
||||
if (parts_to_merge.empty())
|
||||
{
|
||||
SimpleMergeSelector::Settings merge_settings;
|
||||
/// Override value from table settings
|
||||
merge_settings.max_parts_to_merge_at_once = data_settings->max_parts_to_merge_at_once;
|
||||
|
||||
if (aggressive)
|
||||
merge_settings.base = 1;
|
||||
|
||||
|
@ -56,6 +56,7 @@ struct Settings;
|
||||
M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \
|
||||
M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \
|
||||
M(UInt64, non_replicated_deduplication_window, 0, "How many last blocks of hashes should be kept on disk (0 - disabled).", 0) \
|
||||
M(UInt64, max_parts_to_merge_at_once, 100, "Max amount of parts which can be merged at once (0 - disabled). Doesn't affect OPTIMIZE FINAL query.", 0) \
|
||||
\
|
||||
/** Inserts settings. */ \
|
||||
M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \
|
||||
|
@ -268,9 +268,9 @@ void ReplicatedMergeTreeQueue::removeCoveredPartsFromMutations(const String & pa
|
||||
|
||||
bool some_mutations_are_probably_done = false;
|
||||
|
||||
for (auto it = in_partition->second.begin(); it != in_partition->second.end(); ++it)
|
||||
for (auto & it : in_partition->second)
|
||||
{
|
||||
MutationStatus & status = *it->second;
|
||||
MutationStatus & status = *it.second;
|
||||
|
||||
if (remove_part && remove_covered_parts)
|
||||
status.parts_to_do.removePartAndCoveredParts(part_name);
|
||||
|
@ -88,7 +88,7 @@ class SimpleMergeSelector final : public IMergeSelector
|
||||
public:
|
||||
struct Settings
|
||||
{
|
||||
/// Zero means unlimited.
|
||||
/// Zero means unlimited. Can be overridden by the same merge tree setting.
|
||||
size_t max_parts_to_merge_at_once = 100;
|
||||
|
||||
/** Minimum ratio of size of one part to all parts in set of parts to merge (for usual cases).
|
||||
|
@ -58,7 +58,9 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context
|
||||
|
||||
structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
if (structure.empty())
|
||||
throw Exception("Table structure is empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Table structure is empty for table function '{}'",
|
||||
ast_function->formatForErrorMessage());
|
||||
|
||||
if (args.size() == 4)
|
||||
compression_method = args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
|
||||
<settings>
|
||||
<allow_experimental_map_type>1</allow_experimental_map_type>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.3">
|
||||
<test>
|
||||
<preconditions>
|
||||
<table_exists>hits_100m_single</table_exists>
|
||||
</preconditions>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.7">
|
||||
<test>
|
||||
<settings>
|
||||
<max_memory_usage>30000000000</max_memory_usage>
|
||||
</settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
|
||||
<query>SELECT avg(ifNotFinite(arrayAUC(arrayMap(x -> rand(x) / 0x100000000, range(2 + rand() % 100)), arrayMap(x -> rand(x) % 2, range(2 + rand() % 100))), 0)) FROM numbers(100000)</query>
|
||||
</test>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<tags>
|
||||
<tag>search</tag>
|
||||
</tags>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<max_memory_usage>35G</max_memory_usage>
|
||||
</settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<max_memory_usage>15G</max_memory_usage>
|
||||
</settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.4">
|
||||
<test>
|
||||
<create_query>
|
||||
CREATE TABLE simple_key_direct_dictionary_source_table
|
||||
(
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<query>SELECT count() FROM numbers(10000000) WHERE NOT ignore(formatReadableSize(number))</query>
|
||||
<query>SELECT count() FROM numbers(10000000) WHERE NOT ignore(formatReadableQuantity(number))</query>
|
||||
<query>SELECT count() FROM numbers(10000000) WHERE NOT ignore(formatReadableTimeDelta(number))</query>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>gp_hash_func</name>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.6">
|
||||
<test>
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>hash_func</name>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<max_memory_usage>30000000000</max_memory_usage>
|
||||
</settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.4">
|
||||
<test>
|
||||
<settings>
|
||||
<max_insert_threads>8</max_insert_threads>
|
||||
</settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<create_query>
|
||||
CREATE TABLE simple_key_hashed_dictionary_source_table
|
||||
(
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.3">
|
||||
<test>
|
||||
<query>SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? ['Hello', 'World'] : ['a', 'b', 'c'])</query>
|
||||
<query>SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? materialize(['Hello', 'World']) : ['a', 'b', 'c'])</query>
|
||||
<query>SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? ['Hello', 'World'] : materialize(['a', 'b', 'c']))</query>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
|
||||
<create_query>CREATE TABLE lot_of_string_arrays_src (`id` UInt64, `col00` Array(String), `col01` Array(String), `col02` Array(String), `col03` Array(String), `col04` Array(String), `col05` Array(String), `col06` Array(String), `col07` Array(String), `col08` Array(String), `col09` Array(String), `col10` Array(String), `col11` Array(String), `col12` Array(String), `col13` Array(String), `col14` Array(String), `col15` Array(String), `col16` Array(String), `col17` Array(String), `col18` Array(String), `col19` Array(String), `col20` Array(String), `col21` Array(String), `col22` Array(String), `col23` Array(String), `col24` Array(String), `col25` Array(String), `col26` Array(String), `col27` Array(String), `col28` Array(String), `col29` Array(String), `col30` Array(String), `col31` Array(String), `col32` Array(String), `col33` Array(String), `col34` Array(String), `col35` Array(String), `col36` Array(String), `col37` Array(String), `col38` Array(String), `col39` Array(String), `col40` Array(String), `col41` Array(String), `col42` Array(String), `col43` Array(String), `col44` Array(String), `col45` Array(String), `col46` Array(String), `col47` Array(String), `col48` Array(String), `col49` Array(String)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192;</create_query>
|
||||
<create_query>CREATE TABLE lot_of_string_arrays_dst_lowcardinality (`id` UInt64, `col00` Array(LowCardinality(String)), `col01` Array(LowCardinality(String)), `col02` Array(LowCardinality(String)), `col03` Array(LowCardinality(String)), `col04` Array(LowCardinality(String)), `col05` Array(LowCardinality(String)), `col06` Array(LowCardinality(String)), `col07` Array(LowCardinality(String)), `col08` Array(LowCardinality(String)), `col09` Array(LowCardinality(String)), `col10` Array(LowCardinality(String)), `col11` Array(LowCardinality(String)), `col12` Array(LowCardinality(String)), `col13` Array(LowCardinality(String)), `col14` Array(LowCardinality(String)), `col15` Array(LowCardinality(String)), `col16` Array(LowCardinality(String)), `col17` Array(LowCardinality(String)), `col18` Array(LowCardinality(String)), `col19` Array(LowCardinality(String)), `col20` Array(LowCardinality(String)), `col21` Array(LowCardinality(String)), `col22` Array(LowCardinality(String)), `col23` Array(LowCardinality(String)), `col24` Array(LowCardinality(String)), `col25` Array(LowCardinality(String)), `col26` Array(LowCardinality(String)), `col27` Array(LowCardinality(String)), `col28` Array(LowCardinality(String)), `col29` Array(LowCardinality(String)), `col30` Array(LowCardinality(String)), `col31` Array(LowCardinality(String)), `col32` Array(LowCardinality(String)), `col33` Array(LowCardinality(String)), `col34` Array(LowCardinality(String)), `col35` Array(LowCardinality(String)), `col36` Array(LowCardinality(String)), `col37` Array(LowCardinality(String)), `col38` Array(LowCardinality(String)), `col39` Array(LowCardinality(String)), `col40` Array(LowCardinality(String)), `col41` Array(LowCardinality(String)), `col42` Array(LowCardinality(String)), `col43` Array(LowCardinality(String)), `col44` Array(LowCardinality(String)), `col45` Array(LowCardinality(String)), `col46` Array(LowCardinality(String)), `col47` Array(LowCardinality(String)), `col48` Array(LowCardinality(String)), `col49` Array(LowCardinality(String))) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192;</create_query>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<create_query>CREATE TABLE ints (i64 Int64, i32 Int32, i16 Int16, i8 Int8) ENGINE = Memory</create_query>
|
||||
|
||||
<fill_query>INSERT INTO ints SELECT number AS i64, i64 AS i32, i64 AS i16, i64 AS i8 FROM numbers(10000)</fill_query>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.7">
|
||||
<test>
|
||||
<create_query>CREATE TABLE ints (i64 Int64, i32 Int32, i16 Int16, i8 Int8) ENGINE = Memory</create_query>
|
||||
|
||||
<settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>json</name>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<max_threads>1</max_threads>
|
||||
</settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<max_threads>1</max_threads>
|
||||
</settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.6">
|
||||
<test>
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>func_slow</name>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.3">
|
||||
<test>
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>format</name>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<create_query>create table test_parallel_index (x UInt64, y UInt64, z UInt64, INDEX a (y) TYPE minmax GRANULARITY 2,
|
||||
INDEX b (z) TYPE set(8) GRANULARITY 2) engine = MergeTree order by x partition by bitAnd(x, 63 * 64) settings index_granularity = 4;</create_query>
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
|
||||
<preconditions>
|
||||
<table_exists>test.hits</table_exists>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<!--
|
||||
Not sure why it's needed. Maybe it has something to do with the
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<max_threads>4</max_threads>
|
||||
<max_memory_usage>20G</max_memory_usage>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<query>SELECT count() FROM zeros(10000000) WHERE NOT ignore(randomPrintableASCII(10))</query>
|
||||
<query>SELECT count() FROM zeros(10000000) WHERE NOT ignore(randomPrintableASCII(100))</query>
|
||||
<query>SELECT count() FROM zeros(100000) WHERE NOT ignore(randomPrintableASCII(1000))</query>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<allow_experimental_bigint_types>1</allow_experimental_bigint_types>
|
||||
<max_memory_usage>15G</max_memory_usage>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.3">
|
||||
<test>
|
||||
<settings>
|
||||
<output_format_pretty_max_rows>1000000</output_format_pretty_max_rows>
|
||||
<max_threads>1</max_threads>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
|
||||
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
|
||||
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<max_threads>1</max_threads>
|
||||
</settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<settings>
|
||||
<max_memory_usage>30000000000</max_memory_usage>
|
||||
</settings>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>param</name>
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.2">
|
||||
<test>
|
||||
|
||||
<preconditions>
|
||||
<table_exists>hits_10m_single</table_exists>
|
||||
|
155
tests/queries/0_stateless/01053_ssd_dictionary.sh
Executable file
155
tests/queries/0_stateless/01053_ssd_dictionary.sh
Executable file
@ -0,0 +1,155 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
DROP DATABASE IF EXISTS 01053_db;
|
||||
|
||||
CREATE DATABASE 01053_db Engine = Ordinary;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.table_for_dict;
|
||||
|
||||
CREATE TABLE 01053_db.table_for_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64,
|
||||
b Int32,
|
||||
c String
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY id;
|
||||
|
||||
INSERT INTO 01053_db.table_for_dict VALUES (1, 100, -100, 'clickhouse'), (2, 3, 4, 'database'), (5, 6, 7, 'columns'), (10, 9, 8, '');
|
||||
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
|
||||
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
|
||||
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
|
||||
-- Probably we need rewrite it to integration test
|
||||
CREATE DICTIONARY 01053_db.ssd_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64 DEFAULT 0,
|
||||
b Int32 DEFAULT -1,
|
||||
c String DEFAULT 'none'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/0d'));
|
||||
|
||||
SELECT 'TEST_SMALL';
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(4));
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(5));
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(6));
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(2));
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(3));
|
||||
|
||||
SELECT * FROM 01053_db.ssd_dict ORDER BY id;
|
||||
DROP DICTIONARY 01053_db.ssd_dict;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.keys_table;
|
||||
|
||||
CREATE TABLE 01053_db.keys_table
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
ENGINE = StripeLog();
|
||||
|
||||
INSERT INTO 01053_db.keys_table VALUES (1);
|
||||
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (2);
|
||||
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (5);
|
||||
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (10);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
|
||||
CREATE DICTIONARY 01053_db.ssd_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64 DEFAULT 0,
|
||||
b Int32 DEFAULT -1,
|
||||
c String DEFAULT 'none'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY';
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
|
||||
SELECT 'VALUE FROM DISK';
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
|
||||
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(1));
|
||||
|
||||
SELECT 'VALUE FROM RAM BUFFER';
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(10));
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(10));
|
||||
|
||||
SELECT 'VALUES FROM DISK AND RAM BUFFER';
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
|
||||
SELECT 'HAS';
|
||||
SELECT count() FROM 01053_db.keys_table WHERE dictHas('01053_db.ssd_dict', toUInt64(id));
|
||||
|
||||
SELECT 'VALUES NOT FROM TABLE';
|
||||
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
|
||||
|
||||
SELECT 'DUPLICATE KEYS';
|
||||
SELECT arrayJoin([1, 2, 3, 3, 2, 1]) AS id, dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(id));
|
||||
--SELECT
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.keys_table;
|
||||
|
||||
CREATE TABLE 01053_db.keys_table
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY id;
|
||||
|
||||
INSERT INTO 01053_db.keys_table VALUES (1);
|
||||
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (2);
|
||||
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (5);
|
||||
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (10);
|
||||
|
||||
OPTIMIZE TABLE 01053_db.keys_table;
|
||||
|
||||
CREATE DICTIONARY 01053_db.ssd_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64 DEFAULT 0,
|
||||
b Int32 DEFAULT -1
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/2d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 1024));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY (MT)';
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
|
||||
SELECT 'VALUES FROM DISK AND RAM BUFFER (MT)';
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.table_for_dict;
|
||||
|
||||
DROP DATABASE IF EXISTS 01053_db;"
|
@ -1,159 +0,0 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP DATABASE IF EXISTS 01053_db;
|
||||
|
||||
CREATE DATABASE 01053_db Engine = Ordinary;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.table_for_dict;
|
||||
|
||||
CREATE TABLE 01053_db.table_for_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64,
|
||||
b Int32,
|
||||
c String
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY id;
|
||||
|
||||
INSERT INTO 01053_db.table_for_dict VALUES (1, 100, -100, 'clickhouse'), (2, 3, 4, 'database'), (5, 6, 7, 'columns'), (10, 9, 8, '');
|
||||
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
|
||||
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
|
||||
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
|
||||
-- FIXME filesystem error: in create_directory: Permission denied [/var/lib/clickhouse]
|
||||
-- Probably we need rewrite it to integration test
|
||||
CREATE DICTIONARY 01053_db.ssd_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64 DEFAULT 0,
|
||||
b Int32 DEFAULT -1,
|
||||
c String DEFAULT 'none'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/0d'));
|
||||
|
||||
SELECT 'TEST_SMALL';
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(4));
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(5));
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(6));
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(2));
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(3));
|
||||
|
||||
SELECT * FROM 01053_db.ssd_dict ORDER BY id;
|
||||
DROP DICTIONARY 01053_db.ssd_dict;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.keys_table;
|
||||
|
||||
CREATE TABLE 01053_db.keys_table
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
ENGINE = StripeLog();
|
||||
|
||||
INSERT INTO 01053_db.keys_table VALUES (1);
|
||||
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (2);
|
||||
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (5);
|
||||
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (10);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
|
||||
CREATE DICTIONARY 01053_db.ssd_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64 DEFAULT 0,
|
||||
b Int32 DEFAULT -1,
|
||||
c String DEFAULT 'none'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY';
|
||||
-- 118
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
|
||||
SELECT 'VALUE FROM DISK';
|
||||
-- -100
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
|
||||
|
||||
-- 'clickhouse'
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(1));
|
||||
|
||||
SELECT 'VALUE FROM RAM BUFFER';
|
||||
-- 8
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(10));
|
||||
|
||||
-- ''
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(10));
|
||||
|
||||
SELECT 'VALUES FROM DISK AND RAM BUFFER';
|
||||
-- 118
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
|
||||
SELECT 'HAS';
|
||||
-- 1006
|
||||
SELECT count() FROM 01053_db.keys_table WHERE dictHas('01053_db.ssd_dict', toUInt64(id));
|
||||
|
||||
SELECT 'VALUES NOT FROM TABLE';
|
||||
-- 0 -1 none
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
|
||||
|
||||
SELECT 'DUPLICATE KEYS';
|
||||
SELECT arrayJoin([1, 2, 3, 3, 2, 1]) AS id, dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(id));
|
||||
--SELECT
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.keys_table;
|
||||
|
||||
CREATE TABLE 01053_db.keys_table
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY id;
|
||||
|
||||
INSERT INTO 01053_db.keys_table VALUES (1);
|
||||
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (2);
|
||||
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (5);
|
||||
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (10);
|
||||
|
||||
OPTIMIZE TABLE 01053_db.keys_table;
|
||||
|
||||
CREATE DICTIONARY 01053_db.ssd_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64 DEFAULT 0,
|
||||
b Int32 DEFAULT -1
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/2d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 1024));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY (MT)';
|
||||
-- 118
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
|
||||
SELECT 'VALUES FROM DISK AND RAM BUFFER (MT)';
|
||||
-- 118
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.table_for_dict;
|
||||
|
||||
DROP DATABASE IF EXISTS 01053_db;
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user