diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml
index d69168b01ee..1df8cea4ad5 100644
--- a/.github/workflows/backport_branches.yml
+++ b/.github/workflows/backport_branches.yml
@@ -3,6 +3,9 @@ name: BackportPR
env:
# Force the stdout and stderr streams to be unbuffered
PYTHONUNBUFFERED: 1
+ # Export system tables to ClickHouse Cloud
+ CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }}
+ CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }}
on: # yamllint disable-line rule:truthy
push:
diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml
index ae1862e327f..f25611e8666 100644
--- a/.github/workflows/master.yml
+++ b/.github/workflows/master.yml
@@ -3,6 +3,9 @@ name: MasterCI
env:
# Force the stdout and stderr streams to be unbuffered
PYTHONUNBUFFERED: 1
+ # Export system tables to ClickHouse Cloud
+ CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }}
+ CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }}
on: # yamllint disable-line rule:truthy
push:
diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml
index d97b9975c3c..25927954809 100644
--- a/.github/workflows/pull_request.yml
+++ b/.github/workflows/pull_request.yml
@@ -3,6 +3,9 @@ name: PullRequestCI
env:
# Force the stdout and stderr streams to be unbuffered
PYTHONUNBUFFERED: 1
+ # Export system tables to ClickHouse Cloud
+ CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }}
+ CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }}
on: # yamllint disable-line rule:truthy
pull_request:
diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml
index 21284815583..e5527177aa5 100644
--- a/.github/workflows/release_branches.yml
+++ b/.github/workflows/release_branches.yml
@@ -3,6 +3,9 @@ name: ReleaseBranchCI
env:
# Force the stdout and stderr streams to be unbuffered
PYTHONUNBUFFERED: 1
+ # Export system tables to ClickHouse Cloud
+ CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }}
+ CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }}
on: # yamllint disable-line rule:truthy
push:
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 157f8c9fac0..55bcf5fbf3c 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -208,9 +208,6 @@ option(OMIT_HEAVY_DEBUG_SYMBOLS
"Do not generate debugger info for heavy modules (ClickHouse functions and dictionaries, some contrib)"
${OMIT_HEAVY_DEBUG_SYMBOLS_DEFAULT})
-if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")
- set(USE_DEBUG_HELPERS ON)
-endif()
option(USE_DEBUG_HELPERS "Enable debug helpers" ${USE_DEBUG_HELPERS})
option(BUILD_STANDALONE_KEEPER "Build keeper as small standalone binary" OFF)
diff --git a/base/base/JSON.cpp b/base/base/JSON.cpp
index 4c6d97b4444..0b43be38149 100644
--- a/base/base/JSON.cpp
+++ b/base/base/JSON.cpp
@@ -7,8 +7,6 @@
#include
#include
-#include
-
#define JSON_MAX_DEPTH 100
diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h
index 411841e6d9f..fc4e9e551ca 100644
--- a/base/base/wide_integer_impl.h
+++ b/base/base/wide_integer_impl.h
@@ -12,7 +12,6 @@
#include
#include
-#include
#include
// NOLINTBEGIN(*)
@@ -22,6 +21,7 @@
#define CONSTEXPR_FROM_DOUBLE constexpr
using FromDoubleIntermediateType = long double;
#else
+#include
/// `wide_integer_from_builtin` can't be constexpr with non-literal `cpp_bin_float_double_extended`
#define CONSTEXPR_FROM_DOUBLE
using FromDoubleIntermediateType = boost::multiprecision::cpp_bin_float_double_extended;
diff --git a/base/poco/Data/ODBC/src/Unicode_UNIXODBC.cpp b/base/poco/Data/ODBC/src/Unicode_UNIXODBC.cpp
index 4caf097c28a..1c5555f8cf3 100644
--- a/base/poco/Data/ODBC/src/Unicode_UNIXODBC.cpp
+++ b/base/poco/Data/ODBC/src/Unicode_UNIXODBC.cpp
@@ -19,7 +19,6 @@
#include "Poco/UTF16Encoding.h"
#include "Poco/Buffer.h"
#include "Poco/Exception.h"
-#include
using Poco::Buffer;
diff --git a/base/poco/Foundation/src/Task.cpp b/base/poco/Foundation/src/Task.cpp
index a850ae37eff..4303d50d6e8 100644
--- a/base/poco/Foundation/src/Task.cpp
+++ b/base/poco/Foundation/src/Task.cpp
@@ -16,7 +16,6 @@
#include "Poco/TaskManager.h"
#include "Poco/Exception.h"
-#include
#include
diff --git a/base/poco/JSON/src/Object.cpp b/base/poco/JSON/src/Object.cpp
index 7fca65c5b01..b041f570934 100644
--- a/base/poco/JSON/src/Object.cpp
+++ b/base/poco/JSON/src/Object.cpp
@@ -14,7 +14,6 @@
#include "Poco/JSON/Object.h"
#include
-#include
using Poco::Dynamic::Var;
diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp
index c5697b556d1..2712c0c452e 100644
--- a/base/poco/Net/src/HTTPClientSession.cpp
+++ b/base/poco/Net/src/HTTPClientSession.cpp
@@ -26,7 +26,6 @@
#include "Poco/CountingStream.h"
#include "Poco/RegularExpression.h"
#include
-#include
using Poco::NumberFormatter;
diff --git a/docker/README.md b/docker/README.md
index ec52ddd143e..bae02201add 100644
--- a/docker/README.md
+++ b/docker/README.md
@@ -1,5 +1,5 @@
## ClickHouse Dockerfiles
-This directory contain Dockerfiles for `clickhouse-client` and `clickhouse-server`. They are updated in each release.
+This directory contain Dockerfiles for `clickhouse-server`. They are updated in each release.
Also there is bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`.
diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile
deleted file mode 100644
index 1c185daec75..00000000000
--- a/docker/client/Dockerfile
+++ /dev/null
@@ -1,34 +0,0 @@
-FROM ubuntu:18.04
-
-# ARG for quick switch to a given ubuntu mirror
-ARG apt_archive="http://archive.ubuntu.com"
-RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list
-
-ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/"
-ARG version=22.1.1.*
-
-RUN apt-get update \
- && apt-get install --yes --no-install-recommends \
- apt-transport-https \
- ca-certificates \
- dirmngr \
- gnupg \
- && mkdir -p /etc/apt/sources.list.d \
- && apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 \
- && echo $repository > /etc/apt/sources.list.d/clickhouse.list \
- && apt-get update \
- && env DEBIAN_FRONTEND=noninteractive \
- apt-get install --allow-unauthenticated --yes --no-install-recommends \
- clickhouse-client=$version \
- clickhouse-common-static=$version \
- locales \
- tzdata \
- && rm -rf /var/lib/apt/lists/* /var/cache/debconf \
- && apt-get clean
-
-RUN locale-gen en_US.UTF-8
-ENV LANG en_US.UTF-8
-ENV LANGUAGE en_US:en
-ENV LC_ALL en_US.UTF-8
-
-ENTRYPOINT ["/usr/bin/clickhouse-client"]
diff --git a/docker/client/README.md b/docker/client/README.md
deleted file mode 100644
index bbcc7d60794..00000000000
--- a/docker/client/README.md
+++ /dev/null
@@ -1,7 +0,0 @@
-# ClickHouse Client Docker Image
-
-For more information see [ClickHouse Server Docker Image](https://hub.docker.com/r/clickhouse/clickhouse-server/).
-
-## License
-
-View [license information](https://github.com/ClickHouse/ClickHouse/blob/master/LICENSE) for the software contained in this image.
diff --git a/docker/packager/packager b/docker/packager/packager
index b6b9682f418..c0baacccef2 100755
--- a/docker/packager/packager
+++ b/docker/packager/packager
@@ -22,7 +22,7 @@ def check_image_exists_locally(image_name: str) -> bool:
output = subprocess.check_output(
f"docker images -q {image_name} 2> /dev/null", shell=True
)
- return output != ""
+ return output != b""
except subprocess.CalledProcessError:
return False
@@ -46,7 +46,7 @@ def build_image(image_name: str, filepath: Path) -> None:
)
-def pre_build(repo_path: Path, env_variables: List[str]):
+def pre_build(repo_path: Path, env_variables: List[str]) -> None:
if "WITH_PERFORMANCE=1" in env_variables:
current_branch = subprocess.check_output(
"git branch --show-current", shell=True, encoding="utf-8"
@@ -81,8 +81,9 @@ def run_docker_image_with_env(
env_variables: List[str],
ch_root: Path,
ccache_dir: Optional[Path],
-):
+) -> None:
output_dir.mkdir(parents=True, exist_ok=True)
+
env_part = " -e ".join(env_variables)
if env_part:
env_part = " -e " + env_part
@@ -129,9 +130,10 @@ def parse_env_variables(
version: str,
official: bool,
additional_pkgs: bool,
+ with_profiler: bool,
with_coverage: bool,
with_binaries: str,
-):
+) -> List[str]:
DARWIN_SUFFIX = "-darwin"
DARWIN_ARM_SUFFIX = "-darwin-aarch64"
ARM_SUFFIX = "-aarch64"
@@ -322,6 +324,9 @@ def parse_env_variables(
# utils are not included into clickhouse-bundle, so build everything
build_target = "all"
+ if with_profiler:
+ cmake_flags.append("-DENABLE_BUILD_PROFILING=1")
+
if with_coverage:
cmake_flags.append("-DWITH_COVERAGE=1")
@@ -416,6 +421,7 @@ def parse_args() -> argparse.Namespace:
parser.add_argument("--version")
parser.add_argument("--official", action="store_true")
parser.add_argument("--additional-pkgs", action="store_true")
+ parser.add_argument("--with-profiler", action="store_true")
parser.add_argument("--with-coverage", action="store_true")
parser.add_argument(
"--with-binaries", choices=("programs", "tests", ""), default=""
@@ -451,7 +457,7 @@ def parse_args() -> argparse.Namespace:
return args
-def main():
+def main() -> None:
logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s")
args = parse_args()
@@ -479,6 +485,7 @@ def main():
args.version,
args.official,
args.additional_pkgs,
+ args.with_profiler,
args.with_coverage,
args.with_binaries,
)
diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh
index 798d2a40b12..f949e66ab17 100755
--- a/docker/test/performance-comparison/compare.sh
+++ b/docker/test/performance-comparison/compare.sh
@@ -665,9 +665,8 @@ create view partial_query_times as select * from
-- Report for backward-incompatible ('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,
+ as select round(time_median, 3) time,
+ round(time_stddev / time_median, 3) relative_time_stddev,
test, query_index, query_display_name
from partial_query_times
join query_display_names using (test, query_index)
@@ -739,28 +738,26 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv')
;
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.
(left > right ? left / right : right / left) as times_change_float,
isFinite(times_change_float) as times_change_finite,
- toDecimal64(times_change_finite ? times_change_float : 1., 3) as times_change_decimal,
+ round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal,
times_change_finite
? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x'
: '--' as times_change_str
select
- toDecimal64(left, 3), toDecimal64(right, 3), times_change_str,
- toDecimal64(diff, 3), toDecimal64(stat_threshold, 3),
+ round(left, 3), round(right, 3), times_change_str,
+ round(diff, 3), round(stat_threshold, 3),
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')
- 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
+ round(left, 3), round(right, 3), round(diff, 3),
+ round(stat_threshold, 3), unstable_fail, test, query_index, query_display_name
from queries where unstable_show order by stat_threshold desc;
@@ -789,11 +786,10 @@ create view total_speedup as
;
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')
+ ? '-' || toString(round(times_speedup, 3)) || 'x'
+ : '+' || toString(round(1 / times_speedup, 3)) || 'x')
as times_speedup_str
select test, times_speedup_str, queries, bad, changed, unstable
-- Not sure what's the precedence of UNION ALL vs WHERE & ORDER BY, hence all
@@ -817,11 +813,10 @@ create view total_client_time_per_query as select *
'test text, query_index int, client float, server float');
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,
+ as select client, server, round(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;
+ where p > round(1.02, 3) order by p desc;
create table wall_clock_time_per_test engine Memory as select *
from file('wall-clock-times.tsv', TSV, 'test text, real float, user float, system float');
@@ -899,15 +894,14 @@ create view test_times_view_total as
;
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),
+ round(real, 3),
+ round(total_client_time, 3),
queries,
- toDecimal64(query_max, 3),
- toDecimal64(avg_real_per_query, 3),
- toDecimal64(query_min, 3),
+ round(query_max, 3),
+ round(avg_real_per_query, 3),
+ round(query_min, 3),
runs
from (
select * from test_times_view
@@ -919,21 +913,20 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv')
-- report for all queries page, only main metric
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.
(left > right ? left / right : right / left) as times_change_float,
isFinite(times_change_float) as times_change_finite,
- toDecimal64(times_change_finite ? times_change_float : 1., 3) as times_change_decimal,
+ round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal,
times_change_finite
? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x'
: '--' as times_change_str
select changed_fail, unstable_fail,
- toDecimal64(left, 3), toDecimal64(right, 3), times_change_str,
- toDecimal64(isFinite(diff) ? diff : 0, 3),
- toDecimal64(isFinite(stat_threshold) ? stat_threshold : 0, 3),
+ round(left, 3), round(right, 3), times_change_str,
+ round(isFinite(diff) ? diff : 0, 3),
+ round(isFinite(stat_threshold) ? stat_threshold : 0, 3),
test, query_index, query_display_name
from queries order by test, query_index;
@@ -1044,27 +1037,6 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes,
order by count() desc
;
-create table metric_devation engine File(TSVWithNamesAndTypes,
- 'report/metric-deviation.$version.tsv')
- settings output_format_decimal_trailing_zeros = 1
- -- first goes the key used to split the file with grep
- as select test, query_index, query_display_name,
- toDecimal64(d, 3) d, q, metric
- from (
- select
- test, query_index,
- (q[3] - q[1])/q[2] d,
- quantilesExact(0, 0.5, 1)(value) q, metric
- from (select * from unstable_run_metrics
- union all select * from unstable_run_traces
- union all select * from unstable_run_metrics_2) mm
- group by test, query_index, metric
- having isFinite(d) and d > 0.5 and q[3] > 5
- ) metrics
- left join query_display_names using (test, query_index)
- order by test, query_index, d desc
- ;
-
create table stacks engine File(TSV, 'report/stacks.$version.tsv') as
select
-- first goes the key used to split the file with grep
@@ -1173,9 +1145,8 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as
-- Show metrics that have changed
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)
+ round(diff, 3), round(times_diff, 3)
from (
select metric, median(left) as left, median(right) as right,
(right - left) / left diff,
@@ -1226,7 +1197,6 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv')
'$SHA_TO_TEST' :: LowCardinality(String) AS commit_sha,
'${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME:-Performance}' :: LowCardinality(String) AS check_name,
'$(sed -n 's/.*/\1/p' report.html)' :: LowCardinality(String) AS check_status,
- -- TODO toDateTime() can't parse output of 'date', so no time for now.
(($(date +%s) - $CHPC_CHECK_START_TIMESTAMP) * 1000) :: UInt64 AS check_duration_ms,
fromUnixTimestamp($CHPC_CHECK_START_TIMESTAMP) check_start_time,
test_name :: LowCardinality(String) AS test_name ,
diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md
index 8dd5f01cd1c..2140d22b620 100644
--- a/docs/en/operations/utilities/clickhouse-keeper-client.md
+++ b/docs/en/operations/utilities/clickhouse-keeper-client.md
@@ -45,9 +45,14 @@ keeper foo bar
- `ls [path]` -- Lists the nodes for the given path (default: cwd)
- `cd [path]` -- Change the working path (default `.`)
- `set [version]` -- Updates the node's value. Only update if version matches (default: -1)
-- `create ` -- Creates new node
+- `create [mode]` -- Creates new node with the set value
+- `touch ` -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists
- `get ` -- Returns the node's value
- `remove ` -- Remove the node
- `rmr ` -- Recursively deletes path. Confirmation required
- `flwc ` -- Executes four-letter-word command
- `help` -- Prints this message
+- `get_stat [path]` -- Returns the node's stat (default `.`)
+- `find_super_nodes [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`)
+- `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive
+- `find_big_family [path] [n]` -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10)
diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md
index 527ce2434c0..e39bdbae06e 100644
--- a/docs/en/sql-reference/functions/string-functions.md
+++ b/docs/en/sql-reference/functions/string-functions.md
@@ -729,6 +729,30 @@ Returns whether string `str` ends with `suffix`.
endsWith(str, suffix)
```
+## endsWithUTF8
+
+Returns whether string `str` ends with `suffix`, the difference between `endsWithUTF8` and `endsWith` is that `endsWithUTF8` match `str` and `suffix` by UTF-8 characters.
+
+**Syntax**
+
+```sql
+endsWithUTF8(str, suffix)
+```
+
+**Example**
+
+``` sql
+SELECT endsWithUTF8('中国', '\xbd'), endsWith('中国', '\xbd')
+```
+
+Result:
+
+```result
+┌─endsWithUTF8('中国', '½')─┬─endsWith('中国', '½')─┐
+│ 0 │ 1 │
+└──────────────────────────┴──────────────────────┘
+```
+
## startsWith
Returns whether string `str` starts with `prefix`.
@@ -745,6 +769,25 @@ startsWith(str, prefix)
SELECT startsWith('Spider-Man', 'Spi');
```
+## startsWithUTF8
+
+Returns whether string `str` starts with `prefix`, the difference between `startsWithUTF8` and `startsWith` is that `startsWithUTF8` match `str` and `suffix` by UTF-8 characters.
+
+
+**Example**
+
+``` sql
+SELECT startsWithUTF8('中国', '\xe4'), startsWith('中国', '\xe4')
+```
+
+Result:
+
+```result
+┌─startsWithUTF8('中国', '⥩─┬─startsWith('中国', '⥩─┐
+│ 0 │ 1 │
+└────────────────────────────┴────────────────────────┘
+```
+
## trim
Removes the specified characters from the start or end of a string. If not specified otherwise, the function removes whitespace (ASCII-character 32).
diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md
index c69285171ab..a6ced870c18 100644
--- a/docs/en/sql-reference/statements/create/quota.md
+++ b/docs/en/sql-reference/statements/create/quota.md
@@ -11,6 +11,7 @@ Syntax:
``` sql
CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]
+ [IN access_storage_type]
[KEYED BY {user_name | ip_address | client_key | client_key,user_name | client_key,ip_address} | NOT KEYED]
[FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year}
{MAX { {queries | query_selects | query_inserts | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number } [,...] |
diff --git a/docs/en/sql-reference/statements/create/role.md b/docs/en/sql-reference/statements/create/role.md
index 9b14e220e1f..4b6fffe4f60 100644
--- a/docs/en/sql-reference/statements/create/role.md
+++ b/docs/en/sql-reference/statements/create/role.md
@@ -11,6 +11,7 @@ Syntax:
``` sql
CREATE ROLE [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...]
+ [IN access_storage_type]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [CONST|READONLY|WRITABLE|CHANGEABLE_IN_READONLY] | PROFILE 'profile_name'] [,...]
```
diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md
index 83bb2e6bb9a..cd7718793bd 100644
--- a/docs/en/sql-reference/statements/create/row-policy.md
+++ b/docs/en/sql-reference/statements/create/row-policy.md
@@ -16,6 +16,7 @@ Syntax:
``` sql
CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name1 [ON CLUSTER cluster_name1] ON [db1.]table1|db1.*
[, policy_name2 [ON CLUSTER cluster_name2] ON [db2.]table2|db2.* ...]
+ [IN access_storage_type]
[FOR SELECT] USING condition
[AS {PERMISSIVE | RESTRICTIVE}]
[TO {role1 [, role2 ...] | ALL | ALL EXCEPT role1 [, role2 ...]}]
diff --git a/docs/en/sql-reference/statements/create/settings-profile.md b/docs/en/sql-reference/statements/create/settings-profile.md
index 8e221a4d82f..d8afce9d6de 100644
--- a/docs/en/sql-reference/statements/create/settings-profile.md
+++ b/docs/en/sql-reference/statements/create/settings-profile.md
@@ -12,6 +12,7 @@ Syntax:
``` sql
CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1]
[, name2 [ON CLUSTER cluster_name2] ...]
+ [IN access_storage_type]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [CONST|READONLY|WRITABLE|CHANGEABLE_IN_READONLY] | INHERIT 'profile_name'] [,...]
```
diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md
index d168be63c36..11d4eae7bc8 100644
--- a/docs/en/sql-reference/statements/create/user.md
+++ b/docs/en/sql-reference/statements/create/user.md
@@ -14,6 +14,7 @@ CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1]
[, name2 [ON CLUSTER cluster_name2] ...]
[NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name'}]
[HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
+ [IN access_storage_type]
[DEFAULT ROLE role [,...]]
[DEFAULT DATABASE database | NONE]
[GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]]
diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md
index c91457993c4..eed76dbcd5c 100644
--- a/docs/en/sql-reference/statements/drop.md
+++ b/docs/en/sql-reference/statements/drop.md
@@ -49,7 +49,7 @@ Deletes a user.
Syntax:
``` sql
-DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name]
+DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type]
```
## DROP ROLE
@@ -59,7 +59,7 @@ Deletes a role. The deleted role is revoked from all the entities where it was a
Syntax:
``` sql
-DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name]
+DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type]
```
## DROP ROW POLICY
@@ -69,7 +69,7 @@ Deletes a row policy. Deleted row policy is revoked from all the entities where
Syntax:
``` sql
-DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name]
+DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] [FROM access_storage_type]
```
## DROP QUOTA
@@ -79,7 +79,7 @@ Deletes a quota. The deleted quota is revoked from all the entities where it was
Syntax:
``` sql
-DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name]
+DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type]
```
## DROP SETTINGS PROFILE
@@ -89,7 +89,7 @@ Deletes a settings profile. The deleted settings profile is revoked from all the
Syntax:
``` sql
-DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name]
+DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type]
```
## DROP VIEW
diff --git a/docs/en/sql-reference/statements/move.md b/docs/en/sql-reference/statements/move.md
new file mode 100644
index 00000000000..fac738ff711
--- /dev/null
+++ b/docs/en/sql-reference/statements/move.md
@@ -0,0 +1,32 @@
+---
+slug: /en/sql-reference/statements/move
+sidebar_position: 54
+sidebar_label: MOVE
+---
+
+# MOVE access entity statement
+
+This statement allows to move an access entity from one access storage to another.
+
+Syntax:
+
+```sql
+MOVE {USER, ROLE, QUOTA, SETTINGS PROFILE, ROW POLICY} name1 [, name2, ...] TO access_storage_type
+```
+
+Currently, there are five access storages in ClickHouse:
+ - `local_directory`
+ - `memory`
+ - `replicated`
+ - `users_xml` (ro)
+ - `ldap` (ro)
+
+Examples:
+
+```sql
+MOVE USER test TO local_directory
+```
+
+```sql
+MOVE ROLE test TO memory
+```
diff --git a/docs/ru/getting-started/example-datasets/wikistat.md b/docs/ru/getting-started/example-datasets/wikistat.md
deleted file mode 100644
index 479616d667b..00000000000
--- a/docs/ru/getting-started/example-datasets/wikistat.md
+++ /dev/null
@@ -1,32 +0,0 @@
----
-slug: /ru/getting-started/example-datasets/wikistat
-sidebar_position: 17
-sidebar_label: WikiStat
----
-
-# WikiStat {#wikistat}
-
-См: http://dumps.wikimedia.org/other/pagecounts-raw/
-
-Создание таблицы:
-
-``` sql
-CREATE TABLE wikistat
-(
- date Date,
- time DateTime,
- project String,
- subproject String,
- path String,
- hits UInt64,
- size UInt64
-) ENGINE = MergeTree(date, (path, time), 8192);
-```
-
-Загрузка данных:
-
-``` bash
-$ for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt
-$ cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done
-$ ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done
-```
diff --git a/docs/ru/getting-started/example-datasets/wikistat.md b/docs/ru/getting-started/example-datasets/wikistat.md
new file mode 120000
index 00000000000..2d429d00984
--- /dev/null
+++ b/docs/ru/getting-started/example-datasets/wikistat.md
@@ -0,0 +1 @@
+../../../en/getting-started/example-datasets/wikistat.md
\ No newline at end of file
diff --git a/docs/zh/getting-started/example-datasets/wikistat.md b/docs/zh/getting-started/example-datasets/wikistat.md
deleted file mode 100644
index 4ce13b0f1d3..00000000000
--- a/docs/zh/getting-started/example-datasets/wikistat.md
+++ /dev/null
@@ -1,32 +0,0 @@
----
-slug: /zh/getting-started/example-datasets/wikistat
-sidebar_position: 17
-sidebar_label: WikiStat
----
-
-# WikiStat {#wikistat}
-
-参考: http://dumps.wikimedia.org/other/pagecounts-raw/
-
-创建表结构:
-
-``` sql
-CREATE TABLE wikistat
-(
- date Date,
- time DateTime,
- project String,
- subproject String,
- path String,
- hits UInt64,
- size UInt64
-) ENGINE = MergeTree(date, (path, time), 8192);
-```
-
-加载数据:
-
-``` bash
-$ for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt
-$ cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done
-$ ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done
-```
diff --git a/docs/zh/getting-started/example-datasets/wikistat.md b/docs/zh/getting-started/example-datasets/wikistat.md
new file mode 120000
index 00000000000..2d429d00984
--- /dev/null
+++ b/docs/zh/getting-started/example-datasets/wikistat.md
@@ -0,0 +1 @@
+../../../en/getting-started/example-datasets/wikistat.md
\ No newline at end of file
diff --git a/programs/disks/ICommand.cpp b/programs/disks/ICommand.cpp
index 52d1a2196a9..86188fb6db1 100644
--- a/programs/disks/ICommand.cpp
+++ b/programs/disks/ICommand.cpp
@@ -1,4 +1,6 @@
#include "ICommand.h"
+#include
+
namespace DB
{
diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp
index 3afd49e9855..c822a631798 100644
--- a/programs/keeper-client/Commands.cpp
+++ b/programs/keeper-client/Commands.cpp
@@ -1,5 +1,6 @@
#include "Commands.h"
+#include
#include "KeeperClient.h"
@@ -24,8 +25,18 @@ void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con
else
path = client->cwd;
- for (const auto & child : client->zookeeper->getChildren(path))
- std::cout << child << " ";
+ auto children = client->zookeeper->getChildren(path);
+ std::sort(children.begin(), children.end());
+
+ bool need_space = false;
+ for (const auto & child : children)
+ {
+ if (std::exchange(need_space, true))
+ std::cout << " ";
+
+ std::cout << child;
+ }
+
std::cout << "\n";
}
@@ -115,6 +126,21 @@ void CreateCommand::execute(const ASTKeeperQuery * query, KeeperClient * client)
static_cast(query->args[2].safeGet()));
}
+bool TouchCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const
+{
+ String arg;
+ if (!parseKeeperPath(pos, expected, arg))
+ return false;
+ node->args.push_back(std::move(arg));
+
+ return true;
+}
+
+void TouchCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
+{
+ client->zookeeper->createIfNotExists(client->getAbsolutePath(query->args[0].safeGet()), "");
+}
+
bool GetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const
{
String arg;
@@ -130,6 +156,173 @@ void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co
std::cout << client->zookeeper->get(client->getAbsolutePath(query->args[0].safeGet())) << "\n";
}
+bool GetStatCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const
+{
+ String arg;
+ if (!parseKeeperPath(pos, expected, arg))
+ return true;
+
+ node->args.push_back(std::move(arg));
+ return true;
+}
+
+void GetStatCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const
+{
+ Coordination::Stat stat;
+ String path;
+ if (!query->args.empty())
+ path = client->getAbsolutePath(query->args[0].safeGet());
+ else
+ path = client->cwd;
+
+ client->zookeeper->get(path, &stat);
+
+ std::cout << "cZxid = " << stat.czxid << "\n";
+ std::cout << "mZxid = " << stat.mzxid << "\n";
+ std::cout << "pZxid = " << stat.pzxid << "\n";
+ std::cout << "ctime = " << stat.ctime << "\n";
+ std::cout << "mtime = " << stat.mtime << "\n";
+ std::cout << "version = " << stat.version << "\n";
+ std::cout << "cversion = " << stat.cversion << "\n";
+ std::cout << "aversion = " << stat.aversion << "\n";
+ std::cout << "ephemeralOwner = " << stat.ephemeralOwner << "\n";
+ std::cout << "dataLength = " << stat.dataLength << "\n";
+ std::cout << "numChildren = " << stat.numChildren << "\n";
+}
+
+bool FindSuperNodes::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const
+{
+ ASTPtr threshold;
+ if (!ParserUnsignedInteger{}.parse(pos, threshold, expected))
+ return false;
+
+ node->args.push_back(threshold->as().value);
+
+ String path;
+ if (!parseKeeperPath(pos, expected, path))
+ path = ".";
+
+ node->args.push_back(std::move(path));
+ return true;
+}
+
+void FindSuperNodes::execute(const ASTKeeperQuery * query, KeeperClient * client) const
+{
+ auto threshold = query->args[0].safeGet();
+ auto path = client->getAbsolutePath(query->args[1].safeGet());
+
+ Coordination::Stat stat;
+ client->zookeeper->get(path, &stat);
+
+ if (stat.numChildren >= static_cast(threshold))
+ {
+ std::cout << static_cast(path) << "\t" << stat.numChildren << "\n";
+ return;
+ }
+
+ auto children = client->zookeeper->getChildren(path);
+ std::sort(children.begin(), children.end());
+ for (const auto & child : children)
+ {
+ auto next_query = *query;
+ next_query.args[1] = DB::Field(path / child);
+ execute(&next_query, client);
+ }
+}
+
+bool DeleteStaleBackups::parse(IParser::Pos & /* pos */, std::shared_ptr & /* node */, Expected & /* expected */) const
+{
+ return true;
+}
+
+void DeleteStaleBackups::execute(const ASTKeeperQuery * /* query */, KeeperClient * client) const
+{
+ client->askConfirmation(
+ "You are going to delete all inactive backups in /clickhouse/backups.",
+ [client]
+ {
+ fs::path backup_root = "/clickhouse/backups";
+ auto backups = client->zookeeper->getChildren(backup_root);
+ std::sort(backups.begin(), backups.end());
+
+ for (const auto & child : backups)
+ {
+ auto backup_path = backup_root / child;
+ std::cout << "Found backup " << backup_path << ", checking if it's active\n";
+
+ String stage_path = backup_path / "stage";
+ auto stages = client->zookeeper->getChildren(stage_path);
+
+ bool is_active = false;
+ for (const auto & stage : stages)
+ {
+ if (startsWith(stage, "alive"))
+ {
+ is_active = true;
+ break;
+ }
+ }
+
+ if (is_active)
+ {
+ std::cout << "Backup " << backup_path << " is active, not going to delete\n";
+ continue;
+ }
+
+ std::cout << "Backup " << backup_path << " is not active, deleting it\n";
+ client->zookeeper->removeRecursive(backup_path);
+ }
+ });
+}
+
+bool FindBigFamily::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const
+{
+ String path;
+ if (!parseKeeperPath(pos, expected, path))
+ path = ".";
+
+ node->args.push_back(std::move(path));
+
+ ASTPtr count;
+ if (ParserUnsignedInteger{}.parse(pos, count, expected))
+ node->args.push_back(count->as().value);
+ else
+ node->args.push_back(UInt64(10));
+
+ return true;
+}
+
+void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client) const
+{
+ auto path = client->getAbsolutePath(query->args[0].safeGet());
+ auto n = query->args[1].safeGet();
+
+ std::vector> result;
+
+ std::queue queue;
+ queue.push(path);
+ while (!queue.empty())
+ {
+ auto next_path = queue.front();
+ queue.pop();
+
+ auto children = client->zookeeper->getChildren(next_path);
+ std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; });
+
+ auto response = client->zookeeper->get(children);
+
+ for (size_t i = 0; i < response.size(); ++i)
+ {
+ result.emplace_back(response[i].stat.numChildren, children[i]);
+ queue.push(children[i]);
+ }
+ }
+
+ std::sort(result.begin(), result.end(), std::greater());
+ for (UInt64 i = 0; i < std::min(result.size(), static_cast(n)); ++i)
+ std::cout << std::get<1>(result[i]) << "\t" << std::get<0>(result[i]) << "\n";
+}
+
bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const
{
String arg;
@@ -170,7 +363,7 @@ bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptrgetHelpMessage() << "\n";
+ std::cout << pair.second->generateHelpString() << "\n";
}
bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const
diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h
index e4debd53e42..7d64fef1738 100644
--- a/programs/keeper-client/Commands.h
+++ b/programs/keeper-client/Commands.h
@@ -21,6 +21,12 @@ public:
virtual String getName() const = 0;
virtual ~IKeeperClientCommand() = default;
+
+ String generateHelpString() const
+ {
+ return fmt::vformat(getHelpMessage(), fmt::make_format_args(getName()));
+ }
+
};
using Command = std::shared_ptr;
@@ -34,7 +40,7 @@ class LSCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
- String getHelpMessage() const override { return "ls [path] -- Lists the nodes for the given path (default: cwd)"; }
+ String getHelpMessage() const override { return "{} [path] -- Lists the nodes for the given path (default: cwd)"; }
};
class CDCommand : public IKeeperClientCommand
@@ -45,7 +51,7 @@ class CDCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
- String getHelpMessage() const override { return "cd [path] -- Change the working path (default `.`)"; }
+ String getHelpMessage() const override { return "{} [path] -- Change the working path (default `.`)"; }
};
class SetCommand : public IKeeperClientCommand
@@ -58,7 +64,7 @@ class SetCommand : public IKeeperClientCommand
String getHelpMessage() const override
{
- return "set [version] -- Updates the node's value. Only update if version matches (default: -1)";
+ return "{} [version] -- Updates the node's value. Only update if version matches (default: -1)";
}
};
@@ -70,7 +76,18 @@ class CreateCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
- String getHelpMessage() const override { return "create -- Creates new node"; }
+ String getHelpMessage() const override { return "{} [mode] -- Creates new node with the set value"; }
+};
+
+class TouchCommand : public IKeeperClientCommand
+{
+ String getName() const override { return "touch"; }
+
+ bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override;
+
+ void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
+
+ String getHelpMessage() const override { return "{} -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists"; }
};
class GetCommand : public IKeeperClientCommand
@@ -81,9 +98,63 @@ class GetCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
- String getHelpMessage() const override { return "get -- Returns the node's value"; }
+ String getHelpMessage() const override { return "{} -- Returns the node's value"; }
};
+class GetStatCommand : public IKeeperClientCommand
+{
+ String getName() const override { return "get_stat"; }
+
+ bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override;
+
+ void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
+
+ String getHelpMessage() const override { return "{} [path] -- Returns the node's stat (default `.`)"; }
+};
+
+class FindSuperNodes : public IKeeperClientCommand
+{
+ String getName() const override { return "find_super_nodes"; }
+
+ bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override;
+
+ void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
+
+ String getHelpMessage() const override
+ {
+ return "{} [path] -- Finds nodes with number of children larger than some threshold for the given path (default `.`)";
+ }
+};
+
+class DeleteStaleBackups : public IKeeperClientCommand
+{
+ String getName() const override { return "delete_stale_backups"; }
+
+ bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override;
+
+ void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
+
+ String getHelpMessage() const override
+ {
+ return "{} -- Deletes ClickHouse nodes used for backups that are now inactive";
+ }
+};
+
+class FindBigFamily : public IKeeperClientCommand
+{
+ String getName() const override { return "find_big_family"; }
+
+ bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override;
+
+ void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
+
+ String getHelpMessage() const override
+ {
+ return "{} [path] [n] -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10)";
+ }
+};
+
+
class RMCommand : public IKeeperClientCommand
{
String getName() const override { return "rm"; }
@@ -92,7 +163,7 @@ class RMCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
- String getHelpMessage() const override { return "remove -- Remove the node"; }
+ String getHelpMessage() const override { return "{} -- Remove the node"; }
};
class RMRCommand : public IKeeperClientCommand
@@ -103,7 +174,7 @@ class RMRCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
- String getHelpMessage() const override { return "rmr -- Recursively deletes path. Confirmation required"; }
+ String getHelpMessage() const override { return "{} -- Recursively deletes path. Confirmation required"; }
};
class HelpCommand : public IKeeperClientCommand
@@ -114,7 +185,7 @@ class HelpCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
- String getHelpMessage() const override { return "help -- Prints this message"; }
+ String getHelpMessage() const override { return "{} -- Prints this message"; }
};
class FourLetterWordCommand : public IKeeperClientCommand
@@ -125,7 +196,7 @@ class FourLetterWordCommand : public IKeeperClientCommand
void execute(const ASTKeeperQuery * query, KeeperClient * client) const override;
- String getHelpMessage() const override { return "flwc -- Executes four-letter-word command"; }
+ String getHelpMessage() const override { return "{} -- Executes four-letter-word command"; }
};
}
diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp
index 4123a86515c..893be4d25d1 100644
--- a/programs/keeper-client/KeeperClient.cpp
+++ b/programs/keeper-client/KeeperClient.cpp
@@ -176,7 +176,12 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */)
std::make_shared(),
std::make_shared(),
std::make_shared(),
+ std::make_shared(),
std::make_shared(),
+ std::make_shared(),
+ std::make_shared(),
+ std::make_shared(),
+ std::make_shared(),
std::make_shared(),
std::make_shared(),
std::make_shared(),
diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp
index 05cba7f8510..b8bc3c337e3 100644
--- a/src/Access/AccessControl.cpp
+++ b/src/Access/AccessControl.cpp
@@ -418,7 +418,7 @@ void AccessControl::addStoragesFromUserDirectoriesConfig(
String type = key_in_user_directories;
if (size_t bracket_pos = type.find('['); bracket_pos != String::npos)
type.resize(bracket_pos);
- if ((type == "users_xml") || (type == "users_config"))
+ if ((type == "users.xml") || (type == "users_config"))
type = UsersConfigAccessStorage::STORAGE_TYPE;
else if ((type == "local") || (type == "local_directory"))
type = DiskAccessStorage::STORAGE_TYPE;
@@ -528,12 +528,14 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co
return changes_notifier->subscribeForChanges(ids, handler);
}
-std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
+bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
{
- auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists);
- if (id)
+ if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists))
+ {
changes_notifier->sendNotifications();
- return id;
+ return true;
+ }
+ return false;
}
bool AccessControl::removeImpl(const UUID & id, bool throw_if_not_exists)
diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h
index c7b94955a47..74bcc119184 100644
--- a/src/Access/AccessControl.h
+++ b/src/Access/AccessControl.h
@@ -232,7 +232,7 @@ private:
class CustomSettingsPrefixes;
class PasswordComplexityRules;
- std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
+ bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp
index 938881fafff..190c7567b85 100644
--- a/src/Access/DiskAccessStorage.cpp
+++ b/src/Access/DiskAccessStorage.cpp
@@ -498,20 +498,10 @@ std::optional> DiskAccessStorage::readNameWi
}
-std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
-{
- UUID id = generateRandomID();
- if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true))
- return id;
-
- return std::nullopt;
-}
-
-
-bool DiskAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk)
+bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
{
std::lock_guard lock{mutex};
- return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, write_on_disk);
+ return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk = */ true);
}
@@ -745,7 +735,7 @@ void DiskAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists]
{
for (const auto & [id, entity] : my_entities)
- insertWithID(id, entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true);
+ insert(id, entity, replace_if_exists, throw_if_exists);
});
}
diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h
index 069a966c8e9..5d94008b34f 100644
--- a/src/Access/DiskAccessStorage.h
+++ b/src/Access/DiskAccessStorage.h
@@ -13,7 +13,7 @@ class AccessChangesNotifier;
class DiskAccessStorage : public IAccessStorage
{
public:
- static constexpr char STORAGE_TYPE[] = "local directory";
+ static constexpr char STORAGE_TYPE[] = "local_directory";
DiskAccessStorage(const String & storage_name_, const String & directory_path_, AccessChangesNotifier & changes_notifier_, bool readonly_, bool allow_backup_);
~DiskAccessStorage() override;
@@ -39,7 +39,7 @@ private:
std::vector findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override;
- std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
+ bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
@@ -53,7 +53,6 @@ private:
void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS;
void stopListsWritingThread();
- bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk);
bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) TSA_REQUIRES(mutex);
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);
bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);
diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp
index 9468e8d220a..ec25fd505f0 100644
--- a/src/Access/IAccessStorage.cpp
+++ b/src/Access/IAccessStorage.cpp
@@ -93,6 +93,17 @@ String IAccessStorage::readName(const UUID & id) const
}
+bool IAccessStorage::exists(const std::vector & ids) const
+{
+ for (const auto & id : ids)
+ {
+ if (!exists(id))
+ return false;
+ }
+
+ return true;
+}
+
std::optional IAccessStorage::readName(const UUID & id, bool throw_if_not_exists) const
{
if (auto name_and_type = readNameWithType(id, throw_if_not_exists))
@@ -167,38 +178,69 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity)
return *insert(entity, /* replace_if_exists = */ false, /* throw_if_exists = */ true);
}
-
std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
{
- return insertImpl(entity, replace_if_exists, throw_if_exists);
+ auto id = generateRandomID();
+
+ if (insert(id, entity, replace_if_exists, throw_if_exists))
+ return id;
+
+ return std::nullopt;
+}
+
+
+bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
+{
+ return insertImpl(id, entity, replace_if_exists, throw_if_exists);
}
std::vector IAccessStorage::insert(const std::vector & multiple_entities, bool replace_if_exists, bool throw_if_exists)
{
+ return insert(multiple_entities, /* ids = */ {}, replace_if_exists, throw_if_exists);
+}
+
+std::vector IAccessStorage::insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists, bool throw_if_exists)
+{
+ assert(ids.empty() || (multiple_entities.size() == ids.size()));
+
if (multiple_entities.empty())
return {};
if (multiple_entities.size() == 1)
{
- if (auto id = insert(multiple_entities[0], replace_if_exists, throw_if_exists))
- return {*id};
+ UUID id;
+ if (!ids.empty())
+ id = ids[0];
+ else
+ id = generateRandomID();
+
+ if (insert(id, multiple_entities[0], replace_if_exists, throw_if_exists))
+ return {id};
return {};
}
std::vector successfully_inserted;
try
{
- std::vector ids;
- for (const auto & entity : multiple_entities)
+ std::vector new_ids;
+ for (size_t i = 0; i < multiple_entities.size(); ++i)
{
- if (auto id = insertImpl(entity, replace_if_exists, throw_if_exists))
+ const auto & entity = multiple_entities[i];
+
+ UUID id;
+ if (!ids.empty())
+ id = ids[i];
+ else
+ id = generateRandomID();
+
+ if (insert(id, entity, replace_if_exists, throw_if_exists))
{
successfully_inserted.push_back(entity);
- ids.push_back(*id);
+ new_ids.push_back(id);
}
}
- return ids;
+ return new_ids;
}
catch (Exception & e)
{
@@ -244,7 +286,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vector IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool)
+bool IAccessStorage::insertImpl(const UUID &, const AccessEntityPtr & entity, bool, bool)
{
if (isReadOnly())
throwReadonlyCannotInsert(entity->getType(), entity->getName());
diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h
index aa3947201e7..b82537df639 100644
--- a/src/Access/IAccessStorage.h
+++ b/src/Access/IAccessStorage.h
@@ -3,6 +3,8 @@
#include
#include
#include
+#include
+#include
#include
#include
#include
@@ -92,6 +94,7 @@ public:
/// Returns whether there is an entity with such identifier in the storage.
virtual bool exists(const UUID & id) const = 0;
+ bool exists(const std::vector & ids) const;
/// Reads an entity. Throws an exception if not found.
template
@@ -100,6 +103,9 @@ public:
template
std::shared_ptr read(const String & name, bool throw_if_not_exists = true) const;
+ template
+ std::vector read(const std::vector & ids, bool throw_if_not_exists = true) const;
+
/// Reads an entity. Returns nullptr if not found.
template
std::shared_ptr tryRead(const UUID & id) const;
@@ -128,7 +134,9 @@ public:
/// Throws an exception if the specified name already exists.
UUID insert(const AccessEntityPtr & entity);
std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
+ bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true);
+ std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true);
/// Inserts an entity to the storage. Returns ID of a new entry in the storage.
std::optional tryInsert(const AccessEntityPtr & entity);
@@ -179,7 +187,7 @@ protected:
virtual std::vector findAllImpl(AccessEntityType type) const = 0;
virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0;
virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const;
- virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
+ virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
virtual bool removeImpl(const UUID & id, bool throw_if_not_exists);
virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const;
@@ -240,6 +248,19 @@ std::shared_ptr IAccessStorage::read(const String & name, bo
}
+template
+std::vector IAccessStorage::read(const std::vector & ids, bool throw_if_not_exists) const
+{
+ std::vector result;
+ result.reserve(ids.size());
+
+ for (const auto & id : ids)
+ result.push_back(read(id, throw_if_not_exists));
+
+ return result;
+}
+
+
template
std::shared_ptr IAccessStorage::tryRead(const UUID & id) const
{
@@ -265,4 +286,9 @@ std::vector>> IAccessStorage
return entities;
}
+inline bool parseAccessStorageName(IParser::Pos & pos, Expected & expected, String & storage_name)
+{
+ return parseIdentifierOrStringLiteral(pos, expected, storage_name);
+}
+
}
diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp
index c4192ee4552..999e2c41d0b 100644
--- a/src/Access/MemoryAccessStorage.cpp
+++ b/src/Access/MemoryAccessStorage.cpp
@@ -63,17 +63,7 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not
}
-std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
-{
- UUID id = generateRandomID();
- if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists))
- return id;
-
- return std::nullopt;
-}
-
-
-bool MemoryAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
+bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
{
std::lock_guard lock{mutex};
return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists);
@@ -300,7 +290,7 @@ void MemoryAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists]
{
for (const auto & [id, entity] : my_entities)
- insertWithID(id, entity, replace_if_exists, throw_if_exists);
+ insert(id, entity, replace_if_exists, throw_if_exists);
});
}
diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h
index b63132147da..9d3fdb0a388 100644
--- a/src/Access/MemoryAccessStorage.h
+++ b/src/Access/MemoryAccessStorage.h
@@ -6,6 +6,7 @@
#include
#include
#include
+#include
namespace DB
@@ -22,11 +23,6 @@ public:
const char * getStorageType() const override { return STORAGE_TYPE; }
- /// Inserts an entity with a specified ID.
- /// If `replace_if_exists == true` it can replace an existing entry with such ID and also remove an existing entry
- /// with such name & type.
- bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists);
-
/// Removes all entities except the specified list `ids_to_keep`.
/// The function skips IDs not contained in the storage.
void removeAllExcept(const std::vector & ids_to_keep);
@@ -44,7 +40,7 @@ private:
std::optional findImpl(AccessEntityType type, const String & name) const override;
std::vector findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
- std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
+ bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp
index f36a670ddbf..0550c140c17 100644
--- a/src/Access/MultipleAccessStorage.cpp
+++ b/src/Access/MultipleAccessStorage.cpp
@@ -16,6 +16,7 @@ namespace ErrorCodes
{
extern const int ACCESS_ENTITY_ALREADY_EXISTS;
extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND;
+ extern const int ACCESS_ENTITY_NOT_FOUND;
}
using Storage = IAccessStorage;
@@ -178,6 +179,91 @@ ConstStoragePtr MultipleAccessStorage::getStorage(const UUID & id) const
return const_cast(this)->getStorage(id);
}
+StoragePtr MultipleAccessStorage::findStorageByName(const DB::String & storage_name)
+{
+ auto storages = getStoragesInternal();
+ for (const auto & storage : *storages)
+ {
+ if (storage->getStorageName() == storage_name)
+ return storage;
+ }
+
+ return nullptr;
+}
+
+
+ConstStoragePtr MultipleAccessStorage::findStorageByName(const DB::String & storage_name) const
+{
+ return const_cast(this)->findStorageByName(storage_name);
+}
+
+
+StoragePtr MultipleAccessStorage::getStorageByName(const DB::String & storage_name)
+{
+ auto storage = findStorageByName(storage_name);
+ if (storage)
+ return storage;
+
+ throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "Access storage with name {} is not found", storage_name);
+}
+
+
+ConstStoragePtr MultipleAccessStorage::getStorageByName(const DB::String & storage_name) const
+{
+ return const_cast(this)->getStorageByName(storage_name);
+}
+
+StoragePtr MultipleAccessStorage::findExcludingStorage(AccessEntityType type, const DB::String & name, DB::MultipleAccessStorage::StoragePtr exclude) const
+{
+ auto storages = getStoragesInternal();
+ for (const auto & storage : *storages)
+ {
+ if (storage == exclude)
+ continue;
+
+ if (storage->find(type, name))
+ return storage;
+ }
+
+ return nullptr;
+}
+
+void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name)
+{
+ auto source_storage = getStorageByName(source_storage_name);
+ auto destination_storage = getStorageByName(destination_storage_name);
+
+ auto to_move = source_storage->read(ids);
+ bool need_rollback = false;
+
+ try
+ {
+ source_storage->remove(ids);
+ need_rollback = true;
+ destination_storage->insert(to_move, ids);
+ }
+ catch (Exception & e)
+ {
+ String message;
+
+ bool need_comma = false;
+ for (const auto & entity : to_move)
+ {
+ if (std::exchange(need_comma, true))
+ message += ", ";
+
+ message += entity->formatTypeWithName();
+ }
+
+ e.addMessage("while moving {} from {} to {}", message, source_storage_name, destination_storage_name);
+
+ if (need_rollback)
+ source_storage->insert(to_move, ids);
+
+ throw;
+ }
+}
+
AccessEntityPtr MultipleAccessStorage::readImpl(const UUID & id, bool throw_if_not_exists) const
{
if (auto storage = findStorage(id))
@@ -245,7 +331,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode)
}
-std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
+bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
{
std::shared_ptr storage_for_insertion;
@@ -268,13 +354,14 @@ std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & en
getStorageName());
}
- auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists);
- if (id)
+ if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists))
{
std::lock_guard lock{mutex};
- ids_cache.set(*id, storage_for_insertion);
+ ids_cache.set(id, storage_for_insertion);
+ return true;
}
- return id;
+
+ return false;
}
diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h
index 6a0c1bdfc02..069d414f601 100644
--- a/src/Access/MultipleAccessStorage.h
+++ b/src/Access/MultipleAccessStorage.h
@@ -41,6 +41,16 @@ public:
ConstStoragePtr getStorage(const UUID & id) const;
StoragePtr getStorage(const UUID & id);
+ ConstStoragePtr findStorageByName(const String & storage_name) const;
+ StoragePtr findStorageByName(const String & storage_name);
+ ConstStoragePtr getStorageByName(const String & storage_name) const;
+ StoragePtr getStorageByName(const String & storage_name);
+
+ /// Search for an access entity storage, excluding one. Returns nullptr if not found.
+ StoragePtr findExcludingStorage(AccessEntityType type, const String & name, StoragePtr exclude) const;
+
+ void moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name);
+
bool exists(const UUID & id) const override;
bool isBackupAllowed() const override;
@@ -53,7 +63,7 @@ protected:
std::vector findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override;
- std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
+ bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override;
@@ -65,6 +75,8 @@ private:
std::shared_ptr nested_storages TSA_GUARDED_BY(mutex);
mutable CacheBase ids_cache TSA_GUARDED_BY(mutex);
mutable std::mutex mutex;
+
+ mutable std::mutex move_mutex;
};
}
diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp
index 56d68be9268..275672da115 100644
--- a/src/Access/ReplicatedAccessStorage.cpp
+++ b/src/Access/ReplicatedAccessStorage.cpp
@@ -108,17 +108,7 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function)
}
}
-std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
-{
- const UUID id = generateRandomID();
- if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists))
- return id;
-
- return std::nullopt;
-}
-
-
-bool ReplicatedAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
+bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
{
const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType());
const String & name = new_entity->getName();
@@ -619,7 +609,7 @@ AccessEntityPtr ReplicatedAccessStorage::tryReadEntityFromZooKeeper(const zkutil
void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntityPtr & entity)
{
LOG_DEBUG(getLogger(), "Setting id {} to entity named {}", toString(id), entity->getName());
- memory_storage.insertWithID(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false);
+ memory_storage.insert(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false);
}
@@ -711,7 +701,7 @@ void ReplicatedAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists]
{
for (const auto & [id, entity] : my_entities)
- insertWithID(id, entity, replace_if_exists, throw_if_exists);
+ insert(id, entity, replace_if_exists, throw_if_exists);
});
}
diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h
index 555d58e6b04..cddb20860f7 100644
--- a/src/Access/ReplicatedAccessStorage.h
+++ b/src/Access/ReplicatedAccessStorage.h
@@ -46,11 +46,10 @@ private:
std::unique_ptr watching_thread;
std::shared_ptr> watched_queue;
- std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
+ bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) override;
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
- bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists);
bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
bool removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, bool throw_if_not_exists);
bool updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
diff --git a/src/Access/UsersConfigAccessStorage.h b/src/Access/UsersConfigAccessStorage.h
index b533ccbf200..d8ddf07ff40 100644
--- a/src/Access/UsersConfigAccessStorage.h
+++ b/src/Access/UsersConfigAccessStorage.h
@@ -20,7 +20,7 @@ class UsersConfigAccessStorage : public IAccessStorage
{
public:
- static constexpr char STORAGE_TYPE[] = "users.xml";
+ static constexpr char STORAGE_TYPE[] = "users_xml";
UsersConfigAccessStorage(const String & storage_name_, AccessControl & access_control_, bool allow_backup_);
~UsersConfigAccessStorage() override;
diff --git a/src/AggregateFunctions/tests/gtest_ranks.cpp b/src/AggregateFunctions/tests/gtest_ranks.cpp
index b29271cbec7..99c7fbd26d7 100644
--- a/src/AggregateFunctions/tests/gtest_ranks.cpp
+++ b/src/AggregateFunctions/tests/gtest_ranks.cpp
@@ -2,7 +2,6 @@
#include
#include
#include
-#include
#include
diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp
index 2cb7afa4ad6..7b27bd70ce3 100644
--- a/src/Analyzer/Passes/FuseFunctionsPass.cpp
+++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp
@@ -14,6 +14,9 @@
#include
#include
+#include
+
+
namespace DB
{
diff --git a/src/Analyzer/examples/query_analyzer.cpp b/src/Analyzer/examples/query_analyzer.cpp
index 5a20b46b346..869113b10f8 100644
--- a/src/Analyzer/examples/query_analyzer.cpp
+++ b/src/Analyzer/examples/query_analyzer.cpp
@@ -1,4 +1,3 @@
-#include
int main(int argc, char ** argv)
{
diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp
index 8c7a4e830fe..1f90e757b73 100644
--- a/src/Client/ConnectionParameters.cpp
+++ b/src/Client/ConnectionParameters.cpp
@@ -1,6 +1,5 @@
#include "ConnectionParameters.h"
#include
-#include
#include
#include
#include
diff --git a/src/Client/ConnectionString.cpp b/src/Client/ConnectionString.cpp
index 8150ae98c8b..0893ef1af00 100644
--- a/src/Client/ConnectionString.cpp
+++ b/src/Client/ConnectionString.cpp
@@ -6,7 +6,6 @@
#include
#include
-#include
#include
#include
diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp
index 07872774559..2fb6d2c3028 100644
--- a/src/Columns/ColumnObject.cpp
+++ b/src/Columns/ColumnObject.cpp
@@ -2,17 +2,17 @@
#include
#include
#include
-#include
#include
#include
#include
#include
#include
-#include
#include
#include
#include
#include
+#include
+
namespace DB
{
diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp
index 6062ea51941..c3450ff91b4 100644
--- a/src/Columns/tests/gtest_column_sparse.cpp
+++ b/src/Columns/tests/gtest_column_sparse.cpp
@@ -10,6 +10,7 @@
#include
+
using namespace DB;
static pcg64 rng(randomSeed());
diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp
index 44d338c07af..9f137124be0 100644
--- a/src/Common/Config/configReadClient.cpp
+++ b/src/Common/Config/configReadClient.cpp
@@ -3,7 +3,6 @@
#include
#include "ConfigProcessor.h"
#include
-#include
#include
namespace fs = std::filesystem;
diff --git a/src/Common/CounterInFile.h b/src/Common/CounterInFile.h
index d52f4a10050..fe3b74173f6 100644
--- a/src/Common/CounterInFile.h
+++ b/src/Common/CounterInFile.h
@@ -4,7 +4,6 @@
#include
#include
-#include
#include
#include
diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp
index d5e04238ef9..4c21d9c9783 100644
--- a/src/Common/DateLUTImpl.cpp
+++ b/src/Common/DateLUTImpl.cpp
@@ -10,7 +10,6 @@
#include
#include
#include
-#include
/// Embedded timezones.
diff --git a/src/Common/EventNotifier.h b/src/Common/EventNotifier.h
index edf7622512d..1d34f7ad9b1 100644
--- a/src/Common/EventNotifier.h
+++ b/src/Common/EventNotifier.h
@@ -7,11 +7,11 @@
#include