Merge branch 'master' into master

This commit is contained in:
mergify[bot] 2021-08-20 11:50:30 +00:00 committed by GitHub
commit 66c6a85211
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
378 changed files with 10109 additions and 6340 deletions

View File

@ -1,5 +1,8 @@
### ClickHouse release v21.8, 2021-08-12
#### Upgrade Notes
* New version is using `Map` data type for system logs tables (`system.query_log`, `system.query_thread_log`, `system.processes`, `system.opentelemetry_span_log`). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes [#18698](https://github.com/ClickHouse/ClickHouse/issues/18698). [#23934](https://github.com/ClickHouse/ClickHouse/pull/23934), [#25773](https://github.com/ClickHouse/ClickHouse/pull/25773) ([hexiaoting](https://github.com/hexiaoting), [sundy-li](https://github.com/sundy-li), [Maksim Kita](https://github.com/kitaisreal)). If you want to *downgrade* from version 21.8 to older versions, you will need to cleanup system tables with logs manually. Look at `/var/lib/clickhouse/data/system/*_log`.
#### New Features
* Add support for a part of SQL/JSON standard. [#24148](https://github.com/ClickHouse/ClickHouse/pull/24148) ([l1tsolaiki](https://github.com/l1tsolaiki), [Kseniia Sumarokova](https://github.com/kssenii)).

View File

@ -593,7 +593,23 @@ macro (add_executable target)
# disabled for TSAN and gcc since libtsan.a provides overrides too
if (TARGET clickhouse_new_delete)
# operator::new/delete for executables (MemoryTracker stuff)
target_link_libraries (${target} PRIVATE clickhouse_new_delete ${MALLOC_LIBRARIES})
target_link_libraries (${target} PRIVATE clickhouse_new_delete)
endif()
# In case of static jemalloc, because zone_register() is located in zone.c and
# is never used outside (it is declared as constructor) it is omitted
# by the linker, and so jemalloc will not be registered as system
# allocator under osx [1], and clickhouse will SIGSEGV.
#
# [1]: https://github.com/jemalloc/jemalloc/issues/708
#
# About symbol name:
# - _zone_register not zone_register due to Mach-O binary format,
# - _je_zone_register due to JEMALLOC_PRIVATE_NAMESPACE=je_ under OS X.
# - but jemalloc-cmake does not run private_namespace.sh
# so symbol name should be _zone_register
if (ENABLE_JEMALLOC AND MAKE_STATIC_LIBRARIES AND OS_DARWIN)
set_property(TARGET ${target} APPEND PROPERTY LINK_OPTIONS -u_zone_register)
endif()
endif()
endmacro()

View File

@ -152,7 +152,7 @@ namespace wide
template <size_t Bits, typename Signed>
struct integer<Bits, Signed>::_impl
{
static constexpr size_t _Bits = Bits;
static constexpr size_t _bits = Bits;
static constexpr const unsigned byte_count = Bits / 8;
static constexpr const unsigned item_count = byte_count / sizeof(base_type);
static constexpr const unsigned base_bits = sizeof(base_type) * 8;
@ -614,8 +614,8 @@ public:
else
{
static_assert(IsWideInteger<T>::value);
return std::common_type_t<integer<Bits, Signed>, integer<T::_impl::_Bits, Signed>>::_impl::operator_plus(
integer<T::_impl::_Bits, Signed>(lhs), rhs);
return std::common_type_t<integer<Bits, Signed>, integer<T::_impl::_bits, Signed>>::_impl::operator_plus(
integer<T::_impl::_bits, Signed>(lhs), rhs);
}
}
@ -632,8 +632,8 @@ public:
else
{
static_assert(IsWideInteger<T>::value);
return std::common_type_t<integer<Bits, Signed>, integer<T::_impl::_Bits, Signed>>::_impl::operator_minus(
integer<T::_impl::_Bits, Signed>(lhs), rhs);
return std::common_type_t<integer<Bits, Signed>, integer<T::_impl::_bits, Signed>>::_impl::operator_minus(
integer<T::_impl::_bits, Signed>(lhs), rhs);
}
}
@ -857,7 +857,7 @@ public:
else
{
static_assert(IsWideInteger<T>::value);
return std::common_type_t<integer<Bits, Signed>, integer<T::_impl::_Bits, Signed>>::operator_slash(T(lhs), rhs);
return std::common_type_t<integer<Bits, Signed>, integer<T::_impl::_bits, Signed>>::operator_slash(T(lhs), rhs);
}
}
@ -877,7 +877,7 @@ public:
else
{
static_assert(IsWideInteger<T>::value);
return std::common_type_t<integer<Bits, Signed>, integer<T::_impl::_Bits, Signed>>::operator_percent(T(lhs), rhs);
return std::common_type_t<integer<Bits, Signed>, integer<T::_impl::_bits, Signed>>::operator_percent(T(lhs), rhs);
}
}

View File

@ -12,6 +12,7 @@
#include <Common/SymbolIndex.h>
#include <Common/StackTrace.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Core/ServerUUID.h>
#if !defined(ARCADIA_BUILD)
# include "Common/config_version.h"
@ -38,6 +39,13 @@ void setExtras()
if (!anonymize)
sentry_set_extra("server_name", sentry_value_new_string(getFQDNOrHostName().c_str()));
DB::UUID server_uuid = DB::ServerUUID::get();
if (server_uuid != DB::UUIDHelpers::Nil)
{
std::string server_uuid_str = DB::toString(server_uuid);
sentry_set_extra("server_uuid", sentry_value_new_string(server_uuid_str.c_str()));
}
sentry_set_tag("version", VERSION_STRING);
sentry_set_extra("version_githash", sentry_value_new_string(VERSION_GITHASH));
sentry_set_extra("version_describe", sentry_value_new_string(VERSION_DESCRIBE));

View File

@ -1,5 +1,4 @@
#include <sys/auxv.h>
#include "atomic.h"
#include <unistd.h> // __environ
#include <errno.h>
@ -18,7 +17,18 @@ static size_t __find_auxv(unsigned long type)
return (size_t) -1;
}
unsigned long __getauxval(unsigned long type)
__attribute__((constructor)) static void __auxv_init()
{
size_t i;
for (i = 0; __environ[i]; i++);
__auxv = (unsigned long *) (__environ + i + 1);
size_t secure_idx = __find_auxv(AT_SECURE);
if (secure_idx != ((size_t) -1))
__auxv_secure = __auxv[secure_idx];
}
unsigned long getauxval(unsigned long type)
{
if (type == AT_SECURE)
return __auxv_secure;
@ -33,38 +43,3 @@ unsigned long __getauxval(unsigned long type)
errno = ENOENT;
return 0;
}
static void * volatile getauxval_func;
static unsigned long __auxv_init(unsigned long type)
{
if (!__environ)
{
// __environ is not initialized yet so we can't initialize __auxv right now.
// That's normally occurred only when getauxval() is called from some sanitizer's internal code.
errno = ENOENT;
return 0;
}
// Initialize __auxv and __auxv_secure.
size_t i;
for (i = 0; __environ[i]; i++);
__auxv = (unsigned long *) (__environ + i + 1);
size_t secure_idx = __find_auxv(AT_SECURE);
if (secure_idx != ((size_t) -1))
__auxv_secure = __auxv[secure_idx];
// Now we've initialized __auxv, next time getauxval() will only call __get_auxval().
a_cas_p(&getauxval_func, (void *)__auxv_init, (void *)__getauxval);
return __getauxval(type);
}
// First time getauxval() will call __auxv_init().
static void * volatile getauxval_func = (void *)__auxv_init;
unsigned long getauxval(unsigned long type)
{
return ((unsigned long (*)(unsigned long))getauxval_func)(type);
}

View File

@ -2,11 +2,11 @@
# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION,
# only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes.
SET(VERSION_REVISION 54454)
SET(VERSION_REVISION 54455)
SET(VERSION_MAJOR 21)
SET(VERSION_MINOR 9)
SET(VERSION_MINOR 10)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH f063e44131a048ba2d9af8075f03700fd5ec3e69)
SET(VERSION_DESCRIBE v21.9.1.7770-prestable)
SET(VERSION_STRING 21.9.1.7770)
SET(VERSION_GITHASH 09df5018f95edcd0f759d4689ac5d029dd400c2a)
SET(VERSION_DESCRIBE v21.10.1.1-testing)
SET(VERSION_STRING 21.10.1.1)
# end of autochange

View File

@ -1,9 +1,10 @@
# Disabled under OSX until https://github.com/ClickHouse/ClickHouse/issues/27568 is fixed
if (SANITIZE OR NOT (
((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_ARM OR ARCH_PPC64LE))))
((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_ARM OR ARCH_PPC64LE)) OR
(OS_DARWIN AND (CMAKE_BUILD_TYPE STREQUAL "RelWithDebInfo" OR CMAKE_BUILD_TYPE STREQUAL "Debug"))
))
if (ENABLE_JEMALLOC)
message (${RECONFIGURE_MESSAGE_LEVEL}
"jemalloc is disabled implicitly: it doesn't work with sanitizers and can only be used with x86_64, aarch64, or ppc64le Linux or FreeBSD builds")
"jemalloc is disabled implicitly: it doesn't work with sanitizers and can only be used with x86_64, aarch64, or ppc64le Linux or FreeBSD builds and RelWithDebInfo macOS builds.")
endif ()
set (ENABLE_JEMALLOC OFF)
else ()
@ -138,9 +139,5 @@ target_compile_options(jemalloc PRIVATE -Wno-redundant-decls)
target_compile_options(jemalloc PRIVATE -D_GNU_SOURCE)
set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS USE_JEMALLOC=1)
if (MAKE_STATIC_LIBRARIES)
# To detect whether we need to register jemalloc for osx as default zone.
set_property(TARGET jemalloc APPEND PROPERTY INTERFACE_COMPILE_DEFINITIONS BUNDLED_STATIC_JEMALLOC=1)
endif()
message (STATUS "Using jemalloc")

2
contrib/librdkafka vendored

@ -1 +1 @@
Subproject commit 43491d33ca2826531d1e3cae70d4bf1e5249e3c9
Subproject commit b8554f1682062c85ba519eb54ef2f90e02b812cb

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (21.9.1.1) unstable; urgency=low
clickhouse (21.10.1.1) unstable; urgency=low
* Modified source code
-- clickhouse-release <clickhouse-release@yandex-team.ru> Sat, 10 Jul 2021 08:22:49 +0300
-- clickhouse-release <clickhouse-release@yandex-team.ru> Sat, 17 Jul 2021 08:45:03 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.9.1.*
ARG version=21.10.1.*
RUN apt-get update \
&& apt-get install --yes --no-install-recommends \

View File

@ -1,7 +1,7 @@
FROM ubuntu:20.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.9.1.*
ARG version=21.10.1.*
ARG gosu_ver=1.10
# set non-empty deb_location_url url to create a docker image

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=21.9.1.*
ARG version=21.10.1.*
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -641,6 +641,7 @@ create view partial_query_times as select * from
-- Report for partial queries that we could only run on the new server (e.g.
-- queries with new functions added in the tested PR).
create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv')
settings output_format_decimal_trailing_zeros = 1
as select toDecimal64(time_median, 3) time,
toDecimal64(time_stddev / time_median, 3) relative_time_stddev,
test, query_index, query_display_name
@ -713,8 +714,9 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv')
order by test, query_index, metric_name
;
create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') as
with
create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv')
settings output_format_decimal_trailing_zeros = 1
as with
-- server_time is sometimes reported as zero (if it's less than 1 ms),
-- so we have to work around this to not get an error about conversion
-- of NaN to decimal.
@ -730,8 +732,9 @@ create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') as
changed_fail, test, query_index, query_display_name
from queries where changed_show order by abs(diff) desc;
create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') as
select
create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv')
settings output_format_decimal_trailing_zeros = 1
as select
toDecimal64(left, 3), toDecimal64(right, 3), toDecimal64(diff, 3),
toDecimal64(stat_threshold, 3), unstable_fail, test, query_index, query_display_name
from queries where unstable_show order by stat_threshold desc;
@ -761,8 +764,9 @@ create view total_speedup as
from test_speedup
;
create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') as
with
create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv')
settings output_format_decimal_trailing_zeros = 1
as with
(times_speedup >= 1
? '-' || toString(toDecimal64(times_speedup, 3)) || 'x'
: '+' || toString(toDecimal64(1 / times_speedup, 3)) || 'x')
@ -788,8 +792,9 @@ create view total_client_time_per_query as select *
from file('analyze/client-times.tsv', TSV,
'test text, query_index int, client float, server float');
create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') as
select client, server, toDecimal64(client/server, 3) p,
create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv')
settings output_format_decimal_trailing_zeros = 1
as select client, server, toDecimal64(client/server, 3) p,
test, query_display_name
from total_client_time_per_query left join query_display_names using (test, query_index)
where p > toDecimal64(1.02, 3) order by p desc;
@ -874,8 +879,9 @@ create view test_times_view_total as
from test_times_view
;
create table test_times_report engine File(TSV, 'report/test-times.tsv') as
select
create table test_times_report engine File(TSV, 'report/test-times.tsv')
settings output_format_decimal_trailing_zeros = 1
as select
test,
toDecimal64(real, 3),
toDecimal64(total_client_time, 3),
@ -893,8 +899,9 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') as
;
-- report for all queries page, only main metric
create table all_tests_report engine File(TSV, 'report/all-queries.tsv') as
with
create table all_tests_report engine File(TSV, 'report/all-queries.tsv')
settings output_format_decimal_trailing_zeros = 1
as with
-- server_time is sometimes reported as zero (if it's less than 1 ms),
-- so we have to work around this to not get an error about conversion
-- of NaN to decimal.
@ -1057,9 +1064,10 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes,
;
create table metric_devation engine File(TSVWithNamesAndTypes,
'report/metric-deviation.$version.tsv') as
'report/metric-deviation.$version.tsv')
settings output_format_decimal_trailing_zeros = 1
-- first goes the key used to split the file with grep
select test, query_index, query_display_name,
as select test, query_index, query_display_name,
toDecimal64(d, 3) d, q, metric
from (
select
@ -1187,8 +1195,9 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as
;
-- Show metrics that have changed
create table changes engine File(TSV, 'metrics/changes.tsv') as
select metric, left, right,
create table changes engine File(TSV, 'metrics/changes.tsv')
settings output_format_decimal_trailing_zeros = 1
as select metric, left, right,
toDecimal64(diff, 3), toDecimal64(times_diff, 3)
from (
select metric, median(left) as left, median(right) as right,

View File

@ -13,7 +13,7 @@ left_sha=$2
# right_pr=$3 not used for now
right_sha=$4
datasets=${CHPC_DATASETS:-"hits1 hits10 hits100 values"}
datasets=${CHPC_DATASETS-"hits1 hits10 hits100 values"}
declare -A dataset_paths
dataset_paths["hits10"]="https://s3.mds.yandex.net/clickhouse-private-datasets/hits_10m_single/partitions/hits_10m_single.tar"

View File

@ -127,6 +127,15 @@ export PATH
export REF_PR
export REF_SHA
# Try to collect some core dumps. I've seen two patterns in Sandbox:
# 1) |/home/zomb-sandbox/venv/bin/python /home/zomb-sandbox/client/sandbox/bin/coredumper.py %e %p %g %u %s %P %c
# Not sure what this script does (puts them to sandbox resources, logs some messages?),
# and it's not accessible from inside docker anyway.
# 2) something like %e.%p.core.dmp. The dump should end up in the workspace directory.
# At least we remove the ulimit and then try to pack some common file names into output.
ulimit -c unlimited
cat /proc/sys/kernel/core_pattern
# Start the main comparison script.
{ \
time ../download.sh "$REF_PR" "$REF_SHA" "$PR_TO_TEST" "$SHA_TO_TEST" && \
@ -144,8 +153,11 @@ done
dmesg -T > dmesg.log
ls -lath
7z a '-x!*/tmp' /output/output.7z ./*.{log,tsv,html,txt,rep,svg,columns} \
{right,left}/{performance,scripts} {{right,left}/db,db0}/preprocessed_configs \
report analyze benchmark metrics
report analyze benchmark metrics \
./*.core.dmp ./*.core
cp compare.log /output

View File

@ -105,10 +105,6 @@ def process_result(result_path):
description += ", skipped: {}".format(skipped)
if unknown != 0:
description += ", unknown: {}".format(unknown)
# Temporary green for tests with DatabaseReplicated:
if 1 == int(os.environ.get('USE_DATABASE_REPLICATED', 0)):
state = "success"
else:
state = "failure"
description = "Output log doesn't exist"

View File

@ -1,3 +1,8 @@
---
toc_priority: 36
toc_title: Replicated
---
# [experimental] Replicated {#replicated}
The engine is based on the [Atomic](../../engines/database-engines/atomic.md) engine. It supports replication of metadata via DDL log being written to ZooKeeper and executed on all of the replicas for a given database.

View File

@ -38,9 +38,7 @@ A table for the Graphite data should have the following columns for the followin
- Value of the metric. Data type: any numeric.
- Version of the metric. Data type: any numeric.
ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts.
- Version of the metric. Data type: any numeric (ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts).
The names of these columns should be set in the rollup configuration.
@ -132,7 +130,7 @@ Fields for `pattern` and `default` sections:
- `regexp` A pattern for the metric name.
- `age` The minimum age of the data in seconds.
- `precision` How precisely to define the age of the data in seconds. Should be a divisor for 86400 (seconds in a day).
- `function` The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`.
- `function` The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`. Accepted functions: min / max / any / avg. The average is calculated imprecisely, like the average of the averages.
### Configuration Example {#configuration-example}
@ -169,4 +167,7 @@ Fields for `pattern` and `default` sections:
</graphite_rollup>
```
!!! warning "Warning"
Data rollup is performed during merges. Usually, for old partitions, merges are not started, so for rollup it is necessary to trigger an unscheduled merge using [optimize](../../../sql-reference/statements/optimize.md). Or use additional tools, for example [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer).
[Original article](https://clickhouse.tech/docs/en/operations/table_engines/graphitemergetree/) <!--hide-->

View File

@ -137,7 +137,7 @@ CREATE TABLE table_name
) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}', ver)
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SAMPLE BY intHash32(UserID);
```
<details markdown="1">
@ -150,12 +150,12 @@ CREATE TABLE table_name
EventDate DateTime,
CounterID UInt32,
UserID UInt32
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192);
```
</details>
As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the «[macros](../../../operations/server-configuration-parameters/settings/#macros) section of the configuration file.
As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the [macros](../../../operations/server-configuration-parameters/settings.md#macros) section of the configuration file.
Example:

View File

@ -56,6 +56,9 @@ The same thing happens if the subordinate table does not exist when the buffer i
If you need to run ALTER for a subordinate table, and the Buffer table, we recommend first deleting the Buffer table, running ALTER for the subordinate table, then creating the Buffer table again.
!!! attention "Attention"
Running ALTER on the Buffer table in releases made before 28 Sep 2020 will cause a `Block structure mismatch` error (see [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117)), so deleting the Buffer table and then recreating is the only option. It is advisable to check that this error is fixed in your release before trying to run ALTER on the Buffer table.
If the server is restarted abnormally, the data in the buffer is lost.
`FINAL` and `SAMPLE` do not work correctly for Buffer tables. These conditions are passed to the destination table, but are not used for processing data in the buffer. If these features are required we recommend only using the Buffer table for writing, while reading from the destination table.

View File

@ -105,7 +105,7 @@ We use `Decimal` data type to store prices. Everything else is quite straightfor
## Import Data
Upload data into ClickHouse in parallel:
Upload data into ClickHouse:
```
clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO dish FORMAT CSVWithNames" < Dish.csv

View File

@ -18,6 +18,18 @@ Some settings specified in the main configuration file can be overridden in othe
- If `replace` is specified, it replaces the entire element with the specified one.
- If `remove` is specified, it deletes the element.
You can also declare attributes as coming from environment variables by using `from_env="VARIABLE_NAME"`:
```xml
<yandex>
<macros>
<replica from_env="REPLICA" />
<layer from_env="LAYER" />
<shard from_env="SHARD" />
</macros>
</yandex>
```
## Substitution {#substitution}
The config can also define “substitutions”. If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](../operations/server-configuration-parameters/settings.md)).

View File

@ -486,7 +486,7 @@ Parameter substitutions for replicated tables.
Can be omitted if replicated tables are not used.
For more information, see the section [Creating replicated tables](../../engines/table-engines/mergetree-family/replication.md).
For more information, see the section [Creating replicated tables](../../engines/table-engines/mergetree-family/replication.md#creating-replicated-tables).
**Example**
@ -1247,6 +1247,7 @@ Default value: `/var/lib/clickhouse/access/`.
Section of the configuration file that contains settings:
- Path to configuration file with predefined users.
- Path to folder where users created by SQL commands are stored.
- ZooKeeper node path where users created by SQL commands are stored and replicated (experimental).
If this section is specified, the path from [users_config](../../operations/server-configuration-parameters/settings.md#users-config) and [access_control_path](../../operations/server-configuration-parameters/settings.md#access_control_path) won't be used.
@ -1262,6 +1263,9 @@ The `user_directories` section can contain any number of items, the order of the
<local_directory>
<path>/var/lib/clickhouse/access/</path>
</local_directory>
<replicated>
<zookeeper_path>/clickhouse/access/</zookeeper_path>
</replicated>
</user_directories>
```

View File

@ -4,7 +4,7 @@ Contains information about [trace spans](https://opentracing.io/docs/overview/sp
Columns:
- `trace_id` ([UUID](../../sql-reference/data-types/uuid.md) — ID of the trace for executed query.
- `trace_id` ([UUID](../../sql-reference/data-types/uuid.md)) — ID of the trace for executed query.
- `span_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of the `trace span`.

View File

@ -0,0 +1,129 @@
# system.zookeeper_log {#system-zookeeper_log}
This table contains information about the parameters of the request to the ZooKeeper server and the response from it.
For requests, only columns with request parameters are filled in, and the remaining columns are filled with default values (`0` or `NULL`). When the response arrives, the data from the response is added to the other columns.
Columns with request parameters:
- `type` ([Enum](../../sql-reference/data-types/enum.md)) — Event type in the ZooKeeper client. Can have one of the following values:
- `Request` — The request has been sent.
- `Response` — The response was received.
- `Finalize` — The connection is lost, no response was received.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the event happened.
- `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time when the event happened.
- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP address of ZooKeeper server that was used to make the request.
- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — The port of ZooKeeper server that was used to make the request.
- `session_id` ([Int64](../../sql-reference/data-types/int-uint.md)) — The session ID that the ZooKeeper server sets for each connection.
- `xid` ([Int32](../../sql-reference/data-types/int-uint.md)) — The ID of the request within the session. This is usually a sequential request number. It is the same for the request row and the paired `response`/`finalize` row.
- `has_watch` ([UInt8](../../sql-reference/data-types/int-uint.md)) — The request whether the [watch](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#ch_zkWatches) has been set.
- `op_num` ([Enum](../../sql-reference/data-types/enum.md)) — The type of request or response.
- `path` ([String](../../sql-reference/data-types/string.md)) — The path to the ZooKeeper node specified in the request, or an empty string if the request not requires specifying a path.
- `data` ([String](../../sql-reference/data-types/string.md)) — The data written to the ZooKeeper node (for the `SET` and `CREATE` requests — what the request wanted to write, for the response to the `GET` request — what was read) or an empty string.
- `is_ephemeral` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Is the ZooKeeper node being created as an [ephemeral](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#Ephemeral+Nodes).
- `is_sequential` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Is the ZooKeeper node being created as an [sequential](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#Sequence+Nodes+--+Unique+Naming).
- `version` ([Nullable(Int32)](../../sql-reference/data-types/nullable.md)) — The version of the ZooKeeper node that the request expects when executing. This is supported for `CHECK`, `SET`, `REMOVE` requests (is relevant `-1` if the request does not check the version or `NULL` for other requests that do not support version checking).
- `requests_size` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of requests included in the multi request (this is a special request that consists of several consecutive ordinary requests and executes them atomically). All requests included in multi request will have the same `xid`.
- `request_idx` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of the request included in multi request (for multi request — `0`, then in order from `1`).
Columns with request response parameters:
- `zxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — ZooKeeper transaction ID. The serial number issued by the ZooKeeper server in response to a successfully executed request (`0` if the request was not executed/returned an error/the client does not know whether the request was executed).
- `error` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — Error code. Can have many values, here are just some of them:
- `ZOK` — The request was executed seccessfully.
- `ZCONNECTIONLOSS` — The connection was lost.
- `ZOPERATIONTIMEOUT` — The request execution timeout has expired.
- `ZSESSIONEXPIRED` — The session has expired.
- `NULL` — The request is completed.
- `watch_type` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — The type of the `watch` event (for responses with `op_num` = `Watch`), for the remaining responses: `NULL`.
- `watch_state` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — The status of the `watch` event (for responses with `op_num` = `Watch`), for the remaining responses: `NULL`.
- `path_created` ([String](../../sql-reference/data-types/string.md)) — The path to the created ZooKeeper node (for responses to the `CREATE` request), may differ from the `path` if the node is created as a `sequential`.
- `stat_czxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The `zxid` of the change that caused this ZooKeeper node to be created.
- `stat_mzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The `zxid` of the change that last modified this ZooKeeper node.
- `stat_pzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The transaction ID of the change that last modified childern of this ZooKeeper node.
- `stat_version` ([Int32](../../sql-reference/data-types/int-uint.md)) — The number of changes to the data of this ZooKeeper node.
- `stat_cversion` ([Int32](../../sql-reference/data-types/int-uint.md)) — The number of changes to the children of this ZooKeeper node.
- `stat_dataLength` ([Int32](../../sql-reference/data-types/int-uint.md)) — The length of the data field of this ZooKeeper node.
- `stat_numChildren` ([Int32](../../sql-reference/data-types/int-uint.md)) — The number of children of this ZooKeeper node.
- `children` ([Array(String)](../../sql-reference/data-types/array.md)) — The list of child ZooKeeper nodes (for responses to `LIST` request).
**Example**
Query:
``` sql
SELECT * FROM system.zookeeper_log WHERE (session_id = '106662742089334927') AND (xid = '10858') FORMAT Vertical;
```
Result:
``` text
Row 1:
──────
type: Request
event_date: 2021-08-09
event_time: 2021-08-09 21:38:30.291792
address: ::
port: 2181
session_id: 106662742089334927
xid: 10858
has_watch: 1
op_num: List
path: /clickhouse/task_queue/ddl
data:
is_ephemeral: 0
is_sequential: 0
version: ᴺᵁᴸᴸ
requests_size: 0
request_idx: 0
zxid: 0
error: ᴺᵁᴸᴸ
watch_type: ᴺᵁᴸᴸ
watch_state: ᴺᵁᴸᴸ
path_created:
stat_czxid: 0
stat_mzxid: 0
stat_pzxid: 0
stat_version: 0
stat_cversion: 0
stat_dataLength: 0
stat_numChildren: 0
children: []
Row 2:
──────
type: Response
event_date: 2021-08-09
event_time: 2021-08-09 21:38:30.292086
address: ::
port: 2181
session_id: 106662742089334927
xid: 10858
has_watch: 1
op_num: List
path: /clickhouse/task_queue/ddl
data:
is_ephemeral: 0
is_sequential: 0
version: ᴺᵁᴸᴸ
requests_size: 0
request_idx: 0
zxid: 16926267
error: ZOK
watch_type: ᴺᵁᴸᴸ
watch_state: ᴺᵁᴸᴸ
path_created:
stat_czxid: 16925469
stat_mzxid: 16925469
stat_pzxid: 16926179
stat_version: 0
stat_cversion: 7
stat_dataLength: 0
stat_numChildren: 7
children: ['query-0000000006','query-0000000005','query-0000000004','query-0000000003','query-0000000002','query-0000000001','query-0000000000']
```
**See Also**
- [ZooKeeper](../../operations/tips.md#zookeeper)
- [ZooKeeper guide](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html)

View File

@ -197,7 +197,7 @@ Result:
## h3ToGeo {#h3togeo}
Returns `(lon, lat)` that corresponds to the provided H3 index.
Returns the geographical coordinates of longitude and latitude corresponding to the provided [H3](#h3index) index.
**Syntax**
@ -207,20 +207,18 @@ h3ToGeo(h3Index)
**Arguments**
- `h3Index` — H3 Index. Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
- `h3Index` — H3 Index. [UInt64](../../../sql-reference/data-types/int-uint.md).
**Returned values**
- `lon` — Longitude. Type: [Float64](../../../sql-reference/data-types/float.md).
- `lat` — Latitude. Type: [Float64](../../../sql-reference/data-types/float.md).
- A tuple consisting of two values: `tuple(lon,lat)`. `lon` — Longitude. [Float64](../../../sql-reference/data-types/float.md). `lat` — Latitude. [Float64](../../../sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT h3ToGeo(644325524701193974) coordinates;
SELECT h3ToGeo(644325524701193974) AS coordinates;
```
Result:
@ -230,6 +228,7 @@ Result:
│ (37.79506616830252,55.71290243145668) │
└───────────────────────────────────────┘
```
## h3kRing {#h3kring}
Lists all the [H3](#h3index) hexagons in the raduis of `k` from the given hexagon in random order.

View File

@ -1339,3 +1339,149 @@ Result:
│ 2,"good" │
└───────────────────────────────────────────┘
```
## snowflakeToDateTime {#snowflakeToDateTime}
Extract time from snowflake id as DateTime format.
**Syntax**
``` sql
snowflakeToDateTime(value [, time_zone])
```
**Parameters**
- `value``snowflake id`, Int64 value.
- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md).
**Returned value**
- value converted to the `DateTime` data type.
**Example**
Query:
``` sql
SELECT snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC');
```
Result:
``` text
┌─snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC')─┐
│ 2021-08-15 10:57:56 │
└──────────────────────────────────────────────────────────────────┘
```
## snowflakeToDateTime64 {#snowflakeToDateTime64}
Extract time from snowflake id as DateTime64 format.
**Syntax**
``` sql
snowflakeToDateTime64(value [, time_zone])
```
**Parameters**
- `value``snowflake id`, Int64 value.
- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md).
**Returned value**
- value converted to the `DateTime64` data type.
**Example**
Query:
``` sql
SELECT snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC');
```
Result:
``` text
┌─snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC')─┐
│ 2021-08-15 10:58:19.841 │
└────────────────────────────────────────────────────────────────────┘
```
## dateTimeToSnowflake {#dateTimeToSnowflake}
Convert DateTime to the first snowflake id at the giving time.
**Syntax**
``` sql
dateTimeToSnowflake(value)
```
**Parameters**
- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md).
**Returned value**
- `value` converted to the `Int64` data type as the first snowflake id at that time.
**Example**
Query:
``` sql
WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt
SELECT dateTimeToSnowflake(dt);
```
Result:
``` text
┌─dateTimeToSnowflake(dt)─┐
│ 1426860702823350272 │
└─────────────────────────┘
```
## dateTime64ToSnowflake {#dateTime64ToSnowflake}
Convert DateTime64 to the first snowflake id at the giving time.
**Syntax**
``` sql
dateTime64ToSnowflake(value)
```
**Parameters**
- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md).
**Returned value**
- `value` converted to the `Int64` data type as the first snowflake id at that time.
**Example**
Query:
``` sql
WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64
SELECT dateTime64ToSnowflake(dt64);
```
Result:
``` text
┌─dateTime64ToSnowflake(dt64)─┐
│ 1426860704886947840 │
└─────────────────────────────┘
```

View File

@ -384,5 +384,32 @@ ExpressionTransform
(ReadFromStorage)
NumbersMt × 2 0 → 1
```
### EXPLAIN ESTIMATE {#explain-estimate}
Shows the estimated number of rows, marks and parts to be read from the tables while processing the query. Works with tables in the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree) family.
**Example**
Creating a table:
```sql
CREATE TABLE ttt (i Int64) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity = 16, write_final_mark = 0;
INSERT INTO ttt SELECT number FROM numbers(128);
OPTIMIZE TABLE ttt;
```
Query:
```sql
EXPLAIN ESTIMATE SELECT * FROM ttt;
```
Result:
```text
┌─database─┬─table─┬─parts─┬─rows─┬─marks─┐
│ default │ ttt │ 1 │ 128 │ 8 │
└──────────┴───────┴───────┴──────┴───────┘
```
[Оriginal article](https://clickhouse.tech/docs/en/sql-reference/statements/explain/) <!--hide-->

View File

@ -311,12 +311,12 @@ One may execute query after:
- Individual replica path `/replicas/replica_name/` loss.
Replica attaches locally found parts and sends info about them to Zookeeper.
Parts present on replica before metadata loss are not re-fetched from other replicas if not being outdated
(so replica restoration does not mean re-downloading all data over the network).
Parts present on a replica before metadata loss are not re-fetched from other ones if not being outdated (so replica restoration does not mean re-downloading all data over the network).
Caveat: parts in all states are moved to `detached/` folder. Parts active before data loss (Committed) are attached.
!!! warning "Warning"
Parts in all states are moved to `detached/` folder. Parts active before data loss (committed) are attached.
#### Syntax
**Syntax**
```sql
SYSTEM RESTORE REPLICA [db.]replicated_merge_tree_family_table_name [ON CLUSTER cluster_name]
@ -328,11 +328,11 @@ Alternative syntax:
SYSTEM RESTORE REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name
```
#### Example
**Example**
Creating a table on multiple servers. After the replica's metadata in ZooKeeper is lost, the table will attach as read-only as metadata is missing. The last query needs to execute on every replica.
```sql
-- Creating table on multiple servers
CREATE TABLE test(n UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/', '{replica}')
ORDER BY n PARTITION BY n % 10;
@ -341,8 +341,14 @@ INSERT INTO test SELECT * FROM numbers(1000);
-- zookeeper_delete_path("/clickhouse/tables/test", recursive=True) <- root loss.
SYSTEM RESTART REPLICA test; -- Table will attach as readonly as metadata is missing.
SYSTEM RESTORE REPLICA test; -- Need to execute on every replica, another way: RESTORE REPLICA test ON CLUSTER cluster
SYSTEM RESTART REPLICA test;
SYSTEM RESTORE REPLICA test;
```
Another way:
```sql
SYSTEM RESTORE REPLICA test ON CLUSTER cluster;
```
### RESTART REPLICAS {#query_language-system-restart-replicas}

View File

@ -6,12 +6,13 @@ toc_title: cluster
# cluster, clusterAllReplicas {#cluster-clusterallreplicas}
Allows to access all shards in an existing cluster which configured in `remote_servers` section without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. One replica of each shard is queried.
`clusterAllReplicas` - same as `cluster` but all replicas are queried. Each replica in a cluster is used as separate shard/connection.
`clusterAllReplicas` function — same as `cluster`, but all replicas are queried. Each replica in a cluster is used as a separate shard/connection.
!!! note "Note"
All available clusters are listed in the `system.clusters` table.
All available clusters are listed in the [system.clusters](../../operations/system-tables/clusters.md) table.
Signatures:
**Syntax**
``` sql
cluster('cluster_name', db.table[, sharding_key])
@ -19,10 +20,27 @@ cluster('cluster_name', db, table[, sharding_key])
clusterAllReplicas('cluster_name', db.table[, sharding_key])
clusterAllReplicas('cluster_name', db, table[, sharding_key])
```
**Arguments**
`cluster_name` Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers.
- `cluster_name` Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers.
- `db.table` or `db`, `table` - Name of a database and a table.
- `sharding_key` - A sharding key. Optional. Needs to be specified if the cluster has more than one shard.
`sharding_key` - When insert into cluster function with more than one shard, sharding_key need to be provided.
**Returned value**
The dataset from clusters.
**Using Macros**
`cluster_name` can contain macros — substitution in curly brackets. The substituted value is taken from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration file.
Example:
```sql
SELECT * FROM cluster('{cluster}', default.example_table);
```
**Usage and Recommendations**
Using the `cluster` and `clusterAllReplicas` table functions are less efficient than creating a `Distributed` table because in this case, the server connection is re-established for every request. When processing a large number of queries, please always create the `Distributed` table ahead of time, and do not use the `cluster` and `clusterAllReplicas` table functions.

View File

@ -1,3 +1,7 @@
---
toc_priority: 36
toc_title: Replicated
---
# [экспериментальный] Replicated {#replicated}

View File

@ -38,9 +38,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
- Значение метрики. Тип данных: любой числовой.
- Версия метрики. Тип данных: любой числовой.
ClickHouse сохраняет строки с последней версией или последнюю записанную строку, если версии совпадают. Другие строки удаляются при слиянии кусков данных.
- Версия метрики. Тип данных: любой числовой (ClickHouse сохраняет строки с последней версией или последнюю записанную строку, если версии совпадают. Другие строки удаляются при слиянии кусков данных).
Имена этих столбцов должны быть заданы в конфигурации rollup.
@ -173,4 +171,4 @@ default
!!! warning "Внимание"
Прореживание данных производится во время слияний. Обычно для старых партций слияния не запускаются, поэтому для прореживания надо иницировать незапланированное слияние используя [optimize](../../../sql-reference/statements/optimize/). Или использовать дополнительные инструменты, например [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer).
Прореживание данных производится во время слияний. Обычно для старых партций слияния не запускаются, поэтому для прореживания надо иницировать незапланированное слияние используя [optimize](../../../sql-reference/statements/optimize.md). Или использовать дополнительные инструменты, например [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer).

View File

@ -102,7 +102,7 @@ CREATE TABLE table_name
) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}', ver)
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SAMPLE BY intHash32(UserID);
```
<details markdown="1">
@ -115,12 +115,12 @@ CREATE TABLE table_name
EventDate DateTime,
CounterID UInt32,
UserID UInt32
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192);
```
</details>
Как видно в примере, эти параметры могут содержать подстановки в фигурных скобках. Подставляемые значения достаются из конфигурационного файла, из секции «[macros](../../../operations/server-configuration-parameters/settings/#macros)».
Как видно в примере, эти параметры могут содержать подстановки в фигурных скобках. Эти подстановки заменяются на соответствующие значения из конфигурационного файла, из секции [macros](../../../operations/server-configuration-parameters/settings.md#macros).
Пример:

View File

@ -48,7 +48,10 @@ CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10
Если у одного из столбцов таблицы Buffer и подчинённой таблицы не совпадает тип, то в лог сервера будет записано сообщение об ошибке и буфер будет очищен.
То же самое происходит, если подчинённая таблица не существует в момент сброса буфера.
Если есть необходимость выполнить ALTER для подчинённой таблицы и для таблицы Buffer, то рекомендуется удалить таблицу Buffer, затем выполнить ALTER подчинённой таблицы, а затем создать таблицу Buffer заново.
Если есть необходимость выполнить ALTER для подчинённой таблицы и для таблицы Buffer, то рекомендуется удалить таблицу Buffer, затем выполнить ALTER подчинённой таблицы, а после создать таблицу Buffer заново.
!!! attention "Внимание"
В релизах до 28 сентября 2020 года выполнение ALTER на таблице Buffer ломает структуру блоков и вызывает ошибку (см. [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117)), поэтому удаление буфера и его пересоздание — единственный вариант миграции для данного движка. Перед выполнением ALTER на таблице Buffer убедитесь, что в вашей версии эта ошибка устранена.
При нештатном перезапуске сервера, данные, находящиеся в буфере, будут потеряны.

View File

@ -465,9 +465,9 @@ ClickHouse проверяет условия для `min_part_size` и `min_part
Подстановки параметров реплицируемых таблиц.
Можно не указывать, если реплицируемых таблицы не используются.
Можно не указывать, если реплицируемые таблицы не используются.
Подробнее смотрите в разделе «[Создание реплицируемых таблиц](../../engines/table-engines/mergetree-family/replication.md)».
Подробнее смотрите в разделе [Создание реплицируемых таблиц](../../engines/table-engines/mergetree-family/replication.md#creating-replicated-tables).
**Пример**

View File

@ -4,7 +4,7 @@
Столбцы:
- `trace_id` ([UUID](../../sql-reference/data-types/uuid.md) — идентификатор трассировки для выполненного запроса.
- `trace_id` ([UUID](../../sql-reference/data-types/uuid.md)) — идентификатор трассировки для выполненного запроса.
- `span_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — идентификатор `trace span`.

View File

@ -0,0 +1,129 @@
# system.zookeeper_log {#system-zookeeper_log}
Эта таблица содержит информацию о параметрах запроса к серверу ZooKeeper и ответа от него.
Для запросов заполняются только столбцы с параметрами запроса, а остальные столбцы заполняются значениями по умолчанию (`0` или `NULL`). Когда поступает ответ, данные добавляются в столбцы с параметрами ответа на запрос.
Столбцы с параметрами запроса:
- `type` ([Enum](../../sql-reference/data-types/enum.md)) — тип события в клиенте ZooKeeper. Может иметь одно из следующих значений:
- `Request` — запрос отправлен.
- `Response` — ответ получен.
- `Finalize` — соединение разорвано, ответ не получен.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата, когда произошло событие.
- `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — дата и время, когда произошло событие.
- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — IP адрес сервера ZooKeeper, с которого был сделан запрос.
- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — порт сервера ZooKeeper, с которого был сделан запрос.
- `session_id` ([Int64](../../sql-reference/data-types/int-uint.md)) — идентификатор сессии, который сервер ZooKeeper создает для каждого соединения.
- `xid` ([Int32](../../sql-reference/data-types/int-uint.md)) — идентификатор запроса внутри сессии. Обычно это последовательный номер запроса, одинаковый у строки запроса и у парной строки `response`/`finalize`.
- `has_watch` ([UInt8](../../sql-reference/data-types/int-uint.md)) — установлен ли запрос [watch](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#ch_zkWatches).
- `op_num` ([Enum](../../sql-reference/data-types/enum.md)) — тип запроса или ответа на запрос.
- `path` ([String](../../sql-reference/data-types/string.md)) — путь к узлу ZooKeeper, указанный в запросе. Пустая строка, если запрос не требует указания пути.
- `data` ([String](../../sql-reference/data-types/string.md)) — данные, записанные на узле ZooKeeper (для запросов `SET` и `CREATE` — что запрос хотел записать, для ответа на запрос `GET` — что было прочитано), или пустая строка.
- `is_ephemeral` ([UInt8](../../sql-reference/data-types/int-uint.md)) — создается ли узел ZooKeeper как [ephemeral](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#Ephemeral+Nodes).
- `is_sequential` ([UInt8](../../sql-reference/data-types/int-uint.md)) — создается ли узел ZooKeeper как [sequential](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#Sequence+Nodes+--+Unique+Naming).
- `version` ([Nullable(Int32)](../../sql-reference/data-types/nullable.md)) — версия узла ZooKeeper, которую запрос ожидает увидеть при выполнении. Поддерживается для запросов `CHECK`, `SET`, `REMOVE` (`-1` — запрос не проверяет версию, `NULL` — для других запросов, которые не поддерживают проверку версии).
- `requests_size` ([UInt32](../../sql-reference/data-types/int-uint.md)) — количество запросов, включенных в мультизапрос (это специальный запрос, который состоит из нескольких последовательных обычных запросов, выполняющихся атомарно). Все запросы, включенные в мультизапрос, имеют одинаковый `xid`.
- `request_idx` ([UInt32](../../sql-reference/data-types/int-uint.md)) — номер запроса, включенного в мультизапрос (`0` — для мультизапроса, далее по порядку с `1`).
Столбцы с параметрами ответа на запрос:
- `zxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — идентификатор транзакции в ZooKeeper. Последовательный номер, выданный сервером ZooKeeper в ответе на успешно выполненный запрос (`0` — запрос не был выполнен, возвращена ошибка или клиент ZooKeeper не знает, был ли выполнен запрос).
- `error` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — код ошибки. Может иметь много значений, здесь приведены только некоторые из них:
- `ZOK` — запрос успешно выполнен.
- `ZCONNECTIONLOSS` — соединение разорвано.
- `ZOPERATIONTIMEOUT` — истекло время ожидания выполнения запроса.
- `ZSESSIONEXPIRED` — истекло время сессии.
- `NULL` — выполнен запрос.
- `watch_type` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — тип события `watch` (для ответов на запрос при `op_num` = `Watch`), для остальных ответов: `NULL`.
- `watch_state` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — статус события `watch` (для ответов на запрос при `op_num` = `Watch`), для остальных ответов: `NULL`.
- `path_created` ([String](../../sql-reference/data-types/string.md)) — путь к созданному узлу ZooKeeper (для ответов на запрос `CREATE`). Может отличаться от `path`, если узел создается как `sequential`.
- `stat_czxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — идентификатор транзакции, в результате которой был создан узел ZooKeeper.
- `stat_mzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — идентификатор транзакции, которая последней модифицировала узел ZooKeeper.
- `stat_pzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — идентификатор транзакции, которая последней модифицировала дочерние узлы ZooKeeper.
- `stat_version` ([Int32](../../sql-reference/data-types/int-uint.md)) — количество изменений в данных узла ZooKeeper.
- `stat_cversion` ([Int32](../../sql-reference/data-types/int-uint.md)) — количество изменений в дочерних узлах ZooKeeper.
- `stat_dataLength` ([Int32](../../sql-reference/data-types/int-uint.md)) — длина поля данных узла ZooKeeper.
- `stat_numChildren` ([Int32](../../sql-reference/data-types/int-uint.md)) — количество дочерних узлов ZooKeeper.
- `children` ([Array(String)](../../sql-reference/data-types/array.md)) — список дочерних узлов ZooKeeper (для ответов на запрос `LIST`).
**Пример**
Запрос:
``` sql
SELECT * FROM system.zookeeper_log WHERE (session_id = '106662742089334927') AND (xid = '10858') FORMAT Vertical;
```
Результат:
``` text
Row 1:
──────
type: Request
event_date: 2021-08-09
event_time: 2021-08-09 21:38:30.291792
address: ::
port: 2181
session_id: 106662742089334927
xid: 10858
has_watch: 1
op_num: List
path: /clickhouse/task_queue/ddl
data:
is_ephemeral: 0
is_sequential: 0
version: ᴺᵁᴸᴸ
requests_size: 0
request_idx: 0
zxid: 0
error: ᴺᵁᴸᴸ
watch_type: ᴺᵁᴸᴸ
watch_state: ᴺᵁᴸᴸ
path_created:
stat_czxid: 0
stat_mzxid: 0
stat_pzxid: 0
stat_version: 0
stat_cversion: 0
stat_dataLength: 0
stat_numChildren: 0
children: []
Row 2:
──────
type: Response
event_date: 2021-08-09
event_time: 2021-08-09 21:38:30.292086
address: ::
port: 2181
session_id: 106662742089334927
xid: 10858
has_watch: 1
op_num: List
path: /clickhouse/task_queue/ddl
data:
is_ephemeral: 0
is_sequential: 0
version: ᴺᵁᴸᴸ
requests_size: 0
request_idx: 0
zxid: 16926267
error: ZOK
watch_type: ᴺᵁᴸᴸ
watch_state: ᴺᵁᴸᴸ
path_created:
stat_czxid: 16925469
stat_mzxid: 16925469
stat_pzxid: 16926179
stat_version: 0
stat_cversion: 7
stat_dataLength: 0
stat_numChildren: 7
children: ['query-0000000006','query-0000000005','query-0000000004','query-0000000003','query-0000000002','query-0000000001','query-0000000000']
```
**См. также**
- [ZooKeeper](../../operations/tips.md#zookeeper)
- [Руководство по ZooKeeper](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html)

View File

@ -193,6 +193,40 @@ SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index;
└────────────────────┘
```
## h3ToGeo {#h3togeo}
Возвращает географические координаты долготы и широты, соответствующие указанному [H3](#h3index)-индексу.
**Синтаксис**
``` sql
h3ToGeo(h3Index)
```
**Аргументы**
- `h3Index` — [H3](#h3index)-индекс. [UInt64](../../../sql-reference/data-types/int-uint.md).
**Возвращаемые значения**
- кортеж из двух значений: `tuple(lon,lat)`, где `lon` — долгота [Float64](../../../sql-reference/data-types/float.md), `lat` — широта [Float64](../../../sql-reference/data-types/float.md).
**Пример**
Запрос:
``` sql
SELECT h3ToGeo(644325524701193974) coordinates;
```
Результат:
``` text
┌─coordinates───────────────────────────┐
│ (37.79506616830252,55.71290243145668) │
└───────────────────────────────────────┘
```
## h3kRing {#h3kring}
Возвращает [H3](#h3index)-индексы шестигранников в радиусе `k` от данного в произвольном порядке.

View File

@ -385,4 +385,32 @@ ExpressionTransform
NumbersMt × 2 0 → 1
```
### EXPLAIN ESTIMATE {#explain-estimate}
Отображает оценки числа строк, засечек и кусков, которые будут прочитаны при выполнении запроса. Применяется для таблиц семейства [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree).
**Пример**
Создадим таблицу:
```sql
CREATE TABLE ttt (i Int64) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity = 16, write_final_mark = 0;
INSERT INTO ttt SELECT number FROM numbers(128);
OPTIMIZE TABLE ttt;
```
Запрос:
```sql
EXPLAIN ESTIMATE SELECT * FROM ttt;
```
Результат:
```text
┌─database─┬─table─┬─parts─┬─rows─┬─marks─┐
│ default │ ttt │ 1 │ 128 │ 8 │
└──────────┴───────┴───────┴──────┴───────┘
```
[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/explain/) <!--hide-->

View File

@ -36,6 +36,7 @@ toc_title: SYSTEM
- [START REPLICATION QUEUES](#query_language-system-start-replication-queues)
- [SYNC REPLICA](#query_language-system-sync-replica)
- [RESTART REPLICA](#query_language-system-restart-replica)
- [RESTORE REPLICA](#query_language-system-restore-replica)
- [RESTART REPLICAS](#query_language-system-restart-replicas)
## RELOAD EMBEDDED DICTIONARIES] {#query_language-system-reload-emdedded-dictionaries}
@ -287,13 +288,66 @@ SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name
### RESTART REPLICA {#query_language-system-restart-replica}
Реинициализация состояния Zookeeper-сессий для таблицы семейства `ReplicatedMergeTree`. Сравнивает текущее состояние с тем, что хранится в Zookeeper, как источник правды, и добавляет задачи в очередь репликации в Zookeeper, если необходимо.
Инициализация очереди репликации на основе данных ZooKeeper происходит так же, как при attach table. На короткое время таблица станет недоступной для любых операций.
Реинициализирует состояние сессий Zookeeper для таблицы семейства `ReplicatedMergeTree`. Сравнивает текущее состояние с состоянием в Zookeeper (как с эталоном) и при необходимости добавляет задачи в очередь репликации в Zookeeper.
Инициализация очереди репликации на основе данных ZooKeeper происходит так же, как при `ATTACH TABLE`. Некоторое время таблица будет недоступна для любых операций.
``` sql
SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name
```
### RESTORE REPLICA {#query_language-system-restore-replica}
Восстанавливает реплику, если метаданные в Zookeeper потеряны, но сами данные возможно существуют.
Работает только с таблицами семейства `ReplicatedMergeTree` и только если таблица находится в readonly-режиме.
Запрос можно выполнить если:
- потерян корневой путь ZooKeeper `/`;
- потерян путь реплик `/replicas`;
- потерян путь конкретной реплики `/replicas/replica_name/`.
К реплике прикрепляются локально найденные куски, информация о них отправляется в Zookeeper.
Если присутствующие в реплике до потери метаданных данные не устарели, они не скачиваются повторно с других реплик. Поэтому восстановление реплики не означает повторную загрузку всех данных по сети.
!!! warning "Предупреждение"
Потерянные данные в любых состояниях перемещаются в папку `detached/`. Куски, активные до потери данных (находившиеся в состоянии Committed), прикрепляются.
**Синтаксис**
```sql
SYSTEM RESTORE REPLICA [db.]replicated_merge_tree_family_table_name [ON CLUSTER cluster_name]
```
Альтернативный синтаксис:
```sql
SYSTEM RESTORE REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name
```
**Пример**
Создание таблицы на нескольких серверах. После потери корневого пути реплики таблица будет прикреплена только для чтения, так как метаданные отсутствуют. Последний запрос необходимо выполнить на каждой реплике.
```sql
CREATE TABLE test(n UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/', '{replica}')
ORDER BY n PARTITION BY n % 10;
INSERT INTO test SELECT * FROM numbers(1000);
-- zookeeper_delete_path("/clickhouse/tables/test", recursive=True) <- root loss.
SYSTEM RESTART REPLICA test;
SYSTEM RESTORE REPLICA test;
```
Альтернативный способ:
```sql
SYSTEM RESTORE REPLICA test ON CLUSTER cluster;
```
### RESTART REPLICAS {#query_language-system-restart-replicas}
Реинициализация состояния ZooKeeper-сессий для всех `ReplicatedMergeTree` таблиц. Сравнивает текущее состояние реплики с тем, что хранится в ZooKeeper, как c источником правды, и добавляет задачи в очередь репликации в ZooKeeper, если необходимо.

View File

@ -5,22 +5,44 @@ toc_title: cluster
# cluster, clusterAllReplicas {#cluster-clusterallreplicas}
Позволяет обратиться ко всем серверам существующего кластера, который присутствует в таблице `system.clusters` и сконфигурирован в секцци `remote_servers` без создания таблицы типа `Distributed`.
`clusterAllReplicas` - работает также как `cluster` но каждая реплика в кластере будет использована как отдельный шард/отдельное соединение.
Позволяет обратиться ко всем шардам существующего кластера, который сконфигурирован в секции `remote_servers` без создания таблицы типа [Distributed](../../engines/table-engines/special/distributed.md). В запросе используется одна реплика каждого шарда.
Функция `clusterAllReplicas` работает также как `cluster`, но каждая реплика в кластере используется как отдельный шард/отдельное соединение.
Сигнатуры:
!!! note "Примечание"
Все доступные кластеры перечислены в таблице [system.clusters](../../operations/system-tables/clusters.md).
**Синтаксис**
``` sql
cluster('cluster_name', db.table)
cluster('cluster_name', db, table)
clusterAllReplicas('cluster_name', db.table)
clusterAllReplicas('cluster_name', db, table)
cluster('cluster_name', db.table[, sharding_key])
cluster('cluster_name', db, table[, sharding_key])
clusterAllReplicas('cluster_name', db.table[, sharding_key])
clusterAllReplicas('cluster_name', db, table[, sharding_key])
```
**Аргументы**
- `cluster_name` имя кластера, который обозначает подмножество адресов и параметров подключения к удаленным и локальным серверам, входящим в кластер.
- `db.table` или `db`, `table` - имя базы данных и таблицы.
- `sharding_key` - ключ шардирования. Необязательный аргумент. Указывается, если данные добавляются более чем в один шард кластера.
**Возвращаемое значение**
Набор данных из кластеров.
**Использование макросов**
`cluster_name` может содержать макрос — подстановку в фигурных скобках. Эта подстановка заменяется на соответствующее значение из секции [macros](../../operations/server-configuration-parameters/settings.md#macros) конфигурационного файла .
Пример:
```sql
SELECT * FROM cluster('{cluster}', default.example_table);
```
`cluster_name` имя кластера, который обязан присутствовать в таблице `system.clusters` и обозначает подмножество адресов и параметров подключения к удаленным и локальным серверам, входящим в кластер.
**Использование и рекомендации**
Использование табличных функций `cluster` и `clusterAllReplicas` менее оптимальное чем создание таблицы типа `Distributed`, поскольку в этом случае соединение с сервером переустанавливается на каждый запрос. При обработке большого количества запросов, всегда создавайте `Distributed` таблицу заранее и не используйте табличные функции `cluster` и `clusterAllReplicas`.
Использование табличных функций `cluster` и `clusterAllReplicas` менее оптимально, чем создание таблицы типа `Distributed`, поскольку в этом случае при каждом новом запросе устанавливается новое соединение с сервером. При обработке большого количества запросов всегда создавайте `Distributed` таблицу заранее и не используйте табличные функции `cluster` и `clusterAllReplicas`.
Табличные функции `cluster` and `clusterAllReplicas` могут быть полезны в следующих случаях:
@ -30,7 +52,7 @@ clusterAllReplicas('cluster_name', db, table)
Настройки соединения `user`, `password`, `host`, `post`, `compression`, `secure` берутся из секции `<remote_servers>` файлов конфигурации. См. подробности в разделе [Distributed](../../engines/table-engines/special/distributed.md)
**See Also**
**См. также**
- [skip_unavailable_shards](../../operations/settings/settings.md#settings-skip_unavailable_shards)
- [load_balancing](../../operations/settings/settings.md#settings-load_balancing)

View File

@ -1,6 +1,4 @@
---
machine_translated: true
machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd
toc_priority: 49
toc_title: "\u6570\u636E\u5907\u4EFD"
---
@ -36,7 +34,7 @@ ClickHouse允许使用 `ALTER TABLE ... FREEZE PARTITION ...` 查询以创建表
数据可以使用 `ALTER TABLE ... ATTACH PARTITION ...` 从备份中恢复。
有关与分区操作相关的查询的详细信息,请参阅 [更改文档] (../sql-reference/statements/alter.md#alter_manipulations-with-partitions).
有关与分区操作相关的查询的详细信息,请参阅 [更改文档](../sql-reference/statements/alter.md#alter_manipulations-with-partitions).
第三方工具可用于自动化此方法: [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup).

View File

@ -1,9 +1,5 @@
---
machine_translated: true
machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3
---
# 系统。data_type_families {#system_tables-data_type_families}
# system.data_type_families {#system_tables-data_type_families}
包含有关受支持的[数据类型](../../sql-reference/data-types/)的信息.

View File

@ -26,7 +26,7 @@ toc_title: "常见问题"
### 服务器未运行 {#server-is-not-running}
**检查服务器是否运行nnig**
**检查服务器是否正在运行**
命令:

View File

@ -2,6 +2,7 @@
#include "Common/MemoryTracker.h"
#include "Columns/ColumnsNumber.h"
#include "ConnectionParameters.h"
#include "IO/CompressionMethod.h"
#include "QueryFuzzer.h"
#include "Suggest.h"
#include "TestHint.h"
@ -1823,7 +1824,7 @@ private:
void processInsertQuery()
{
const auto parsed_insert_query = parsed_query->as<ASTInsertQuery &>();
if (!parsed_insert_query.data && (is_interactive || (!stdin_is_a_tty && std_in.eof())))
if ((!parsed_insert_query.data && !parsed_insert_query.infile) && (is_interactive || (!stdin_is_a_tty && std_in.eof())))
throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT);
connection->sendQuery(
@ -1894,7 +1895,24 @@ private:
if (!parsed_insert_query)
return;
if (parsed_insert_query->data)
if (parsed_insert_query->infile)
{
const auto & in_file_node = parsed_insert_query->infile->as<ASTLiteral &>();
const auto in_file = in_file_node.value.safeGet<std::string>();
auto in_buffer = wrapReadBufferWithCompressionMethod(std::make_unique<ReadBufferFromFile>(in_file), chooseCompressionMethod(in_file, ""));
try
{
sendDataFrom(*in_buffer, sample, columns_description);
}
catch (Exception & e)
{
e.addMessage("data for INSERT was parsed from file");
throw;
}
}
else if (parsed_insert_query->data)
{
/// Send data contained in the query.
ReadBufferFromMemory data_in(parsed_insert_query->data, parsed_insert_query->end - parsed_insert_query->data);

View File

@ -17,6 +17,7 @@
#include <Poco/Version.h>
#include <Poco/Environment.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Core/ServerUUID.h>
#include <filesystem>
#include <IO/UseSSL.h>
@ -326,6 +327,8 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
}
}
DB::ServerUUID::load(path + "/uuid", log);
const Settings & settings = global_context->getSettingsRef();
GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 100));

View File

@ -12,6 +12,7 @@
#include <Interpreters/executeQuery.h>
#include <Interpreters/loadMetadata.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/Session.h>
#include <Common/Exception.h>
#include <Common/Macros.h>
#include <Common/Config/ConfigProcessor.h>
@ -374,14 +375,13 @@ void LocalServer::processQueries()
if (!parse_res.second)
throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR);
/// we can't mutate global global_context (can lead to races, as it was already passed to some background threads)
/// so we can't reuse it safely as a query context and need a copy here
auto context = Context::createCopy(global_context);
/// Authenticate and create a context to execute queries.
Session session{global_context, ClientInfo::Interface::TCP};
session.authenticate("default", "", Poco::Net::SocketAddress{});
context->makeSessionContext();
context->makeQueryContext();
context->setUser("default", "", Poco::Net::SocketAddress{});
/// Use the same context for all queries.
auto context = session.makeQueryContext();
context->makeSessionContext(); /// initial_create_query requires a session context to be set.
context->setCurrentQueryId("");
applyCmdSettings(context);

View File

@ -39,6 +39,7 @@
#include <Common/getMappedArea.h>
#include <Common/remapExecutable.h>
#include <Common/TLDListsHolder.h>
#include <Core/ServerUUID.h>
#include <IO/HTTPCommon.h>
#include <IO/ReadHelpers.h>
#include <IO/UseSSL.h>
@ -79,7 +80,6 @@
#include <Server/HTTP/HTTPServer.h>
#include <filesystem>
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
# include "Common/config_version.h"
@ -146,7 +146,6 @@ static bool jemallocOptionEnabled(const char *name)
static bool jemallocOptionEnabled(const char *) { return 0; }
#endif
int mainEntryClickHouseServer(int argc, char ** argv)
{
DB::Server app;
@ -667,13 +666,14 @@ if (ThreadFuzzer::instance().isEffective())
global_context->setRemoteHostFilter(config());
std::string path = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
fs::path path = path_str;
std::string default_database = config().getString("default_database", "default");
/// Check that the process user id matches the owner of the data.
const auto effective_user_id = geteuid();
struct stat statbuf;
if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid)
if (stat(path_str.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid)
{
const auto effective_user = getUserName(effective_user_id);
const auto data_owner = getUserName(statbuf.st_uid);
@ -690,9 +690,11 @@ if (ThreadFuzzer::instance().isEffective())
}
}
global_context->setPath(path);
global_context->setPath(path_str);
StatusFile status{path + "status", StatusFile::write_full_info};
StatusFile status{path / "status", StatusFile::write_full_info};
DB::ServerUUID::load(path / "uuid", log);
/// Try to increase limit on number of open files.
{
@ -726,7 +728,7 @@ if (ThreadFuzzer::instance().isEffective())
/// Storage with temporary data for processing of heavy queries.
{
std::string tmp_path = config().getString("tmp_path", path + "tmp/");
std::string tmp_path = config().getString("tmp_path", path / "tmp/");
std::string tmp_policy = config().getString("tmp_policy", "");
const VolumePtr & volume = global_context->setTemporaryStorage(tmp_path, tmp_policy);
for (const DiskPtr & disk : volume->getDisks())
@ -738,7 +740,7 @@ if (ThreadFuzzer::instance().isEffective())
* Examples: do repair of local data; clone all replicated tables from replica.
*/
{
auto flags_path = fs::path(path) / "flags/";
auto flags_path = path / "flags/";
fs::create_directories(flags_path);
global_context->setFlagsPath(flags_path);
}
@ -747,29 +749,29 @@ if (ThreadFuzzer::instance().isEffective())
*/
{
std::string user_files_path = config().getString("user_files_path", fs::path(path) / "user_files/");
std::string user_files_path = config().getString("user_files_path", path / "user_files/");
global_context->setUserFilesPath(user_files_path);
fs::create_directories(user_files_path);
}
{
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", fs::path(path) / "dictionaries_lib/");
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", path / "dictionaries_lib/");
global_context->setDictionariesLibPath(dictionaries_lib_path);
fs::create_directories(dictionaries_lib_path);
}
/// top_level_domains_lists
{
const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/");
const std::string & top_level_domains_path = config().getString("top_level_domains_path", path / "top_level_domains/");
TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config());
}
{
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
fs::create_directories(path / "data/");
fs::create_directories(path / "metadata/");
/// Directory with metadata of tables, which was marked as dropped by Atomic database
fs::create_directories(fs::path(path) / "metadata_dropped/");
fs::create_directories(path / "metadata_dropped/");
}
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
@ -952,7 +954,7 @@ if (ThreadFuzzer::instance().isEffective())
#endif
/// Set path for format schema files
fs::path format_schema_path(config().getString("format_schema_path", fs::path(path) / "format_schemas/"));
fs::path format_schema_path(config().getString("format_schema_path", path / "format_schemas/"));
global_context->setFormatSchemaPath(format_schema_path);
fs::create_directories(format_schema_path);
@ -1088,7 +1090,7 @@ if (ThreadFuzzer::instance().isEffective())
/// system logs may copy global context.
global_context->setCurrentDatabaseNameInGlobalContext(default_database);
LOG_INFO(log, "Loading metadata from {}", path);
LOG_INFO(log, "Loading metadata from {}", path_str);
try
{
@ -1428,7 +1430,6 @@ if (ThreadFuzzer::instance().isEffective())
/// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread.
async_metrics.start();
global_context->enableNamedSessions();
{
String level_str = config().getString("text_log.level", "");

View File

@ -1,6 +1,7 @@
#include <Access/AccessControlManager.h>
#include <Access/MultipleAccessStorage.h>
#include <Access/MemoryAccessStorage.h>
#include <Access/ReplicatedAccessStorage.h>
#include <Access/UsersConfigAccessStorage.h>
#include <Access/DiskAccessStorage.h>
#include <Access/LDAPAccessStorage.h>
@ -225,6 +226,22 @@ void AccessControlManager::startPeriodicReloadingUsersConfigs()
}
}
void AccessControlManager::addReplicatedStorage(
const String & storage_name_,
const String & zookeeper_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_)
{
auto storages = getStoragesPtr();
for (const auto & storage : *storages)
{
if (auto replicated_storage = typeid_cast<std::shared_ptr<ReplicatedAccessStorage>>(storage))
return;
}
auto new_storage = std::make_shared<ReplicatedAccessStorage>(storage_name_, zookeeper_path_, get_zookeeper_function_);
addStorage(new_storage);
LOG_DEBUG(getLogger(), "Added {} access storage '{}'", String(new_storage->getStorageType()), new_storage->getStorageName());
new_storage->startup();
}
void AccessControlManager::addDiskStorage(const String & directory_, bool readonly_)
{
@ -322,6 +339,11 @@ void AccessControlManager::addStoragesFromUserDirectoriesConfig(
{
addLDAPStorage(name, config, prefix);
}
else if (type == ReplicatedAccessStorage::STORAGE_TYPE)
{
String zookeeper_path = config.getString(prefix + ".zookeeper_path");
addReplicatedStorage(name, zookeeper_path, get_zookeeper_function);
}
else
throw Exception("Unknown storage type '" + type + "' at " + prefix + " in config", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}

View File

@ -84,6 +84,10 @@ public:
/// Adds LDAPAccessStorage which allows querying remote LDAP server for user info.
void addLDAPStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_);
void addReplicatedStorage(const String & storage_name,
const String & zookeeper_path,
const zkutil::GetZooKeeper & get_zookeeper_function);
/// Adds storages from <users_directories> config.
void addStoragesFromUserDirectoriesConfig(const Poco::Util::AbstractConfiguration & config,
const String & key,

View File

@ -0,0 +1,175 @@
#include <Access/AccessEntityIO.h>
#include <Access/IAccessEntity.h>
#include <Access/IAccessStorage.h>
#include <Access/Quota.h>
#include <Access/Role.h>
#include <Access/RowPolicy.h>
#include <Access/SettingsProfile.h>
#include <Access/User.h>
#include <Core/Defines.h>
#include <Interpreters/InterpreterCreateQuotaQuery.h>
#include <Interpreters/InterpreterCreateRoleQuery.h>
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Interpreters/InterpreterCreateSettingsProfileQuery.h>
#include <Interpreters/InterpreterCreateUserQuery.h>
#include <Interpreters/InterpreterGrantQuery.h>
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ParserCreateQuotaQuery.h>
#include <Parsers/ParserCreateRoleQuery.h>
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ParserCreateSettingsProfileQuery.h>
#include <Parsers/ParserCreateUserQuery.h>
#include <Parsers/ParserGrantQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION;
}
using EntityType = IAccessStorage::EntityType;
using EntityTypeInfo = IAccessStorage::EntityTypeInfo;
namespace
{
/// Special parser for the 'ATTACH access entity' queries.
class ParserAttachAccessEntity : public IParserBase
{
protected:
const char * getName() const override { return "ATTACH access entity query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
ParserCreateUserQuery create_user_p;
ParserCreateRoleQuery create_role_p;
ParserCreateRowPolicyQuery create_policy_p;
ParserCreateQuotaQuery create_quota_p;
ParserCreateSettingsProfileQuery create_profile_p;
ParserGrantQuery grant_p;
create_user_p.useAttachMode();
create_role_p.useAttachMode();
create_policy_p.useAttachMode();
create_quota_p.useAttachMode();
create_profile_p.useAttachMode();
grant_p.useAttachMode();
return create_user_p.parse(pos, node, expected) || create_role_p.parse(pos, node, expected)
|| create_policy_p.parse(pos, node, expected) || create_quota_p.parse(pos, node, expected)
|| create_profile_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected);
}
};
}
String serializeAccessEntity(const IAccessEntity & entity)
{
/// Build list of ATTACH queries.
ASTs queries;
queries.push_back(InterpreterShowCreateAccessEntityQuery::getAttachQuery(entity));
if ((entity.getType() == EntityType::USER) || (entity.getType() == EntityType::ROLE))
boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity));
/// Serialize the list of ATTACH queries to a string.
WriteBufferFromOwnString buf;
for (const ASTPtr & query : queries)
{
formatAST(*query, buf, false, true);
buf.write(";\n", 2);
}
return buf.str();
}
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path)
{
ASTs queries;
ParserAttachAccessEntity parser;
const char * begin = definition.data(); /// begin of current query
const char * pos = begin; /// parser moves pos from begin to the end of current query
const char * end = begin + definition.size();
while (pos < end)
{
queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
while (isWhitespaceASCII(*pos) || *pos == ';')
++pos;
}
/// Interpret the AST to build an access entity.
std::shared_ptr<User> user;
std::shared_ptr<Role> role;
std::shared_ptr<RowPolicy> policy;
std::shared_ptr<Quota> quota;
std::shared_ptr<SettingsProfile> profile;
AccessEntityPtr res;
for (const auto & query : queries)
{
if (auto * create_user_query = query->as<ASTCreateUserQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = user = std::make_unique<User>();
InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query);
}
else if (auto * create_role_query = query->as<ASTCreateRoleQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = role = std::make_unique<Role>();
InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query);
}
else if (auto * create_policy_query = query->as<ASTCreateRowPolicyQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = policy = std::make_unique<RowPolicy>();
InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query);
}
else if (auto * create_quota_query = query->as<ASTCreateQuotaQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = quota = std::make_unique<Quota>();
InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query);
}
else if (auto * create_profile_query = query->as<ASTCreateSettingsProfileQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = profile = std::make_unique<SettingsProfile>();
InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query);
}
else if (auto * grant_query = query->as<ASTGrantQuery>())
{
if (!user && !role)
throw Exception(
"A user or role should be attached before grant in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
if (user)
InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query);
else
InterpreterGrantQuery::updateRoleFromQuery(*role, *grant_query);
}
else
throw Exception("No interpreter found for query " + query->getID(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
}
if (!res)
throw Exception("No access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
return res;
}
}

View File

@ -0,0 +1,12 @@
#pragma once
#include <Access/IAccessEntity.h>
namespace DB
{
String serializeAccessEntity(const IAccessEntity & entity);
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path);
}

View File

@ -70,6 +70,7 @@ public:
/// Returns the current user. The function can return nullptr.
UserPtr getUser() const;
String getUserName() const;
std::optional<UUID> getUserID() const { return getParams().user_id; }
/// Returns information about current and enabled roles.
std::shared_ptr<const EnabledRolesInfo> getRolesInfo() const;

View File

@ -26,6 +26,8 @@ protected:
String user_name;
};
/// Does not check the password/credentials and that the specified host is allowed.
/// (Used only internally in cluster, if the secret matches)
class AlwaysAllowCredentials
: public Credentials
{

View File

@ -4,41 +4,20 @@
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromString.h>
#include <Access/AccessEntityIO.h>
#include <Access/User.h>
#include <Access/Role.h>
#include <Access/RowPolicy.h>
#include <Access/Quota.h>
#include <Access/SettingsProfile.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ParserCreateUserQuery.h>
#include <Parsers/ParserCreateRoleQuery.h>
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ParserCreateQuotaQuery.h>
#include <Parsers/ParserCreateSettingsProfileQuery.h>
#include <Parsers/ParserGrantQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/InterpreterCreateUserQuery.h>
#include <Interpreters/InterpreterCreateRoleQuery.h>
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Interpreters/InterpreterCreateQuotaQuery.h>
#include <Interpreters/InterpreterCreateSettingsProfileQuery.h>
#include <Interpreters/InterpreterGrantQuery.h>
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Common/quoteString.h>
#include <Core/Defines.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Stringifier.h>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
#include <filesystem>
#include <fstream>
@ -49,7 +28,6 @@ namespace ErrorCodes
{
extern const int DIRECTORY_DOESNT_EXIST;
extern const int FILE_DOESNT_EXIST;
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION;
}
@ -58,34 +36,6 @@ namespace
using EntityType = IAccessStorage::EntityType;
using EntityTypeInfo = IAccessStorage::EntityTypeInfo;
/// Special parser for the 'ATTACH access entity' queries.
class ParserAttachAccessEntity : public IParserBase
{
protected:
const char * getName() const override { return "ATTACH access entity query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
ParserCreateUserQuery create_user_p;
ParserCreateRoleQuery create_role_p;
ParserCreateRowPolicyQuery create_policy_p;
ParserCreateQuotaQuery create_quota_p;
ParserCreateSettingsProfileQuery create_profile_p;
ParserGrantQuery grant_p;
create_user_p.useAttachMode();
create_role_p.useAttachMode();
create_policy_p.useAttachMode();
create_quota_p.useAttachMode();
create_profile_p.useAttachMode();
grant_p.useAttachMode();
return create_user_p.parse(pos, node, expected) || create_role_p.parse(pos, node, expected)
|| create_policy_p.parse(pos, node, expected) || create_quota_p.parse(pos, node, expected)
|| create_profile_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected);
}
};
/// Reads a file containing ATTACH queries and then parses it to build an access entity.
AccessEntityPtr readEntityFile(const String & file_path)
@ -96,80 +46,7 @@ namespace
readStringUntilEOF(file_contents, in);
/// Parse the file contents.
ASTs queries;
ParserAttachAccessEntity parser;
const char * begin = file_contents.data(); /// begin of current query
const char * pos = begin; /// parser moves pos from begin to the end of current query
const char * end = begin + file_contents.size();
while (pos < end)
{
queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
while (isWhitespaceASCII(*pos) || *pos == ';')
++pos;
}
/// Interpret the AST to build an access entity.
std::shared_ptr<User> user;
std::shared_ptr<Role> role;
std::shared_ptr<RowPolicy> policy;
std::shared_ptr<Quota> quota;
std::shared_ptr<SettingsProfile> profile;
AccessEntityPtr res;
for (const auto & query : queries)
{
if (auto * create_user_query = query->as<ASTCreateUserQuery>())
{
if (res)
throw Exception("Two access entities in one file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = user = std::make_unique<User>();
InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query);
}
else if (auto * create_role_query = query->as<ASTCreateRoleQuery>())
{
if (res)
throw Exception("Two access entities in one file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = role = std::make_unique<Role>();
InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query);
}
else if (auto * create_policy_query = query->as<ASTCreateRowPolicyQuery>())
{
if (res)
throw Exception("Two access entities in one file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = policy = std::make_unique<RowPolicy>();
InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query);
}
else if (auto * create_quota_query = query->as<ASTCreateQuotaQuery>())
{
if (res)
throw Exception("Two access entities are attached in the same file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = quota = std::make_unique<Quota>();
InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query);
}
else if (auto * create_profile_query = query->as<ASTCreateSettingsProfileQuery>())
{
if (res)
throw Exception("Two access entities are attached in the same file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = profile = std::make_unique<SettingsProfile>();
InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query);
}
else if (auto * grant_query = query->as<ASTGrantQuery>())
{
if (!user && !role)
throw Exception("A user or role should be attached before grant in file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
if (user)
InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query);
else
InterpreterGrantQuery::updateRoleFromQuery(*role, *grant_query);
}
else
throw Exception("No interpreter found for query " + query->getID(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
}
if (!res)
throw Exception("No access entities attached in file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
return res;
return deserializeAccessEntity(file_contents, file_path);
}
@ -186,24 +63,10 @@ namespace
}
}
/// Writes ATTACH queries for building a specified access entity to a file.
void writeEntityFile(const String & file_path, const IAccessEntity & entity)
{
/// Build list of ATTACH queries.
ASTs queries;
queries.push_back(InterpreterShowCreateAccessEntityQuery::getAttachQuery(entity));
if ((entity.getType() == EntityType::USER) || (entity.getType() == EntityType::ROLE))
boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity));
/// Serialize the list of ATTACH queries to a string.
WriteBufferFromOwnString buf;
for (const ASTPtr & query : queries)
{
formatAST(*query, buf, false, true);
buf.write(";\n", 2);
}
String file_contents = buf.str();
String file_contents = serializeAccessEntity(entity);
/// First we save *.tmp file and then we rename if everything's ok.
auto tmp_file_path = std::filesystem::path{file_path}.replace_extension(".tmp");

View File

@ -0,0 +1,618 @@
#include <Access/AccessEntityIO.h>
#include <Access/MemoryAccessStorage.h>
#include <Access/ReplicatedAccessStorage.h>
#include <IO/ReadHelpers.h>
#include <boost/container/flat_set.hpp>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/escapeForFileName.h>
#include <common/range.h>
#include <common/sleep.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NO_ZOOKEEPER;
}
static UUID parseUUID(const String & text)
{
UUID uuid = UUIDHelpers::Nil;
auto buffer = ReadBufferFromMemory(text.data(), text.length());
readUUIDText(uuid, buffer);
return uuid;
}
ReplicatedAccessStorage::ReplicatedAccessStorage(
const String & storage_name_,
const String & zookeeper_path_,
zkutil::GetZooKeeper get_zookeeper_)
: IAccessStorage(storage_name_)
, zookeeper_path(zookeeper_path_)
, get_zookeeper(get_zookeeper_)
{
if (zookeeper_path.empty())
throw Exception("ZooKeeper path must be non-empty", ErrorCodes::BAD_ARGUMENTS);
if (zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1);
/// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
if (zookeeper_path.front() != '/')
zookeeper_path = "/" + zookeeper_path;
}
ReplicatedAccessStorage::~ReplicatedAccessStorage()
{
ReplicatedAccessStorage::shutdown();
}
void ReplicatedAccessStorage::startup()
{
initializeZookeeper();
worker_thread = ThreadFromGlobalPool(&ReplicatedAccessStorage::runWorkerThread, this);
}
void ReplicatedAccessStorage::shutdown()
{
bool prev_stop_flag = stop_flag.exchange(true);
if (!prev_stop_flag)
{
/// Notify the worker thread to stop waiting for new queue items
refresh_queue.push(UUIDHelpers::Nil);
worker_thread.join();
}
}
template <typename Func>
static void retryOnZooKeeperUserError(size_t attempts, Func && function)
{
while (attempts > 0)
{
try
{
function();
return;
}
catch (zkutil::KeeperException & keeper_exception)
{
if (Coordination::isUserError(keeper_exception.code) && attempts > 1)
attempts -= 1;
else
throw;
}
}
}
UUID ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists)
{
const UUID id = generateRandomID();
const EntityTypeInfo type_info = EntityTypeInfo::get(new_entity->getType());
const String & name = new_entity->getName();
LOG_DEBUG(getLogger(), "Inserting entity of type {} named {} with id {}", type_info.name, name, toString(id));
auto zookeeper = get_zookeeper();
retryOnZooKeeperUserError(10, [&]{ insertZooKeeper(zookeeper, id, new_entity, replace_if_exists); });
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
refreshEntityNoLock(zookeeper, id, notifications);
return id;
}
void ReplicatedAccessStorage::insertZooKeeper(
const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists)
{
const String & name = new_entity->getName();
const EntityType type = new_entity->getType();
const EntityTypeInfo type_info = EntityTypeInfo::get(type);
const String entity_uuid = toString(id);
/// The entity data will be stored here, this ensures all entities have unique ids
const String entity_path = zookeeper_path + "/uuid/" + entity_uuid;
/// Then we create a znode with the entity name, inside the znode of each entity type
/// This ensure all entities of the same type have a unique name
const String name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(name);
Coordination::Requests ops;
const String new_entity_definition = serializeAccessEntity(*new_entity);
ops.emplace_back(zkutil::makeCreateRequest(entity_path, new_entity_definition, zkutil::CreateMode::Persistent));
/// The content of the "name" znode is the uuid of the entity owning that name
ops.emplace_back(zkutil::makeCreateRequest(name_path, entity_uuid, zkutil::CreateMode::Persistent));
Coordination::Responses responses;
const Coordination::Error res = zookeeper->tryMulti(ops, responses);
if (res == Coordination::Error::ZNODEEXISTS)
{
if (responses[0]->error == Coordination::Error::ZNODEEXISTS)
{
/// The UUID already exists, simply fail.
/// To fail with a nice error message, we need info about what already exists.
/// This itself could fail if the conflicting uuid disappears in the meantime.
/// If that happens, then we'll just retry from the start.
String existing_entity_definition = zookeeper->get(entity_path);
AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path);
EntityType existing_type = existing_entity->getType();
String existing_name = existing_entity->getName();
throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name);
}
else if (replace_if_exists)
{
/// The name already exists for this type.
/// If asked to, we need to replace the existing entity.
/// First get the uuid of the existing entity
/// This itself could fail if the conflicting name disappears in the meantime.
/// If that happens, then we'll just retry from the start.
Coordination::Stat name_stat;
String existing_entity_uuid = zookeeper->get(name_path, &name_stat);
const String existing_entity_path = zookeeper_path + "/uuid/" + existing_entity_uuid;
Coordination::Requests replace_ops;
replace_ops.emplace_back(zkutil::makeRemoveRequest(existing_entity_path, -1));
replace_ops.emplace_back(zkutil::makeCreateRequest(entity_path, new_entity_definition, zkutil::CreateMode::Persistent));
replace_ops.emplace_back(zkutil::makeSetRequest(name_path, entity_uuid, name_stat.version));
/// If this fails, then we'll just retry from the start.
zookeeper->multi(replace_ops);
}
else
{
throwNameCollisionCannotInsert(type, name);
}
}
else
{
zkutil::KeeperMultiException::check(res, ops, responses);
}
}
void ReplicatedAccessStorage::removeImpl(const UUID & id)
{
LOG_DEBUG(getLogger(), "Removing entity {}", toString(id));
auto zookeeper = get_zookeeper();
retryOnZooKeeperUserError(10, [&] { removeZooKeeper(zookeeper, id); });
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
removeEntityNoLock(id, notifications);
}
void ReplicatedAccessStorage::removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id)
{
const String entity_uuid = toString(id);
const String entity_path = zookeeper_path + "/uuid/" + entity_uuid;
String entity_definition;
Coordination::Stat entity_stat;
const bool uuid_exists = zookeeper->tryGet(entity_path, entity_definition, &entity_stat);
if (!uuid_exists)
throwNotFound(id);
const AccessEntityPtr entity = deserializeAccessEntity(entity_definition, entity_path);
const EntityTypeInfo type_info = EntityTypeInfo::get(entity->getType());
const String & name = entity->getName();
const String entity_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(name);
Coordination::Requests ops;
ops.emplace_back(zkutil::makeRemoveRequest(entity_path, entity_stat.version));
ops.emplace_back(zkutil::makeRemoveRequest(entity_name_path, -1));
/// If this fails, then we'll just retry from the start.
zookeeper->multi(ops);
}
void ReplicatedAccessStorage::updateImpl(const UUID & id, const UpdateFunc & update_func)
{
LOG_DEBUG(getLogger(), "Updating entity {}", toString(id));
auto zookeeper = get_zookeeper();
retryOnZooKeeperUserError(10, [&] { updateZooKeeper(zookeeper, id, update_func); });
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
refreshEntityNoLock(zookeeper, id, notifications);
}
void ReplicatedAccessStorage::updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func)
{
const String entity_uuid = toString(id);
const String entity_path = zookeeper_path + "/uuid/" + entity_uuid;
String old_entity_definition;
Coordination::Stat stat;
const bool uuid_exists = zookeeper->tryGet(entity_path, old_entity_definition, &stat);
if (!uuid_exists)
throwNotFound(id);
const AccessEntityPtr old_entity = deserializeAccessEntity(old_entity_definition, entity_path);
const AccessEntityPtr new_entity = update_func(old_entity);
if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());
const EntityTypeInfo type_info = EntityTypeInfo::get(new_entity->getType());
Coordination::Requests ops;
const String new_entity_definition = serializeAccessEntity(*new_entity);
ops.emplace_back(zkutil::makeSetRequest(entity_path, new_entity_definition, stat.version));
const String & old_name = old_entity->getName();
const String & new_name = new_entity->getName();
if (new_name != old_name)
{
auto old_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(old_name);
auto new_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(new_name);
ops.emplace_back(zkutil::makeRemoveRequest(old_name_path, -1));
ops.emplace_back(zkutil::makeCreateRequest(new_name_path, entity_uuid, zkutil::CreateMode::Persistent));
}
Coordination::Responses responses;
const Coordination::Error res = zookeeper->tryMulti(ops, responses);
if (res == Coordination::Error::ZNODEEXISTS)
{
throwNameCollisionCannotRename(new_entity->getType(), old_name, new_name);
}
else if (res == Coordination::Error::ZNONODE)
{
throwNotFound(id);
}
else
{
zkutil::KeeperMultiException::check(res, ops, responses);
}
}
void ReplicatedAccessStorage::runWorkerThread()
{
LOG_DEBUG(getLogger(), "Started worker thread");
while (!stop_flag)
{
try
{
if (!initialized)
initializeZookeeper();
refresh();
}
catch (...)
{
tryLogCurrentException(getLogger(), "Unexpected error, will try to restart worker thread:");
resetAfterError();
sleepForSeconds(5);
}
}
}
void ReplicatedAccessStorage::resetAfterError()
{
initialized = false;
UUID id;
while (refresh_queue.tryPop(id)) {}
std::lock_guard lock{mutex};
for (const auto type : collections::range(EntityType::MAX))
entries_by_name_and_type[static_cast<size_t>(type)].clear();
entries_by_id.clear();
}
void ReplicatedAccessStorage::initializeZookeeper()
{
assert(!initialized);
auto zookeeper = get_zookeeper();
if (!zookeeper)
throw Exception("Can't have Replicated access without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
createRootNodes(zookeeper);
refreshEntities(zookeeper);
initialized = true;
}
void ReplicatedAccessStorage::createRootNodes(const zkutil::ZooKeeperPtr & zookeeper)
{
zookeeper->createAncestors(zookeeper_path);
zookeeper->createIfNotExists(zookeeper_path, "");
zookeeper->createIfNotExists(zookeeper_path + "/uuid", "");
for (const auto type : collections::range(EntityType::MAX))
{
/// Create a znode for each type of AccessEntity
const auto type_info = EntityTypeInfo::get(type);
zookeeper->createIfNotExists(zookeeper_path + "/" + type_info.unique_char, "");
}
}
void ReplicatedAccessStorage::refresh()
{
UUID id;
if (refresh_queue.tryPop(id, /* timeout_ms: */ 10000))
{
if (stop_flag)
return;
auto zookeeper = get_zookeeper();
if (id == UUIDHelpers::Nil)
refreshEntities(zookeeper);
else
refreshEntity(zookeeper, id);
}
}
void ReplicatedAccessStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper)
{
LOG_DEBUG(getLogger(), "Refreshing entities list");
const String zookeeper_uuids_path = zookeeper_path + "/uuid";
auto watch_entities_list = [this](const Coordination::WatchResponse &)
{
refresh_queue.push(UUIDHelpers::Nil);
};
Coordination::Stat stat;
const auto entity_uuid_strs = zookeeper->getChildrenWatch(zookeeper_uuids_path, &stat, watch_entities_list);
std::unordered_set<UUID> entity_uuids;
entity_uuids.reserve(entity_uuid_strs.size());
for (const String & entity_uuid_str : entity_uuid_strs)
entity_uuids.insert(parseUUID(entity_uuid_str));
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
std::vector<UUID> entities_to_remove;
/// Locally remove entities that were removed from ZooKeeper
for (const auto & pair : entries_by_id)
{
const UUID & entity_uuid = pair.first;
if (!entity_uuids.contains(entity_uuid))
entities_to_remove.push_back(entity_uuid);
}
for (const auto & entity_uuid : entities_to_remove)
removeEntityNoLock(entity_uuid, notifications);
/// Locally add entities that were added to ZooKeeper
for (const auto & entity_uuid : entity_uuids)
{
const auto it = entries_by_id.find(entity_uuid);
if (it == entries_by_id.end())
refreshEntityNoLock(zookeeper, entity_uuid, notifications);
}
LOG_DEBUG(getLogger(), "Refreshing entities list finished");
}
void ReplicatedAccessStorage::refreshEntity(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id)
{
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
refreshEntityNoLock(zookeeper, id, notifications);
}
void ReplicatedAccessStorage::refreshEntityNoLock(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, Notifications & notifications)
{
LOG_DEBUG(getLogger(), "Refreshing entity {}", toString(id));
const auto watch_entity = [this, id](const Coordination::WatchResponse & response)
{
if (response.type == Coordination::Event::CHANGED)
refresh_queue.push(id);
};
Coordination::Stat entity_stat;
const String entity_path = zookeeper_path + "/uuid/" + toString(id);
String entity_definition;
const bool exists = zookeeper->tryGetWatch(entity_path, entity_definition, &entity_stat, watch_entity);
if (exists)
{
const AccessEntityPtr entity = deserializeAccessEntity(entity_definition, entity_path);
setEntityNoLock(id, entity, notifications);
}
else
{
removeEntityNoLock(id, notifications);
}
}
void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntityPtr & entity, Notifications & notifications)
{
LOG_DEBUG(getLogger(), "Setting id {} to entity named {}", toString(id), entity->getName());
const EntityType type = entity->getType();
const String & name = entity->getName();
/// If the type+name already exists and is a different entity, remove old entity
auto & entries_by_name = entries_by_name_and_type[static_cast<size_t>(type)];
if (auto it = entries_by_name.find(name); it != entries_by_name.end() && it->second->id != id)
{
removeEntityNoLock(it->second->id, notifications);
}
/// If the entity already exists under a different type+name, remove old type+name
if (auto it = entries_by_id.find(id); it != entries_by_id.end())
{
const AccessEntityPtr & existing_entity = it->second.entity;
const EntityType existing_type = existing_entity->getType();
const String & existing_name = existing_entity->getName();
if (existing_type != type || existing_name != name)
{
auto & existing_entries_by_name = entries_by_name_and_type[static_cast<size_t>(existing_type)];
existing_entries_by_name.erase(existing_name);
}
}
auto & entry = entries_by_id[id];
entry.id = id;
entry.entity = entity;
entries_by_name[name] = &entry;
prepareNotifications(entry, false, notifications);
}
void ReplicatedAccessStorage::removeEntityNoLock(const UUID & id, Notifications & notifications)
{
LOG_DEBUG(getLogger(), "Removing entity with id {}", toString(id));
const auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
{
LOG_DEBUG(getLogger(), "Id {} not found, ignoring removal", toString(id));
return;
}
const Entry & entry = it->second;
const EntityType type = entry.entity->getType();
const String & name = entry.entity->getName();
prepareNotifications(entry, true, notifications);
auto & entries_by_name = entries_by_name_and_type[static_cast<size_t>(type)];
const auto name_it = entries_by_name.find(name);
if (name_it == entries_by_name.end())
LOG_WARNING(getLogger(), "Entity {} not found in names, ignoring removal of name", toString(id));
else if (name_it->second != &(it->second))
LOG_WARNING(getLogger(), "Name {} not pointing to entity {}, ignoring removal of name", name, toString(id));
else
entries_by_name.erase(name);
entries_by_id.erase(id);
LOG_DEBUG(getLogger(), "Removed entity with id {}", toString(id));
}
std::optional<UUID> ReplicatedAccessStorage::findImpl(EntityType type, const String & name) const
{
std::lock_guard lock{mutex};
const auto & entries_by_name = entries_by_name_and_type[static_cast<size_t>(type)];
const auto it = entries_by_name.find(name);
if (it == entries_by_name.end())
return {};
const Entry * entry = it->second;
return entry->id;
}
std::vector<UUID> ReplicatedAccessStorage::findAllImpl(EntityType type) const
{
std::lock_guard lock{mutex};
std::vector<UUID> result;
result.reserve(entries_by_id.size());
for (const auto & [id, entry] : entries_by_id)
if (entry.entity->isTypeOf(type))
result.emplace_back(id);
return result;
}
bool ReplicatedAccessStorage::existsImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
return entries_by_id.count(id);
}
AccessEntityPtr ReplicatedAccessStorage::readImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
const auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
throwNotFound(id);
const Entry & entry = it->second;
return entry.entity;
}
String ReplicatedAccessStorage::readNameImpl(const UUID & id) const
{
return readImpl(id)->getName();
}
void ReplicatedAccessStorage::prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const
{
const AccessEntityPtr entity = remove ? nullptr : entry.entity;
for (const auto & handler : entry.handlers_by_id)
notifications.push_back({handler, entry.id, entity});
for (const auto & handler : handlers_by_type[static_cast<size_t>(entry.entity->getType())])
notifications.push_back({handler, entry.id, entity});
}
scope_guard ReplicatedAccessStorage::subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const
{
std::lock_guard lock{mutex};
auto & handlers = handlers_by_type[static_cast<size_t>(type)];
handlers.push_back(handler);
auto handler_it = std::prev(handlers.end());
return [this, type, handler_it]
{
std::lock_guard lock2{mutex};
auto & handlers2 = handlers_by_type[static_cast<size_t>(type)];
handlers2.erase(handler_it);
};
}
scope_guard ReplicatedAccessStorage::subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const
{
std::lock_guard lock{mutex};
const auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
return {};
const Entry & entry = it->second;
auto handler_it = entry.handlers_by_id.insert(entry.handlers_by_id.end(), handler);
return [this, id, handler_it]
{
std::lock_guard lock2{mutex};
auto it2 = entries_by_id.find(id);
if (it2 != entries_by_id.end())
{
const Entry & entry2 = it2->second;
entry2.handlers_by_id.erase(handler_it);
}
};
}
bool ReplicatedAccessStorage::hasSubscriptionImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
const auto & it = entries_by_id.find(id);
if (it != entries_by_id.end())
{
const Entry & entry = it->second;
return !entry.handlers_by_id.empty();
}
return false;
}
bool ReplicatedAccessStorage::hasSubscriptionImpl(EntityType type) const
{
std::lock_guard lock{mutex};
const auto & handlers = handlers_by_type[static_cast<size_t>(type)];
return !handlers.empty();
}
}

View File

@ -0,0 +1,87 @@
#pragma once
#include <Access/IAccessStorage.h>
#include <Common/ThreadPool.h>
#include <Common/ZooKeeper/Common.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <common/scope_guard.h>
#include <Coordination/ThreadSafeQueue.h>
#include <atomic>
#include <list>
#include <memory>
#include <mutex>
#include <unordered_map>
namespace DB
{
/// Implementation of IAccessStorage which keeps all data in zookeeper.
class ReplicatedAccessStorage : public IAccessStorage
{
public:
static constexpr char STORAGE_TYPE[] = "replicated";
ReplicatedAccessStorage(const String & storage_name, const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper);
virtual ~ReplicatedAccessStorage() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
virtual void startup();
virtual void shutdown();
private:
String zookeeper_path;
zkutil::GetZooKeeper get_zookeeper;
std::atomic<bool> initialized = false;
std::atomic<bool> stop_flag = false;
ThreadFromGlobalPool worker_thread;
ThreadSafeQueue<UUID> refresh_queue;
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
void removeImpl(const UUID & id) override;
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
void insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists);
void removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id);
void updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func);
void runWorkerThread();
void resetAfterError();
void initializeZookeeper();
void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper);
void refresh();
void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper);
void refreshEntity(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id);
void refreshEntityNoLock(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, Notifications & notifications);
void setEntityNoLock(const UUID & id, const AccessEntityPtr & entity, Notifications & notifications);
void removeEntityNoLock(const UUID & id, Notifications & notifications);
struct Entry
{
UUID id;
AccessEntityPtr entity;
mutable std::list<OnChangedHandler> handlers_by_id;
};
std::optional<UUID> findImpl(EntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(EntityType type) const override;
bool existsImpl(const UUID & id) const override;
AccessEntityPtr readImpl(const UUID & id) const override;
String readNameImpl(const UUID & id) const override;
bool canInsertImpl(const AccessEntityPtr &) const override { return true; }
void prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const;
scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override;
scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override;
bool hasSubscriptionImpl(const UUID & id) const override;
bool hasSubscriptionImpl(EntityType type) const override;
mutable std::mutex mutex;
std::unordered_map<UUID, Entry> entries_by_id;
std::unordered_map<String, Entry *> entries_by_name_and_type[static_cast<size_t>(EntityType::MAX)];
mutable std::list<OnChangedHandler> handlers_by_type[static_cast<size_t>(EntityType::MAX)];
};
}

View File

@ -10,6 +10,7 @@ PEERDIR(
SRCS(
AccessControlManager.cpp
AccessEntityIO.cpp
AccessRights.cpp
AccessRightsElement.cpp
AllowedClientHosts.cpp
@ -34,6 +35,7 @@ SRCS(
Quota.cpp
QuotaCache.cpp
QuotaUsage.cpp
ReplicatedAccessStorage.cpp
Role.cpp
RoleCache.cpp
RolesOrUsersSet.cpp

View File

@ -5,6 +5,7 @@
#include <Common/SipHash.h>
#include <Common/assert_cast.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/IDataType.h>
namespace DB

View File

@ -5,6 +5,7 @@
#include <Poco/Net/HTTPRequest.h>
#include <Poco/URI.h>
#include <filesystem>
#include <thread>
namespace fs = std::filesystem;

View File

@ -373,7 +373,9 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead
except_list,
is_draining ? drain_timeout : receive_timeout);
if (n == 0)
/// We treat any error as timeout for simplicity.
/// And we also check if read_list is still empty just in case.
if (n <= 0 || read_list.empty())
{
auto err_msg = fmt::format("Timeout exceeded while reading from {}", dumpAddressesUnlocked());
for (ReplicaState & state : replica_states)
@ -389,9 +391,7 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead
}
}
/// TODO Absolutely wrong code: read_list could be empty; motivation of rand is unclear.
/// This code path is disabled by default.
/// TODO Motivation of rand is unclear.
auto & socket = read_list[thread_local_rng() % read_list.size()];
if (fd_to_replica_state_idx.empty())
{

View File

@ -565,7 +565,7 @@ void ColumnArray::expand(const IColumn::Filter & mask, bool inverted)
while (index >= 0)
{
offsets_data[index] = last_offset;
if (mask[index] ^ inverted)
if (!!mask[index] ^ inverted)
{
if (from < 0)
throw Exception("Too many bytes in mask", ErrorCodes::LOGICAL_ERROR);

View File

@ -354,7 +354,7 @@ void ColumnFixedString::expand(const IColumn::Filter & mask, bool inverted)
chars.resize_fill(mask.size() * n, 0);
while (index >= 0)
{
if (mask[index] ^ inverted)
if (!!mask[index] ^ inverted)
{
if (from < 0)
throw Exception("Too many bytes in mask", ErrorCodes::LOGICAL_ERROR);

View File

@ -178,7 +178,7 @@ void ColumnString::expand(const IColumn::Filter & mask, bool inverted)
while (index >= 0)
{
offsets_data[index] = last_offset;
if (mask[index] ^ inverted)
if (!!mask[index] ^ inverted)
{
if (from < 0)
throw Exception("Too many bytes in mask", ErrorCodes::LOGICAL_ERROR);

View File

@ -1,6 +1,6 @@
#pragma once
#include <Core/Block.h>
#include <Columns/IColumn.h>
namespace DB

View File

@ -26,7 +26,7 @@ void expandDataByMask(PaddedPODArray<T> & data, const PaddedPODArray<UInt8> & ma
data.resize(mask.size());
while (index >= 0)
{
if (mask[index] ^ inverted)
if (!!mask[index] ^ inverted)
{
if (from < 0)
throw Exception("Too many bytes in mask", ErrorCodes::LOGICAL_ERROR);

View File

@ -20,7 +20,7 @@ template <typename T>
static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
{
writeChar('\'', buf);
writeText(x.getValue(), x.getScale(), buf);
writeText(x.getValue(), x.getScale(), buf, {});
writeChar('\'', buf);
}

View File

@ -26,7 +26,7 @@ template <typename T>
static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
{
writeChar('\'', buf);
writeText(x.getValue(), x.getScale(), buf);
writeText(x.getValue(), x.getScale(), buf, {});
writeChar('\'', buf);
}

View File

@ -191,10 +191,11 @@ void ThreadPoolImpl<Thread>::wait()
template <typename Thread>
ThreadPoolImpl<Thread>::~ThreadPoolImpl()
{
/// Note: should not use logger from here,
/// because it can be an instance of GlobalThreadPool that is a global variable
/// and the destruction order of global variables is unspecified.
finalize();
/// wait() hadn't been called, log exception at least.
if (first_exception)
DB::tryLogException(first_exception, __PRETTY_FUNCTION__);
}
template <typename Thread>
@ -273,21 +274,11 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
}
catch (...)
{
ALLOW_ALLOCATIONS_IN_SCOPE;
/// job should be reset before decrementing scheduled_jobs to
/// ensure that the Job destroyed before wait() returns.
job = {};
{
/// In case thread pool will not be terminated on exception
/// (this is the case for GlobalThreadPool),
/// than first_exception may be overwritten and got lost,
/// and this usually is an error, since this will finish the thread,
/// and for this the caller may not be ready.
if (!shutdown_on_exception)
DB::tryLogException(std::current_exception(), __PRETTY_FUNCTION__);
std::unique_lock lock(mutex);
if (!first_exception)
first_exception = std::current_exception(); // NOLINT

View File

@ -122,6 +122,24 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p
return path_starts_with_prefix_path;
}
bool symlinkStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path)
{
/// Differs from pathStartsWith in how `path` is normalized before comparison.
/// Make `path` absolute if it was relative and put it into normalized form: remove
/// `.` and `..` and extra `/`. Path is not canonized because otherwise path will
/// not be a path of a symlink itself.
auto absolute_path = std::filesystem::absolute(path);
absolute_path = absolute_path.lexically_normal(); /// Normalize path.
auto absolute_prefix_path = std::filesystem::absolute(prefix_path);
absolute_prefix_path = absolute_prefix_path.lexically_normal(); /// Normalize 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);
@ -130,6 +148,13 @@ bool pathStartsWith(const String & path, const String & prefix_path)
return pathStartsWith(filesystem_path, filesystem_prefix_path);
}
bool symlinkStartsWith(const String & path, const String & prefix_path)
{
auto filesystem_path = std::filesystem::path(path);
auto filesystem_prefix_path = std::filesystem::path(prefix_path);
return symlinkStartsWith(filesystem_path, filesystem_prefix_path);
}
}

View File

@ -35,6 +35,8 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p
/// Returns true if path starts with prefix path
bool pathStartsWith(const String & path, const String & prefix_path);
bool symlinkStartsWith(const String & path, const String & prefix_path);
}
namespace FS

View File

@ -1,25 +0,0 @@
#if defined(OS_DARWIN) && defined(BUNDLED_STATIC_JEMALLOC)
extern "C"
{
extern void zone_register();
}
struct InitializeJemallocZoneAllocatorForOSX
{
InitializeJemallocZoneAllocatorForOSX()
{
/// In case of OSX jemalloc register itself as a default zone allocator.
///
/// But when you link statically then zone_register() will not be called,
/// and even will be optimized out:
///
/// It is ok to call it twice (i.e. in case of shared libraries)
/// Since zone_register() is a no-op if the default zone is already replaced with something.
///
/// https://github.com/jemalloc/jemalloc/issues/708
zone_register();
}
} initializeJemallocZoneAllocatorForOSX;
#endif

View File

@ -277,5 +277,4 @@ GTEST_TEST(WideInteger, DecimalFormatting)
Int128 fractional = DecimalUtils::getFractionalPart(x, 2);
EXPECT_EQ(fractional, 40);
EXPECT_EQ(decimalFractional(fractional, 2), "40");
}

View File

@ -439,11 +439,14 @@ bool NO_INLINE decompressImpl(
{
s = *ip++;
length += s;
} while (unlikely(s == 255));
} while (unlikely(s == 255 && ip < input_end));
};
/// Get literal length.
if (unlikely(ip >= input_end))
return false;
const unsigned token = *ip++;
length = token >> 4;
if (length == 0x0F)
@ -464,18 +467,18 @@ bool NO_INLINE decompressImpl(
/// output: xyzHello, w
/// ^-op (we will overwrite excessive bytes on next iteration)
{
auto * target = std::min(copy_end, output_end);
wildCopy<copy_amount>(op, ip, target); /// Here we can write up to copy_amount - 1 bytes after buffer.
if (unlikely(copy_end > output_end))
return false;
if (target == output_end)
wildCopy<copy_amount>(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer.
if (copy_end == output_end)
return true;
}
ip += length;
op = copy_end;
if (unlikely(ip > input_end))
if (unlikely(ip + 1 >= input_end))
return false;
/// Get match offset.
@ -528,8 +531,9 @@ bool NO_INLINE decompressImpl(
copy<copy_amount>(op, match); /// copy_amount + copy_amount - 1 - 4 * 2 bytes after buffer.
if (length > copy_amount * 2)
{
auto * target = std::min(copy_end, output_end);
wildCopy<copy_amount>(op + copy_amount, match + copy_amount, target);
if (unlikely(copy_end > output_end))
return false;
wildCopy<copy_amount>(op + copy_amount, match + copy_amount, copy_end);
}
op = copy_end;

View File

@ -248,25 +248,42 @@ struct KeeperStorageCreateRequest final : public KeeperStorageRequest
Coordination::ZooKeeperCreateResponse & response = dynamic_cast<Coordination::ZooKeeperCreateResponse &>(*response_ptr);
Coordination::ZooKeeperCreateRequest & request = dynamic_cast<Coordination::ZooKeeperCreateRequest &>(*zk_request);
if (container.contains(request.path))
{
response.error = Coordination::Error::ZNODEEXISTS;
}
else
{
auto parent_path = parentPath(request.path);
auto it = container.find(parent_path);
if (it == container.end())
{
response.error = Coordination::Error::ZNONODE;
return { response_ptr, undo };
}
else if (it->value.stat.ephemeralOwner != 0)
{
response.error = Coordination::Error::ZNOCHILDRENFOREPHEMERALS;
return { response_ptr, undo };
}
else
std::string path_created = request.path;
if (request.is_sequential)
{
auto seq_num = it->value.seq_num;
std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
seq_num_str.exceptions(std::ios::failbit);
seq_num_str << std::setw(10) << std::setfill('0') << seq_num;
path_created += seq_num_str.str();
}
if (container.contains(path_created))
{
response.error = Coordination::Error::ZNODEEXISTS;
return { response_ptr, undo };
}
auto child_path = getBaseName(path_created);
if (child_path.empty())
{
response.error = Coordination::Error::ZBADARGUMENTS;
return { response_ptr, undo };
}
auto & session_auth_ids = storage.session_and_auth[session_id];
KeeperStorage::Node created_node;
@ -293,21 +310,7 @@ struct KeeperStorageCreateRequest final : public KeeperStorageRequest
created_node.data = request.data;
created_node.is_sequental = request.is_sequential;
std::string path_created = request.path;
if (request.is_sequential)
{
auto seq_num = it->value.seq_num;
std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
seq_num_str.exceptions(std::ios::failbit);
seq_num_str << std::setw(10) << std::setfill('0') << seq_num;
path_created += seq_num_str.str();
}
int32_t parent_cversion = request.parent_cversion;
auto child_path = getBaseName(path_created);
int64_t prev_parent_zxid;
int32_t prev_parent_cversion;
container.updateValue(parent_path, [child_path, zxid, &prev_parent_zxid,
@ -356,9 +359,6 @@ struct KeeperStorageCreateRequest final : public KeeperStorageRequest
};
response.error = Coordination::Error::ZOK;
}
}
return { response_ptr, undo };
}
};

View File

@ -22,6 +22,85 @@ namespace ErrorCodes
extern const int POSITION_OUT_OF_BOUND;
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int AMBIGUOUS_COLUMN_NAME;
}
template <typename ReturnType>
static ReturnType onError(const std::string & message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (std::is_same_v<ReturnType, void>)
throw Exception(message, code);
else
return false;
};
template <typename ReturnType>
static ReturnType checkColumnStructure(const ColumnWithTypeAndName & actual, const ColumnWithTypeAndName & expected,
const std::string & context_description, bool allow_remove_constants, int code)
{
if (actual.name != expected.name)
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different names of columns:\n"
+ actual.dumpStructure() + "\n" + expected.dumpStructure(), code);
if (!actual.type->equals(*expected.type))
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different types:\n"
+ actual.dumpStructure() + "\n" + expected.dumpStructure(), code);
if (!actual.column || !expected.column)
return ReturnType(true);
const IColumn * actual_column = actual.column.get();
/// If we allow to remove constants, and expected column is not const, then unwrap actual constant column.
if (allow_remove_constants && !isColumnConst(*expected.column))
{
if (const auto * column_const = typeid_cast<const ColumnConst *>(actual_column))
actual_column = &column_const->getDataColumn();
}
if (actual_column->getName() != expected.column->getName())
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different columns:\n"
+ actual.dumpStructure() + "\n" + expected.dumpStructure(), code);
if (isColumnConst(*actual.column) && isColumnConst(*expected.column))
{
Field actual_value = assert_cast<const ColumnConst &>(*actual.column).getField();
Field expected_value = assert_cast<const ColumnConst &>(*expected.column).getField();
if (actual_value != expected_value)
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different values of constants, actual: "
+ applyVisitor(FieldVisitorToString(), actual_value) + ", expected: " + applyVisitor(FieldVisitorToString(), expected_value),
code);
}
return ReturnType(true);
}
template <typename ReturnType>
static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description, bool allow_remove_constants)
{
size_t columns = rhs.columns();
if (lhs.columns() != columns)
return onError<ReturnType>("Block structure mismatch in " + context_description + " stream: different number of columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0; i < columns; ++i)
{
const auto & actual = lhs.getByPosition(i);
const auto & expected = rhs.getByPosition(i);
if constexpr (std::is_same_v<ReturnType, bool>)
{
if (!checkColumnStructure<ReturnType>(actual, expected, context_description, allow_remove_constants, ErrorCodes::LOGICAL_ERROR))
return false;
}
else
checkColumnStructure<ReturnType>(actual, expected, context_description, allow_remove_constants, ErrorCodes::LOGICAL_ERROR);
}
return ReturnType(true);
}
@ -57,24 +136,41 @@ void Block::insert(size_t position, ColumnWithTypeAndName elem)
throw Exception("Position out of bound in Block::insert(), max position = "
+ toString(data.size()), ErrorCodes::POSITION_OUT_OF_BOUND);
if (elem.name.empty())
throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
for (auto & name_pos : index_by_name)
if (name_pos.second >= position)
++name_pos.second;
index_by_name.emplace(elem.name, position);
auto [it, inserted] = index_by_name.emplace(elem.name, position);
if (!inserted)
checkColumnStructure<void>(data[it->second], elem,
"(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME);
data.emplace(data.begin() + position, std::move(elem));
}
void Block::insert(ColumnWithTypeAndName elem)
{
index_by_name.emplace(elem.name, data.size());
if (elem.name.empty())
throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
auto [it, inserted] = index_by_name.emplace(elem.name, data.size());
if (!inserted)
checkColumnStructure<void>(data[it->second], elem,
"(columns with identical name must have identical structure)", true, ErrorCodes::AMBIGUOUS_COLUMN_NAME);
data.emplace_back(std::move(elem));
}
void Block::insertUnique(ColumnWithTypeAndName elem)
{
if (elem.name.empty())
throw Exception("Column name in Block cannot be empty", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
if (index_by_name.end() == index_by_name.find(elem.name))
insert(std::move(elem));
}
@ -487,67 +583,6 @@ DataTypes Block::getDataTypes() const
}
template <typename ReturnType>
static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description, bool allow_remove_constants)
{
auto on_error = [](const std::string & message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (std::is_same_v<ReturnType, void>)
throw Exception(message, code);
else
return false;
};
size_t columns = rhs.columns();
if (lhs.columns() != columns)
return on_error("Block structure mismatch in " + context_description + " stream: different number of columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0; i < columns; ++i)
{
const auto & expected = rhs.getByPosition(i);
const auto & actual = lhs.getByPosition(i);
if (actual.name != expected.name)
return on_error("Block structure mismatch in " + context_description + " stream: different names of columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
if (!actual.type->equals(*expected.type))
return on_error("Block structure mismatch in " + context_description + " stream: different types:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
if (!actual.column || !expected.column)
continue;
const IColumn * actual_column = actual.column.get();
/// If we allow to remove constants, and expected column is not const, then unwrap actual constant column.
if (allow_remove_constants && !isColumnConst(*expected.column))
{
if (const auto * column_const = typeid_cast<const ColumnConst *>(actual_column))
actual_column = &column_const->getDataColumn();
}
if (actual_column->getName() != expected.column->getName())
return on_error("Block structure mismatch in " + context_description + " stream: different columns:\n"
+ lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
if (isColumnConst(*actual.column) && isColumnConst(*expected.column))
{
Field actual_value = assert_cast<const ColumnConst &>(*actual.column).getField();
Field expected_value = assert_cast<const ColumnConst &>(*expected.column).getField();
if (actual_value != expected_value)
return on_error("Block structure mismatch in " + context_description + " stream: different values of constants, actual: "
+ applyVisitor(FieldVisitorToString(), actual_value) + ", expected: " + applyVisitor(FieldVisitorToString(), expected_value),
ErrorCodes::LOGICAL_ERROR);
}
}
return ReturnType(true);
}
bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs)
{
return checkBlockStructure<bool>(lhs, rhs, {}, false);

View File

@ -23,65 +23,99 @@ inline Field getBinaryValue(UInt8 type, ReadBuffer & buf)
{
switch (type)
{
case Field::Types::Null: {
return DB::Field();
case Field::Types::Null:
{
return Field();
}
case Field::Types::UInt64: {
case Field::Types::UInt64:
{
UInt64 value;
DB::readVarUInt(value, buf);
readVarUInt(value, buf);
return value;
}
case Field::Types::UInt128: {
case Field::Types::UInt128:
{
UInt128 value;
DB::readBinary(value, buf);
readBinary(value, buf);
return value;
}
case Field::Types::Int64: {
case Field::Types::UInt256:
{
UInt256 value;
readBinary(value, buf);
return value;
}
case Field::Types::UUID:
{
UUID value;
readBinary(value, buf);
return value;
}
case Field::Types::Int64:
{
Int64 value;
DB::readVarInt(value, buf);
readVarInt(value, buf);
return value;
}
case Field::Types::Float64: {
case Field::Types::Int128:
{
Int128 value;
readBinary(value, buf);
return value;
}
case Field::Types::Int256:
{
Int256 value;
readBinary(value, buf);
return value;
}
case Field::Types::Float64:
{
Float64 value;
DB::readFloatBinary(value, buf);
readFloatBinary(value, buf);
return value;
}
case Field::Types::String: {
case Field::Types::String:
{
std::string value;
DB::readStringBinary(value, buf);
readStringBinary(value, buf);
return value;
}
case Field::Types::Array: {
case Field::Types::Array:
{
Array value;
DB::readBinary(value, buf);
readBinary(value, buf);
return value;
}
case Field::Types::Tuple: {
case Field::Types::Tuple:
{
Tuple value;
DB::readBinary(value, buf);
readBinary(value, buf);
return value;
}
case Field::Types::Map: {
case Field::Types::Map:
{
Map value;
DB::readBinary(value, buf);
readBinary(value, buf);
return value;
}
case Field::Types::AggregateFunctionState: {
case Field::Types::AggregateFunctionState:
{
AggregateFunctionStateData value;
DB::readStringBinary(value.name, buf);
DB::readStringBinary(value.data, buf);
readStringBinary(value.name, buf);
readStringBinary(value.data, buf);
return value;
}
}
return DB::Field();
return Field();
}
void readBinary(Array & x, ReadBuffer & buf)
{
size_t size;
UInt8 type;
DB::readBinary(type, buf);
DB::readBinary(size, buf);
readBinary(type, buf);
readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
x.push_back(getBinaryValue(type, buf));
@ -93,8 +127,8 @@ void writeBinary(const Array & x, WriteBuffer & buf)
size_t size = x.size();
if (size)
type = x.front().getType();
DB::writeBinary(type, buf);
DB::writeBinary(size, buf);
writeBinary(type, buf);
writeBinary(size, buf);
for (const auto & elem : x)
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem);
@ -102,19 +136,19 @@ void writeBinary(const Array & x, WriteBuffer & buf)
void writeText(const Array & x, WriteBuffer & buf)
{
DB::String res = applyVisitor(FieldVisitorToString(), DB::Field(x));
String res = applyVisitor(FieldVisitorToString(), Field(x));
buf.write(res.data(), res.size());
}
void readBinary(Tuple & x, ReadBuffer & buf)
{
size_t size;
DB::readBinary(size, buf);
readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
{
UInt8 type;
DB::readBinary(type, buf);
readBinary(type, buf);
x.push_back(getBinaryValue(type, buf));
}
}
@ -122,30 +156,30 @@ void readBinary(Tuple & x, ReadBuffer & buf)
void writeBinary(const Tuple & x, WriteBuffer & buf)
{
const size_t size = x.size();
DB::writeBinary(size, buf);
writeBinary(size, buf);
for (const auto & elem : x)
{
const UInt8 type = elem.getType();
DB::writeBinary(type, buf);
writeBinary(type, buf);
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem);
}
}
void writeText(const Tuple & x, WriteBuffer & buf)
{
writeFieldText(DB::Field(x), buf);
writeFieldText(Field(x), buf);
}
void readBinary(Map & x, ReadBuffer & buf)
{
size_t size;
DB::readBinary(size, buf);
readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
{
UInt8 type;
DB::readBinary(type, buf);
readBinary(type, buf);
x.push_back(getBinaryValue(type, buf));
}
}
@ -153,19 +187,19 @@ void readBinary(Map & x, ReadBuffer & buf)
void writeBinary(const Map & x, WriteBuffer & buf)
{
const size_t size = x.size();
DB::writeBinary(size, buf);
writeBinary(size, buf);
for (const auto & elem : x)
{
const UInt8 type = elem.getType();
DB::writeBinary(type, buf);
writeBinary(type, buf);
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem);
}
}
void writeText(const Map & x, WriteBuffer & buf)
{
writeFieldText(DB::Field(x), buf);
writeFieldText(Field(x), buf);
}
template <typename T>

View File

@ -974,9 +974,9 @@ __attribute__ ((noreturn)) inline void writeText(const AggregateFunctionStateDat
}
template <typename T>
inline void writeText(const DecimalField<T> & value, WriteBuffer & buf)
inline void writeText(const DecimalField<T> & value, WriteBuffer & buf, bool trailing_zeros = false)
{
writeText(value.getValue(), value.getScale(), buf);
writeText(value.getValue(), value.getScale(), buf, trailing_zeros);
}
template <typename T>

View File

@ -2,8 +2,7 @@
#include <Core/MySQL/PacketsConnection.h>
#include <Poco/RandomStream.h>
#include <Poco/SHA1Engine.h>
#include <Access/User.h>
#include <Access/AccessControlManager.h>
#include <Interpreters/Session.h>
#include <common/logger_useful.h>
#include <Common/OpenSSLHelpers.h>
@ -73,7 +72,7 @@ Native41::Native41(const String & password, const String & auth_plugin_data)
}
void Native41::authenticate(
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
const String & user_name, Session & session, std::optional<String> auth_response,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool, const Poco::Net::SocketAddress & address)
{
if (!auth_response)
@ -86,7 +85,7 @@ void Native41::authenticate(
if (auth_response->empty())
{
context->setUser(user_name, "", address);
session.authenticate(user_name, "", address);
return;
}
@ -96,9 +95,7 @@ void Native41::authenticate(
+ " bytes, received: " + std::to_string(auth_response->size()) + " bytes.",
ErrorCodes::UNKNOWN_EXCEPTION);
auto user = context->getAccessControlManager().read<User>(user_name);
Poco::SHA1Engine::Digest double_sha1_value = user->authentication.getPasswordDoubleSHA1();
Poco::SHA1Engine::Digest double_sha1_value = session.getPasswordDoubleSHA1(user_name);
assert(double_sha1_value.size() == Poco::SHA1Engine::DIGEST_SIZE);
Poco::SHA1Engine engine;
@ -111,7 +108,7 @@ void Native41::authenticate(
{
password_sha1[i] = digest[i] ^ static_cast<unsigned char>((*auth_response)[i]);
}
context->setUser(user_name, password_sha1, address);
session.authenticate(user_name, password_sha1, address);
}
#if USE_SSL
@ -136,7 +133,7 @@ Sha256Password::Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logg
}
void Sha256Password::authenticate(
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
const String & user_name, Session & session, std::optional<String> auth_response,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address)
{
if (!auth_response)
@ -231,7 +228,7 @@ void Sha256Password::authenticate(
password.pop_back();
}
context->setUser(user_name, password, address);
session.authenticate(user_name, password, address);
}
#endif

View File

@ -15,6 +15,7 @@
namespace DB
{
class Session;
namespace MySQLProtocol
{
@ -32,7 +33,7 @@ public:
virtual String getAuthPluginData() = 0;
virtual void authenticate(
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
const String & user_name, Session & session, std::optional<String> auth_response,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) = 0;
};
@ -49,7 +50,7 @@ public:
String getAuthPluginData() override { return scramble; }
void authenticate(
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
const String & user_name, Session & session, std::optional<String> auth_response,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool /* is_secure_connection */, const Poco::Net::SocketAddress & address) override;
private:
@ -69,7 +70,7 @@ public:
String getAuthPluginData() override { return scramble; }
void authenticate(
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
const String & user_name, Session & session, std::optional<String> auth_response,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) override;
private:

View File

@ -110,7 +110,7 @@ void insertPostgreSQLValue(
readDateTime64Text(time, 6, in, assert_cast<const DataTypeDateTime64 *>(data_type.get())->getTimeZone());
if (time < 0)
time = 0;
assert_cast<ColumnDecimal<Decimal64> &>(column).insertValue(time);
assert_cast<DataTypeDateTime64::ColumnType &>(column).insertValue(time);
break;
}
case ExternalResultDescription::ValueType::vtDecimal32: [[fallthrough]];

View File

@ -1,13 +1,11 @@
#pragma once
#include <Access/AccessControlManager.h>
#include <Access/User.h>
#include <functional>
#include <Interpreters/Context.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Session.h>
#include <common/logger_useful.h>
#include <Poco/Format.h>
#include <Poco/RegularExpression.h>
@ -803,12 +801,13 @@ protected:
static void setPassword(
const String & user_name,
const String & password,
ContextMutablePtr context,
Session & session,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address)
{
try {
context->setUser(user_name, password, address);
try
{
session.authenticate(user_name, password, address);
}
catch (const Exception &)
{
@ -822,7 +821,7 @@ protected:
public:
virtual void authenticate(
const String & user_name,
ContextMutablePtr context,
Session & session,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address) = 0;
@ -836,11 +835,11 @@ class NoPasswordAuth : public AuthenticationMethod
public:
void authenticate(
const String & user_name,
ContextMutablePtr context,
Session & session,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address) override
{
setPassword(user_name, "", context, mt, address);
return setPassword(user_name, "", session, mt, address);
}
Authentication::Type getType() const override
@ -854,7 +853,7 @@ class CleartextPasswordAuth : public AuthenticationMethod
public:
void authenticate(
const String & user_name,
ContextMutablePtr context,
Session & session,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address) override
{
@ -864,7 +863,7 @@ public:
if (type == Messaging::FrontMessageType::PASSWORD_MESSAGE)
{
std::unique_ptr<Messaging::PasswordMessage> password = mt.receive<Messaging::PasswordMessage>();
setPassword(user_name, password->password, context, mt, address);
return setPassword(user_name, password->password, session, mt, address);
}
else
throw Exception(
@ -897,16 +896,15 @@ public:
void authenticate(
const String & user_name,
ContextMutablePtr context,
Session & session,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address)
{
auto user = context->getAccessControlManager().read<User>(user_name);
Authentication::Type user_auth_type = user->authentication.getType();
Authentication::Type user_auth_type = session.getAuthenticationType(user_name);
if (type_to_method.find(user_auth_type) != type_to_method.end())
{
type_to_method[user_auth_type]->authenticate(user_name, context, mt, address);
type_to_method[user_auth_type]->authenticate(user_name, session, mt, address);
mt.send(Messaging::AuthenticationOk(), true);
LOG_DEBUG(log, "Authentication for user {} was successful.", user_name);
return;

56
src/Core/ServerUUID.cpp Normal file
View File

@ -0,0 +1,56 @@
#include <Core/ServerUUID.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_CREATE_FILE;
}
void ServerUUID::load(const fs::path & server_uuid_file, Poco::Logger * log)
{
/// Write a uuid file containing a unique uuid if the file doesn't already exist during server start.
if (fs::exists(server_uuid_file))
{
try
{
UUID uuid;
ReadBufferFromFile in(server_uuid_file);
readUUIDText(uuid, in);
assertEOF(in);
server_uuid = uuid;
return;
}
catch (...)
{
/// As for now it's ok to just overwrite it, because persistency in not essential.
LOG_ERROR(log, "Cannot read server UUID from file {}: {}. Will overwrite it",
server_uuid_file.string(), getCurrentExceptionMessage(true));
}
}
try
{
UUID new_uuid = UUIDHelpers::generateV4();
auto uuid_str = toString(new_uuid);
WriteBufferFromFile out(server_uuid_file);
out.write(uuid_str.data(), uuid_str.size());
out.sync();
out.finalize();
server_uuid = new_uuid;
}
catch (...)
{
throw Exception(ErrorCodes::CANNOT_CREATE_FILE, "Caught Exception {} while writing the Server UUID file {}",
getCurrentExceptionMessage(false), server_uuid_file.string());
}
}
}

26
src/Core/ServerUUID.h Normal file
View File

@ -0,0 +1,26 @@
#pragma once
#include <Core/UUID.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace Poco
{
class Logger;
}
namespace DB
{
class ServerUUID
{
inline static UUID server_uuid = UUIDHelpers::Nil;
public:
/// Returns persistent UUID of current clickhouse-server or clickhouse-keeper instance.
static UUID get() { return server_uuid; }
/// Loads server UUID from file or creates new one. Should be called on daemon startup.
static void load(const fs::path & server_uuid_file, Poco::Logger * log);
};
}

View File

@ -561,6 +561,7 @@ class IColumn;
M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \
M(Bool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \
M(String, output_format_tsv_null_representation, "\\N", "Custom NULL representation in TSV format", 0) \
M(Bool, output_format_decimal_trailing_zeros, false, "Output trailing zeros when printing Decimal values. E.g. 1.230000 instead of 1.23.", 0) \
\
M(UInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
M(Float, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \

View File

@ -44,10 +44,10 @@ void SerializationDecimal<T>::readText(T & x, ReadBuffer & istr, UInt32 precisio
}
template <typename T>
void SerializationDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
void SerializationDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
T value = assert_cast<const ColumnType &>(column).getData()[row_num];
writeText(value, this->scale, ostr);
writeText(value, this->scale, ostr, settings.decimal_trailing_zeros);
}
template <typename T>

View File

@ -151,7 +151,7 @@ Columns CacheDictionary<dictionary_key_type>::getColumns(
* use default value.
*/
if (dictionary_key_type == DictionaryKeyType::complex)
if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -268,7 +268,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::hasKeys(const Columns & k
* Check that key was fetched during update for that key set true in result array.
*/
if (dictionary_key_type == DictionaryKeyType::complex)
if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types);
@ -364,7 +364,7 @@ ColumnPtr CacheDictionary<dictionary_key_type>::getHierarchy(
ColumnPtr key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]]) const
{
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
size_t keys_found;
auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found);
@ -382,7 +382,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::isInHierarchy(
ColumnPtr in_key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]]) const
{
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
size_t keys_found;
auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found);
@ -492,7 +492,7 @@ Pipe CacheDictionary<dictionary_key_type>::read(const Names & column_names, size
/// Write lock on storage
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names);
else
{
@ -534,7 +534,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
std::vector<UInt64> requested_keys_vector;
std::vector<size_t> requested_complex_key_rows;
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
requested_keys_vector.reserve(requested_keys.size());
else
requested_complex_key_rows.reserve(requested_keys.size());
@ -546,7 +546,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
if (key_index_to_state_from_storage[i].isExpired()
|| key_index_to_state_from_storage[i].isNotFound())
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
requested_keys_vector.emplace_back(requested_keys[i]);
else
requested_complex_key_rows.emplace_back(i);
@ -572,7 +572,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
Stopwatch watch;
QueryPipeline pipeline;
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
pipeline.init(current_source_ptr->loadIds(requested_keys_vector));
else
pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows));
@ -684,7 +684,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
}
}
template class CacheDictionary<DictionaryKeyType::simple>;
template class CacheDictionary<DictionaryKeyType::complex>;
template class CacheDictionary<DictionaryKeyType::Simple>;
template class CacheDictionary<DictionaryKeyType::Complex>;
}

View File

@ -51,8 +51,7 @@ template <DictionaryKeyType dictionary_key_type>
class CacheDictionary final : public IDictionary
{
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by cache dictionary");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
CacheDictionary(
const StorageID & dict_id_,
@ -118,7 +117,7 @@ public:
DictionaryKeyType getKeyType() const override
{
return dictionary_key_type == DictionaryKeyType::simple ? DictionaryKeyType::simple : DictionaryKeyType::complex;
return dictionary_key_type == DictionaryKeyType::Simple ? DictionaryKeyType::Simple : DictionaryKeyType::Complex;
}
ColumnPtr getColumn(
@ -141,7 +140,7 @@ public:
std::exception_ptr getLastException() const override;
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); }
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); }
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override;
@ -151,7 +150,7 @@ public:
const DataTypePtr & key_type) const override;
private:
using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>;
using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>;
static MutableColumns aggregateColumnsInOrderOfKeys(
const PaddedPODArray<KeyType> & keys,
@ -219,7 +218,7 @@ private:
};
extern template class CacheDictionary<DictionaryKeyType::simple>;
extern template class CacheDictionary<DictionaryKeyType::complex>;
extern template class CacheDictionary<DictionaryKeyType::Simple>;
extern template class CacheDictionary<DictionaryKeyType::Complex>;
}

View File

@ -41,8 +41,7 @@ class CacheDictionaryStorage final : public ICacheDictionaryStorage
static constexpr size_t max_collision_length = 10;
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryStorage");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
explicit CacheDictionaryStorage(
const DictionaryStructure & dictionary_structure,
@ -62,19 +61,19 @@ public:
String getName() const override
{
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
return "Cache";
else
return "ComplexKeyCache";
}
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; }
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; }
SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for complex key storage");
@ -82,7 +81,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -90,7 +89,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertDefaultKeysImpl(keys);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
@ -98,19 +97,19 @@ public:
PaddedPODArray<UInt64> getCachedSimpleKeys() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return getCachedKeysImpl();
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage");
}
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; }
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::Complex; }
ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & column_fetch_requests) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, column_fetch_requests);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage");
@ -118,7 +117,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage");
@ -126,7 +125,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertDefaultKeysImpl(keys);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
@ -134,7 +133,7 @@ public:
PaddedPODArray<StringRef> getCachedComplexKeys() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return getCachedKeysImpl();
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage");

View File

@ -14,8 +14,8 @@ namespace ErrorCodes
extern const int TIMEOUT_EXCEEDED;
}
template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>;
template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>;
template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
template <DictionaryKeyType dictionary_key_type>
CacheDictionaryUpdateQueue<dictionary_key_type>::CacheDictionaryUpdateQueue(
@ -155,7 +155,7 @@ void CacheDictionaryUpdateQueue<dictionary_key_type>::updateThreadFunction()
}
}
template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>;
template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>;
template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
}

View File

@ -39,7 +39,7 @@ template <DictionaryKeyType dictionary_key_type>
class CacheDictionaryUpdateUnit
{
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
/// Constructor for complex keys update request
explicit CacheDictionaryUpdateUnit(
@ -85,8 +85,8 @@ private:
template <DictionaryKeyType dictionary_key_type>
using CacheDictionaryUpdateUnitPtr = std::shared_ptr<CacheDictionaryUpdateUnit<dictionary_key_type>>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
struct CacheDictionaryUpdateQueueConfiguration
{
@ -110,7 +110,6 @@ class CacheDictionaryUpdateQueue
public:
/// Client of update queue must provide this function in constructor and perform update using update unit.
using UpdateFunction = std::function<void (CacheDictionaryUpdateUnitPtr<dictionary_key_type>)>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryUpdateQueue");
CacheDictionaryUpdateQueue(
String dictionary_name_for_logs_,
@ -167,7 +166,7 @@ private:
std::atomic<bool> finished{false};
};
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>;
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>;
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
}

View File

@ -17,7 +17,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory)
[[maybe_unused]] const Poco::Util::AbstractConfiguration & config,
[[maybe_unused]] const std::string & config_prefix,
[[maybe_unused]] Block & sample_block,
ContextPtr /* context */,
ContextPtr /* global_context */,
const std::string & /* default_database */,
bool /*created_from_ddl*/) -> DictionarySourcePtr
{

View File

@ -7,6 +7,7 @@
#include <Interpreters/ExpressionActions.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <IO/ConnectionTimeouts.h>
#include <Interpreters/Session.h>
#include <Interpreters/executeQuery.h>
#include <Common/isLocalAddress.h>
#include <common/logger_useful.h>
@ -63,19 +64,18 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
const DictionaryStructure & dict_struct_,
const Configuration & configuration_,
const Block & sample_block_,
ContextPtr context_)
ContextMutablePtr context_,
std::shared_ptr<Session> local_session_)
: update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
, configuration{configuration_}
, query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks}
, sample_block{sample_block_}
, context(Context::createCopy(context_))
, local_session(local_session_)
, context(context_)
, pool{createPool(configuration)}
, load_all_query{query_builder.composeLoadAllQuery()}
{
/// Query context is needed because some code in executeQuery function may assume it exists.
/// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock.
context->makeQueryContext();
}
ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other)
@ -85,11 +85,11 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar
, invalidate_query_response{other.invalidate_query_response}
, query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks}
, sample_block{other.sample_block}
, local_session(other.local_session)
, context(Context::createCopy(other.context))
, pool{createPool(configuration)}
, load_all_query{other.load_all_query}
{
context->makeQueryContext();
}
std::string ClickHouseDictionarySource::getUpdateFieldAndDate()
@ -222,14 +222,13 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
ContextPtr context,
ContextPtr global_context,
const std::string & default_database [[maybe_unused]],
bool /* created_from_ddl */) -> DictionarySourcePtr
{
bool secure = config.getBool(config_prefix + ".secure", false);
auto context_copy = Context::createCopy(context);
UInt16 default_port = getPortFromContext(context_copy, secure);
UInt16 default_port = getPortFromContext(global_context, secure);
std::string settings_config_prefix = config_prefix + ".clickhouse";
std::string host = config.getString(settings_config_prefix + ".host", "localhost");
@ -252,12 +251,18 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
.secure = config.getBool(settings_config_prefix + ".secure", false)
};
/// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication).
ContextMutablePtr context;
std::shared_ptr<Session> local_session;
if (configuration.is_local)
{
context_copy->setUser(configuration.user, configuration.password, Poco::Net::SocketAddress("127.0.0.1", 0));
context_copy = copyContextAndApplySettings(config_prefix, context_copy, config);
/// Start local session in case when the dictionary is loaded in-process (without TCP communication).
local_session = std::make_shared<Session>(global_context, ClientInfo::Interface::TCP);
local_session->authenticate(configuration.user, configuration.password, Poco::Net::SocketAddress{"127.0.0.1", 0});
context = local_session->makeQueryContext();
context->applySettingsChanges(readSettingsFromDictionaryConfig(config, config_prefix));
}
else
context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
String dictionary_name = config.getString(".dictionary.name", "");
String dictionary_database = config.getString(".dictionary.database", "");
@ -265,7 +270,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
if (dictionary_name == configuration.table && dictionary_database == configuration.db)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouseDictionarySource table cannot be dictionary table");
return std::make_unique<ClickHouseDictionarySource>(dict_struct, configuration, sample_block, context_copy);
return std::make_unique<ClickHouseDictionarySource>(dict_struct, configuration, sample_block, context, local_session);
};
factory.registerSource("clickhouse", create_table_source);

View File

@ -39,7 +39,8 @@ public:
const DictionaryStructure & dict_struct_,
const Configuration & configuration_,
const Block & sample_block_,
ContextPtr context);
ContextMutablePtr context_,
std::shared_ptr<Session> local_session_);
/// copy-constructor is provided in order to support cloneability
ClickHouseDictionarySource(const ClickHouseDictionarySource & other);
@ -81,6 +82,7 @@ private:
mutable std::string invalidate_query_response;
ExternalQueryBuilder query_builder;
Block sample_block;
std::shared_ptr<Session> local_session;
ContextMutablePtr context;
ConnectionPoolWithFailoverPtr pool;
const std::string load_all_query;

View File

@ -31,7 +31,7 @@ DictionaryPtr DictionaryFactory::create(
const std::string & name,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
ContextPtr context,
ContextPtr global_context,
bool created_from_ddl) const
{
Poco::Util::AbstractConfiguration::Keys keys;
@ -45,12 +45,9 @@ 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", ""), created_from_ddl);
name, config, config_prefix + ".source", dict_struct, global_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)
context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, name);
const auto & layout_type = keys.front();
{
@ -58,7 +55,7 @@ DictionaryPtr DictionaryFactory::create(
if (found != registered_layouts.end())
{
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);
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl);
}
}
@ -68,10 +65,10 @@ DictionaryPtr DictionaryFactory::create(
layout_type);
}
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr context) const
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr global_context) const
{
auto configuration = getDictionaryConfigurationFromAST(ast, context);
return DictionaryFactory::create(name, *configuration, "dictionary", context, true);
auto configuration = getDictionaryConfigurationFromAST(ast, global_context);
return DictionaryFactory::create(name, *configuration, "dictionary", global_context, true);
}
bool DictionaryFactory::isComplex(const std::string & layout_type) const

View File

@ -36,13 +36,13 @@ public:
const std::string & name,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
ContextPtr context,
ContextPtr global_context,
bool created_from_ddl) const;
/// Create dictionary from DDL-query
DictionaryPtr create(const std::string & name,
const ASTCreateQuery & ast,
ContextPtr context) const;
ContextPtr global_context) const;
using LayoutCreateFunction = std::function<DictionaryPtr(
const std::string & name,
@ -50,7 +50,7 @@ public:
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr,
ContextPtr context,
ContextPtr global_context,
bool created_from_ddl)>;
bool isComplex(const std::string & layout_type) const;

View File

@ -0,0 +1,48 @@
#include "DictionaryHelpers.h"
namespace DB
{
MutableColumns deserializeColumnsFromKeys(
const DictionaryStructure & dictionary_structure,
const PaddedPODArray<StringRef> & keys,
size_t start,
size_t end)
{
MutableColumns result_columns;
result_columns.reserve(dictionary_structure.key->size());
for (const DictionaryAttribute & attribute : *dictionary_structure.key)
result_columns.emplace_back(attribute.type->createColumn());
for (size_t index = start; index < end; ++index)
{
const auto & key = keys[index];
const auto * ptr = key.data;
for (auto & result_column : result_columns)
ptr = result_column->deserializeAndInsertFromArena(ptr);
}
return result_columns;
}
ColumnsWithTypeAndName deserializeColumnsWithTypeAndNameFromKeys(
const DictionaryStructure & dictionary_structure,
const PaddedPODArray<StringRef> & keys,
size_t start,
size_t end)
{
ColumnsWithTypeAndName result;
MutableColumns columns = deserializeColumnsFromKeys(dictionary_structure, keys, start, end);
for (size_t i = 0, num_columns = columns.size(); i < num_columns; ++i)
{
const auto & dictionary_attribute = (*dictionary_structure.key)[i];
result.emplace_back(ColumnWithTypeAndName{std::move(columns[i]), dictionary_attribute.type, dictionary_attribute.name});
}
return result;
}
}

View File

@ -380,14 +380,14 @@ template <DictionaryKeyType key_type>
class DictionaryKeysArenaHolder;
template <>
class DictionaryKeysArenaHolder<DictionaryKeyType::simple>
class DictionaryKeysArenaHolder<DictionaryKeyType::Simple>
{
public:
static Arena * getComplexKeyArena() { return nullptr; }
};
template <>
class DictionaryKeysArenaHolder<DictionaryKeyType::complex>
class DictionaryKeysArenaHolder<DictionaryKeyType::Complex>
{
public:
@ -402,8 +402,7 @@ template <DictionaryKeyType key_type>
class DictionaryKeysExtractor
{
public:
using KeyType = std::conditional_t<key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor");
using KeyType = std::conditional_t<key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_)
: key_columns(key_columns_)
@ -411,7 +410,7 @@ public:
{
assert(!key_columns.empty());
if constexpr (key_type == DictionaryKeyType::simple)
if constexpr (key_type == DictionaryKeyType::Simple)
{
key_columns[0] = key_columns[0]->convertToFullColumnIfConst();
@ -437,7 +436,7 @@ public:
{
assert(current_key_index < keys_size);
if constexpr (key_type == DictionaryKeyType::simple)
if constexpr (key_type == DictionaryKeyType::Simple)
{
const auto & column_vector = static_cast<const ColumnVector<UInt64> &>(*key_columns[0]);
const auto & data = column_vector.getData();
@ -465,7 +464,7 @@ public:
void rollbackCurrentKey() const
{
if constexpr (key_type == DictionaryKeyType::complex)
if constexpr (key_type == DictionaryKeyType::Complex)
complex_key_arena->rollback(current_complex_key.size);
}
@ -497,6 +496,20 @@ private:
Arena * complex_key_arena;
};
/// Deserialize columns from keys array using dictionary structure
MutableColumns deserializeColumnsFromKeys(
const DictionaryStructure & dictionary_structure,
const PaddedPODArray<StringRef> & keys,
size_t start,
size_t end);
/// Deserialize columns with type and name from keys array using dictionary structure
ColumnsWithTypeAndName deserializeColumnsWithTypeAndNameFromKeys(
const DictionaryStructure & dictionary_structure,
const PaddedPODArray<StringRef> & keys,
size_t start,
size_t end);
/** Merge block with blocks from stream. If there are duplicate keys in block they are filtered out.
* In result block_to_update will be merged with blocks from stream.
* Note: readPrefix readImpl readSuffix will be called on stream object during function execution.
@ -507,8 +520,7 @@ void mergeBlockWithPipe(
Block & block_to_update,
Pipe pipe)
{
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by updatePreviousyLoadedBlockWithStream");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
Columns saved_block_key_columns;
saved_block_key_columns.reserve(key_columns_size);

View File

@ -29,7 +29,7 @@ DictionarySourceData::DictionarySourceData(
, key_type(DictionaryInputStreamKeyType::ComplexKey)
{
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
fillKeyColumns(keys, 0, keys.size(), dictionary_structure, key_columns);
key_columns = deserializeColumnsWithTypeAndNameFromKeys(dictionary_structure, keys, 0, keys.size());
}
DictionarySourceData::DictionarySourceData(
@ -132,7 +132,7 @@ Block DictionarySourceData::fillBlock(
{
ColumnPtr column;
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
column = dictionary->getColumn(
attribute.name,
@ -158,32 +158,4 @@ Block DictionarySourceData::fillBlock(
return Block(block_columns);
}
void DictionarySourceData::fillKeyColumns(
const PaddedPODArray<StringRef> & keys,
size_t start,
size_t size,
const DictionaryStructure & dictionary_structure,
ColumnsWithTypeAndName & result)
{
MutableColumns columns;
columns.reserve(dictionary_structure.key->size());
for (const DictionaryAttribute & attribute : *dictionary_structure.key)
columns.emplace_back(attribute.type->createColumn());
for (size_t index = start; index < size; ++index)
{
const auto & key = keys[index];
const auto *ptr = key.data;
for (auto & column : columns)
ptr = column->deserializeAndInsertFromArena(ptr);
}
for (size_t i = 0, num_columns = columns.size(); i < num_columns; ++i)
{
const auto & dictionary_attribute = (*dictionary_structure.key)[i];
result.emplace_back(ColumnWithTypeAndName{std::move(columns[i]), dictionary_attribute.type, dictionary_attribute.name});
}
}
}

View File

@ -51,13 +51,6 @@ private:
const DataTypes & types,
ColumnsWithTypeAndName && view) const;
static void fillKeyColumns(
const PaddedPODArray<StringRef> & keys,
size_t start,
size_t size,
const DictionaryStructure & dictionary_structure,
ColumnsWithTypeAndName & result);
const size_t num_rows;
std::shared_ptr<const IDictionary> dictionary;
std::unordered_set<std::string> column_names;

Some files were not shown because too many files have changed in this diff Show More