mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge remote-tracking branch 'upstream/master' into interactive-metrics-table
This commit is contained in:
commit
1ea72d3453
@ -27,6 +27,8 @@ Checks: [
|
||||
'-bugprone-not-null-terminated-result',
|
||||
'-bugprone-reserved-identifier', # useful but too slow, TODO retry when https://reviews.llvm.org/rG1c282052624f9d0bd273bde0b47b30c96699c6c7 is merged
|
||||
'-bugprone-unchecked-optional-access',
|
||||
'-bugprone-crtp-constructor-accessibility',
|
||||
'-bugprone-suspicious-stringview-data-usage',
|
||||
|
||||
'-cert-dcl16-c',
|
||||
'-cert-dcl37-c',
|
||||
@ -36,6 +38,7 @@ Checks: [
|
||||
'-cert-msc51-cpp',
|
||||
'-cert-oop54-cpp',
|
||||
'-cert-oop57-cpp',
|
||||
'-cert-err33-c', # Misreports on clang-19: it warns about all functions containing 'remove' in the name, not only about the standard library.
|
||||
|
||||
'-clang-analyzer-optin.performance.Padding',
|
||||
|
||||
@ -99,6 +102,7 @@ Checks: [
|
||||
'-modernize-use-emplace',
|
||||
'-modernize-use-nodiscard',
|
||||
'-modernize-use-trailing-return-type',
|
||||
'-modernize-use-designated-initializers',
|
||||
|
||||
'-performance-enum-size',
|
||||
'-performance-inefficient-string-concatenation',
|
||||
|
4
.github/workflows/backport_branches.yml
vendored
4
.github/workflows/backport_branches.yml
vendored
@ -36,10 +36,6 @@ jobs:
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
echo "Testing the main ci directory"
|
||||
python3 -m unittest discover -s . -p 'test_*.py'
|
||||
for dir in *_lambda/; do
|
||||
echo "Testing $dir"
|
||||
python3 -m unittest discover -s "$dir" -p 'test_*.py'
|
||||
done
|
||||
- name: PrepareRunConfig
|
||||
id: runconfig
|
||||
run: |
|
||||
|
4
.github/workflows/master.yml
vendored
4
.github/workflows/master.yml
vendored
@ -33,10 +33,6 @@ jobs:
|
||||
# cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
# echo "Testing the main ci directory"
|
||||
# python3 -m unittest discover -s . -p 'test_*.py'
|
||||
# for dir in *_lambda/; do
|
||||
# echo "Testing $dir"
|
||||
# python3 -m unittest discover -s "$dir" -p 'test_*.py'
|
||||
# done
|
||||
- name: PrepareRunConfig
|
||||
id: runconfig
|
||||
run: |
|
||||
|
4
.github/workflows/merge_queue.yml
vendored
4
.github/workflows/merge_queue.yml
vendored
@ -30,10 +30,6 @@ jobs:
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
echo "Testing the main ci directory"
|
||||
python3 -m unittest discover -s . -p 'test_*.py'
|
||||
for dir in *_lambda/; do
|
||||
echo "Testing $dir"
|
||||
python3 -m unittest discover -s "$dir" -p 'test_*.py'
|
||||
done
|
||||
- name: PrepareRunConfig
|
||||
id: runconfig
|
||||
run: |
|
||||
|
4
.github/workflows/pull_request.yml
vendored
4
.github/workflows/pull_request.yml
vendored
@ -48,10 +48,6 @@ jobs:
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
echo "Testing the main ci directory"
|
||||
python3 -m unittest discover -s . -p 'test_*.py'
|
||||
for dir in *_lambda/; do
|
||||
echo "Testing $dir"
|
||||
python3 -m unittest discover -s "$dir" -p 'test_*.py'
|
||||
done
|
||||
- name: PrepareRunConfig
|
||||
id: runconfig
|
||||
run: |
|
||||
|
4
.github/workflows/release_branches.yml
vendored
4
.github/workflows/release_branches.yml
vendored
@ -33,10 +33,6 @@ jobs:
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
echo "Testing the main ci directory"
|
||||
python3 -m unittest discover -s . -p 'test_*.py'
|
||||
for dir in *_lambda/; do
|
||||
echo "Testing $dir"
|
||||
python3 -m unittest discover -s "$dir" -p 'test_*.py'
|
||||
done
|
||||
- name: PrepareRunConfig
|
||||
id: runconfig
|
||||
run: |
|
||||
|
2
contrib/grpc
vendored
2
contrib/grpc
vendored
@ -1 +1 @@
|
||||
Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df
|
||||
Subproject commit 1716359d2e28d304a250f9df0e6c0ccad03de8db
|
2
contrib/libunwind
vendored
2
contrib/libunwind
vendored
@ -1 +1 @@
|
||||
Subproject commit d6a01c46327e56fd86beb8aaa31591fcd9a6b7df
|
||||
Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2
|
@ -4,9 +4,6 @@ set(LIBUNWIND_CXX_SOURCES
|
||||
"${LIBUNWIND_SOURCE_DIR}/src/libunwind.cpp"
|
||||
"${LIBUNWIND_SOURCE_DIR}/src/Unwind-EHABI.cpp"
|
||||
"${LIBUNWIND_SOURCE_DIR}/src/Unwind-seh.cpp")
|
||||
if (APPLE)
|
||||
set(LIBUNWIND_CXX_SOURCES ${LIBUNWIND_CXX_SOURCES} "${LIBUNWIND_SOURCE_DIR}/src/Unwind_AppleExtras.cpp")
|
||||
endif ()
|
||||
|
||||
set(LIBUNWIND_C_SOURCES
|
||||
"${LIBUNWIND_SOURCE_DIR}/src/UnwindLevel1.c"
|
||||
@ -32,6 +29,7 @@ set_target_properties(unwind PROPERTIES FOLDER "contrib/libunwind-cmake")
|
||||
|
||||
target_include_directories(unwind SYSTEM BEFORE PUBLIC $<BUILD_INTERFACE:${LIBUNWIND_SOURCE_DIR}/include>)
|
||||
target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_NO_HEAP=1)
|
||||
target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_REMEMBER_STACK_ALLOC=1)
|
||||
# NOTE: from this macros sizeof(unw_context_t)/sizeof(unw_cursor_t) is depends, so it should be set always
|
||||
target_compile_definitions(unwind PUBLIC -D_LIBUNWIND_IS_NATIVE_ONLY)
|
||||
|
||||
|
@ -1298,7 +1298,6 @@ elseif(ARCH_PPC64LE)
|
||||
${OPENSSL_SOURCE_DIR}/crypto/camellia/camellia.c
|
||||
${OPENSSL_SOURCE_DIR}/crypto/camellia/cmll_cbc.c
|
||||
${OPENSSL_SOURCE_DIR}/crypto/chacha/chacha_enc.c
|
||||
${OPENSSL_SOURCE_DIR}/crypto/mem_clr.c
|
||||
${OPENSSL_SOURCE_DIR}/crypto/rc4/rc4_enc.c
|
||||
${OPENSSL_SOURCE_DIR}/crypto/rc4/rc4_skey.c
|
||||
${OPENSSL_SOURCE_DIR}/crypto/sha/keccak1600.c
|
||||
|
@ -26,7 +26,10 @@ RUN apt-get update \
|
||||
zstd \
|
||||
--yes --no-install-recommends \
|
||||
&& apt-get clean \
|
||||
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
|
||||
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \
|
||||
&& groupadd --system --gid 1000 clickhouse \
|
||||
&& useradd --system --gid 1000 --uid 1000 -m clickhouse
|
||||
# ^ For some reason, groupadd and useradd are needed for tests with 'expect', but I don't know, why.
|
||||
|
||||
COPY requirements.txt /
|
||||
RUN pip3 install --no-cache-dir -r /requirements.txt
|
||||
|
@ -9,7 +9,7 @@ trap 'kill $(jobs -pr) ||:' EXIT
|
||||
stage=${stage:-}
|
||||
|
||||
# Compiler version, normally set by Dockerfile
|
||||
export LLVM_VERSION=${LLVM_VERSION:-17}
|
||||
export LLVM_VERSION=${LLVM_VERSION:-18}
|
||||
|
||||
# A variable to pass additional flags to CMake.
|
||||
# Here we explicitly default it to nothing so that bash doesn't complain about
|
||||
|
@ -4,6 +4,9 @@
|
||||
source /setup_export_logs.sh
|
||||
set -e -x
|
||||
|
||||
MAX_RUN_TIME=${MAX_RUN_TIME:-3600}
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 3600 : MAX_RUN_TIME))
|
||||
|
||||
# Choose random timezone for this test run
|
||||
TZ="$(rg -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)"
|
||||
echo "Choosen random timezone $TZ"
|
||||
@ -242,7 +245,22 @@ function run_tests()
|
||||
}
|
||||
|
||||
export -f run_tests
|
||||
timeout "$MAX_RUN_TIME" bash -c run_tests ||:
|
||||
|
||||
function timeout_with_logging() {
|
||||
local exit_code=0
|
||||
|
||||
timeout -s TERM --preserve-status "${@}" || exit_code="${?}"
|
||||
|
||||
if [[ "${exit_code}" -eq "124" ]]
|
||||
then
|
||||
echo "The command 'timeout ${*}' has been killed by timeout"
|
||||
fi
|
||||
|
||||
return $exit_code
|
||||
}
|
||||
|
||||
TIMEOUT=$((MAX_RUN_TIME - 700))
|
||||
timeout_with_logging "$TIMEOUT" bash -c run_tests ||:
|
||||
|
||||
echo "Files in current directory"
|
||||
ls -la ./
|
||||
|
@ -12,12 +12,6 @@ MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 7200 : MAX_RUN_TIME))
|
||||
USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0}
|
||||
USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0}
|
||||
|
||||
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0
|
||||
|
||||
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
|
||||
RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0
|
||||
fi
|
||||
|
||||
# Choose random timezone for this test run.
|
||||
#
|
||||
# NOTE: that clickhouse-test will randomize session_timezone by itself as well
|
||||
@ -101,53 +95,6 @@ if [ "$NUM_TRIES" -gt "1" ]; then
|
||||
mkdir -p /var/run/clickhouse-server
|
||||
fi
|
||||
|
||||
# Run a CH instance to execute sequential tests on it in parallel with all other tests.
|
||||
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
|
||||
mkdir -p /var/run/clickhouse-server3 /etc/clickhouse-server3 /var/lib/clickhouse3
|
||||
cp -r -L /etc/clickhouse-server/* /etc/clickhouse-server3/
|
||||
|
||||
sudo chown clickhouse:clickhouse /var/run/clickhouse-server3 /var/lib/clickhouse3 /etc/clickhouse-server3/
|
||||
sudo chown -R clickhouse:clickhouse /etc/clickhouse-server3/*
|
||||
|
||||
function replace(){
|
||||
sudo find /etc/clickhouse-server3/ -type f -name '*.xml' -exec sed -i "$1" {} \;
|
||||
}
|
||||
|
||||
replace "s|<port>9000</port>|<port>19000</port>|g"
|
||||
replace "s|<port>9440</port>|<port>19440</port>|g"
|
||||
replace "s|<port>9988</port>|<port>19988</port>|g"
|
||||
replace "s|<port>9234</port>|<port>19234</port>|g"
|
||||
replace "s|<port>9181</port>|<port>19181</port>|g"
|
||||
replace "s|<https_port>8443</https_port>|<https_port>18443</https_port>|g"
|
||||
replace "s|<tcp_port>9000</tcp_port>|<tcp_port>19000</tcp_port>|g"
|
||||
replace "s|<tcp_port>9181</tcp_port>|<tcp_port>19181</tcp_port>|g"
|
||||
replace "s|<tcp_port_secure>9440</tcp_port_secure>|<tcp_port_secure>19440</tcp_port_secure>|g"
|
||||
replace "s|<tcp_with_proxy_port>9010</tcp_with_proxy_port>|<tcp_with_proxy_port>19010</tcp_with_proxy_port>|g"
|
||||
replace "s|<mysql_port>9004</mysql_port>|<mysql_port>19004</mysql_port>|g"
|
||||
replace "s|<postgresql_port>9005</postgresql_port>|<postgresql_port>19005</postgresql_port>|g"
|
||||
replace "s|<interserver_http_port>9009</interserver_http_port>|<interserver_http_port>19009</interserver_http_port>|g"
|
||||
replace "s|8123|18123|g"
|
||||
replace "s|/var/lib/clickhouse/|/var/lib/clickhouse3/|g"
|
||||
replace "s|/etc/clickhouse-server/|/etc/clickhouse-server3/|g"
|
||||
# distributed cache
|
||||
replace "s|<tcp_port>10001</tcp_port>|<tcp_port>10003</tcp_port>|g"
|
||||
replace "s|<tcp_port>10002</tcp_port>|<tcp_port>10004</tcp_port>|g"
|
||||
|
||||
sudo -E -u clickhouse /usr/bin/clickhouse server --daemon --config /etc/clickhouse-server3/config.xml \
|
||||
--pid-file /var/run/clickhouse-server3/clickhouse-server.pid \
|
||||
-- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \
|
||||
--logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \
|
||||
--tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \
|
||||
--prometheus.port 19988 --keeper_server.raft_configuration.server.port 19234 --keeper_server.tcp_port 19181 \
|
||||
--mysql_port 19004 --postgresql_port 19005
|
||||
|
||||
for _ in {1..100}
|
||||
do
|
||||
clickhouse-client --port 19000 --query "SELECT 1" && break
|
||||
sleep 1
|
||||
done
|
||||
fi
|
||||
|
||||
# simplest way to forward env variables to server
|
||||
sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server/config.xml --daemon --pid-file /var/run/clickhouse-server/clickhouse-server.pid
|
||||
|
||||
@ -183,9 +130,6 @@ if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
--keeper_server.tcp_port 29181 --keeper_server.server_id 3 \
|
||||
--prometheus.port 29988 \
|
||||
--macros.shard s2 # It doesn't work :(
|
||||
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours)
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited)
|
||||
fi
|
||||
|
||||
if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
|
||||
@ -210,9 +154,6 @@ if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
|
||||
--keeper_server.tcp_port 19181 --keeper_server.server_id 2 \
|
||||
--prometheus.port 19988 \
|
||||
--macros.replica r2 # It doesn't work :(
|
||||
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours)
|
||||
MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited)
|
||||
fi
|
||||
|
||||
# Wait for the server to start, but not for too long.
|
||||
@ -223,7 +164,6 @@ do
|
||||
done
|
||||
|
||||
setup_logs_replication
|
||||
|
||||
attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01
|
||||
|
||||
function fn_exists() {
|
||||
@ -284,11 +224,7 @@ function run_tests()
|
||||
else
|
||||
# All other configurations are OK.
|
||||
ADDITIONAL_OPTIONS+=('--jobs')
|
||||
ADDITIONAL_OPTIONS+=('5')
|
||||
fi
|
||||
|
||||
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
|
||||
ADDITIONAL_OPTIONS+=('--run-sequential-tests-in-parallel')
|
||||
ADDITIONAL_OPTIONS+=('7')
|
||||
fi
|
||||
|
||||
if [[ -n "$RUN_BY_HASH_NUM" ]] && [[ -n "$RUN_BY_HASH_TOTAL" ]]; then
|
||||
@ -373,9 +309,6 @@ done
|
||||
# Because it's the simplest way to read it when server has crashed.
|
||||
sudo clickhouse stop ||:
|
||||
|
||||
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
|
||||
sudo clickhouse stop --pid-path /var/run/clickhouse-server3 ||:
|
||||
fi
|
||||
|
||||
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||:
|
||||
@ -393,12 +326,6 @@ rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server.log ||:
|
||||
rg -A50 -Fa "============" /var/log/clickhouse-server/stderr.log ||:
|
||||
zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst &
|
||||
|
||||
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
|
||||
rg -Fa "<Fatal>" /var/log/clickhouse-server3/clickhouse-server.log ||:
|
||||
rg -A50 -Fa "============" /var/log/clickhouse-server3/stderr.log ||:
|
||||
zstd --threads=0 < /var/log/clickhouse-server3/clickhouse-server.log > /test_output/clickhouse-server3.log.zst &
|
||||
fi
|
||||
|
||||
data_path_config="--path=/var/lib/clickhouse/"
|
||||
if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then
|
||||
# We need s3 storage configuration (but it's more likely that clickhouse-local will fail for some reason)
|
||||
@ -419,10 +346,6 @@ if [ $failed_to_save_logs -ne 0 ]; then
|
||||
do
|
||||
clickhouse-local "$data_path_config" --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
|
||||
|
||||
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
|
||||
clickhouse-local --path /var/lib/clickhouse3/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.3.tsv.zst ||:
|
||||
fi
|
||||
|
||||
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||:
|
||||
clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||:
|
||||
@ -464,12 +387,6 @@ rm -rf /var/lib/clickhouse/data/system/*/
|
||||
tar -chf /test_output/store.tar /var/lib/clickhouse/store ||:
|
||||
tar -chf /test_output/metadata.tar /var/lib/clickhouse/metadata/*.sql ||:
|
||||
|
||||
if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then
|
||||
rm -rf /var/lib/clickhouse3/data/system/*/
|
||||
tar -chf /test_output/store.tar /var/lib/clickhouse3/store ||:
|
||||
tar -chf /test_output/metadata.tar /var/lib/clickhouse3/metadata/*.sql ||:
|
||||
fi
|
||||
|
||||
|
||||
if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
|
||||
rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server1.log ||:
|
||||
|
@ -17,6 +17,7 @@ ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18
|
||||
|
||||
RUN apt-get update \
|
||||
&& apt-get install \
|
||||
sudo \
|
||||
apt-transport-https \
|
||||
apt-utils \
|
||||
ca-certificates \
|
||||
|
@ -185,6 +185,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va
|
||||
- `--format, -f` – Use the specified default format to output the result.
|
||||
- `--vertical, -E` – If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `–format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables.
|
||||
- `--time, -t` – If specified, print the query execution time to ‘stderr’ in non-interactive mode.
|
||||
- `--memory-usage` – If specified, print memory usage to ‘stderr’ in non-interactive mode]. Possible values: 'none' - do not print memory usage, 'default' - print number of bytes, 'readable' - print memory usage in human-readable format.
|
||||
- `--stacktrace` – If specified, also print the stack trace if an exception occurs.
|
||||
- `--config-file` – The name of the configuration file.
|
||||
- `--secure` – If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl).
|
||||
@ -339,7 +340,7 @@ clickhouse-client clickhouse://some_user%40some_mail.com@localhost:9000
|
||||
Connect to one of provides hosts: `192.168.1.15`, `192.168.1.25`.
|
||||
|
||||
``` bash
|
||||
clickhouse-client clickhouse://192.168.1.15,192.168.1.25
|
||||
clickhouse-client clickhouse://192.168.1.15,192.168.1.25
|
||||
```
|
||||
|
||||
### Configuration Files {#configuration_files}
|
||||
@ -367,7 +368,7 @@ Example of a config file:
|
||||
```
|
||||
|
||||
Or the same config in a YAML format:
|
||||
|
||||
|
||||
```yaml
|
||||
user: username
|
||||
password: 'password'
|
||||
|
@ -67,6 +67,7 @@ The supported formats are:
|
||||
| [Prometheus](#prometheus) | ✗ | ✔ |
|
||||
| [Protobuf](#protobuf) | ✔ | ✔ |
|
||||
| [ProtobufSingle](#protobufsingle) | ✔ | ✔ |
|
||||
| [ProtobufList](#protobuflist) | ✔ | ✔ |
|
||||
| [Avro](#data-format-avro) | ✔ | ✔ |
|
||||
| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ |
|
||||
| [Parquet](#data-format-parquet) | ✔ | ✔ |
|
||||
@ -1952,6 +1953,35 @@ SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf
|
||||
|
||||
Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters.
|
||||
|
||||
## ProtobufList {#protobuflist}
|
||||
|
||||
Similar to Protobuf but rows are represented as a sequence of sub-messages contained in a message with fixed name "Envelope".
|
||||
|
||||
Usage example:
|
||||
|
||||
``` sql
|
||||
SELECT * FROM test.table FORMAT ProtobufList SETTINGS format_schema = 'schemafile:MessageType'
|
||||
```
|
||||
|
||||
``` bash
|
||||
cat protobuflist_messages.bin | clickhouse-client --query "INSERT INTO test.table FORMAT ProtobufList SETTINGS format_schema='schemafile:MessageType'"
|
||||
```
|
||||
|
||||
where the file `schemafile.proto` looks like this:
|
||||
|
||||
``` capnp
|
||||
syntax = "proto3";
|
||||
message Envelope {
|
||||
message MessageType {
|
||||
string name = 1;
|
||||
string surname = 2;
|
||||
uint32 birthDate = 3;
|
||||
repeated string phoneNumbers = 4;
|
||||
};
|
||||
MessageType row = 1;
|
||||
};
|
||||
```
|
||||
|
||||
## Avro {#data-format-avro}
|
||||
|
||||
[Apache Avro](https://avro.apache.org/) is a row-oriented data serialization framework developed within Apache’s Hadoop project.
|
||||
|
@ -124,7 +124,7 @@ which is equal to
|
||||
|
||||
#### Default values for from_env and from_zk attributes
|
||||
|
||||
It's possible to set the default value and substitute it only if the environment variable or zookeeper node is set using `replace="1"`.
|
||||
It's possible to set the default value and substitute it only if the environment variable or zookeeper node is set using `replace="1"` (must be declared before from_env).
|
||||
|
||||
With previous example, but `MAX_QUERY_SIZE` is unset:
|
||||
|
||||
@ -132,7 +132,7 @@ With previous example, but `MAX_QUERY_SIZE` is unset:
|
||||
<clickhouse>
|
||||
<profiles>
|
||||
<default>
|
||||
<max_query_size from_env="MAX_QUERY_SIZE" replace="1">150000</max_query_size>
|
||||
<max_query_size replace="1" from_env="MAX_QUERY_SIZE">150000</max_query_size>
|
||||
</default>
|
||||
</profiles>
|
||||
</clickhouse>
|
||||
|
35
docs/en/operations/system-tables/detached_tables.md
Normal file
35
docs/en/operations/system-tables/detached_tables.md
Normal file
@ -0,0 +1,35 @@
|
||||
---
|
||||
slug: /en/operations/system-tables/detached_tables
|
||||
---
|
||||
# detached_tables
|
||||
|
||||
Contains information of each detached table.
|
||||
|
||||
Columns:
|
||||
|
||||
- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in.
|
||||
|
||||
- `table` ([String](../../sql-reference/data-types/string.md)) — Table name.
|
||||
|
||||
- `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid (Atomic database).
|
||||
|
||||
- `metadata_path` ([String](../../sql-reference/data-types/string.md)) - Path to the table metadata in the file system.
|
||||
|
||||
- `is_permanently` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag indicates that the table was detached PERMANENTLY.
|
||||
|
||||
|
||||
**Example**
|
||||
|
||||
```sql
|
||||
SELECT * FROM system.detached_tables FORMAT Vertical;
|
||||
```
|
||||
|
||||
```text
|
||||
Row 1:
|
||||
──────
|
||||
database: base
|
||||
table: t1
|
||||
uuid: 81b1c20a-b7c6-4116-a2ce-7583fb6b6736
|
||||
metadata_path: /var/lib/clickhouse/store/461/461cf698-fd0b-406d-8c01-5d8fd5748a91/t1.sql
|
||||
is_permanently: 1
|
||||
```
|
@ -9,7 +9,6 @@ Columns:
|
||||
|
||||
- `name` ([String](../../sql-reference/data-types/string.md)) – The name of the function.
|
||||
- `is_aggregate` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Whether the function is an aggregate function.
|
||||
- `is_deterministic` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md))) - Whether the function is deterministic.
|
||||
- `case_insensitive`, ([UInt8](../../sql-reference/data-types/int-uint.md)) - Whether the function name can be used case-insensitively.
|
||||
- `alias_to`, ([String](../../sql-reference/data-types/string.md)) - The original function name, if the function name is an alias.
|
||||
- `create_query`, ([String](../../sql-reference/data-types/enum.md)) - Unused.
|
||||
|
@ -56,7 +56,6 @@ Functions:
|
||||
|
||||
## Related content
|
||||
|
||||
- [Reducing ClickHouse Storage Cost with the Low Cardinality Type – Lessons from an Instana Engineer](https://altinity.com/blog/2020-5-20-reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer)
|
||||
- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf)
|
||||
- Blog: [Optimizing ClickHouse with Schemas and Codecs](https://clickhouse.com/blog/optimize-clickhouse-codecs-compression-schema)
|
||||
- Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse)
|
||||
- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf)
|
||||
|
@ -76,7 +76,7 @@ WHERE macro = 'test';
|
||||
└───────┴──────────────┘
|
||||
```
|
||||
|
||||
## FQDN
|
||||
## fqdn
|
||||
|
||||
Returns the fully qualified domain name of the ClickHouse server.
|
||||
|
||||
@ -86,7 +86,7 @@ Returns the fully qualified domain name of the ClickHouse server.
|
||||
fqdn();
|
||||
```
|
||||
|
||||
Aliases: `fullHostName`, 'FQDN'.
|
||||
Aliases: `fullHostName`, `FQDN`.
|
||||
|
||||
**Returned value**
|
||||
|
||||
|
@ -567,12 +567,13 @@ While no standard or recommendation exists for the epoch of Snowflake IDs, imple
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
generateSnowflakeID([expr])
|
||||
generateSnowflakeID([expr, [machine_id]])
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `expr` — An arbitrary [expression](../../sql-reference/syntax.md#syntax-expressions) used to bypass [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in a query. The value of the expression has no effect on the returned Snowflake ID. Optional.
|
||||
- `machine_id` — A machine ID, the lowest 10 bits are used. [Int64](../data-types/int-uint.md). Optional.
|
||||
|
||||
**Returned value**
|
||||
|
||||
@ -608,6 +609,16 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2);
|
||||
└────────────────────────┴────────────────────────┘
|
||||
```
|
||||
|
||||
**Example with expression and a machine ID**
|
||||
|
||||
```
|
||||
SELECT generateSnowflakeID('expr', 1);
|
||||
|
||||
┌─generateSnowflakeID('expr', 1)─┐
|
||||
│ 7201148511606784002 │
|
||||
└────────────────────────────────┘
|
||||
```
|
||||
|
||||
## snowflakeToDateTime
|
||||
|
||||
:::warning
|
||||
|
@ -1,6 +1,6 @@
|
||||
---
|
||||
slug: /en/sql-reference/table-functions/azureBlobStorageCluster
|
||||
sidebar_position: 55
|
||||
sidebar_position: 15
|
||||
sidebar_label: azureBlobStorageCluster
|
||||
title: "azureBlobStorageCluster Table Function"
|
||||
---
|
||||
|
@ -356,7 +356,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7"
|
||||
|
||||
#### 新功能 {#new-feature-1}
|
||||
|
||||
- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。 这个新功能是由Altinity的特殊要求添加到错误修正版本中的。
|
||||
- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。
|
||||
[#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy)
|
||||
|
||||
### ClickHouse版本v20.1.2.4,2020-01-22 {#clickhouse-release-v20-1-2-4-2020-01-22}
|
||||
|
@ -82,14 +82,14 @@ FROM LEFT_RIGHT
|
||||
SELECT
|
||||
left,
|
||||
right,
|
||||
if(left < right, 'left is smaller than right', 'right is greater or equal than left') AS is_smaller
|
||||
if(left < right, 'left is smaller than right', 'right is smaller or equal than left') AS is_smaller
|
||||
FROM LEFT_RIGHT
|
||||
WHERE isNotNull(left) AND isNotNull(right)
|
||||
|
||||
┌─left─┬─right─┬─is_smaller──────────────────────────┐
|
||||
│ 1 │ 3 │ left is smaller than right │
|
||||
│ 2 │ 2 │ right is greater or equal than left │
|
||||
│ 3 │ 1 │ right is greater or equal than left │
|
||||
│ 2 │ 2 │ right is smaller or equal than left │
|
||||
│ 3 │ 1 │ right is smaller or equal than left │
|
||||
└──────┴───────┴─────────────────────────────────────┘
|
||||
```
|
||||
|
||||
|
@ -4,6 +4,9 @@ if (USE_CLANG_TIDY)
|
||||
set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}")
|
||||
endif ()
|
||||
|
||||
set(MAX_LINKER_MEMORY 3500)
|
||||
include(../cmake/limit_jobs.cmake)
|
||||
|
||||
include(${ClickHouse_SOURCE_DIR}/cmake/split_debug_symbols.cmake)
|
||||
|
||||
# The `clickhouse` binary is a multi purpose tool that contains multiple execution modes (client, server, etc.),
|
||||
|
@ -186,6 +186,8 @@ void Client::parseConnectionsCredentials(Poco::Util::AbstractConfiguration & con
|
||||
history_file = home_path + "/" + history_file.substr(1);
|
||||
config.setString("history_file", history_file);
|
||||
}
|
||||
if (config.has(prefix + ".accept-invalid-certificate"))
|
||||
config.setBool("accept-invalid-certificate", config.getBool(prefix + ".accept-invalid-certificate"));
|
||||
}
|
||||
|
||||
if (!connection_name.empty() && !connection_found)
|
||||
@ -277,6 +279,12 @@ void Client::initialize(Poco::Util::Application & self)
|
||||
else if (config().has("connection"))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "--connection was specified, but config does not exist");
|
||||
|
||||
if (config().has("accept-invalid-certificate"))
|
||||
{
|
||||
config().setString("openSSL.client.invalidCertificateHandler.name", "AcceptCertificateHandler");
|
||||
config().setString("openSSL.client.verificationMode", "none");
|
||||
}
|
||||
|
||||
/** getenv is thread-safe in Linux glibc and in all sane libc implementations.
|
||||
* But the standard does not guarantee that subsequent calls will not rewrite the value by returned pointer.
|
||||
*
|
||||
@ -732,7 +740,7 @@ bool Client::processWithFuzzing(const String & full_query)
|
||||
}
|
||||
if (auto *q = orig_ast->as<ASTSetQuery>())
|
||||
{
|
||||
if (auto *setDialect = q->changes.tryGet("dialect"); setDialect && setDialect->safeGet<String>() == "kusto")
|
||||
if (auto *set_dialect = q->changes.tryGet("dialect"); set_dialect && set_dialect->safeGet<String>() == "kusto")
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
<!-- Config set into /etc/clickhouse-client/. It's used if no other configs are found. -->
|
||||
<config>
|
||||
<!-- Shorthand for self-signed combination in openSSL section below: <accept-invalid-certificate>1</accept-invalid-certificate> -->
|
||||
<openSSL>
|
||||
<client> <!-- Used for connection to server's secure tcp port -->
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
@ -72,6 +73,7 @@
|
||||
|
||||
Default: "hostname" will be used. -->
|
||||
<name>default</name>
|
||||
<!-- For self-signed server certificate when connecting to secure tcp: <accept-invalid-certificate>1</accept-invalid-certificate> -->
|
||||
<!-- Host that will be used for connection. -->
|
||||
<hostname>127.0.0.1</hostname>
|
||||
<port>9000</port>
|
||||
|
@ -45,16 +45,17 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv)
|
||||
keeper_context->setDigestEnabled(true);
|
||||
keeper_context->setSnapshotDisk(std::make_shared<DiskLocal>("Keeper-snapshots", options["output-dir"].as<std::string>()));
|
||||
|
||||
DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false);
|
||||
/// TODO(hanfei): support rocksdb here
|
||||
DB::KeeperMemoryStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false);
|
||||
|
||||
DB::deserializeKeeperStorageFromSnapshotsDir(storage, options["zookeeper-snapshots-dir"].as<std::string>(), logger);
|
||||
storage.initializeSystemNodes();
|
||||
|
||||
DB::deserializeLogsAndApplyToStorage(storage, options["zookeeper-logs-dir"].as<std::string>(), logger);
|
||||
DB::SnapshotMetadataPtr snapshot_meta = std::make_shared<DB::SnapshotMetadata>(storage.getZXID(), 1, std::make_shared<nuraft::cluster_config>());
|
||||
DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta);
|
||||
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, snapshot_meta);
|
||||
|
||||
DB::KeeperSnapshotManager manager(1, keeper_context);
|
||||
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(1, keeper_context);
|
||||
auto snp = manager.serializeSnapshotToBuffer(snapshot);
|
||||
auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID());
|
||||
std::cout << "Snapshot serialized to path:" << fs::path(file_info->disk->getPath()) / file_info->path << std::endl;
|
||||
|
@ -423,6 +423,7 @@ void LocalServer::connect()
|
||||
{
|
||||
connection_parameters = ConnectionParameters(getClientConfiguration(), "localhost");
|
||||
|
||||
/// This is needed for table function input(...).
|
||||
ReadBuffer * in;
|
||||
auto table_file = getClientConfiguration().getString("table-file", "-");
|
||||
if (table_file == "-" || table_file == "stdin")
|
||||
|
@ -2731,8 +2731,7 @@ void Server::createInterserverServers(
|
||||
|
||||
void Server::stopServers(
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
const ServerType & server_type
|
||||
) const
|
||||
const ServerType & server_type) const
|
||||
{
|
||||
LoggerRawPtr log = &logger();
|
||||
|
||||
|
@ -129,8 +129,7 @@ private:
|
||||
|
||||
void stopServers(
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
const ServerType & server_type
|
||||
) const;
|
||||
const ServerType & server_type) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -118,10 +118,10 @@ AggregateFunctionPtr createAggregateFunctionAnalysisOfVariance(const std::string
|
||||
void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory & factory)
|
||||
{
|
||||
AggregateFunctionProperties properties = { .is_order_dependent = false };
|
||||
factory.registerFunction("analysisOfVariance", {createAggregateFunctionAnalysisOfVariance, properties}, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("analysisOfVariance", {createAggregateFunctionAnalysisOfVariance, properties}, AggregateFunctionFactory::Case::Insensitive);
|
||||
|
||||
/// This is widely used term
|
||||
factory.registerAlias("anova", "analysisOfVariance", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("anova", "analysisOfVariance", AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -361,9 +361,9 @@ void registerAggregateFunctionsAny(AggregateFunctionFactory & factory)
|
||||
AggregateFunctionProperties default_properties = {.returns_default_when_only_null = false, .is_order_dependent = true};
|
||||
|
||||
factory.registerFunction("any", {createAggregateFunctionAny, default_properties});
|
||||
factory.registerAlias("any_value", "any", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("first_value", "any", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("any_value", "any", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("first_value", "any", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerFunction("anyLast", {createAggregateFunctionAnyLast, default_properties});
|
||||
factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
}
|
||||
|
@ -221,11 +221,11 @@ void registerAggregateFunctionsAnyRespectNulls(AggregateFunctionFactory & factor
|
||||
= {.returns_default_when_only_null = false, .is_order_dependent = true, .is_window_function = true};
|
||||
|
||||
factory.registerFunction("any_respect_nulls", {createAggregateFunctionAnyRespectNulls, default_properties_for_respect_nulls});
|
||||
factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
|
||||
|
||||
factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls});
|
||||
factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive);
|
||||
|
||||
/// Must happen after registering any and anyLast
|
||||
factory.registerNullsActionTransformation("any", "any_respect_nulls");
|
||||
|
@ -46,6 +46,6 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const
|
||||
|
||||
void registerAggregateFunctionAvg(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
}
|
||||
|
@ -234,9 +234,9 @@ void registerAggregateFunctionsBitwise(AggregateFunctionFactory & factory)
|
||||
factory.registerFunction("groupBitXor", createAggregateFunctionBitwise<AggregateFunctionGroupBitXorData>);
|
||||
|
||||
/// Aliases for compatibility with MySQL.
|
||||
factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -9,7 +9,7 @@ template <typename T1, typename T2> using AggregateFunctionCorr = AggregateFunct
|
||||
|
||||
void registerAggregateFunctionsStatisticsCorr(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction("corr", createAggregateFunctionStatisticsBinary<AggregateFunctionCorr, StatisticsFunctionKind::corr>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("corr", createAggregateFunctionStatisticsBinary<AggregateFunctionCorr, StatisticsFunctionKind::corr>, AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -37,7 +37,7 @@ AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, cons
|
||||
void registerAggregateFunctionCount(AggregateFunctionFactory & factory)
|
||||
{
|
||||
AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false };
|
||||
factory.registerFunction("count", {createAggregateFunctionCount, properties}, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("count", {createAggregateFunctionCount, properties}, AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -13,8 +13,8 @@ void registerAggregateFunctionsStatisticsCovar(AggregateFunctionFactory & factor
|
||||
factory.registerFunction("covarPop", createAggregateFunctionStatisticsBinary<AggregateFunctionCovar, StatisticsFunctionKind::covarPop>);
|
||||
|
||||
/// Synonyms for compatibility.
|
||||
factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -29,7 +29,7 @@ const String & getAggregateFunctionCanonicalNameIfAny(const String & name)
|
||||
return AggregateFunctionFactory::instance().getCanonicalNameIfAny(name);
|
||||
}
|
||||
|
||||
void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, CaseSensitiveness case_sensitiveness)
|
||||
void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, Case case_sensitiveness)
|
||||
{
|
||||
if (creator_with_properties.creator == nullptr)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: "
|
||||
@ -39,7 +39,7 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: the aggregate function name '{}' is not unique",
|
||||
name);
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive)
|
||||
if (case_sensitiveness == Case::Insensitive)
|
||||
{
|
||||
auto key = Poco::toLower(name);
|
||||
if (!case_insensitive_aggregate_functions.emplace(key, creator_with_properties).second)
|
||||
|
@ -60,7 +60,7 @@ public:
|
||||
void registerFunction(
|
||||
const String & name,
|
||||
Value creator,
|
||||
CaseSensitiveness case_sensitiveness = CaseSensitive);
|
||||
Case case_sensitiveness = Case::Sensitive);
|
||||
|
||||
/// Register how to transform from one aggregate function to other based on NullsAction
|
||||
/// Registers them both ways:
|
||||
|
@ -840,8 +840,8 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory)
|
||||
AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true };
|
||||
|
||||
factory.registerFunction("groupArray", { createAggregateFunctionGroupArray<false>, properties });
|
||||
factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties });
|
||||
factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray<true>, properties });
|
||||
}
|
||||
|
@ -150,8 +150,18 @@ public:
|
||||
|
||||
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
|
||||
{
|
||||
readVarUInt(this->data(place).version, buf);
|
||||
this->data(place).value.read(buf);
|
||||
auto & set = this->data(place).value;
|
||||
auto & version = this->data(place).version;
|
||||
size_t size;
|
||||
readVarUInt(version, buf);
|
||||
readVarUInt(size, buf);
|
||||
set.reserve(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
int key;
|
||||
readIntBinary(key, buf);
|
||||
set.insert(key);
|
||||
}
|
||||
}
|
||||
|
||||
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
|
||||
@ -292,7 +302,7 @@ public:
|
||||
}
|
||||
return new_map;
|
||||
};
|
||||
auto new_map = rhs_value.size() < set.size() ? create_new_map(rhs_value, set) : create_new_map(set, rhs_value);
|
||||
auto new_map = create_new_map(set, rhs_value);
|
||||
set = std::move(new_map);
|
||||
}
|
||||
}
|
||||
@ -316,11 +326,9 @@ public:
|
||||
readVarUInt(version, buf);
|
||||
readVarUInt(size, buf);
|
||||
set.reserve(size);
|
||||
UInt64 elem_version;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto key = readStringBinaryInto(*arena, buf);
|
||||
readVarUInt(elem_version, buf);
|
||||
set.insert(key);
|
||||
}
|
||||
}
|
||||
|
@ -277,7 +277,7 @@ void registerAggregateFunctionGroupConcat(AggregateFunctionFactory & factory)
|
||||
AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true };
|
||||
|
||||
factory.registerFunction("groupConcat", { createAggregateFunctionGroupConcat, properties });
|
||||
factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -350,7 +350,7 @@ AggregateFunctionPtr createAggregateFunctionKolmogorovSmirnovTest(
|
||||
|
||||
void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction("kolmogorovSmirnovTest", createAggregateFunctionKolmogorovSmirnovTest, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("kolmogorovSmirnovTest", createAggregateFunctionKolmogorovSmirnovTest, AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -15,11 +15,11 @@ void registerAggregateFunctionsStatisticsSecondMoment(AggregateFunctionFactory &
|
||||
factory.registerFunction("stddevPop", createAggregateFunctionStatisticsUnary<AggregateFunctionSecondMoment, StatisticsFunctionKind::stddevPop>);
|
||||
|
||||
/// Synonyms for compatibility.
|
||||
factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("STD", "stddevPop", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("STD", "stddevPop", AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -72,7 +72,7 @@ AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const
|
||||
|
||||
void registerAggregateFunctionSum(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction("sum", createAggregateFunctionSum<AggregateFunctionSumSimple>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("sum", createAggregateFunctionSum<AggregateFunctionSumSimple>, AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerFunction("sumWithOverflow", createAggregateFunctionSum<AggregateFunctionSumWithOverflow>);
|
||||
factory.registerFunction("sumKahan", createAggregateFunctionSum<AggregateFunctionSumKahan>);
|
||||
}
|
||||
|
@ -535,9 +535,9 @@ void registerAggregateFunctionTopK(AggregateFunctionFactory & factory)
|
||||
|
||||
factory.registerFunction("topK", { createAggregateFunctionTopK<false, false>, properties });
|
||||
factory.registerFunction("topKWeighted", { createAggregateFunctionTopK<true, false>, properties });
|
||||
factory.registerFunction("approx_top_k", { createAggregateFunctionTopK<false, true>, properties }, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("approx_top_sum", { createAggregateFunctionTopK<true, true>, properties }, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerAlias("approx_top_count", "approx_top_k", AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("approx_top_k", { createAggregateFunctionTopK<false, true>, properties }, AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerFunction("approx_top_sum", { createAggregateFunctionTopK<true, true>, properties }, AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerAlias("approx_top_count", "approx_top_k", AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -195,8 +195,8 @@ AggregateFunctionPtr createAggregateFunctionMinMax(
|
||||
|
||||
void registerAggregateFunctionsMinMax(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction("min", createAggregateFunctionMinMax<true>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("max", createAggregateFunctionMinMax<false>, AggregateFunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction("min", createAggregateFunctionMinMax<true>, AggregateFunctionFactory::Case::Insensitive);
|
||||
factory.registerFunction("max", createAggregateFunctionMinMax<false>, AggregateFunctionFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,6 +12,7 @@
|
||||
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -616,6 +617,7 @@ private:
|
||||
bool is_any_nullable = false;
|
||||
Tuple args;
|
||||
args.reserve(equals_functions.size());
|
||||
DataTypes tuple_element_types;
|
||||
/// first we create tuple from RHS of equals functions
|
||||
for (const auto & equals : equals_functions)
|
||||
{
|
||||
@ -628,16 +630,18 @@ private:
|
||||
if (const auto * rhs_literal = equals_arguments[1]->as<ConstantNode>())
|
||||
{
|
||||
args.push_back(rhs_literal->getValue());
|
||||
tuple_element_types.push_back(rhs_literal->getResultType());
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto * lhs_literal = equals_arguments[0]->as<ConstantNode>();
|
||||
assert(lhs_literal);
|
||||
args.push_back(lhs_literal->getValue());
|
||||
tuple_element_types.push_back(lhs_literal->getResultType());
|
||||
}
|
||||
}
|
||||
|
||||
auto rhs_node = std::make_shared<ConstantNode>(std::move(args));
|
||||
auto rhs_node = std::make_shared<ConstantNode>(std::move(args), std::make_shared<DataTypeTuple>(std::move(tuple_element_types)));
|
||||
|
||||
auto in_function = std::make_shared<FunctionNode>("in");
|
||||
|
||||
|
@ -2919,6 +2919,17 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
|
||||
|
||||
resolveExpressionNode(in_second_argument, scope, false /*allow_lambda_expression*/, true /*allow_table_expression*/);
|
||||
}
|
||||
|
||||
/// Edge case when the first argument of IN is scalar subquery.
|
||||
auto & in_first_argument = function_in_arguments_nodes[0];
|
||||
auto first_argument_type = in_first_argument->getNodeType();
|
||||
if (first_argument_type == QueryTreeNodeType::QUERY || first_argument_type == QueryTreeNodeType::UNION)
|
||||
{
|
||||
IdentifierResolveScope subquery_scope(in_first_argument, &scope /*parent_scope*/);
|
||||
subquery_scope.subquery_depth = scope.subquery_depth + 1;
|
||||
|
||||
evaluateScalarSubqueryIfNeeded(in_first_argument, subquery_scope);
|
||||
}
|
||||
}
|
||||
|
||||
/// Initialize function argument columns
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Common/NetException.h>
|
||||
#include <Common/SignalHandlers.h>
|
||||
#include <Common/tryGetFileNameByFileDescriptor.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
@ -302,7 +303,13 @@ public:
|
||||
};
|
||||
|
||||
|
||||
ClientBase::~ClientBase() = default;
|
||||
ClientBase::~ClientBase()
|
||||
{
|
||||
writeSignalIDtoSignalPipe(SignalListener::StopThread);
|
||||
signal_listener_thread.join();
|
||||
HandledSignals::instance().reset();
|
||||
}
|
||||
|
||||
ClientBase::ClientBase(
|
||||
int in_fd_,
|
||||
int out_fd_,
|
||||
@ -2104,9 +2111,18 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
|
||||
progress_table.writeFinalTable();
|
||||
output_stream << std::endl << std::endl;
|
||||
}
|
||||
else if (getClientConfiguration().getBool("print-time-to-stderr", false))
|
||||
else
|
||||
{
|
||||
error_stream << progress_indication.elapsedSeconds() << "\n";
|
||||
const auto & config = getClientConfiguration();
|
||||
if (config.getBool("print-time-to-stderr", false))
|
||||
error_stream << progress_indication.elapsedSeconds() << "\n";
|
||||
|
||||
const auto & print_memory_mode = config.getString("print-memory-to-stderr", "");
|
||||
auto peak_memeory_usage = std::max<Int64>(progress_indication.getMemoryUsage().peak, 0);
|
||||
if (print_memory_mode == "default")
|
||||
error_stream << peak_memeory_usage << "\n";
|
||||
else if (print_memory_mode == "readable")
|
||||
error_stream << formatReadableSizeWithBinarySuffix(peak_memeory_usage) << "\n";
|
||||
}
|
||||
|
||||
if (!is_interactive && getClientConfiguration().getBool("print-num-processed-rows", false))
|
||||
@ -3071,6 +3087,7 @@ void ClientBase::init(int argc, char ** argv)
|
||||
("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.")
|
||||
("wait_for_suggestions_to_load", "Load suggestion data synchonously.")
|
||||
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
|
||||
("memory-usage", po::value<std::string>()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'")
|
||||
|
||||
("echo", "in batch mode, print query before execution")
|
||||
|
||||
@ -3097,6 +3114,8 @@ void ClientBase::init(int argc, char ** argv)
|
||||
("max_memory_usage_in_client", po::value<std::string>(), "Set memory limit in client/local server")
|
||||
|
||||
("fuzzer-args", po::value<std::string>(), "Command line arguments for the LLVM's libFuzzer driver. Only relevant if the application is compiled with libFuzzer.")
|
||||
|
||||
("client_logs_file", po::value<std::string>(), "Path to a file for writing client logs. Currently we only have fatal logs (when the client crashes)")
|
||||
;
|
||||
|
||||
addOptions(options_description);
|
||||
@ -3156,6 +3175,14 @@ void ClientBase::init(int argc, char ** argv)
|
||||
/// Output execution time to stderr in batch mode.
|
||||
if (options.count("time"))
|
||||
getClientConfiguration().setBool("print-time-to-stderr", true);
|
||||
if (options.count("memory-usage"))
|
||||
{
|
||||
const auto & memory_usage_mode = options["memory-usage"].as<std::string>();
|
||||
if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable")
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode);
|
||||
getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode);
|
||||
}
|
||||
|
||||
if (options.count("query"))
|
||||
queries = options["query"].as<std::vector<std::string>>();
|
||||
if (options.count("query_id"))
|
||||
@ -3271,6 +3298,25 @@ void ClientBase::init(int argc, char ** argv)
|
||||
total_memory_tracker.setDescription("(total)");
|
||||
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
||||
}
|
||||
|
||||
/// Print stacktrace in case of crash
|
||||
HandledSignals::instance().setupTerminateHandler();
|
||||
HandledSignals::instance().setupCommonDeadlySignalHandlers();
|
||||
/// We don't setup signal handlers for SIGINT, SIGQUIT, SIGTERM because we don't
|
||||
/// have an option for client to shutdown gracefully.
|
||||
|
||||
fatal_channel_ptr = new Poco::SplitterChannel;
|
||||
fatal_console_channel_ptr = new Poco::ConsoleChannel;
|
||||
fatal_channel_ptr->addChannel(fatal_console_channel_ptr);
|
||||
if (options.count("client_logs_file"))
|
||||
{
|
||||
fatal_file_channel_ptr = new Poco::SimpleFileChannel(options["client_logs_file"].as<std::string>());
|
||||
fatal_channel_ptr->addChannel(fatal_file_channel_ptr);
|
||||
}
|
||||
|
||||
fatal_log = createLogger("ClientBase", fatal_channel_ptr.get(), Poco::Message::PRIO_FATAL);
|
||||
signal_listener = std::make_unique<SignalListener>(nullptr, fatal_log);
|
||||
signal_listener_thread.start(*signal_listener);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -13,6 +13,9 @@
|
||||
#include <Core/ExternalTable.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Poco/ConsoleChannel.h>
|
||||
#include <Poco/SimpleFileChannel.h>
|
||||
#include <Poco/SplitterChannel.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Client/Suggest.h>
|
||||
#include <boost/program_options.hpp>
|
||||
@ -213,6 +216,13 @@ protected:
|
||||
SharedContextHolder shared_context;
|
||||
ContextMutablePtr global_context;
|
||||
|
||||
LoggerPtr fatal_log;
|
||||
Poco::AutoPtr<Poco::SplitterChannel> fatal_channel_ptr;
|
||||
Poco::AutoPtr<Poco::Channel> fatal_console_channel_ptr;
|
||||
Poco::AutoPtr<Poco::Channel> fatal_file_channel_ptr;
|
||||
Poco::Thread signal_listener_thread;
|
||||
std::unique_ptr<Poco::Runnable> signal_listener;
|
||||
|
||||
bool is_interactive = false; /// Use either interactive line editing interface or batch mode.
|
||||
bool is_multiquery = false;
|
||||
bool delayed_interactive = false;
|
||||
|
@ -54,8 +54,6 @@ public:
|
||||
|
||||
struct ReplicaInfo
|
||||
{
|
||||
bool collaborate_with_initiator{false};
|
||||
size_t all_replicas_count{0};
|
||||
size_t number_of_current_replica{0};
|
||||
};
|
||||
|
||||
|
@ -175,4 +175,5 @@ private:
|
||||
|
||||
ReadBuffer * in;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -142,13 +142,12 @@ void MultiplexedConnections::sendQuery(
|
||||
modified_settings.group_by_two_level_threshold = 0;
|
||||
modified_settings.group_by_two_level_threshold_bytes = 0;
|
||||
}
|
||||
}
|
||||
|
||||
if (replica_info)
|
||||
{
|
||||
client_info.collaborate_with_initiator = true;
|
||||
client_info.count_participating_replicas = replica_info->all_replicas_count;
|
||||
client_info.number_of_current_replica = replica_info->number_of_current_replica;
|
||||
}
|
||||
if (replica_info)
|
||||
{
|
||||
client_info.collaborate_with_initiator = true;
|
||||
client_info.number_of_current_replica = replica_info->number_of_current_replica;
|
||||
}
|
||||
|
||||
/// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting.
|
||||
|
@ -1101,4 +1101,10 @@ void ColumnObject::finalize()
|
||||
checkObjectHasNoAmbiguosPaths(getKeys());
|
||||
}
|
||||
|
||||
void ColumnObject::updateHashFast(SipHash & hash) const
|
||||
{
|
||||
for (const auto & entry : subcolumns)
|
||||
for (auto & part : entry->data.data)
|
||||
part->updateHashFast(hash);
|
||||
}
|
||||
}
|
||||
|
@ -253,7 +253,7 @@ public:
|
||||
const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); }
|
||||
void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); }
|
||||
void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); }
|
||||
void updateHashFast(SipHash &) const override { throwMustBeConcrete(); }
|
||||
void updateHashFast(SipHash & hash) const override;
|
||||
void expand(const Filter &, bool) override { throwMustBeConcrete(); }
|
||||
bool hasEqualValues() const override { throwMustBeConcrete(); }
|
||||
size_t byteSizeAt(size_t) const override { throwMustBeConcrete(); }
|
||||
|
@ -201,6 +201,7 @@ bool ColumnTuple::tryInsert(const Field & x)
|
||||
return false;
|
||||
}
|
||||
}
|
||||
++column_length;
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -236,6 +237,7 @@ void ColumnTuple::doInsertManyFrom(const IColumn & src, size_t position, size_t
|
||||
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
columns[i]->insertManyFrom(*src_tuple.columns[i], position, length);
|
||||
column_length += length;
|
||||
}
|
||||
|
||||
void ColumnTuple::insertDefault()
|
||||
|
@ -187,12 +187,9 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
|
||||
#if USE_GWP_ASAN
|
||||
if (unlikely(GWPAsan::GuardedAlloc.shouldSample()))
|
||||
{
|
||||
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
|
||||
if (void * ptr = GWPAsan::GuardedAlloc.allocate(new_size, alignment))
|
||||
{
|
||||
auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
|
||||
trace_free.onFree(buf, old_size);
|
||||
|
||||
memcpy(ptr, buf, std::min(old_size, new_size));
|
||||
free(buf, old_size);
|
||||
trace_alloc.onAlloc(buf, new_size);
|
||||
@ -209,6 +206,7 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
|
||||
}
|
||||
else
|
||||
{
|
||||
[[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed);
|
||||
}
|
||||
}
|
||||
@ -231,13 +229,17 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
|
||||
if (alignment <= MALLOC_MIN_ALIGNMENT)
|
||||
{
|
||||
/// Resize malloc'd memory region with no special alignment requirement.
|
||||
auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
/// Realloc can do 2 possible things:
|
||||
/// - expand existing memory region
|
||||
/// - allocate new memory block and free the old one
|
||||
/// Because we don't know which option will be picked we need to make sure there is enough
|
||||
/// memory for all options
|
||||
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
|
||||
trace_free.onFree(buf, old_size);
|
||||
|
||||
void * new_buf = ::realloc(buf, new_size);
|
||||
if (nullptr == new_buf)
|
||||
{
|
||||
[[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
throw DB::ErrnoException(
|
||||
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY,
|
||||
"Allocator: Cannot realloc from {} to {}",
|
||||
@ -246,6 +248,8 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
|
||||
}
|
||||
|
||||
buf = new_buf;
|
||||
auto trace_free = CurrentMemoryTracker::free(old_size);
|
||||
trace_free.onFree(buf, old_size);
|
||||
trace_alloc.onAlloc(buf, new_size);
|
||||
|
||||
if constexpr (clear_memory)
|
||||
|
187
src/Common/CollectionOfDerived.h
Normal file
187
src/Common/CollectionOfDerived.h
Normal file
@ -0,0 +1,187 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/defines.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <typeindex>
|
||||
#include <vector>
|
||||
#include <string>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/* This is a collections of objects derived from ItemBase.
|
||||
* Collection contains no more than one instance for each derived type.
|
||||
* The derived type is used to access the instance.
|
||||
*/
|
||||
|
||||
template<class ItemBase>
|
||||
class CollectionOfDerivedItems
|
||||
{
|
||||
public:
|
||||
using Self = CollectionOfDerivedItems<ItemBase>;
|
||||
using ItemPtr = std::shared_ptr<ItemBase>;
|
||||
|
||||
private:
|
||||
struct Rec
|
||||
{
|
||||
std::type_index type_idx;
|
||||
ItemPtr ptr;
|
||||
|
||||
bool operator<(const Rec & other) const
|
||||
{
|
||||
return type_idx < other.type_idx;
|
||||
}
|
||||
|
||||
bool operator<(const std::type_index & value) const
|
||||
{
|
||||
return type_idx < value;
|
||||
}
|
||||
|
||||
bool operator==(const Rec & other) const
|
||||
{
|
||||
return type_idx == other.type_idx;
|
||||
}
|
||||
};
|
||||
using Records = std::vector<Rec>;
|
||||
|
||||
public:
|
||||
void swap(Self & other) noexcept
|
||||
{
|
||||
records.swap(other.records);
|
||||
}
|
||||
|
||||
void clear()
|
||||
{
|
||||
records.clear();
|
||||
}
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
return records.empty();
|
||||
}
|
||||
|
||||
size_t size() const
|
||||
{
|
||||
return records.size();
|
||||
}
|
||||
|
||||
Self clone() const
|
||||
{
|
||||
Self result;
|
||||
result.records.reserve(records.size());
|
||||
for (const auto & rec : records)
|
||||
result.records.emplace_back(rec.type_idx, rec.ptr->clone());
|
||||
return result;
|
||||
}
|
||||
|
||||
// append items for other inscnace only if there is no such item in current instance
|
||||
void appendIfUniq(Self && other)
|
||||
{
|
||||
auto middle_idx = records.size();
|
||||
std::move(other.records.begin(), other.records.end(), std::back_inserter(records));
|
||||
// merge is stable
|
||||
std::inplace_merge(records.begin(), records.begin() + middle_idx, records.end());
|
||||
// remove duplicates
|
||||
records.erase(std::unique(records.begin(), records.end()), records.end());
|
||||
|
||||
assert(std::is_sorted(records.begin(), records.end()));
|
||||
assert(isUniqTypes());
|
||||
}
|
||||
|
||||
template <class T>
|
||||
void add(std::shared_ptr<T> info)
|
||||
{
|
||||
static_assert(std::is_base_of_v<ItemBase, T>, "Template parameter must inherit items base class");
|
||||
return addImpl(std::type_index(typeid(T)), std::move(info));
|
||||
}
|
||||
|
||||
template <class T>
|
||||
std::shared_ptr<T> get() const
|
||||
{
|
||||
static_assert(std::is_base_of_v<ItemBase, T>, "Template parameter must inherit items base class");
|
||||
auto it = getImpl(std::type_index(typeid(T)));
|
||||
if (it == records.cend())
|
||||
return nullptr;
|
||||
auto cast = std::dynamic_pointer_cast<T>(it->ptr);
|
||||
chassert(cast);
|
||||
return cast;
|
||||
}
|
||||
|
||||
template <class T>
|
||||
std::shared_ptr<T> extract()
|
||||
{
|
||||
static_assert(std::is_base_of_v<ItemBase, T>, "Template parameter must inherit items base class");
|
||||
auto it = getImpl(std::type_index(typeid(T)));
|
||||
if (it == records.cend())
|
||||
return nullptr;
|
||||
auto cast = std::dynamic_pointer_cast<T>(it->ptr);
|
||||
chassert(cast);
|
||||
|
||||
records.erase(it);
|
||||
return cast;
|
||||
}
|
||||
|
||||
std::string debug() const
|
||||
{
|
||||
std::string result;
|
||||
|
||||
for (auto & rec : records)
|
||||
{
|
||||
result.append(rec.type_idx.name());
|
||||
result.append(" ");
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private:
|
||||
bool isUniqTypes() const
|
||||
{
|
||||
auto uniq_it = std::adjacent_find(records.begin(), records.end());
|
||||
return uniq_it == records.end();
|
||||
}
|
||||
|
||||
void addImpl(std::type_index type_idx, ItemPtr item)
|
||||
{
|
||||
auto it = std::lower_bound(records.begin(), records.end(), type_idx);
|
||||
|
||||
if (it == records.end())
|
||||
{
|
||||
records.emplace_back(type_idx, item);
|
||||
return;
|
||||
}
|
||||
|
||||
if (it->type_idx == type_idx)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "inserted items must be unique by their type, type {} is inserted twice", type_idx.name());
|
||||
|
||||
|
||||
records.emplace(it, type_idx, item);
|
||||
}
|
||||
|
||||
Records::const_iterator getImpl(std::type_index type_idx) const
|
||||
{
|
||||
auto it = std::lower_bound(records.cbegin(), records.cend(), type_idx);
|
||||
|
||||
if (it == records.cend())
|
||||
return records.cend();
|
||||
|
||||
if (it->type_idx != type_idx)
|
||||
return records.cend();
|
||||
|
||||
return it;
|
||||
}
|
||||
|
||||
Records records;
|
||||
};
|
||||
|
||||
}
|
@ -39,16 +39,16 @@ protected:
|
||||
|
||||
public:
|
||||
/// For compatibility with SQL, it's possible to specify that certain function name is case insensitive.
|
||||
enum CaseSensitiveness
|
||||
enum Case
|
||||
{
|
||||
CaseSensitive,
|
||||
CaseInsensitive
|
||||
Sensitive,
|
||||
Insensitive
|
||||
};
|
||||
|
||||
/** Register additional name for value
|
||||
* real_name have to be already registered.
|
||||
*/
|
||||
void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive)
|
||||
void registerAlias(const String & alias_name, const String & real_name, Case case_sensitiveness = Sensitive)
|
||||
{
|
||||
const auto & creator_map = getMap();
|
||||
const auto & case_insensitive_creator_map = getCaseInsensitiveMap();
|
||||
@ -66,12 +66,12 @@ public:
|
||||
}
|
||||
|
||||
/// We need sure the real_name exactly exists when call the function directly.
|
||||
void registerAliasUnchecked(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive)
|
||||
void registerAliasUnchecked(const String & alias_name, const String & real_name, Case case_sensitiveness = Sensitive)
|
||||
{
|
||||
String alias_name_lowercase = Poco::toLower(alias_name);
|
||||
const String factory_name = getFactoryName();
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive)
|
||||
if (case_sensitiveness == Insensitive)
|
||||
{
|
||||
if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", factory_name, alias_name);
|
||||
|
@ -116,32 +116,32 @@ public:
|
||||
return elements;
|
||||
}
|
||||
|
||||
bool exists(const std::string & path) const override
|
||||
bool exists(const std::string & file_name) const override
|
||||
{
|
||||
return fs::exists(getPath(path));
|
||||
return fs::exists(getPath(file_name));
|
||||
}
|
||||
|
||||
std::string read(const std::string & path) const override
|
||||
std::string read(const std::string & file_name) const override
|
||||
{
|
||||
ReadBufferFromFile in(getPath(path));
|
||||
ReadBufferFromFile in(getPath(file_name));
|
||||
std::string data;
|
||||
readStringUntilEOF(data, in);
|
||||
return data;
|
||||
}
|
||||
|
||||
void write(const std::string & path, const std::string & data, bool replace) override
|
||||
void write(const std::string & file_name, const std::string & data, bool replace) override
|
||||
{
|
||||
if (!replace && fs::exists(path))
|
||||
if (!replace && fs::exists(file_name))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS,
|
||||
"Metadata file {} for named collection already exists",
|
||||
path);
|
||||
file_name);
|
||||
}
|
||||
|
||||
fs::create_directories(root_path);
|
||||
|
||||
auto tmp_path = getPath(path + ".tmp");
|
||||
auto tmp_path = getPath(file_name + ".tmp");
|
||||
WriteBufferFromFile out(tmp_path, data.size(), O_WRONLY | O_CREAT | O_EXCL);
|
||||
writeString(data, out);
|
||||
|
||||
@ -150,28 +150,32 @@ public:
|
||||
out.sync();
|
||||
out.close();
|
||||
|
||||
fs::rename(tmp_path, getPath(path));
|
||||
fs::rename(tmp_path, getPath(file_name));
|
||||
}
|
||||
|
||||
void remove(const std::string & path) override
|
||||
void remove(const std::string & file_name) override
|
||||
{
|
||||
if (!removeIfExists(getPath(path)))
|
||||
if (!removeIfExists(file_name))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
|
||||
"Cannot remove `{}`, because it doesn't exist", path);
|
||||
"Cannot remove `{}`, because it doesn't exist", file_name);
|
||||
}
|
||||
}
|
||||
|
||||
bool removeIfExists(const std::string & path) override
|
||||
bool removeIfExists(const std::string & file_name) override
|
||||
{
|
||||
return fs::remove(getPath(path));
|
||||
return fs::remove(getPath(file_name));
|
||||
}
|
||||
|
||||
private:
|
||||
std::string getPath(const std::string & path) const
|
||||
std::string getPath(const std::string & file_name) const
|
||||
{
|
||||
return fs::path(root_path) / path;
|
||||
const auto file_name_as_path = fs::path(file_name);
|
||||
if (file_name_as_path.is_absolute())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Filename {} cannot be an absolute path", file_name);
|
||||
|
||||
return fs::path(root_path) / file_name_as_path;
|
||||
}
|
||||
|
||||
/// Delete .tmp files. They could be left undeleted in case of
|
||||
@ -264,49 +268,49 @@ public:
|
||||
return children;
|
||||
}
|
||||
|
||||
bool exists(const std::string & path) const override
|
||||
bool exists(const std::string & file_name) const override
|
||||
{
|
||||
return getClient()->exists(getPath(path));
|
||||
return getClient()->exists(getPath(file_name));
|
||||
}
|
||||
|
||||
std::string read(const std::string & path) const override
|
||||
std::string read(const std::string & file_name) const override
|
||||
{
|
||||
return getClient()->get(getPath(path));
|
||||
return getClient()->get(getPath(file_name));
|
||||
}
|
||||
|
||||
void write(const std::string & path, const std::string & data, bool replace) override
|
||||
void write(const std::string & file_name, const std::string & data, bool replace) override
|
||||
{
|
||||
if (replace)
|
||||
{
|
||||
getClient()->createOrUpdate(getPath(path), data, zkutil::CreateMode::Persistent);
|
||||
getClient()->createOrUpdate(getPath(file_name), data, zkutil::CreateMode::Persistent);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto code = getClient()->tryCreate(getPath(path), data, zkutil::CreateMode::Persistent);
|
||||
auto code = getClient()->tryCreate(getPath(file_name), data, zkutil::CreateMode::Persistent);
|
||||
|
||||
if (code == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS,
|
||||
"Metadata file {} for named collection already exists",
|
||||
path);
|
||||
file_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void remove(const std::string & path) override
|
||||
void remove(const std::string & file_name) override
|
||||
{
|
||||
getClient()->remove(getPath(path));
|
||||
getClient()->remove(getPath(file_name));
|
||||
}
|
||||
|
||||
bool removeIfExists(const std::string & path) override
|
||||
bool removeIfExists(const std::string & file_name) override
|
||||
{
|
||||
auto code = getClient()->tryRemove(getPath(path));
|
||||
auto code = getClient()->tryRemove(getPath(file_name));
|
||||
if (code == Coordination::Error::ZOK)
|
||||
return true;
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
return false;
|
||||
throw Coordination::Exception::fromPath(code, getPath(path));
|
||||
throw Coordination::Exception::fromPath(code, getPath(file_name));
|
||||
}
|
||||
|
||||
private:
|
||||
@ -320,9 +324,13 @@ private:
|
||||
return zookeeper_client;
|
||||
}
|
||||
|
||||
std::string getPath(const std::string & path) const
|
||||
std::string getPath(const std::string & file_name) const
|
||||
{
|
||||
return fs::path(root_path) / path;
|
||||
const auto file_name_as_path = fs::path(file_name);
|
||||
if (file_name_as_path.is_absolute())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Filename {} cannot be an absolute path", file_name);
|
||||
|
||||
return fs::path(root_path) / file_name_as_path;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -442,8 +442,6 @@ The server successfully detected this situation and will download merged part fr
|
||||
M(ReadBufferFromS3InitMicroseconds, "Time spent initializing connection to S3.", ValueType::Microseconds) \
|
||||
M(ReadBufferFromS3Bytes, "Bytes read from S3.", ValueType::Bytes) \
|
||||
M(ReadBufferFromS3RequestsErrors, "Number of exceptions while reading from S3.", ValueType::Number) \
|
||||
M(ReadBufferFromS3ResetSessions, "Number of HTTP sessions that were reset in ReadBufferFromS3.", ValueType::Number) \
|
||||
M(ReadBufferFromS3PreservedSessions, "Number of HTTP sessions that were preserved in ReadBufferFromS3.", ValueType::Number) \
|
||||
\
|
||||
M(WriteBufferFromS3Microseconds, "Time spent on writing to S3.", ValueType::Microseconds) \
|
||||
M(WriteBufferFromS3Bytes, "Bytes written to S3.", ValueType::Bytes) \
|
||||
|
@ -72,11 +72,6 @@ public:
|
||||
/// How much seconds passed since query execution start.
|
||||
double elapsedSeconds() const { return getElapsedNanoseconds() / 1e9; }
|
||||
|
||||
void updateThreadEventData(HostToTimesMap & new_hosts_data);
|
||||
|
||||
private:
|
||||
double getCPUUsage();
|
||||
|
||||
struct MemoryUsage
|
||||
{
|
||||
UInt64 total = 0;
|
||||
@ -86,6 +81,11 @@ private:
|
||||
|
||||
MemoryUsage getMemoryUsage() const;
|
||||
|
||||
void updateThreadEventData(HostToTimesMap & new_hosts_data);
|
||||
|
||||
private:
|
||||
double getCPUUsage();
|
||||
|
||||
UInt64 getElapsedNanoseconds() const;
|
||||
|
||||
/// This flag controls whether to show the progress bar. We start showing it after
|
||||
|
635
src/Common/SignalHandlers.cpp
Normal file
635
src/Common/SignalHandlers.cpp
Normal file
@ -0,0 +1,635 @@
|
||||
#include <Common/SignalHandlers.h>
|
||||
#include <Common/config_version.h>
|
||||
#include <Common/getHashOfLoadedBinary.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Daemon/BaseDaemon.h>
|
||||
#include <Daemon/SentryWriter.h>
|
||||
#include <base/sleep.h>
|
||||
#include <base/getThreadId.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#pragma clang diagnostic ignored "-Wreserved-identifier"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_SET_SIGNAL_HANDLER;
|
||||
extern const int CANNOT_SEND_SIGNAL;
|
||||
}
|
||||
}
|
||||
|
||||
using namespace DB;
|
||||
|
||||
|
||||
void call_default_signal_handler(int sig)
|
||||
{
|
||||
if (SIG_ERR == signal(sig, SIG_DFL))
|
||||
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
|
||||
|
||||
if (0 != raise(sig))
|
||||
throw ErrnoException(ErrorCodes::CANNOT_SEND_SIGNAL, "Cannot send signal");
|
||||
}
|
||||
|
||||
|
||||
void writeSignalIDtoSignalPipe(int sig)
|
||||
{
|
||||
auto saved_errno = errno; /// We must restore previous value of errno in signal handler.
|
||||
|
||||
char buf[signal_pipe_buf_size];
|
||||
auto & signal_pipe = HandledSignals::instance().signal_pipe;
|
||||
WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf);
|
||||
writeBinary(sig, out);
|
||||
out.next();
|
||||
|
||||
errno = saved_errno;
|
||||
}
|
||||
|
||||
void closeLogsSignalHandler(int sig, siginfo_t *, void *)
|
||||
{
|
||||
DENY_ALLOCATIONS_IN_SCOPE;
|
||||
writeSignalIDtoSignalPipe(sig);
|
||||
}
|
||||
|
||||
void terminateRequestedSignalHandler(int sig, siginfo_t *, void *)
|
||||
{
|
||||
DENY_ALLOCATIONS_IN_SCOPE;
|
||||
writeSignalIDtoSignalPipe(sig);
|
||||
}
|
||||
|
||||
|
||||
void signalHandler(int sig, siginfo_t * info, void * context)
|
||||
{
|
||||
if (asynchronous_stack_unwinding && sig == SIGSEGV)
|
||||
siglongjmp(asynchronous_stack_unwinding_signal_jump_buffer, 1);
|
||||
|
||||
DENY_ALLOCATIONS_IN_SCOPE;
|
||||
auto saved_errno = errno; /// We must restore previous value of errno in signal handler.
|
||||
|
||||
char buf[signal_pipe_buf_size];
|
||||
auto & signal_pipe = HandledSignals::instance().signal_pipe;
|
||||
WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf);
|
||||
|
||||
const ucontext_t * signal_context = reinterpret_cast<ucontext_t *>(context);
|
||||
const StackTrace stack_trace(*signal_context);
|
||||
|
||||
#if USE_GWP_ASAN
|
||||
if (const auto fault_address = reinterpret_cast<uintptr_t>(info->si_addr);
|
||||
GWPAsan::isGWPAsanError(fault_address))
|
||||
GWPAsan::printReport(fault_address);
|
||||
#endif
|
||||
|
||||
writeBinary(sig, out);
|
||||
writePODBinary(*info, out);
|
||||
writePODBinary(signal_context, out);
|
||||
writePODBinary(stack_trace, out);
|
||||
writeVectorBinary(Exception::enable_job_stack_trace ? Exception::thread_frame_pointers : std::vector<StackTrace::FramePointers>{}, out);
|
||||
writeBinary(static_cast<UInt32>(getThreadId()), out);
|
||||
writePODBinary(current_thread, out);
|
||||
|
||||
out.next();
|
||||
|
||||
if (sig != SIGTSTP) /// This signal is used for debugging.
|
||||
{
|
||||
/// The time that is usually enough for separate thread to print info into log.
|
||||
/// Under MSan full stack unwinding with DWARF info about inline functions takes 101 seconds in one case.
|
||||
for (size_t i = 0; i < 300; ++i)
|
||||
{
|
||||
/// We will synchronize with the thread printing the messages with an atomic variable to finish earlier.
|
||||
if (HandledSignals::instance().fatal_error_printed.test())
|
||||
break;
|
||||
|
||||
/// This coarse method of synchronization is perfectly ok for fatal signals.
|
||||
sleepForSeconds(1);
|
||||
}
|
||||
|
||||
/// Wait for all logs flush operations
|
||||
sleepForSeconds(3);
|
||||
call_default_signal_handler(sig);
|
||||
}
|
||||
|
||||
errno = saved_errno;
|
||||
}
|
||||
|
||||
|
||||
[[noreturn]] void terminate_handler()
|
||||
{
|
||||
static thread_local bool terminating = false;
|
||||
if (terminating)
|
||||
abort();
|
||||
|
||||
terminating = true;
|
||||
|
||||
std::string log_message;
|
||||
|
||||
if (std::current_exception())
|
||||
log_message = "Terminate called for uncaught exception:\n" + getCurrentExceptionMessage(true);
|
||||
else
|
||||
log_message = "Terminate called without an active exception";
|
||||
|
||||
/// POSIX.1 says that write(2)s of less than PIPE_BUF bytes must be atomic - man 7 pipe
|
||||
/// And the buffer should not be too small because our exception messages can be large.
|
||||
static constexpr size_t buf_size = PIPE_BUF;
|
||||
|
||||
if (log_message.size() > buf_size - 16)
|
||||
log_message.resize(buf_size - 16);
|
||||
|
||||
char buf[buf_size];
|
||||
auto & signal_pipe = HandledSignals::instance().signal_pipe;
|
||||
WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], buf_size, buf);
|
||||
|
||||
writeBinary(static_cast<int>(SignalListener::StdTerminate), out);
|
||||
writeBinary(static_cast<UInt32>(getThreadId()), out);
|
||||
writeBinary(log_message, out);
|
||||
out.next();
|
||||
|
||||
abort();
|
||||
}
|
||||
|
||||
#if defined(SANITIZER)
|
||||
template <typename T>
|
||||
struct ValueHolder
|
||||
{
|
||||
ValueHolder(T value_) : value(value_)
|
||||
{}
|
||||
|
||||
T value;
|
||||
};
|
||||
|
||||
extern "C" void __sanitizer_set_death_callback(void (*)());
|
||||
|
||||
/// Sanitizers may not expect some function calls from death callback.
|
||||
/// Let's try to disable instrumentation to avoid possible issues.
|
||||
/// However, this callback may call other functions that are still instrumented.
|
||||
/// We can try [[clang::always_inline]] attribute for statements in future (available in clang-15)
|
||||
/// See https://github.com/google/sanitizers/issues/1543 and https://github.com/google/sanitizers/issues/1549.
|
||||
static DISABLE_SANITIZER_INSTRUMENTATION void sanitizerDeathCallback()
|
||||
{
|
||||
DENY_ALLOCATIONS_IN_SCOPE;
|
||||
/// Also need to send data via pipe. Otherwise it may lead to deadlocks or failures in printing diagnostic info.
|
||||
|
||||
char buf[signal_pipe_buf_size];
|
||||
auto & signal_pipe = HandledSignals::instance().signal_pipe;
|
||||
WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf);
|
||||
|
||||
const StackTrace stack_trace;
|
||||
|
||||
writeBinary(SignalListener::SanitizerTrap, out);
|
||||
writePODBinary(stack_trace, out);
|
||||
/// We create a dummy struct with a constructor so DISABLE_SANITIZER_INSTRUMENTATION is not applied to it
|
||||
/// otherwise, Memory sanitizer can't know that values initiialized inside this function are actually initialized
|
||||
/// because instrumentations are disabled leading to false positives later on
|
||||
ValueHolder<UInt32> thread_id{static_cast<UInt32>(getThreadId())};
|
||||
writeBinary(thread_id.value, out);
|
||||
writePODBinary(current_thread, out);
|
||||
|
||||
out.next();
|
||||
|
||||
/// The time that is usually enough for separate thread to print info into log.
|
||||
sleepForSeconds(20);
|
||||
}
|
||||
#endif
|
||||
|
||||
|
||||
void HandledSignals::addSignalHandler(const std::vector<int> & signals, signal_function handler, bool register_signal)
|
||||
{
|
||||
struct sigaction sa;
|
||||
memset(&sa, 0, sizeof(sa));
|
||||
sa.sa_sigaction = handler;
|
||||
sa.sa_flags = SA_SIGINFO;
|
||||
|
||||
#if defined(OS_DARWIN)
|
||||
sigemptyset(&sa.sa_mask);
|
||||
for (auto signal : signals)
|
||||
sigaddset(&sa.sa_mask, signal);
|
||||
#else
|
||||
if (sigemptyset(&sa.sa_mask))
|
||||
throw Poco::Exception("Cannot set signal handler.");
|
||||
|
||||
for (auto signal : signals)
|
||||
if (sigaddset(&sa.sa_mask, signal))
|
||||
throw Poco::Exception("Cannot set signal handler.");
|
||||
#endif
|
||||
|
||||
for (auto signal : signals)
|
||||
if (sigaction(signal, &sa, nullptr))
|
||||
throw Poco::Exception("Cannot set signal handler.");
|
||||
|
||||
if (register_signal)
|
||||
std::copy(signals.begin(), signals.end(), std::back_inserter(handled_signals));
|
||||
}
|
||||
|
||||
void blockSignals(const std::vector<int> & signals)
|
||||
{
|
||||
sigset_t sig_set;
|
||||
|
||||
#if defined(OS_DARWIN)
|
||||
sigemptyset(&sig_set);
|
||||
for (auto signal : signals)
|
||||
sigaddset(&sig_set, signal);
|
||||
#else
|
||||
if (sigemptyset(&sig_set))
|
||||
throw Poco::Exception("Cannot block signal.");
|
||||
|
||||
for (auto signal : signals)
|
||||
if (sigaddset(&sig_set, signal))
|
||||
throw Poco::Exception("Cannot block signal.");
|
||||
#endif
|
||||
|
||||
if (pthread_sigmask(SIG_BLOCK, &sig_set, nullptr))
|
||||
throw Poco::Exception("Cannot block signal.");
|
||||
}
|
||||
|
||||
|
||||
void SignalListener::run()
|
||||
{
|
||||
static_assert(PIPE_BUF >= 512);
|
||||
static_assert(signal_pipe_buf_size <= PIPE_BUF, "Only write of PIPE_BUF to pipe is atomic and the minimal known PIPE_BUF across supported platforms is 512");
|
||||
char buf[signal_pipe_buf_size];
|
||||
auto & signal_pipe = HandledSignals::instance().signal_pipe;
|
||||
ReadBufferFromFileDescriptor in(signal_pipe.fds_rw[0], signal_pipe_buf_size, buf);
|
||||
|
||||
while (!in.eof())
|
||||
{
|
||||
int sig = 0;
|
||||
readBinary(sig, in);
|
||||
// We may log some specific signals afterwards, with different log
|
||||
// levels and more info, but for completeness we log all signals
|
||||
// here at trace level.
|
||||
// Don't use strsignal here, because it's not thread-safe.
|
||||
LOG_TRACE(log, "Received signal {}", sig);
|
||||
|
||||
if (sig == StopThread)
|
||||
{
|
||||
LOG_INFO(log, "Stop SignalListener thread");
|
||||
break;
|
||||
}
|
||||
else if (sig == SIGHUP)
|
||||
{
|
||||
LOG_DEBUG(log, "Received signal to close logs.");
|
||||
BaseDaemon::instance().closeLogs(BaseDaemon::instance().logger());
|
||||
LOG_INFO(log, "Opened new log file after received signal.");
|
||||
}
|
||||
else if (sig == StdTerminate)
|
||||
{
|
||||
UInt32 thread_num;
|
||||
std::string message;
|
||||
|
||||
readBinary(thread_num, in);
|
||||
readBinary(message, in);
|
||||
|
||||
onTerminate(message, thread_num);
|
||||
}
|
||||
else if (sig == SIGINT ||
|
||||
sig == SIGQUIT ||
|
||||
sig == SIGTERM)
|
||||
{
|
||||
if (daemon)
|
||||
daemon->handleSignal(sig);
|
||||
}
|
||||
else
|
||||
{
|
||||
siginfo_t info{};
|
||||
ucontext_t * context{};
|
||||
StackTrace stack_trace(NoCapture{});
|
||||
std::vector<StackTrace::FramePointers> thread_frame_pointers;
|
||||
UInt32 thread_num{};
|
||||
ThreadStatus * thread_ptr{};
|
||||
|
||||
if (sig != SanitizerTrap)
|
||||
{
|
||||
readPODBinary(info, in);
|
||||
readPODBinary(context, in);
|
||||
}
|
||||
|
||||
readPODBinary(stack_trace, in);
|
||||
if (sig != SanitizerTrap)
|
||||
readVectorBinary(thread_frame_pointers, in);
|
||||
readBinary(thread_num, in);
|
||||
readPODBinary(thread_ptr, in);
|
||||
|
||||
/// This allows to receive more signals if failure happens inside onFault function.
|
||||
/// Example: segfault while symbolizing stack trace.
|
||||
try
|
||||
{
|
||||
std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); }).detach();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Likely cannot allocate thread
|
||||
onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void SignalListener::onTerminate(std::string_view message, UInt32 thread_num) const
|
||||
{
|
||||
size_t pos = message.find('\n');
|
||||
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) {}",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", thread_num, message.substr(0, pos));
|
||||
|
||||
/// Print trace from std::terminate exception line-by-line to make it easy for grep.
|
||||
while (pos != std::string_view::npos)
|
||||
{
|
||||
++pos;
|
||||
size_t next_pos = message.find('\n', pos);
|
||||
size_t size = next_pos;
|
||||
if (next_pos != std::string_view::npos)
|
||||
size = next_pos - pos;
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(message.substr(pos, size)));
|
||||
pos = next_pos;
|
||||
}
|
||||
}
|
||||
|
||||
void SignalListener::onFault(
|
||||
int sig,
|
||||
const siginfo_t & info,
|
||||
ucontext_t * context,
|
||||
const StackTrace & stack_trace,
|
||||
const std::vector<StackTrace::FramePointers> & thread_frame_pointers,
|
||||
UInt32 thread_num,
|
||||
DB::ThreadStatus * thread_ptr) const
|
||||
try
|
||||
{
|
||||
ThreadStatus thread_status;
|
||||
|
||||
/// First log those fields that are safe to access and that should not cause new fault.
|
||||
/// That way we will have some duplicated info in the log but we don't loose important info
|
||||
/// in case of double fault.
|
||||
|
||||
LOG_FATAL(log, "########## Short fault info ############");
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "",
|
||||
thread_num, sig);
|
||||
|
||||
std::string signal_description = "Unknown signal";
|
||||
|
||||
/// Some of these are not really signals, but our own indications on failure reason.
|
||||
if (sig == StdTerminate)
|
||||
signal_description = "std::terminate";
|
||||
else if (sig == SanitizerTrap)
|
||||
signal_description = "sanitizer trap";
|
||||
else if (sig >= 0)
|
||||
signal_description = strsignal(sig); // NOLINT(concurrency-mt-unsafe) // it is not thread-safe but ok in this context
|
||||
|
||||
LOG_FATAL(log, "Signal description: {}", signal_description);
|
||||
|
||||
String error_message;
|
||||
|
||||
if (sig != SanitizerTrap)
|
||||
error_message = signalToErrorMessage(sig, info, *context);
|
||||
else
|
||||
error_message = "Sanitizer trap.";
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(error_message));
|
||||
|
||||
String bare_stacktrace_str;
|
||||
if (stack_trace.getSize())
|
||||
{
|
||||
/// Write bare stack trace (addresses) just in case if we will fail to print symbolized stack trace.
|
||||
/// NOTE: This still require memory allocations and mutex lock inside logger.
|
||||
/// BTW we can also print it to stderr using write syscalls.
|
||||
|
||||
WriteBufferFromOwnString bare_stacktrace;
|
||||
writeString("Stack trace:", bare_stacktrace);
|
||||
for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i)
|
||||
{
|
||||
writeChar(' ', bare_stacktrace);
|
||||
writePointerHex(stack_trace.getFramePointers()[i], bare_stacktrace);
|
||||
}
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(bare_stacktrace.str()));
|
||||
bare_stacktrace_str = bare_stacktrace.str();
|
||||
}
|
||||
|
||||
/// Now try to access potentially unsafe data in thread_ptr.
|
||||
|
||||
String query_id;
|
||||
String query;
|
||||
|
||||
/// Send logs from this thread to client if possible.
|
||||
/// It will allow client to see failure messages directly.
|
||||
if (thread_ptr)
|
||||
{
|
||||
query_id = thread_ptr->getQueryId();
|
||||
query = thread_ptr->getQueryForLog();
|
||||
|
||||
if (auto logs_queue = thread_ptr->getInternalTextLogsQueue())
|
||||
{
|
||||
CurrentThread::attachInternalTextLogsQueue(logs_queue, LogsLevel::trace);
|
||||
}
|
||||
}
|
||||
|
||||
LOG_FATAL(log, "########################################");
|
||||
|
||||
if (query_id.empty())
|
||||
{
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (no query) Received signal {} ({})",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "",
|
||||
thread_num, signal_description, sig);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (query_id: {}) (query: {}) Received signal {} ({})",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "",
|
||||
thread_num, query_id, query, signal_description, sig);
|
||||
}
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(error_message));
|
||||
|
||||
if (!bare_stacktrace_str.empty())
|
||||
{
|
||||
LOG_FATAL(log, fmt::runtime(bare_stacktrace_str));
|
||||
}
|
||||
|
||||
/// Write symbolized stack trace line by line for better grep-ability.
|
||||
stack_trace.toStringEveryLine([&](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); });
|
||||
|
||||
/// In case it's a scheduled job write all previous jobs origins call stacks
|
||||
std::for_each(thread_frame_pointers.rbegin(), thread_frame_pointers.rend(),
|
||||
[this](const StackTrace::FramePointers & frame_pointers)
|
||||
{
|
||||
if (size_t size = std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin())
|
||||
{
|
||||
LOG_FATAL(log, "========================================");
|
||||
WriteBufferFromOwnString bare_stacktrace;
|
||||
writeString("Job's origin stack trace:", bare_stacktrace);
|
||||
std::for_each_n(frame_pointers.begin(), size,
|
||||
[&bare_stacktrace](const void * ptr)
|
||||
{
|
||||
writeChar(' ', bare_stacktrace);
|
||||
writePointerHex(ptr, bare_stacktrace);
|
||||
}
|
||||
);
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(bare_stacktrace.str()));
|
||||
|
||||
StackTrace::toStringEveryLine(const_cast<void **>(frame_pointers.data()), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); });
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
/// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace.
|
||||
/// Please keep the below log messages in-sync with the ones in programs/server/Server.cpp
|
||||
|
||||
if (daemon && daemon->stored_binary_hash.empty())
|
||||
{
|
||||
LOG_FATAL(log, "Integrity check of the executable skipped because the reference checksum could not be read.");
|
||||
}
|
||||
else if (daemon)
|
||||
{
|
||||
String calculated_binary_hash = getHashOfLoadedBinaryHex();
|
||||
if (calculated_binary_hash == daemon->stored_binary_hash)
|
||||
{
|
||||
LOG_FATAL(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_FATAL(
|
||||
log,
|
||||
"Calculated checksum of the executable ({0}) does not correspond"
|
||||
" to the reference checksum stored in the executable ({1})."
|
||||
" This may indicate one of the following:"
|
||||
" - the executable was changed just after startup;"
|
||||
" - the executable was corrupted on disk due to faulty hardware;"
|
||||
" - the loaded executable was corrupted in memory due to faulty hardware;"
|
||||
" - the file was intentionally modified;"
|
||||
" - a logical error in the code.",
|
||||
calculated_binary_hash,
|
||||
daemon->stored_binary_hash);
|
||||
}
|
||||
}
|
||||
#endif
|
||||
|
||||
/// Write crash to system.crash_log table if available.
|
||||
if (collectCrashLog)
|
||||
collectCrashLog(sig, thread_num, query_id, stack_trace);
|
||||
|
||||
Context::getGlobalContextInstance()->handleCrash();
|
||||
|
||||
/// Send crash report to developers (if configured)
|
||||
if (sig != SanitizerTrap)
|
||||
{
|
||||
if (daemon)
|
||||
{
|
||||
if (auto * sentry = SentryWriter::getInstance())
|
||||
sentry->onSignal(sig, error_message, stack_trace.getFramePointers(), stack_trace.getOffset(), stack_trace.getSize());
|
||||
}
|
||||
|
||||
/// Advice the user to send it manually.
|
||||
if (std::string_view(VERSION_OFFICIAL).contains("official build"))
|
||||
{
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
/// Approximate support period, upper bound.
|
||||
if (time(nullptr) - date_lut.makeDate(2000 + VERSION_MAJOR, VERSION_MINOR, 1) < (365 + 30) * 86400)
|
||||
{
|
||||
LOG_FATAL(log, "Report this error to https://github.com/ClickHouse/ClickHouse/issues");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_FATAL(log, "ClickHouse version {} is old and should be upgraded to the latest version.", VERSION_STRING);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_FATAL(log, "This ClickHouse version is not official and should be upgraded to the official build.");
|
||||
}
|
||||
}
|
||||
|
||||
/// List changed settings.
|
||||
if (!query_id.empty())
|
||||
{
|
||||
ContextPtr query_context = thread_ptr->getQueryContext();
|
||||
if (query_context)
|
||||
{
|
||||
String changed_settings = query_context->getSettingsRef().toString();
|
||||
|
||||
if (changed_settings.empty())
|
||||
LOG_FATAL(log, "No settings were changed");
|
||||
else
|
||||
LOG_FATAL(log, "Changed settings: {}", changed_settings);
|
||||
}
|
||||
}
|
||||
|
||||
/// When everything is done, we will try to send these error messages to the client.
|
||||
if (thread_ptr)
|
||||
thread_ptr->onFatalError();
|
||||
|
||||
HandledSignals::instance().fatal_error_printed.test_and_set();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// onFault is called from the std::thread, and it should catch all exceptions; otherwise, you can get unrelated fatal errors.
|
||||
PreformattedMessage message = getCurrentExceptionMessageAndPattern(true);
|
||||
LOG_FATAL(log, message);
|
||||
}
|
||||
|
||||
HandledSignals::HandledSignals()
|
||||
{
|
||||
signal_pipe.setNonBlockingWrite();
|
||||
signal_pipe.tryIncreaseSize(1 << 20);
|
||||
}
|
||||
|
||||
void HandledSignals::reset()
|
||||
{
|
||||
/// Reset signals to SIG_DFL to avoid trying to write to the signal_pipe that will be closed after.
|
||||
for (int sig : handled_signals)
|
||||
{
|
||||
if (SIG_ERR == signal(sig, SIG_DFL))
|
||||
{
|
||||
try
|
||||
{
|
||||
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
|
||||
}
|
||||
catch (ErrnoException &)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
signal_pipe.close();
|
||||
}
|
||||
|
||||
HandledSignals::~HandledSignals()
|
||||
{
|
||||
reset();
|
||||
};
|
||||
|
||||
HandledSignals & HandledSignals::instance()
|
||||
{
|
||||
static HandledSignals res;
|
||||
return res;
|
||||
}
|
||||
|
||||
void HandledSignals::setupTerminateHandler()
|
||||
{
|
||||
std::set_terminate(terminate_handler);
|
||||
}
|
||||
|
||||
void HandledSignals::setupCommonDeadlySignalHandlers()
|
||||
{
|
||||
/// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime.
|
||||
addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, true);
|
||||
|
||||
#if defined(SANITIZER)
|
||||
__sanitizer_set_death_callback(sanitizerDeathCallback);
|
||||
#endif
|
||||
}
|
||||
|
||||
void HandledSignals::setupCommonTerminateRequestSignalHandlers()
|
||||
{
|
||||
addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, true);
|
||||
}
|
110
src/Common/SignalHandlers.h
Normal file
110
src/Common/SignalHandlers.h
Normal file
@ -0,0 +1,110 @@
|
||||
#pragma once
|
||||
#include <csignal>
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <Common/PipeFDs.h>
|
||||
#include <Common/StackTrace.h>
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Poco/Runnable.h>
|
||||
|
||||
class BaseDaemon;
|
||||
|
||||
/** Reset signal handler to the default and send signal to itself.
|
||||
* It's called from user signal handler to write core dump.
|
||||
*/
|
||||
void call_default_signal_handler(int sig);
|
||||
|
||||
const size_t signal_pipe_buf_size =
|
||||
sizeof(int)
|
||||
+ sizeof(siginfo_t)
|
||||
+ sizeof(ucontext_t*)
|
||||
+ sizeof(StackTrace)
|
||||
+ sizeof(UInt64)
|
||||
+ sizeof(UInt32)
|
||||
+ sizeof(void*);
|
||||
|
||||
using signal_function = void(int, siginfo_t*, void*);
|
||||
|
||||
void writeSignalIDtoSignalPipe(int sig);
|
||||
|
||||
/** Signal handler for HUP */
|
||||
void closeLogsSignalHandler(int sig, siginfo_t *, void *);
|
||||
|
||||
void terminateRequestedSignalHandler(int sig, siginfo_t *, void *);
|
||||
|
||||
|
||||
/** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log.
|
||||
*/
|
||||
void signalHandler(int sig, siginfo_t * info, void * context);
|
||||
|
||||
|
||||
/** To use with std::set_terminate.
|
||||
* Collects slightly more info than __gnu_cxx::__verbose_terminate_handler,
|
||||
* and send it to pipe. Other thread will read this info from pipe and asynchronously write it to log.
|
||||
* Look at libstdc++-v3/libsupc++/vterminate.cc for example.
|
||||
*/
|
||||
[[noreturn]] void terminate_handler();
|
||||
|
||||
/// Avoid link time dependency on DB/Interpreters - will use this function only when linked.
|
||||
__attribute__((__weak__)) void collectCrashLog(
|
||||
Int32 signal, UInt64 thread_id, const String & query_id, const StackTrace & stack_trace);
|
||||
|
||||
|
||||
void blockSignals(const std::vector<int> & signals);
|
||||
|
||||
|
||||
/** The thread that read info about signal or std::terminate from pipe.
|
||||
* On HUP, close log files (for new files to be opened later).
|
||||
* On information about std::terminate, write it to log.
|
||||
* On other signals, write info to log.
|
||||
*/
|
||||
class SignalListener : public Poco::Runnable
|
||||
{
|
||||
public:
|
||||
static constexpr int StdTerminate = -1;
|
||||
static constexpr int StopThread = -2;
|
||||
static constexpr int SanitizerTrap = -3;
|
||||
|
||||
explicit SignalListener(BaseDaemon * daemon_, LoggerPtr log_)
|
||||
: daemon(daemon_), log(log_)
|
||||
{
|
||||
}
|
||||
|
||||
void run() override;
|
||||
|
||||
private:
|
||||
BaseDaemon * daemon;
|
||||
LoggerPtr log;
|
||||
|
||||
void onTerminate(std::string_view message, UInt32 thread_num) const;
|
||||
|
||||
void onFault(
|
||||
int sig,
|
||||
const siginfo_t & info,
|
||||
ucontext_t * context,
|
||||
const StackTrace & stack_trace,
|
||||
const std::vector<StackTrace::FramePointers> & thread_frame_pointers,
|
||||
UInt32 thread_num,
|
||||
DB::ThreadStatus * thread_ptr) const;
|
||||
};
|
||||
|
||||
struct HandledSignals
|
||||
{
|
||||
std::vector<int> handled_signals;
|
||||
DB::PipeFDs signal_pipe;
|
||||
std::atomic_flag fatal_error_printed;
|
||||
|
||||
HandledSignals();
|
||||
~HandledSignals();
|
||||
|
||||
void setupTerminateHandler();
|
||||
void setupCommonDeadlySignalHandlers();
|
||||
void setupCommonTerminateRequestSignalHandlers();
|
||||
|
||||
void addSignalHandler(const std::vector<int> & signals, signal_function handler, bool register_signal);
|
||||
|
||||
void reset();
|
||||
|
||||
static HandledSignals & instance();
|
||||
};
|
@ -13,14 +13,14 @@
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <base/find_symbols.h>
|
||||
#include <base/sort.h>
|
||||
#include <base/map.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include "Common/ZooKeeper/IKeeper.h"
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
@ -114,7 +114,11 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr<Coordination::IKeeper>
|
||||
/// availability_zones is empty on server startup or after config reloading
|
||||
/// We will keep the az info when starting new sessions
|
||||
availability_zones = args.availability_zones;
|
||||
LOG_TEST(log, "Availability zones from config: [{}], client: {}", fmt::join(availability_zones, ", "), args.client_availability_zone);
|
||||
|
||||
LOG_TEST(log, "Availability zones from config: [{}], client: {}",
|
||||
fmt::join(collections::map(availability_zones, [](auto s){ return DB::quoteString(s); }), ", "),
|
||||
DB::quoteString(args.client_availability_zone));
|
||||
|
||||
if (args.availability_zone_autodetect)
|
||||
updateAvailabilityZones();
|
||||
}
|
||||
|
@ -80,7 +80,7 @@ inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align)
|
||||
throw std::bad_alloc{};
|
||||
}
|
||||
|
||||
inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept
|
||||
inline ALWAYS_INLINE void * newNoExcept(std::size_t size) noexcept
|
||||
{
|
||||
#if USE_GWP_ASAN
|
||||
if (unlikely(GWPAsan::GuardedAlloc.shouldSample()))
|
||||
@ -99,7 +99,7 @@ inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept
|
||||
return malloc(size);
|
||||
}
|
||||
|
||||
inline ALWAYS_INLINE void * newNoExept(std::size_t size, std::align_val_t align) noexcept
|
||||
inline ALWAYS_INLINE void * newNoExcept(std::size_t size, std::align_val_t align) noexcept
|
||||
{
|
||||
#if USE_GWP_ASAN
|
||||
if (unlikely(GWPAsan::GuardedAlloc.shouldSample()))
|
||||
|
@ -87,7 +87,7 @@ void * operator new(std::size_t size, const std::nothrow_t &) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::trackMemory(size, trace);
|
||||
void * ptr = Memory::newNoExept(size);
|
||||
void * ptr = Memory::newNoExcept(size);
|
||||
trace.onAlloc(ptr, actual_size);
|
||||
return ptr;
|
||||
}
|
||||
@ -96,7 +96,7 @@ void * operator new[](std::size_t size, const std::nothrow_t &) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::trackMemory(size, trace);
|
||||
void * ptr = Memory::newNoExept(size);
|
||||
void * ptr = Memory::newNoExcept(size);
|
||||
trace.onAlloc(ptr, actual_size);
|
||||
return ptr;
|
||||
}
|
||||
@ -105,7 +105,7 @@ void * operator new(std::size_t size, std::align_val_t align, const std::nothrow
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::trackMemory(size, trace, align);
|
||||
void * ptr = Memory::newNoExept(size, align);
|
||||
void * ptr = Memory::newNoExcept(size, align);
|
||||
trace.onAlloc(ptr, actual_size);
|
||||
return ptr;
|
||||
}
|
||||
@ -114,7 +114,7 @@ void * operator new[](std::size_t size, std::align_val_t align, const std::nothr
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::trackMemory(size, trace, align);
|
||||
void * ptr = Memory::newNoExept(size, align);
|
||||
void * ptr = Memory::newNoExcept(size, align);
|
||||
trace.onAlloc(ptr, actual_size);
|
||||
return ptr;
|
||||
}
|
||||
|
@ -55,6 +55,7 @@ struct Settings;
|
||||
M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests, set to 0 to disable", 0) \
|
||||
M(UInt64, raft_limits_reconnect_limit, 50, "If connection to a peer is silent longer than this limit * (multiplied by heartbeat interval), we re-establish the connection.", 0) \
|
||||
M(Bool, async_replication, false, "Enable async replication. All write and read guarantees are preserved while better performance is achieved. Settings is disabled by default to not break backwards compatibility.", 0) \
|
||||
M(Bool, experimental_use_rocksdb, false, "Use rocksdb as backend storage", 0) \
|
||||
M(UInt64, latest_logs_cache_size_threshold, 1 * 1024 * 1024 * 1024, "Maximum total size of in-memory cache of latest log entries.", 0) \
|
||||
M(UInt64, commit_logs_cache_size_threshold, 500 * 1024 * 1024, "Maximum total size of in-memory cache of log entries needed next for commit.", 0) \
|
||||
M(UInt64, disk_move_retries_wait_ms, 1000, "How long to wait between retries after a failure which happened while a file was being moved between disks.", 0) \
|
||||
|
@ -183,8 +183,6 @@
|
||||
M(ReadBufferFromS3InitMicroseconds) \
|
||||
M(ReadBufferFromS3Bytes) \
|
||||
M(ReadBufferFromS3RequestsErrors) \
|
||||
M(ReadBufferFromS3ResetSessions) \
|
||||
M(ReadBufferFromS3PreservedSessions) \
|
||||
\
|
||||
M(WriteBufferFromS3Microseconds) \
|
||||
M(WriteBufferFromS3Bytes) \
|
||||
|
@ -5,18 +5,27 @@
|
||||
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/Defines.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/S3/Credentials.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Poco/Util/JSONConfiguration.h>
|
||||
#include <Coordination/KeeperConstants.h>
|
||||
#include <Server/CloudPlacementInfo.h>
|
||||
#include <Coordination/KeeperFeatureFlags.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/DiskSelector.h>
|
||||
#include <IO/S3/Credentials.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
#include "config.h"
|
||||
#if USE_ROCKSDB
|
||||
#include <rocksdb/table.h>
|
||||
#include <rocksdb/convenience.h>
|
||||
#include <rocksdb/utilities/db_ttl.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -24,6 +33,8 @@ namespace ErrorCodes
|
||||
{
|
||||
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ROCKSDB_ERROR;
|
||||
|
||||
}
|
||||
|
||||
@ -41,6 +52,95 @@ KeeperContext::KeeperContext(bool standalone_keeper_, CoordinationSettingsPtr co
|
||||
system_nodes_with_data[keeper_api_version_path] = toString(static_cast<uint8_t>(KeeperApiVersion::WITH_MULTI_READ));
|
||||
}
|
||||
|
||||
#if USE_ROCKSDB
|
||||
using RocksDBOptions = std::unordered_map<std::string, std::string>;
|
||||
|
||||
static RocksDBOptions getOptionsFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & path)
|
||||
{
|
||||
RocksDBOptions options;
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(path, keys);
|
||||
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
const String key_path = path + "." + key;
|
||||
options[key] = config.getString(key_path);
|
||||
}
|
||||
|
||||
return options;
|
||||
}
|
||||
|
||||
static rocksdb::Options getRocksDBOptionsFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
rocksdb::Status status;
|
||||
rocksdb::Options base;
|
||||
|
||||
base.create_if_missing = true;
|
||||
base.compression = rocksdb::CompressionType::kZSTD;
|
||||
base.statistics = rocksdb::CreateDBStatistics();
|
||||
/// It is too verbose by default, and in fact we don't care about rocksdb logs at all.
|
||||
base.info_log_level = rocksdb::ERROR_LEVEL;
|
||||
|
||||
rocksdb::Options merged = base;
|
||||
rocksdb::BlockBasedTableOptions table_options;
|
||||
|
||||
if (config.has("keeper_server.rocksdb.options"))
|
||||
{
|
||||
auto config_options = getOptionsFromConfig(config, "keeper_server.rocksdb.options");
|
||||
status = rocksdb::GetDBOptionsFromMap(merged, config_options, &merged);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.options' : {}",
|
||||
status.ToString());
|
||||
}
|
||||
}
|
||||
if (config.has("rocksdb.column_family_options"))
|
||||
{
|
||||
auto column_family_options = getOptionsFromConfig(config, "rocksdb.column_family_options");
|
||||
status = rocksdb::GetColumnFamilyOptionsFromMap(merged, column_family_options, &merged);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.column_family_options' at: {}", status.ToString());
|
||||
}
|
||||
}
|
||||
if (config.has("rocksdb.block_based_table_options"))
|
||||
{
|
||||
auto block_based_table_options = getOptionsFromConfig(config, "rocksdb.block_based_table_options");
|
||||
status = rocksdb::GetBlockBasedTableOptionsFromMap(table_options, block_based_table_options, &table_options);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.block_based_table_options' at: {}", status.ToString());
|
||||
}
|
||||
}
|
||||
|
||||
merged.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options));
|
||||
return merged;
|
||||
}
|
||||
#endif
|
||||
|
||||
KeeperContext::Storage KeeperContext::getRocksDBPathFromConfig(const Poco::Util::AbstractConfiguration & config) const
|
||||
{
|
||||
const auto create_local_disk = [](const auto & path)
|
||||
{
|
||||
if (fs::exists(path))
|
||||
fs::remove_all(path);
|
||||
fs::create_directories(path);
|
||||
|
||||
return std::make_shared<DiskLocal>("LocalRocksDBDisk", path);
|
||||
};
|
||||
if (config.has("keeper_server.rocksdb_path"))
|
||||
return create_local_disk(config.getString("keeper_server.rocksdb_path"));
|
||||
|
||||
if (config.has("keeper_server.storage_path"))
|
||||
return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")} / "rocksdb");
|
||||
|
||||
if (standalone_keeper)
|
||||
return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "rocksdb");
|
||||
else
|
||||
return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/rocksdb");
|
||||
}
|
||||
|
||||
void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_)
|
||||
{
|
||||
dispatcher = dispatcher_;
|
||||
@ -59,6 +159,14 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config,
|
||||
|
||||
initializeFeatureFlags(config);
|
||||
initializeDisks(config);
|
||||
|
||||
#if USE_ROCKSDB
|
||||
if (config.getBool("keeper_server.coordination_settings.experimental_use_rocksdb", false))
|
||||
{
|
||||
rocksdb_options = std::make_shared<rocksdb::Options>(getRocksDBOptionsFromConfig(config));
|
||||
digest_enabled = false; /// TODO: support digest
|
||||
}
|
||||
#endif
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -94,6 +202,8 @@ void KeeperContext::initializeDisks(const Poco::Util::AbstractConfiguration & co
|
||||
{
|
||||
disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance(), diskValidator);
|
||||
|
||||
rocksdb_storage = getRocksDBPathFromConfig(config);
|
||||
|
||||
log_storage = getLogsPathFromConfig(config);
|
||||
|
||||
if (config.has("keeper_server.latest_log_storage_disk"))
|
||||
@ -262,6 +372,37 @@ void KeeperContext::dumpConfiguration(WriteBufferFromOwnString & buf) const
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void KeeperContext::setRocksDBDisk(DiskPtr disk)
|
||||
{
|
||||
rocksdb_storage = std::move(disk);
|
||||
}
|
||||
|
||||
DiskPtr KeeperContext::getTemporaryRocksDBDisk() const
|
||||
{
|
||||
DiskPtr rocksdb_disk = getDisk(rocksdb_storage);
|
||||
if (!rocksdb_disk)
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "rocksdb storage is not initialized");
|
||||
}
|
||||
auto uuid_str = formatUUID(UUIDHelpers::generateV4());
|
||||
String path_to_create = "rocks_" + std::string(uuid_str.data(), uuid_str.size());
|
||||
rocksdb_disk->createDirectory(path_to_create);
|
||||
return std::make_shared<DiskLocal>("LocalTmpRocksDBDisk", fullPath(rocksdb_disk, path_to_create));
|
||||
}
|
||||
|
||||
void KeeperContext::setRocksDBOptions(std::shared_ptr<rocksdb::Options> rocksdb_options_)
|
||||
{
|
||||
if (rocksdb_options_ != nullptr)
|
||||
rocksdb_options = rocksdb_options_;
|
||||
else
|
||||
{
|
||||
#if USE_ROCKSDB
|
||||
rocksdb_options = std::make_shared<rocksdb::Options>(getRocksDBOptionsFromConfig(Poco::Util::JSONConfiguration()));
|
||||
#endif
|
||||
}
|
||||
}
|
||||
|
||||
KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const
|
||||
{
|
||||
const auto create_local_disk = [](const auto & path)
|
||||
|
@ -6,6 +6,11 @@
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
|
||||
namespace rocksdb
|
||||
{
|
||||
struct Options;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -62,6 +67,12 @@ public:
|
||||
|
||||
constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; }
|
||||
|
||||
void setRocksDBDisk(DiskPtr disk);
|
||||
DiskPtr getTemporaryRocksDBDisk() const;
|
||||
|
||||
void setRocksDBOptions(std::shared_ptr<rocksdb::Options> rocksdb_options_ = nullptr);
|
||||
std::shared_ptr<rocksdb::Options> getRocksDBOptions() const { return rocksdb_options; }
|
||||
|
||||
UInt64 getKeeperMemorySoftLimit() const { return memory_soft_limit; }
|
||||
void updateKeeperMemorySoftLimit(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
@ -90,6 +101,7 @@ private:
|
||||
void initializeFeatureFlags(const Poco::Util::AbstractConfiguration & config);
|
||||
void initializeDisks(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
Storage getRocksDBPathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
|
||||
Storage getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
|
||||
Storage getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
|
||||
Storage getStatePathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
|
||||
@ -111,12 +123,15 @@ private:
|
||||
|
||||
std::shared_ptr<DiskSelector> disk_selector;
|
||||
|
||||
Storage rocksdb_storage;
|
||||
Storage log_storage;
|
||||
Storage latest_log_storage;
|
||||
Storage snapshot_storage;
|
||||
Storage latest_snapshot_storage;
|
||||
Storage state_file_storage;
|
||||
|
||||
std::shared_ptr<rocksdb::Options> rocksdb_options;
|
||||
|
||||
std::vector<std::string> old_log_disk_names;
|
||||
std::vector<std::string> old_snapshot_disk_names;
|
||||
|
||||
|
@ -117,13 +117,13 @@ void KeeperDispatcher::requestThread()
|
||||
RaftAppendResult prev_result = nullptr;
|
||||
/// Requests from previous iteration. We store them to be able
|
||||
/// to send errors to the client.
|
||||
KeeperStorage::RequestsForSessions prev_batch;
|
||||
KeeperStorageBase::RequestsForSessions prev_batch;
|
||||
|
||||
const auto & shutdown_called = keeper_context->isShutdownCalled();
|
||||
|
||||
while (!shutdown_called)
|
||||
{
|
||||
KeeperStorage::RequestForSession request;
|
||||
KeeperStorageBase::RequestForSession request;
|
||||
|
||||
auto coordination_settings = configuration_and_settings->coordination_settings;
|
||||
uint64_t max_wait = coordination_settings->operation_timeout_ms.totalMilliseconds();
|
||||
@ -153,7 +153,7 @@ void KeeperDispatcher::requestThread()
|
||||
continue;
|
||||
}
|
||||
|
||||
KeeperStorage::RequestsForSessions current_batch;
|
||||
KeeperStorageBase::RequestsForSessions current_batch;
|
||||
size_t current_batch_bytes_size = 0;
|
||||
|
||||
bool has_read_request = false;
|
||||
@ -311,7 +311,7 @@ void KeeperDispatcher::responseThread()
|
||||
const auto & shutdown_called = keeper_context->isShutdownCalled();
|
||||
while (!shutdown_called)
|
||||
{
|
||||
KeeperStorage::ResponseForSession response_for_session;
|
||||
KeeperStorageBase::ResponseForSession response_for_session;
|
||||
|
||||
uint64_t max_wait = configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds();
|
||||
|
||||
@ -402,7 +402,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ
|
||||
return false;
|
||||
}
|
||||
|
||||
KeeperStorage::RequestForSession request_info;
|
||||
KeeperStorageBase::RequestForSession request_info;
|
||||
request_info.request = request;
|
||||
using namespace std::chrono;
|
||||
request_info.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count();
|
||||
@ -448,7 +448,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf
|
||||
snapshots_queue,
|
||||
keeper_context,
|
||||
snapshot_s3,
|
||||
[this](uint64_t /*log_idx*/, const KeeperStorage::RequestForSession & request_for_session)
|
||||
[this](uint64_t /*log_idx*/, const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
{
|
||||
/// check if we have queue of read requests depending on this request to be committed
|
||||
@ -540,7 +540,7 @@ void KeeperDispatcher::shutdown()
|
||||
update_configuration_thread.join();
|
||||
}
|
||||
|
||||
KeeperStorage::RequestForSession request_for_session;
|
||||
KeeperStorageBase::RequestForSession request_for_session;
|
||||
|
||||
/// Set session expired for all pending requests
|
||||
while (requests_queue && requests_queue->tryPop(request_for_session))
|
||||
@ -551,7 +551,7 @@ void KeeperDispatcher::shutdown()
|
||||
setResponse(request_for_session.session_id, response);
|
||||
}
|
||||
|
||||
KeeperStorage::RequestsForSessions close_requests;
|
||||
KeeperStorageBase::RequestsForSessions close_requests;
|
||||
{
|
||||
/// Clear all registered sessions
|
||||
std::lock_guard lock(session_to_response_callback_mutex);
|
||||
@ -565,7 +565,7 @@ void KeeperDispatcher::shutdown()
|
||||
auto request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close);
|
||||
request->xid = Coordination::CLOSE_XID;
|
||||
using namespace std::chrono;
|
||||
KeeperStorage::RequestForSession request_info
|
||||
KeeperStorageBase::RequestForSession request_info
|
||||
{
|
||||
.session_id = session,
|
||||
.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count(),
|
||||
@ -663,7 +663,7 @@ void KeeperDispatcher::sessionCleanerTask()
|
||||
auto request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close);
|
||||
request->xid = Coordination::CLOSE_XID;
|
||||
using namespace std::chrono;
|
||||
KeeperStorage::RequestForSession request_info
|
||||
KeeperStorageBase::RequestForSession request_info
|
||||
{
|
||||
.session_id = dead_session,
|
||||
.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count(),
|
||||
@ -711,16 +711,16 @@ void KeeperDispatcher::finishSession(int64_t session_id)
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error)
|
||||
void KeeperDispatcher::addErrorResponses(const KeeperStorageBase::RequestsForSessions & requests_for_sessions, Coordination::Error error)
|
||||
{
|
||||
for (const auto & request_for_session : requests_for_sessions)
|
||||
{
|
||||
KeeperStorage::ResponsesForSessions responses;
|
||||
KeeperStorageBase::ResponsesForSessions responses;
|
||||
auto response = request_for_session.request->makeResponse();
|
||||
response->xid = request_for_session.request->xid;
|
||||
response->zxid = 0;
|
||||
response->error = error;
|
||||
if (!responses_queue.push(DB::KeeperStorage::ResponseForSession{request_for_session.session_id, response}))
|
||||
if (!responses_queue.push(DB::KeeperStorageBase::ResponseForSession{request_for_session.session_id, response}))
|
||||
throw Exception(ErrorCodes::SYSTEM_ERROR,
|
||||
"Could not push error response xid {} zxid {} error message {} to responses queue",
|
||||
response->xid,
|
||||
@ -730,7 +730,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperDispatcher::forceWaitAndProcessResult(
|
||||
RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions, bool clear_requests_on_success)
|
||||
RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions, bool clear_requests_on_success)
|
||||
{
|
||||
if (!result->has_result())
|
||||
result->get();
|
||||
@ -755,7 +755,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms)
|
||||
{
|
||||
/// New session id allocation is a special request, because we cannot process it in normal
|
||||
/// way: get request -> put to raft -> set response for registered callback.
|
||||
KeeperStorage::RequestForSession request_info;
|
||||
KeeperStorageBase::RequestForSession request_info;
|
||||
std::shared_ptr<Coordination::ZooKeeperSessionIDRequest> request = std::make_shared<Coordination::ZooKeeperSessionIDRequest>();
|
||||
/// Internal session id. It's a temporary number which is unique for each client on this server
|
||||
/// but can be same on different servers.
|
||||
|
@ -26,7 +26,7 @@ using ZooKeeperResponseCallback = std::function<void(const Coordination::ZooKeep
|
||||
class KeeperDispatcher
|
||||
{
|
||||
private:
|
||||
using RequestsQueue = ConcurrentBoundedQueue<KeeperStorage::RequestForSession>;
|
||||
using RequestsQueue = ConcurrentBoundedQueue<KeeperStorageBase::RequestForSession>;
|
||||
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
|
||||
using ClusterUpdateQueue = ConcurrentBoundedQueue<ClusterUpdateAction>;
|
||||
|
||||
@ -95,18 +95,18 @@ private:
|
||||
|
||||
/// Add error responses for requests to responses queue.
|
||||
/// Clears requests.
|
||||
void addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error);
|
||||
void addErrorResponses(const KeeperStorageBase::RequestsForSessions & requests_for_sessions, Coordination::Error error);
|
||||
|
||||
/// Forcefully wait for result and sets errors if something when wrong.
|
||||
/// Clears both arguments
|
||||
nuraft::ptr<nuraft::buffer> forceWaitAndProcessResult(
|
||||
RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions, bool clear_requests_on_success);
|
||||
RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions, bool clear_requests_on_success);
|
||||
|
||||
public:
|
||||
std::mutex read_request_queue_mutex;
|
||||
|
||||
/// queue of read requests that can be processed after a request with specific session ID and XID is committed
|
||||
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, KeeperStorage::RequestsForSessions>> read_request_queue;
|
||||
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, KeeperStorageBase::RequestsForSessions>> read_request_queue;
|
||||
|
||||
/// Just allocate some objects, real initialization is done by `intialize method`
|
||||
KeeperDispatcher();
|
||||
@ -192,7 +192,7 @@ public:
|
||||
|
||||
Keeper4LWInfo getKeeper4LWInfo() const;
|
||||
|
||||
const KeeperStateMachine & getStateMachine() const
|
||||
const IKeeperStateMachine & getStateMachine() const
|
||||
{
|
||||
return *server->getKeeperStateMachine();
|
||||
}
|
||||
|
@ -123,7 +123,7 @@ KeeperServer::KeeperServer(
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
KeeperContextPtr keeper_context_,
|
||||
KeeperSnapshotManagerS3 & snapshot_manager_s3,
|
||||
KeeperStateMachine::CommitCallback commit_callback)
|
||||
IKeeperStateMachine::CommitCallback commit_callback)
|
||||
: server_id(configuration_and_settings_->server_id)
|
||||
, log(getLogger("KeeperServer"))
|
||||
, is_recovering(config.getBool("keeper_server.force_recovery", false))
|
||||
@ -134,13 +134,28 @@ KeeperServer::KeeperServer(
|
||||
if (keeper_context->getCoordinationSettings()->quorum_reads)
|
||||
LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower.");
|
||||
|
||||
state_machine = nuraft::cs_new<KeeperStateMachine>(
|
||||
responses_queue_,
|
||||
snapshots_queue_,
|
||||
keeper_context,
|
||||
config.getBool("keeper_server.upload_snapshot_on_exit", false) ? &snapshot_manager_s3 : nullptr,
|
||||
commit_callback,
|
||||
checkAndGetSuperdigest(configuration_and_settings_->super_digest));
|
||||
#if USE_ROCKSDB
|
||||
const auto & coordination_settings = keeper_context->getCoordinationSettings();
|
||||
if (coordination_settings->experimental_use_rocksdb)
|
||||
{
|
||||
state_machine = nuraft::cs_new<KeeperStateMachine<KeeperRocksStorage>>(
|
||||
responses_queue_,
|
||||
snapshots_queue_,
|
||||
keeper_context,
|
||||
config.getBool("keeper_server.upload_snapshot_on_exit", false) ? &snapshot_manager_s3 : nullptr,
|
||||
commit_callback,
|
||||
checkAndGetSuperdigest(configuration_and_settings_->super_digest));
|
||||
LOG_WARNING(log, "Use RocksDB as Keeper backend storage.");
|
||||
}
|
||||
else
|
||||
#endif
|
||||
state_machine = nuraft::cs_new<KeeperStateMachine<KeeperMemoryStorage>>(
|
||||
responses_queue_,
|
||||
snapshots_queue_,
|
||||
keeper_context,
|
||||
config.getBool("keeper_server.upload_snapshot_on_exit", false) ? &snapshot_manager_s3 : nullptr,
|
||||
commit_callback,
|
||||
checkAndGetSuperdigest(configuration_and_settings_->super_digest));
|
||||
|
||||
state_manager = nuraft::cs_new<KeeperStateManager>(
|
||||
server_id,
|
||||
@ -522,7 +537,7 @@ namespace
|
||||
{
|
||||
|
||||
// Serialize the request for the log entry
|
||||
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestForSession & request_for_session)
|
||||
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
DB::WriteBufferFromNuraftBuffer write_buf;
|
||||
DB::writeIntBinary(request_for_session.session_id, write_buf);
|
||||
@ -530,7 +545,7 @@ nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestFor
|
||||
DB::writeIntBinary(request_for_session.time, write_buf);
|
||||
/// we fill with dummy values to eliminate unnecessary copy later on when we will write correct values
|
||||
DB::writeIntBinary(static_cast<int64_t>(0), write_buf); /// zxid
|
||||
DB::writeIntBinary(KeeperStorage::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag
|
||||
DB::writeIntBinary(KeeperStorageBase::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag
|
||||
DB::writeIntBinary(static_cast<uint64_t>(0), write_buf); /// digest value
|
||||
/// if new fields are added, update KeeperStateMachine::ZooKeeperLogSerializationVersion along with parseRequest function and PreAppendLog callback handler
|
||||
return write_buf.getBuffer();
|
||||
@ -538,7 +553,7 @@ nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestFor
|
||||
|
||||
}
|
||||
|
||||
void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & request_for_session)
|
||||
void KeeperServer::putLocalReadRequest(const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
if (!request_for_session.request->isReadRequest())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot process non-read request locally");
|
||||
@ -546,7 +561,7 @@ void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession &
|
||||
state_machine->processReadRequest(request_for_session);
|
||||
}
|
||||
|
||||
RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions)
|
||||
RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorageBase::RequestsForSessions & requests_for_sessions)
|
||||
{
|
||||
std::vector<nuraft::ptr<nuraft::buffer>> entries;
|
||||
entries.reserve(requests_for_sessions.size());
|
||||
@ -789,7 +804,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
|
||||
auto entry_buf = entry->get_buf_ptr();
|
||||
|
||||
KeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version;
|
||||
IKeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version;
|
||||
auto request_for_session = state_machine->parseRequest(*entry_buf, /*final=*/false, &serialization_version);
|
||||
request_for_session->zxid = next_zxid;
|
||||
if (!state_machine->preprocess(*request_for_session))
|
||||
@ -799,10 +814,10 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
|
||||
/// older versions of Keeper can send logs that are missing some fields
|
||||
size_t bytes_missing = 0;
|
||||
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
bytes_missing += sizeof(request_for_session->time);
|
||||
|
||||
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_ZXID_DIGEST)
|
||||
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_ZXID_DIGEST)
|
||||
bytes_missing += sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value);
|
||||
|
||||
if (bytes_missing != 0)
|
||||
@ -816,19 +831,19 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
size_t write_buffer_header_size
|
||||
= sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value);
|
||||
|
||||
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
write_buffer_header_size += sizeof(request_for_session->time);
|
||||
|
||||
auto * buffer_start = reinterpret_cast<BufferBase::Position>(entry_buf->data_begin() + entry_buf->size() - write_buffer_header_size);
|
||||
|
||||
WriteBufferFromPointer write_buf(buffer_start, write_buffer_header_size);
|
||||
|
||||
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
|
||||
writeIntBinary(request_for_session->time, write_buf);
|
||||
|
||||
writeIntBinary(request_for_session->zxid, write_buf);
|
||||
writeIntBinary(request_for_session->digest->version, write_buf);
|
||||
if (request_for_session->digest->version != KeeperStorage::NO_DIGEST)
|
||||
if (request_for_session->digest->version != KeeperStorageBase::NO_DIGEST)
|
||||
writeIntBinary(request_for_session->digest->value, write_buf);
|
||||
|
||||
write_buf.finalize();
|
||||
|
@ -24,7 +24,7 @@ class KeeperServer
|
||||
private:
|
||||
const int server_id;
|
||||
|
||||
nuraft::ptr<KeeperStateMachine> state_machine;
|
||||
nuraft::ptr<IKeeperStateMachine> state_machine;
|
||||
|
||||
nuraft::ptr<KeeperStateManager> state_manager;
|
||||
|
||||
@ -79,26 +79,26 @@ public:
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
KeeperContextPtr keeper_context_,
|
||||
KeeperSnapshotManagerS3 & snapshot_manager_s3,
|
||||
KeeperStateMachine::CommitCallback commit_callback);
|
||||
IKeeperStateMachine::CommitCallback commit_callback);
|
||||
|
||||
/// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings.
|
||||
void startup(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6 = true);
|
||||
|
||||
/// Put local read request and execute in state machine directly and response into
|
||||
/// responses queue
|
||||
void putLocalReadRequest(const KeeperStorage::RequestForSession & request);
|
||||
void putLocalReadRequest(const KeeperStorageBase::RequestForSession & request);
|
||||
|
||||
bool isRecovering() const { return is_recovering; }
|
||||
bool reconfigEnabled() const { return enable_reconfiguration; }
|
||||
|
||||
/// Put batch of requests into Raft and get result of put. Responses will be set separately into
|
||||
/// responses_queue.
|
||||
RaftAppendResult putRequestBatch(const KeeperStorage::RequestsForSessions & requests);
|
||||
RaftAppendResult putRequestBatch(const KeeperStorageBase::RequestsForSessions & requests);
|
||||
|
||||
/// Return set of the non-active sessions
|
||||
std::vector<int64_t> getDeadSessions();
|
||||
|
||||
nuraft::ptr<KeeperStateMachine> getKeeperStateMachine() const { return state_machine; }
|
||||
nuraft::ptr<IKeeperStateMachine> getKeeperStateMachine() const { return state_machine; }
|
||||
|
||||
void forceRecovery();
|
||||
|
||||
|
@ -66,7 +66,8 @@ namespace
|
||||
return base;
|
||||
}
|
||||
|
||||
void writeNode(const KeeperStorage::Node & node, SnapshotVersion version, WriteBuffer & out)
|
||||
template<typename Node>
|
||||
void writeNode(const Node & node, SnapshotVersion version, WriteBuffer & out)
|
||||
{
|
||||
writeBinary(node.getData(), out);
|
||||
|
||||
@ -86,7 +87,7 @@ namespace
|
||||
writeBinary(node.aversion, out);
|
||||
writeBinary(node.ephemeralOwner(), out);
|
||||
if (version < SnapshotVersion::V6)
|
||||
writeBinary(static_cast<int32_t>(node.data_size), out);
|
||||
writeBinary(static_cast<int32_t>(node.getData().size()), out);
|
||||
writeBinary(node.numChildren(), out);
|
||||
writeBinary(node.pzxid, out);
|
||||
|
||||
@ -96,7 +97,8 @@ namespace
|
||||
writeBinary(node.sizeInBytes(), out);
|
||||
}
|
||||
|
||||
void readNode(KeeperStorage::Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map)
|
||||
template<typename Node>
|
||||
void readNode(Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map)
|
||||
{
|
||||
readVarUInt(node.data_size, in);
|
||||
if (node.data_size != 0)
|
||||
@ -195,7 +197,8 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context)
|
||||
template<typename Storage>
|
||||
void KeeperStorageSnapshot<Storage>::serialize(const KeeperStorageSnapshot<Storage> & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context)
|
||||
{
|
||||
writeBinary(static_cast<uint8_t>(snapshot.version), out);
|
||||
serializeSnapshotMetadata(snapshot.snapshot_meta, out);
|
||||
@ -205,11 +208,11 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
||||
writeBinary(snapshot.zxid, out);
|
||||
if (keeper_context->digestEnabled())
|
||||
{
|
||||
writeBinary(static_cast<uint8_t>(KeeperStorage::CURRENT_DIGEST_VERSION), out);
|
||||
writeBinary(static_cast<uint8_t>(Storage::CURRENT_DIGEST_VERSION), out);
|
||||
writeBinary(snapshot.nodes_digest, out);
|
||||
}
|
||||
else
|
||||
writeBinary(static_cast<uint8_t>(KeeperStorage::NO_DIGEST), out);
|
||||
writeBinary(static_cast<uint8_t>(Storage::NO_DIGEST), out);
|
||||
}
|
||||
|
||||
writeBinary(snapshot.session_id, out);
|
||||
@ -255,7 +258,6 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
||||
/// slightly bigger than required.
|
||||
if (node.mzxid > snapshot.zxid)
|
||||
break;
|
||||
|
||||
writeBinary(path, out);
|
||||
writeNode(node, snapshot.version, out);
|
||||
|
||||
@ -282,7 +284,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
||||
writeBinary(session_id, out);
|
||||
writeBinary(timeout, out);
|
||||
|
||||
KeeperStorage::AuthIDs ids;
|
||||
KeeperStorageBase::AuthIDs ids;
|
||||
if (snapshot.session_and_auth.contains(session_id))
|
||||
ids = snapshot.session_and_auth.at(session_id);
|
||||
|
||||
@ -303,7 +305,8 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context)
|
||||
template<typename Storage>
|
||||
void KeeperStorageSnapshot<Storage>::deserialize(SnapshotDeserializationResult<Storage> & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context)
|
||||
{
|
||||
uint8_t version;
|
||||
readBinary(version, in);
|
||||
@ -312,7 +315,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported snapshot version {}", version);
|
||||
|
||||
deserialization_result.snapshot_meta = deserializeSnapshotMetadata(in);
|
||||
KeeperStorage & storage = *deserialization_result.storage;
|
||||
Storage & storage = *deserialization_result.storage;
|
||||
|
||||
bool recalculate_digest = keeper_context->digestEnabled();
|
||||
if (version >= SnapshotVersion::V5)
|
||||
@ -320,11 +323,11 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
readBinary(storage.zxid, in);
|
||||
uint8_t digest_version;
|
||||
readBinary(digest_version, in);
|
||||
if (digest_version != KeeperStorage::DigestVersion::NO_DIGEST)
|
||||
if (digest_version != Storage::DigestVersion::NO_DIGEST)
|
||||
{
|
||||
uint64_t nodes_digest;
|
||||
readBinary(nodes_digest, in);
|
||||
if (digest_version == KeeperStorage::CURRENT_DIGEST_VERSION)
|
||||
if (digest_version == Storage::CURRENT_DIGEST_VERSION)
|
||||
{
|
||||
storage.nodes_digest = nodes_digest;
|
||||
recalculate_digest = false;
|
||||
@ -374,8 +377,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
|
||||
size_t snapshot_container_size;
|
||||
readBinary(snapshot_container_size, in);
|
||||
|
||||
storage.container.reserve(snapshot_container_size);
|
||||
if constexpr (!use_rocksdb)
|
||||
storage.container.reserve(snapshot_container_size);
|
||||
|
||||
if (recalculate_digest)
|
||||
storage.nodes_digest = 0;
|
||||
@ -389,7 +392,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
in.readStrict(path_data.get(), path_size);
|
||||
std::string_view path{path_data.get(), path_size};
|
||||
|
||||
KeeperStorage::Node node{};
|
||||
typename Storage::Node node{};
|
||||
readNode(node, in, current_version, storage.acl_map);
|
||||
|
||||
using enum Coordination::PathMatchResult;
|
||||
@ -421,7 +424,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT)
|
||||
{
|
||||
LOG_ERROR(getLogger("KeeperSnapshotManager"), "{}. Ignoring it", get_error_msg());
|
||||
node = KeeperStorage::Node{};
|
||||
node = typename Storage::Node{};
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
@ -433,8 +436,9 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
}
|
||||
|
||||
auto ephemeral_owner = node.ephemeralOwner();
|
||||
if (!node.isEphemeral() && node.numChildren() > 0)
|
||||
node.getChildren().reserve(node.numChildren());
|
||||
if constexpr (!use_rocksdb)
|
||||
if (!node.isEphemeral() && node.numChildren() > 0)
|
||||
node.getChildren().reserve(node.numChildren());
|
||||
|
||||
if (ephemeral_owner != 0)
|
||||
storage.ephemerals[node.ephemeralOwner()].insert(std::string{path});
|
||||
@ -447,36 +451,38 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
|
||||
LOG_TRACE(getLogger("KeeperSnapshotManager"), "Building structure for children nodes");
|
||||
|
||||
for (const auto & itr : storage.container)
|
||||
if constexpr (!use_rocksdb)
|
||||
{
|
||||
if (itr.key != "/")
|
||||
for (const auto & itr : storage.container)
|
||||
{
|
||||
auto parent_path = parentNodePath(itr.key);
|
||||
storage.container.updateValue(
|
||||
parent_path, [path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path)); });
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & itr : storage.container)
|
||||
{
|
||||
if (itr.key != "/")
|
||||
{
|
||||
if (itr.value.numChildren() != static_cast<int32_t>(itr.value.getChildren().size()))
|
||||
if (itr.key != "/")
|
||||
{
|
||||
auto parent_path = parentNodePath(itr.key);
|
||||
storage.container.updateValue(
|
||||
parent_path, [path = itr.key](typename Storage::Node & value) { value.addChild(getBaseNodeName(path)); });
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & itr : storage.container)
|
||||
{
|
||||
if (itr.key != "/")
|
||||
{
|
||||
if (itr.value.numChildren() != static_cast<int32_t>(itr.value.getChildren().size()))
|
||||
{
|
||||
#ifdef NDEBUG
|
||||
/// TODO (alesapin) remove this, it should be always CORRUPTED_DATA.
|
||||
LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}"
|
||||
" is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
|
||||
/// TODO (alesapin) remove this, it should be always CORRUPTED_DATA.
|
||||
LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}"
|
||||
" is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
|
||||
#else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}"
|
||||
" is different from actual children size {} for node {}",
|
||||
itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}"
|
||||
" is different from actual children size {} for node {}",
|
||||
itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
|
||||
#endif
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
size_t active_sessions_size;
|
||||
readBinary(active_sessions_size, in);
|
||||
|
||||
@ -493,14 +499,14 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
size_t session_auths_size;
|
||||
readBinary(session_auths_size, in);
|
||||
|
||||
KeeperStorage::AuthIDs ids;
|
||||
typename Storage::AuthIDs ids;
|
||||
size_t session_auth_counter = 0;
|
||||
while (session_auth_counter < session_auths_size)
|
||||
{
|
||||
String scheme, id;
|
||||
readBinary(scheme, in);
|
||||
readBinary(id, in);
|
||||
ids.emplace_back(KeeperStorage::AuthID{scheme, id});
|
||||
ids.emplace_back(typename Storage::AuthID{scheme, id});
|
||||
|
||||
session_auth_counter++;
|
||||
}
|
||||
@ -523,7 +529,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
|
||||
}
|
||||
}
|
||||
|
||||
KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_)
|
||||
template<typename Storage>
|
||||
KeeperStorageSnapshot<Storage>::KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_)
|
||||
: storage(storage_)
|
||||
, snapshot_meta(std::make_shared<SnapshotMetadata>(up_to_log_idx_, 0, std::make_shared<nuraft::cluster_config>()))
|
||||
, session_id(storage->session_id_counter)
|
||||
@ -540,8 +547,9 @@ KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t
|
||||
session_and_auth = storage->session_and_auth;
|
||||
}
|
||||
|
||||
KeeperStorageSnapshot::KeeperStorageSnapshot(
|
||||
KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_)
|
||||
template<typename Storage>
|
||||
KeeperStorageSnapshot<Storage>::KeeperStorageSnapshot(
|
||||
Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_)
|
||||
: storage(storage_)
|
||||
, snapshot_meta(snapshot_meta_)
|
||||
, session_id(storage->session_id_counter)
|
||||
@ -558,12 +566,14 @@ KeeperStorageSnapshot::KeeperStorageSnapshot(
|
||||
session_and_auth = storage->session_and_auth;
|
||||
}
|
||||
|
||||
KeeperStorageSnapshot::~KeeperStorageSnapshot()
|
||||
template<typename Storage>
|
||||
KeeperStorageSnapshot<Storage>::~KeeperStorageSnapshot()
|
||||
{
|
||||
storage->disableSnapshotMode();
|
||||
}
|
||||
|
||||
KeeperSnapshotManager::KeeperSnapshotManager(
|
||||
template<typename Storage>
|
||||
KeeperSnapshotManager<Storage>::KeeperSnapshotManager(
|
||||
size_t snapshots_to_keep_,
|
||||
const KeeperContextPtr & keeper_context_,
|
||||
bool compress_snapshots_zstd_,
|
||||
@ -651,7 +661,8 @@ KeeperSnapshotManager::KeeperSnapshotManager(
|
||||
moveSnapshotsIfNeeded();
|
||||
}
|
||||
|
||||
SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx)
|
||||
template<typename Storage>
|
||||
SnapshotFileInfoPtr KeeperSnapshotManager<Storage>::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx)
|
||||
{
|
||||
ReadBufferFromNuraftBuffer reader(buffer);
|
||||
|
||||
@ -680,7 +691,8 @@ SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft:
|
||||
return snapshot_file_info;
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk()
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager<Storage>::deserializeLatestSnapshotBufferFromDisk()
|
||||
{
|
||||
while (!existing_snapshots.empty())
|
||||
{
|
||||
@ -701,7 +713,8 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBuff
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager<Storage>::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const
|
||||
{
|
||||
const auto & [snapshot_path, snapshot_disk, size] = *existing_snapshots.at(up_to_log_idx);
|
||||
WriteBufferFromNuraftBuffer writer;
|
||||
@ -710,7 +723,8 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFrom
|
||||
return writer.getBuffer();
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager<Storage>::serializeSnapshotToBuffer(const KeeperStorageSnapshot<Storage> & snapshot) const
|
||||
{
|
||||
std::unique_ptr<WriteBufferFromNuraftBuffer> writer = std::make_unique<WriteBufferFromNuraftBuffer>();
|
||||
auto * buffer_raw_ptr = writer.get();
|
||||
@ -720,13 +734,13 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::serializeSnapshotToBuffer(con
|
||||
else
|
||||
compressed_writer = std::make_unique<CompressedWriteBuffer>(*writer);
|
||||
|
||||
KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context);
|
||||
KeeperStorageSnapshot<Storage>::serialize(snapshot, *compressed_writer, keeper_context);
|
||||
compressed_writer->finalize();
|
||||
return buffer_raw_ptr->getBuffer();
|
||||
}
|
||||
|
||||
|
||||
bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
|
||||
template<typename Storage>
|
||||
bool KeeperSnapshotManager<Storage>::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
|
||||
{
|
||||
static constexpr unsigned char ZSTD_COMPRESSED_MAGIC[4] = {0x28, 0xB5, 0x2F, 0xFD};
|
||||
|
||||
@ -737,7 +751,8 @@ bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
|
||||
return memcmp(magic_from_buffer, ZSTD_COMPRESSED_MAGIC, 4) == 0;
|
||||
}
|
||||
|
||||
SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const
|
||||
template<typename Storage>
|
||||
SnapshotDeserializationResult<Storage> KeeperSnapshotManager<Storage>::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const
|
||||
{
|
||||
bool is_zstd_compressed = isZstdCompressed(buffer);
|
||||
|
||||
@ -749,14 +764,15 @@ SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuff
|
||||
else
|
||||
compressed_reader = std::make_unique<CompressedReadBuffer>(*reader);
|
||||
|
||||
SnapshotDeserializationResult result;
|
||||
result.storage = std::make_unique<KeeperStorage>(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false);
|
||||
KeeperStorageSnapshot::deserialize(result, *compressed_reader, keeper_context);
|
||||
SnapshotDeserializationResult<Storage> result;
|
||||
result.storage = std::make_unique<Storage>(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false);
|
||||
KeeperStorageSnapshot<Storage>::deserialize(result, *compressed_reader, keeper_context);
|
||||
result.storage->initializeSystemNodes();
|
||||
return result;
|
||||
}
|
||||
|
||||
SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot()
|
||||
template<typename Storage>
|
||||
SnapshotDeserializationResult<Storage> KeeperSnapshotManager<Storage>::restoreFromLatestSnapshot()
|
||||
{
|
||||
if (existing_snapshots.empty())
|
||||
return {};
|
||||
@ -767,23 +783,27 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot()
|
||||
return deserializeSnapshotFromBuffer(buffer);
|
||||
}
|
||||
|
||||
DiskPtr KeeperSnapshotManager::getDisk() const
|
||||
template<typename Storage>
|
||||
DiskPtr KeeperSnapshotManager<Storage>::getDisk() const
|
||||
{
|
||||
return keeper_context->getSnapshotDisk();
|
||||
}
|
||||
|
||||
DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const
|
||||
template<typename Storage>
|
||||
DiskPtr KeeperSnapshotManager<Storage>::getLatestSnapshotDisk() const
|
||||
{
|
||||
return keeper_context->getLatestSnapshotDisk();
|
||||
}
|
||||
|
||||
void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded()
|
||||
template<typename Storage>
|
||||
void KeeperSnapshotManager<Storage>::removeOutdatedSnapshotsIfNeeded()
|
||||
{
|
||||
while (existing_snapshots.size() > snapshots_to_keep)
|
||||
removeSnapshot(existing_snapshots.begin()->first);
|
||||
}
|
||||
|
||||
void KeeperSnapshotManager::moveSnapshotsIfNeeded()
|
||||
template<typename Storage>
|
||||
void KeeperSnapshotManager<Storage>::moveSnapshotsIfNeeded()
|
||||
{
|
||||
/// move snapshots to correct disks
|
||||
|
||||
@ -813,7 +833,8 @@ void KeeperSnapshotManager::moveSnapshotsIfNeeded()
|
||||
|
||||
}
|
||||
|
||||
void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx)
|
||||
template<typename Storage>
|
||||
void KeeperSnapshotManager<Storage>::removeSnapshot(uint64_t log_idx)
|
||||
{
|
||||
auto itr = existing_snapshots.find(log_idx);
|
||||
if (itr == existing_snapshots.end())
|
||||
@ -823,7 +844,8 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx)
|
||||
existing_snapshots.erase(itr);
|
||||
}
|
||||
|
||||
SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot)
|
||||
template<typename Storage>
|
||||
SnapshotFileInfoPtr KeeperSnapshotManager<Storage>::serializeSnapshotToDisk(const KeeperStorageSnapshot<Storage> & snapshot)
|
||||
{
|
||||
auto up_to_log_idx = snapshot.snapshot_meta->get_last_log_idx();
|
||||
auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd);
|
||||
@ -842,7 +864,7 @@ SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperS
|
||||
else
|
||||
compressed_writer = std::make_unique<CompressedWriteBuffer>(*writer);
|
||||
|
||||
KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context);
|
||||
KeeperStorageSnapshot<Storage>::serialize(snapshot, *compressed_writer, keeper_context);
|
||||
compressed_writer->finalize();
|
||||
compressed_writer->sync();
|
||||
|
||||
@ -864,14 +886,16 @@ SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperS
|
||||
return snapshot_file_info;
|
||||
}
|
||||
|
||||
size_t KeeperSnapshotManager::getLatestSnapshotIndex() const
|
||||
template<typename Storage>
|
||||
size_t KeeperSnapshotManager<Storage>::getLatestSnapshotIndex() const
|
||||
{
|
||||
if (!existing_snapshots.empty())
|
||||
return existing_snapshots.rbegin()->first;
|
||||
return 0;
|
||||
}
|
||||
|
||||
SnapshotFileInfoPtr KeeperSnapshotManager::getLatestSnapshotInfo() const
|
||||
template<typename Storage>
|
||||
SnapshotFileInfoPtr KeeperSnapshotManager<Storage>::getLatestSnapshotInfo() const
|
||||
{
|
||||
if (!existing_snapshots.empty())
|
||||
{
|
||||
@ -890,4 +914,10 @@ SnapshotFileInfoPtr KeeperSnapshotManager::getLatestSnapshotInfo() const
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template struct KeeperStorageSnapshot<KeeperMemoryStorage>;
|
||||
template class KeeperSnapshotManager<KeeperMemoryStorage>;
|
||||
#if USE_ROCKSDB
|
||||
template struct KeeperStorageSnapshot<KeeperRocksStorage>;
|
||||
template class KeeperSnapshotManager<KeeperRocksStorage>;
|
||||
#endif
|
||||
}
|
||||
|
@ -34,10 +34,11 @@ enum SnapshotVersion : uint8_t
|
||||
static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V6;
|
||||
|
||||
/// What is stored in binary snapshot
|
||||
template<typename Storage>
|
||||
struct SnapshotDeserializationResult
|
||||
{
|
||||
/// Storage
|
||||
KeeperStoragePtr storage;
|
||||
std::unique_ptr<Storage> storage;
|
||||
/// Snapshot metadata (up_to_log_idx and so on)
|
||||
SnapshotMetadataPtr snapshot_meta;
|
||||
/// Cluster config
|
||||
@ -52,21 +53,31 @@ struct SnapshotDeserializationResult
|
||||
///
|
||||
/// This representation of snapshot have to be serialized into NuRaft
|
||||
/// buffer and send over network or saved to file.
|
||||
template<typename Storage>
|
||||
struct KeeperStorageSnapshot
|
||||
{
|
||||
#if USE_ROCKSDB
|
||||
static constexpr bool use_rocksdb = std::is_same_v<Storage, KeeperRocksStorage>;
|
||||
#else
|
||||
static constexpr bool use_rocksdb = false;
|
||||
#endif
|
||||
|
||||
public:
|
||||
KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr);
|
||||
KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr);
|
||||
|
||||
KeeperStorageSnapshot(
|
||||
KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr);
|
||||
Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr);
|
||||
|
||||
KeeperStorageSnapshot(const KeeperStorageSnapshot<Storage>&) = delete;
|
||||
KeeperStorageSnapshot(KeeperStorageSnapshot<Storage>&&) = default;
|
||||
|
||||
~KeeperStorageSnapshot();
|
||||
|
||||
static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context);
|
||||
static void serialize(const KeeperStorageSnapshot<Storage> & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context);
|
||||
|
||||
static void deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context);
|
||||
static void deserialize(SnapshotDeserializationResult<Storage> & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context);
|
||||
|
||||
KeeperStorage * storage;
|
||||
Storage * storage;
|
||||
|
||||
SnapshotVersion version = CURRENT_SNAPSHOT_VERSION;
|
||||
/// Snapshot metadata
|
||||
@ -77,11 +88,11 @@ public:
|
||||
/// so we have for loop for (i = 0; i < snapshot_container_size; ++i) { doSmth(begin + i); }
|
||||
size_t snapshot_container_size;
|
||||
/// Iterator to the start of the storage
|
||||
KeeperStorage::Container::const_iterator begin;
|
||||
Storage::Container::const_iterator begin;
|
||||
/// Active sessions and their timeouts
|
||||
SessionAndTimeout session_and_timeout;
|
||||
/// Sessions credentials
|
||||
KeeperStorage::SessionAndAuth session_and_auth;
|
||||
Storage::SessionAndAuth session_and_auth;
|
||||
/// ACLs cache for better performance. Without we cannot deserialize storage.
|
||||
std::unordered_map<uint64_t, Coordination::ACLs> acl_map;
|
||||
/// Cluster config from snapshot, can be empty
|
||||
@ -105,14 +116,16 @@ struct SnapshotFileInfo
|
||||
};
|
||||
|
||||
using SnapshotFileInfoPtr = std::shared_ptr<SnapshotFileInfo>;
|
||||
|
||||
using KeeperStorageSnapshotPtr = std::shared_ptr<KeeperStorageSnapshot>;
|
||||
using CreateSnapshotCallback = std::function<std::shared_ptr<SnapshotFileInfo>(KeeperStorageSnapshotPtr &&, bool)>;
|
||||
|
||||
using SnapshotMetaAndStorage = std::pair<SnapshotMetadataPtr, KeeperStoragePtr>;
|
||||
#if USE_ROCKSDB
|
||||
using KeeperStorageSnapshotPtr = std::variant<std::shared_ptr<KeeperStorageSnapshot<KeeperMemoryStorage>>, std::shared_ptr<KeeperStorageSnapshot<KeeperRocksStorage>>>;
|
||||
#else
|
||||
using KeeperStorageSnapshotPtr = std::variant<std::shared_ptr<KeeperStorageSnapshot<KeeperMemoryStorage>>>;
|
||||
#endif
|
||||
using CreateSnapshotCallback = std::function<SnapshotFileInfoPtr(KeeperStorageSnapshotPtr &&, bool)>;
|
||||
|
||||
/// Class responsible for snapshots serialization and deserialization. Each snapshot
|
||||
/// has it's path on disk and log index.
|
||||
template<typename Storage>
|
||||
class KeeperSnapshotManager
|
||||
{
|
||||
public:
|
||||
@ -124,18 +137,18 @@ public:
|
||||
size_t storage_tick_time_ = 500);
|
||||
|
||||
/// Restore storage from latest available snapshot
|
||||
SnapshotDeserializationResult restoreFromLatestSnapshot();
|
||||
SnapshotDeserializationResult<Storage> restoreFromLatestSnapshot();
|
||||
|
||||
/// Compress snapshot and serialize it to buffer
|
||||
nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const;
|
||||
nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot<Storage> & snapshot) const;
|
||||
|
||||
/// Serialize already compressed snapshot to disk (return path)
|
||||
SnapshotFileInfoPtr serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx);
|
||||
|
||||
/// Serialize snapshot directly to disk
|
||||
SnapshotFileInfoPtr serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot);
|
||||
SnapshotFileInfoPtr serializeSnapshotToDisk(const KeeperStorageSnapshot<Storage> & snapshot);
|
||||
|
||||
SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
|
||||
SnapshotDeserializationResult<Storage> deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
|
||||
|
||||
/// Deserialize snapshot with log index up_to_log_idx from disk into compressed nuraft buffer.
|
||||
nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const;
|
||||
|
@ -44,7 +44,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
KeeperStateMachine::KeeperStateMachine(
|
||||
IKeeperStateMachine::IKeeperStateMachine(
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
const KeeperContextPtr & keeper_context_,
|
||||
@ -52,12 +52,6 @@ KeeperStateMachine::KeeperStateMachine(
|
||||
CommitCallback commit_callback_,
|
||||
const std::string & superdigest_)
|
||||
: commit_callback(commit_callback_)
|
||||
, snapshot_manager(
|
||||
keeper_context_->getCoordinationSettings()->snapshots_to_keep,
|
||||
keeper_context_,
|
||||
keeper_context_->getCoordinationSettings()->compress_snapshots_with_zstd_format,
|
||||
superdigest_,
|
||||
keeper_context_->getCoordinationSettings()->dead_session_check_period_ms.totalMilliseconds())
|
||||
, responses_queue(responses_queue_)
|
||||
, snapshots_queue(snapshots_queue_)
|
||||
, min_request_size_to_cache(keeper_context_->getCoordinationSettings()->min_request_size_for_cache)
|
||||
@ -68,6 +62,32 @@ KeeperStateMachine::KeeperStateMachine(
|
||||
{
|
||||
}
|
||||
|
||||
template<typename Storage>
|
||||
KeeperStateMachine<Storage>::KeeperStateMachine(
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
// const CoordinationSettingsPtr & coordination_settings_,
|
||||
const KeeperContextPtr & keeper_context_,
|
||||
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
|
||||
IKeeperStateMachine::CommitCallback commit_callback_,
|
||||
const std::string & superdigest_)
|
||||
: IKeeperStateMachine(
|
||||
responses_queue_,
|
||||
snapshots_queue_,
|
||||
/// coordination_settings_,
|
||||
keeper_context_,
|
||||
snapshot_manager_s3_,
|
||||
commit_callback_,
|
||||
superdigest_),
|
||||
snapshot_manager(
|
||||
keeper_context_->getCoordinationSettings()->snapshots_to_keep,
|
||||
keeper_context_,
|
||||
keeper_context_->getCoordinationSettings()->compress_snapshots_with_zstd_format,
|
||||
superdigest_,
|
||||
keeper_context_->getCoordinationSettings()->dead_session_check_period_ms.totalMilliseconds())
|
||||
{
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -78,7 +98,8 @@ bool isLocalDisk(const IDisk & disk)
|
||||
|
||||
}
|
||||
|
||||
void KeeperStateMachine::init()
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::init()
|
||||
{
|
||||
/// Do everything without mutexes, no other threads exist.
|
||||
LOG_DEBUG(log, "Totally have {} snapshots", snapshot_manager.totalSnapshots());
|
||||
@ -123,7 +144,7 @@ void KeeperStateMachine::init()
|
||||
LOG_DEBUG(log, "No existing snapshots, last committed log index {}", last_committed_idx);
|
||||
|
||||
if (!storage)
|
||||
storage = std::make_unique<KeeperStorage>(
|
||||
storage = std::make_unique<Storage>(
|
||||
keeper_context->getCoordinationSettings()->dead_session_check_period_ms.totalMilliseconds(), superdigest, keeper_context);
|
||||
}
|
||||
|
||||
@ -131,13 +152,13 @@ namespace
|
||||
{
|
||||
|
||||
void assertDigest(
|
||||
const KeeperStorage::Digest & expected,
|
||||
const KeeperStorage::Digest & actual,
|
||||
const KeeperStorageBase::Digest & expected,
|
||||
const KeeperStorageBase::Digest & actual,
|
||||
const Coordination::ZooKeeperRequest & request,
|
||||
uint64_t log_idx,
|
||||
bool committing)
|
||||
{
|
||||
if (!KeeperStorage::checkDigest(expected, actual))
|
||||
if (!KeeperStorageBase::checkDigest(expected, actual))
|
||||
{
|
||||
LOG_FATAL(
|
||||
getLogger("KeeperStateMachine"),
|
||||
@ -170,7 +191,8 @@ struct TSA_SCOPED_LOCKABLE LockGuardWithStats final
|
||||
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data)
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine<Storage>::pre_commit(uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
auto result = nuraft::buffer::alloc(sizeof(log_idx));
|
||||
nuraft::buffer_serializer ss(result);
|
||||
@ -191,10 +213,10 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::pre_commit(uint64_t log_idx, nur
|
||||
return result;
|
||||
}
|
||||
|
||||
std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version)
|
||||
std::shared_ptr<KeeperStorageBase::RequestForSession> IKeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version)
|
||||
{
|
||||
ReadBufferFromNuraftBuffer buffer(data);
|
||||
auto request_for_session = std::make_shared<KeeperStorage::RequestForSession>();
|
||||
auto request_for_session = std::make_shared<KeeperStorageBase::RequestForSession>();
|
||||
readIntBinary(request_for_session->session_id, buffer);
|
||||
|
||||
int32_t length;
|
||||
@ -267,7 +289,7 @@ std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseReque
|
||||
|
||||
request_for_session->digest.emplace();
|
||||
readIntBinary(request_for_session->digest->version, buffer);
|
||||
if (request_for_session->digest->version != KeeperStorage::DigestVersion::NO_DIGEST || !buffer.eof())
|
||||
if (request_for_session->digest->version != KeeperStorageBase::DigestVersion::NO_DIGEST || !buffer.eof())
|
||||
readIntBinary(request_for_session->digest->value, buffer);
|
||||
}
|
||||
|
||||
@ -283,7 +305,8 @@ std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseReque
|
||||
return request_for_session;
|
||||
}
|
||||
|
||||
bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session)
|
||||
template<typename Storage>
|
||||
bool KeeperStateMachine<Storage>::preprocess(const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
const auto op_num = request_for_session.request->getOpNum();
|
||||
if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig)
|
||||
@ -317,10 +340,11 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req
|
||||
return true;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::reconfigure(const KeeperStorageBase::RequestForSession& request_for_session)
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session);
|
||||
KeeperStorageBase::ResponseForSession response = processReconfiguration(request_for_session);
|
||||
if (!responses_queue.push(response))
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed);
|
||||
@ -330,8 +354,9 @@ void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& req
|
||||
}
|
||||
}
|
||||
|
||||
KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
|
||||
const KeeperStorage::RequestForSession & request_for_session)
|
||||
template<typename Storage>
|
||||
KeeperStorageBase::ResponseForSession KeeperStateMachine<Storage>::processReconfiguration(
|
||||
const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest);
|
||||
|
||||
@ -340,7 +365,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
|
||||
const int64_t zxid = request_for_session.zxid;
|
||||
|
||||
using enum Coordination::Error;
|
||||
auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorage::ResponseForSession
|
||||
auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorageBase::ResponseForSession
|
||||
{
|
||||
auto res = std::make_shared<Coordination::ZooKeeperReconfigResponse>();
|
||||
res->xid = request.xid;
|
||||
@ -397,7 +422,8 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
|
||||
return { session_id, std::move(response) };
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)
|
||||
template<typename Storage>
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine<Storage>::commit(const uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
auto request_for_session = parseRequest(data, true);
|
||||
if (!request_for_session->zxid)
|
||||
@ -408,7 +434,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
if (!keeper_context->localLogsPreprocessed() && !preprocess(*request_for_session))
|
||||
return nullptr;
|
||||
|
||||
auto try_push = [&](const KeeperStorage::ResponseForSession & response)
|
||||
auto try_push = [&](const KeeperStorageBase::ResponseForSession & response)
|
||||
{
|
||||
if (!responses_queue.push(response))
|
||||
{
|
||||
@ -430,7 +456,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
std::shared_ptr<Coordination::ZooKeeperSessionIDResponse> response = std::make_shared<Coordination::ZooKeeperSessionIDResponse>();
|
||||
response->internal_id = session_id_request.internal_id;
|
||||
response->server_id = session_id_request.server_id;
|
||||
KeeperStorage::ResponseForSession response_for_session;
|
||||
KeeperStorageBase::ResponseForSession response_for_session;
|
||||
response_for_session.session_id = -1;
|
||||
response_for_session.response = response;
|
||||
response_for_session.request = request_for_session->request;
|
||||
@ -451,7 +477,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
}
|
||||
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
KeeperStorage::ResponsesForSessions responses_for_sessions
|
||||
KeeperStorageBase::ResponsesForSessions responses_for_sessions
|
||||
= storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid);
|
||||
|
||||
for (auto & response_for_session : responses_for_sessions)
|
||||
@ -482,7 +508,8 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
|
||||
template<typename Storage>
|
||||
bool KeeperStateMachine<Storage>::apply_snapshot(nuraft::snapshot & s)
|
||||
{
|
||||
LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx());
|
||||
nuraft::ptr<nuraft::buffer> latest_snapshot_ptr;
|
||||
@ -509,7 +536,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
|
||||
{ /// deserialize and apply snapshot to storage
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
|
||||
SnapshotDeserializationResult snapshot_deserialization_result;
|
||||
SnapshotDeserializationResult<Storage> snapshot_deserialization_result;
|
||||
if (latest_snapshot_ptr)
|
||||
snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr);
|
||||
else
|
||||
@ -530,7 +557,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
|
||||
}
|
||||
|
||||
|
||||
void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf)
|
||||
void IKeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf)
|
||||
{
|
||||
std::lock_guard lock(cluster_config_lock);
|
||||
auto tmp = new_conf->serialize();
|
||||
@ -538,7 +565,7 @@ void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr<nuraf
|
||||
keeper_context->setLastCommitIndex(log_idx);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
|
||||
void IKeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
/// Don't rollback anything until the first commit because nothing was preprocessed
|
||||
if (!keeper_context->localLogsPreprocessed())
|
||||
@ -554,7 +581,8 @@ void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
|
||||
rollbackRequest(*request_for_session, false);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing)
|
||||
{
|
||||
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
|
||||
return;
|
||||
@ -563,7 +591,8 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession
|
||||
storage->rollbackRequest(request_for_session.zxid, allow_missing);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::rollbackRequestNoLock(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing)
|
||||
{
|
||||
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
|
||||
return;
|
||||
@ -571,14 +600,15 @@ void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSe
|
||||
storage->rollbackRequest(request_for_session.zxid, allow_missing);
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::snapshot> KeeperStateMachine::last_snapshot()
|
||||
nuraft::ptr<nuraft::snapshot> IKeeperStateMachine::last_snapshot()
|
||||
{
|
||||
/// Just return the latest snapshot.
|
||||
std::lock_guard lock(snapshots_lock);
|
||||
return latest_snapshot_meta;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done)
|
||||
{
|
||||
LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx());
|
||||
|
||||
@ -587,14 +617,15 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
|
||||
CreateSnapshotTask snapshot_task;
|
||||
{ /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking.
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
snapshot_task.snapshot = std::make_shared<KeeperStorageSnapshot>(storage.get(), snapshot_meta_copy, getClusterConfig());
|
||||
snapshot_task.snapshot = std::make_shared<KeeperStorageSnapshot<Storage>>(storage.get(), snapshot_meta_copy, getClusterConfig());
|
||||
}
|
||||
|
||||
/// create snapshot task for background execution (in snapshot thread)
|
||||
snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot, bool execute_only_cleanup)
|
||||
snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot_, bool execute_only_cleanup)
|
||||
{
|
||||
nuraft::ptr<std::exception> exception(nullptr);
|
||||
bool ret = false;
|
||||
auto && snapshot = std::get<std::shared_ptr<KeeperStorageSnapshot<Storage>>>(std::move(snapshot_));
|
||||
if (!execute_only_cleanup)
|
||||
{
|
||||
try
|
||||
@ -683,7 +714,8 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
|
||||
LOG_WARNING(log, "Cannot push snapshot task into queue");
|
||||
}
|
||||
|
||||
void KeeperStateMachine::save_logical_snp_obj(
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::save_logical_snp_obj(
|
||||
nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool /*is_first_obj*/, bool /*is_last_obj*/)
|
||||
{
|
||||
LOG_DEBUG(log, "Saving snapshot {} obj_id {}", s.get_last_log_idx(), obj_id);
|
||||
@ -748,7 +780,7 @@ static int bufferFromFile(LoggerPtr log, const std::string & path, nuraft::ptr<n
|
||||
return 0;
|
||||
}
|
||||
|
||||
int KeeperStateMachine::read_logical_snp_obj(
|
||||
int IKeeperStateMachine::read_logical_snp_obj(
|
||||
nuraft::snapshot & s, void *& /*user_snp_ctx*/, uint64_t obj_id, nuraft::ptr<nuraft::buffer> & data_out, bool & is_last_obj)
|
||||
{
|
||||
LOG_DEBUG(log, "Reading snapshot {} obj_id {}", s.get_last_log_idx(), obj_id);
|
||||
@ -788,7 +820,8 @@ int KeeperStateMachine::read_logical_snp_obj(
|
||||
return 1;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session)
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
{
|
||||
/// Pure local request, just process it with storage
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
@ -804,103 +837,120 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperStateMachine::shutdownStorage()
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::shutdownStorage()
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
storage->finalize();
|
||||
}
|
||||
|
||||
std::vector<int64_t> KeeperStateMachine::getDeadSessions()
|
||||
template<typename Storage>
|
||||
std::vector<int64_t> KeeperStateMachine<Storage>::getDeadSessions()
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getDeadSessions();
|
||||
}
|
||||
|
||||
int64_t KeeperStateMachine::getNextZxid() const
|
||||
template<typename Storage>
|
||||
int64_t KeeperStateMachine<Storage>::getNextZxid() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getNextZXID();
|
||||
}
|
||||
|
||||
KeeperStorage::Digest KeeperStateMachine::getNodesDigest() const
|
||||
template<typename Storage>
|
||||
KeeperStorageBase::Digest KeeperStateMachine<Storage>::getNodesDigest() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getNodesDigest(false);
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getLastProcessedZxid() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getLastProcessedZxid() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getZXID();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getNodesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getNodesCount() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getNodesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getTotalWatchesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getTotalWatchesCount() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getTotalWatchesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getWatchedPathsCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getWatchedPathsCount() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getWatchedPathsCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getSessionsWithWatchesCount() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getSessionsWithWatchesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getTotalEphemeralNodesCount() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getTotalEphemeralNodesCount();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getSessionWithEphemeralNodesCount() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getSessionWithEphemeralNodesCount();
|
||||
}
|
||||
|
||||
void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::dumpWatches(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
storage->dumpWatches(buf);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
storage->dumpWatchesByPath(buf);
|
||||
}
|
||||
|
||||
void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
storage->dumpSessionsAndEphemerals(buf);
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getApproximateDataSize() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getApproximateDataSize() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getApproximateDataSize();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getKeyArenaSize() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getKeyArenaSize() const
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
return storage->getArenaDataSize();
|
||||
}
|
||||
|
||||
uint64_t KeeperStateMachine::getLatestSnapshotSize() const
|
||||
template<typename Storage>
|
||||
uint64_t KeeperStateMachine<Storage>::getLatestSnapshotSize() const
|
||||
{
|
||||
auto snapshot_info = [&]
|
||||
{
|
||||
@ -923,7 +973,7 @@ uint64_t KeeperStateMachine::getLatestSnapshotSize() const
|
||||
return size;
|
||||
}
|
||||
|
||||
ClusterConfigPtr KeeperStateMachine::getClusterConfig() const
|
||||
ClusterConfigPtr IKeeperStateMachine::getClusterConfig() const
|
||||
{
|
||||
std::lock_guard lock(cluster_config_lock);
|
||||
if (cluster_config)
|
||||
@ -935,11 +985,18 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::recalculateStorageStats()
|
||||
template<typename Storage>
|
||||
void KeeperStateMachine<Storage>::recalculateStorageStats()
|
||||
{
|
||||
LockGuardWithStats lock(storage_and_responses_lock);
|
||||
LOG_INFO(log, "Recalculating storage stats");
|
||||
storage->recalculateStats();
|
||||
LOG_INFO(log, "Done recalculating storage stats");
|
||||
}
|
||||
|
||||
template class KeeperStateMachine<KeeperMemoryStorage>;
|
||||
#if USE_ROCKSDB
|
||||
template class KeeperStateMachine<KeeperRocksStorage>;
|
||||
#endif
|
||||
|
||||
}
|
||||
|
@ -11,26 +11,24 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
using ResponsesQueue = ConcurrentBoundedQueue<KeeperStorage::ResponseForSession>;
|
||||
using ResponsesQueue = ConcurrentBoundedQueue<KeeperStorageBase::ResponseForSession>;
|
||||
using SnapshotsQueue = ConcurrentBoundedQueue<CreateSnapshotTask>;
|
||||
|
||||
/// ClickHouse Keeper state machine. Wrapper for KeeperStorage.
|
||||
/// Responsible for entries commit, snapshots creation and so on.
|
||||
class KeeperStateMachine : public nuraft::state_machine
|
||||
class IKeeperStateMachine : public nuraft::state_machine
|
||||
{
|
||||
public:
|
||||
using CommitCallback = std::function<void(uint64_t, const KeeperStorage::RequestForSession &)>;
|
||||
using CommitCallback = std::function<void(uint64_t, const KeeperStorageBase::RequestForSession &)>;
|
||||
|
||||
KeeperStateMachine(
|
||||
IKeeperStateMachine(
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
const KeeperContextPtr & keeper_context_,
|
||||
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
|
||||
CommitCallback commit_callback_ = {},
|
||||
const std::string & superdigest_ = "");
|
||||
CommitCallback commit_callback_,
|
||||
const std::string & superdigest_);
|
||||
|
||||
/// Read state from the latest snapshot
|
||||
void init();
|
||||
virtual void init() = 0;
|
||||
|
||||
enum ZooKeeperLogSerializationVersion
|
||||
{
|
||||
@ -47,89 +45,66 @@ public:
|
||||
///
|
||||
/// final - whether it's the final time we will fetch the request so we can safely remove it from cache
|
||||
/// serialization_version - information about which fields were parsed from the buffer so we can modify the buffer accordingly
|
||||
std::shared_ptr<KeeperStorage::RequestForSession> parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr);
|
||||
std::shared_ptr<KeeperStorageBase::RequestForSession> parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr);
|
||||
|
||||
bool preprocess(const KeeperStorage::RequestForSession & request_for_session);
|
||||
virtual bool preprocess(const KeeperStorageBase::RequestForSession & request_for_session) = 0;
|
||||
|
||||
nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override;
|
||||
|
||||
nuraft::ptr<nuraft::buffer> commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT
|
||||
|
||||
/// Save new cluster config to our snapshot (copy of the config stored in StateManager)
|
||||
void commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf) override; /// NOLINT
|
||||
|
||||
void rollback(uint64_t log_idx, nuraft::buffer & data) override;
|
||||
|
||||
// allow_missing - whether the transaction we want to rollback can be missing from storage
|
||||
// (can happen in case of exception during preprocessing)
|
||||
void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing);
|
||||
|
||||
void rollbackRequestNoLock(
|
||||
const KeeperStorage::RequestForSession & request_for_session,
|
||||
bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS;
|
||||
virtual void rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) = 0;
|
||||
|
||||
uint64_t last_commit_index() override { return keeper_context->lastCommittedIndex(); }
|
||||
|
||||
/// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state.
|
||||
bool apply_snapshot(nuraft::snapshot & s) override;
|
||||
|
||||
nuraft::ptr<nuraft::snapshot> last_snapshot() override;
|
||||
|
||||
/// Create new snapshot from current state.
|
||||
void create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done) override;
|
||||
void create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done) override = 0;
|
||||
|
||||
/// Save snapshot which was send by leader to us. After that we will apply it in apply_snapshot.
|
||||
void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override;
|
||||
void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override = 0;
|
||||
|
||||
/// Better name is `serialize snapshot` -- save existing snapshot (created by create_snapshot) into
|
||||
/// in-memory buffer data_out.
|
||||
int read_logical_snp_obj(
|
||||
nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr<nuraft::buffer> & data_out, bool & is_last_obj) override;
|
||||
|
||||
// This should be used only for tests or keeper-data-dumper because it violates
|
||||
// TSA -- we can't acquire the lock outside of this class or return a storage under lock
|
||||
// in a reasonable way.
|
||||
KeeperStorage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS
|
||||
{
|
||||
return *storage;
|
||||
}
|
||||
|
||||
void shutdownStorage();
|
||||
virtual void shutdownStorage() = 0;
|
||||
|
||||
ClusterConfigPtr getClusterConfig() const;
|
||||
|
||||
/// Process local read request
|
||||
void processReadRequest(const KeeperStorage::RequestForSession & request_for_session);
|
||||
virtual void processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) = 0;
|
||||
|
||||
std::vector<int64_t> getDeadSessions();
|
||||
virtual std::vector<int64_t> getDeadSessions() = 0;
|
||||
|
||||
int64_t getNextZxid() const;
|
||||
virtual int64_t getNextZxid() const = 0;
|
||||
|
||||
KeeperStorage::Digest getNodesDigest() const;
|
||||
virtual KeeperStorageBase::Digest getNodesDigest() const = 0;
|
||||
|
||||
/// Introspection functions for 4lw commands
|
||||
uint64_t getLastProcessedZxid() const;
|
||||
virtual uint64_t getLastProcessedZxid() const = 0;
|
||||
|
||||
uint64_t getNodesCount() const;
|
||||
uint64_t getTotalWatchesCount() const;
|
||||
uint64_t getWatchedPathsCount() const;
|
||||
uint64_t getSessionsWithWatchesCount() const;
|
||||
virtual uint64_t getNodesCount() const = 0;
|
||||
virtual uint64_t getTotalWatchesCount() const = 0;
|
||||
virtual uint64_t getWatchedPathsCount() const = 0;
|
||||
virtual uint64_t getSessionsWithWatchesCount() const = 0;
|
||||
|
||||
void dumpWatches(WriteBufferFromOwnString & buf) const;
|
||||
void dumpWatchesByPath(WriteBufferFromOwnString & buf) const;
|
||||
void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const;
|
||||
virtual void dumpWatches(WriteBufferFromOwnString & buf) const = 0;
|
||||
virtual void dumpWatchesByPath(WriteBufferFromOwnString & buf) const = 0;
|
||||
virtual void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const = 0;
|
||||
|
||||
uint64_t getSessionWithEphemeralNodesCount() const;
|
||||
uint64_t getTotalEphemeralNodesCount() const;
|
||||
uint64_t getApproximateDataSize() const;
|
||||
uint64_t getKeyArenaSize() const;
|
||||
uint64_t getLatestSnapshotSize() const;
|
||||
virtual uint64_t getSessionWithEphemeralNodesCount() const = 0;
|
||||
virtual uint64_t getTotalEphemeralNodesCount() const = 0;
|
||||
virtual uint64_t getApproximateDataSize() const = 0;
|
||||
virtual uint64_t getKeyArenaSize() const = 0;
|
||||
virtual uint64_t getLatestSnapshotSize() const = 0;
|
||||
|
||||
void recalculateStorageStats();
|
||||
virtual void recalculateStorageStats() = 0;
|
||||
|
||||
void reconfigure(const KeeperStorage::RequestForSession& request_for_session);
|
||||
virtual void reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) = 0;
|
||||
|
||||
private:
|
||||
protected:
|
||||
CommitCallback commit_callback;
|
||||
/// In our state machine we always have a single snapshot which is stored
|
||||
/// in memory in compressed (serialized) format.
|
||||
@ -137,12 +112,9 @@ private:
|
||||
std::shared_ptr<SnapshotFileInfo> latest_snapshot_info;
|
||||
nuraft::ptr<nuraft::buffer> latest_snapshot_buf = nullptr;
|
||||
|
||||
/// Main state machine logic
|
||||
KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock);
|
||||
CoordinationSettingsPtr coordination_settings;
|
||||
|
||||
/// Save/Load and Serialize/Deserialize logic for snapshots.
|
||||
KeeperSnapshotManager snapshot_manager;
|
||||
|
||||
/// Put processed responses into this queue
|
||||
ResponsesQueue & responses_queue;
|
||||
|
||||
@ -159,7 +131,7 @@ private:
|
||||
/// for request.
|
||||
mutable std::mutex storage_and_responses_lock;
|
||||
|
||||
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, std::shared_ptr<KeeperStorage::RequestForSession>>> parsed_request_cache;
|
||||
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, std::shared_ptr<KeeperStorageBase::RequestForSession>>> parsed_request_cache;
|
||||
uint64_t min_request_size_to_cache{0};
|
||||
/// we only need to protect the access to the map itself
|
||||
/// requests can be modified from anywhere without lock because a single request
|
||||
@ -181,7 +153,104 @@ private:
|
||||
|
||||
KeeperSnapshotManagerS3 * snapshot_manager_s3;
|
||||
|
||||
KeeperStorage::ResponseForSession processReconfiguration(const KeeperStorage::RequestForSession & request_for_session)
|
||||
TSA_REQUIRES(storage_and_responses_lock);
|
||||
virtual KeeperStorageBase::ResponseForSession processReconfiguration(
|
||||
const KeeperStorageBase::RequestForSession& request_for_session)
|
||||
TSA_REQUIRES(storage_and_responses_lock) = 0;
|
||||
|
||||
};
|
||||
|
||||
/// ClickHouse Keeper state machine. Wrapper for KeeperStorage.
|
||||
/// Responsible for entries commit, snapshots creation and so on.
|
||||
template<typename Storage>
|
||||
class KeeperStateMachine : public IKeeperStateMachine
|
||||
{
|
||||
public:
|
||||
/// using CommitCallback = std::function<void(uint64_t, const KeeperStorage::RequestForSession &)>;
|
||||
|
||||
KeeperStateMachine(
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
/// const CoordinationSettingsPtr & coordination_settings_,
|
||||
const KeeperContextPtr & keeper_context_,
|
||||
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
|
||||
CommitCallback commit_callback_ = {},
|
||||
const std::string & superdigest_ = "");
|
||||
|
||||
/// Read state from the latest snapshot
|
||||
void init() override;
|
||||
|
||||
bool preprocess(const KeeperStorageBase::RequestForSession & request_for_session) override;
|
||||
|
||||
nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override;
|
||||
|
||||
nuraft::ptr<nuraft::buffer> commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT
|
||||
|
||||
// allow_missing - whether the transaction we want to rollback can be missing from storage
|
||||
// (can happen in case of exception during preprocessing)
|
||||
void rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) override;
|
||||
|
||||
void rollbackRequestNoLock(
|
||||
const KeeperStorageBase::RequestForSession & request_for_session,
|
||||
bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS;
|
||||
|
||||
/// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state.
|
||||
bool apply_snapshot(nuraft::snapshot & s) override;
|
||||
|
||||
/// Create new snapshot from current state.
|
||||
void create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done) override;
|
||||
|
||||
/// Save snapshot which was send by leader to us. After that we will apply it in apply_snapshot.
|
||||
void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override;
|
||||
|
||||
// This should be used only for tests or keeper-data-dumper because it violates
|
||||
// TSA -- we can't acquire the lock outside of this class or return a storage under lock
|
||||
// in a reasonable way.
|
||||
Storage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS
|
||||
{
|
||||
return *storage;
|
||||
}
|
||||
|
||||
void shutdownStorage() override;
|
||||
|
||||
/// Process local read request
|
||||
void processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) override;
|
||||
|
||||
std::vector<int64_t> getDeadSessions() override;
|
||||
|
||||
int64_t getNextZxid() const override;
|
||||
|
||||
KeeperStorageBase::Digest getNodesDigest() const override;
|
||||
|
||||
/// Introspection functions for 4lw commands
|
||||
uint64_t getLastProcessedZxid() const override;
|
||||
|
||||
uint64_t getNodesCount() const override;
|
||||
uint64_t getTotalWatchesCount() const override;
|
||||
uint64_t getWatchedPathsCount() const override;
|
||||
uint64_t getSessionsWithWatchesCount() const override;
|
||||
|
||||
void dumpWatches(WriteBufferFromOwnString & buf) const override;
|
||||
void dumpWatchesByPath(WriteBufferFromOwnString & buf) const override;
|
||||
void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const override;
|
||||
|
||||
uint64_t getSessionWithEphemeralNodesCount() const override;
|
||||
uint64_t getTotalEphemeralNodesCount() const override;
|
||||
uint64_t getApproximateDataSize() const override;
|
||||
uint64_t getKeyArenaSize() const override;
|
||||
uint64_t getLatestSnapshotSize() const override;
|
||||
|
||||
void recalculateStorageStats() override;
|
||||
|
||||
void reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) override;
|
||||
|
||||
private:
|
||||
/// Main state machine logic
|
||||
std::unique_ptr<Storage> storage; //TSA_PT_GUARDED_BY(storage_and_responses_lock);
|
||||
|
||||
/// Save/Load and Serialize/Deserialize logic for snapshots.
|
||||
KeeperSnapshotManager<Storage> snapshot_manager;
|
||||
|
||||
KeeperStorageBase::ResponseForSession processReconfiguration(const KeeperStorageBase::RequestForSession & request_for_session)
|
||||
TSA_REQUIRES(storage_and_responses_lock) override;
|
||||
};
|
||||
}
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -8,188 +8,384 @@
|
||||
|
||||
#include <absl/container/flat_hash_set.h>
|
||||
|
||||
#include "config.h"
|
||||
#if USE_ROCKSDB
|
||||
#include <Coordination/RocksDBContainer.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class KeeperContext;
|
||||
using KeeperContextPtr = std::shared_ptr<KeeperContext>;
|
||||
|
||||
struct KeeperStorageRequestProcessor;
|
||||
using KeeperStorageRequestProcessorPtr = std::shared_ptr<KeeperStorageRequestProcessor>;
|
||||
using ResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr &)>;
|
||||
using ChildrenSet = absl::flat_hash_set<StringRef, StringRefHash>;
|
||||
using SessionAndTimeout = std::unordered_map<int64_t, int64_t>;
|
||||
|
||||
struct KeeperStorageSnapshot;
|
||||
|
||||
/// Keeper state machine almost equal to the ZooKeeper's state machine.
|
||||
/// Implements all logic of operations, data changes, sessions allocation.
|
||||
/// In-memory and not thread safe.
|
||||
class KeeperStorage
|
||||
/// KeeperRocksNodeInfo is used in RocksDB keeper.
|
||||
/// It is serialized directly as POD to RocksDB.
|
||||
struct KeeperRocksNodeInfo
|
||||
{
|
||||
public:
|
||||
/// Node should have as minimal size as possible to reduce memory footprint
|
||||
/// of stored nodes
|
||||
/// New fields should be added to the struct only if it's really necessary
|
||||
struct Node
|
||||
int64_t czxid{0};
|
||||
int64_t mzxid{0};
|
||||
int64_t pzxid{0};
|
||||
uint64_t acl_id = 0; /// 0 -- no ACL by default
|
||||
|
||||
int64_t mtime{0};
|
||||
|
||||
int32_t version{0};
|
||||
int32_t cversion{0};
|
||||
int32_t aversion{0};
|
||||
|
||||
int32_t seq_num = 0;
|
||||
mutable UInt64 digest = 0; /// we cached digest for this node.
|
||||
|
||||
/// as ctime can't be negative because it stores the timestamp when the
|
||||
/// node was created, we can use the MSB for a bool
|
||||
struct
|
||||
{
|
||||
int64_t czxid{0};
|
||||
int64_t mzxid{0};
|
||||
int64_t pzxid{0};
|
||||
uint64_t acl_id = 0; /// 0 -- no ACL by default
|
||||
bool is_ephemeral : 1;
|
||||
int64_t ctime : 63;
|
||||
} is_ephemeral_and_ctime{false, 0};
|
||||
|
||||
int64_t mtime{0};
|
||||
|
||||
std::unique_ptr<char[]> data{nullptr};
|
||||
uint32_t data_size{0};
|
||||
|
||||
int32_t version{0};
|
||||
int32_t cversion{0};
|
||||
int32_t aversion{0};
|
||||
|
||||
mutable uint64_t cached_digest = 0;
|
||||
|
||||
Node() = default;
|
||||
|
||||
Node & operator=(const Node & other);
|
||||
Node(const Node & other);
|
||||
|
||||
Node & operator=(Node && other) noexcept;
|
||||
Node(Node && other) noexcept;
|
||||
|
||||
bool empty() const;
|
||||
|
||||
bool isEphemeral() const
|
||||
{
|
||||
return is_ephemeral_and_ctime.is_ephemeral;
|
||||
}
|
||||
|
||||
int64_t ephemeralOwner() const
|
||||
{
|
||||
if (isEphemeral())
|
||||
return ephemeral_or_children_data.ephemeral_owner;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
void setEphemeralOwner(int64_t ephemeral_owner)
|
||||
{
|
||||
is_ephemeral_and_ctime.is_ephemeral = ephemeral_owner != 0;
|
||||
ephemeral_or_children_data.ephemeral_owner = ephemeral_owner;
|
||||
}
|
||||
|
||||
int32_t numChildren() const
|
||||
{
|
||||
if (isEphemeral())
|
||||
return 0;
|
||||
|
||||
return ephemeral_or_children_data.children_info.num_children;
|
||||
}
|
||||
|
||||
void setNumChildren(int32_t num_children)
|
||||
{
|
||||
ephemeral_or_children_data.children_info.num_children = num_children;
|
||||
}
|
||||
|
||||
void increaseNumChildren()
|
||||
{
|
||||
chassert(!isEphemeral());
|
||||
++ephemeral_or_children_data.children_info.num_children;
|
||||
}
|
||||
|
||||
void decreaseNumChildren()
|
||||
{
|
||||
chassert(!isEphemeral());
|
||||
--ephemeral_or_children_data.children_info.num_children;
|
||||
}
|
||||
|
||||
int32_t seqNum() const
|
||||
{
|
||||
if (isEphemeral())
|
||||
return 0;
|
||||
|
||||
return ephemeral_or_children_data.children_info.seq_num;
|
||||
}
|
||||
|
||||
void setSeqNum(int32_t seq_num)
|
||||
{
|
||||
ephemeral_or_children_data.children_info.seq_num = seq_num;
|
||||
}
|
||||
|
||||
void increaseSeqNum()
|
||||
{
|
||||
chassert(!isEphemeral());
|
||||
++ephemeral_or_children_data.children_info.seq_num;
|
||||
}
|
||||
|
||||
int64_t ctime() const
|
||||
{
|
||||
return is_ephemeral_and_ctime.ctime;
|
||||
}
|
||||
|
||||
void setCtime(uint64_t ctime)
|
||||
{
|
||||
is_ephemeral_and_ctime.ctime = ctime;
|
||||
}
|
||||
|
||||
void copyStats(const Coordination::Stat & stat);
|
||||
|
||||
void setResponseStat(Coordination::Stat & response_stat) const;
|
||||
|
||||
/// Object memory size
|
||||
uint64_t sizeInBytes() const;
|
||||
|
||||
void setData(const String & new_data);
|
||||
|
||||
std::string_view getData() const noexcept { return {data.get(), data_size}; }
|
||||
|
||||
void addChild(StringRef child_path);
|
||||
|
||||
void removeChild(StringRef child_path);
|
||||
|
||||
const auto & getChildren() const noexcept { return children; }
|
||||
auto & getChildren() { return children; }
|
||||
|
||||
// Invalidate the calculated digest so it's recalculated again on the next
|
||||
// getDigest call
|
||||
void invalidateDigestCache() const;
|
||||
|
||||
// get the calculated digest of the node
|
||||
UInt64 getDigest(std::string_view path) const;
|
||||
|
||||
// copy only necessary information for preprocessing and digest calculation
|
||||
// (e.g. we don't need to copy list of children)
|
||||
void shallowCopy(const Node & other);
|
||||
private:
|
||||
/// as ctime can't be negative because it stores the timestamp when the
|
||||
/// node was created, we can use the MSB for a bool
|
||||
/// ephemeral notes cannot have children so a node can set either
|
||||
/// ephemeral_owner OR seq_num + num_children
|
||||
union
|
||||
{
|
||||
int64_t ephemeral_owner;
|
||||
struct
|
||||
{
|
||||
bool is_ephemeral : 1;
|
||||
int64_t ctime : 63;
|
||||
} is_ephemeral_and_ctime{false, 0};
|
||||
int32_t seq_num;
|
||||
int32_t num_children;
|
||||
} children_info;
|
||||
} ephemeral_or_children_data{0};
|
||||
|
||||
/// ephemeral notes cannot have children so a node can set either
|
||||
/// ephemeral_owner OR seq_num + num_children
|
||||
union
|
||||
{
|
||||
int64_t ephemeral_owner;
|
||||
struct
|
||||
{
|
||||
int32_t seq_num;
|
||||
int32_t num_children;
|
||||
} children_info;
|
||||
} ephemeral_or_children_data{0};
|
||||
bool isEphemeral() const
|
||||
{
|
||||
return is_ephemeral_and_ctime.is_ephemeral;
|
||||
}
|
||||
|
||||
ChildrenSet children{};
|
||||
};
|
||||
int64_t ephemeralOwner() const
|
||||
{
|
||||
if (isEphemeral())
|
||||
return ephemeral_or_children_data.ephemeral_owner;
|
||||
|
||||
#if !defined(ADDRESS_SANITIZER) && !defined(MEMORY_SANITIZER)
|
||||
static_assert(
|
||||
sizeof(ListNode<Node>) <= 144,
|
||||
"std::list node containing ListNode<Node> is > 160 bytes (sizeof(ListNode<Node>) + 16 bytes for pointers) which will increase "
|
||||
"memory consumption");
|
||||
return 0;
|
||||
}
|
||||
|
||||
void setEphemeralOwner(int64_t ephemeral_owner)
|
||||
{
|
||||
is_ephemeral_and_ctime.is_ephemeral = ephemeral_owner != 0;
|
||||
ephemeral_or_children_data.ephemeral_owner = ephemeral_owner;
|
||||
}
|
||||
|
||||
int32_t numChildren() const
|
||||
{
|
||||
if (isEphemeral())
|
||||
return 0;
|
||||
|
||||
return ephemeral_or_children_data.children_info.num_children;
|
||||
}
|
||||
|
||||
void setNumChildren(int32_t num_children)
|
||||
{
|
||||
ephemeral_or_children_data.children_info.num_children = num_children;
|
||||
}
|
||||
|
||||
/// dummy interface for test
|
||||
void addChild(StringRef) {}
|
||||
auto getChildren() const
|
||||
{
|
||||
return std::vector<int>(numChildren());
|
||||
}
|
||||
|
||||
void increaseNumChildren()
|
||||
{
|
||||
chassert(!isEphemeral());
|
||||
++ephemeral_or_children_data.children_info.num_children;
|
||||
}
|
||||
|
||||
void decreaseNumChildren()
|
||||
{
|
||||
chassert(!isEphemeral());
|
||||
--ephemeral_or_children_data.children_info.num_children;
|
||||
}
|
||||
|
||||
int32_t seqNum() const
|
||||
{
|
||||
if (isEphemeral())
|
||||
return 0;
|
||||
|
||||
return ephemeral_or_children_data.children_info.seq_num;
|
||||
}
|
||||
|
||||
void setSeqNum(int32_t seq_num_)
|
||||
{
|
||||
ephemeral_or_children_data.children_info.seq_num = seq_num_;
|
||||
}
|
||||
|
||||
void increaseSeqNum()
|
||||
{
|
||||
chassert(!isEphemeral());
|
||||
++ephemeral_or_children_data.children_info.seq_num;
|
||||
}
|
||||
|
||||
int64_t ctime() const
|
||||
{
|
||||
return is_ephemeral_and_ctime.ctime;
|
||||
}
|
||||
|
||||
void setCtime(uint64_t ctime)
|
||||
{
|
||||
is_ephemeral_and_ctime.ctime = ctime;
|
||||
}
|
||||
|
||||
void copyStats(const Coordination::Stat & stat);
|
||||
};
|
||||
|
||||
/// KeeperRocksNode is the memory structure used by RocksDB
|
||||
struct KeeperRocksNode : public KeeperRocksNodeInfo
|
||||
{
|
||||
#if USE_ROCKSDB
|
||||
friend struct RocksDBContainer<KeeperRocksNode>;
|
||||
#endif
|
||||
using Meta = KeeperRocksNodeInfo;
|
||||
|
||||
uint64_t size_bytes = 0; // only for compatible, should be deprecated
|
||||
|
||||
uint64_t sizeInBytes() const { return data_size + sizeof(KeeperRocksNodeInfo); }
|
||||
void setData(String new_data)
|
||||
{
|
||||
data_size = static_cast<uint32_t>(new_data.size());
|
||||
if (data_size != 0)
|
||||
{
|
||||
data = std::unique_ptr<char[]>(new char[new_data.size()]);
|
||||
memcpy(data.get(), new_data.data(), data_size);
|
||||
}
|
||||
}
|
||||
|
||||
void shallowCopy(const KeeperRocksNode & other)
|
||||
{
|
||||
czxid = other.czxid;
|
||||
mzxid = other.mzxid;
|
||||
pzxid = other.pzxid;
|
||||
acl_id = other.acl_id; /// 0 -- no ACL by default
|
||||
|
||||
mtime = other.mtime;
|
||||
|
||||
is_ephemeral_and_ctime = other.is_ephemeral_and_ctime;
|
||||
|
||||
ephemeral_or_children_data = other.ephemeral_or_children_data;
|
||||
|
||||
data_size = other.data_size;
|
||||
if (data_size != 0)
|
||||
{
|
||||
data = std::unique_ptr<char[]>(new char[data_size]);
|
||||
memcpy(data.get(), other.data.get(), data_size);
|
||||
}
|
||||
|
||||
version = other.version;
|
||||
cversion = other.cversion;
|
||||
aversion = other.aversion;
|
||||
|
||||
/// cached_digest = other.cached_digest;
|
||||
}
|
||||
void invalidateDigestCache() const;
|
||||
UInt64 getDigest(std::string_view path) const;
|
||||
String getEncodedString();
|
||||
void decodeFromString(const String & buffer_str);
|
||||
void recalculateSize() {}
|
||||
std::string_view getData() const noexcept { return {data.get(), data_size}; }
|
||||
|
||||
void setResponseStat(Coordination::Stat & response_stat) const
|
||||
{
|
||||
response_stat.czxid = czxid;
|
||||
response_stat.mzxid = mzxid;
|
||||
response_stat.ctime = ctime();
|
||||
response_stat.mtime = mtime;
|
||||
response_stat.version = version;
|
||||
response_stat.cversion = cversion;
|
||||
response_stat.aversion = aversion;
|
||||
response_stat.ephemeralOwner = ephemeralOwner();
|
||||
response_stat.dataLength = static_cast<int32_t>(data_size);
|
||||
response_stat.numChildren = numChildren();
|
||||
response_stat.pzxid = pzxid;
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
serialized = false;
|
||||
}
|
||||
bool empty() const
|
||||
{
|
||||
return data_size == 0 && mzxid == 0;
|
||||
}
|
||||
std::unique_ptr<char[]> data{nullptr};
|
||||
uint32_t data_size{0};
|
||||
private:
|
||||
bool serialized = false;
|
||||
};
|
||||
|
||||
/// KeeperMemNode should have as minimal size as possible to reduce memory footprint
|
||||
/// of stored nodes
|
||||
/// New fields should be added to the struct only if it's really necessary
|
||||
struct KeeperMemNode
|
||||
{
|
||||
int64_t czxid{0};
|
||||
int64_t mzxid{0};
|
||||
int64_t pzxid{0};
|
||||
uint64_t acl_id = 0; /// 0 -- no ACL by default
|
||||
|
||||
int64_t mtime{0};
|
||||
|
||||
std::unique_ptr<char[]> data{nullptr};
|
||||
uint32_t data_size{0};
|
||||
|
||||
int32_t version{0};
|
||||
int32_t cversion{0};
|
||||
int32_t aversion{0};
|
||||
|
||||
mutable uint64_t cached_digest = 0;
|
||||
|
||||
KeeperMemNode() = default;
|
||||
|
||||
KeeperMemNode & operator=(const KeeperMemNode & other);
|
||||
KeeperMemNode(const KeeperMemNode & other);
|
||||
|
||||
KeeperMemNode & operator=(KeeperMemNode && other) noexcept;
|
||||
KeeperMemNode(KeeperMemNode && other) noexcept;
|
||||
|
||||
bool empty() const;
|
||||
|
||||
bool isEphemeral() const
|
||||
{
|
||||
return is_ephemeral_and_ctime.is_ephemeral;
|
||||
}
|
||||
|
||||
int64_t ephemeralOwner() const
|
||||
{
|
||||
if (isEphemeral())
|
||||
return ephemeral_or_children_data.ephemeral_owner;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
void setEphemeralOwner(int64_t ephemeral_owner)
|
||||
{
|
||||
is_ephemeral_and_ctime.is_ephemeral = ephemeral_owner != 0;
|
||||
ephemeral_or_children_data.ephemeral_owner = ephemeral_owner;
|
||||
}
|
||||
|
||||
int32_t numChildren() const
|
||||
{
|
||||
if (isEphemeral())
|
||||
return 0;
|
||||
|
||||
return ephemeral_or_children_data.children_info.num_children;
|
||||
}
|
||||
|
||||
void setNumChildren(int32_t num_children)
|
||||
{
|
||||
ephemeral_or_children_data.children_info.num_children = num_children;
|
||||
}
|
||||
|
||||
void increaseNumChildren()
|
||||
{
|
||||
chassert(!isEphemeral());
|
||||
++ephemeral_or_children_data.children_info.num_children;
|
||||
}
|
||||
|
||||
void decreaseNumChildren()
|
||||
{
|
||||
chassert(!isEphemeral());
|
||||
--ephemeral_or_children_data.children_info.num_children;
|
||||
}
|
||||
|
||||
int32_t seqNum() const
|
||||
{
|
||||
if (isEphemeral())
|
||||
return 0;
|
||||
|
||||
return ephemeral_or_children_data.children_info.seq_num;
|
||||
}
|
||||
|
||||
void setSeqNum(int32_t seq_num)
|
||||
{
|
||||
ephemeral_or_children_data.children_info.seq_num = seq_num;
|
||||
}
|
||||
|
||||
void increaseSeqNum()
|
||||
{
|
||||
chassert(!isEphemeral());
|
||||
++ephemeral_or_children_data.children_info.seq_num;
|
||||
}
|
||||
|
||||
int64_t ctime() const
|
||||
{
|
||||
return is_ephemeral_and_ctime.ctime;
|
||||
}
|
||||
|
||||
void setCtime(uint64_t ctime)
|
||||
{
|
||||
is_ephemeral_and_ctime.ctime = ctime;
|
||||
}
|
||||
|
||||
void copyStats(const Coordination::Stat & stat);
|
||||
|
||||
void setResponseStat(Coordination::Stat & response_stat) const;
|
||||
|
||||
/// Object memory size
|
||||
uint64_t sizeInBytes() const;
|
||||
|
||||
void setData(const String & new_data);
|
||||
|
||||
std::string_view getData() const noexcept { return {data.get(), data_size}; }
|
||||
|
||||
void addChild(StringRef child_path);
|
||||
|
||||
void removeChild(StringRef child_path);
|
||||
|
||||
const auto & getChildren() const noexcept { return children; }
|
||||
auto & getChildren() { return children; }
|
||||
|
||||
// Invalidate the calculated digest so it's recalculated again on the next
|
||||
// getDigest call
|
||||
void invalidateDigestCache() const;
|
||||
|
||||
// get the calculated digest of the node
|
||||
UInt64 getDigest(std::string_view path) const;
|
||||
|
||||
// copy only necessary information for preprocessing and digest calculation
|
||||
// (e.g. we don't need to copy list of children)
|
||||
void shallowCopy(const KeeperMemNode & other);
|
||||
private:
|
||||
/// as ctime can't be negative because it stores the timestamp when the
|
||||
/// node was created, we can use the MSB for a bool
|
||||
struct
|
||||
{
|
||||
bool is_ephemeral : 1;
|
||||
int64_t ctime : 63;
|
||||
} is_ephemeral_and_ctime{false, 0};
|
||||
|
||||
/// ephemeral notes cannot have children so a node can set either
|
||||
/// ephemeral_owner OR seq_num + num_children
|
||||
union
|
||||
{
|
||||
int64_t ephemeral_owner;
|
||||
struct
|
||||
{
|
||||
int32_t seq_num;
|
||||
int32_t num_children;
|
||||
} children_info;
|
||||
} ephemeral_or_children_data{0};
|
||||
|
||||
ChildrenSet children{};
|
||||
};
|
||||
|
||||
class KeeperStorageBase
|
||||
{
|
||||
public:
|
||||
|
||||
enum DigestVersion : uint8_t
|
||||
{
|
||||
@ -200,7 +396,11 @@ public:
|
||||
V4 = 4 // 0 is not a valid digest value
|
||||
};
|
||||
|
||||
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V4;
|
||||
struct Digest
|
||||
{
|
||||
DigestVersion version{DigestVersion::NO_DIGEST};
|
||||
uint64_t value{0};
|
||||
};
|
||||
|
||||
struct ResponseForSession
|
||||
{
|
||||
@ -210,16 +410,6 @@ public:
|
||||
};
|
||||
using ResponsesForSessions = std::vector<ResponseForSession>;
|
||||
|
||||
struct Digest
|
||||
{
|
||||
DigestVersion version{DigestVersion::NO_DIGEST};
|
||||
uint64_t value{0};
|
||||
};
|
||||
|
||||
static bool checkDigest(const Digest & first, const Digest & second);
|
||||
|
||||
static String generateDigest(const String & userdata);
|
||||
|
||||
struct RequestForSession
|
||||
{
|
||||
int64_t session_id;
|
||||
@ -229,6 +419,7 @@ public:
|
||||
std::optional<Digest> digest;
|
||||
int64_t log_idx{0};
|
||||
};
|
||||
using RequestsForSessions = std::vector<RequestForSession>;
|
||||
|
||||
struct AuthID
|
||||
{
|
||||
@ -238,9 +429,6 @@ public:
|
||||
bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; }
|
||||
};
|
||||
|
||||
using RequestsForSessions = std::vector<RequestForSession>;
|
||||
|
||||
using Container = SnapshotableHashTable<Node>;
|
||||
using Ephemerals = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
||||
using SessionAndWatcher = std::unordered_map<int64_t, std::unordered_set<std::string>>;
|
||||
using SessionIDs = std::unordered_set<int64_t>;
|
||||
@ -250,6 +438,38 @@ public:
|
||||
using SessionAndAuth = std::unordered_map<int64_t, AuthIDs>;
|
||||
using Watches = std::unordered_map<String /* path, relative of root_path */, SessionIDs>;
|
||||
|
||||
static bool checkDigest(const Digest & first, const Digest & second);
|
||||
|
||||
};
|
||||
|
||||
/// Keeper state machine almost equal to the ZooKeeper's state machine.
|
||||
/// Implements all logic of operations, data changes, sessions allocation.
|
||||
/// In-memory and not thread safe.
|
||||
template<typename Container_>
|
||||
class KeeperStorage : public KeeperStorageBase
|
||||
{
|
||||
public:
|
||||
using Container = Container_;
|
||||
using Node = Container::Node;
|
||||
|
||||
#if !defined(ADDRESS_SANITIZER) && !defined(MEMORY_SANITIZER)
|
||||
static_assert(
|
||||
sizeof(ListNode<Node>) <= 144,
|
||||
"std::list node containing ListNode<Node> is > 160 bytes (sizeof(ListNode<Node>) + 16 bytes for pointers) which will increase "
|
||||
"memory consumption");
|
||||
#endif
|
||||
|
||||
|
||||
#if USE_ROCKSDB
|
||||
static constexpr bool use_rocksdb = std::is_same_v<Container_, RocksDBContainer<KeeperRocksNode>>;
|
||||
#else
|
||||
static constexpr bool use_rocksdb = false;
|
||||
#endif
|
||||
|
||||
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V4;
|
||||
|
||||
static String generateDigest(const String & userdata);
|
||||
|
||||
int64_t session_id_counter{1};
|
||||
|
||||
SessionAndAuth session_and_auth;
|
||||
@ -393,7 +613,7 @@ public:
|
||||
std::unordered_map<std::string, std::list<const Delta *>, Hash, Equal> deltas_for_path;
|
||||
|
||||
std::list<Delta> deltas;
|
||||
KeeperStorage & storage;
|
||||
KeeperStorage<Container> & storage;
|
||||
};
|
||||
|
||||
UncommittedState uncommitted_state{*this};
|
||||
@ -530,10 +750,16 @@ public:
|
||||
/// Set of methods for creating snapshots
|
||||
|
||||
/// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version.
|
||||
void enableSnapshotMode(size_t up_to_version) { container.enableSnapshotMode(up_to_version); }
|
||||
void enableSnapshotMode(size_t up_to_version)
|
||||
{
|
||||
container.enableSnapshotMode(up_to_version);
|
||||
}
|
||||
|
||||
/// Turn off snapshot mode.
|
||||
void disableSnapshotMode() { container.disableSnapshotMode(); }
|
||||
void disableSnapshotMode()
|
||||
{
|
||||
container.disableSnapshotMode();
|
||||
}
|
||||
|
||||
Container::const_iterator getSnapshotIteratorBegin() const { return container.begin(); }
|
||||
|
||||
@ -572,6 +798,9 @@ private:
|
||||
void addDigest(const Node & node, std::string_view path);
|
||||
};
|
||||
|
||||
using KeeperStoragePtr = std::unique_ptr<KeeperStorage>;
|
||||
using KeeperMemoryStorage = KeeperStorage<SnapshotableHashTable<KeeperMemNode>>;
|
||||
#if USE_ROCKSDB
|
||||
using KeeperRocksStorage = KeeperStorage<RocksDBContainer<KeeperRocksNode>>;
|
||||
#endif
|
||||
|
||||
}
|
||||
|
460
src/Coordination/RocksDBContainer.h
Normal file
460
src/Coordination/RocksDBContainer.h
Normal file
@ -0,0 +1,460 @@
|
||||
#pragma once
|
||||
#include <base/StringRef.h>
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/KeeperContext.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
|
||||
#include <rocksdb/convenience.h>
|
||||
#include <rocksdb/options.h>
|
||||
#include <rocksdb/status.h>
|
||||
#include <rocksdb/table.h>
|
||||
#include <rocksdb/snapshot.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ROCKSDB_ERROR;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/// The key-value format of rocks db will be
|
||||
/// - key: Int8 (depth of the path) + String (path)
|
||||
/// - value: SizeOf(keeperRocksNodeInfo) (meta of the node) + String (data)
|
||||
|
||||
template <class Node_>
|
||||
struct RocksDBContainer
|
||||
{
|
||||
using Node = Node_;
|
||||
|
||||
private:
|
||||
/// MockNode is only use in test to mock `getChildren()` and `getData()`
|
||||
struct MockNode
|
||||
{
|
||||
std::vector<int> children;
|
||||
std::string data;
|
||||
MockNode(size_t children_num, std::string_view data_)
|
||||
: children(std::vector<int>(children_num)),
|
||||
data(data_)
|
||||
{
|
||||
}
|
||||
|
||||
std::vector<int> getChildren() { return children; }
|
||||
std::string getData() { return data; }
|
||||
};
|
||||
|
||||
UInt16 getKeyDepth(const std::string & key)
|
||||
{
|
||||
UInt16 depth = 0;
|
||||
for (size_t i = 0; i < key.size(); i++)
|
||||
{
|
||||
if (key[i] == '/' && i + 1 != key.size())
|
||||
depth ++;
|
||||
}
|
||||
return depth;
|
||||
}
|
||||
|
||||
std::string getEncodedKey(const std::string & key, bool child_prefix = false)
|
||||
{
|
||||
WriteBufferFromOwnString key_buffer;
|
||||
UInt16 depth = getKeyDepth(key) + (child_prefix ? 1 : 0);
|
||||
writeIntBinary(depth, key_buffer);
|
||||
writeString(key, key_buffer);
|
||||
return key_buffer.str();
|
||||
}
|
||||
|
||||
static std::string_view getDecodedKey(const std::string_view & key)
|
||||
{
|
||||
return std::string_view(key.begin() + 2, key.end());
|
||||
}
|
||||
|
||||
|
||||
struct KVPair
|
||||
{
|
||||
StringRef key;
|
||||
Node value;
|
||||
};
|
||||
|
||||
using ValueUpdater = std::function<void(Node & node)>;
|
||||
|
||||
public:
|
||||
|
||||
/// This is an iterator wrapping rocksdb iterator and the kv result.
|
||||
struct const_iterator
|
||||
{
|
||||
std::shared_ptr<rocksdb::Iterator> iter;
|
||||
|
||||
std::shared_ptr<const KVPair> pair;
|
||||
|
||||
const_iterator() = default;
|
||||
|
||||
explicit const_iterator(std::shared_ptr<KVPair> pair_) : pair(std::move(pair_)) {}
|
||||
|
||||
explicit const_iterator(rocksdb::Iterator * iter_) : iter(iter_)
|
||||
{
|
||||
updatePairFromIter();
|
||||
}
|
||||
|
||||
const KVPair & operator * () const
|
||||
{
|
||||
return *pair;
|
||||
}
|
||||
|
||||
const KVPair * operator->() const
|
||||
{
|
||||
return pair.get();
|
||||
}
|
||||
|
||||
bool operator != (const const_iterator & other) const
|
||||
{
|
||||
return !(*this == other);
|
||||
}
|
||||
|
||||
bool operator == (const const_iterator & other) const
|
||||
{
|
||||
if (pair == nullptr && other == nullptr)
|
||||
return true;
|
||||
if (pair == nullptr || other == nullptr)
|
||||
return false;
|
||||
return pair->key.toView() == other->key.toView() && iter == other.iter;
|
||||
}
|
||||
|
||||
bool operator == (std::nullptr_t) const
|
||||
{
|
||||
return iter == nullptr;
|
||||
}
|
||||
|
||||
bool operator != (std::nullptr_t) const
|
||||
{
|
||||
return iter != nullptr;
|
||||
}
|
||||
|
||||
explicit operator bool() const
|
||||
{
|
||||
return iter != nullptr;
|
||||
}
|
||||
|
||||
const_iterator & operator ++()
|
||||
{
|
||||
iter->Next();
|
||||
updatePairFromIter();
|
||||
return *this;
|
||||
}
|
||||
|
||||
private:
|
||||
void updatePairFromIter()
|
||||
{
|
||||
if (iter && iter->Valid())
|
||||
{
|
||||
auto new_pair = std::make_shared<KVPair>();
|
||||
new_pair->key = StringRef(getDecodedKey(iter->key().ToStringView()));
|
||||
ReadBufferFromOwnString buffer(iter->value().ToStringView());
|
||||
typename Node::Meta & meta = new_pair->value;
|
||||
readPODBinary(meta, buffer);
|
||||
readVarUInt(new_pair->value.data_size, buffer);
|
||||
if (new_pair->value.data_size)
|
||||
{
|
||||
new_pair->value.data = std::unique_ptr<char[]>(new char[new_pair->value.data_size]);
|
||||
buffer.readStrict(new_pair->value.data.get(), new_pair->value.data_size);
|
||||
}
|
||||
pair = new_pair;
|
||||
}
|
||||
else
|
||||
{
|
||||
pair = nullptr;
|
||||
iter = nullptr;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
bool initialized = false;
|
||||
|
||||
const const_iterator end_ptr;
|
||||
|
||||
void initialize(const KeeperContextPtr & context)
|
||||
{
|
||||
DiskPtr disk = context->getTemporaryRocksDBDisk();
|
||||
if (disk == nullptr)
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get rocksdb disk");
|
||||
}
|
||||
auto options = context->getRocksDBOptions();
|
||||
if (options == nullptr)
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get rocksdb options");
|
||||
}
|
||||
rocksdb_dir = disk->getPath();
|
||||
rocksdb::DB * db;
|
||||
auto status = rocksdb::DB::Open(*options, rocksdb_dir, &db);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Failed to open rocksdb path at: {}: {}",
|
||||
rocksdb_dir, status.ToString());
|
||||
}
|
||||
rocksdb_ptr = std::unique_ptr<rocksdb::DB>(db);
|
||||
write_options.disableWAL = true;
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
~RocksDBContainer()
|
||||
{
|
||||
if (initialized)
|
||||
{
|
||||
rocksdb_ptr->Close();
|
||||
rocksdb_ptr = nullptr;
|
||||
|
||||
std::filesystem::remove_all(rocksdb_dir);
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<std::pair<std::string, Node>> getChildren(const std::string & key_)
|
||||
{
|
||||
rocksdb::ReadOptions read_options;
|
||||
read_options.total_order_seek = true;
|
||||
|
||||
std::string key = key_;
|
||||
if (!key.ends_with('/'))
|
||||
key += '/';
|
||||
size_t len = key.size() + 2;
|
||||
|
||||
auto iter = std::unique_ptr<rocksdb::Iterator>(rocksdb_ptr->NewIterator(read_options));
|
||||
std::string encoded_string = getEncodedKey(key, true);
|
||||
rocksdb::Slice prefix(encoded_string);
|
||||
std::vector<std::pair<std::string, Node>> result;
|
||||
for (iter->Seek(prefix); iter->Valid() && iter->key().starts_with(prefix); iter->Next())
|
||||
{
|
||||
Node node;
|
||||
ReadBufferFromOwnString buffer(iter->value().ToStringView());
|
||||
typename Node::Meta & meta = node;
|
||||
/// We do not read data here
|
||||
readPODBinary(meta, buffer);
|
||||
std::string real_key(iter->key().data() + len, iter->key().size() - len);
|
||||
// std::cout << "real key: " << real_key << std::endl;
|
||||
result.emplace_back(std::move(real_key), std::move(node));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
bool contains(const std::string & path)
|
||||
{
|
||||
const std::string & encoded_key = getEncodedKey(path);
|
||||
std::string buffer_str;
|
||||
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &buffer_str);
|
||||
if (status.IsNotFound())
|
||||
return false;
|
||||
if (!status.ok())
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during executing contains. The error message is {}.", status.ToString());
|
||||
return true;
|
||||
}
|
||||
|
||||
const_iterator find(StringRef key_)
|
||||
{
|
||||
/// rocksdb::PinnableSlice slice;
|
||||
const std::string & encoded_key = getEncodedKey(key_.toString());
|
||||
std::string buffer_str;
|
||||
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &buffer_str);
|
||||
if (status.IsNotFound())
|
||||
return end();
|
||||
if (!status.ok())
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during executing find. The error message is {}.", status.ToString());
|
||||
ReadBufferFromOwnString buffer(buffer_str);
|
||||
auto kv = std::make_shared<KVPair>();
|
||||
kv->key = key_;
|
||||
typename Node::Meta & meta = kv->value;
|
||||
readPODBinary(meta, buffer);
|
||||
/// TODO: Sometimes we don't need to load data.
|
||||
readVarUInt(kv->value.data_size, buffer);
|
||||
if (kv->value.data_size)
|
||||
{
|
||||
kv->value.data = std::unique_ptr<char[]>(new char[kv->value.data_size]);
|
||||
buffer.readStrict(kv->value.data.get(), kv->value.data_size);
|
||||
}
|
||||
return const_iterator(kv);
|
||||
}
|
||||
|
||||
MockNode getValue(StringRef key)
|
||||
{
|
||||
auto it = find(key);
|
||||
chassert(it != end());
|
||||
return MockNode(it->value.numChildren(), it->value.getData());
|
||||
}
|
||||
|
||||
const_iterator updateValue(StringRef key_, ValueUpdater updater)
|
||||
{
|
||||
/// rocksdb::PinnableSlice slice;
|
||||
const std::string & key = key_.toString();
|
||||
const std::string & encoded_key = getEncodedKey(key);
|
||||
std::string buffer_str;
|
||||
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &buffer_str);
|
||||
if (!status.ok())
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during find. The error message is {}.", status.ToString());
|
||||
auto kv = std::make_shared<KVPair>();
|
||||
kv->key = key_;
|
||||
kv->value.decodeFromString(buffer_str);
|
||||
/// storage->removeDigest(node, key);
|
||||
updater(kv->value);
|
||||
insertOrReplace(key, kv->value);
|
||||
return const_iterator(kv);
|
||||
}
|
||||
|
||||
bool insert(const std::string & key, Node & value)
|
||||
{
|
||||
std::string value_str;
|
||||
const std::string & encoded_key = getEncodedKey(key);
|
||||
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &value_str);
|
||||
if (status.ok())
|
||||
{
|
||||
return false;
|
||||
}
|
||||
else if (status.IsNotFound())
|
||||
{
|
||||
status = rocksdb_ptr->Put(write_options, encoded_key, value.getEncodedString());
|
||||
if (status.ok())
|
||||
{
|
||||
counter++;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during insert. The error message is {}.", status.ToString());
|
||||
}
|
||||
|
||||
void insertOrReplace(const std::string & key, Node & value)
|
||||
{
|
||||
const std::string & encoded_key = getEncodedKey(key);
|
||||
/// storage->addDigest(value, key);
|
||||
std::string value_str;
|
||||
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &value_str);
|
||||
bool increase_counter = false;
|
||||
if (status.IsNotFound())
|
||||
increase_counter = true;
|
||||
else if (!status.ok())
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during get. The error message is {}.", status.ToString());
|
||||
|
||||
status = rocksdb_ptr->Put(write_options, encoded_key, value.getEncodedString());
|
||||
if (status.ok())
|
||||
counter += increase_counter;
|
||||
else
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during insert. The error message is {}.", status.ToString());
|
||||
}
|
||||
|
||||
using KeyPtr = std::unique_ptr<char[]>;
|
||||
|
||||
/// To be compatible with SnapshotableHashTable, will remove later;
|
||||
KeyPtr allocateKey(size_t size)
|
||||
{
|
||||
return KeyPtr{new char[size]};
|
||||
}
|
||||
|
||||
void insertOrReplace(KeyPtr key_data, size_t key_size, Node value)
|
||||
{
|
||||
std::string key(key_data.get(), key_size);
|
||||
insertOrReplace(key, value);
|
||||
}
|
||||
|
||||
bool erase(const std::string & key)
|
||||
{
|
||||
/// storage->removeDigest(value, key);
|
||||
const std::string & encoded_key = getEncodedKey(key);
|
||||
|
||||
auto status = rocksdb_ptr->Delete(write_options, encoded_key);
|
||||
if (status.IsNotFound())
|
||||
return false;
|
||||
if (status.ok())
|
||||
{
|
||||
counter--;
|
||||
return true;
|
||||
}
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during erase. The error message is {}.", status.ToString());
|
||||
}
|
||||
|
||||
void recalculateDataSize() {}
|
||||
void reverse(size_t size_) {(void)size_;}
|
||||
|
||||
uint64_t getApproximateDataSize() const
|
||||
{
|
||||
/// use statistics from rocksdb
|
||||
return counter * sizeof(Node);
|
||||
}
|
||||
|
||||
void enableSnapshotMode(size_t version)
|
||||
{
|
||||
chassert(!snapshot_mode);
|
||||
snapshot_mode = true;
|
||||
snapshot_up_to_version = version;
|
||||
snapshot_size = counter;
|
||||
++current_version;
|
||||
|
||||
snapshot = rocksdb_ptr->GetSnapshot();
|
||||
}
|
||||
|
||||
void disableSnapshotMode()
|
||||
{
|
||||
chassert(snapshot_mode);
|
||||
snapshot_mode = false;
|
||||
rocksdb_ptr->ReleaseSnapshot(snapshot);
|
||||
}
|
||||
|
||||
void clearOutdatedNodes() {}
|
||||
|
||||
std::pair<size_t, size_t> snapshotSizeWithVersion() const
|
||||
{
|
||||
if (!snapshot_mode)
|
||||
return std::make_pair(counter, current_version);
|
||||
else
|
||||
return std::make_pair(snapshot_size, current_version);
|
||||
}
|
||||
|
||||
const_iterator begin() const
|
||||
{
|
||||
rocksdb::ReadOptions read_options;
|
||||
read_options.total_order_seek = true;
|
||||
if (snapshot_mode)
|
||||
read_options.snapshot = snapshot;
|
||||
auto * iter = rocksdb_ptr->NewIterator(read_options);
|
||||
iter->SeekToFirst();
|
||||
return const_iterator(iter);
|
||||
}
|
||||
|
||||
const_iterator end() const
|
||||
{
|
||||
return end_ptr;
|
||||
}
|
||||
|
||||
size_t size() const
|
||||
{
|
||||
return counter;
|
||||
}
|
||||
|
||||
uint64_t getArenaDataSize() const
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
uint64_t keyArenaSize() const
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
private:
|
||||
String rocksdb_dir;
|
||||
|
||||
std::unique_ptr<rocksdb::DB> rocksdb_ptr;
|
||||
rocksdb::WriteOptions write_options;
|
||||
|
||||
const rocksdb::Snapshot * snapshot;
|
||||
|
||||
bool snapshot_mode{false};
|
||||
size_t current_version{0};
|
||||
size_t snapshot_up_to_version{0};
|
||||
size_t snapshot_size{0};
|
||||
size_t counter{0};
|
||||
|
||||
};
|
||||
|
||||
}
|
@ -212,9 +212,9 @@ private:
|
||||
updateDataSize(INSERT_OR_REPLACE, key.size, new_value_size, old_value_size, !snapshot_mode);
|
||||
}
|
||||
|
||||
|
||||
public:
|
||||
|
||||
using Node = V;
|
||||
using iterator = typename List::iterator;
|
||||
using const_iterator = typename List::const_iterator;
|
||||
using ValueUpdater = std::function<void(V & value)>;
|
||||
@ -364,6 +364,7 @@ public:
|
||||
{
|
||||
auto map_it = map.find(key);
|
||||
if (map_it != map.end())
|
||||
/// return std::make_shared<KVPair>(KVPair{map_it->getMapped()->key, map_it->getMapped()->value});
|
||||
return map_it->getMapped();
|
||||
return list.end();
|
||||
}
|
||||
|
@ -43,7 +43,8 @@ void deserializeSnapshotMagic(ReadBuffer & in)
|
||||
throw Exception(ErrorCodes::CORRUPTED_DATA, "Incorrect magic header in file, expected {}, got {}", SNP_HEADER, magic_header);
|
||||
}
|
||||
|
||||
int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in)
|
||||
template<typename Storage>
|
||||
int64_t deserializeSessionAndTimeout(Storage & storage, ReadBuffer & in)
|
||||
{
|
||||
int32_t count;
|
||||
Coordination::read(count, in);
|
||||
@ -62,7 +63,8 @@ int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in)
|
||||
return max_session_id;
|
||||
}
|
||||
|
||||
void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in)
|
||||
template<typename Storage>
|
||||
void deserializeACLMap(Storage & storage, ReadBuffer & in)
|
||||
{
|
||||
int32_t count;
|
||||
Coordination::read(count, in);
|
||||
@ -90,7 +92,8 @@ void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in)
|
||||
}
|
||||
}
|
||||
|
||||
int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerPtr log)
|
||||
template<typename Storage>
|
||||
int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log)
|
||||
{
|
||||
int64_t max_zxid = 0;
|
||||
std::string path;
|
||||
@ -98,7 +101,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP
|
||||
size_t count = 0;
|
||||
while (path != "/")
|
||||
{
|
||||
KeeperStorage::Node node{};
|
||||
typename Storage::Node node{};
|
||||
String data;
|
||||
Coordination::read(data, in);
|
||||
node.setData(data);
|
||||
@ -146,14 +149,15 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP
|
||||
if (itr.key != "/")
|
||||
{
|
||||
auto parent_path = parentNodePath(itr.key);
|
||||
storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); value.increaseNumChildren(); });
|
||||
storage.container.updateValue(parent_path, [my_path = itr.key] (typename Storage::Node & value) { value.addChild(getBaseNodeName(my_path)); value.increaseNumChildren(); });
|
||||
}
|
||||
}
|
||||
|
||||
return max_zxid;
|
||||
}
|
||||
|
||||
void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, LoggerPtr log)
|
||||
template<typename Storage>
|
||||
void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, LoggerPtr log)
|
||||
{
|
||||
LOG_INFO(log, "Deserializing storage snapshot {}", snapshot_path);
|
||||
int64_t zxid = getZxidFromName(snapshot_path);
|
||||
@ -192,9 +196,11 @@ void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::st
|
||||
LOG_INFO(log, "Finished, snapshot ZXID {}", storage.zxid);
|
||||
}
|
||||
|
||||
void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, LoggerPtr log)
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
template<typename Storage>
|
||||
void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, LoggerPtr log)
|
||||
{
|
||||
namespace fs = std::filesystem;
|
||||
std::map<int64_t, std::string> existing_snapshots;
|
||||
for (const auto & p : fs::directory_iterator(path))
|
||||
{
|
||||
@ -480,7 +486,8 @@ bool hasErrorsInMultiRequest(Coordination::ZooKeeperRequestPtr request)
|
||||
|
||||
}
|
||||
|
||||
bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, LoggerPtr /*log*/)
|
||||
template<typename Storage>
|
||||
bool deserializeTxn(Storage & storage, ReadBuffer & in, LoggerPtr /*log*/)
|
||||
{
|
||||
int64_t checksum;
|
||||
Coordination::read(checksum, in);
|
||||
@ -535,7 +542,8 @@ bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, LoggerPtr /*log*/)
|
||||
return true;
|
||||
}
|
||||
|
||||
void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, LoggerPtr log)
|
||||
template<typename Storage>
|
||||
void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, LoggerPtr log)
|
||||
{
|
||||
ReadBufferFromFile reader(log_path);
|
||||
|
||||
@ -559,9 +567,9 @@ void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string
|
||||
LOG_INFO(log, "Finished {} deserialization, totally read {} records", log_path, counter);
|
||||
}
|
||||
|
||||
void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, LoggerPtr log)
|
||||
template<typename Storage>
|
||||
void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, LoggerPtr log)
|
||||
{
|
||||
namespace fs = std::filesystem;
|
||||
std::map<int64_t, std::string> existing_logs;
|
||||
for (const auto & p : fs::directory_iterator(path))
|
||||
{
|
||||
@ -595,4 +603,9 @@ void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string
|
||||
}
|
||||
}
|
||||
|
||||
template void deserializeKeeperStorageFromSnapshot<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & snapshot_path, LoggerPtr log);
|
||||
template void deserializeKeeperStorageFromSnapshotsDir<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & path, LoggerPtr log);
|
||||
template void deserializeLogAndApplyToStorage<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & log_path, LoggerPtr log);
|
||||
template void deserializeLogsAndApplyToStorage<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & path, LoggerPtr log);
|
||||
|
||||
}
|
||||
|
@ -5,12 +5,16 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, LoggerPtr log);
|
||||
template<typename Storage>
|
||||
void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, LoggerPtr log);
|
||||
|
||||
void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, LoggerPtr log);
|
||||
template<typename Storage>
|
||||
void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, LoggerPtr log);
|
||||
|
||||
void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, LoggerPtr log);
|
||||
template<typename Storage>
|
||||
void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, LoggerPtr log);
|
||||
|
||||
void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, LoggerPtr log);
|
||||
template<typename Storage>
|
||||
void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, LoggerPtr log);
|
||||
|
||||
}
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -36,7 +36,7 @@ class IColumn;
|
||||
M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\
|
||||
M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \
|
||||
M(UInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \
|
||||
M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \
|
||||
M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size in rows for reading", 0) \
|
||||
M(UInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.", 0) \
|
||||
M(UInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.", 0) \
|
||||
M(UInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.", 0) \
|
||||
@ -151,7 +151,7 @@ class IColumn;
|
||||
M(UInt64, max_local_write_bandwidth, 0, "The maximum speed of local writes in bytes per second.", 0) \
|
||||
M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams, and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \
|
||||
M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \
|
||||
\
|
||||
M(Bool, dictionary_validate_primary_key_type, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64.", 0) \
|
||||
M(Bool, distributed_insert_skip_read_only_replicas, false, "If true, INSERT into Distributed will skip read-only replicas.", 0) \
|
||||
M(Bool, distributed_foreground_insert, false, "If setting is enabled, insert query into distributed waits until data are sent to all nodes in a cluster. \n\nEnables or disables synchronous data insertion into a `Distributed` table.\n\nBy default, when inserting data into a Distributed table, the ClickHouse server sends data to cluster nodes in the background. When `distributed_foreground_insert` = 1, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).", 0) ALIAS(insert_distributed_sync) \
|
||||
M(UInt64, distributed_background_insert_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) ALIAS(insert_distributed_timeout) \
|
||||
@ -609,9 +609,8 @@ class IColumn;
|
||||
M(Bool, optimize_time_filter_with_preimage, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')", 0) \
|
||||
M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \
|
||||
M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there are constants there", 0) \
|
||||
M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \
|
||||
M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views. Use true to always deduplicate in dependent tables.", 0) \
|
||||
M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \
|
||||
M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \
|
||||
M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \
|
||||
M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \
|
||||
M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \
|
||||
@ -977,6 +976,7 @@ class IColumn;
|
||||
|
||||
#define OBSOLETE_SETTINGS(M, ALIAS) \
|
||||
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
|
||||
MAKE_OBSOLETE(M, Bool, update_insert_deduplication_token_in_dependent_materialized_views, 0) \
|
||||
MAKE_OBSOLETE(M, UInt64, max_memory_usage_for_all_queries, 0) \
|
||||
MAKE_OBSOLETE(M, UInt64, multiple_joins_rewriter_version, 0) \
|
||||
MAKE_OBSOLETE(M, Bool, enable_debug_queries, false) \
|
||||
|
@ -67,6 +67,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."},
|
||||
{"input_format_json_ignore_key_case", false, false, "Ignore json key case while read json field from string."},
|
||||
{"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."},
|
||||
{"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."},
|
||||
{"collect_hash_table_stats_during_joins", false, true, "New setting."},
|
||||
{"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."},
|
||||
{"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."},
|
||||
|
@ -20,7 +20,6 @@
|
||||
#endif
|
||||
#include <cerrno>
|
||||
#include <cstring>
|
||||
#include <csignal>
|
||||
#include <unistd.h>
|
||||
|
||||
#include <algorithm>
|
||||
@ -37,6 +36,7 @@
|
||||
#include <Poco/Pipe.h>
|
||||
|
||||
#include <Common/ErrorHandlers.h>
|
||||
#include <Common/SignalHandlers.h>
|
||||
#include <base/argsToConfig.h>
|
||||
#include <base/getThreadId.h>
|
||||
#include <base/coverage.h>
|
||||
@ -54,7 +54,6 @@
|
||||
#include <Common/Config/ConfigProcessor.h>
|
||||
#include <Common/SymbolIndex.h>
|
||||
#include <Common/getExecutablePath.h>
|
||||
#include <Common/getHashOfLoadedBinary.h>
|
||||
#include <Common/Elf.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/logger_useful.h>
|
||||
@ -79,8 +78,6 @@ namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_SET_SIGNAL_HANDLER;
|
||||
extern const int CANNOT_SEND_SIGNAL;
|
||||
extern const int SYSTEM_ERROR;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
@ -88,114 +85,6 @@ namespace DB
|
||||
|
||||
using namespace DB;
|
||||
|
||||
PipeFDs signal_pipe;
|
||||
|
||||
|
||||
/** Reset signal handler to the default and send signal to itself.
|
||||
* It's called from user signal handler to write core dump.
|
||||
*/
|
||||
static void call_default_signal_handler(int sig)
|
||||
{
|
||||
if (SIG_ERR == signal(sig, SIG_DFL))
|
||||
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
|
||||
|
||||
if (0 != raise(sig))
|
||||
throw ErrnoException(ErrorCodes::CANNOT_SEND_SIGNAL, "Cannot send signal");
|
||||
}
|
||||
|
||||
static const size_t signal_pipe_buf_size =
|
||||
sizeof(int)
|
||||
+ sizeof(siginfo_t)
|
||||
+ sizeof(ucontext_t*)
|
||||
+ sizeof(StackTrace)
|
||||
+ sizeof(UInt64)
|
||||
+ sizeof(UInt32)
|
||||
+ sizeof(void*);
|
||||
|
||||
using signal_function = void(int, siginfo_t*, void*);
|
||||
|
||||
static void writeSignalIDtoSignalPipe(int sig)
|
||||
{
|
||||
auto saved_errno = errno; /// We must restore previous value of errno in signal handler.
|
||||
|
||||
char buf[signal_pipe_buf_size];
|
||||
WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf);
|
||||
writeBinary(sig, out);
|
||||
out.next();
|
||||
|
||||
errno = saved_errno;
|
||||
}
|
||||
|
||||
/** Signal handler for HUP */
|
||||
static void closeLogsSignalHandler(int sig, siginfo_t *, void *)
|
||||
{
|
||||
DENY_ALLOCATIONS_IN_SCOPE;
|
||||
writeSignalIDtoSignalPipe(sig);
|
||||
}
|
||||
|
||||
static void terminateRequestedSignalHandler(int sig, siginfo_t *, void *)
|
||||
{
|
||||
DENY_ALLOCATIONS_IN_SCOPE;
|
||||
writeSignalIDtoSignalPipe(sig);
|
||||
}
|
||||
|
||||
|
||||
static std::atomic_flag fatal_error_printed;
|
||||
|
||||
/** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log.
|
||||
*/
|
||||
static void signalHandler(int sig, siginfo_t * info, void * context)
|
||||
{
|
||||
if (asynchronous_stack_unwinding && sig == SIGSEGV)
|
||||
siglongjmp(asynchronous_stack_unwinding_signal_jump_buffer, 1);
|
||||
|
||||
DENY_ALLOCATIONS_IN_SCOPE;
|
||||
auto saved_errno = errno; /// We must restore previous value of errno in signal handler.
|
||||
|
||||
char buf[signal_pipe_buf_size];
|
||||
WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf);
|
||||
|
||||
const ucontext_t * signal_context = reinterpret_cast<ucontext_t *>(context);
|
||||
const StackTrace stack_trace(*signal_context);
|
||||
|
||||
#if USE_GWP_ASAN
|
||||
if (const auto fault_address = reinterpret_cast<uintptr_t>(info->si_addr);
|
||||
GWPAsan::isGWPAsanError(fault_address))
|
||||
GWPAsan::printReport(fault_address);
|
||||
#endif
|
||||
|
||||
writeBinary(sig, out);
|
||||
writePODBinary(*info, out);
|
||||
writePODBinary(signal_context, out);
|
||||
writePODBinary(stack_trace, out);
|
||||
writeVectorBinary(Exception::enable_job_stack_trace ? Exception::thread_frame_pointers : std::vector<StackTrace::FramePointers>{}, out);
|
||||
writeBinary(static_cast<UInt32>(getThreadId()), out);
|
||||
writePODBinary(current_thread, out);
|
||||
|
||||
out.next();
|
||||
|
||||
if (sig != SIGTSTP) /// This signal is used for debugging.
|
||||
{
|
||||
/// The time that is usually enough for separate thread to print info into log.
|
||||
/// Under MSan full stack unwinding with DWARF info about inline functions takes 101 seconds in one case.
|
||||
for (size_t i = 0; i < 300; ++i)
|
||||
{
|
||||
/// We will synchronize with the thread printing the messages with an atomic variable to finish earlier.
|
||||
if (fatal_error_printed.test())
|
||||
break;
|
||||
|
||||
/// This coarse method of synchronization is perfectly ok for fatal signals.
|
||||
sleepForSeconds(1);
|
||||
}
|
||||
|
||||
/// Wait for all logs flush operations
|
||||
sleepForSeconds(3);
|
||||
call_default_signal_handler(sig);
|
||||
}
|
||||
|
||||
errno = saved_errno;
|
||||
}
|
||||
|
||||
|
||||
static bool getenvBool(const char * name)
|
||||
{
|
||||
@ -207,445 +96,6 @@ static bool getenvBool(const char * name)
|
||||
}
|
||||
|
||||
|
||||
/// Avoid link time dependency on DB/Interpreters - will use this function only when linked.
|
||||
__attribute__((__weak__)) void collectCrashLog(
|
||||
Int32 signal, UInt64 thread_id, const String & query_id, const StackTrace & stack_trace);
|
||||
|
||||
|
||||
/** The thread that read info about signal or std::terminate from pipe.
|
||||
* On HUP, close log files (for new files to be opened later).
|
||||
* On information about std::terminate, write it to log.
|
||||
* On other signals, write info to log.
|
||||
*/
|
||||
class SignalListener : public Poco::Runnable
|
||||
{
|
||||
public:
|
||||
static constexpr int StdTerminate = -1;
|
||||
static constexpr int StopThread = -2;
|
||||
static constexpr int SanitizerTrap = -3;
|
||||
|
||||
explicit SignalListener(BaseDaemon & daemon_)
|
||||
: log(getLogger("BaseDaemon"))
|
||||
, daemon(daemon_)
|
||||
{
|
||||
}
|
||||
|
||||
void run() override
|
||||
{
|
||||
static_assert(PIPE_BUF >= 512);
|
||||
static_assert(signal_pipe_buf_size <= PIPE_BUF, "Only write of PIPE_BUF to pipe is atomic and the minimal known PIPE_BUF across supported platforms is 512");
|
||||
char buf[signal_pipe_buf_size];
|
||||
ReadBufferFromFileDescriptor in(signal_pipe.fds_rw[0], signal_pipe_buf_size, buf);
|
||||
|
||||
while (!in.eof())
|
||||
{
|
||||
int sig = 0;
|
||||
readBinary(sig, in);
|
||||
// We may log some specific signals afterwards, with different log
|
||||
// levels and more info, but for completeness we log all signals
|
||||
// here at trace level.
|
||||
// Don't use strsignal here, because it's not thread-safe.
|
||||
LOG_TRACE(log, "Received signal {}", sig);
|
||||
|
||||
if (sig == StopThread)
|
||||
{
|
||||
LOG_INFO(log, "Stop SignalListener thread");
|
||||
break;
|
||||
}
|
||||
else if (sig == SIGHUP)
|
||||
{
|
||||
LOG_DEBUG(log, "Received signal to close logs.");
|
||||
BaseDaemon::instance().closeLogs(BaseDaemon::instance().logger());
|
||||
LOG_INFO(log, "Opened new log file after received signal.");
|
||||
}
|
||||
else if (sig == StdTerminate)
|
||||
{
|
||||
UInt32 thread_num;
|
||||
std::string message;
|
||||
|
||||
readBinary(thread_num, in);
|
||||
readBinary(message, in);
|
||||
|
||||
onTerminate(message, thread_num);
|
||||
}
|
||||
else if (sig == SIGINT ||
|
||||
sig == SIGQUIT ||
|
||||
sig == SIGTERM)
|
||||
{
|
||||
daemon.handleSignal(sig);
|
||||
}
|
||||
else
|
||||
{
|
||||
siginfo_t info{};
|
||||
ucontext_t * context{};
|
||||
StackTrace stack_trace(NoCapture{});
|
||||
std::vector<StackTrace::FramePointers> thread_frame_pointers;
|
||||
UInt32 thread_num{};
|
||||
ThreadStatus * thread_ptr{};
|
||||
|
||||
if (sig != SanitizerTrap)
|
||||
{
|
||||
readPODBinary(info, in);
|
||||
readPODBinary(context, in);
|
||||
}
|
||||
|
||||
readPODBinary(stack_trace, in);
|
||||
|
||||
if (sig != SanitizerTrap)
|
||||
readVectorBinary(thread_frame_pointers, in);
|
||||
|
||||
readBinary(thread_num, in);
|
||||
readPODBinary(thread_ptr, in);
|
||||
|
||||
/// This allows to receive more signals if failure happens inside onFault function.
|
||||
/// Example: segfault while symbolizing stack trace.
|
||||
try
|
||||
{
|
||||
std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); }).detach();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Likely cannot allocate thread
|
||||
onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
LoggerPtr log;
|
||||
BaseDaemon & daemon;
|
||||
|
||||
void onTerminate(std::string_view message, UInt32 thread_num) const
|
||||
{
|
||||
size_t pos = message.find('\n');
|
||||
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) {}",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon.build_id, daemon.git_hash, thread_num, message.substr(0, pos));
|
||||
|
||||
/// Print trace from std::terminate exception line-by-line to make it easy for grep.
|
||||
while (pos != std::string_view::npos)
|
||||
{
|
||||
++pos;
|
||||
size_t next_pos = message.find('\n', pos);
|
||||
size_t size = next_pos;
|
||||
if (next_pos != std::string_view::npos)
|
||||
size = next_pos - pos;
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(message.substr(pos, size)));
|
||||
pos = next_pos;
|
||||
}
|
||||
}
|
||||
|
||||
void onFault(
|
||||
int sig,
|
||||
const siginfo_t & info,
|
||||
ucontext_t * context,
|
||||
const StackTrace & stack_trace,
|
||||
const std::vector<StackTrace::FramePointers> & thread_frame_pointers,
|
||||
UInt32 thread_num,
|
||||
ThreadStatus * thread_ptr) const
|
||||
try
|
||||
{
|
||||
ThreadStatus thread_status;
|
||||
|
||||
/// First log those fields that are safe to access and that should not cause new fault.
|
||||
/// That way we will have some duplicated info in the log but we don't loose important info
|
||||
/// in case of double fault.
|
||||
|
||||
LOG_FATAL(log, "########## Short fault info ############");
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon.build_id, daemon.git_hash,
|
||||
thread_num, sig);
|
||||
|
||||
std::string signal_description = "Unknown signal";
|
||||
|
||||
/// Some of these are not really signals, but our own indications on failure reason.
|
||||
if (sig == StdTerminate)
|
||||
signal_description = "std::terminate";
|
||||
else if (sig == SanitizerTrap)
|
||||
signal_description = "sanitizer trap";
|
||||
else if (sig >= 0)
|
||||
signal_description = strsignal(sig); // NOLINT(concurrency-mt-unsafe) // it is not thread-safe but ok in this context
|
||||
|
||||
LOG_FATAL(log, "Signal description: {}", signal_description);
|
||||
|
||||
String error_message;
|
||||
|
||||
if (sig != SanitizerTrap)
|
||||
error_message = signalToErrorMessage(sig, info, *context);
|
||||
else
|
||||
error_message = "Sanitizer trap.";
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(error_message));
|
||||
|
||||
String bare_stacktrace_str;
|
||||
if (stack_trace.getSize())
|
||||
{
|
||||
/// Write bare stack trace (addresses) just in case if we will fail to print symbolized stack trace.
|
||||
/// NOTE: This still require memory allocations and mutex lock inside logger.
|
||||
/// BTW we can also print it to stderr using write syscalls.
|
||||
|
||||
WriteBufferFromOwnString bare_stacktrace;
|
||||
writeString("Stack trace:", bare_stacktrace);
|
||||
for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i)
|
||||
{
|
||||
writeChar(' ', bare_stacktrace);
|
||||
writePointerHex(stack_trace.getFramePointers()[i], bare_stacktrace);
|
||||
}
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(bare_stacktrace.str()));
|
||||
bare_stacktrace_str = bare_stacktrace.str();
|
||||
}
|
||||
|
||||
/// Now try to access potentially unsafe data in thread_ptr.
|
||||
|
||||
String query_id;
|
||||
String query;
|
||||
|
||||
/// Send logs from this thread to client if possible.
|
||||
/// It will allow client to see failure messages directly.
|
||||
if (thread_ptr)
|
||||
{
|
||||
query_id = thread_ptr->getQueryId();
|
||||
query = thread_ptr->getQueryForLog();
|
||||
|
||||
if (auto logs_queue = thread_ptr->getInternalTextLogsQueue())
|
||||
{
|
||||
CurrentThread::attachInternalTextLogsQueue(logs_queue, LogsLevel::trace);
|
||||
}
|
||||
}
|
||||
|
||||
LOG_FATAL(log, "########################################");
|
||||
|
||||
if (query_id.empty())
|
||||
{
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (no query) Received signal {} ({})",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon.build_id, daemon.git_hash,
|
||||
thread_num, signal_description, sig);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (query_id: {}) (query: {}) Received signal {} ({})",
|
||||
VERSION_STRING, VERSION_OFFICIAL, daemon.build_id, daemon.git_hash,
|
||||
thread_num, query_id, query, signal_description, sig);
|
||||
}
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(error_message));
|
||||
|
||||
if (!bare_stacktrace_str.empty())
|
||||
{
|
||||
LOG_FATAL(log, fmt::runtime(bare_stacktrace_str));
|
||||
}
|
||||
|
||||
/// Write symbolized stack trace line by line for better grep-ability.
|
||||
stack_trace.toStringEveryLine([&](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); });
|
||||
|
||||
/// In case it's a scheduled job write all previous jobs origins call stacks
|
||||
std::for_each(thread_frame_pointers.rbegin(), thread_frame_pointers.rend(),
|
||||
[this](const StackTrace::FramePointers & frame_pointers)
|
||||
{
|
||||
if (size_t size = std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin())
|
||||
{
|
||||
LOG_FATAL(log, "========================================");
|
||||
WriteBufferFromOwnString bare_stacktrace;
|
||||
writeString("Job's origin stack trace:", bare_stacktrace);
|
||||
std::for_each_n(frame_pointers.begin(), size,
|
||||
[&bare_stacktrace](const void * ptr)
|
||||
{
|
||||
writeChar(' ', bare_stacktrace);
|
||||
writePointerHex(ptr, bare_stacktrace);
|
||||
}
|
||||
);
|
||||
|
||||
LOG_FATAL(log, fmt::runtime(bare_stacktrace.str()));
|
||||
|
||||
StackTrace::toStringEveryLine(const_cast<void **>(frame_pointers.data()), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); });
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
/// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace.
|
||||
/// Please keep the below log messages in-sync with the ones in programs/server/Server.cpp
|
||||
|
||||
if (daemon.stored_binary_hash.empty())
|
||||
{
|
||||
LOG_FATAL(log, "Integrity check of the executable skipped because the reference checksum could not be read.");
|
||||
}
|
||||
else
|
||||
{
|
||||
String calculated_binary_hash = getHashOfLoadedBinaryHex();
|
||||
if (calculated_binary_hash == daemon.stored_binary_hash)
|
||||
{
|
||||
LOG_FATAL(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_FATAL(
|
||||
log,
|
||||
"Calculated checksum of the executable ({0}) does not correspond"
|
||||
" to the reference checksum stored in the executable ({1})."
|
||||
" This may indicate one of the following:"
|
||||
" - the executable was changed just after startup;"
|
||||
" - the executable was corrupted on disk due to faulty hardware;"
|
||||
" - the loaded executable was corrupted in memory due to faulty hardware;"
|
||||
" - the file was intentionally modified;"
|
||||
" - a logical error in the code.",
|
||||
calculated_binary_hash,
|
||||
daemon.stored_binary_hash);
|
||||
}
|
||||
}
|
||||
#endif
|
||||
|
||||
/// Write crash to system.crash_log table if available.
|
||||
if (collectCrashLog)
|
||||
collectCrashLog(sig, thread_num, query_id, stack_trace);
|
||||
|
||||
Context::getGlobalContextInstance()->handleCrash();
|
||||
|
||||
/// Send crash report to developers (if configured)
|
||||
if (sig != SanitizerTrap)
|
||||
{
|
||||
if (auto * sentry = SentryWriter::getInstance())
|
||||
sentry->onSignal(sig, error_message, stack_trace.getFramePointers(), stack_trace.getOffset(), stack_trace.getSize());
|
||||
|
||||
/// Advice the user to send it manually.
|
||||
if (std::string_view(VERSION_OFFICIAL).contains("official build"))
|
||||
{
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
/// Approximate support period, upper bound.
|
||||
if (time(nullptr) - date_lut.makeDate(2000 + VERSION_MAJOR, VERSION_MINOR, 1) < (365 + 30) * 86400)
|
||||
{
|
||||
LOG_FATAL(log, "Report this error to https://github.com/ClickHouse/ClickHouse/issues");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_FATAL(log, "ClickHouse version {} is old and should be upgraded to the latest version.", VERSION_STRING);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_FATAL(log, "This ClickHouse version is not official and should be upgraded to the official build.");
|
||||
}
|
||||
}
|
||||
|
||||
/// List changed settings.
|
||||
if (!query_id.empty())
|
||||
{
|
||||
ContextPtr query_context = thread_ptr->getQueryContext();
|
||||
if (query_context)
|
||||
{
|
||||
String changed_settings = query_context->getSettingsRef().toString();
|
||||
|
||||
if (changed_settings.empty())
|
||||
LOG_FATAL(log, "No settings were changed");
|
||||
else
|
||||
LOG_FATAL(log, "Changed settings: {}", changed_settings);
|
||||
}
|
||||
}
|
||||
|
||||
/// When everything is done, we will try to send these error messages to the client.
|
||||
if (thread_ptr)
|
||||
thread_ptr->onFatalError();
|
||||
|
||||
fatal_error_printed.test_and_set();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// onFault is called from the std::thread, and it should catch all exceptions; otherwise, you can get unrelated fatal errors.
|
||||
PreformattedMessage message = getCurrentExceptionMessageAndPattern(true);
|
||||
LOG_FATAL(getLogger(__PRETTY_FUNCTION__), message);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
#if defined(SANITIZER)
|
||||
|
||||
template <typename T>
|
||||
struct ValueHolder
|
||||
{
|
||||
ValueHolder(T value_) : value(value_)
|
||||
{}
|
||||
|
||||
T value;
|
||||
};
|
||||
|
||||
extern "C" void __sanitizer_set_death_callback(void (*)());
|
||||
|
||||
/// Sanitizers may not expect some function calls from death callback.
|
||||
/// Let's try to disable instrumentation to avoid possible issues.
|
||||
/// However, this callback may call other functions that are still instrumented.
|
||||
/// We can try [[clang::always_inline]] attribute for statements in future (available in clang-15)
|
||||
/// See https://github.com/google/sanitizers/issues/1543 and https://github.com/google/sanitizers/issues/1549.
|
||||
static DISABLE_SANITIZER_INSTRUMENTATION void sanitizerDeathCallback()
|
||||
{
|
||||
DENY_ALLOCATIONS_IN_SCOPE;
|
||||
/// Also need to send data via pipe. Otherwise it may lead to deadlocks or failures in printing diagnostic info.
|
||||
|
||||
char buf[signal_pipe_buf_size];
|
||||
WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf);
|
||||
|
||||
const StackTrace stack_trace;
|
||||
|
||||
writeBinary(SignalListener::SanitizerTrap, out);
|
||||
writePODBinary(stack_trace, out);
|
||||
/// We create a dummy struct with a constructor so DISABLE_SANITIZER_INSTRUMENTATION is not applied to it
|
||||
/// otherwise, Memory sanitizer can't know that values initiialized inside this function are actually initialized
|
||||
/// because instrumentations are disabled leading to false positives later on
|
||||
ValueHolder<UInt32> thread_id{static_cast<UInt32>(getThreadId())};
|
||||
writeBinary(thread_id.value, out);
|
||||
writePODBinary(current_thread, out);
|
||||
|
||||
out.next();
|
||||
|
||||
/// The time that is usually enough for separate thread to print info into log.
|
||||
sleepForSeconds(20);
|
||||
}
|
||||
#endif
|
||||
|
||||
|
||||
/** To use with std::set_terminate.
|
||||
* Collects slightly more info than __gnu_cxx::__verbose_terminate_handler,
|
||||
* and send it to pipe. Other thread will read this info from pipe and asynchronously write it to log.
|
||||
* Look at libstdc++-v3/libsupc++/vterminate.cc for example.
|
||||
*/
|
||||
[[noreturn]] static void terminate_handler()
|
||||
{
|
||||
static thread_local bool terminating = false;
|
||||
if (terminating)
|
||||
abort();
|
||||
|
||||
terminating = true;
|
||||
|
||||
std::string log_message;
|
||||
|
||||
if (std::current_exception())
|
||||
log_message = "Terminate called for uncaught exception:\n" + getCurrentExceptionMessage(true);
|
||||
else
|
||||
log_message = "Terminate called without an active exception";
|
||||
|
||||
/// POSIX.1 says that write(2)s of less than PIPE_BUF bytes must be atomic - man 7 pipe
|
||||
/// And the buffer should not be too small because our exception messages can be large.
|
||||
static constexpr size_t buf_size = PIPE_BUF;
|
||||
|
||||
if (log_message.size() > buf_size - 16)
|
||||
log_message.resize(buf_size - 16);
|
||||
|
||||
char buf[buf_size];
|
||||
WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], buf_size, buf);
|
||||
|
||||
writeBinary(static_cast<int>(SignalListener::StdTerminate), out);
|
||||
writeBinary(static_cast<UInt32>(getThreadId()), out);
|
||||
writeBinary(log_message, out);
|
||||
out.next();
|
||||
|
||||
abort();
|
||||
}
|
||||
|
||||
|
||||
static std::string createDirectory(const std::string & file)
|
||||
{
|
||||
fs::path path = fs::path(file).parent_path();
|
||||
@ -698,22 +148,7 @@ BaseDaemon::~BaseDaemon()
|
||||
{
|
||||
writeSignalIDtoSignalPipe(SignalListener::StopThread);
|
||||
signal_listener_thread.join();
|
||||
/// Reset signals to SIG_DFL to avoid trying to write to the signal_pipe that will be closed after.
|
||||
for (int sig : handled_signals)
|
||||
if (SIG_ERR == signal(sig, SIG_DFL))
|
||||
{
|
||||
try
|
||||
{
|
||||
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
|
||||
}
|
||||
catch (ErrnoException &)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
signal_pipe.close();
|
||||
|
||||
HandledSignals::instance().reset();
|
||||
SentryWriter::resetInstance();
|
||||
}
|
||||
|
||||
@ -752,6 +187,8 @@ void BaseDaemon::closeFDs()
|
||||
#else
|
||||
fs::path proc_path{"/proc/self/fd"};
|
||||
#endif
|
||||
|
||||
const auto & signal_pipe = HandledSignals::instance().signal_pipe;
|
||||
if (fs::is_directory(proc_path)) /// Hooray, proc exists
|
||||
{
|
||||
/// in /proc/self/fd directory filenames are numeric file descriptors.
|
||||
@ -972,56 +409,6 @@ void BaseDaemon::initialize(Application & self)
|
||||
}
|
||||
|
||||
|
||||
static void addSignalHandler(const std::vector<int> & signals, signal_function handler, std::vector<int> * out_handled_signals)
|
||||
{
|
||||
struct sigaction sa;
|
||||
memset(&sa, 0, sizeof(sa));
|
||||
sa.sa_sigaction = handler;
|
||||
sa.sa_flags = SA_SIGINFO;
|
||||
|
||||
#if defined(OS_DARWIN)
|
||||
sigemptyset(&sa.sa_mask);
|
||||
for (auto signal : signals)
|
||||
sigaddset(&sa.sa_mask, signal);
|
||||
#else
|
||||
if (sigemptyset(&sa.sa_mask))
|
||||
throw Poco::Exception("Cannot set signal handler.");
|
||||
|
||||
for (auto signal : signals)
|
||||
if (sigaddset(&sa.sa_mask, signal))
|
||||
throw Poco::Exception("Cannot set signal handler.");
|
||||
#endif
|
||||
|
||||
for (auto signal : signals)
|
||||
if (sigaction(signal, &sa, nullptr))
|
||||
throw Poco::Exception("Cannot set signal handler.");
|
||||
|
||||
if (out_handled_signals)
|
||||
std::copy(signals.begin(), signals.end(), std::back_inserter(*out_handled_signals));
|
||||
}
|
||||
|
||||
|
||||
static void blockSignals(const std::vector<int> & signals)
|
||||
{
|
||||
sigset_t sig_set;
|
||||
|
||||
#if defined(OS_DARWIN)
|
||||
sigemptyset(&sig_set);
|
||||
for (auto signal : signals)
|
||||
sigaddset(&sig_set, signal);
|
||||
#else
|
||||
if (sigemptyset(&sig_set))
|
||||
throw Poco::Exception("Cannot block signal.");
|
||||
|
||||
for (auto signal : signals)
|
||||
if (sigaddset(&sig_set, signal))
|
||||
throw Poco::Exception("Cannot block signal.");
|
||||
#endif
|
||||
|
||||
if (pthread_sigmask(SIG_BLOCK, &sig_set, nullptr))
|
||||
throw Poco::Exception("Cannot block signal.");
|
||||
}
|
||||
|
||||
extern const char * GIT_HASH;
|
||||
|
||||
void BaseDaemon::initializeTerminationAndSignalProcessing()
|
||||
@ -1045,29 +432,21 @@ void BaseDaemon::initializeTerminationAndSignalProcessing()
|
||||
};
|
||||
}
|
||||
}
|
||||
std::set_terminate(terminate_handler);
|
||||
|
||||
/// We want to avoid SIGPIPE when working with sockets and pipes, and just handle return value/errno instead.
|
||||
blockSignals({SIGPIPE});
|
||||
|
||||
/// Setup signal handlers.
|
||||
/// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime.
|
||||
addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, &handled_signals);
|
||||
addSignalHandler({SIGHUP}, closeLogsSignalHandler, &handled_signals);
|
||||
addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, &handled_signals);
|
||||
|
||||
#if defined(SANITIZER)
|
||||
__sanitizer_set_death_callback(sanitizerDeathCallback);
|
||||
#endif
|
||||
HandledSignals::instance().setupTerminateHandler();
|
||||
HandledSignals::instance().setupCommonDeadlySignalHandlers();
|
||||
HandledSignals::instance().setupCommonTerminateRequestSignalHandlers();
|
||||
HandledSignals::instance().addSignalHandler({SIGHUP}, closeLogsSignalHandler, true);
|
||||
|
||||
/// Set up Poco ErrorHandler for Poco Threads.
|
||||
static KillingErrorHandler killing_error_handler;
|
||||
Poco::ErrorHandler::set(&killing_error_handler);
|
||||
|
||||
signal_pipe.setNonBlockingWrite();
|
||||
signal_pipe.tryIncreaseSize(1 << 20);
|
||||
|
||||
signal_listener = std::make_unique<SignalListener>(*this);
|
||||
signal_listener = std::make_unique<SignalListener>(this, getLogger("BaseDaemon"));
|
||||
signal_listener_thread.start(*signal_listener);
|
||||
|
||||
#if defined(__ELF__) && !defined(OS_FREEBSD)
|
||||
@ -1273,7 +652,7 @@ void BaseDaemon::setupWatchdog()
|
||||
/// Forward signals to the child process.
|
||||
if (forward_signals)
|
||||
{
|
||||
addSignalHandler(
|
||||
HandledSignals::instance().addSignalHandler(
|
||||
{SIGHUP, SIGINT, SIGQUIT, SIGTERM},
|
||||
[](int sig, siginfo_t *, void *)
|
||||
{
|
||||
@ -1289,7 +668,7 @@ void BaseDaemon::setupWatchdog()
|
||||
(void)res;
|
||||
}
|
||||
},
|
||||
nullptr);
|
||||
false);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1306,6 +685,10 @@ void BaseDaemon::setupWatchdog()
|
||||
int status = 0;
|
||||
do
|
||||
{
|
||||
// Close log files to prevent keeping descriptors of unlinked rotated files.
|
||||
// On next log write files will be reopened.
|
||||
closeLogs(logger());
|
||||
|
||||
if (-1 != waitpid(pid, &status, WUNTRACED | WCONTINUED) || errno == ECHILD)
|
||||
{
|
||||
if (WIFSTOPPED(status))
|
||||
|
@ -168,8 +168,6 @@ protected:
|
||||
String git_hash;
|
||||
String stored_binary_hash;
|
||||
|
||||
std::vector<int> handled_signals;
|
||||
|
||||
bool should_setup_watchdog = false;
|
||||
char * argv0 = nullptr;
|
||||
};
|
||||
|
@ -17,7 +17,7 @@ SerializationPtr DataTypeDate::doGetDefaultSerialization() const
|
||||
|
||||
void registerDataTypeDate(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared<DataTypeDate>()); }, DataTypeFactory::CaseInsensitive);
|
||||
factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared<DataTypeDate>()); }, DataTypeFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ Field DataTypeDate32::getDefault() const
|
||||
void registerDataTypeDate32(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType(
|
||||
"Date32", [] { return DataTypePtr(std::make_shared<DataTypeDate32>()); }, DataTypeFactory::CaseInsensitive);
|
||||
"Date32", [] { return DataTypePtr(std::make_shared<DataTypeDate32>()); }, DataTypeFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -15,8 +15,8 @@ void registerDataTypeDomainBool(DataTypeFactory & factory)
|
||||
std::make_unique<DataTypeCustomFixedName>("Bool"), std::make_unique<SerializationBool>(type->getDefaultSerialization())));
|
||||
});
|
||||
|
||||
factory.registerAlias("bool", "Bool", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("boolean", "Bool", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("bool", "Bool", DataTypeFactory::Case::Insensitive);
|
||||
factory.registerAlias("boolean", "Bool", DataTypeFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -318,7 +318,7 @@ void registerDataTypeEnum(DataTypeFactory & factory)
|
||||
factory.registerDataType("Enum", create);
|
||||
|
||||
/// MySQL
|
||||
factory.registerAlias("ENUM", "Enum", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("ENUM", "Enum", DataTypeFactory::Case::Insensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -175,7 +175,7 @@ DataTypePtr DataTypeFactory::getCustom(DataTypeCustomDescPtr customization) cons
|
||||
}
|
||||
|
||||
|
||||
void DataTypeFactory::registerDataType(const String & family_name, Value creator, CaseSensitiveness case_sensitiveness)
|
||||
void DataTypeFactory::registerDataType(const String & family_name, Value creator, Case case_sensitiveness)
|
||||
{
|
||||
if (creator == nullptr)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type family {} has been provided a null constructor", family_name);
|
||||
@ -189,12 +189,12 @@ void DataTypeFactory::registerDataType(const String & family_name, Value creator
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type family name '{}' is not unique",
|
||||
family_name);
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive
|
||||
if (case_sensitiveness == Case::Insensitive
|
||||
&& !case_insensitive_data_types.emplace(family_name_lowercase, creator).second)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the case insensitive data type family name '{}' is not unique", family_name);
|
||||
}
|
||||
|
||||
void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness)
|
||||
void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, Case case_sensitiveness)
|
||||
{
|
||||
if (creator == nullptr)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type {} has been provided a null constructor",
|
||||
@ -208,7 +208,7 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator
|
||||
}, case_sensitiveness);
|
||||
}
|
||||
|
||||
void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness)
|
||||
void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, Case case_sensitiveness)
|
||||
{
|
||||
registerDataType(family_name, [creator](const ASTPtr & ast)
|
||||
{
|
||||
@ -219,7 +219,7 @@ void DataTypeFactory::registerDataTypeCustom(const String & family_name, Creator
|
||||
}, case_sensitiveness);
|
||||
}
|
||||
|
||||
void DataTypeFactory::registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness)
|
||||
void DataTypeFactory::registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, Case case_sensitiveness)
|
||||
{
|
||||
registerDataTypeCustom(name, [name, creator](const ASTPtr & ast)
|
||||
{
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user