Merge remote-tracking branch 'origin' into cache-for-object-storage-table-engines

This commit is contained in:
kssenii 2024-09-30 21:17:36 +02:00
commit 50b6fd3969
997 changed files with 11353 additions and 2982 deletions

View File

@ -34,7 +34,7 @@ curl https://clickhouse.com/ | sh
Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know.
* [v24.9 Community Call](https://clickhouse.com/company/events/v24-9-community-release-call) - September 26
* [v24.10 Community Call](https://clickhouse.com/company/events/v24-10-community-release-call) - October 31
## Upcoming Events

View File

@ -0,0 +1,17 @@
# docker build -t clickhouse/style-test .
FROM ubuntu:22.04
RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
aspell \
libxml2-utils \
python3-pip \
locales \
git \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8
ENV LC_ALL=en_US.UTF-8
COPY requirements.txt /
RUN pip3 install --no-cache-dir -r requirements.txt

View File

@ -0,0 +1,4 @@
requests==2.32.3
yamllint==1.26.3
codespell==2.2.1
https://clickhouse-builds.s3.amazonaws.com/packages/praktika-0.1-py3-none-any.whl

410
ci_v2/jobs/check_style.py Normal file
View File

@ -0,0 +1,410 @@
import math
import multiprocessing
import os
import re
import sys
from concurrent.futures import ProcessPoolExecutor
from pathlib import Path
from praktika.result import Result
from praktika.utils import Shell, Utils
NPROC = multiprocessing.cpu_count()
def chunk_list(data, n):
"""Split the data list into n nearly equal-sized chunks."""
chunk_size = math.ceil(len(data) / n)
for i in range(0, len(data), chunk_size):
yield data[i : i + chunk_size]
def run_check_concurrent(check_name, check_function, files, nproc=NPROC):
stop_watch = Utils.Stopwatch()
if not files:
print(f"File list is empty [{files}]")
raise
file_chunks = list(chunk_list(files, nproc))
results = []
# Run check_function concurrently on each chunk
with ProcessPoolExecutor(max_workers=NPROC) as executor:
futures = [executor.submit(check_function, chunk) for chunk in file_chunks]
# Wait for results and process them (optional)
for future in futures:
try:
res = future.result()
if res and res not in results:
results.append(res)
except Exception as e:
results.append(f"Exception in {check_name}: {e}")
result = Result(
name=check_name,
status=Result.Status.SUCCESS if not results else Result.Status.FAILED,
start_time=stop_watch.start_time,
duration=stop_watch.duration,
info=f"errors: {results}" if results else "",
)
return result
def run_simple_check(check_name, check_function, **kwargs):
stop_watch = Utils.Stopwatch()
error = check_function(**kwargs)
result = Result(
name=check_name,
status=Result.Status.SUCCESS if not error else Result.Status.FAILED,
start_time=stop_watch.start_time,
duration=stop_watch.duration,
info=error,
)
return result
def run_check(check_name, check_function, files):
return run_check_concurrent(check_name, check_function, files, nproc=1)
def check_duplicate_includes(file_path):
includes = []
with open(file_path, "r", encoding="utf-8", errors="ignore") as f:
for line in f:
if re.match(r"^#include ", line):
includes.append(line.strip())
include_counts = {line: includes.count(line) for line in includes}
duplicates = {line: count for line, count in include_counts.items() if count > 1}
if duplicates:
return f"{file_path}: {duplicates}"
return ""
def check_whitespaces(file_paths):
for file in file_paths:
exit_code, out, err = Shell.get_res_stdout_stderr(
f'./ci_v2/jobs/scripts/check_style/double_whitespaces.pl "{file}"',
verbose=False,
)
if out or err:
return out + " err: " + err
return ""
def check_yamllint(file_paths):
file_paths = " ".join([f"'{file}'" for file in file_paths])
exit_code, out, err = Shell.get_res_stdout_stderr(
f"yamllint --config-file=./.yamllint {file_paths}", verbose=False
)
return out or err
def check_xmllint(file_paths):
if not isinstance(file_paths, list):
file_paths = [file_paths]
file_paths = " ".join([f"'{file}'" for file in file_paths])
exit_code, out, err = Shell.get_res_stdout_stderr(
f"xmllint --noout --nonet {file_paths}", verbose=False
)
return out or err
def check_functional_test_cases(files):
"""
Queries with event_date should have yesterday() not today()
NOTE: it is not that accuate, but at least something.
"""
patterns = [
re.compile(
r"(?i)where.*?\bevent_date\s*(=|>=)\s*today\(\)(?!\s*-\s*1)",
re.IGNORECASE | re.DOTALL,
)
]
errors = []
for test_case in files:
try:
with open(test_case, "r", encoding="utf-8", errors="replace") as f:
file_content = " ".join(
f.read().splitlines()
) # Combine lines into a single string
# Check if any pattern matches in the concatenated string
if any(pattern.search(file_content) for pattern in patterns):
errors.append(
f"event_date should be filtered using >=yesterday() in {test_case} (to avoid flakiness)"
)
except Exception as e:
errors.append(f"Error checking {test_case}: {e}")
for test_case in files:
if "fail" in test_case:
errors.append(f"test case {test_case} includes 'fail' in its name")
return " ".join(errors)
def check_gaps_in_tests_numbers(file_paths, gap_threshold=100):
test_numbers = set()
pattern = re.compile(r"(\d+)")
for file in file_paths:
file_name = os.path.basename(file)
match = pattern.search(file_name)
if match:
test_numbers.add(int(match.group(1)))
sorted_numbers = sorted(test_numbers)
large_gaps = []
for i in range(1, len(sorted_numbers)):
prev_num = sorted_numbers[i - 1]
next_num = sorted_numbers[i]
diff = next_num - prev_num
if diff >= gap_threshold:
large_gaps.append(f"Gap ({prev_num}, {next_num}) > {gap_threshold}")
return large_gaps
def check_broken_links(path, exclude_paths):
broken_symlinks = []
for path in Path(path).rglob("*"):
if any(exclude_path in str(path) for exclude_path in exclude_paths):
continue
if path.is_symlink():
if not path.exists():
broken_symlinks.append(str(path))
if broken_symlinks:
for symlink in broken_symlinks:
print(symlink)
return f"Broken symlinks found: {broken_symlinks}"
else:
return ""
def check_cpp_code():
res, out, err = Shell.get_res_stdout_stderr(
"./ci_v2/jobs/scripts/check_style/check_cpp.sh"
)
if err:
out += err
return out
def check_repo_submodules():
res, out, err = Shell.get_res_stdout_stderr(
"./ci_v2/jobs/scripts/check_style/check_submodules.sh"
)
if err:
out += err
return out
def check_other():
res, out, err = Shell.get_res_stdout_stderr(
"./ci_v2/jobs/scripts/check_style/checks_to_refactor.sh"
)
if err:
out += err
return out
def check_codespell():
res, out, err = Shell.get_res_stdout_stderr(
"./ci_v2/jobs/scripts/check_style/check_typos.sh"
)
if err:
out += err
return out
def check_aspell():
res, out, err = Shell.get_res_stdout_stderr(
"./ci_v2/jobs/scripts/check_style/check_aspell.sh"
)
if err:
out += err
return out
def check_mypy():
res, out, err = Shell.get_res_stdout_stderr(
"./ci_v2/jobs/scripts/check_style/check-mypy"
)
if err:
out += err
return out
def check_pylint():
res, out, err = Shell.get_res_stdout_stderr(
"./ci_v2/jobs/scripts/check_style/check-pylint"
)
if err:
out += err
return out
def check_file_names(files):
files_set = set()
for file in files:
file_ = file.lower()
if file_ in files_set:
return f"Non-uniq file name in lower case: {file}"
files_set.add(file_)
return ""
if __name__ == "__main__":
results = []
stop_watch = Utils.Stopwatch()
all_files = Utils.traverse_paths(
include_paths=["."],
exclude_paths=[
"./.git",
"./contrib",
"./build",
],
not_exists_ok=True, # ./build may exist if runs locally
)
cpp_files = Utils.traverse_paths(
include_paths=["./src", "./base", "./programs", "./utils"],
exclude_paths=[
"./base/glibc-compatibility",
"./contrib/consistent-hashing",
"./base/widechar_width",
],
file_suffixes=[".h", ".cpp"],
)
yaml_workflow_files = Utils.traverse_paths(
include_paths=["./.github"],
exclude_paths=[],
file_suffixes=[".yaml", ".yml"],
)
xml_files = Utils.traverse_paths(
include_paths=["."],
exclude_paths=["./.git", "./contrib/"],
file_suffixes=[".xml"],
)
functional_test_files = Utils.traverse_paths(
include_paths=["./tests/queries"],
exclude_paths=[],
file_suffixes=[".sql", ".sh", ".py", ".j2"],
)
results.append(
Result(
name="Read Files",
status=Result.Status.SUCCESS,
start_time=stop_watch.start_time,
duration=stop_watch.duration,
)
)
results.append(
run_check_concurrent(
check_name="Whitespace Check",
check_function=check_whitespaces,
files=cpp_files,
)
)
results.append(
run_check_concurrent(
check_name="YamlLint Check",
check_function=check_yamllint,
files=yaml_workflow_files,
)
)
results.append(
run_check_concurrent(
check_name="XmlLint Check",
check_function=check_xmllint,
files=xml_files,
)
)
results.append(
run_check_concurrent(
check_name="Functional Tests scripts smoke check",
check_function=check_functional_test_cases,
files=functional_test_files,
)
)
results.append(
run_check(
check_name="Check Tests Numbers",
check_function=check_gaps_in_tests_numbers,
files=functional_test_files,
)
)
results.append(
run_simple_check(
check_name="Check Broken Symlinks",
check_function=check_broken_links,
path="./",
exclude_paths=["contrib/", "metadata/", "programs/server/data"],
)
)
results.append(
run_simple_check(
check_name="Check CPP code",
check_function=check_cpp_code,
)
)
results.append(
run_simple_check(
check_name="Check Submodules",
check_function=check_repo_submodules,
)
)
results.append(
run_check(
check_name="Check File Names",
check_function=check_file_names,
files=all_files,
)
)
results.append(
run_simple_check(
check_name="Check Many Different Things",
check_function=check_other,
)
)
results.append(
run_simple_check(
check_name="Check Codespell",
check_function=check_codespell,
)
)
results.append(
run_simple_check(
check_name="Check Aspell",
check_function=check_aspell,
)
)
res = Result.create_from(results=results, stopwatch=stop_watch).dump()
if not res.is_ok():
print("Style check: failed")
for result in results:
if not result.is_ok():
print("Failed check:")
print(" | ", result)
sys.exit(1)
else:
print("Style check: ok")

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,59 @@
#!/usr/bin/env bash
# force-enable double star globbing
shopt -s globstar
# Perform spell checking on the docs
if [[ ${1:-} == "--help" ]] || [[ ${1:-} == "-h" ]]; then
echo "Usage $0 [--help|-h] [-i [filename]]"
echo " --help|-h: print this help"
echo " -i: interactive mode. If filename is specified, check only this file, otherwise check all files"
exit 0
fi
ROOT_PATH="."
CHECK_LANG=en
ASPELL_IGNORE_PATH="${ROOT_PATH}/utils/check-style/aspell-ignore/${CHECK_LANG}"
if [[ ${1:-} == "-i" ]]; then
if [[ ! -z ${2:-} ]]; then
FILES_TO_CHECK=${ROOT_PATH}/docs/${CHECK_LANG}/${2}
else
FILES_TO_CHECK=${ROOT_PATH}/docs/${CHECK_LANG}/**/*.md
fi
for fname in ${FILES_TO_CHECK}; do
echo "Checking $fname"
aspell --personal=aspell-dict.txt --add-sgml-skip=code --encoding=utf-8 --mode=markdown -W 3 --lang=${CHECK_LANG} --home-dir=${ASPELL_IGNORE_PATH} -c "$fname"
done
exit
fi
STATUS=0
for fname in ${ROOT_PATH}/docs/${CHECK_LANG}/**/*.md; do
errors=$(cat "$fname" \
| aspell list \
-W 3 \
--personal=aspell-dict.txt \
--add-sgml-skip=code \
--encoding=utf-8 \
--mode=markdown \
--lang=${CHECK_LANG} \
--home-dir=${ASPELL_IGNORE_PATH} \
| sort | uniq)
if [ ! -z "$errors" ]; then
STATUS=1
echo "====== $fname ======"
echo "$errors"
fi
done
if (( STATUS != 0 )); then
echo "====== Errors found ======"
echo "To exclude some words add them to the dictionary file \"${ASPELL_IGNORE_PATH}/aspell-dict.txt\""
echo "You can also run ${0} -i to see the errors interactively and fix them or add to the dictionary file"
fi
exit ${STATUS}

View File

@ -0,0 +1,339 @@
#!/usr/bin/env bash
# For code formatting we have clang-format.
#
# But it's not sane to apply clang-format for whole code base,
# because it sometimes makes worse for properly formatted files.
#
# It's only reasonable to blindly apply clang-format only in cases
# when the code is likely to be out of style.
#
# For this purpose we have a script that will use very primitive heuristics
# (simple regexps) to check if the code is likely to have basic style violations.
# and then to run formatter only for the specified files.
LC_ALL="en_US.UTF-8"
ROOT_PATH="."
EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|poco/|memcpy/|consistent-hashing|benchmark|tests/.*.cpp|utils/keeper-bench/example.yaml'
# From [1]:
# But since array_to_string_internal() in array.c still loops over array
# elements and concatenates them into a string, it's probably not more
# efficient than the looping solutions proposed, but it's more readable.
#
# [1]: https://stackoverflow.com/a/15394738/328260
function in_array()
{
local IFS="|"
local value=$1 && shift
[[ "${IFS}${*}${IFS}" =~ "${IFS}${value}${IFS}" ]]
}
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null |
grep -vP $EXCLUDE_DIRS |
xargs grep $@ -P '((class|struct|namespace|enum|if|for|while|else|throw|switch).*|\)(\s*const)?(\s*override)?\s*)\{$|\s$|^ {1,3}[^\* ]\S|\t|^\s*(if|else if|if constexpr|else if constexpr|for|while|catch|switch)\(|\( [^\s\\]|\S \)' |
# a curly brace not in a new line, but not for the case of C++11 init or agg. initialization | trailing whitespace | number of ws not a multiple of 4, but not in the case of comment continuation | missing whitespace after for/if/while... before opening brace | whitespaces inside braces
grep -v -P '(//|:\s+\*|\$\(\()| \)"'
# single-line comment | continuation of a multiline comment | a typical piece of embedded shell code | something like ending of raw string literal
# Tabs
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null |
grep -vP $EXCLUDE_DIRS |
xargs grep $@ -F $'\t'
# // namespace comments are unneeded
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null |
grep -vP $EXCLUDE_DIRS |
xargs grep $@ -P '}\s*//+\s*namespace\s*'
# Broken symlinks
find -L $ROOT_PATH -type l 2>/dev/null | grep -v contrib && echo "^ Broken symlinks found"
# Duplicated or incorrect setting declarations
SETTINGS_FILE=$(mktemp)
cat $ROOT_PATH/src/Core/Settings.cpp $ROOT_PATH/src/Core/FormatFactorySettingsDeclaration.h | grep "M(" | awk '{print substr($2, 0, length($2) - 1) " " substr($1, 3, length($1) - 3) " SettingsDeclaration" }' > ${SETTINGS_FILE}
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep "extern const Settings" -T | awk '{print substr($5, 0, length($5) -1) " " substr($4, 9) " " substr($1, 0, length($1) - 1)}' >> ${SETTINGS_FILE}
# Duplicate extern declarations for settings
awk '{if (seen[$0]++) print $3 " -> " $1 ;}' ${SETTINGS_FILE} | while read line;
do
echo "Found duplicated setting declaration in: $line"
done
# Incorrect declarations for settings
for setting in $(awk '{print $1 " " $2}' ${SETTINGS_FILE} | sort | uniq | awk '{ print $1 }' | sort | uniq -d);
do
expected=$(grep "^$setting " ${SETTINGS_FILE} | grep SettingsDeclaration | awk '{ print $2 }')
grep "^$setting " ${SETTINGS_FILE} | grep -v " $expected" | awk '{ print $3 " found setting " $1 " with type " $2 }' | while read line;
do
echo "In $line but it should be $expected"
done
done
rm ${SETTINGS_FILE}
# Unused/Undefined/Duplicates ErrorCodes/ProfileEvents/CurrentMetrics
declare -A EXTERN_TYPES
EXTERN_TYPES[ErrorCodes]=int
EXTERN_TYPES[ProfileEvents]=Event
EXTERN_TYPES[CurrentMetrics]=Metric
EXTERN_TYPES_EXCLUDES=(
ProfileEvents::global_counters
ProfileEvents::Event
ProfileEvents::Count
ProfileEvents::Counters
ProfileEvents::end
ProfileEvents::increment
ProfileEvents::incrementForLogMessage
ProfileEvents::getName
ProfileEvents::Timer
ProfileEvents::Type
ProfileEvents::TypeEnum
ProfileEvents::dumpToMapColumn
ProfileEvents::getProfileEvents
ProfileEvents::ThreadIdToCountersSnapshot
ProfileEvents::LOCAL_NAME
ProfileEvents::keeper_profile_events
ProfileEvents::CountersIncrement
CurrentMetrics::add
CurrentMetrics::sub
CurrentMetrics::get
CurrentMetrics::set
CurrentMetrics::end
CurrentMetrics::Increment
CurrentMetrics::Metric
CurrentMetrics::values
CurrentMetrics::Value
CurrentMetrics::keeper_metrics
ErrorCodes::ErrorCode
ErrorCodes::getName
ErrorCodes::increment
ErrorCodes::end
ErrorCodes::values
ErrorCodes::values[i]
ErrorCodes::getErrorCodeByName
ErrorCodes::Value
)
for extern_type in ${!EXTERN_TYPES[@]}; do
type_of_extern=${EXTERN_TYPES[$extern_type]}
allowed_chars='[_A-Za-z]+'
# Unused
# NOTE: to fix automatically, replace echo with:
# sed -i "/extern const $type_of_extern $val/d" $file
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | {
# NOTE: the check is pretty dumb and distinguish only by the type_of_extern,
# and this matches with zkutil::CreateMode
grep -v -e 'src/Common/ZooKeeper/Types.h' -e 'src/Coordination/KeeperConstants.cpp'
} | {
grep -vP $EXCLUDE_DIRS | xargs grep -l -P "extern const $type_of_extern $allowed_chars"
} | while read file; do
grep -P "extern const $type_of_extern $allowed_chars;" $file | sed -r -e "s/^.*?extern const $type_of_extern ($allowed_chars);.*?$/\1/" | while read val; do
if ! grep -q "$extern_type::$val" $file; then
# Excludes for SOFTWARE_EVENT/HARDWARE_EVENT/CACHE_EVENT in ThreadProfileEvents.cpp
if [[ ! $extern_type::$val =~ ProfileEvents::Perf.* ]]; then
echo "$extern_type::$val is defined but not used in file $file"
fi
fi
done
done
# Undefined
# NOTE: to fix automatically, replace echo with:
# ( grep -q -F 'namespace $extern_type' $file && \
# sed -i -r "0,/(\s*)extern const $type_of_extern [$allowed_chars]+/s//\1extern const $type_of_extern $val;\n&/" $file || \
# awk '{ print; if (ns == 1) { ns = 2 }; if (ns == 2) { ns = 0; print "namespace $extern_type\n{\n extern const $type_of_extern '$val';\n}" } }; /namespace DB/ { ns = 1; };' < $file > ${file}.tmp && mv ${file}.tmp $file )
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | {
grep -vP $EXCLUDE_DIRS | xargs grep -l -P "$extern_type::$allowed_chars"
} | while read file; do
grep -P "$extern_type::$allowed_chars" $file | grep -P -v '^\s*//' | sed -r -e "s/^.*?$extern_type::($allowed_chars).*?$/\1/" | while read val; do
if ! grep -q "extern const $type_of_extern $val" $file; then
if ! in_array "$extern_type::$val" "${EXTERN_TYPES_EXCLUDES[@]}"; then
echo "$extern_type::$val is used in file $file but not defined"
fi
fi
done
done
# Duplicates
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | {
grep -vP $EXCLUDE_DIRS | xargs grep -l -P "$extern_type::$allowed_chars"
} | while read file; do
grep -P "extern const $type_of_extern $allowed_chars;" $file | sort | uniq -c | grep -v -P ' +1 ' && echo "Duplicate $extern_type in file $file"
done
done
# Three or more consecutive empty lines
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
while read file; do awk '/^$/ { ++i; if (i > 2) { print "More than two consecutive empty lines in file '$file'" } } /./ { i = 0 }' $file; done
# Check that every header file has #pragma once in first line
find $ROOT_PATH/{src,programs,utils} -name '*.h' |
grep -vP $EXCLUDE_DIRS |
while read file; do [[ $(head -n1 $file) != '#pragma once' ]] && echo "File $file must have '#pragma once' in first line"; done
# Too many exclamation marks
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -F '!!!' | grep -P '.' && echo "Too many exclamation marks (looks dirty, unconfident)."
# Exclamation mark in a message
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -F '!",' | grep -P '.' && echo "No need for an exclamation mark (looks dirty, unconfident)."
# Trailing whitespaces
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -n -P ' $' | grep -n -P '.' && echo "^ Trailing whitespaces."
# Forbid stringstream because it's easy to use them incorrectly and hard to debug possible issues
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -P 'std::[io]?stringstream' | grep -v "STYLE_CHECK_ALLOW_STD_STRING_STREAM" && echo "Use WriteBufferFromOwnString or ReadBufferFromString instead of std::stringstream"
# Forbid std::cerr/std::cout in src (fine in programs/utils)
std_cerr_cout_excludes=(
/examples/
/tests/
_fuzzer
# OK
src/Common/ProgressIndication.cpp
# only under #ifdef DBMS_HASH_MAP_DEBUG_RESIZES, that is used only in tests
src/Common/HashTable/HashTable.h
# SensitiveDataMasker::printStats()
src/Common/SensitiveDataMasker.cpp
# StreamStatistics::print()
src/Compression/LZ4_decompress_faster.cpp
# ContextSharedPart with subsequent std::terminate()
src/Interpreters/Context.cpp
# IProcessor::dump()
src/Processors/IProcessor.cpp
src/Client/ClientApplicationBase.cpp
src/Client/ClientBase.cpp
src/Client/LineReader.cpp
src/Client/QueryFuzzer.cpp
src/Client/Suggest.cpp
src/Client/ClientBase.h
src/Client/LineReader.h
src/Client/ReplxxLineReader.h
src/Bridge/IBridge.cpp
src/Daemon/BaseDaemon.cpp
src/Loggers/Loggers.cpp
src/Common/GWPAsan.cpp
src/Common/ProgressIndication.h
)
sources_with_std_cerr_cout=( $(
find $ROOT_PATH/{src,base} -name '*.h' -or -name '*.cpp' | \
grep -vP $EXCLUDE_DIRS | \
grep -F -v $(printf -- "-e %s " "${std_cerr_cout_excludes[@]}") | \
xargs grep -F --with-filename -e std::cerr -e std::cout | cut -d: -f1 | sort -u
) )
# Exclude comments
for src in "${sources_with_std_cerr_cout[@]}"; do
# suppress stderr, since it may contain warning for #pargma once in headers
if gcc -fpreprocessed -dD -E "$src" 2>/dev/null | grep -F -q -e std::cerr -e std::cout; then
echo "$src: uses std::cerr/std::cout"
fi
done
expect_tests=( $(find $ROOT_PATH/tests/queries -name '*.expect') )
for test_case in "${expect_tests[@]}"; do
pattern="^exp_internal -f \$CLICKHOUSE_TMP/\$basename.debuglog 0$"
grep -q "$pattern" "$test_case" || echo "Missing '$pattern' in '$test_case'"
if grep -q "^spawn.*CLICKHOUSE_CLIENT_BINARY$" "$test_case"; then
pattern="^spawn.*CLICKHOUSE_CLIENT_BINARY.*--history_file$"
grep -q "$pattern" "$test_case" || echo "Missing '$pattern' in '$test_case'"
fi
# Otherwise expect_after/expect_before will not bail without stdin attached
# (and actually this is a hack anyway, correct way is to use $any_spawn_id)
pattern="-i \$any_spawn_id timeout"
grep -q -- "$pattern" "$test_case" || echo "Missing '$pattern' in '$test_case'"
pattern="-i \$any_spawn_id eof"
grep -q -- "$pattern" "$test_case" || echo "Missing '$pattern' in '$test_case'"
done
# Forbid non-unique error codes
if [[ "$(grep -Po "M\([0-9]*," $ROOT_PATH/src/Common/ErrorCodes.cpp | wc -l)" != "$(grep -Po "M\([0-9]*," $ROOT_PATH/src/Common/ErrorCodes.cpp | sort | uniq | wc -l)" ]]
then
echo "ErrorCodes.cpp contains non-unique error codes"
fi
# Check that there is no system-wide libraries/headers in use.
#
# NOTE: it is better to override find_path/find_library in cmake, but right now
# it is not possible, see [1] for the reference.
#
# [1]: git grep --recurse-submodules -e find_library -e find_path contrib
if git grep -e find_path -e find_library -- :**CMakeLists.txt; then
echo "There is find_path/find_library usage. ClickHouse should use everything bundled. Consider adding one more contrib module."
fi
# Forbid std::filesystem::is_symlink and std::filesystem::read_symlink, because it's easy to use them incorrectly
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -P '::(is|read)_symlink' | grep -v "STYLE_CHECK_ALLOW_STD_FS_SYMLINK" && echo "Use DB::FS::isSymlink and DB::FS::readSymlink instead"
# Forbid __builtin_unreachable(), because it's hard to debug when it becomes reachable
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -P '__builtin_unreachable' && echo "Use UNREACHABLE() from defines.h instead"
# Forbid mt19937() and random_device() which are outdated and slow
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -P '(std::mt19937|std::mersenne_twister_engine|std::random_device)' && echo "Use pcg64_fast (from pcg_random.h) and randomSeed (from Common/randomSeed.h) instead"
# Require checking return value of close(),
# since it can hide fd misuse and break other places.
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -e ' close(.*fd' -e ' ::close(' | grep -v = && echo "Return value of close() should be checked"
# A small typo can lead to debug code in release builds, see https://github.com/ClickHouse/ClickHouse/pull/47647
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -l -F '#ifdef NDEBUG' | xargs -I@FILE awk '/#ifdef NDEBUG/ { inside = 1; dirty = 1 } /#endif/ { if (inside && dirty) { print "File @FILE has suspicious #ifdef NDEBUG, possibly confused with #ifndef NDEBUG" }; inside = 0 } /#else/ { dirty = 0 }' @FILE
# If a user is doing dynamic or typeid cast with a pointer, and immediately dereferencing it, it is unsafe.
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep --line-number -P '(dynamic|typeid)_cast<[^>]+\*>\([^\(\)]+\)->' | grep -P '.' && echo "It's suspicious when you are doing a dynamic_cast or typeid_cast with a pointer and immediately dereferencing it. Use references instead of pointers or check a pointer to nullptr."
# Check for bad punctuation: whitespace before comma.
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '\w ,' | grep -v 'bad punctuation is ok here' && echo "^ There is bad punctuation: whitespace before comma. You should write it like this: 'Hello, world!'"
# Check usage of std::regex which is too bloated and slow.
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number 'std::regex' | grep -P '.' && echo "^ Please use re2 instead of std::regex"
# Cyrillic characters hiding inside Latin.
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | grep -v StorageSystemContributors.generated.cpp | xargs grep -P --line-number '[a-zA-Z][а-яА-ЯёЁ]|[а-яА-ЯёЁ][a-zA-Z]' && echo "^ Cyrillic characters found in unexpected place."
# Orphaned header files.
join -v1 <(find $ROOT_PATH/{src,programs,utils} -name '*.h' -printf '%f\n' | sort | uniq) <(find $ROOT_PATH/{src,programs,utils} -name '*.cpp' -or -name '*.c' -or -name '*.h' -or -name '*.S' | xargs grep --no-filename -o -P '[\w-]+\.h' | sort | uniq) |
grep . && echo '^ Found orphan header files.'
# Don't allow dynamic compiler check with CMake, because we are using hermetic, reproducible, cross-compiled, static (TLDR, good) builds.
ls -1d $ROOT_PATH/contrib/*-cmake | xargs -I@ find @ -name 'CMakeLists.txt' -or -name '*.cmake' | xargs grep --with-filename -i -P 'check_c_compiler_flag|check_cxx_compiler_flag|check_c_source_compiles|check_cxx_source_compiles|check_include_file|check_symbol_exists|cmake_push_check_state|cmake_pop_check_state|find_package|CMAKE_REQUIRED_FLAGS|CheckIncludeFile|CheckCCompilerFlag|CheckCXXCompilerFlag|CheckCSourceCompiles|CheckCXXSourceCompiles|CheckCSymbolExists|CheckCXXSymbolExists' | grep -v Rust && echo "^ It's not allowed to have dynamic compiler checks with CMake."
# Wrong spelling of abbreviations, e.g. SQL is right, Sql is wrong. XMLHttpRequest is very wrong.
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -P 'Sql|Html|Xml|Cpu|Tcp|Udp|Http|Db|Json|Yaml' | grep -v -P 'RabbitMQ|Azure|Aws|aws|Avro|IO/S3' &&
echo "Abbreviations such as SQL, XML, HTTP, should be in all caps. For example, SQL is right, Sql is wrong. XMLHttpRequest is very wrong."
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -F -i 'ErrorCodes::LOGICAL_ERROR, "Logical error:' &&
echo "If an exception has LOGICAL_ERROR code, there is no need to include the text 'Logical error' in the exception message, because then the phrase 'Logical error' will be printed twice."
# There shouldn't be any code snippets under GPL or LGPL
find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | xargs grep -i -F 'General Public License' && echo "There shouldn't be any code snippets under GPL or LGPL"
PATTERN="allow_";
DIFF=$(comm -3 <(grep -o "\b$PATTERN\w*\b" $ROOT_PATH/src/Core/Settings.cpp | sort -u) <(grep -o -h "\b$PATTERN\w*\b" $ROOT_PATH/src/Databases/enableAllExperimentalSettings.cpp $ROOT_PATH/utils/check-style/experimental_settings_ignore.txt | sort -u));
[ -n "$DIFF" ] && echo "$DIFF" && echo "^^ Detected 'allow_*' settings that might need to be included in src/Databases/enableAllExperimentalSettings.cpp" && echo "Alternatively, consider adding an exception to utils/check-style/experimental_settings_ignore.txt"

View File

@ -0,0 +1,37 @@
#!/usr/bin/env bash
# The script checks if all submodules defined in $GIT_ROOT/.gitmodules exist in $GIT_ROOT/contrib
set -e
GIT_ROOT="."
cd "$GIT_ROOT"
# Remove keys for submodule.*.path parameters, the values are separated by \0
# and check if the directory exists
git config --file .gitmodules --null --get-regexp path | sed -z 's|.*\n||' | \
xargs -P100 -0 --no-run-if-empty -I{} bash -c 'if ! test -d '"'{}'"'; then echo Directory for submodule {} is not found; exit 1; fi' 2>&1
# And check that the submodule is fine
git config --file .gitmodules --null --get-regexp path | sed -z 's|.*\n||' | \
xargs -P100 -0 --no-run-if-empty -I{} git submodule status -q '{}' 2>&1
# All submodules should be from https://github.com/
git config --file "$ROOT_PATH/.gitmodules" --get-regexp 'submodule\..+\.url' | \
while read -r line; do
name=${line#submodule.}; name=${name%.url*}
url=${line#* }
[[ "$url" != 'https://github.com/'* ]] && echo "All submodules should be from https://github.com/, submodule '$name' has '$url'"
done
# All submodules should be of this form: [submodule "contrib/libxyz"] (for consistency, the submodule name does matter too much)
# - restrict the check to top-level .gitmodules file
git config --file "$ROOT_PATH/.gitmodules" --get-regexp 'submodule\..+\.path' | \
while read -r line; do
name=${line#submodule.}; name=${name%.path*}
path=${line#* }
[ "$name" != "$path" ] && echo "Submodule name '$name' is not equal to it's path '$path'"
done

View File

@ -0,0 +1,15 @@
#!/usr/bin/env bash
# Check for typos in code.
ROOT_PATH="."
#FIXME: check all (or almost all) repo
codespell \
--skip "*generated*,*gperf*,*.bin,*.mrk*,*.idx,checksums.txt,*.dat,*.pyc,*.kate-swp,*obfuscateQueries.cpp,d3-*.js,*.min.js,*.sum,${ROOT_PATH}/utils/check-style/aspell-ignore" \
--ignore-words "${ROOT_PATH}/utils/check-style/codespell-ignore-words.list" \
--exclude-file "${ROOT_PATH}/utils/check-style/codespell-ignore-lines.list" \
--quiet-level 2 \
"$ROOT_PATH"/{src,base,programs,utils} \
$@ | grep -P '.' \
&& echo -e "\nFound some typos in code.\nSee the files utils/check-style/codespell* if you want to add an exception."

View File

@ -0,0 +1,98 @@
#!/bin/bash
ROOT_PATH="."
# Queries to system.query_log/system.query_thread_log should have current_database = currentDatabase() condition
# NOTE: it is not that accurate, but at least something.
tests_with_query_log=( $(
find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' |
xargs grep --with-filename -e system.query_log -e system.query_thread_log | cut -d: -f1 | sort -u
) )
for test_case in "${tests_with_query_log[@]}"; do
grep -qE current_database.*currentDatabase "$test_case" || {
grep -qE 'current_database.*\$CLICKHOUSE_DATABASE' "$test_case"
} || echo "Queries to system.query_log/system.query_thread_log does not have current_database = currentDatabase() condition in $test_case"
done
grep -iE 'SYSTEM STOP MERGES;?$' -R $ROOT_PATH/tests/queries && echo "Merges cannot be disabled globally in fast/stateful/stateless tests, because it will break concurrently running queries"
# Queries to:
tables_with_database_column=(
system.tables
system.parts
system.detached_parts
system.parts_columns
system.columns
system.projection_parts
system.mutations
)
# should have database = currentDatabase() condition
#
# NOTE: it is not that accuate, but at least something.
tests_with_database_column=( $(
find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' |
xargs grep --with-filename $(printf -- "-e %s " "${tables_with_database_column[@]}") |
grep -v -e ':--' -e ':#' |
cut -d: -f1 | sort -u
) )
for test_case in "${tests_with_database_column[@]}"; do
grep -qE database.*currentDatabase "$test_case" || {
grep -qE 'database.*\$CLICKHOUSE_DATABASE' "$test_case"
} || {
# explicit database
grep -qE "database[ ]*=[ ]*'" "$test_case"
} || {
echo "Queries to ${tables_with_database_column[*]} does not have database = currentDatabase()/\$CLICKHOUSE_DATABASE condition in $test_case"
}
done
# Queries with ReplicatedMergeTree
# NOTE: it is not that accuate, but at least something.
tests_with_replicated_merge_tree=( $(
find $ROOT_PATH/tests/queries -iname '*.sql' -or -iname '*.sh' -or -iname '*.py' -or -iname '*.j2' |
xargs grep --with-filename -e "Replicated.*MergeTree[ ]*(.*" | cut -d: -f1 | sort -u
) )
for test_case in "${tests_with_replicated_merge_tree[@]}"; do
case "$test_case" in
*.gen.*)
;;
*.sh)
test_case_zk_prefix="\(\$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX\|{database}\)"
grep -q -e "Replicated.*MergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "Replicated.*MergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)"
;;
*.sql|*.sql.j2)
test_case_zk_prefix="\({database}\|currentDatabase()\|{uuid}\|{default_path_test}\)"
grep -q -e "Replicated.*MergeTree[ ]*(.*$test_case_zk_prefix" "$test_case" || echo "Replicated.*MergeTree should contain '$test_case_zk_prefix' in zookeeper path to avoid overlaps ($test_case)"
;;
*.py)
# Right now there is not such tests anyway
echo "No ReplicatedMergeTree style check for *.py ($test_case)"
;;
esac
done
# The stateful directory should only contain the tests that depend on the test dataset (hits or visits).
find $ROOT_PATH/tests/queries/1_stateful -name '*.sql' -or -name '*.sh' | grep -v '00076_system_columns_bytes' | xargs -I{} bash -c 'grep -q -P "hits|visits" "{}" || echo "The test {} does not depend on the test dataset (hits or visits table) and should be located in the 0_stateless directory. You can also add an exception to the check-style script."'
# Check for existence of __init__.py files
for i in "${ROOT_PATH}"/tests/integration/test_*; do FILE="${i}/__init__.py"; [ ! -f "${FILE}" ] && echo "${FILE} should exist for every integration test"; done
# Check for executable bit on non-executable files
find $ROOT_PATH/{src,base,programs,utils,tests,docs,cmake} '(' -name '*.cpp' -or -name '*.h' -or -name '*.sql' -or -name '*.j2' -or -name '*.xml' -or -name '*.reference' -or -name '*.txt' -or -name '*.md' ')' -and -executable | grep -P '.' && echo "These files should not be executable."
# Check for BOM
find $ROOT_PATH/{src,base,programs,utils,tests,docs,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' | xargs grep -l -F $'\xEF\xBB\xBF' | grep -P '.' && echo "Files should not have UTF-8 BOM"
find $ROOT_PATH/{src,base,programs,utils,tests,docs,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' | xargs grep -l -F $'\xFF\xFE' | grep -P '.' && echo "Files should not have UTF-16LE BOM"
find $ROOT_PATH/{src,base,programs,utils,tests,docs,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' | xargs grep -l -F $'\xFE\xFF' | grep -P '.' && echo "Files should not have UTF-16BE BOM"
# Conflict markers
find $ROOT_PATH/{src,base,programs,utils,tests,docs,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' |
xargs grep -P '^(<<<<<<<|=======|>>>>>>>)$' | grep -P '.' && echo "Conflict markers are found in files"
# DOS/Windows newlines
find $ROOT_PATH/{base,src,programs,utils,docs} -name '*.md' -or -name '*.h' -or -name '*.cpp' -or -name '*.js' -or -name '*.py' -or -name '*.html' | xargs grep -l -P '\r$' && echo "^ Files contain DOS/Windows newlines (\r\n instead of \n)."
# # workflows check
# act --list --directory="$ROOT_PATH" 1>/dev/null 2>&1 || act --list --directory="$ROOT_PATH" 2>&1
# actionlint -ignore 'reusable workflow call.+' || :

View File

@ -0,0 +1,37 @@
#!/usr/bin/perl
use strict;
# Find double whitespace such as "a, b, c" that looks very ugly and annoying.
# But skip double whitespaces if they are used as an alignment - by comparing to surrounding lines.
my $ret = 0;
foreach my $file (@ARGV)
{
my @array;
open (FH,'<',$file);
while (<FH>)
{
push @array, $_;
}
for (my $i = 1; $i < $#array; ++$i)
{
if ($array[$i] =~ ',( {2,3})[^ /]')
{
# https://stackoverflow.com/questions/87380/how-can-i-find-the-location-of-a-regex-match-in-perl
if ((substr($array[$i - 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/) # whitespaces are not part of alignment
&& (substr($array[$i + 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/)
&& $array[$i] !~ /(-?\d+\w*,\s+){3,}/) # this is not a number table like { 10, -1, 2 }
{
print($file . ":" . ($i + 1) . $array[$i]);
$ret = 1;
}
}
}
}
exit $ret;

View File

@ -0,0 +1,251 @@
from praktika import Docker, Secret
S3_BUCKET_NAME = "clickhouse-builds"
S3_BUCKET_HTTP_ENDPOINT = "clickhouse-builds.s3.amazonaws.com"
class RunnerLabels:
CI_SERVICES = "ci_services"
CI_SERVICES_EBS = "ci_services_ebs"
BASE_BRANCH = "master"
SECRETS = [
Secret.Config(
name="dockerhub_robot_password",
type=Secret.Type.AWS_SSM_VAR,
),
Secret.Config(
name="woolenwolf_gh_app.clickhouse-app-id",
type=Secret.Type.AWS_SSM_SECRET,
),
Secret.Config(
name="woolenwolf_gh_app.clickhouse-app-key",
type=Secret.Type.AWS_SSM_SECRET,
),
]
DOCKERS = [
# Docker.Config(
# name="clickhouse/binary-builder",
# path="./docker/packager/binary-builder",
# arm64=True,
# amd64=True,
# depends_on=[],
# ),
# Docker.Config(
# name="clickhouse/cctools",
# path="./docker/packager/cctools",
# arm64=True,
# amd64=True,
# depends_on=[],
# ),
# Docker.Config(
# name="clickhouse/test-old-centos",
# path="./docker/test/compatibility/centos",
# arm64=True,
# amd64=True,
# depends_on=[],
# ),
# Docker.Config(
# name="clickhouse/test-old-ubuntu",
# path="./docker/test/compatibility/ubuntu",
# arm64=True,
# amd64=True,
# depends_on=[],
# ),
# Docker.Config(
# name="clickhouse/test-util",
# path="./docker/test/util",
# arm64=True,
# amd64=True,
# depends_on=[],
# ),
# Docker.Config(
# name="clickhouse/integration-test",
# path="./docker/test/integration/base",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/fuzzer",
# path="./docker/test/fuzzer",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/performance-comparison",
# path="./docker/test/performance-comparison",
# arm64=True,
# amd64=True,
# depends_on=[],
# ),
# Docker.Config(
# name="clickhouse/fasttest",
# path="./docker/test/fasttest",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-util"],
# ),
# Docker.Config(
# name="clickhouse/test-base",
# path="./docker/test/base",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-util"],
# ),
# Docker.Config(
# name="clickhouse/clickbench",
# path="./docker/test/clickbench",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/keeper-jepsen-test",
# path="./docker/test/keeper-jepsen",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/server-jepsen-test",
# path="./docker/test/server-jepsen",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/sqllogic-test",
# path="./docker/test/sqllogic",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/sqltest",
# path="./docker/test/sqltest",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/stateless-test",
# path="./docker/test/stateless",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/stateful-test",
# path="./docker/test/stateful",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/stateless-test"],
# ),
# Docker.Config(
# name="clickhouse/stress-test",
# path="./docker/test/stress",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/stateful-test"],
# ),
# Docker.Config(
# name="clickhouse/unit-test",
# path="./docker/test/unit",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/integration-tests-runner",
# path="./docker/test/integration/runner",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
Docker.Config(
name="clickhouse/style-test",
path="./ci_v2/docker/style-test",
platforms=Docker.Platforms.arm_amd,
depends_on=[],
),
# Docker.Config(
# name="clickhouse/docs-builder",
# path="./docker/docs/builder",
# arm64=True,
# amd64=True,
# depends_on=["clickhouse/test-base"],
# ),
]
# TODO:
# "docker/test/integration/s3_proxy": {
# "name": "clickhouse/s3-proxy",
# "dependent": []
# },
# "docker/test/integration/resolver": {
# "name": "clickhouse/python-bottle",
# "dependent": []
# },
# "docker/test/integration/helper_container": {
# "name": "clickhouse/integration-helper",
# "dependent": []
# },
# "docker/test/integration/mysql_golang_client": {
# "name": "clickhouse/mysql-golang-client",
# "dependent": []
# },
# "docker/test/integration/dotnet_client": {
# "name": "clickhouse/dotnet-client",
# "dependent": []
# },
# "docker/test/integration/mysql_java_client": {
# "name": "clickhouse/mysql-java-client",
# "dependent": []
# },
# "docker/test/integration/mysql_js_client": {
# "name": "clickhouse/mysql-js-client",
# "dependent": []
# },
# "docker/test/integration/mysql_php_client": {
# "name": "clickhouse/mysql-php-client",
# "dependent": []
# },
# "docker/test/integration/postgresql_java_client": {
# "name": "clickhouse/postgresql-java-client",
# "dependent": []
# },
# "docker/test/integration/kerberos_kdc": {
# "only_amd64": true,
# "name": "clickhouse/kerberos-kdc",
# "dependent": []
# },
# "docker/test/integration/kerberized_hadoop": {
# "only_amd64": true,
# "name": "clickhouse/kerberized-hadoop",
# "dependent": []
# },
# "docker/test/sqlancer": {
# "name": "clickhouse/sqlancer-test",
# "dependent": []
# },
# "docker/test/install/deb": {
# "name": "clickhouse/install-deb-test",
# "dependent": []
# },
# "docker/test/install/rpm": {
# "name": "clickhouse/install-rpm-test",
# "dependent": []
# },
# "docker/test/integration/nginx_dav": {
# "name": "clickhouse/nginx-dav",
# "dependent": []
# }
class JobNames:
STYLE_CHECK = "Style Check"

View File

@ -0,0 +1,20 @@
from ci_v2.settings.definitions import (
S3_BUCKET_HTTP_ENDPOINT,
S3_BUCKET_NAME,
RunnerLabels,
)
S3_ARTIFACT_PATH = f"{S3_BUCKET_NAME}/artifacts"
CI_CONFIG_RUNS_ON = [RunnerLabels.CI_SERVICES]
DOCKER_BUILD_RUNS_ON = [RunnerLabels.CI_SERVICES_EBS]
CACHE_S3_PATH = f"{S3_BUCKET_NAME}/ci_ch_cache"
HTML_S3_PATH = f"{S3_BUCKET_NAME}/reports"
S3_BUCKET_TO_HTTP_ENDPOINT = {S3_BUCKET_NAME: S3_BUCKET_HTTP_ENDPOINT}
DOCKERHUB_USERNAME = "robotclickhouse"
DOCKERHUB_SECRET = "dockerhub_robot_password"
CI_DB_DB_NAME = "default"
CI_DB_TABLE_NAME = "checks"
INSTALL_PYTHON_REQS_FOR_NATIVE_JOBS = ""

View File

@ -0,0 +1,44 @@
from typing import List
from ci_v2.settings.definitions import (
BASE_BRANCH,
DOCKERS,
SECRETS,
JobNames,
RunnerLabels,
)
from praktika import Job, Workflow
style_check_job = Job.Config(
name=JobNames.STYLE_CHECK,
runs_on=[RunnerLabels.CI_SERVICES],
command="python3 ./ci_v2/jobs/check_style.py",
run_in_docker="clickhouse/style-test",
)
workflow = Workflow.Config(
name="PR",
event=Workflow.Event.PULL_REQUEST,
base_branches=[BASE_BRANCH],
jobs=[
style_check_job,
],
dockers=DOCKERS,
secrets=SECRETS,
enable_cache=True,
enable_report=True,
enable_merge_ready_status=True,
)
WORKFLOWS = [
workflow,
] # type: List[Workflow.Config]
if __name__ == "__main__":
# example: local job test inside praktika environment
from praktika.runner import Runner
Runner.generate_dummy_environment(workflow, style_check_job)
Runner().run(workflow, style_check_job)

View File

@ -69,6 +69,11 @@ set (SRCS_PROGRAM_OPTIONS
"${LIBRARY_DIR}/libs/program_options/src/winmain.cpp"
)
# Always compile this file with the highest possible level of optimizations, even in Debug builds.
# Otherwise the client takes too long to start and SQL stateless tests (many options to parse)
# https://github.com/ClickHouse/ClickHouse/issues/65745
set_source_files_properties(${SRCS_PROGRAM_OPTIONS} PROPERTIES COMPILE_FLAGS "-O3")
add_library (_boost_program_options ${SRCS_PROGRAM_OPTIONS})
add_library (boost::program_options ALIAS _boost_program_options)
target_include_directories (_boost_program_options SYSTEM BEFORE PUBLIC ${LIBRARY_DIR})

View File

@ -164,15 +164,6 @@ target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE)
# Because our coverage callbacks call malloc, and recursive call of malloc could not work.
target_compile_options(_jemalloc PRIVATE ${WITHOUT_COVERAGE_FLAGS_LIST})
if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")
target_compile_definitions(_jemalloc PRIVATE
-DJEMALLOC_DEBUG=1
# Usage examples:
# - MALLOC_CONF=log:.
# - MALLOC_CONF='log:core.malloc.exit|core.sallocx.entry|core.sdallocx.entry'
-DJEMALLOC_LOG=1)
endif ()
target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_PROF=1)
# jemalloc provides support two unwind flavors:

View File

@ -2,8 +2,8 @@
# To run this script you must install docker and piddeptree python package
#
import subprocess
import os
import subprocess
import sys

View File

@ -1,9 +1,10 @@
#!/usr/bin/env python3
from argparse import ArgumentParser
import os
import jinja2
import itertools
import os
from argparse import ArgumentParser
import jinja2
def removesuffix(text, suffix):

View File

@ -1,7 +1,8 @@
import datetime
import os
import subprocess
import datetime
from flask import Flask, flash, request, redirect, url_for
from flask import Flask, flash, redirect, request, url_for
def run_command(command, wait=False):

View File

@ -1,9 +1,9 @@
#!/usr/bin/env python3
import os
import logging
import argparse
import csv
import logging
import os
def process_result(result_folder):

View File

@ -1,12 +1,12 @@
#!/usr/bin/env python3
import os
import yaml
import html
import os
import random
import string
from clickhouse_driver import Client
import yaml
from clickhouse_driver import Client
client = Client(host="localhost", port=9000)
settings = {

View File

@ -232,7 +232,7 @@ Below you can find some quick links which may be useful when writing code for Cl
- [The code style guide](https://clickhouse.com/docs/en/development/style/).
- [Adding third-party libraries](https://clickhouse.com/docs/en/development/contrib/#adding-third-party-libraries)
- [Writing tests](https://clickhouse.com/docs/en/development/tests/)
- [List of open issues](https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3Ahacktoberfest)
- [List of open issues](https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22)
## Writing Documentation {#writing-documentation}

View File

@ -7,7 +7,7 @@ description: Analyzing Stack Overflow data with ClickHouse
# Analyzing Stack Overflow data with ClickHouse
This dataset contains every `Post`, `User`, `Vote`, `Comment`, `Badge, `PostHistory`, and `PostLink` that has occurred on Stack Overflow.
This dataset contains every `Posts`, `Users`, `Votes`, `Comments`, `Badges`, `PostHistory`, and `PostLinks` that has occurred on Stack Overflow.
Users can either download pre-prepared Parquet versions of the data, containing every post up to April 2024, or download the latest data in XML format and load this. Stack Overflow provide updates to this data periodically - historically every 3 months.
@ -159,7 +159,7 @@ INSERT INTO stackoverflow.badges SELECT * FROM s3('https://datasets-documentatio
0 rows in set. Elapsed: 6.635 sec. Processed 51.29 million rows, 797.05 MB (7.73 million rows/s., 120.13 MB/s.)
```
### `PostLinks`
### PostLinks
```sql
CREATE TABLE stackoverflow.postlinks
@ -178,7 +178,7 @@ INSERT INTO stackoverflow.postlinks SELECT * FROM s3('https://datasets-documenta
0 rows in set. Elapsed: 1.534 sec. Processed 6.55 million rows, 129.70 MB (4.27 million rows/s., 84.57 MB/s.)
```
### `PostHistory`
### PostHistory
```sql
CREATE TABLE stackoverflow.posthistory

View File

@ -5683,6 +5683,24 @@ Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting
Default value: `false`.
## show_create_query_identifier_quoting_rule
Define identifier quoting behavior of the show create query result:
- `when_necessary`: When the identifiers is one of `{"distinct", "all", "table"}`, or it can cause ambiguity: column names, dictionary attribute names.
- `always`: Always quote identifiers.
- `user_display`: When the identifiers is a keyword.
Default value: `when_necessary`.
## show_create_query_identifier_quoting_style
Define identifier quoting style of the show create query result:
- `Backticks`: \`clickhouse\` style.
- `DoubleQuotes`: "postgres" style
- `BackticksMySQL`: \`mysql\` style, most same as `Backticks`, but it uses '``' to escape '`'
Default value: `Backticks`.
## mongodb_throw_on_unsupported_query
If enabled, MongoDB tables will return an error when a MongoDB query can't be built.

View File

@ -51,6 +51,40 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed
If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the sipHash128 function instead.
If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))).
## RIPEMD160
Produces [RIPEMD-160](https://en.wikipedia.org/wiki/RIPEMD) hash value.
**Syntax**
```sql
RIPEMD160(input)
```
**Parameters**
- `input`: Input string. [String](../data-types/string.md)
**Returned value**
- A 160-bit `RIPEMD-160` hash value of type [FixedString(20)](../data-types/fixedstring.md).
**Example**
Use the [hex](../functions/encoding-functions.md/#hex) function to represent the result as a hex-encoded string.
Query:
```sql
SELECT HEX(RIPEMD160('The quick brown fox jumps over the lazy dog'));
```
```response
┌─HEX(RIPEMD160('The quick brown fox jumps over the lazy dog'))─┐
│ 37F332F68DB77BD9D7EDD4969571AD671CF9DD3B │
└───────────────────────────────────────────────────────────────┘
```
## sipHash64
Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value.

View File

@ -2789,6 +2789,45 @@ Result:
- [Custom Settings](../../operations/settings/index.md#custom_settings)
## getSettingOrDefault
Returns the current value of a [custom setting](../../operations/settings/index.md#custom_settings) or returns the default value specified in the 2nd argument if the custom setting is not set in the current profile.
**Syntax**
```sql
getSettingOrDefault('custom_setting', default_value);
```
**Parameter**
- `custom_setting` — The setting name. [String](../data-types/string.md).
- `default_value` — Value to return if custom_setting is not set. Value may be of any data type or Null.
**Returned value**
- The setting's current value or default_value if setting is not set.
**Example**
```sql
SELECT getSettingOrDefault('custom_undef1', 'my_value');
SELECT getSettingOrDefault('custom_undef2', 100);
SELECT getSettingOrDefault('custom_undef3', NULL);
```
Result:
```
my_value
100
NULL
```
**See Also**
- [Custom Settings](../../operations/settings/index.md#custom_settings)
## isDecimalOverflow
Checks whether the [Decimal](../data-types/decimal.md) value is outside its precision or outside the specified precision.

View File

@ -12,10 +12,10 @@ Syntax:
``` sql
ALTER USER [IF EXISTS] name1 [RENAME TO new_name |, name2 [,...]]
[ON CLUSTER cluster_name]
[NOT IDENTIFIED | IDENTIFIED | ADD 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' | SAN 'TYPE:subject_alt_name'}]
[NOT IDENTIFIED | RESET AUTHENTICATION METHODS TO NEW | {IDENTIFIED | ADD IDENTIFIED} {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']}
[, {[{plaintext_password | sha256_password | sha256_hash | ...}] BY {'password' | 'hash'}} | {ldap SERVER 'server_name'} | {...} | ... [,...]]]
[[ADD | DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
[VALID UNTIL datetime]
[RESET AUTHENTICATION METHODS TO NEW]
[DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ]
[GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY | WRITABLE] | PROFILE 'profile_name'] [,...]

View File

@ -43,6 +43,19 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine]
Creates a table with the same structure as another table. You can specify a different engine for the table. If the engine is not specified, the same engine will be used as for the `db2.name2` table.
### With a Schema and Data Cloned from Another Table
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name CLONE AS [db2.]name2 [ENGINE = engine]
```
Creates a table with the same structure as another table. You can specify a different engine for the table. If the engine is not specified, the same engine will be used as for the `db2.name2` table. After the new table is created, all partitions from `db2.name2` are attached to it. In other words, the data of `db2.name2` is cloned into `db.table_name` upon creation. This query is equivalent to the following:
``` sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine];
ALTER TABLE [db.]table_name ATTACH PARTITION ALL FROM [db2].name2;
```
### From a Table Function
``` sql

View File

@ -11,10 +11,10 @@ Syntax:
``` sql
CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [, name2 [,...]] [ON CLUSTER cluster_name]
[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' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']}]
[NOT IDENTIFIED | IDENTIFIED {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']}
[, {[{plaintext_password | sha256_password | sha256_hash | ...}] BY {'password' | 'hash'}} | {ldap SERVER 'server_name'} | {...} | ... [,...]]]
[HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
[VALID UNTIL datetime]
[RESET AUTHENTICATION METHODS TO NEW]
[IN access_storage_type]
[DEFAULT ROLE role [,...]]
[DEFAULT DATABASE database | NONE]

View File

@ -13,8 +13,10 @@ sidebar_label: USER
``` sql
ALTER USER [IF EXISTS] name1 [RENAME TO new_name |, name2 [,...]]
[ON CLUSTER cluster_name]
[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']}]
[NOT IDENTIFIED | RESET AUTHENTICATION METHODS TO NEW | {IDENTIFIED | ADD IDENTIFIED} {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']}
[, {[{plaintext_password | sha256_password | sha256_hash | ...}] BY {'password' | 'hash'}} | {ldap SERVER 'server_name'} | {...} | ... [,...]]]
[[ADD | DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
[VALID UNTIL datetime]
[DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ]
[GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]]
[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY | WRITABLE] | PROFILE 'profile_name'] [,...]

View File

@ -12,8 +12,11 @@ sidebar_label: "Пользователь"
``` sql
CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [, name2 [,...]] [ON CLUSTER cluster_name]
[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' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'}]
[NOT IDENTIFIED | IDENTIFIED {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']}
[, {[{plaintext_password | sha256_password | sha256_hash | ...}] BY {'password' | 'hash'}} | {ldap SERVER 'server_name'} | {...} | ... [,...]]]
[HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
[VALID UNTIL datetime]
[IN access_storage_type]
[DEFAULT ROLE role [,...]]
[DEFAULT DATABASE database | NONE]
[GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]]

View File

@ -26,7 +26,7 @@ public:
String path_from = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow<String>(options, "path-from"));
std::optional<String> path_to = getValueFromCommandLineOptionsWithOptional<String>(options, "path-to");
auto in = disk.getDisk()->readFile(path_from);
auto in = disk.getDisk()->readFile(path_from, getReadSettings());
std::unique_ptr<WriteBufferFromFileBase> out = {};
if (path_to.has_value())
{

View File

@ -39,7 +39,7 @@ public:
else
{
String relative_path_from = disk.getRelativeFromRoot(path_from.value());
return disk.getDisk()->readFile(relative_path_from);
return disk.getDisk()->readFile(relative_path_from, getReadSettings());
}
}();

View File

@ -164,7 +164,7 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
std::string table_name = params.get("table_name");
LOG_TRACE(log, "DB name: '{}', table name: '{}'", db_name, table_name);
auto quoting_style = IdentifierQuotingStyle::None;
auto quoting_style = IdentifierQuotingStyle::Backticks;
#if USE_ODBC
quoting_style = getQuotingStyle(connection_handler);
#endif

View File

@ -38,7 +38,7 @@ IdentifierQuotingStyle getQuotingStyle(nanodbc::ConnectionHolderPtr connection)
{
auto identifier_quote = getIdentifierQuote(connection);
if (identifier_quote.empty())
return IdentifierQuotingStyle::None;
return IdentifierQuotingStyle::Backticks;
else if (identifier_quote[0] == '`')
return IdentifierQuotingStyle::Backticks;
else if (identifier_quote[0] == '"')

View File

@ -30,7 +30,7 @@ disable = '''
[tool.isort]
profile = "black"
src_paths = ["src", "tests/ci", "tests/sqllogic"]
src_paths = ["src", "tests/ci", "tests/sqllogic", "tests/queries", "tests/integration"]
[tool.black]
required-version = 24

View File

@ -30,6 +30,7 @@ namespace ErrorCodes
{
extern const int CANNOT_RESTORE_TABLE;
extern const int ACCESS_ENTITY_ALREADY_EXISTS;
extern const int ACCESS_ENTITY_NOT_FOUND;
extern const int LOGICAL_ERROR;
}
@ -41,6 +42,7 @@ namespace
{
std::unordered_map<UUID, AccessEntityPtr> entities;
std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies;
std::unordered_map<UUID, AccessEntityPtr> dependents;
BackupEntryPtr toBackupEntry() const
{
@ -72,6 +74,24 @@ namespace
}
}
if (!dependents.empty())
{
if (!dependencies.empty())
writeText("\n", buf);
writeText("DEPENDENTS\n", buf);
for (const auto & [id, entity] : dependents)
{
writeText(id, buf);
writeChar('\t', buf);
writeText(entity->getTypeInfo().name, buf);
writeChar('\t', buf);
writeText(entity->getName(), buf);
writeChar('\n', buf);
writeText(serializeAccessEntity(*entity), buf);
writeChar('\n', buf);
}
}
return std::make_shared<BackupEntryFromMemory>(buf.str());
}
@ -81,59 +101,71 @@ namespace
{
AccessEntitiesInBackup res;
bool dependencies_found = false;
bool reading_dependencies = false;
bool reading_dependents = false;
while (!buf->eof())
{
String line;
readStringUntilNewlineInto(line, *buf);
buf->ignore();
if (line == "DEPENDENCIES")
{
dependencies_found = true;
break;
reading_dependencies = true;
reading_dependents = false;
continue;
}
else if (line == "DEPENDENTS")
{
reading_dependents = true;
reading_dependencies = false;
continue;
}
else if (line.empty())
{
continue;
}
UUID id = parse<UUID>(line.substr(0, line.find('\t')));
line.clear();
size_t separator1 = line.find('\t');
size_t separator2 = line.find('\t', separator1 + 1);
if ((separator1 == String::npos) || (separator2 == String::npos))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Separators not found in line {}", line);
String queries;
while (!buf->eof())
UUID id = parse<UUID>(line.substr(0, separator1));
AccessEntityType type = AccessEntityTypeInfo::parseType(line.substr(separator1 + 1, separator2 - separator1 - 1));
String name = line.substr(separator2 + 1);
if (reading_dependencies)
{
String query;
readStringUntilNewlineInto(query, *buf);
buf->ignore();
if (query.empty())
break;
if (!queries.empty())
queries.append("\n");
queries.append(query);
res.dependencies.emplace(id, std::pair{name, type});
}
AccessEntityPtr entity = deserializeAccessEntity(queries);
res.entities.emplace(id, entity);
}
if (dependencies_found)
{
while (!buf->eof())
else
{
String id_as_string;
readStringInto(id_as_string, *buf);
buf->ignore();
UUID id = parse<UUID>(id_as_string);
String queries;
while (!buf->eof())
{
String query;
readStringUntilNewlineInto(query, *buf);
buf->ignore();
if (query.empty())
break;
if (!queries.empty())
queries.append("\n");
queries.append(query);
}
String type_as_string;
readStringInto(type_as_string, *buf);
buf->ignore();
AccessEntityType type = AccessEntityTypeInfo::parseType(type_as_string);
AccessEntityPtr entity = deserializeAccessEntity(queries);
String name;
readStringInto(name, *buf);
buf->ignore();
if (name != entity->getName())
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Unexpected name {} is specified for {}", name, entity->formatTypeWithName());
if (type != entity->getType())
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Unexpected type {} is specified for {}", AccessEntityTypeInfo::get(type).name, entity->formatTypeWithName());
if (!res.entities.contains(id))
res.dependencies.emplace(id, std::pair{name, type});
if (reading_dependents)
res.dependents.emplace(id, entity);
else
res.entities.emplace(id, entity);
}
}
@ -146,190 +178,59 @@ namespace
}
}
};
std::vector<UUID> findDependencies(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities)
{
std::vector<UUID> res;
for (const auto & entity : entities | boost::adaptors::map_values)
insertAtEnd(res, entity->findDependencies());
/// Remove duplicates in the list of dependencies (some entities can refer to other entities).
::sort(res.begin(), res.end());
res.erase(std::unique(res.begin(), res.end()), res.end());
for (const auto & id : entities | boost::adaptors::map_keys)
{
auto it = std::lower_bound(res.begin(), res.end(), id);
if ((it != res.end()) && (*it == id))
res.erase(it);
}
return res;
}
std::unordered_map<UUID, std::pair<String, AccessEntityType>> readDependenciesNamesAndTypes(const std::vector<UUID> & dependencies, const AccessControl & access_control)
{
std::unordered_map<UUID, std::pair<String, AccessEntityType>> res;
for (const auto & id : dependencies)
{
if (auto name_and_type = access_control.tryReadNameWithType(id))
res.emplace(id, name_and_type.value());
}
return res;
}
/// Checks if new entities (which we're going to restore) already exist,
/// and either skips them or throws an exception depending on the restore settings.
void checkExistingEntities(std::vector<std::pair<UUID, AccessEntityPtr>> & entities,
std::unordered_map<UUID, UUID> & old_to_new_id,
const AccessControl & access_control,
RestoreAccessCreationMode creation_mode)
{
if (creation_mode == RestoreAccessCreationMode::kReplace)
return;
auto should_skip = [&](const std::pair<UUID, AccessEntityPtr> & id_and_entity)
{
const auto & id = id_and_entity.first;
const auto & entity = *id_and_entity.second;
auto existing_id = access_control.find(entity.getType(), entity.getName());
if (!existing_id)
{
return false;
}
else if (creation_mode == RestoreAccessCreationMode::kCreateIfNotExists)
{
old_to_new_id[id] = *existing_id;
return true;
}
else
{
throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Cannot restore {} because it already exists", entity.formatTypeWithName());
}
};
std::erase_if(entities, should_skip);
}
/// If new entities (which we're going to restore) depend on other entities which are not going to be restored or not present in the backup
/// then we should try to replace those dependencies with already existing entities.
void resolveDependencies(const std::unordered_map<UUID, std::pair<String, AccessEntityType>> & dependencies,
std::unordered_map<UUID, UUID> & old_to_new_ids,
const AccessControl & access_control,
bool allow_unresolved_dependencies)
{
for (const auto & [id, name_and_type] : dependencies)
{
std::optional<UUID> new_id;
if (allow_unresolved_dependencies)
new_id = access_control.find(name_and_type.second, name_and_type.first);
else
new_id = access_control.getID(name_and_type.second, name_and_type.first);
if (new_id)
old_to_new_ids.emplace(id, *new_id);
}
}
/// Generates random IDs for the new entities.
void generateRandomIDs(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, std::unordered_map<UUID, UUID> & old_to_new_ids)
{
Poco::UUIDGenerator generator;
for (auto & [id, entity] : entities)
{
UUID new_id;
generator.createRandom().copyTo(reinterpret_cast<char *>(&new_id));
old_to_new_ids.emplace(id, new_id);
id = new_id;
}
}
/// Updates dependencies of the new entities using a specified map.
void replaceDependencies(std::vector<std::pair<UUID, AccessEntityPtr>> & entities,
const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
for (auto & entity : entities | boost::adaptors::map_values)
IAccessEntity::replaceDependencies(entity, old_to_new_ids);
}
AccessRightsElements getRequiredAccessToRestore(const std::vector<std::pair<UUID, AccessEntityPtr>> & entities)
{
AccessRightsElements res;
for (const auto & entity : entities | boost::adaptors::map_values)
{
auto entity_type = entity->getType();
switch (entity_type)
{
case User::TYPE:
{
const auto & user = typeid_cast<const User &>(*entity);
res.emplace_back(AccessType::CREATE_USER);
auto elements = user.access.getElements();
for (auto & element : elements)
{
if (element.is_partial_revoke)
continue;
element.grant_option = true;
res.emplace_back(element);
}
if (!user.granted_roles.isEmpty())
res.emplace_back(AccessType::ROLE_ADMIN);
break;
}
case Role::TYPE:
{
const auto & role = typeid_cast<const Role &>(*entity);
res.emplace_back(AccessType::CREATE_ROLE);
auto elements = role.access.getElements();
for (auto & element : elements)
{
if (element.is_partial_revoke)
continue;
element.grant_option = true;
res.emplace_back(element);
}
if (!role.granted_roles.isEmpty())
res.emplace_back(AccessType::ROLE_ADMIN);
break;
}
case SettingsProfile::TYPE:
{
res.emplace_back(AccessType::CREATE_SETTINGS_PROFILE);
break;
}
case RowPolicy::TYPE:
{
const auto & policy = typeid_cast<const RowPolicy &>(*entity);
res.emplace_back(AccessType::CREATE_ROW_POLICY, policy.getDatabase(), policy.getTableName());
break;
}
case Quota::TYPE:
{
res.emplace_back(AccessType::CREATE_QUOTA);
break;
}
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown type: {}", toString(entity_type));
}
}
return res;
}
}
std::pair<String, BackupEntryPtr> makeBackupEntryForAccess(
const std::vector<std::pair<UUID, AccessEntityPtr>> & access_entities,
const String & data_path_in_backup,
size_t counter,
const AccessControl & access_control)
std::pair<String, BackupEntryPtr> makeBackupEntryForAccessEntities(
const std::vector<UUID> & entities_ids,
const std::unordered_map<UUID, AccessEntityPtr> & all_entities,
bool write_dependents,
const String & data_path_in_backup)
{
auto dependencies = readDependenciesNamesAndTypes(findDependencies(access_entities), access_control);
AccessEntitiesInBackup ab;
boost::range::copy(access_entities, std::inserter(ab.entities, ab.entities.end()));
ab.dependencies = std::move(dependencies);
String filename = fmt::format("access{:02}.txt", counter + 1); /// access01.txt, access02.txt, ...
std::unordered_set<UUID> entities_ids_set;
for (const auto & id : entities_ids)
entities_ids_set.emplace(id);
for (const auto & id : entities_ids)
{
auto it = all_entities.find(id);
if (it != all_entities.end())
{
AccessEntityPtr entity = it->second;
ab.entities.emplace(id, entity);
auto dependencies = entity->findDependencies();
for (const auto & dependency_id : dependencies)
{
if (!entities_ids_set.contains(dependency_id))
{
auto it_dependency = all_entities.find(dependency_id);
if (it_dependency != all_entities.end())
{
auto dependency_entity = it_dependency->second;
ab.dependencies.emplace(dependency_id, std::make_pair(dependency_entity->getName(), dependency_entity->getType()));
}
}
}
}
}
if (write_dependents)
{
for (const auto & [id, possible_dependent] : all_entities)
{
if (!entities_ids_set.contains(id) && possible_dependent->hasDependencies(entities_ids_set))
{
auto dependent = possible_dependent->clone();
dependent->clearAllExceptDependencies();
ab.dependents.emplace(id, dependent);
}
}
}
String filename = fmt::format("access-{}.txt", UUIDHelpers::generateV4());
String file_path_in_backup = fs::path{data_path_in_backup} / filename;
return {file_path_in_backup, ab.toBackupEntry()};
}
@ -339,61 +240,411 @@ AccessRestorerFromBackup::AccessRestorerFromBackup(
const BackupPtr & backup_, const RestoreSettings & restore_settings_)
: backup(backup_)
, creation_mode(restore_settings_.create_access)
, allow_unresolved_dependencies(restore_settings_.allow_unresolved_access_dependencies)
, skip_unresolved_dependencies(restore_settings_.skip_unresolved_access_dependencies)
, update_dependents(restore_settings_.update_access_entities_dependents)
, log(getLogger("AccessRestorerFromBackup"))
{
}
AccessRestorerFromBackup::~AccessRestorerFromBackup() = default;
void AccessRestorerFromBackup::addDataPath(const String & data_path)
void AccessRestorerFromBackup::addDataPath(const String & data_path_in_backup)
{
if (!data_paths.emplace(data_path).second)
return;
if (loaded)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Access entities already loaded");
fs::path data_path_in_backup_fs = data_path;
Strings filenames = backup->listFiles(data_path, /*recursive*/ false);
if (filenames.empty())
return;
for (const String & filename : filenames)
{
if (!filename.starts_with("access") || !filename.ends_with(".txt"))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File name {} doesn't match the wildcard \"access*.txt\"",
String{data_path_in_backup_fs / filename});
}
::sort(filenames.begin(), filenames.end());
for (const String & filename : filenames)
{
String filepath_in_backup = data_path_in_backup_fs / filename;
auto read_buffer_from_backup = backup->readFile(filepath_in_backup);
auto ab = AccessEntitiesInBackup::fromBackupEntry(std::move(read_buffer_from_backup), filepath_in_backup);
boost::range::copy(ab.entities, std::back_inserter(entities));
boost::range::copy(ab.dependencies, std::inserter(dependencies, dependencies.end()));
}
for (const auto & id : entities | boost::adaptors::map_keys)
dependencies.erase(id);
if (std::find(data_paths_in_backup.begin(), data_paths_in_backup.end(), data_path_in_backup) == data_paths_in_backup.end())
data_paths_in_backup.emplace_back(data_path_in_backup);
}
void AccessRestorerFromBackup::loadFromBackup()
{
if (loaded)
return;
/// Parse files "access*.txt" found in the added data paths in the backup.
for (size_t data_path_index = 0; data_path_index != data_paths_in_backup.size(); ++data_path_index)
{
const String & data_path_in_backup = data_paths_in_backup[data_path_index];
fs::path data_path_in_backup_fs = data_path_in_backup;
Strings filenames = backup->listFiles(data_path_in_backup_fs, /*recursive*/ false);
if (filenames.empty())
continue;
for (const String & filename : filenames)
{
if (!filename.starts_with("access") || !filename.ends_with(".txt"))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File name {} doesn't match the wildcard \"access*.txt\"",
String{data_path_in_backup_fs / filename});
}
for (const String & filename : filenames)
{
String filepath_in_backup = data_path_in_backup_fs / filename;
AccessEntitiesInBackup ab;
try
{
auto read_buffer_from_backup = backup->readFile(filepath_in_backup);
ab = AccessEntitiesInBackup::fromBackupEntry(std::move(read_buffer_from_backup), filepath_in_backup);
}
catch (Exception & e)
{
e.addMessage("While reading access entities from {} in backup", filepath_in_backup);
throw;
}
for (const auto & [id, entity] : ab.entities)
{
auto it = entity_infos.find(id);
if (it == entity_infos.end())
{
it = entity_infos.emplace(id, EntityInfo{.id = id, .name = entity->getName(), .type = entity->getType()}).first;
}
EntityInfo & entity_info = it->second;
entity_info.entity = entity;
entity_info.restore = true;
entity_info.data_path_index = data_path_index;
}
for (const auto & [id, name_and_type] : ab.dependencies)
{
auto it = entity_infos.find(id);
if (it == entity_infos.end())
{
it = entity_infos.emplace(id, EntityInfo{.id = id, .name = name_and_type.first, .type = name_and_type.second}).first;
}
EntityInfo & entity_info = it->second;
entity_info.is_dependency = true;
}
for (const auto & [id, entity] : ab.dependents)
{
auto it = entity_infos.find(id);
if (it == entity_infos.end())
{
it = entity_infos.emplace(id, EntityInfo{.id = id, .name = entity->getName(), .type = entity->getType()}).first;
}
EntityInfo & entity_info = it->second;
if (!entity_info.restore)
entity_info.entity = entity;
}
}
}
loaded = true;
}
AccessRightsElements AccessRestorerFromBackup::getRequiredAccess() const
{
return getRequiredAccessToRestore(entities);
if (!loaded)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Access entities not loaded");
AccessRightsElements res;
for (const auto & [id, entity_info] : entity_infos)
{
if (!entity_info.restore)
continue;
const auto & entity = entity_info.entity;
auto entity_type = entity->getType();
switch (entity_type)
{
case User::TYPE:
{
const auto & user = typeid_cast<const User &>(*entity);
res.emplace_back(AccessType::CREATE_USER);
auto elements = user.access.getElements();
for (auto & element : elements)
{
if (element.is_partial_revoke)
continue;
element.grant_option = true;
res.emplace_back(element);
}
if (!user.granted_roles.isEmpty())
res.emplace_back(AccessType::ROLE_ADMIN);
break;
}
case Role::TYPE:
{
const auto & role = typeid_cast<const Role &>(*entity);
res.emplace_back(AccessType::CREATE_ROLE);
auto elements = role.access.getElements();
for (auto & element : elements)
{
if (element.is_partial_revoke)
continue;
element.grant_option = true;
res.emplace_back(element);
}
if (!role.granted_roles.isEmpty())
res.emplace_back(AccessType::ROLE_ADMIN);
break;
}
case SettingsProfile::TYPE:
{
res.emplace_back(AccessType::CREATE_SETTINGS_PROFILE);
break;
}
case RowPolicy::TYPE:
{
const auto & policy = typeid_cast<const RowPolicy &>(*entity);
res.emplace_back(AccessType::CREATE_ROW_POLICY, policy.getDatabase(), policy.getTableName());
break;
}
case Quota::TYPE:
{
res.emplace_back(AccessType::CREATE_QUOTA);
break;
}
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown type: {}", toString(entity_type));
}
}
return res;
}
std::vector<std::pair<UUID, AccessEntityPtr>> AccessRestorerFromBackup::getAccessEntities(const AccessControl & access_control) const
void AccessRestorerFromBackup::generateRandomIDsAndResolveDependencies(const AccessControl & access_control)
{
auto new_entities = entities;
if (ids_assigned)
return;
if (!loaded)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Access entities not loaded");
/// Calculate `new_id` for each entity info.
/// Check which ones of the loaded access entities already exist.
/// Generate random UUIDs for access entities which we're going to restore if they don't exist.
for (auto & [id, entity_info] : entity_infos)
{
const String & name = entity_info.name;
auto type = entity_info.type;
if (entity_info.restore && (creation_mode == RestoreAccessCreationMode::kReplace))
{
entity_info.new_id = UUIDHelpers::generateV4();
LOG_TRACE(log, "{}: Generated new UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), *entity_info.new_id);
continue;
}
if (auto existing_id = access_control.find(type, name))
{
if (entity_info.restore && (creation_mode == RestoreAccessCreationMode::kCreate))
{
throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Cannot restore {} because it already exists",
AccessEntityTypeInfo::get(type).formatEntityNameWithType(name));
}
bool was_going_to_restore = entity_info.restore;
entity_info.new_id = *existing_id;
entity_info.restore = false;
LOG_TRACE(log, "{}: Found with UUID {}{}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), *existing_id,
(was_going_to_restore ? ", will not restore" : ""));
}
else
{
if (entity_info.restore)
{
entity_info.new_id = UUIDHelpers::generateV4();
LOG_TRACE(log, "{}: Generated new UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), *entity_info.new_id);
}
else if (skip_unresolved_dependencies)
{
LOG_TRACE(log, "{}: Not found, ignoring", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name));
}
else
{
throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "Cannot resolve {} while restoring from backup",
AccessEntityTypeInfo::get(type).formatEntityNameWithType(name));
}
}
}
/// Prepare map from old UUIDs to new UUIDs.
std::unordered_set<UUID> ids_to_restore;
std::unordered_map<UUID, UUID> old_to_new_ids;
checkExistingEntities(new_entities, old_to_new_ids, access_control, creation_mode);
resolveDependencies(dependencies, old_to_new_ids, access_control, allow_unresolved_dependencies);
generateRandomIDs(new_entities, old_to_new_ids);
replaceDependencies(new_entities, old_to_new_ids);
std::unordered_set<UUID> unresolved_ids;
return new_entities;
for (const auto & [id, entity_info] : entity_infos)
{
if (entity_info.restore)
ids_to_restore.insert(id);
if (entity_info.new_id)
old_to_new_ids[id] = *entity_info.new_id;
else
unresolved_ids.insert(id);
}
/// Calculate `is_dependent` for each entity info.
if (update_dependents)
{
for (auto & [id, entity_info] : entity_infos)
{
if (!entity_info.restore && entity_info.new_id && entity_info.entity && entity_info.entity->hasDependencies(ids_to_restore))
entity_info.is_dependent = true;
}
}
/// Remap the UUIDs of dependencies in the access entities we're going to restore.
for (auto & [id, entity_info] : entity_infos)
{
if (entity_info.restore || entity_info.is_dependent)
{
auto new_entity = entity_info.entity->clone();
new_entity->replaceDependencies(old_to_new_ids);
new_entity->removeDependencies(unresolved_ids);
entity_info.entity = new_entity;
}
if (entity_info.restore && data_path_with_entities_to_restore.empty())
data_path_with_entities_to_restore = data_paths_in_backup[entity_info.data_path_index];
}
ids_assigned = true;
}
AccessEntitiesToRestore AccessRestorerFromBackup::getEntitiesToRestore(const String & data_path_in_backup) const
{
if (!ids_assigned)
throw Exception(ErrorCodes::LOGICAL_ERROR, "IDs not assigned");
if (data_path_in_backup != data_path_with_entities_to_restore)
return {};
AccessEntitiesToRestore res;
res.new_entities.reserve(entity_infos.size());
res.dependents.reserve(entity_infos.size());
for (const auto & [id, entity_info] : entity_infos)
{
if (entity_info.restore)
res.new_entities.emplace_back(*entity_info.new_id, entity_info.entity);
if (entity_info.is_dependent)
res.dependents.emplace_back(AccessEntitiesToRestore::Dependent{*entity_info.new_id, entity_info.entity});
}
return res;
}
void restoreAccessEntitiesFromBackup(
IAccessStorage & destination_access_storage,
const AccessEntitiesToRestore & entities_to_restore,
const RestoreSettings & restore_settings)
{
if (entities_to_restore.new_entities.empty())
return; /// Nothing to restore.
auto log = getLogger("AccessRestorerFromBackup");
bool replace_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kReplace);
bool throw_if_exists = (restore_settings.create_access == RestoreAccessCreationMode::kCreate);
bool update_dependents = restore_settings.update_access_entities_dependents;
std::unordered_set<UUID> restored_ids;
std::unordered_map<UUID, UUID> new_to_existing_ids;
AccessEntitiesToRestore::Dependents additional_dependents;
additional_dependents.reserve(entities_to_restore.new_entities.size());
for (const auto & [id, entity] : entities_to_restore.new_entities)
{
const String & name = entity->getName();
auto type = entity->getType();
LOG_TRACE(log, "{}: Adding with UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), id);
UUID existing_id;
if (destination_access_storage.insert(id, entity, replace_if_exists, throw_if_exists, &existing_id))
{
LOG_TRACE(log, "{}: Added successfully", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name));
restored_ids.emplace(id);
}
else
{
/// Couldn't insert `entity` because there is an existing entity with the same name.
LOG_TRACE(log, "{}: Not added because already exists with UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), existing_id);
new_to_existing_ids[id] = existing_id;
if (update_dependents)
additional_dependents.emplace_back(AccessEntitiesToRestore::Dependent{existing_id, entity});
}
}
if (!new_to_existing_ids.empty())
{
std::vector<UUID> ids_to_update;
ids_to_update.reserve(restored_ids.size());
boost::copy(restored_ids, std::inserter(ids_to_update, ids_to_update.end()));
std::unordered_set<UUID> new_ids;
boost::copy(new_to_existing_ids | boost::adaptors::map_keys, std::inserter(new_ids, new_ids.end()));
/// If new entities restored from backup have dependencies on other entities from backup which were not restored because they existed,
/// then we should correct those dependencies.
auto update_func = [&](const AccessEntityPtr & entity, const UUID &) -> AccessEntityPtr
{
if (!entity->hasDependencies(new_ids))
return entity;
LOG_TRACE(log, "{}: Updating dependencies", entity->formatTypeWithName());
auto res = entity->clone();
res->replaceDependencies(new_to_existing_ids);
return res;
};
/// It's totally ok if some UUIDs from `ids_to_update` don't exist anymore, that's why we use tryUpdate() here.
destination_access_storage.tryUpdate(ids_to_update, update_func);
}
auto do_update_dependents = [&](const AccessEntitiesToRestore::Dependents & dependents)
{
if (dependents.empty())
return;
std::vector<UUID> ids_to_update;
ids_to_update.reserve(dependents.size());
std::unordered_map<UUID, AccessEntityPtr> id_to_source;
for (const auto & dependent : dependents)
{
const UUID & id = dependent.existing_id;
if (!destination_access_storage.isReadOnly(id))
{
ids_to_update.emplace_back(id);
auto modified_source = dependent.source->clone();
modified_source->replaceDependencies(new_to_existing_ids);
id_to_source[id] = modified_source;
}
}
/// If new entities restored from backup have dependencies on other entities from backup which were not restored because they existed,
/// then we should correct those dependencies.
auto update_func = [&](const AccessEntityPtr & entity, const UUID & existing_id) -> AccessEntityPtr
{
const auto & source = *id_to_source.at(existing_id);
if (!source.hasDependencies(restored_ids))
return entity;
LOG_TRACE(log, "{}: Updating dependent", entity->formatTypeWithName());
auto res = entity->clone();
res->copyDependenciesFrom(source, restored_ids);
return res;
};
/// It's totally ok if some UUIDs from `ids_to_update` don't exist anymore, that's why we use tryUpdate() here.
destination_access_storage.tryUpdate(ids_to_update, update_func);
};
do_update_dependents(entities_to_restore.dependents);
do_update_dependents(additional_dependents);
}
}

View File

@ -1,8 +1,8 @@
#pragma once
#include <Common/Logger.h>
#include <Core/UUID.h>
#include <unordered_map>
#include <unordered_set>
namespace DB
@ -12,6 +12,7 @@ enum class AccessEntityType : uint8_t;
struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
class AccessRightsElements;
class IAccessStorage;
class IBackup;
using BackupPtr = std::shared_ptr<const IBackup>;
class IBackupEntry;
@ -20,15 +21,42 @@ struct RestoreSettings;
enum class RestoreAccessCreationMode : uint8_t;
/// Makes a backup of access entities of a specified type.
std::pair<String, BackupEntryPtr> makeBackupEntryForAccess(
const std::vector<std::pair<UUID, AccessEntityPtr>> & access_entities,
const String & data_path_in_backup,
size_t counter,
const AccessControl & access_control);
/// Makes a backup entry for of a set of access entities.
std::pair<String, BackupEntryPtr> makeBackupEntryForAccessEntities(
const std::vector<UUID> & entities_ids,
const std::unordered_map<UUID, AccessEntityPtr> & all_entities,
bool write_dependents,
const String & data_path_in_backup);
struct AccessEntitiesToRestore
{
/// Access entities loaded from backup with new randomly generated UUIDs.
std::vector<std::pair<UUID /* new_id */, AccessEntityPtr /* new_entity */>> new_entities;
/// Dependents are access entities which exist already and they should be updated after restoring.
/// For example, if there were a role granted to a user: `CREATE USER user1; CREATE ROLE role1; GRANT role1 TO user1`,
/// and we're restoring only role `role1` because user `user1` already exists,
/// then user `user1` should be modified after restoring role `role1` to add this grant `GRANT role1 TO user1`.
struct Dependent
{
/// UUID of an existing access entities.
UUID existing_id;
/// Source access entity from backup to copy dependencies from.
AccessEntityPtr source;
};
using Dependents = std::vector<Dependent>;
Dependents dependents;
};
/// Restores access entities from a backup.
void restoreAccessEntitiesFromBackup(
IAccessStorage & access_storage,
const AccessEntitiesToRestore & entities_to_restore,
const RestoreSettings & restore_settings);
/// Loads access entities from a backup and prepares them for insertion into an access storage.
class AccessRestorerFromBackup
{
public:
@ -36,21 +64,75 @@ public:
~AccessRestorerFromBackup();
/// Adds a data path to loads access entities from.
void addDataPath(const String & data_path);
void addDataPath(const String & data_path_in_backup);
/// Loads access entities from the backup.
void loadFromBackup();
/// Checks that the current user can do restoring.
/// Function loadFromBackup() must be called before that.
AccessRightsElements getRequiredAccess() const;
/// Inserts all access entities loaded from all the paths added by addDataPath().
std::vector<std::pair<UUID, AccessEntityPtr>> getAccessEntities(const AccessControl & access_control) const;
/// Generates random IDs for access entities we're restoring to insert them into an access storage;
/// and finds IDs of existing access entities which are used as dependencies.
void generateRandomIDsAndResolveDependencies(const AccessControl & access_control);
/// Returns access entities loaded from backup and prepared for insertion into an access storage.
/// Both functions loadFromBackup() and generateRandomIDsAndResolveDependencies() must be called before that.
AccessEntitiesToRestore getEntitiesToRestore(const String & data_path_in_backup) const;
private:
BackupPtr backup;
RestoreAccessCreationMode creation_mode;
bool allow_unresolved_dependencies = false;
std::vector<std::pair<UUID, AccessEntityPtr>> entities;
std::unordered_map<UUID, std::pair<String, AccessEntityType>> dependencies;
std::unordered_set<String> data_paths;
const BackupPtr backup;
const RestoreAccessCreationMode creation_mode;
const bool skip_unresolved_dependencies;
const bool update_dependents;
const LoggerPtr log;
/// Whether loadFromBackup() finished.
bool loaded = false;
/// Whether generateRandomIDsAndResolveDependencies() finished.
bool ids_assigned = false;
Strings data_paths_in_backup;
String data_path_with_entities_to_restore;
/// Information about an access entity loaded from the backup.
struct EntityInfo
{
UUID id;
String name;
AccessEntityType type;
AccessEntityPtr entity = nullptr; /// Can be nullptr if `restore=false`.
/// Index in `data_paths_in_backup`.
size_t data_path_index = 0;
/// Whether we're going to restore this entity.
/// For example,
/// in case of `RESTORE TABLE system.roles` this flag is true for all the roles loaded from the backup, and
/// in case of `RESTORE ALL` this flag is always true.
bool restore = false;
/// Whether this entity info was added as a dependency of another entity which we're going to restore.
/// For example, if we're going to restore the following user: `CREATE USER user1 DEFAULT ROLE role1, role2 SETTINGS PROFILE profile1, profile2`
/// then `restore=true` for `user1` and `is_dependency=true` for `role1`, `role2`, `profile1`, `profile2`.
/// Flags `restore` and `is_dependency` both can be set at the same time.
bool is_dependency = false;
/// Whether this entity info is a dependent of another entity which we're going to restore.
/// For example, if we're going to restore role `role1` and there is also the following user stored in the backup:
/// `CREATE USER user1 DEFAULT ROLE role1`, then `is_dependent=true` for `user1`.
/// This flags is set by generateRandomIDsAndResolveDependencies().
bool is_dependent = false;
/// New UUID for this entity - either randomly generated or copied from an existing entity.
/// This UUID is assigned by generateRandomIDsAndResolveDependencies().
std::optional<UUID> new_id = std::nullopt;
};
std::unordered_map<UUID, EntityInfo> entity_infos;
};
}

View File

@ -629,9 +629,9 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po
}
}
void AccessControl::restoreFromBackup(RestorerFromBackup & restorer)
void AccessControl::restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup)
{
MultipleAccessStorage::restoreFromBackup(restorer);
MultipleAccessStorage::restoreFromBackup(restorer, data_path_in_backup);
changes_notifier->sendNotifications();
}

View File

@ -124,7 +124,7 @@ public:
AuthResult authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address, const String & forwarded_address) const;
/// Makes a backup of access entities.
void restoreFromBackup(RestorerFromBackup & restorer) override;
void restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup) override;
void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config);

View File

@ -594,6 +594,14 @@ public:
optimizeTree();
}
void makeDifference(const Node & other)
{
Node rhs = other;
makeDifferenceRec(*this, rhs);
flags -= other.flags;
optimizeTree();
}
ProtoElements getElements() const
{
ProtoElements res;
@ -1109,6 +1117,32 @@ private:
result.wildcard_grant = wildcard_grant || rhs.wildcard_grant;
}
void makeDifferenceRec(Node & result, Node & rhs)
{
if (rhs.children)
{
for (auto & rhs_child : *rhs.children)
{
auto & result_child = result.getLeaf(rhs_child.node_name, rhs_child.level, !rhs_child.isLeaf());
auto & lhs_child = getLeaf(rhs_child.node_name, rhs_child.level, !rhs_child.isLeaf());
lhs_child.makeDifferenceRec(result_child, rhs_child);
}
}
if (children)
{
for (auto & lhs_child : *children)
{
auto & result_child = result.getLeaf(lhs_child.node_name, lhs_child.level, !lhs_child.isLeaf());
auto & rhs_child = rhs.getLeaf(lhs_child.node_name, lhs_child.level, !lhs_child.isLeaf());
lhs_child.makeDifferenceRec(result_child, rhs_child);
}
}
result.flags = flags - rhs.flags;
result.wildcard_grant = wildcard_grant || rhs.wildcard_grant;
}
void modifyFlagsRec(const ModifyFlagsFunction & function, bool grant_option, bool & flags_added, bool & flags_removed)
{
if (children)
@ -1581,6 +1615,22 @@ void AccessRights::makeIntersection(const AccessRights & other)
}
void AccessRights::makeDifference(const AccessRights & other)
{
auto helper = [](std::unique_ptr<Node> & root_node, const std::unique_ptr<Node> & other_root_node)
{
if (!root_node || !other_root_node)
return;
root_node->makeDifference(*other_root_node);
if (!root_node->flags && !root_node->children)
root_node = nullptr;
};
helper(root, other.root);
helper(root_with_grant_option, other.root_with_grant_option);
}
void AccessRights::modifyFlags(const ModifyFlagsFunction & function)
{
if (!root)

View File

@ -150,6 +150,9 @@ public:
/// Makes an intersection of access rights.
void makeIntersection(const AccessRights & other);
/// Makes a difference (relative complement) of access rights.
void makeDifference(const AccessRights & other);
/// Traverse the tree and modify each access flags.
using ModifyFlagsFunction = std::function<AccessFlags(
const AccessFlags & flags,

View File

@ -233,11 +233,6 @@ void AccessRightsElement::replaceEmptyDatabase(const String & current_database)
String AccessRightsElement::toString() const { return toStringImpl(*this, true); }
String AccessRightsElement::toStringWithoutOptions() const { return toStringImpl(*this, false); }
String AccessRightsElement::toStringForAccessTypeSource() const
{
String result{access_flags.toKeywords().front()};
return result + " ON *.*";
}
bool AccessRightsElements::empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); }

View File

@ -92,7 +92,6 @@ struct AccessRightsElement
/// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table".
String toString() const;
String toStringWithoutOptions() const;
String toStringForAccessTypeSource() const;
void formatColumnNames(WriteBuffer & buffer) const;
void formatONClause(WriteBuffer & buffer, bool hilite = false) const;

View File

@ -645,9 +645,25 @@ bool ContextAccess::checkAccessImplHelper(const ContextPtr & context, AccessFlag
AccessRightsElement{access_flags, fmt_args...}.toStringWithoutOptions());
}
AccessRights difference;
difference.grant(flags, fmt_args...);
AccessRights original_rights = difference;
difference.makeDifference(*getAccessRights());
if (difference == original_rights)
{
return access_denied(ErrorCodes::ACCESS_DENIED,
"{}: Not enough privileges. To execute this query, it's necessary to have the grant {}",
AccessRightsElement{access_flags, fmt_args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : ""));
}
return access_denied(ErrorCodes::ACCESS_DENIED,
"{}: Not enough privileges. To execute this query, it's necessary to have the grant {}",
AccessRightsElement{access_flags, fmt_args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : ""));
"{}: Not enough privileges. To execute this query, it's necessary to have the grant {}. "
"(Missing permissions: {}){}",
AccessRightsElement{access_flags, fmt_args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : ""),
difference.getElements().toStringWithoutOptions(),
grant_option ? ". You can try to use the `GRANT CURRENT GRANTS(...)` statement" : "");
};
/// As we check the SOURCES from the Table Engine logic, direct prompt about Table Engine would be misleading
@ -671,18 +687,7 @@ bool ContextAccess::checkAccessImplHelper(const ContextPtr & context, AccessFlag
if (new_flags.isEmpty())
return access_denied_no_grant(flags, args...);
if (grant_option && acs->isGranted(flags, args...))
{
return access_denied(ErrorCodes::ACCESS_DENIED,
"{}: Not enough privileges. "
"The required privileges have been granted, but without grant option. "
"To execute this query, it's necessary to have the grant {} WITH GRANT OPTION",
AccessRightsElement{new_flags}.toStringForAccessTypeSource());
}
return access_denied(ErrorCodes::ACCESS_DENIED,
"{}: Not enough privileges. To execute this query, it's necessary to have the grant {}",
AccessRightsElement{new_flags}.toStringForAccessTypeSource() + (grant_option ? " WITH GRANT OPTION" : ""));
return access_denied_no_grant(new_flags);
}
return access_denied_no_grant(flags, args...);

View File

@ -676,7 +676,7 @@ bool DiskAccessStorage::updateNoLock(const UUID & id, const UpdateFunc & update_
if (!entry.entity)
entry.entity = readAccessEntityFromDisk(id);
auto old_entity = entry.entity;
auto new_entity = update_func(old_entity);
auto new_entity = update_func(old_entity, id);
if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());

View File

@ -176,6 +176,16 @@ std::vector<UUID> GrantedRoles::findDependencies() const
return res;
}
bool GrantedRoles::hasDependencies(const std::unordered_set<UUID> & ids) const
{
for (const auto & role_id : roles)
{
if (ids.contains(role_id))
return true;
}
return false;
}
void GrantedRoles::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
std::vector<UUID> new_ids;
@ -221,4 +231,56 @@ void GrantedRoles::replaceDependencies(const std::unordered_map<UUID, UUID> & ol
}
}
void GrantedRoles::copyDependenciesFrom(const GrantedRoles & src, const std::unordered_set<UUID> & ids)
{
bool found = false;
for (const auto & role_id : src.roles)
{
if (ids.contains(role_id))
{
roles.emplace(role_id);
found = true;
}
}
if (found)
{
for (const auto & role_id : src.roles_with_admin_option)
{
if (ids.contains(role_id))
roles_with_admin_option.emplace(role_id);
}
}
}
void GrantedRoles::removeDependencies(const std::unordered_set<UUID> & ids)
{
bool found = false;
for (auto it = roles.begin(); it != roles.end();)
{
if (ids.contains(*it))
{
it = roles.erase(it);
found = true;
}
else
{
++it;
}
}
if (found)
{
for (auto it = roles_with_admin_option.begin(); it != roles_with_admin_option.end();)
{
if (ids.contains(*it))
it = roles_with_admin_option.erase(it);
else
++it;
}
}
}
}

View File

@ -58,7 +58,10 @@ public:
friend bool operator !=(const GrantedRoles & left, const GrantedRoles & right) { return !(left == right); }
std::vector<UUID> findDependencies() const;
bool hasDependencies(const std::unordered_set<UUID> & ids) const;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids);
void copyDependenciesFrom(const GrantedRoles & src, const std::unordered_set<UUID> & ids);
void removeDependencies(const std::unordered_set<UUID> & ids);
private:
boost::container::flat_set<UUID> roles;

View File

@ -9,28 +9,4 @@ bool IAccessEntity::equal(const IAccessEntity & other) const
return (name == other.name) && (getType() == other.getType());
}
void IAccessEntity::replaceDependencies(std::shared_ptr<const IAccessEntity> & entity, const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
if (old_to_new_ids.empty())
return;
bool need_replace_dependencies = false;
auto dependencies = entity->findDependencies();
for (const auto & dependency : dependencies)
{
if (old_to_new_ids.contains(dependency))
{
need_replace_dependencies = true;
break;
}
}
if (!need_replace_dependencies)
return;
auto new_entity = entity->clone();
new_entity->replaceDependencies(old_to_new_ids);
entity = new_entity;
}
}

View File

@ -48,10 +48,13 @@ struct IAccessEntity
/// Finds all dependencies.
virtual std::vector<UUID> findDependencies() const { return {}; }
virtual bool hasDependencies(const std::unordered_set<UUID> & /* ids */) const { return false; }
/// Replaces dependencies according to a specified map.
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) { doReplaceDependencies(old_to_new_ids); }
static void replaceDependencies(std::shared_ptr<const IAccessEntity> & entity, const std::unordered_map<UUID, UUID> & old_to_new_ids);
virtual void replaceDependencies(const std::unordered_map<UUID, UUID> & /* old_to_new_ids */) {}
virtual void copyDependenciesFrom(const IAccessEntity & /* src */, const std::unordered_set<UUID> & /* ids */) {}
virtual void removeDependencies(const std::unordered_set<UUID> & /* ids */) {}
virtual void clearAllExceptDependencies() {}
/// Whether this access entity should be written to a backup.
virtual bool isBackupAllowed() const { return false; }
@ -67,8 +70,6 @@ protected:
{
return std::make_shared<EntityClassT>(typeid_cast<const EntityClassT &>(*this));
}
virtual void doReplaceDependencies(const std::unordered_map<UUID, UUID> & /* old_to_new_ids */) {}
};
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;

View File

@ -4,8 +4,10 @@
#include <Access/User.h>
#include <Access/AccessBackup.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/RestorerFromBackup.h>
#include <Backups/IBackupCoordination.h>
#include <Backups/IRestoreCoordination.h>
#include <Backups/RestoreSettings.h>
#include <Backups/RestorerFromBackup.h>
#include <Common/Exception.h>
#include <Common/quoteString.h>
#include <Common/callOnce.h>
@ -14,6 +16,7 @@
#include <Poco/UUIDGenerator.h>
#include <Poco/Logger.h>
#include <base/FnTraits.h>
#include <base/range.h>
#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <boost/range/adaptor/map.hpp>
@ -71,6 +74,18 @@ std::vector<UUID> IAccessStorage::find(AccessEntityType type, const Strings & na
}
std::vector<UUID> IAccessStorage::findAllImpl() const
{
std::vector<UUID> res;
for (auto type : collections::range(AccessEntityType::MAX))
{
auto ids = findAllImpl(type);
res.insert(res.end(), ids.begin(), ids.end());
}
return res;
}
UUID IAccessStorage::getID(AccessEntityType type, const String & name) const
{
auto id = findImpl(type, name);
@ -598,67 +613,59 @@ void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, c
if (!isBackupAllowed())
throwBackupNotAllowed();
auto entities = readAllWithIDs(type);
std::erase_if(entities, [](const std::pair<UUID, AccessEntityPtr> & x) { return !x.second->isBackupAllowed(); });
if (entities.empty())
auto entities_ids = findAll(type);
if (entities_ids.empty())
return;
auto backup_entry = makeBackupEntryForAccess(
entities,
data_path_in_backup,
backup_entries_collector.getAccessCounter(type),
backup_entries_collector.getContext()->getAccessControl());
auto backup_entry_with_path = makeBackupEntryForAccessEntities(
entities_ids,
backup_entries_collector.getAllAccessEntities(),
backup_entries_collector.getBackupSettings().write_access_entities_dependents,
data_path_in_backup);
backup_entries_collector.addBackupEntry(backup_entry);
if (isReplicated())
{
auto backup_coordination = backup_entries_collector.getBackupCoordination();
auto replication_id = getReplicationID();
backup_coordination->addReplicatedAccessFilePath(replication_id, type, backup_entry_with_path.first);
backup_entries_collector.addPostTask(
[backup_entry = backup_entry_with_path.second,
replication_id,
type,
&backup_entries_collector,
backup_coordination]
{
for (const String & path : backup_coordination->getReplicatedAccessFilePaths(replication_id, type))
backup_entries_collector.addBackupEntry(path, backup_entry);
});
}
else
{
backup_entries_collector.addBackupEntry(backup_entry_with_path);
}
}
void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup)
{
if (!isRestoreAllowed())
throwRestoreNotAllowed();
if (isReplicated() && !acquireReplicatedRestore(restorer))
return;
auto entities = restorer.getAccessEntitiesToRestore();
if (entities.empty())
return;
auto create_access = restorer.getRestoreSettings().create_access;
bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace);
bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate);
restorer.addDataRestoreTask([this, entities_to_restore = std::move(entities), replace_if_exists, throw_if_exists] mutable
if (isReplicated())
{
std::unordered_map<UUID, UUID> new_to_existing_ids;
for (auto & [id, entity] : entities_to_restore)
{
UUID existing_entity_id;
if (!insert(id, entity, replace_if_exists, throw_if_exists, &existing_entity_id))
{
/// Couldn't insert `entity` because there is an existing entity with the same name.
new_to_existing_ids[id] = existing_entity_id;
}
}
auto restore_coordination = restorer.getRestoreCoordination();
if (!restore_coordination->acquireReplicatedAccessStorage(getReplicationID()))
return;
}
if (!new_to_existing_ids.empty())
restorer.addDataRestoreTask(
[this, &restorer, data_path_in_backup]
{
/// If new entities restored from backup have dependencies on other entities from backup which were not restored because they existed,
/// then we should correct those dependencies.
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto res = entity;
IAccessEntity::replaceDependencies(res, new_to_existing_ids);
return res;
};
std::vector<UUID> ids;
ids.reserve(entities_to_restore.size());
boost::copy(entities_to_restore | boost::adaptors::map_keys, std::back_inserter(ids));
tryUpdate(ids, update_func);
}
});
auto entities_to_restore = restorer.getAccessEntitiesToRestore(data_path_in_backup);
const auto & restore_settings = restorer.getRestoreSettings();
restoreAccessEntitiesFromBackup(*this, entities_to_restore, restore_settings);
});
}

View File

@ -66,6 +66,7 @@ public:
/// Returns true if this storage is replicated.
virtual bool isReplicated() const { return false; }
virtual String getReplicationID() const { return ""; }
/// Starts periodic reloading and updating of entities in this storage.
virtual void startPeriodicReloading() {}
@ -90,8 +91,9 @@ public:
/// Returns the identifiers of all the entities of a specified type contained in the storage.
std::vector<UUID> findAll(AccessEntityType type) const;
template <typename EntityClassT>
std::vector<UUID> findAll() const { return findAll(EntityClassT::TYPE); }
/// Returns the identifiers of all the entities in the storage.
template <typename EntityClassT = IAccessEntity>
std::vector<UUID> findAll() const;
/// Searches for an entity with specified type and name. Returns std::nullopt if not found.
std::optional<UUID> find(AccessEntityType type, const String & name) const;
@ -148,7 +150,7 @@ public:
std::optional<std::pair<String, AccessEntityType>> tryReadNameWithType(const UUID & id) const;
/// Reads all entities and returns them with their IDs.
template <typename EntityClassT>
template <typename EntityClassT = IAccessEntity>
std::vector<std::pair<UUID, std::shared_ptr<const EntityClassT>>> readAllWithIDs() const;
std::vector<std::pair<UUID, AccessEntityPtr>> readAllWithIDs(AccessEntityType type) const;
@ -180,7 +182,7 @@ public:
/// Removes multiple entities from the storage. Returns the list of successfully dropped.
std::vector<UUID> tryRemove(const std::vector<UUID> & ids);
using UpdateFunc = std::function<AccessEntityPtr(const AccessEntityPtr &)>;
using UpdateFunc = std::function<AccessEntityPtr(const AccessEntityPtr &, const UUID &)>;
/// Updates an entity stored in the storage. Throws an exception if couldn't update.
bool update(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists = true);
@ -214,11 +216,12 @@ public:
/// Makes a backup of this access storage.
virtual void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const;
virtual void restoreFromBackup(RestorerFromBackup & restorer);
virtual void restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup);
protected:
virtual std::optional<UUID> findImpl(AccessEntityType type, const String & name) const = 0;
virtual std::vector<UUID> findAllImpl(AccessEntityType type) const = 0;
virtual std::vector<UUID> findAllImpl() const;
virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0;
virtual std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const;
virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id);
@ -267,6 +270,16 @@ private:
};
template <typename EntityClassT>
std::vector<UUID> IAccessStorage::findAll() const
{
if constexpr (std::is_same_v<EntityClassT, IAccessEntity>)
return findAllImpl();
else
return findAllImpl(EntityClassT::TYPE);
}
template <typename EntityClassT>
std::shared_ptr<const EntityClassT> IAccessStorage::read(const UUID & id, bool throw_if_not_exists) const
{

View File

@ -163,7 +163,7 @@ void LDAPAccessStorage::applyRoleChangeNoLock(bool grant, const UUID & role_id,
// Update the granted roles of the relevant users.
if (!user_ids.empty())
{
auto update_func = [&role_id, &grant] (const AccessEntityPtr & entity_) -> AccessEntityPtr
auto update_func = [&role_id, &grant] (const AccessEntityPtr & entity_, const UUID &) -> AccessEntityPtr
{
if (auto user = typeid_cast<std::shared_ptr<const User>>(entity_))
{
@ -301,7 +301,7 @@ void LDAPAccessStorage::updateAssignedRolesNoLock(const UUID & id, const String
if (it != external_role_hashes.end() && it->second == external_roles_hash)
return;
auto update_func = [this, &external_roles, external_roles_hash] (const AccessEntityPtr & entity_) -> AccessEntityPtr
auto update_func = [this, &external_roles, external_roles_hash] (const AccessEntityPtr & entity_, const UUID &) -> AccessEntityPtr
{
if (auto user = typeid_cast<std::shared_ptr<const User>>(entity_))
{

View File

@ -204,7 +204,7 @@ bool MemoryAccessStorage::updateNoLock(const UUID & id, const UpdateFunc & updat
Entry & entry = it->second;
auto old_entity = entry.entity;
auto new_entity = update_func(old_entity);
auto new_entity = update_func(old_entity, id);
if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());

View File

@ -416,7 +416,7 @@ bool MultipleAccessStorage::updateImpl(const UUID & id, const UpdateFunc & updat
{
if (auto old_entity = storage_for_updating->tryRead(id))
{
auto new_entity = update_func(old_entity);
auto new_entity = update_func(old_entity, id);
if (new_entity->getName() != old_entity->getName())
{
for (const auto & storage : *storages)
@ -508,7 +508,7 @@ void MultipleAccessStorage::backup(BackupEntriesCollector & backup_entries_colle
throwBackupNotAllowed();
}
void MultipleAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
void MultipleAccessStorage::restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup)
{
auto storages = getStoragesInternal();
@ -516,7 +516,7 @@ void MultipleAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
{
if (storage->isRestoreAllowed())
{
storage->restoreFromBackup(restorer);
storage->restoreFromBackup(restorer, data_path_in_backup);
return;
}
}

View File

@ -59,7 +59,7 @@ public:
bool isBackupAllowed() const override;
bool isRestoreAllowed() const override;
void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override;
void restoreFromBackup(RestorerFromBackup & restorer) override;
void restoreFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup) override;
bool containsStorage(std::string_view storage_type) const;
protected:

View File

@ -24,9 +24,33 @@ std::vector<UUID> Quota::findDependencies() const
return to_roles.findDependencies();
}
void Quota::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
bool Quota::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return to_roles.hasDependencies(ids);
}
void Quota::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
to_roles.replaceDependencies(old_to_new_ids);
}
void Quota::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_quota = typeid_cast<const Quota &>(src);
to_roles.copyDependenciesFrom(src_quota.to_roles, ids);
}
void Quota::removeDependencies(const std::unordered_set<UUID> & ids)
{
to_roles.removeDependencies(ids);
}
void Quota::clearAllExceptDependencies()
{
all_limits.clear();
key_type = QuotaKeyType::NONE;
}
}

View File

@ -47,7 +47,12 @@ struct Quota : public IAccessEntity
AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override;
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return true; }
};

View File

@ -4,10 +4,6 @@
#include <Access/ReplicatedAccessStorage.h>
#include <Access/AccessChangesNotifier.h>
#include <Access/AccessBackup.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/IBackupCoordination.h>
#include <Backups/IRestoreCoordination.h>
#include <Backups/RestorerFromBackup.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/Context.h>
#include <Common/ZooKeeper/KeeperException.h>
@ -359,7 +355,7 @@ bool ReplicatedAccessStorage::updateZooKeeper(const zkutil::ZooKeeperPtr & zooke
}
const AccessEntityPtr old_entity = deserializeAccessEntity(old_entity_definition, entity_path);
const AccessEntityPtr new_entity = update_func(old_entity);
const AccessEntityPtr new_entity = update_func(old_entity, id);
if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());
@ -684,44 +680,4 @@ AccessEntityPtr ReplicatedAccessStorage::readImpl(const UUID & id, bool throw_if
return memory_storage.read(id, throw_if_not_exists);
}
void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const
{
if (!isBackupAllowed())
throwBackupNotAllowed();
auto entities = readAllWithIDs(type);
std::erase_if(entities, [](const std::pair<UUID, AccessEntityPtr> & x) { return !x.second->isBackupAllowed(); });
if (entities.empty())
return;
auto backup_entry_with_path = makeBackupEntryForAccess(
entities,
data_path_in_backup,
backup_entries_collector.getAccessCounter(type),
backup_entries_collector.getContext()->getAccessControl());
auto backup_coordination = backup_entries_collector.getBackupCoordination();
backup_coordination->addReplicatedAccessFilePath(zookeeper_path, type, backup_entry_with_path.first);
backup_entries_collector.addPostTask(
[backup_entry = backup_entry_with_path.second,
my_zookeeper_path = zookeeper_path,
type,
&backup_entries_collector,
backup_coordination]
{
for (const String & path : backup_coordination->getReplicatedAccessFilePaths(my_zookeeper_path, type))
backup_entries_collector.addBackupEntry(path, backup_entry);
});
}
bool ReplicatedAccessStorage::acquireReplicatedRestore(RestorerFromBackup & restorer) const
{
auto restore_coordination = restorer.getRestoreCoordination();
return restore_coordination->acquireReplicatedAccessStorage(zookeeper_path);
}
}

View File

@ -26,7 +26,9 @@ public:
void shutdown() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
bool isReplicated() const override { return true; }
String getReplicationID() const override { return zookeeper_path; }
void startPeriodicReloading() override { startWatchingThread(); }
void stopPeriodicReloading() override { stopWatchingThread(); }
@ -35,7 +37,6 @@ public:
bool exists(const UUID & id) const override;
bool isBackupAllowed() const override { return backup_allowed; }
void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override;
private:
String zookeeper_path;
@ -80,7 +81,6 @@ private:
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
bool acquireReplicatedRestore(RestorerFromBackup & restorer) const override;
mutable std::mutex mutex;
MemoryAccessStorage memory_storage TSA_GUARDED_BY(mutex);

View File

@ -21,10 +21,36 @@ std::vector<UUID> Role::findDependencies() const
return res;
}
void Role::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
bool Role::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return granted_roles.hasDependencies(ids) || settings.hasDependencies(ids);
}
void Role::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
granted_roles.replaceDependencies(old_to_new_ids);
settings.replaceDependencies(old_to_new_ids);
}
void Role::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_role = typeid_cast<const Role &>(src);
granted_roles.copyDependenciesFrom(src_role.granted_roles, ids);
settings.copyDependenciesFrom(src_role.settings, ids);
}
void Role::removeDependencies(const std::unordered_set<UUID> & ids)
{
granted_roles.removeDependencies(ids);
settings.removeDependencies(ids);
}
void Role::clearAllExceptDependencies()
{
access = {};
settings.removeSettingsKeepProfiles();
}
}

View File

@ -21,7 +21,12 @@ struct Role : public IAccessEntity
AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override;
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
};

View File

@ -295,6 +295,23 @@ std::vector<UUID> RolesOrUsersSet::findDependencies() const
return res;
}
bool RolesOrUsersSet::hasDependencies(const std::unordered_set<UUID> & dependencies_ids) const
{
for (const auto & id : ids)
{
if (dependencies_ids.contains(id))
return true;
}
for (const auto & id : except_ids)
{
if (dependencies_ids.contains(id))
return true;
}
return false;
}
void RolesOrUsersSet::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
std::vector<UUID> new_ids;
@ -337,4 +354,41 @@ void RolesOrUsersSet::replaceDependencies(const std::unordered_map<UUID, UUID> &
boost::range::copy(new_ids, std::inserter(except_ids, except_ids.end()));
}
void RolesOrUsersSet::copyDependenciesFrom(const RolesOrUsersSet & src, const std::unordered_set<UUID> & dependencies_ids)
{
if (all != src.all)
return;
for (const auto & id : src.ids)
{
if (dependencies_ids.contains(id))
ids.emplace(id);
}
for (const auto & id : src.except_ids)
{
if (dependencies_ids.contains(id))
except_ids.emplace(id);
}
}
void RolesOrUsersSet::removeDependencies(const std::unordered_set<UUID> & dependencies_ids)
{
for (auto it = ids.begin(); it != ids.end();)
{
if (dependencies_ids.contains(*it))
it = ids.erase(it);
else
++it;
}
for (auto it = except_ids.begin(); it != except_ids.end();)
{
if (dependencies_ids.contains(*it))
except_ids.erase(it);
else
++it;
}
}
}

View File

@ -64,7 +64,10 @@ struct RolesOrUsersSet
friend bool operator !=(const RolesOrUsersSet & lhs, const RolesOrUsersSet & rhs) { return !(lhs == rhs); }
std::vector<UUID> findDependencies() const;
bool hasDependencies(const std::unordered_set<UUID> & dependencies_ids) const;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids);
void copyDependenciesFrom(const RolesOrUsersSet & src, const std::unordered_set<UUID> & dependencies_ids);
void removeDependencies(const std::unordered_set<UUID> & dependencies_ids);
bool all = false;
boost::container::flat_set<UUID> ids;

View File

@ -63,9 +63,33 @@ std::vector<UUID> RowPolicy::findDependencies() const
return to_roles.findDependencies();
}
void RowPolicy::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
bool RowPolicy::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return to_roles.hasDependencies(ids);
}
void RowPolicy::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
to_roles.replaceDependencies(old_to_new_ids);
}
void RowPolicy::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_policy = typeid_cast<const RowPolicy &>(src);
to_roles.copyDependenciesFrom(src_policy.to_roles, ids);
}
void RowPolicy::removeDependencies(const std::unordered_set<UUID> & ids)
{
to_roles.removeDependencies(ids);
}
void RowPolicy::clearAllExceptDependencies()
{
for (auto & filter : filters)
filter = {};
}
}

View File

@ -50,7 +50,12 @@ struct RowPolicy : public IAccessEntity
AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override;
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return true; }
/// Which roles or users should use this row policy.

View File

@ -21,10 +21,35 @@ std::vector<UUID> SettingsProfile::findDependencies() const
return res;
}
void SettingsProfile::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
bool SettingsProfile::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return elements.hasDependencies(ids) || to_roles.hasDependencies(ids);
}
void SettingsProfile::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
elements.replaceDependencies(old_to_new_ids);
to_roles.replaceDependencies(old_to_new_ids);
}
void SettingsProfile::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_profile = typeid_cast<const SettingsProfile &>(src);
elements.copyDependenciesFrom(src_profile.elements, ids);
to_roles.copyDependenciesFrom(src_profile.to_roles, ids);
}
void SettingsProfile::removeDependencies(const std::unordered_set<UUID> & ids)
{
elements.removeDependencies(ids);
to_roles.removeDependencies(ids);
}
void SettingsProfile::clearAllExceptDependencies()
{
elements.removeSettingsKeepProfiles();
}
}

View File

@ -22,7 +22,12 @@ struct SettingsProfile : public IAccessEntity
AccessEntityType getType() const override { return TYPE; }
std::vector<UUID> findDependencies() const override;
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return elements.isBackupAllowed(); }
};

View File

@ -158,6 +158,18 @@ std::vector<UUID> SettingsProfileElements::findDependencies() const
}
bool SettingsProfileElements::hasDependencies(const std::unordered_set<UUID> & ids) const
{
std::vector<UUID> res;
for (const auto & element : *this)
{
if (element.parent_profile && ids.contains(*element.parent_profile))
return true;
}
return false;
}
void SettingsProfileElements::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
for (auto & element : *this)
@ -176,6 +188,38 @@ void SettingsProfileElements::replaceDependencies(const std::unordered_map<UUID,
}
void SettingsProfileElements::copyDependenciesFrom(const SettingsProfileElements & src, const std::unordered_set<UUID> & ids)
{
SettingsProfileElements new_elements;
for (const auto & element : src)
{
if (element.parent_profile && ids.contains(*element.parent_profile))
{
SettingsProfileElement new_element;
new_element.parent_profile = *element.parent_profile;
new_elements.emplace_back(new_element);
}
}
insert(begin(), new_elements.begin(), new_elements.end());
}
void SettingsProfileElements::removeDependencies(const std::unordered_set<UUID> & ids)
{
std::erase_if(
*this, [&](const SettingsProfileElement & element) { return element.parent_profile && ids.contains(*element.parent_profile); });
}
void SettingsProfileElements::removeSettingsKeepProfiles()
{
for (auto & element : *this)
element.setting_name.clear();
std::erase_if(*this, [&](const SettingsProfileElement & element) { return element.setting_name.empty() && !element.parent_profile; });
}
void SettingsProfileElements::merge(const SettingsProfileElements & other)
{
insert(end(), other.begin(), other.end());

View File

@ -63,7 +63,12 @@ public:
std::shared_ptr<ASTSettingsProfileElements> toASTWithNames(const AccessControl & access_control) const;
std::vector<UUID> findDependencies() const;
bool hasDependencies(const std::unordered_set<UUID> & ids) const;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids);
void copyDependenciesFrom(const SettingsProfileElements & src, const std::unordered_set<UUID> & ids);
void removeDependencies(const std::unordered_set<UUID> & ids);
void removeSettingsKeepProfiles();
void merge(const SettingsProfileElements & other);

View File

@ -49,7 +49,12 @@ std::vector<UUID> User::findDependencies() const
return res;
}
void User::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
bool User::hasDependencies(const std::unordered_set<UUID> & ids) const
{
return default_roles.hasDependencies(ids) || granted_roles.hasDependencies(ids) || grantees.hasDependencies(ids) || settings.hasDependencies(ids);
}
void User::replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids)
{
default_roles.replaceDependencies(old_to_new_ids);
granted_roles.replaceDependencies(old_to_new_ids);
@ -57,4 +62,33 @@ void User::doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_n
settings.replaceDependencies(old_to_new_ids);
}
void User::copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids)
{
if (getType() != src.getType())
return;
const auto & src_user = typeid_cast<const User &>(src);
default_roles.copyDependenciesFrom(src_user.default_roles, ids);
granted_roles.copyDependenciesFrom(src_user.granted_roles, ids);
grantees.copyDependenciesFrom(src_user.grantees, ids);
settings.copyDependenciesFrom(src_user.settings, ids);
}
void User::removeDependencies(const std::unordered_set<UUID> & ids)
{
default_roles.removeDependencies(ids);
granted_roles.removeDependencies(ids);
grantees.removeDependencies(ids);
settings.removeDependencies(ids);
}
void User::clearAllExceptDependencies()
{
authentication_methods.clear();
allowed_client_hosts = AllowedClientHosts::AnyHostTag{};
access = {};
settings.removeSettingsKeepProfiles();
default_database = {};
valid_until = 0;
}
}

View File

@ -32,7 +32,12 @@ struct User : public IAccessEntity
void setName(const String & name_) override;
std::vector<UUID> findDependencies() const override;
void doReplaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
bool hasDependencies(const std::unordered_set<UUID> & ids) const override;
void replaceDependencies(const std::unordered_map<UUID, UUID> & old_to_new_ids) override;
void copyDependenciesFrom(const IAccessEntity & src, const std::unordered_set<UUID> & ids) override;
void removeDependencies(const std::unordered_set<UUID> & ids) override;
void clearAllExceptDependencies() override;
bool isBackupAllowed() const override { return settings.isBackupAllowed(); }
};

View File

@ -445,6 +445,48 @@ TEST(AccessRights, Intersection)
ASSERT_EQ(lhs.toString(), "GRANT SELECT ON toaster.*");
}
TEST(AccessRights, Difference)
{
AccessRights lhs, rhs;
lhs.grant(AccessType::SELECT);
rhs.grant(AccessType::SELECT);
rhs.revoke(AccessType::SELECT, "system");
lhs.makeDifference(rhs);
ASSERT_EQ(lhs.toString(), "GRANT SELECT ON system.*");
lhs = {};
rhs = {};
lhs.grantWildcard(AccessType::SELECT, "toast");
rhs.grant(AccessType::SELECT);
rhs.revoke(AccessType::SELECT, "toaster");
lhs.makeDifference(rhs);
ASSERT_EQ(lhs.toString(), "GRANT SELECT ON toaster.*");
lhs = {};
rhs = {};
lhs.grantWildcard(AccessType::SELECT, "toast");
lhs.grant(AccessType::CREATE_TABLE, "jam");
auto lhs_old = lhs;
lhs.makeDifference(rhs);
ASSERT_EQ(lhs, lhs_old);
lhs = {};
rhs = {};
lhs.grant(AccessType::SELECT, "toast");
rhs.grant(AccessType::CREATE_TABLE, "jam");
lhs_old = lhs;
lhs.makeDifference(rhs);
ASSERT_EQ(lhs, lhs_old);
lhs = {};
rhs = {};
lhs.grant(AccessType::ALL);
rhs.grant(AccessType::ALL);
rhs.revoke(AccessType::SELECT, "system");
lhs.makeDifference(rhs);
ASSERT_EQ(lhs.toString(), "GRANT SELECT ON system.*");
}
TEST(AccessRights, Contains)
{
AccessRights lhs, rhs;

View File

@ -423,6 +423,14 @@ QueryTreeNodePtr IdentifierResolver::tryResolveTableIdentifierFromDatabaseCatalo
else
storage = DatabaseCatalog::instance().tryGetTable(storage_id, context);
if (!storage && storage_id.hasUUID())
{
// If `storage_id` has UUID, it is possible that the UUID is removed from `DatabaseCatalog` after `context->resolveStorageID(storage_id)`
// We try to get the table with the database name and the table name.
auto database = DatabaseCatalog::instance().tryGetDatabase(storage_id.getDatabaseName());
if (database)
storage = database->tryGetTable(table_name, context);
}
if (!storage)
return {};

View File

@ -570,7 +570,7 @@ void BackupCoordinationRemote::prepareReplicatedAccess() const
if (replicated_access)
return;
std::vector<BackupCoordinationReplicatedAccess::FilePathForAccessEntitry> file_path_for_access_entities;
std::vector<BackupCoordinationReplicatedAccess::FilePathForAccessEntity> file_path_for_access_entities;
auto holder = with_retries.createRetriesControlHolder("prepareReplicatedAccess");
holder.retries_ctl.retryLoop(
[&, &zk = holder.faulty_zookeeper]()

View File

@ -1,5 +1,9 @@
#include <Backups/BackupCoordinationReplicatedAccess.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -7,7 +11,7 @@ namespace DB
BackupCoordinationReplicatedAccess::BackupCoordinationReplicatedAccess() = default;
BackupCoordinationReplicatedAccess::~BackupCoordinationReplicatedAccess() = default;
void BackupCoordinationReplicatedAccess::addFilePath(FilePathForAccessEntitry && file_path_for_access_entity)
void BackupCoordinationReplicatedAccess::addFilePath(FilePathForAccessEntity && file_path_for_access_entity)
{
const auto & access_zk_path = file_path_for_access_entity.access_zk_path;
const auto & access_entity_type = file_path_for_access_entity.access_entity_type;
@ -28,10 +32,19 @@ Strings BackupCoordinationReplicatedAccess::getFilePaths(const String & access_z
return {};
const auto & file_paths = it->second;
if (file_paths.host_to_store_access != host_id)
if ((file_paths.host_to_store_access != host_id) || file_paths.file_paths.empty())
return {};
Strings res{file_paths.file_paths.begin(), file_paths.file_paths.end()};
/// Use the same filename for all the paths in backup.
/// Those filenames have format "access-<UUID>.txt", where UUID is random.
/// It's not really necessary, however it looks better if those files have the same filename
/// for a backup of ReplicatedAccessStorage on different hosts.
Strings res;
res.reserve(file_paths.file_paths.size());
String filename = fs::path{*file_paths.file_paths.begin()}.filename();
for (const auto & file_path : file_paths.file_paths)
res.emplace_back(fs::path{file_path}.replace_filename(filename));
return res;
}

View File

@ -2,7 +2,7 @@
#include <Core/Types.h>
#include <map>
#include <unordered_set>
#include <set>
namespace DB
@ -28,7 +28,7 @@ public:
BackupCoordinationReplicatedAccess();
~BackupCoordinationReplicatedAccess();
struct FilePathForAccessEntitry
struct FilePathForAccessEntity
{
String access_zk_path;
AccessEntityType access_entity_type;
@ -37,7 +37,7 @@ public:
};
/// Adds a path to access*.txt file keeping access entities of a ReplicatedAccessStorage.
void addFilePath(FilePathForAccessEntitry && file_path_for_access_entity);
void addFilePath(FilePathForAccessEntity && file_path_for_access_entity);
/// Returns all paths added by addFilePath() if `host_id` is a host chosen to store access.
Strings getFilePaths(const String & access_zk_path, AccessEntityType access_entity_type, const String & host_id) const;
@ -47,7 +47,7 @@ private:
struct FilePathsAndHost
{
std::unordered_set<String> file_paths;
std::set<String> file_paths;
String host_to_store_access;
};

View File

@ -32,6 +32,9 @@ namespace BackupCoordinationStage
/// Finding databases and tables in the backup which we're going to restore.
constexpr const char * FINDING_TABLES_IN_BACKUP = "finding tables in backup";
/// Loading system access tables and then checking if the current user has enough access to restore.
constexpr const char * CHECKING_ACCESS_RIGHTS = "checking access rights";
/// Creating databases or finding them and checking their definitions.
constexpr const char * CREATING_DATABASES = "creating databases";

View File

@ -1,4 +1,5 @@
#include <Access/Common/AccessEntityType.h>
#include <Access/AccessControl.h>
#include <Backups/BackupCoordinationStage.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/BackupEntryFromMemory.h>
@ -910,11 +911,20 @@ void BackupEntriesCollector::runPostTasks()
LOG_TRACE(log, "All post tasks successfully executed");
}
size_t BackupEntriesCollector::getAccessCounter(AccessEntityType type)
std::unordered_map<UUID, AccessEntityPtr> BackupEntriesCollector::getAllAccessEntities()
{
std::lock_guard lock(mutex);
access_counters.resize(static_cast<size_t>(AccessEntityType::MAX));
return access_counters[static_cast<size_t>(type)]++;
if (!all_access_entities)
{
all_access_entities.emplace();
auto entities_with_ids = context->getAccessControl().readAllWithIDs();
for (const auto & [id, entity] : entities_with_ids)
{
if (entity->isBackupAllowed())
all_access_entities->emplace(id, entity);
}
}
return *all_access_entities;
}
}

View File

@ -21,7 +21,8 @@ class IBackupCoordination;
class IDatabase;
using DatabasePtr = std::shared_ptr<IDatabase>;
struct StorageID;
enum class AccessEntityType : uint8_t;
struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
class QueryStatus;
using QueryStatusPtr = std::shared_ptr<QueryStatus>;
@ -49,6 +50,9 @@ public:
ContextPtr getContext() const { return context; }
const ZooKeeperRetriesInfo & getZooKeeperRetriesInfo() const { return global_zookeeper_retries_info; }
/// Returns all access entities which can be put into a backup.
std::unordered_map<UUID, AccessEntityPtr> getAllAccessEntities();
/// Adds a backup entry which will be later returned by run().
/// These function can be called by implementations of IStorage::backupData() in inherited storage classes.
void addBackupEntry(const String & file_name, BackupEntryPtr backup_entry);
@ -61,9 +65,6 @@ public:
/// 1) we need to join (in a backup) the data of replicated tables gathered on different hosts.
void addPostTask(std::function<void()> task);
/// Returns an incremental counter used to backup access control.
size_t getAccessCounter(AccessEntityType type);
private:
void calculateRootPathInBackup();
@ -177,9 +178,10 @@ private:
std::vector<std::pair<String, String>> previous_databases_metadata;
std::vector<std::pair<QualifiedTableName, String>> previous_tables_metadata;
std::optional<std::unordered_map<UUID, AccessEntityPtr>> all_access_entities;
BackupEntries backup_entries;
std::queue<std::function<void()>> post_tasks;
std::vector<size_t> access_counters;
ThreadPool & threadpool;
std::mutex mutex;

View File

@ -37,6 +37,7 @@ namespace ErrorCodes
M(Bool, check_parts) \
M(Bool, check_projection_parts) \
M(Bool, allow_backup_broken_projections) \
M(Bool, write_access_entities_dependents) \
M(Bool, internal) \
M(String, host_id) \
M(OptionalUUID, backup_uuid)

View File

@ -77,6 +77,11 @@ struct BackupSettings
/// Allow to create backup with broken projections.
bool allow_backup_broken_projections = false;
/// Whether dependents of access entities should be written along with the access entities.
/// For example, if a role is granted to a user and we're making a backup of system.roles (but not system.users)
/// this is whether the backup will contain information to grant the role to the corresponding user again.
bool write_access_entities_dependents = true;
/// Internal, should not be specified by user.
/// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER.
bool internal = false;

View File

@ -160,7 +160,8 @@ namespace
M(UInt64, replica_num_in_backup) \
M(Bool, allow_non_empty_tables) \
M(RestoreAccessCreationMode, create_access) \
M(Bool, allow_unresolved_access_dependencies) \
M(Bool, skip_unresolved_access_dependencies) \
M(Bool, update_access_entities_dependents) \
M(RestoreUDFCreationMode, create_function) \
M(Bool, allow_s3_native_copy) \
M(Bool, use_same_s3_credentials_for_base_backup) \
@ -187,7 +188,12 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
else
LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER)
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name);
/// `allow_unresolved_access_dependencies` is an obsolete name.
if (setting.name == "allow_unresolved_access_dependencies")
res.skip_unresolved_access_dependencies = SettingFieldBool{setting.value}.value;
else
throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name);
}
}

View File

@ -100,9 +100,25 @@ struct RestoreSettings
/// How the RESTORE command will handle if an user (or role or profile) which it's going to restore already exists.
RestoreAccessCreationMode create_access = RestoreAccessCreationMode::kCreateIfNotExists;
/// Skip dependencies of access entities which can't be resolved.
/// For example, if an user has a profile assigned and that profile is not in the backup and doesn't exist locally.
bool allow_unresolved_access_dependencies = false;
/// Ignore dependencies or dependents (with update_access_entities_dependents=true) of access entities in the case if they can't be resolved.
/// For example: if a backup contains a profile assigned to a user: `CREATE PROFILE p1; CREATE USER u1 SETTINGS PROFILE p1`
/// and now we're restoring only user `u1` and profile `p1` doesn't exists, then
/// this flag is whether RESTORE should continue with restoring user `u1` without assigning profile `p1`.
/// Another example: if a backup contains a role granted to a user: `CREATE USER u2; CREATE ROLE r2; GRANT r2 TO u2`
/// and now we're restoring only user `u2` and role `r2` doesn't exist, then
/// this flag is whether RESTORE should continue with restoring user `u2` without that grant.
/// If this flag is false then RESTORE will throw an exception in that case.
bool skip_unresolved_access_dependencies = false;
/// Try to update dependents of restored access entities.
/// For example: if a backup contains a profile assigned to a user: `CREATE PROFILE p1; CREATE USER u1 SETTINGS PROFILE p1`
/// and now we're restoring only profile `p1` and user `u1` already exists, then
/// this flag is whether restored profile `p1` should be assigned to user `u1` again.
/// Another example, if a backup contains a role granted to a user: `CREATE USER u2; CREATE ROLE r2; GRANT r2 TO u2`
/// and now we're restoring only role `r2` and user `u2` already exists, then
/// this flag is whether restored role `r2` should be granted to user `u2` again.
/// If this flag is false then RESTORE won't update existing access entities.
bool update_access_entities_dependents = true;
/// How the RESTORE command will handle if a user-defined function which it's going to restore already exists.
RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists;

View File

@ -141,6 +141,8 @@ void RestorerFromBackup::run(Mode mode)
waitFutures();
/// Check access rights.
setStage(Stage::CHECKING_ACCESS_RIGHTS);
loadSystemAccessTables();
checkAccessForObjectsFoundInBackup();
if (mode == Mode::CHECK_ACCESS_ONLY)
@ -487,25 +489,6 @@ void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_
res_table_info.partitions.emplace();
insertAtEnd(*res_table_info.partitions, *partitions);
}
/// Special handling for ACL-related system tables.
if (!restore_settings.structure_only && isSystemAccessTableName(table_name))
{
if (!access_restorer)
access_restorer = std::make_unique<AccessRestorerFromBackup>(backup, restore_settings);
try
{
/// addDataPath() will parse access*.txt files and extract access entities from them.
/// We need to do that early because we need those access entities to check access.
access_restorer->addDataPath(data_path_in_backup);
}
catch (Exception & e)
{
e.addMessage("While parsing data of {} from backup", tableNameWithTypeToString(table_name.database, table_name.table, false));
throw;
}
}
}
void RestorerFromBackup::findDatabaseInBackup(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names)
@ -629,6 +612,27 @@ size_t RestorerFromBackup::getNumTables() const
return table_infos.size();
}
void RestorerFromBackup::loadSystemAccessTables()
{
if (restore_settings.structure_only)
return;
/// Special handling for ACL-related system tables.
std::lock_guard lock{mutex};
for (const auto & [table_name, table_info] : table_infos)
{
if (isSystemAccessTableName(table_name))
{
if (!access_restorer)
access_restorer = std::make_unique<AccessRestorerFromBackup>(backup, restore_settings);
access_restorer->addDataPath(table_info.data_path_in_backup);
}
}
if (access_restorer)
access_restorer->loadFromBackup();
}
void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
{
AccessRightsElements required_access;
@ -713,6 +717,15 @@ void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
context->checkAccess(required_access);
}
AccessEntitiesToRestore RestorerFromBackup::getAccessEntitiesToRestore(const String & data_path_in_backup) const
{
std::lock_guard lock{mutex};
if (!access_restorer)
return {};
access_restorer->generateRandomIDsAndResolveDependencies(context->getAccessControl());
return access_restorer->getEntitiesToRestore(data_path_in_backup);
}
void RestorerFromBackup::createDatabases()
{
Strings database_names;
@ -1071,19 +1084,6 @@ void RestorerFromBackup::runDataRestoreTasks()
}
}
std::vector<std::pair<UUID, AccessEntityPtr>> RestorerFromBackup::getAccessEntitiesToRestore()
{
std::lock_guard lock{mutex};
if (!access_restorer || access_restored)
return {};
/// getAccessEntitiesToRestore() will return entities only when called first time (we don't want to restore the same entities again).
access_restored = true;
return access_restorer->getAccessEntities(context->getAccessControl());
}
void RestorerFromBackup::throwTableIsNotEmpty(const StorageID & storage_id)
{
throw Exception(

View File

@ -20,8 +20,7 @@ struct StorageID;
class IDatabase;
using DatabasePtr = std::shared_ptr<IDatabase>;
class AccessRestorerFromBackup;
struct IAccessEntity;
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
struct AccessEntitiesToRestore;
class QueryStatus;
using QueryStatusPtr = std::shared_ptr<QueryStatus>;
@ -68,7 +67,7 @@ public:
void addDataRestoreTasks(DataRestoreTasks && new_tasks);
/// Returns the list of access entities to restore.
std::vector<std::pair<UUID, AccessEntityPtr>> getAccessEntitiesToRestore();
AccessEntitiesToRestore getAccessEntitiesToRestore(const String & data_path_in_backup) const;
/// Throws an exception that a specified table is already non-empty.
[[noreturn]] static void throwTableIsNotEmpty(const StorageID & storage_id);
@ -101,6 +100,7 @@ private:
size_t getNumDatabases() const;
size_t getNumTables() const;
void loadSystemAccessTables();
void checkAccessForObjectsFoundInBackup() const;
void createDatabases();

View File

@ -1,5 +1,6 @@
#include <gtest/gtest.h>
#include <Common/tests/gtest_global_context.h>
#include <Backups/BackupEntryFromAppendOnlyFile.h>
#include <Backups/BackupEntryFromImmutableFile.h>
#include <Backups/BackupEntryFromSmallFile.h>
@ -217,8 +218,9 @@ TEST_F(BackupEntriesTest, PartialChecksumBeforeFullChecksum)
TEST_F(BackupEntriesTest, BackupEntryFromSmallFile)
{
auto read_settings = getContext().context->getReadSettings();
writeFile(local_disk, "a.txt");
auto entry = std::make_shared<BackupEntryFromSmallFile>(local_disk, "a.txt", ReadSettings{});
auto entry = std::make_shared<BackupEntryFromSmallFile>(local_disk, "a.txt", read_settings);
local_disk->removeFile("a.txt");
@ -234,12 +236,13 @@ TEST_F(BackupEntriesTest, BackupEntryFromSmallFile)
TEST_F(BackupEntriesTest, DecryptedEntriesFromEncryptedDisk)
{
auto read_settings = getContext().context->getReadSettings();
{
writeFile(encrypted_disk, "a.txt");
std::pair<BackupEntryPtr, bool /* partial_checksum_allowed */> test_cases[]
= {{std::make_shared<BackupEntryFromImmutableFile>(encrypted_disk, "a.txt"), false},
{std::make_shared<BackupEntryFromAppendOnlyFile>(encrypted_disk, "a.txt"), true},
{std::make_shared<BackupEntryFromSmallFile>(encrypted_disk, "a.txt", ReadSettings{}), true}};
{std::make_shared<BackupEntryFromSmallFile>(encrypted_disk, "a.txt", read_settings), true}};
for (const auto & [entry, partial_checksum_allowed] : test_cases)
{
EXPECT_EQ(entry->getSize(), 9);
@ -258,7 +261,7 @@ TEST_F(BackupEntriesTest, DecryptedEntriesFromEncryptedDisk)
BackupEntryPtr entries[]
= {std::make_shared<BackupEntryFromImmutableFile>(encrypted_disk, "empty.txt"),
std::make_shared<BackupEntryFromAppendOnlyFile>(encrypted_disk, "empty.txt"),
std::make_shared<BackupEntryFromSmallFile>(encrypted_disk, "empty.txt", ReadSettings{})};
std::make_shared<BackupEntryFromSmallFile>(encrypted_disk, "empty.txt", read_settings)};
for (const auto & entry : entries)
{
EXPECT_EQ(entry->getSize(), 0);
@ -283,12 +286,13 @@ TEST_F(BackupEntriesTest, DecryptedEntriesFromEncryptedDisk)
TEST_F(BackupEntriesTest, EncryptedEntriesFromEncryptedDisk)
{
auto read_settings = getContext().context->getReadSettings();
{
writeFile(encrypted_disk, "a.txt");
BackupEntryPtr entries[]
= {std::make_shared<BackupEntryFromImmutableFile>(encrypted_disk, "a.txt", /* copy_encrypted= */ true),
std::make_shared<BackupEntryFromAppendOnlyFile>(encrypted_disk, "a.txt", /* copy_encrypted= */ true),
std::make_shared<BackupEntryFromSmallFile>(encrypted_disk, "a.txt", ReadSettings{}, /* copy_encrypted= */ true)};
std::make_shared<BackupEntryFromSmallFile>(encrypted_disk, "a.txt", read_settings, /* copy_encrypted= */ true)};
auto encrypted_checksum = getChecksum(entries[0]);
EXPECT_NE(encrypted_checksum, NO_CHECKSUM);
@ -322,7 +326,7 @@ TEST_F(BackupEntriesTest, EncryptedEntriesFromEncryptedDisk)
BackupEntryPtr entries[]
= {std::make_shared<BackupEntryFromImmutableFile>(encrypted_disk, "empty.txt", /* copy_encrypted= */ true),
std::make_shared<BackupEntryFromAppendOnlyFile>(encrypted_disk, "empty.txt", /* copy_encrypted= */ true),
std::make_shared<BackupEntryFromSmallFile>(encrypted_disk, "empty.txt", ReadSettings{}, /* copy_encrypted= */ true)};
std::make_shared<BackupEntryFromSmallFile>(encrypted_disk, "empty.txt", read_settings, /* copy_encrypted= */ true)};
for (const auto & entry : entries)
{
EXPECT_EQ(entry->getSize(), 0);

View File

@ -243,7 +243,7 @@ protected:
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Failed to parse quoting style from '{}' for service {}",
character, BridgeHelperMixin::serviceAlias());
else if (character.empty())
quote_style = IdentifierQuotingStyle::None;
quote_style = IdentifierQuotingStyle::Backticks;
else if (character[0] == '`')
quote_style = IdentifierQuotingStyle::Backticks;
else if (character[0] == '"')

View File

@ -2825,11 +2825,12 @@ void ClientBase::runLibFuzzer() {}
void ClientBase::clearTerminal()
{
/// Clear from cursor until end of screen.
/// Move to the beginning of the line
/// and clear until end of screen.
/// It is needed if garbage is left in terminal.
/// Show cursor. It can be left hidden by invocation of previous programs.
/// A test for this feature: perl -e 'print "x"x100000'; echo -ne '\033[0;0H\033[?25l'; clickhouse-client
output_stream << "\033[0J" "\033[?25h";
output_stream << "\r" "\033[0J" "\033[?25h";
}
void ClientBase::showClientVersion()

View File

@ -10,7 +10,6 @@
#include <IO/Operators.h>
#include <Functions/FunctionFactory.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Storages/StorageFactory.h>
#include <DataTypes/DataTypeFactory.h>
#include <Interpreters/Context.h>
#include <Client/Connection.h>
@ -85,7 +84,6 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti
add_column("name", "columns", true, suggestion_limit);
}
/// FIXME: This query does not work with the new analyzer because of bug https://github.com/ClickHouse/ClickHouse/issues/50669
query = "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" + query + ") WHERE notEmpty(res)";
return query;
}

View File

@ -17,7 +17,7 @@ public:
bool isCancelled() const { return *counter > 0; }
/// Temporarily blocks corresponding actions (while the returned object is alive)
/// Temporary blocks corresponding actions (while the returned object is alive)
friend class ActionLock;
[[nodiscard]] ActionLock cancel() { return ActionLock(*this); }

View File

@ -112,12 +112,12 @@ public:
}
catch (...)
{
int err = close(fd);
[[maybe_unused]] int err = close(fd);
chassert(!err || errno == EINTR);
throw;
}
int err = close(fd);
[[maybe_unused]] int err = close(fd);
chassert(!err || errno == EINTR);
return res;
}
@ -182,12 +182,12 @@ public:
}
catch (...)
{
int err = close(fd);
[[maybe_unused]] int err = close(fd);
chassert(!err || errno == EINTR);
throw;
}
int err = close(fd);
[[maybe_unused]] int err = close(fd);
chassert(!err || errno == EINTR);
}

View File

@ -95,7 +95,7 @@ Epoll::~Epoll()
{
if (epoll_fd != -1)
{
int err = close(epoll_fd);
[[maybe_unused]] int err = close(epoll_fd);
chassert(!err || errno == EINTR);
}
}

View File

@ -452,7 +452,7 @@
M(553, LZMA_STREAM_ENCODER_FAILED) \
M(554, LZMA_STREAM_DECODER_FAILED) \
M(555, ROCKSDB_ERROR) \
M(556, SYNC_MYSQL_USER_ACCESS_ERROR)\
M(556, SYNC_MYSQL_USER_ACCESS_ERROR) \
M(557, UNKNOWN_UNION) \
M(558, EXPECTED_ALL_OR_DISTINCT) \
M(559, INVALID_GRPC_QUERY_INFO) \
@ -578,7 +578,7 @@
M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \
M(698, INVALID_REDIS_STORAGE_TYPE) \
M(699, INVALID_REDIS_TABLE_STRUCTURE) \
M(700, USER_SESSION_LIMIT_EXCEEDED) \
M(700, USER_SESSION_LIMIT_EXCEEDED) \
M(701, CLUSTER_DOESNT_EXIST) \
M(702, CLIENT_INFO_DOES_NOT_MATCH) \
M(703, INVALID_IDENTIFIER) \
@ -610,15 +610,16 @@
M(729, ILLEGAL_TIME_SERIES_TAGS) \
M(730, REFRESH_FAILED) \
M(731, QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE) \
\
M(733, TABLE_IS_BEING_RESTARTED) \
\
M(900, DISTRIBUTED_CACHE_ERROR) \
M(901, CANNOT_USE_DISTRIBUTED_CACHE) \
\
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \
M(1001, STD_EXCEPTION) \
M(1002, UNKNOWN_EXCEPTION) \
/* See END */
/* See END */
#ifdef APPLY_FOR_EXTERNAL_ERROR_CODES
#define APPLY_FOR_ERROR_CODES(M) APPLY_FOR_BUILTIN_ERROR_CODES(M) APPLY_FOR_EXTERNAL_ERROR_CODES(M)

View File

@ -57,7 +57,7 @@ EventFD::~EventFD()
{
if (fd != -1)
{
int err = close(fd);
[[maybe_unused]] int err = close(fd);
chassert(!err || errno == EINTR);
}
}

View File

@ -2,7 +2,6 @@
#if defined(OS_LINUX)
#include <cstddef>
#include <cstdint>

View File

@ -176,7 +176,7 @@ void FileChecker::load()
if (!fileReallyExists(files_info_path))
return;
std::unique_ptr<ReadBuffer> in = disk ? disk->readFile(files_info_path) : std::make_unique<ReadBufferFromFile>(files_info_path);
std::unique_ptr<ReadBuffer> in = disk ? disk->readFile(files_info_path, getReadSettings()) : std::make_unique<ReadBufferFromFile>(files_info_path);
WriteBufferFromOwnString out;
/// The JSON library does not support whitespace. We delete them. Inefficient.

View File

@ -283,7 +283,7 @@ NetlinkMetricsProvider::NetlinkMetricsProvider()
{
if (netlink_socket_fd >= 0)
{
int err = close(netlink_socket_fd);
[[maybe_unused]] int err = close(netlink_socket_fd);
chassert(!err || errno == EINTR);
}
throw;
@ -320,7 +320,7 @@ NetlinkMetricsProvider::~NetlinkMetricsProvider()
{
if (netlink_socket_fd >= 0)
{
int err = close(netlink_socket_fd);
[[maybe_unused]] int err = close(netlink_socket_fd);
chassert(!err || errno == EINTR);
}
}

View File

@ -98,7 +98,7 @@ ProcfsMetricsProvider::ProcfsMetricsProvider(pid_t /*tid*/)
thread_stat_fd = ::open(thread_stat, O_RDONLY | O_CLOEXEC);
if (-1 == thread_stat_fd)
{
int err = ::close(thread_schedstat_fd);
[[maybe_unused]] int err = ::close(thread_schedstat_fd);
chassert(!err || errno == EINTR);
throwWithFailedToOpenFile(thread_stat);
}

View File

@ -100,7 +100,7 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_)
}
catch (...)
{
int err = close(fd);
[[maybe_unused]] int err = close(fd);
chassert(!err || errno == EINTR);
throw;
}

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