Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into interactive-mode-for-clickhouse-local

This commit is contained in:
kssenii 2021-09-16 16:13:04 +00:00
commit c88dc460d4
1308 changed files with 8551 additions and 2661 deletions

View File

@ -152,6 +152,7 @@ if (CMAKE_GENERATOR STREQUAL "Ninja" AND NOT DISABLE_COLORED_BUILD)
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-color=always")
endif ()
include (cmake/check_flags.cmake)
include (cmake/add_warning.cmake)
if (NOT MSVC)
@ -166,7 +167,8 @@ if (COMPILER_CLANG)
set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges")
endif ()
if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 12.0.0)
if (HAS_USE_CTOR_HOMING)
# For more info see https://blog.llvm.org/posts/2021-04-05-constructor-homing-for-debug-info/
if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Xclang -fuse-ctor-homing")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Xclang -fuse-ctor-homing")
@ -192,7 +194,7 @@ endif ()
# Make sure the final executable has symbols exported
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic")
find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-12" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy")
find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-13" "llvm-objcopy-12" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy")
if (NOT OBJCOPY_PATH AND OS_DARWIN)
find_program (BREW_PATH NAMES "brew")
@ -379,7 +381,7 @@ if (COMPILER_CLANG)
endif ()
# Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled
find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8")
find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8")
if (LLVM_AR_PATH)
message(STATUS "Using llvm-ar: ${LLVM_AR_PATH}.")
@ -388,7 +390,7 @@ if (COMPILER_CLANG)
message(WARNING "Cannot find llvm-ar. System ar will be used instead. It does not work with ThinLTO.")
endif ()
find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8")
find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8")
if (LLVM_RANLIB_PATH)
message(STATUS "Using llvm-ranlib: ${LLVM_RANLIB_PATH}.")
@ -629,9 +631,6 @@ include_directories(${ConfigIncludePath})
# Add as many warnings as possible for our own code.
include (cmake/warnings.cmake)
# Check if needed compiler flags are supported
include (cmake/check_flags.cmake)
add_subdirectory (base)
add_subdirectory (src)
add_subdirectory (programs)

View File

@ -16,6 +16,10 @@ extern "C"
}
#endif
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
namespace
{

View File

@ -1,3 +1,7 @@
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
/// This code was based on the code by Fedor Korotkiy (prime@yandex-team.ru) for YT product in Yandex.
#include <common/defines.h>

View File

@ -1,6 +1,10 @@
#pragma once
#include <cstddef>
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
constexpr size_t KiB = 1024;
constexpr size_t MiB = 1024 * KiB;
constexpr size_t GiB = 1024 * MiB;

View File

@ -1,3 +1,7 @@
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
#include <daemon/BaseDaemon.h>
#include <daemon/SentryWriter.h>

View File

@ -49,6 +49,8 @@ if (NOT USE_INTERNAL_MYSQL_LIBRARY AND OPENSSL_INCLUDE_DIR)
target_include_directories (mysqlxx SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR})
endif ()
target_no_warning(mysqlxx reserved-macro-identifier)
if (NOT USE_INTERNAL_MYSQL_LIBRARY AND USE_STATIC_LIBRARIES)
message(WARNING "Statically linking with system mysql/mariadb only works "
"if mysql client libraries are built with same openssl version as "

View File

@ -79,7 +79,7 @@ PoolWithFailover PoolFactory::get(const Poco::Util::AbstractConfiguration & conf
std::lock_guard<std::mutex> lock(impl->mutex);
if (auto entry = impl->pools.find(config_name); entry != impl->pools.end())
{
return *(entry->second.get());
return *(entry->second);
}
else
{
@ -100,7 +100,7 @@ PoolWithFailover PoolFactory::get(const Poco::Util::AbstractConfiguration & conf
impl->pools.insert_or_assign(config_name, pool);
impl->pools_by_ids.insert_or_assign(entry_name, config_name);
}
return *(pool.get());
return *pool;
}
}

View File

@ -27,6 +27,10 @@
#define _PATH_TTY "/dev/tty"
#endif
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
#include <termios.h>
#include <signal.h>
#include <ctype.h>

View File

@ -6,7 +6,7 @@ if (ENABLE_CLANG_TIDY)
message(FATAL_ERROR "clang-tidy requires CMake version at least 3.6.")
endif()
find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-12" "clang-tidy-11" "clang-tidy-10" "clang-tidy-9" "clang-tidy-8")
find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-13" "clang-tidy-12" "clang-tidy-11" "clang-tidy-10" "clang-tidy-9" "clang-tidy-8")
if (CLANG_TIDY_PATH)
message(STATUS

View File

@ -4,3 +4,4 @@ include (CheckCCompilerFlag)
check_cxx_compiler_flag("-Wsuggest-destructor-override" HAS_SUGGEST_DESTRUCTOR_OVERRIDE)
check_cxx_compiler_flag("-Wshadow" HAS_SHADOW)
check_cxx_compiler_flag("-Wsuggest-override" HAS_SUGGEST_OVERRIDE)
check_cxx_compiler_flag("-Xclang -fuse-ctor-homing" HAS_USE_CTOR_HOMING)

View File

@ -1,6 +1,6 @@
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=13
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list

View File

@ -4,7 +4,7 @@ set -e
#ccache -s # uncomment to display CCache statistics
mkdir -p /server/build_docker
cd /server/build_docker
cmake -G Ninja /server "-DCMAKE_C_COMPILER=$(command -v clang-12)" "-DCMAKE_CXX_COMPILER=$(command -v clang++-12)"
cmake -G Ninja /server "-DCMAKE_C_COMPILER=$(command -v clang-13)" "-DCMAKE_CXX_COMPILER=$(command -v clang++-13)"
# Set the number of build jobs to the half of number of virtual CPU cores (rounded up).
# By default, ninja use all virtual CPU cores, that leads to very high memory consumption without much improvement in build time.

View File

@ -1,7 +1,7 @@
# docker build -t clickhouse/binary-builder .
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=13
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
@ -44,13 +44,11 @@ RUN apt-get update \
gdb \
git \
gperf \
libicu-dev \
libreadline-dev \
clang-12 \
clang-tidy-12 \
lld-12 \
llvm-12 \
llvm-12-dev \
clang-${LLVM_VERSION} \
clang-tidy-${LLVM_VERSION} \
lld-${LLVM_VERSION} \
llvm-${LLVM_VERSION} \
llvm-${LLVM_VERSION}-dev \
libicu-dev \
libreadline-dev \
moreutils \

View File

@ -1,7 +1,7 @@
# docker build -t clickhouse/deb-builder .
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=13
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list
@ -37,17 +37,17 @@ RUN curl -O https://clickhouse-datasets.s3.yandex.net/utils/1/dpkg-deb \
RUN apt-get update \
&& apt-get install \
alien \
clang-12 \
clang-tidy-12 \
clang-${LLVM_VERSION} \
clang-tidy-${LLVM_VERSION} \
cmake \
debhelper \
devscripts \
gdb \
git \
gperf \
lld-12 \
llvm-12 \
llvm-12-dev \
lld-${LLVM_VERSION} \
llvm-${LLVM_VERSION} \
llvm-${LLVM_VERSION}-dev \
moreutils \
ninja-build \
perl \

View File

@ -205,7 +205,8 @@ if __name__ == "__main__":
parser.add_argument("--build-type", choices=("debug", ""), default="")
parser.add_argument("--compiler", choices=("clang-11", "clang-11-darwin", "clang-11-darwin-aarch64", "clang-11-aarch64",
"clang-12", "clang-12-darwin", "clang-12-darwin-aarch64", "clang-12-aarch64",
"clang-11-freebsd", "clang-12-freebsd", "gcc-11"), default="clang-12")
"clang-13", "clang-13-darwin", "clang-13-darwin-aarch64", "clang-13-aarch64",
"clang-11-freebsd", "clang-12-freebsd", "clang-13-freebsd", "gcc-11"), default="clang-13")
parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="")
parser.add_argument("--unbundled", action="store_true")
parser.add_argument("--split-binary", action="store_true")

View File

@ -1,7 +1,7 @@
# docker build -t clickhouse/test-base .
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=13
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list

View File

@ -11,7 +11,7 @@ RUN apt-get update && apt-get --yes --allow-unauthenticated install clang-9 libl
# https://github.com/ClickHouse-Extras/woboq_codebrowser/commit/37e15eaf377b920acb0b48dbe82471be9203f76b
RUN git clone https://github.com/ClickHouse-Extras/woboq_codebrowser
RUN cd woboq_codebrowser && cmake . -DCMAKE_BUILD_TYPE=Release -DCMAKE_CXX_COMPILER=clang\+\+-12 -DCMAKE_C_COMPILER=clang-12 && make -j
RUN cd woboq_codebrowser && cmake . -DCMAKE_BUILD_TYPE=Release -DCMAKE_CXX_COMPILER=clang\+\+-13 -DCMAKE_C_COMPILER=clang-13 && make -j
ENV CODEGEN=/woboq_codebrowser/generator/codebrowser_generator
ENV CODEINDEX=/woboq_codebrowser/indexgenerator/codebrowser_indexgenerator
@ -24,7 +24,7 @@ ENV SHA=nosha
ENV DATA="data"
CMD mkdir -p $BUILD_DIRECTORY && cd $BUILD_DIRECTORY && \
cmake $SOURCE_DIRECTORY -DCMAKE_CXX_COMPILER=/usr/bin/clang\+\+-12 -DCMAKE_C_COMPILER=/usr/bin/clang-12 -DCMAKE_EXPORT_COMPILE_COMMANDS=ON -DENABLE_EMBEDDED_COMPILER=0 -DENABLE_S3=0 && \
cmake $SOURCE_DIRECTORY -DCMAKE_CXX_COMPILER=/usr/bin/clang\+\+-13 -DCMAKE_C_COMPILER=/usr/bin/clang-13 -DCMAKE_EXPORT_COMPILE_COMMANDS=ON -DENABLE_EMBEDDED_COMPILER=0 -DENABLE_S3=0 && \
mkdir -p $HTML_RESULT_DIRECTORY && \
$CODEGEN -b $BUILD_DIRECTORY -a -o $HTML_RESULT_DIRECTORY -p ClickHouse:$SOURCE_DIRECTORY:$SHA -d $DATA | ts '%Y-%m-%d %H:%M:%S' && \
cp -r $STATIC_DATA $HTML_RESULT_DIRECTORY/ &&\

View File

@ -80,7 +80,7 @@ LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-client --query "RENAM
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits"
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-client --query "SHOW TABLES FROM test"
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-test -j 8 --testname --shard --zookeeper --print-time --use-skip-list 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_result.txt
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-test -j 8 --testname --shard --zookeeper --print-time 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_result.txt
readarray -t FAILED_TESTS < <(awk '/FAIL|TIMEOUT|ERROR/ { print substr($3, 1, length($3)-1) }' "/test_result.txt")
@ -97,7 +97,7 @@ then
echo "Going to run again: ${FAILED_TESTS[*]}"
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-test --order=random --testname --shard --zookeeper --use-skip-list "${FAILED_TESTS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a /test_result.txt
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-test --order=random --testname --shard --zookeeper "${FAILED_TESTS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a /test_result.txt
else
echo "No failed tests"
fi

View File

@ -1,7 +1,7 @@
# docker build -t clickhouse/fasttest .
FROM ubuntu:20.04
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=12
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=13
RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list

View File

@ -9,7 +9,7 @@ trap 'kill $(jobs -pr) ||:' EXIT
stage=${stage:-}
# Compiler version, normally set by Dockerfile
export LLVM_VERSION=${LLVM_VERSION:-12}
export LLVM_VERSION=${LLVM_VERSION:-13}
# A variable to pass additional flags to CMake.
# Here we explicitly default it to nothing so that bash doesn't complain about
@ -262,153 +262,8 @@ function run_tests
start_server
TESTS_TO_SKIP=(
00105_shard_collations
00109_shard_totals_after_having
00110_external_sort
00302_http_compression
00417_kill_query
00436_convert_charset
00490_special_line_separators_and_characters_outside_of_bmp
00652_replicated_mutations_zookeeper
00682_empty_parts_merge
00701_rollup
00834_cancel_http_readonly_queries_on_client_close
00911_tautological_compare
# Hyperscan
00926_multimatch
00929_multi_match_edit_distance
01681_hyperscan_debug_assertion
02004_max_hyperscan_regex_length
01176_mysql_client_interactive # requires mysql client
01031_mutations_interpreter_and_context
01053_ssd_dictionary # this test mistakenly requires acces to /var/lib/clickhouse -- can't run this locally, disabled
01083_expressions_in_engine_arguments
01092_memory_profiler
01098_msgpack_format
01098_temporary_and_external_tables
01103_check_cpu_instructions_at_startup # avoid dependency on qemu -- invonvenient when running locally
01193_metadata_loading
01238_http_memory_tracking # max_memory_usage_for_user can interfere another queries running concurrently
01251_dict_is_in_infinite_loop
01259_dictionary_custom_settings_ddl
01268_dictionary_direct_layout
01280_ssd_complex_key_dictionary
01281_group_by_limit_memory_tracking # max_memory_usage_for_user can interfere another queries running concurrently
01318_encrypt # Depends on OpenSSL
01318_decrypt # Depends on OpenSSL
01663_aes_msan # Depends on OpenSSL
01667_aes_args_check # Depends on OpenSSL
01683_codec_encrypted # Depends on OpenSSL
01776_decrypt_aead_size_check # Depends on OpenSSL
01811_filter_by_null # Depends on OpenSSL
02012_sha512_fixedstring # Depends on OpenSSL
01281_unsucceeded_insert_select_queries_counter
01292_create_user
01294_lazy_database_concurrent
01305_replica_create_drop_zookeeper
01354_order_by_tuple_collate_const
01355_ilike
01411_bayesian_ab_testing
01798_uniq_theta_sketch
01799_long_uniq_theta_sketch
01890_stem # depends on libstemmer_c
02003_compress_bz2 # depends on bzip2
01059_storage_file_compression # depends on brotli and bzip2
collate
collation
_orc_
arrow
avro
base64
brotli
capnproto
client
ddl_dictionaries
h3
hashing
hdfs
java_hash
json
limit_memory
live_view
memory_leak
memory_limit
mysql
odbc
parallel_alter
parquet
protobuf
secure
sha256
xz
# Not sure why these two fail even in sequential mode. Disabled for now
# to make some progress.
00646_url_engine
00974_query_profiler
# In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default
01504_rocksdb
01686_rocksdb
# Look at DistributedFilesToInsert, so cannot run in parallel.
01460_DistributedFilesToInsert
01541_max_memory_usage_for_user_long
# Require python libraries like scipy, pandas and numpy
01322_ttest_scipy
01561_mann_whitney_scipy
01545_system_errors
# Checks system.errors
01563_distributed_query_finish
# nc - command not found
01601_proxy_protocol
01622_defaults_for_url_engine
# JSON functions
01666_blns
# Requires postgresql-client
01802_test_postgresql_protocol_with_row_policy
# Depends on AWS
01801_s3_cluster
02012_settings_clause_for_s3
# needs psql
01889_postgresql_protocol_null_fields
# needs pv
01923_network_receive_time_metric_insert
01889_sqlite_read_write
# needs s2
01849_geoToS2
01851_s2_to_geo
01852_s2_get_neighbours
01853_s2_cells_intersect
01854_s2_cap_contains
01854_s2_cap_union
# needs s3
01944_insert_partition_by
# depends on Go
02013_zlib_read_after_eof
# Accesses CH via mysql table function (which is unavailable)
01747_system_session_log_long
)
time clickhouse-test --hung-check -j 8 --order=random --use-skip-list \
--no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" \
time clickhouse-test --hung-check -j 8 --order=random \
--fast-tests-only --no-long --testname --shard --zookeeper \
-- "$FASTTEST_FOCUS" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee "$FASTTEST_OUTPUT/test_log.txt"

View File

@ -12,7 +12,7 @@ stage=${stage:-}
script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
echo "$script_dir"
repo_dir=ch
BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-12_debug_none_bundled_unsplitted_disable_False_binary"}
BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-13_debug_none_bundled_unsplitted_disable_False_binary"}
function clone
{

View File

@ -2,7 +2,7 @@
set -euo pipefail
CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-12_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse"}
CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-13_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse"}
CLICKHOUSE_REPO_PATH=${CLICKHOUSE_REPO_PATH:=""}

View File

@ -38,7 +38,7 @@ RUN set -x \
&& dpkg -i "${PKG_VERSION}.deb"
CMD echo "Running PVS version $PKG_VERSION" && cd /repo_folder && pvs-studio-analyzer credentials $LICENCE_NAME $LICENCE_KEY -o ./licence.lic \
&& cmake . -D"ENABLE_EMBEDDED_COMPILER"=OFF -D"USE_INTERNAL_PROTOBUF_LIBRARY"=OFF -D"USE_INTERNAL_GRPC_LIBRARY"=OFF -DCMAKE_C_COMPILER=clang-12 -DCMAKE_CXX_COMPILER=clang\+\+-12 \
&& cmake . -D"ENABLE_EMBEDDED_COMPILER"=OFF -D"USE_INTERNAL_PROTOBUF_LIBRARY"=OFF -D"USE_INTERNAL_GRPC_LIBRARY"=OFF -DCMAKE_C_COMPILER=clang-13 -DCMAKE_CXX_COMPILER=clang\+\+-13 \
&& ninja re2_st clickhouse_grpc_protos \
&& pvs-studio-analyzer analyze -o pvs-studio.log -e contrib -j 4 -l ./licence.lic; \
cp /repo_folder/pvs-studio.log /test_output; \

View File

@ -108,7 +108,7 @@ function run_tests()
ADDITIONAL_OPTIONS+=('--replicated-database')
fi
clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --use-skip-list --print-time "${ADDITIONAL_OPTIONS[@]}" \
clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \
"$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
}

View File

@ -97,7 +97,7 @@ function run_tests()
fi
clickhouse-test --testname --shard --zookeeper --hung-check --print-time \
--use-skip-list --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
--test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
}

View File

@ -13,8 +13,4 @@ dpkg -i package_folder/clickhouse-test_*.deb
service clickhouse-server start && sleep 5
if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then
SKIP_LIST_OPT="--use-skip-list"
fi
clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" "$ADDITIONAL_OPTIONS" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
clickhouse-test --testname --shard --zookeeper "$ADDITIONAL_OPTIONS" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt

View File

@ -10,14 +10,6 @@ import logging
import time
def get_skip_list_cmd(path):
with open(path, 'r') as f:
for line in f:
if '--use-skip-list' in line:
return '--use-skip-list'
return ''
def get_options(i):
options = []
client_options = []
@ -56,8 +48,6 @@ def get_options(i):
def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_time_limit):
skip_list_opt = get_skip_list_cmd(cmd)
global_time_limit_option = ''
if global_time_limit:
global_time_limit_option = "--global_time_limit={}".format(global_time_limit)
@ -66,7 +56,7 @@ def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_t
pipes = []
for i in range(0, len(output_paths)):
f = open(output_paths[i], 'w')
full_command = "{} {} {} {} {}".format(cmd, skip_list_opt, get_options(i), global_time_limit_option, skip_tests_option)
full_command = "{} {} {} {}".format(cmd, get_options(i), global_time_limit_option, skip_tests_option)
logging.info("Run func tests '%s'", full_command)
p = Popen(full_command, shell=True, stdout=f, stderr=f)
pipes.append(p)

View File

@ -23,7 +23,7 @@ $ sudo apt-get install git cmake python ninja-build
Or cmake3 instead of cmake on older systems.
### Install clang-12 (recommended) {#install-clang-12}
### Install clang-13 (recommended) {#install-clang-13}
On Ubuntu/Debian you can use the automatic installation script (check [official webpage](https://apt.llvm.org/))
@ -33,11 +33,11 @@ sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)"
For other Linux distribution - check the availability of the [prebuild packages](https://releases.llvm.org/download.html) or build clang [from sources](https://clang.llvm.org/get_started.html).
#### Use clang-12 for Builds
#### Use clang-13 for Builds
``` bash
$ export CC=clang-12
$ export CXX=clang++-12
$ export CC=clang-13
$ export CXX=clang++-13
```
Gcc can also be used though it is discouraged.

View File

@ -288,5 +288,7 @@ If the data in ZooKeeper was lost or damaged, you can save data by moving it to
- [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size)
- [background_fetches_pool_size](../../../operations/settings/settings.md#background_fetches_pool_size)
- [execute_merges_on_single_replica_time_threshold](../../../operations/settings/settings.md#execute-merges-on-single-replica-time-threshold)
- [max_replicated_fetches_network_bandwidth](../../../operations/settings/merge-tree-settings.md#max_replicated_fetches_network_bandwidth)
- [max_replicated_sends_network_bandwidth](../../../operations/settings/merge-tree-settings.md#max_replicated_sends_network_bandwidth)
[Original article](https://clickhouse.tech/docs/en/operations/table_engines/replication/) <!--hide-->

View File

@ -23,7 +23,6 @@ The supported formats are:
| [CustomSeparated](#format-customseparated) | ✔ | ✔ |
| [Values](#data-format-values) | ✔ | ✔ |
| [Vertical](#vertical) | ✗ | ✔ |
| [VerticalRaw](#verticalraw) | ✗ | ✔ |
| [JSON](#json) | ✗ | ✔ |
| [JSONAsString](#jsonasstring) | ✔ | ✗ |
| [JSONStrings](#jsonstrings) | ✗ | ✔ |
@ -60,6 +59,7 @@ The supported formats are:
| [LineAsString](#lineasstring) | ✔ | ✗ |
| [Regexp](#data-format-regexp) | ✔ | ✗ |
| [RawBLOB](#rawblob) | ✔ | ✔ |
| [MsgPack](#msgpack) | ✔ | ✔ |
You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](../operations/settings/settings.md) section.
@ -943,10 +943,6 @@ test: string with 'quotes' and with some special
This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table).
## VerticalRaw {#verticalraw}
Similar to [Vertical](#vertical), but with escaping disabled. This format is only suitable for outputting query results, not for parsing (receiving data and inserting it in the table).
## XML {#xml}
XML format is suitable only for output, not for parsing. Example:
@ -1551,4 +1547,31 @@ Result:
f9725a22f9191e064120d718e26862a9 -
```
[Original article](https://clickhouse.tech/docs/en/interfaces/formats/) <!--hide-->
## MsgPack {#msgpack}
ClickHouse supports reading and writing [MessagePack](https://msgpack.org/) data files.
### Data Types Matching {#data-types-matching-msgpack}
| MsgPack data type | ClickHouse data type |
|---------------------------------|----------------------------------------------------------------------------------|
| `uint N`, `positive fixint` | [UIntN](../sql-reference/data-types/int-uint.md) |
| `int N` | [IntN](../sql-reference/data-types/int-uint.md) |
| `fixstr`, `str 8`, `str 16`, `str 32` | [String](../sql-reference/data-types/string.md), [FixedString](../sql-reference/data-types/fixedstring.md) |
| `float 32` | [Float32](../sql-reference/data-types/float.md) |
| `float 64` | [Float64](../sql-reference/data-types/float.md) |
| `uint 16` | [Date](../sql-reference/data-types/date.md) |
| `uint 32` | [DateTime](../sql-reference/data-types/datetime.md) |
| `uint 64` | [DateTime64](../sql-reference/data-types/datetime.md) |
| `fixarray`, `array 16`, `array 32`| [Array](../sql-reference/data-types/array.md) |
| `nil` | [Nothing](../sql-reference/data-types/special-data-types/nothing.md) |
Example:
Writing to a file ".msgpk":
```sql
$ clickhouse-client --query="CREATE TABLE msgpack (array Array(UInt8)) ENGINE = Memory;"
$ clickhouse-client --query="INSERT INTO msgpack VALUES ([0, 1, 2, 3, 42, 253, 254, 255]), ([255, 254, 253, 42, 3, 2, 1, 0])";
$ clickhouse-client --query="SELECT * FROM msgpack FORMAT MsgPack" > tmp_msgpack.msgpk;
```

View File

@ -84,7 +84,7 @@ Features:
- Table data preview.
- Full-text search.
By default, DBeaver does not connect using a session (the CLI for example does). If you require session support (for example to set settings for your session), edit the driver connection properties and set session_id to a random string (it uses the http connection under the hood). Then you can use any setting from the query window
By default, DBeaver does not connect using a session (the CLI for example does). If you require session support (for example to set settings for your session), edit the driver connection properties and set `session_id` to a random string (it uses the http connection under the hood). Then you can use any setting from the query window.
### clickhouse-cli {#clickhouse-cli}
@ -113,6 +113,22 @@ Features:
[MindsDB](https://mindsdb.com/) is an open-source AI layer for databases including ClickHouse that allows you to effortlessly develop, train and deploy state-of-the-art machine learning models. MindsDB Studio(GUI) allows you to train new models from database, interpret predictions made by the model, identify potential data biases, and evaluate and visualize model accuracy using the Explainable AI function to adapt and tune your Machine Learning models faster.
### DBM {#dbm}
[DBM](https://dbm.incubator.edurt.io/) DBM is a visual management tool for ClickHouse!
Features:
- Support query history (pagination, clear all, etc.)
- Support selected sql clauses query
- Support terminating query
- Support table management (metadata, delete, preview)
- Support database management (delete, create)
- Support custom query
- Support multiple data sources management(connection test, monitoring)
- Support monitor (processor, connection, query)
- Support migrate data
## Commercial {#commercial}
### DataGrip {#datagrip}
@ -190,20 +206,4 @@ SeekTable is [free](https://www.seektable.com/help/cloud-pricing) for personal/i
[Chadmin](https://github.com/bun4uk/chadmin) is a simple UI where you can visualize your currently running queries on your ClickHouse cluster and info about them and kill them if you want.
### DBM {#dbm}
[DBM](https://dbm.incubator.edurt.io/) DBM is a visual management tool for ClickHouse!
Features:
- Support query history (pagination, clear all, etc.)
- Support selected sql clauses query
- Support terminating query
- Support table management (metadata, delete, preview)
- Support database management (delete, create)
- Support custom query
- Support multiple data sources management(connection test, monitoring)
- Support monitor (processor, connection, query)
- Support migrate data
[Original article](https://clickhouse.tech/docs/en/interfaces/third-party/gui/) <!--hide-->

View File

@ -3,58 +3,58 @@ toc_priority: 66
toc_title: ClickHouse Keeper
---
# [pre-production] clickhouse-keeper
# [pre-production] ClickHouse Keeper
ClickHouse server use [ZooKeeper](https://zookeeper.apache.org/) coordination system for data [replication](../engines/table-engines/mergetree-family/replication.md) and [distributed DDL](../sql-reference/distributed-ddl.md) queries execution. ClickHouse Keeper is an alternative coordination system compatible with ZooKeeper.
ClickHouse server uses [ZooKeeper](https://zookeeper.apache.org/) coordination system for data [replication](../engines/table-engines/mergetree-family/replication.md) and [distributed DDL](../sql-reference/distributed-ddl.md) queries execution. ClickHouse Keeper is an alternative coordination system compatible with ZooKeeper.
!!! warning "Warning"
This feature currently in pre-production stage. We test it in our CI and on small internal installations.
This feature is currently in the pre-production stage. We test it in our CI and on small internal installations.
## Implementation details
ZooKeeper is one of the first well-known open-source coordination systems. It's implemented in Java, has quite a simple and powerful data model. ZooKeeper's coordination algorithm called ZAB (ZooKeeper Atomic Broadcast) doesn't provide linearizability guarantees for reads, because each ZooKeeper node serves reads locally. Unlike ZooKeeper `clickhouse-keeper` written in C++ and use [RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft). This algorithm allows to have linearizability for reads and writes, has several open-source implementations in different languages.
ZooKeeper is one of the first well-known open-source coordination systems. It's implemented in Java, has quite a simple and powerful data model. ZooKeeper's coordination algorithm called ZAB (ZooKeeper Atomic Broadcast) doesn't provide linearizability guarantees for reads, because each ZooKeeper node serves reads locally. Unlike ZooKeeper ClickHouse Keeper is written in C++ and uses [RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft). This algorithm allows to have linearizability for reads and writes, has several open-source implementations in different languages.
By default, `clickhouse-keeper` provides the same guarantees as ZooKeeper (linearizable writes, non-linearizable reads). It has a compatible client-server protocol, so any standard ZooKeeper client can be used to interact with `clickhouse-keeper`. Snapshots and logs have an incompatible format with ZooKeeper, but `clickhouse-keeper-converter` tool allows to convert ZooKeeper data to `clickhouse-keeper` snapshot. Interserver protocol in `clickhouse-keeper` also incompatible with ZooKeeper so mixed ZooKeeper/clickhouse-keeper cluster is impossible.
By default, ClickHouse Keeper provides the same guarantees as ZooKeeper (linearizable writes, non-linearizable reads). It has a compatible client-server protocol, so any standard ZooKeeper client can be used to interact with ClickHouse Keeper. Snapshots and logs have an incompatible format with ZooKeeper, but `clickhouse-keeper-converter` tool allows to convert ZooKeeper data to ClickHouse Keeper snapshot. Interserver protocol in ClickHouse Keeper is also incompatible with ZooKeeper so mixed ZooKeeper / ClickHouse Keeper cluster is impossible.
## Configuration
`clickhouse-keeper` can be used as a standalone replacement for ZooKeeper or as an internal part of the `clickhouse-server`, but in both cases configuration is almost the same `.xml` file. The main `clickhouse-keeper` configuration tag is `<keeper_server>`. Keeper configuration has the following parameters:
ClickHouse Keeper can be used as a standalone replacement for ZooKeeper or as an internal part of the ClickHouse server, but in both cases configuration is almost the same `.xml` file. The main ClickHouse Keeper configuration tag is `<keeper_server>`. Keeper configuration has the following parameters:
- `tcp_port`the port for a client to connect (default for ZooKeeper is `2181`)
- `tcp_port_secure`the secure port for a client to connect
- `server_id`unique server id, each participant of the clickhouse-keeper cluster must have a unique number (1, 2, 3, and so on)
- `log_storage_path`path to coordination logs, better to store logs on the non-busy device (same for ZooKeeper)
- `snapshot_storage_path`path to coordination snapshots
- `tcp_port`Port for a client to connect (default for ZooKeeper is `2181`).
- `tcp_port_secure`Secure port for a client to connect.
- `server_id`Unique server id, each participant of the ClickHouse Keeper cluster must have a unique number (1, 2, 3, and so on).
- `log_storage_path`Path to coordination logs, better to store logs on the non-busy device (same for ZooKeeper).
- `snapshot_storage_path`Path to coordination snapshots.
Other common parameters are inherited from clickhouse-server config (`listen_host`, `logger` and so on).
Other common parameters are inherited from the ClickHouse server config (`listen_host`, `logger`, and so on).
Internal coordination settings are located in `<keeper_server>.<coordination_settings>` section:
- `operation_timeout_ms`timeout for a single client operation (default: 10000)
- `session_timeout_ms`timeout for client session (default: 30000)
- `dead_session_check_period_ms`how often clickhouse-keeper check dead sessions and remove them (default: 500)
- `heart_beat_interval_ms`how often a clickhouse-keeper leader will send heartbeats to followers (default: 500)
- `election_timeout_lower_bound_ms`if follower didn't receive heartbeats from the leader in this interval, then it can initiate leader election (default: 1000)
- `election_timeout_upper_bound_ms`if follower didn't receive heartbeats from the leader in this interval, then it must initiate leader election (default: 2000)
- `rotate_log_storage_interval`how many log records to store in a single file (default: 100000)
- `reserved_log_items`how many coordination log records to store before compaction (default: 100000)
- `snapshot_distance`how often clickhouse-keeper will create new snapshots (in the number of records in logs) (default: 100000)
- `snapshots_to_keep`how many snapshots to keep (default: 3)
- `stale_log_gap`the threshold when leader consider follower as stale and send snapshot to it instead of logs (default: 10000)
- `fresh_log_gap` - when node became fresh (default: 200)
- `max_requests_batch_size` - max size of batch in requests count before it will be sent to RAFT (default: 100)
- `force_sync`call `fsync` on each write to coordination log (default: true)
- `quorum_reads` - execute read requests as writes through whole RAFT consesus with similar speed (default: false)
- `raft_logs_level`text logging level about coordination (trace, debug, and so on) (default: system default)
- `auto_forwarding` - allow to forward write requests from followers to leader (default: true)
- `shutdown_timeout`wait to finish internal connections and shutdown (ms) (default: 5000)
- `startup_timeout`if the server doesn't connect to other quorum participants in the specified timeout it will terminate (ms) (default: 30000)
- `operation_timeout_ms`Timeout for a single client operation (ms) (default: 10000).
- `session_timeout_ms`Timeout for client session (ms) (default: 30000).
- `dead_session_check_period_ms`How often ClickHouse Keeper check dead sessions and remove them (ms) (default: 500).
- `heart_beat_interval_ms`How often a ClickHouse Keeper leader will send heartbeats to followers (ms) (default: 500).
- `election_timeout_lower_bound_ms`If the follower didn't receive heartbeats from the leader in this interval, then it can initiate leader election (default: 1000).
- `election_timeout_upper_bound_ms`If the follower didn't receive heartbeats from the leader in this interval, then it must initiate leader election (default: 2000).
- `rotate_log_storage_interval`How many log records to store in a single file (default: 100000).
- `reserved_log_items`How many coordination log records to store before compaction (default: 100000).
- `snapshot_distance`How often ClickHouse Keeper will create new snapshots (in the number of records in logs) (default: 100000).
- `snapshots_to_keep`How many snapshots to keep (default: 3).
- `stale_log_gap`Threshold when leader considers follower as stale and sends the snapshot to it instead of logs (default: 10000).
- `fresh_log_gap` — When node became fresh (default: 200).
- `max_requests_batch_size` - Max size of batch in requests count before it will be sent to RAFT (default: 100).
- `force_sync`Call `fsync` on each write to coordination log (default: true).
- `quorum_reads` — Execute read requests as writes through whole RAFT consensus with similar speed (default: false).
- `raft_logs_level`Text logging level about coordination (trace, debug, and so on) (default: system default).
- `auto_forwarding` — Allow to forward write requests from followers to the leader (default: true).
- `shutdown_timeout`Wait to finish internal connections and shutdown (ms) (default: 5000).
- `startup_timeout`If the server doesn't connect to other quorum participants in the specified timeout it will terminate (ms) (default: 30000).
Quorum configuration is located in `<keeper_server>.<raft_configuration>` section and contain servers description. The only parameter for the whole quorum is `secure`, which enables encrypted connection for communication between quorum participants. The main parameters for each `<server>` are:
- `id`server_id in quorum
- `hostname`hostname where this server placed
- `port`port where this server listen for connections
- `id`Server identifier in a quorum.
- `hostname`Hostname where this server is placed.
- `port`Port where this server listens for connections.
Examples of configuration for quorum with three nodes can be found in [integration tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/integration) with `test_keeper_` prefix. Example configuration for server #1:
@ -94,7 +94,7 @@ Examples of configuration for quorum with three nodes can be found in [integrati
## How to run
`clickhouse-keeper` is bundled into `clickhouse-server` package, just add configuration of `<keeper_server>` and start clickhouse-server as always. If you want to run standalone `clickhouse-keeper` you can start it in a similar way with:
ClickHouse Keeper is bundled into the ClickHouse server package, just add configuration of `<keeper_server>` and start ClickHouse server as always. If you want to run standalone ClickHouse Keeper you can start it in a similar way with:
```bash
clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon
@ -102,17 +102,18 @@ clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon
## [experimental] Migration from ZooKeeper
Seamlessly migration from ZooKeeper to `clickhouse-keeper` is impossible you have to stop your ZooKeeper cluster, convert data and start `clickhouse-keeper`. `clickhouse-keeper-converter` tool allows to convert ZooKeeper logs and snapshots to `clickhouse-keeper` snapshot. It works only with ZooKeeper > 3.4. Steps for migration:
Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration:
1. Stop all ZooKeeper nodes.
2. [optional, but recommended] Found ZooKeeper leader node, start and stop it again. It will force ZooKeeper to create consistent snapshot.
2. Optional, but recommended: find ZooKeeper leader node, start and stop it again. It will force ZooKeeper to create a consistent snapshot.
3. Run `clickhouse-keeper-converter` on leader, example
3. Run `clickhouse-keeper-converter` on a leader, for example:
```bash
clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 --zookeeper-snapshots-dir /var/lib/zookeeper/version-2 --output-dir /path/to/clickhouse/keeper/snapshots
```
4. Copy snapshot to `clickhouse-server` nodes with configured `keeper` or start `clickhouse-keeper` instead of ZooKeeper. Snapshot must persist on all nodes, otherwise empty nodes can be faster and one of them can becamse leader.
4. Copy snapshot to ClickHouse server nodes with a configured `keeper` or start ClickHouse Keeper instead of ZooKeeper. The snapshot must persist on all nodes, otherwise, empty nodes can be faster and one of them can become a leader.
[Original article](https://clickhouse.tech/docs/en/operations/clickhouse-keeper/) <!--hide-->

View File

@ -69,29 +69,85 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression.
</compression>
```
<!--
## encryption {#server-settings-encryption}
Configures a command to obtain a key to be used by [encryption codecs](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). The command, or a shell script, is expected to write a Base64-encoded key of any length to the stdout.
Configures a command to obtain a key to be used by [encryption codecs](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). Key (or keys) should be written in enviroment variables or be set in configuration file.
Keys can be hex or string. Their length must be equal to 16.
**Example**
For Linux with systemd:
Load from config:
```xml
<encryption>
<key_command>/usr/bin/systemd-ask-password --id="clickhouse-server" --timeout=0 "Enter the ClickHouse encryption passphrase:" | base64</key_command>
</encryption>
<encryption_codecs>
<aes_128_gcm_siv>
<key>12345567812345678</key>
</aes_128_gcm_siv>
</encryption_codecs>
```
For other systems:
!!! note "NOTE"
Storing keys in configuration file is not recommended. It isn't secure. You can move the keys into a separate config file on a secure disk and put a symlink to that config file to `config.d/` folder.
Load from config, when key is in hex:
```xml
<encryption>
<key_command><![CDATA[IFS=; echo -n >/dev/tty "Enter the ClickHouse encryption passphrase: "; stty=`stty -F /dev/tty -g`; stty -F /dev/tty -echo; read k </dev/tty; stty -F /dev/tty "$stty"; echo -n $k | base64]]></key_command>
</encryption>
<encryption_codecs>
<aes_128_gcm_siv>
<key_hex>00112233445566778899aabbccddeeff</key_hex>
</aes_128_gcm_siv>
</encryption_codecs>
```
-->
Load key from environment variable:
```xml
<encryption_codecs>
<aes_128_gcm_siv>
<key_hex from_env="KEY"></key_hex>
</aes_128_gcm_siv>
</encryption_codecs>
```
Where current_key_id sets the current key for encryption, and all specified keys can be used for decryption.
All this methods can be applied for multiple keys:
```xml
<encryption_codecs>
<aes_128_gcm_siv>
<key_hex id="0">00112233445566778899aabbccddeeff</key_hex>
<key_hex id="1" from_env=".."></key_hex>
<current_key_id>1</current_key_id>
</aes_128_gcm_siv>
</encryption_codecs>
```
Where `current_key_id` shows current key for encryption.
Also user can add nonce that must be 12 bytes long (by default encryption and decryption will use nonce consisting of zero bytes):
```xml
<encryption_codecs>
<aes_128_gcm_siv>
<nonce>0123456789101</nonce>
</aes_128_gcm_siv>
</encryption_codecs>
```
Or it can be set in hex:
```xml
<encryption_codecs>
<aes_128_gcm_siv>
<nonce_hex>abcdefabcdef</nonce_hex>
</aes_128_gcm_siv>
</encryption_codecs>
```
Everything above can be applied for `aes_256_gcm_siv` (but key must be 32 bytes length).
## custom_settings_prefixes {#custom_settings_prefixes}
List of prefixes for [custom settings](../../operations/settings/index.md#custom_settings). The prefixes must be separated with commas.

View File

@ -181,6 +181,44 @@ Possible values:
Default value: 0.
## max_replicated_fetches_network_bandwidth {#max_replicated_fetches_network_bandwidth}
Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) fetches. This setting is applied to a particular table, unlike the [max_replicated_fetches_network_bandwidth_for_server](settings.md#max_replicated_fetches_network_bandwidth_for_server) setting, which is applied to the server.
You can limit both server network and network for a particular table, but for this the value of the table-level setting should be less than server-level one. Otherwise the server considers only the `max_replicated_fetches_network_bandwidth_for_server` setting.
The setting isn't followed perfectly accurately.
Possible values:
- Positive integer.
- 0 — Unlimited.
Default value: `0`.
**Usage**
Could be used for throttling speed when replicating data to add or replace new nodes.
## max_replicated_sends_network_bandwidth {#max_replicated_sends_network_bandwidth}
Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) sends. This setting is applied to a particular table, unlike the [max_replicated_sends_network_bandwidth_for_server](settings.md#max_replicated_sends_network_bandwidth_for_server) setting, which is applied to the server.
You can limit both server network and network for a particular table, but for this the value of the table-level setting should be less than server-level one. Otherwise the server considers only the `max_replicated_sends_network_bandwidth_for_server` setting.
The setting isn't followed perfectly accurately.
Possible values:
- Positive integer.
- 0 — Unlimited.
Default value: `0`.
**Usage**
Could be used for throttling speed when replicating data to add or replace new nodes.
## old_parts_lifetime {#old-parts-lifetime}
The time (in seconds) of storing inactive parts to protect against data loss during spontaneous server reboots.

View File

@ -1140,6 +1140,40 @@ Possible values:
Default value: `5`.
## max_replicated_fetches_network_bandwidth_for_server {#max_replicated_fetches_network_bandwidth_for_server}
Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) fetches for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_fetches_network_bandwidth](../../operations/settings/merge-tree-settings.md#max_replicated_fetches_network_bandwidth) setting.
The setting isn't followed perfectly accurately.
Possible values:
- Positive integer.
- 0 — Unlimited.
Default value: `0`.
**Usage**
Could be used for throttling speed when replicating the data to add or replace new nodes.
## max_replicated_sends_network_bandwidth_for_server {#max_replicated_sends_network_bandwidth_for_server}
Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) sends for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_sends_network_bandwidth](../../operations/settings/merge-tree-settings.md#max_replicated_sends_network_bandwidth) setting.
The setting isn't followed perfectly accurately.
Possible values:
- Positive integer.
- 0 — Unlimited.
Default value: `0`.
**Usage**
Could be used for throttling speed when replicating the data to add or replace new nodes.
## connect_timeout_with_failover_ms {#connect-timeout-with-failover-ms}
The timeout in milliseconds for connecting to a remote server for a Distributed table engine, if the shard and replica sections are used in the cluster definition.
@ -3532,3 +3566,91 @@ Possible values:
- Positive integer.
Default value: `1000`.
## max_hyperscan_regexp_length {#max-hyperscan-regexp-length}
Defines the maximum length for each regular expression in the [hyperscan multi-match functions](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn).
Possible values:
- Positive integer.
- 0 - The length is not limited.
Default value: `0`.
**Example**
Query:
```sql
SELECT multiMatchAny('abcd', ['ab','bcd','c','d']) SETTINGS max_hyperscan_regexp_length = 3;
```
Result:
```text
┌─multiMatchAny('abcd', ['ab', 'bcd', 'c', 'd'])─┐
│ 1 │
└────────────────────────────────────────────────┘
```
Query:
```sql
SELECT multiMatchAny('abcd', ['ab','bcd','c','d']) SETTINGS max_hyperscan_regexp_length = 2;
```
Result:
```text
Exception: Regexp length too large.
```
**See Also**
- [max_hyperscan_regexp_total_length](#max-hyperscan-regexp-total-length)
## max_hyperscan_regexp_total_length {#max-hyperscan-regexp-total-length}
Sets the maximum length total of all regular expressions in each [hyperscan multi-match function](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn).
Possible values:
- Positive integer.
- 0 - The length is not limited.
Default value: `0`.
**Example**
Query:
```sql
SELECT multiMatchAny('abcd', ['a','b','c','d']) SETTINGS max_hyperscan_regexp_total_length = 5;
```
Result:
```text
┌─multiMatchAny('abcd', ['a', 'b', 'c', 'd'])─┐
│ 1 │
└─────────────────────────────────────────────┘
```
Query:
```sql
SELECT multiMatchAny('abcd', ['ab','bc','c','d']) SETTINGS max_hyperscan_regexp_total_length = 5;
```
Result:
```text
Exception: Total regexp lengths too large.
```
**See Also**
- [max_hyperscan_regexp_length](#max-hyperscan-regexp-length)

View File

@ -25,14 +25,13 @@ Aliases:
- `Int32``INT`, `INT4`, `INTEGER`.
- `Int64``BIGINT`.
## Uint Ranges {#uint-ranges}
## UInt Ranges {#uint-ranges}
- `UInt8` — \[0 : 255\]
- `UInt16` — \[0 : 65535\]
- `UInt32` — \[0 : 4294967295\]
- `UInt64` — \[0 : 18446744073709551615\]
- `UInt128` — \[0 : 340282366920938463463374607431768211455\]
- `UInt256` — \[0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935\]
`UInt128` is not supported yet.
[Original article](https://clickhouse.tech/docs/en/data_types/int_uint/) <!--hide-->

View File

@ -175,7 +175,10 @@ Example: The table contains discounts for each advertiser in the format:
+---------|-------------|-------------|------+
```
To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). These elements must contain elements `name` and`type` (if `type` is not specified, the default type will be used - Date). `type` can be any numeric type (Date / DateTime / UInt64 / Int32 / others).
To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). These elements must contain elements `name` and `type` (if `type` is not specified, the default type will be used - Date). `type` can be any numeric type (Date / DateTime / UInt64 / Int32 / others).
!!! warning "Warning"
Values of `range_min` and `range_max` should fit in `Int64` type.
Example:
@ -226,34 +229,33 @@ Configuration example:
``` xml
<yandex>
<dictionary>
<dictionary>
...
...
<layout>
<range_hashed />
</layout>
<layout>
<range_hashed />
</layout>
<structure>
<id>
<name>Abcdef</name>
</id>
<range_min>
<name>StartTimeStamp</name>
<type>UInt64</type>
</range_min>
<range_max>
<name>EndTimeStamp</name>
<type>UInt64</type>
</range_max>
<attribute>
<name>XXXType</name>
<type>String</type>
<null_value />
</attribute>
</structure>
<structure>
<id>
<name>Abcdef</name>
</id>
<range_min>
<name>StartTimeStamp</name>
<type>UInt64</type>
</range_min>
<range_max>
<name>EndTimeStamp</name>
<type>UInt64</type>
</range_max>
<attribute>
<name>XXXType</name>
<type>String</type>
<null_value />
</attribute>
</structure>
</dictionary>
</dictionary>
</yandex>
```

View File

@ -29,7 +29,7 @@ Returns the round number with largest absolute value that has an absolute value
Rounds a value to a specified number of decimal places.
The function returns the nearest number of the specified order. In case when given number has equal distance to surrounding numbers, the function uses bankers rounding for float number types and rounds away from zero for the other number types.
The function returns the nearest number of the specified order. In case when given number has equal distance to surrounding numbers, the function uses bankers rounding for float number types and rounds away from zero for the other number types (Decimal).
``` sql
round(expression [, decimal_places])
@ -49,7 +49,7 @@ The rounded number of the same type as the input number.
### Examples {#examples}
**Example of use**
**Example of use with Float**
``` sql
SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3
@ -63,6 +63,20 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3
└─────┴──────────────────────────┘
```
**Example of use with Decimal**
``` sql
SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3
```
``` text
┌──────x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐
│ 0.0000 │ 0.0000 │
│ 0.5000 │ 1.0000 │
│ 1.0000 │ 1.0000 │
└────────┴──────────────────────────────────────────────────┘
```
**Examples of rounding**
Rounding to the nearest number.

View File

@ -26,7 +26,6 @@ aの結果 `SELECT`、および実行する `INSERT`ファイルバックアッ
| [カスタム区切り](#format-customseparated) | ✔ | ✔ |
| [](#data-format-values) | ✔ | ✔ |
| [垂直](#vertical) | ✗ | ✔ |
| [VerticalRaw](#verticalraw) | ✗ | ✔ |
| [JSON](#json) | ✗ | ✔ |
| [JSONCompact](#jsoncompact) | ✗ | ✔ |
| [JSONEachRow](#jsoneachrow) | ✔ | ✔ |
@ -819,10 +818,6 @@ test: string with 'quotes' and with some special
この形式は、クエリ結果の出力にのみ適していますが、解析(テーブルに挿入するデータの取得)には適していません。
## VerticalRaw {#verticalraw}
に類似した [垂直](#vertical) しかし、エスケープ無効で。 この形式は、クエリ結果の出力にのみ適しており、解析(データの受信とテーブルへの挿入)には適していません。
## XML {#xml}
XML形式は出力にのみ適しており、解析には適していません。 例:

View File

@ -15,7 +15,7 @@ toc_title: PostgreSQL
``` sql
CREATE DATABASE test_database
ENGINE = PostgreSQL('host:port', 'database', 'user', 'password'[, `use_table_cache`]);
ENGINE = PostgreSQL('host:port', 'database', 'user', 'password'[, `schema`, `use_table_cache`]);
```
**Параметры движка**
@ -24,6 +24,7 @@ ENGINE = PostgreSQL('host:port', 'database', 'user', 'password'[, `use_table_cac
- `database` — имя удаленной БД.
- `user` — пользователь PostgreSQL.
- `password` — пароль пользователя.
- `schema` — схема PostgreSQL.
- `use_table_cache` — определяет кеширование структуры таблиц БД. Необязательный параметр. Значение по умолчанию: `0`.
## Поддерживаемые типы данных {#data_types-support}

View File

@ -253,4 +253,5 @@ $ sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data
- [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size)
- [background_fetches_pool_size](../../../operations/settings/settings.md#background_fetches_pool_size)
- [execute_merges_on_single_replica_time_threshold](../../../operations/settings/settings.md#execute-merges-on-single-replica-time-threshold)
- [max_replicated_fetches_network_bandwidth](../../../operations/settings/merge-tree-settings.md#max_replicated_fetches_network_bandwidth)
- [max_replicated_sends_network_bandwidth](../../../operations/settings/merge-tree-settings.md#max_replicated_sends_network_bandwidth)

View File

@ -22,7 +22,6 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT
| [CustomSeparated](#format-customseparated) | ✔ | ✔ |
| [Values](#data-format-values) | ✔ | ✔ |
| [Vertical](#vertical) | ✗ | ✔ |
| [VerticalRaw](#verticalraw) | ✗ | ✔ |
| [JSON](#json) | ✗ | ✔ |
| [JSONAsString](#jsonasstring) | ✔ | ✗ |
| [JSONStrings](#jsonstrings) | ✗ | ✔ |
@ -59,6 +58,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT
| [LineAsString](#lineasstring) | ✔ | ✗ |
| [Regexp](#data-format-regexp) | ✔ | ✗ |
| [RawBLOB](#rawblob) | ✔ | ✔ |
| [MsgPack](#msgpack) | ✔ | ✔ |
Вы можете регулировать некоторые параметры работы с форматами с помощью настроек ClickHouse. За дополнительной информацией обращайтесь к разделу [Настройки](../operations/settings/settings.md).
@ -915,10 +915,6 @@ test: string with 'quotes' and with some special
Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу).
## VerticalRaw {#verticalraw}
Аналогичен [Vertical](#vertical), но с отключенным выходом. Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу).
## XML {#xml}
Формат XML подходит только для вывода данных, не для парсинга. Пример:
@ -1464,3 +1460,32 @@ $ clickhouse-client --query "SELECT * FROM {some_table} FORMAT RawBLOB" | md5sum
``` text
f9725a22f9191e064120d718e26862a9 -
```
## MsgPack {#msgpack}
ClickHouse поддерживает запись и чтение из файлов в формате [MessagePack](https://msgpack.org/).
### Соответствие типов данных {#data-types-matching-msgpack}
| Тип данных MsgPack | Тип данных ClickHouse |
|---------------------------------|------------------------------------------------------------------------------------|
| `uint N`, `positive fixint` | [UIntN](../sql-reference/data-types/int-uint.md) |
| `int N` | [IntN](../sql-reference/data-types/int-uint.md) |
| `fixstr`, `str 8`, `str 16`, `str 32` | [String](../sql-reference/data-types/string.md), [FixedString](../sql-reference/data-types/fixedstring.md) |
| `float 32` | [Float32](../sql-reference/data-types/float.md) |
| `float 64` | [Float64](../sql-reference/data-types/float.md) |
| `uint 16` | [Date](../sql-reference/data-types/date.md) |
| `uint 32` | [DateTime](../sql-reference/data-types/datetime.md) |
| `uint 64` | [DateTime64](../sql-reference/data-types/datetime.md) |
| `fixarray`, `array 16`, `array 32`| [Array](../sql-reference/data-types/array.md) |
| `nil` | [Nothing](../sql-reference/data-types/special-data-types/nothing.md) |
Пример:
Запись в файл ".msgpk":
```sql
$ clickhouse-client --query="CREATE TABLE msgpack (array Array(UInt8)) ENGINE = Memory;"
$ clickhouse-client --query="INSERT INTO msgpack VALUES ([0, 1, 2, 3, 42, 253, 254, 255]), ([255, 254, 253, 42, 3, 2, 1, 0])";
$ clickhouse-client --query="SELECT * FROM msgpack FORMAT MsgPack" > tmp_msgpack.msgpk;
```

View File

@ -75,11 +75,13 @@ toc_title: "Визуальные интерфейсы от сторонних р
Основные возможности:
- Построение запросов с подсветкой синтаксиса;
- Просмотр таблиц;
- Автодополнение команд;
- Построение запросов с подсветкой синтаксиса.
- Просмотр таблиц.
- Автодополнение команд.
- Полнотекстовый поиск.
По умолчанию DBeaver не использует сессии при подключении (в отличие от CLI, например). Если вам нужна поддержка сессий (например, для установки настроек на сессию), измените настройки подключения драйвера и укажите для настройки `session_id` любое произвольное значение (драйвер использует подключение по http). После этого вы можете использовать любую настройку (setting) в окне запроса.
### clickhouse-cli {#clickhouse-cli}
[clickhouse-cli](https://github.com/hatarist/clickhouse-cli) - это альтернативный клиент командной строки для ClickHouse, написанный на Python 3.

View File

@ -0,0 +1,119 @@
---
toc_priority: 66
toc_title: ClickHouse Keeper
---
# [пре-продакшн] ClickHouse Keeper
Сервер ClickHouse использует сервис координации [ZooKeeper](https://zookeeper.apache.org/) для [репликации](../engines/table-engines/mergetree-family/replication.md) данных и выполнения [распределенных DDL запросов](../sql-reference/distributed-ddl.md). ClickHouse Keeper — это альтернативный сервис координации, совместимый с ZooKeeper.
!!! warning "Предупреждение"
ClickHouse Keeper находится в стадии пре-продакшн и тестируется в CI ClickHouse и на нескольких внутренних инсталляциях.
## Детали реализации
ZooKeeper — один из первых широко известных сервисов координации с открытым исходным кодом. Он реализован на языке программирования Java, имеет достаточно простую и мощную модель данных. Алгоритм координации Zookeeper называется ZAB (ZooKeeper Atomic Broadcast). Он не гарантирует линеаризуемость операций чтения, поскольку каждый узел ZooKeeper обслуживает чтения локально. В отличие от ZooKeeper, ClickHouse Keeper реализован на C++ и использует алгоритм [RAFT](https://raft.github.io/), [реализация](https://github.com/eBay/NuRaft). Этот алгоритм позволяет достичь линеаризуемости чтения и записи, имеет несколько реализаций с открытым исходным кодом на разных языках.
По умолчанию ClickHouse Keeper предоставляет те же гарантии, что и ZooKeeper (линеаризуемость записей, последовательная согласованность чтений). У него есть совместимый клиент-серверный протокол, поэтому любой стандартный клиент ZooKeeper может использоваться для взаимодействия с ClickHouse Keeper. Снэпшоты и журналы имеют несовместимый с ZooKeeper формат, однако можно конвертировать данные Zookeeper в снэпшот ClickHouse Keeper с помощью `clickhouse-keeper-converter`. Межсерверный протокол ClickHouse Keeper также несовместим с ZooKeeper, поэтому создание смешанного кластера ZooKeeper / ClickHouse Keeper невозможно.
## Конфигурация
ClickHouse Keeper может использоваться как равноценная замена ZooKeeper или как внутренняя часть сервера ClickHouse, но в обоих случаях конфигурация представлена файлом `.xml`. Главный тег конфигурации ClickHouse Keeper — это `<keeper_server>`. Параметры конфигурации:
- `tcp_port` — порт для подключения клиента (по умолчанию для ZooKeeper: `2181`).
- `tcp_port_secure` — зашифрованный порт для подключения клиента.
- `server_id` — уникальный идентификатор сервера, каждый участник кластера должен иметь уникальный номер&nbsp;(1,&nbsp;2,&nbsp;3&nbsp;и&nbsp;т.&nbsp;д.).
- `log_storage_path` — путь к журналам координации, лучше хранить их на незанятом устройстве (актуально и для ZooKeeper).
- `snapshot_storage_path` — путь к снэпшотам координации.
Другие общие параметры наследуются из конфигурации сервера ClickHouse (`listen_host`, `logger`, и т. д.).
Настройки внутренней координации находятся в `<keeper_server>.<coordination_settings>`:
- `operation_timeout_ms` — максимальное время ожидания для одной клиентской операции в миллисекундах (по умолчанию: 10000).
- `session_timeout_ms` — максимальное время ожидания для клиентской сессии в миллисекундах (по умолчанию: 30000).
- `dead_session_check_period_ms` — частота, с которой ClickHouse Keeper проверяет мертвые сессии и удаляет их, в миллисекундах (по умолчанию: 500).
- `heart_beat_interval_ms` — частота, с которой узел-лидер ClickHouse Keeper отправляет хартбиты узлам-последователям, в миллисекундах (по умолчанию: 500).
- `election_timeout_lower_bound_ms` — время, после которого последователь может инициировать выборы лидера, если не получил от него сердцебиения (по умолчанию: 1000).
- `election_timeout_upper_bound_ms` — время, после которого последователь должен инициировать выборы лидера, если не получил от него сердцебиения (по умолчанию: 2000).
- `rotate_log_storage_interval` — количество записей в журнале координации для хранения в одном файле (по умолчанию: 100000).
- `reserved_log_items` — минимальное количество записей в журнале координации которые нужно сохранять после снятия снепшота (по умолчанию: 100000).
- `snapshot_distance` — частота, с которой ClickHouse Keeper делает новые снэпшоты (по количеству записей в журналах), в миллисекундах (по умолчанию: 100000).
- `snapshots_to_keep` — количество снэпшотов для сохранения (по умолчанию: 3).
- `stale_log_gap` — время, после которого лидер считает последователя устаревшим и отправляет ему снэпшот вместо журналов (по умолчанию: 10000).
- `fresh_log_gap` — максимальное отставание от лидера в количестве записей журнала после которого последователь считает себя не отстающим (по умолчанию: 200).
- `max_requests_batch_size` — количество запросов на запись, которые будут сгруппированы в один перед отправкой через RAFT (по умолчанию: 100).
- `force_sync` — вызывать `fsync` при каждой записи в журнал координации (по умолчанию: true).
- `quorum_reads` — выполнять запросы чтения аналогично запросам записи через весь консенсус RAFT с негативным эффектом на производительность и размер журналов (по умолчанию: false).
- `raft_logs_level` — уровень логгирования сообщений в текстовый лог (trace, debug и т. д.) (по умолчанию: information).
- `auto_forwarding` — разрешить пересылку запросов на запись от последователей лидеру (по умолчанию: true).
- `shutdown_timeout` — время ожидания завершения внутренних подключений и выключения, в миллисекундах (по умолчанию: 5000).
- `startup_timeout` — время отключения сервера, если он не подключается к другим участникам кворума, в миллисекундах (по умолчанию: 30000).
Конфигурация кворума находится в `<keeper_server>.<raft_configuration>` и содержит описание серверов. Единственный параметр для всего кворума — `secure`, который включает зашифрованное соединение для связи между участниками кворума. Параметры для каждого `<server>`:
- `id` — идентификатор сервера в кворуме.
- `hostname` — имя хоста, на котором размещен сервер.
- `port` — порт, на котором серверу доступны соединения для внутренней коммуникации.
Примеры конфигурации кворума с тремя узлами можно найти в [интеграционных тестах](https://github.com/ClickHouse/ClickHouse/tree/master/tests/integration) с префиксом `test_keeper_`. Пример конфигурации для сервера №1:
```xml
<keeper_server>
<tcp_port>2181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>30000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>zoo1</hostname>
<port>9444</port>
</server>
<server>
<id>2</id>
<hostname>zoo2</hostname>
<port>9444</port>
</server>
<server>
<id>3</id>
<hostname>zoo3</hostname>
<port>9444</port>
</server>
</raft_configuration>
</keeper_server>
```
## Как запустить
ClickHouse Keeper входит в пакет` clickhouse-server`, просто добавьте кофигурацию `<keeper_server>` и запустите сервер ClickHouse как обычно. Если вы хотите запустить ClickHouse Keeper автономно, сделайте это аналогичным способом:
```bash
clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon
```
## [экспериментально] Переход с ZooKeeper
Плавный переход с ZooKeeper на ClickHouse Keeper невозможен, необходимо остановить кластер ZooKeeper, преобразовать данные и запустить ClickHouse Keeper. Утилита `clickhouse-keeper-converter` конвертирует журналы и снэпшоты ZooKeeper в снэпшот ClickHouse Keeper. Работа утилиты проверена только для версий ZooKeeper выше 3.4. Для миграции необходимо выполнить следующие шаги:
1. Остановите все узлы ZooKeeper.
2. Необязательно, но рекомендуется: найдите узел-лидер ZooKeeper, запустите и снова остановите его. Это заставит ZooKeeper создать консистентный снэпшот.
3. Запустите `clickhouse-keeper-converter` на лидере, например:
```bash
clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 --zookeeper-snapshots-dir /var/lib/zookeeper/version-2 --output-dir /path/to/clickhouse/keeper/snapshots
```
4. Скопируйте снэпшот на узлы сервера ClickHouse с настроенным `keeper` или запустите ClickHouse Keeper вместо ZooKeeper. Снэпшот должен сохраняться на всех узлах: в противном случае пустые узлы могут захватить лидерство и сконвертированные данные могут быть отброшены на старте.
[Original article](https://clickhouse.tech/docs/en/operations/clickhouse-keeper/) <!--hide-->

View File

@ -201,6 +201,44 @@ Eсли суммарное число активных кусков во все
Значение по умолчанию: `0`.
## max_replicated_fetches_network_bandwidth {#max_replicated_fetches_network_bandwidth}
Ограничивает максимальную скорость скачивания данных в сети (в байтах в секунду) для синхронизаций между [репликами](../../engines/table-engines/mergetree-family/replication.md). Настройка применяется к конкретной таблице, в отличие от [max_replicated_fetches_network_bandwidth_for_server](settings.md#max_replicated_fetches_network_bandwidth_for_server), которая применяется к серверу.
Можно ограничить скорость обмена данными как для всего сервера, так и для конкретной таблицы, но для этого значение табличной настройки должно быть меньше серверной. Иначе сервер будет учитывать только настройку `max_replicated_fetches_network_bandwidth_for_server`.
Настройка соблюдается неточно.
Возможные значения:
- Любое целое положительное число.
- 0 — Скорость не ограничена.
Значение по умолчанию: `0`.
**Использование**
Может быть использована для ограничения скорости передачи данных при репликации данных для добавления или замены новых узлов.
## max_replicated_sends_network_bandwidth {#max_replicated_sends_network_bandwidth}
Ограничивает максимальную скорость отправки данных по сети (в байтах в секунду) для синхронизации между [репликами](../../engines/table-engines/mergetree-family/replication.md). Настройка применяется к конкретной таблице, в отличие от [max_replicated_sends_network_bandwidth_for_server](settings.md#max_replicated_sends_network_bandwidth_for_server), которая применяется к серверу.
Можно ограничить скорость обмена данными как для всего сервера, так и для конкретной таблицы, но для этого значение табличной настройки должно быть меньше серверной. Иначе сервер будет учитывать только настройку `max_replicated_sends_network_bandwidth_for_server`.
Настройка следуется неточно.
Возможные значения:
- Любое целое положительное число.
- 0 — Скорость не ограничена.
Значение по умолчанию: `0`.
**Использование**
Может быть использована для ограничения скорости сети при репликации данных для добавления или замены новых узлов.
## max_bytes_to_merge_at_max_space_in_pool {#max-bytes-to-merge-at-max-space-in-pool}
Максимальный суммарный размер кусков (в байтах) в одном слиянии, если есть свободные ресурсы в фоновом пуле.

View File

@ -1098,6 +1098,40 @@ SELECT type, query FROM system.query_log WHERE log_comment = 'log_comment test'
Значение по умолчанию: `5`.
## max_replicated_fetches_network_bandwidth_for_server {#max_replicated_fetches_network_bandwidth_for_server}
Ограничивает максимальную скорость обмена данными в сети (в байтах в секунду) для синхронизации между [репликами](../../engines/table-engines/mergetree-family/replication.md). Применяется только при запуске сервера. Можно также ограничить скорость для конкретной таблицы с помощью настройки [max_replicated_fetches_network_bandwidth](../../operations/settings/merge-tree-settings.md#max_replicated_fetches_network_bandwidth).
Значение настройки соблюдается неточно.
Возможные значения:
- Любое целое положительное число.
- 0 — Скорость не ограничена.
Значение по умолчанию: `0`.
**Использование**
Может быть использована для ограничения скорости сети при репликации данных для добавления или замены новых узлов.
## max_replicated_sends_network_bandwidth_for_server {#max_replicated_sends_network_bandwidth_for_server}
Ограничивает максимальную скорость обмена данными в сети (в байтах в секунду) для [репликационных](../../engines/table-engines/mergetree-family/replication.md) отправок. Применяется только при запуске сервера. Можно также ограничить скорость для конкретной таблицы с помощью настройки [max_replicated_sends_network_bandwidth](../../operations/settings/merge-tree-settings.md#max_replicated_sends_network_bandwidth).
Значение настройки соблюдается неточно.
Возможные значения:
- Любое целое положительное число.
- 0 — Скорость не ограничена.
Значение по умолчанию: `0`.
**Использование**
Может быть использована для ограничения скорости сети при репликации данных для добавления или замены новых узлов.
## connect_timeout_with_failover_ms {#connect-timeout-with-failover-ms}
Таймаут в миллисекундах на соединение с удалённым сервером, для движка таблиц Distributed, если используются секции shard и replica в описании кластера.
@ -3341,3 +3375,91 @@ SETTINGS index_granularity = 8192 │
- Положительное целое число.
Значение по умолчанию: `1000`.
## max_hyperscan_regexp_length {#max-hyperscan-regexp-length}
Задает максимальную длину каждого регулярного выражения в [hyperscan-функциях](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn) поиска множественных совпадений в строке.
Возможные значения:
- Положительное целое число.
- 0 - длина не ограничена.
Значение по умолчанию: `0`.
**Пример**
Запрос:
```sql
SELECT multiMatchAny('abcd', ['ab','bcd','c','d']) SETTINGS max_hyperscan_regexp_length = 3;
```
Результат:
```text
┌─multiMatchAny('abcd', ['ab', 'bcd', 'c', 'd'])─┐
│ 1 │
└────────────────────────────────────────────────┘
```
Запрос:
```sql
SELECT multiMatchAny('abcd', ['ab','bcd','c','d']) SETTINGS max_hyperscan_regexp_length = 2;
```
Результат:
```text
Exception: Regexp length too large.
```
**См. также**
- [max_hyperscan_regexp_total_length](#max-hyperscan-regexp-total-length)
## max_hyperscan_regexp_total_length {#max-hyperscan-regexp-total-length}
Задает максимальную общую длину всех регулярных выражений в каждой [hyperscan-функции](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn) поиска множественных совпадений в строке.
Возможные значения:
- Положительное целое число.
- 0 - длина не ограничена.
Значение по умолчанию: `0`.
**Пример**
Запрос:
```sql
SELECT multiMatchAny('abcd', ['a','b','c','d']) SETTINGS max_hyperscan_regexp_total_length = 5;
```
Результат:
```text
┌─multiMatchAny('abcd', ['a', 'b', 'c', 'd'])─┐
│ 1 │
└─────────────────────────────────────────────┘
```
Запрос:
```sql
SELECT multiMatchAny('abcd', ['ab','bc','c','d']) SETTINGS max_hyperscan_regexp_total_length = 5;
```
Результат:
```text
Exception: Total regexp lengths too large.
```
**См. также**
- [max_hyperscan_regexp_length](#max-hyperscan-regexp-length)

View File

@ -78,10 +78,11 @@ active_replicas: 2
- `log_max_index` (`UInt64`) - максимальный номер записи в общем логе действий.
- `log_pointer` (`UInt64`) - максимальный номер записи из общего лога действий, которую реплика скопировала в свою очередь для выполнения, плюс единица. Если log_pointer сильно меньше log_max_index, значит что-то не так.
- `last_queue_update` (`DateTime`) - When the queue was updated last time.
- `absolute_delay` (`UInt64`) - How big lag in seconds the current replica has.
- `last_queue_update` (`DateTime`) - время последнего обновления запроса.
- `absolute_delay` (`UInt64`) - задержка (в секундах) для текущей реплики.
- `total_replicas` (`UInt8`) - общее число известных реплик этой таблицы.
- `active_replicas` (`UInt8`) - число реплик этой таблицы, имеющих сессию в ZK; то есть, число работающих реплик.
- `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — соответствие между именем реплики и признаком активности реплики.
Если запрашивать все столбцы, то таблица может работать слегка медленно, так как на каждую строчку делается несколько чтений из ZK.
Если не запрашивать последние 4 столбца (log_max_index, log_pointer, total_replicas, active_replicas), то таблица работает быстро.

View File

@ -50,7 +50,7 @@ AS state FROM train_data;
После сохранения состояния в таблице мы можем использовать его несколько раз для прогнозирования или смёржить с другими состояниями и создать новые, улучшенные модели.
``` sql
```sql
WITH (SELECT state FROM your_model) AS model SELECT
evalMLMethod(model, param1, param2) FROM test_data
```
@ -65,9 +65,9 @@ evalMLMethod(model, param1, param2) FROM test_data
<!-- -->
``` sql
SELECT state1 + state2 FROM your_models
```
```sql
SELECT state1 + state2 FROM your_models
```
где таблица `your_models` содержит обе модели. Запрос вернёт новый объект `AggregateFunctionState`.
@ -75,9 +75,9 @@ evalMLMethod(model, param1, param2) FROM test_data
<!-- -->
``` sql
SELECT stochasticLinearRegression(0.01)(target, param1, param2) FROM train_data
```
```sql
SELECT stochasticLinearRegression(0.01)(target, param1, param2) FROM train_data
```
Подобный запрос строит модель и возвращает её веса, отвечающие параметрам моделей и смещение. Таким образом, в приведенном выше примере запрос вернет столбец с тремя значениями.

View File

@ -27,7 +27,7 @@ N может быть отрицательным.
Округляет значение до указанного десятичного разряда.
Функция возвращает ближайшее значение указанного порядка. В случае, когда заданное число равноудалено от чисел необходимого порядка, функция возвращает то из них, которое имеет ближайшую чётную цифру (банковское округление).
Функция возвращает ближайшее значение указанного порядка. В случае, когда заданное число равноудалено от чисел необходимого порядка, для типов с плавающей точкой (Float32/64) функция возвращает то из них, которое имеет ближайшую чётную цифру (банковское округление), для типов с фиксированной точкой (Decimal) функция использует округление в бо́льшую по модулю сторону (математическое округление).
``` sql
round(expression [, decimal_places])
@ -47,7 +47,7 @@ round(expression [, decimal_places])
### Примеры {#primery}
**Пример использования**
**Пример использования с Float**
``` sql
SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3
@ -61,6 +61,21 @@ SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3
└─────┴──────────────────────────┘
```
**Пример использования с Decimal**
``` sql
SELECT cast(number / 2 AS Decimal(10,4)) AS x, round(x) FROM system.numbers LIMIT 3
```
``` text
┌──────x─┬─round(CAST(divide(number, 2), 'Decimal(10, 4)'))─┐
│ 0.0000 │ 0.0000 │
│ 0.5000 │ 1.0000 │
│ 1.0000 │ 1.0000 │
└────────┴──────────────────────────────────────────────────┘
```
**Примеры округления**
Округление до ближайшего числа.

View File

@ -6,13 +6,13 @@ toc_title: 云
# ClickHouse 云服务提供商 {#clickhouse-cloud-service-providers}
!!! info "注意"
如果您已经推出具有托管 ClickHouse 服务的公共云,请随时[提交一个 pull request](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/cloud.md) 将其添加到以下列表。
如果您已经推出具有托管ClickHouse服务的公共云请随时[提交一个 pull request](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/cloud.md)将其添加到以下列表。
## Yandex 云 {#yandex-cloud}
[Yandex ClickHouse 托管服务](https://cloud.yandex.com/services/managed-clickhouse?utm_source=referrals&utm_medium=clickhouseofficialsite&utm_campaign=link3) 提供以下主要功能:
[Yandex ClickHouse托管服务](https://cloud.yandex.com/services/managed-clickhouse?utm_source=referrals&utm_medium=clickhouseofficialsite&utm_campaign=link3)提供以下主要功能:
- 用于 ClickHouse 复制的完全托管的 ZooKeeper 服务 [ClickHouse复制](../engines/table-engines/mergetree-family/replication.md)
- 用于[ClickHouse replication](../engines/table-engines/mergetree-family/replication.md)的完全托管的ZooKeeper服务
- 多种存储类型选择
- 不同可用区副本
- 加密与隔离
@ -20,34 +20,43 @@ toc_title: 云
## Altinity.Cloud {#altinity.cloud}
[Altinity.Cloud](https://altinity.com/cloud-database/) 是针对 Amazon 公共云的完全托管的 ClickHouse-as-a-Service
[Altinity.Cloud](https://altinity.com/cloud-database/)是针对Amazon公共云的完全托管的ClickHouse-as-a-Service
- 在 Amazon 资源上快速部署 ClickHouse 集群
- 在Amazon资源上快速部署ClickHouse集群
- 轻松进行横向扩展/纵向扩展以及节点的垂直扩展
- 具有公共端点或VPC对等的租户隔离
- 可配置存储类型以及卷配置
- 跨可用区扩展以实现性能和高可用性
- 内置监控和SQL查询编辑器
## 阿里云 {#alibaba-cloud}
## 阿里云{#alibaba-cloud}
[阿里云ClickHouse 托管服务](https://www.alibabacloud.com/zh/product/clickhouse) 提供以下主要功能:
[阿里云ClickHouse托管服务](https://www.alibabacloud.com/zh/product/clickhouse)提供以下主要功能:
- 基于阿里飞天分布式系统的高可靠云盘存储引擎
- 按需扩容,无需手动进行数据搬迁
- 支持单节点、单副本、多节点、多副本多种架构,支持冷热数据分层
- 支持访问白名单和一键恢复,多层网络安全防护,云盘加密
- 与云上日志系统、数据库、数据应用工具无缝集成
- 内置监控和数据库管理平台
- 专业的数据库专家技术支持和服务
- 基于阿里飞天分布式系统的高可靠云盘存储引擎
- 按需扩容,无需手动进行数据搬迁
- 支持单节点、单副本、多节点、多副本多种架构,支持冷热数据分层
- 支持访问白名单和一键恢复,多层网络安全防护,云盘加密
- 与云上日志系统、数据库、数据应用工具无缝集成
- 内置监控和数据库管理平台
- 专业的数据库专家技术支持和服务
## SberCloud {#sbercloud}
[SberCloud.Advanced](https://sbercloud.ru/en/advanced)提供[MapReduce Service (MRS)](https://docs.sbercloud.ru/mrs/ug/topics/ug__clickhouse.html), 一个可靠、安全且易于使用的企业级平台用于存储、处理和分析大数据。MRS允许您快速创建和管理ClickHouse集群。
- 一个ClickHouse实例由三个ZooKeeper节点和多个ClickHouse节点组成。 Dedicated Replica模式用于保证双数据副本的高可靠性。
- MRS提供平滑弹性伸缩能力快速满足集群存储容量或CPU计算资源不足场景下的业务增长需求。当您扩展集群中ClickHouse节点的容量时MRS提供一键式数据平衡工具让您主动进行数据平衡。 您可以根据业务特点确定数据均衡方式和时间,保证业务的可用性,实现平滑扩展。
- MRS采用弹性负载均衡保障高可用部署架构自动将用户访问流量分配到多个后端节点将服务能力扩展到外部系统提高容错能力。 通过ELB轮询机制数据写入本地表从不同节点的分布式表中读取。 这样就保证了数据读写负载和应用访问的高可用。
## 腾讯云 {#tencent-cloud}
[腾讯云的 ClickHouse 托管服务](https://cloud.tencent.com/product/cdwch)提供以下主要功能:
[腾讯云ClickHouse托管服务](https://cloud.tencent.com/product/cdwch)提供以下主要功能:
- 易于部署和管理, 集成监控与警报服务
- 高可用高扩展
- 通过集群级别的 VPC 保证安全可靠
- 按需定价,无需前期成本或长期承诺
- 易于在腾讯云上部署和管理
- 高度可扩展和可用
- 集成监控和警报服务
- 每个集群VPC隔离的高安全性
- 按需定价,无前期成本或长期承诺
{## [原始文章](https://clickhouse.tech/docs/en/commercial/cloud/) ##}

View File

@ -6,7 +6,7 @@ toc_title: 简介
# ClickHouse 商业服务 {#clickhouse-commercial-services}
本节是专门从事 ClickHouse 的服务提供商的目录,它们是一些独立的公司,不一定与 Yandex 有关系
此部分是专门从事ClickHouse的商业服务提供商的目录。 他们是独立的公司不一定隶属于Yandex
服务类别:
@ -14,4 +14,4 @@ toc_title: 简介
- [支持](../commercial/support.md)
!!! note "对于服务提供商"
如果您碰巧是其中之一,可以随时提交一个 pull request将您的公司添加到对应的章节如果服务不属于现有的任何目录也可以添加新的章节。提交关于文档的 pull request 最简单的方式是点击右上角的“铅笔”编辑按钮。如果您的服务在某些本地市场上有售,请确保在本地化的文档界面中也提及它(或至少在 pull request 请求描述中指出)。
如果您碰巧代表其中之一请随时提交一个pull request将您的公司添加到相应部分如果服务不适合现有类别甚至可以添加新部分。 提交关于文档的pull request最简单的方式是点击右上角的“铅笔”编辑按钮。 如果您的服务在某些本地市场可用请确保也在本地化文档页面中提及它或至少在pull request请求描述中指出)。

View File

@ -6,16 +6,20 @@ toc_title: 支持
# ClickHouse 商业支持服务提供商 {#clickhouse-commercial-support-service-providers}
!!! info "注意"
如果您已经推出 ClickHouse 商业支持服务,请随时[提交一个 pull request](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/support.md) 将其添加到以下列表。
如果您已经推出ClickHouse商业支持服务请随时[提交一个pull request](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/support.md)将其添加到以下列表。
## Yandex.Cloud
来自ClickHouse作者的ClickHouse全球支持。 支持内部部署和云部署。 在clickhouse-support@yandex-team.com上询问详细信息
## Altinity {#altinity}
Altinity 自从 2017 年开始为企业提供 ClickHouse 支持服务。Altinity 的客户范围包含百强企业到初创企业等。访问 [www.altinity.com](https://www.altinity.com/) 了解更多信息。
Altinity自2017年以来一直为企业ClickHouse提供支持和服务。 Altinity的客户范围从财富100强企业到初创公司。访问 [www.altinity.com](https://www.altinity.com/)了解更多信息。
## Mafiree {#mafiree}
[服务说明](http://mafiree.com/clickhouse-analytics-services.php)
[Service description](http://mafiree.com/clickhouse-analytics-services.php)
## MinervaDB {#minervadb}
[服务说明](https://minervadb.com/index.php/clickhouse-consulting-and-support-by-minervadb/)
[Service description](https://minervadb.com/index.php/clickhouse-consulting-and-support-by-minervadb/)

View File

@ -0,0 +1,150 @@
# 如何将测试查询添加到 ClickHouse CI
ClickHouse有数百个甚至数千个功能。 每个提交都由包含数千个测试用例的一组复杂测试进行检查。
核心功能经过了很多的测试但是ClickHouse CI可以发现一些极端情况和不同的功能组合。
我们看到的大多数错误/回归都发生在测试覆盖率较差的`灰色区域`中。
我们非常有兴趣通过测试涵盖实现生活中使用的大多数可能的场景和功能组合。
## 为什么要添加测试
为什么/何时应将测试用例添加到ClickHouse代码中
1) 您使用了一些复杂的场景/功能组合/您有一些可能未被广泛使用的情况
2) 您会看到更改日志中没有通知的版本之间的某些行为发生了变化
3) 您只是想帮助提高ClickHouse的质量并确保您使用的功能在未来的版本中不会被破坏
4) 一旦测试被添加/接受,您可以确保您检查的角落案例永远不会被意外损坏。
5) 你将成为伟大的开源社区的一份子
6) 您的名字将出现在`system.contributors`表中!
7) 你会让世界变得更好。
### 要做的步骤
#### 先决条件
我假设你运行一些Linux机器你可以在其他操作系统上使用 docker/虚拟机)和任何现代浏览器/互联网连接并且你有一些基本的Linux和SQL技能。
不需要任何高度专业化的知识(因此您不需要了解 C++ 或了解ClickHouse CI的工作原理
#### 准备
1) [create GitHub account](https://github.com/join) (如果你还没有)
2) [setup git](https://docs.github.com/en/free-pro-team@latest/github/getting-started-with-github/set-up-git)
```bash
# for Ubuntu
sudo apt-get update
sudo apt-get install git
git config --global user.name "John Doe" # fill with your name
git config --global user.email "email@example.com" # fill with your email
```
3) [fork ClickHouse project](https://docs.github.com/en/free-pro-team@latest/github/getting-started-with-github/fork-a-repo) - 打开 [https://github.com/ClickHouse/ClickHouse](https://github.com/ClickHouse/ClickHouse) and press fork button in the top right corner:
![fork repo](https://github-images.s3.amazonaws.com/help/bootcamp/Bootcamp-Fork.png)
4) 例如将代码fork克隆到PC上的某个文件夹, `~/workspace/ClickHouse`
```
mkdir ~/workspace && cd ~/workspace
git clone https://github.com/< your GitHub username>/ClickHouse
cd ClickHouse
git remote add upstream https://github.com/ClickHouse/ClickHouse
```
#### 测试的新分支
1) 从最新的clickhouse master创建一个新分支
```
cd ~/workspace/ClickHouse
git fetch upstream
git checkout -b name_for_a_branch_with_my_test upstream/master
```
#### 安装并运行 clickhouse
1) 安装`clickhouse-server` (参考[离线文档](https://clickhouse.tech/docs/en/getting-started/install/))
2) 安装测试配置它将使用Zookeeper模拟实现并调整一些设置
```
cd ~/workspace/ClickHouse/tests/config
sudo ./install.sh
```
3) 运行clickhouse-server
```
sudo systemctl restart clickhouse-server
```
#### 创建测试文件
1) 找到测试的编号 - 在`tests/queries/0_stateless/`中找到编号最大的文件
```sh
$ cd ~/workspace/ClickHouse
$ ls tests/queries/0_stateless/[0-9]*.reference | tail -n 1
tests/queries/0_stateless/01520_client_print_query_id.reference
```
目前,测试的最后一个数字是`01520`,所以我的测试将有数字`01521`
2) 使用您测试的功能的下一个编号和名称创建一个SQL文件
```sh
touch tests/queries/0_stateless/01521_dummy_test.sql
```
3) 使用您最喜欢的编辑器编辑SQL文件请参阅下面的创建测试提示
```sh
vim tests/queries/0_stateless/01521_dummy_test.sql
```
4) 运行测试,并将其结果放入参考文件中:
```
clickhouse-client -nmT < tests/queries/0_stateless/01521_dummy_test.sql | tee tests/queries/0_stateless/01521_dummy_test.reference
```
5) 确保一切正确,如果测试输出不正确(例如由于某些错误),请使用文本编辑器调整参考文件。
#### 如何创建一个好的测试
- 测试应该是
- 最小 - 仅创建与测试功能相关的表,删除不相关的列和部分查询
- 快速 - 不应超过几秒钟(更好的亚秒)
- 正确 - 失败则功能不起作用
- 确定性的
- 隔离/无状态
- 不要依赖一些环境的东西
- 尽可能不要依赖时间
- 尝试覆盖极端情况(zeros / Nulls / empty sets / throwing exceptions)
- 要测试该查询返回错误,您可以在查询后添加特殊注释:`-- { serverError 60 }`或`-- { clientError 20 }`
- 不要切换数据库(除非必要)
- 如果需要,您可以在同一节点上创建多个表副本
- 您可以在需要时使用测试集群定义之一(请参阅 system.clusters
- 使用 `number` / `numbers_mt` / `zeros` / `zeros_mt`和类似的查询要在适用时初始化数据
- 在测试之后和测试之前清理创建的对象DROP IF EXISTS - 在有一些脏状态的情况下
- 优先选择同步操作模式 (mutations, merges)
- 以`0_stateless`文件夹中的其他SQL文件为例
- 确保您想要测试的特性/特性组合尚未被现有测试覆盖
#### 测试命名规则
正确地命名测试非常重要因此可以在clickhouse-test调用中关闭一些测试子集。
| Tester flag| 测试名称中应该包含什么 | 什么时候应该添加标志 |
|---|---|---|---|
| `--[no-]zookeeper`| "zookeeper"或"replica" | 测试使用来自ReplicatedMergeTree家族的表 |
| `--[no-]shard` | "shard"或"distributed"或"global"| 使用到127.0.0.2或类似的连接进行测试 |
| `--[no-]long` | "long"或"deadlock"或"race" | 测试运行时间超过60秒 |
#### Commit / push / 创建PR.
1) commit & push您的修改
```sh
cd ~/workspace/ClickHouse
git add tests/queries/0_stateless/01521_dummy_test.sql
git add tests/queries/0_stateless/01521_dummy_test.reference
git commit # use some nice commit message when possible
git push origin HEAD
```
2) 使用一个在推送过程中显示的链接创建一个到master的PR
3) 调整PR标题和内容在`Changelog category (leave one)`中保留
`Build/Testing/Packaging Improvement`,如果需要,请填写其余字段。

View File

@ -23,7 +23,6 @@ ClickHouse可以接受和返回各种格式的数据。受支持的输入格式
| [CustomSeparated](#format-customseparated) | ✔ | ✔ |
| [Values](#data-format-values) | ✔ | ✔ |
| [Vertical](#vertical) | ✗ | ✔ |
| [VerticalRaw](#verticalraw) | ✗ | ✔ |
| [JSON](#json) | ✗ | ✔ |
| [JSONAsString](#jsonasstring) | ✔ | ✗ |
| [JSONStrings](#jsonstrings) | ✗ | ✔ |
@ -951,31 +950,6 @@ SELECT * FROM t_null FORMAT Vertical
该格式仅适用于输出查询结果,但不适用于解析输入(将数据插入到表中)。
## VerticalRaw {#verticalraw}
`Vertical` 格式不同点在于,行是不会被转义的。
这种格式仅仅适用于输出,但不适用于解析输入(将数据插入到表中)。
示例:
:) SHOW CREATE TABLE geonames FORMAT VerticalRaw;
Row 1:
──────
statement: CREATE TABLE default.geonames ( geonameid UInt32, date Date DEFAULT CAST('2017-12-08' AS Date)) ENGINE = MergeTree(date, geonameid, 8192)
:) SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT VerticalRaw;
Row 1:
──────
test: string with 'quotes' and with some special
characters
和 Vertical 格式相比:
:) SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical;
Row 1:
──────
test: string with \'quotes\' and \t with some special \n characters
## XML {#xml}
该格式仅适用于输出查询结果,但不适用于解析输入,示例:

View File

@ -72,6 +72,22 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix).
[clickhouse-flamegraph](https://github.com/Slach/clickhouse-flamegraph) 是一个可视化的专业工具`system.trace_log`如[flamegraph](http://www.brendangregg.com/flamegraphs.html).
### DBM {#dbm}
[DBM](https://dbm.incubator.edurt.io/) DBM是一款ClickHouse可视化管理工具!
特征:
- 支持查询历史(分页、全部清除等)
- 支持选中的sql子句查询(多窗口等)
- 支持终止查询
- 支持表管理
- 支持数据库管理
- 支持自定义查询
- 支持多数据源管理(连接测试、监控)
- 支持监控(处理进程、连接、查询)
- 支持迁移数据
## 商业 {#shang-ye}
### Holistics {#holistics-software}
@ -99,20 +115,4 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix).
- 重构。
- 搜索和导航。
### DBM {#dbm}
[DBM](https://dbm.incubator.edurt.io/) DBM是一款ClickHouse可视化管理工具!
特征:
- 支持查询历史(分页、全部清除等)
- 支持选中的sql子句查询(多窗口等)
- 支持终止查询
- 支持表管理
- 支持数据库管理
- 支持自定义查询
- 支持多数据源管理(连接测试、监控)
- 支持监控(处理进程、连接、查询)
- 支持迁移数据
[来源文章](https://clickhouse.tech/docs/zh/interfaces/third-party/gui/) <!--hide-->

View File

@ -0,0 +1,51 @@
#include "TestTags.h"
#include <cstring>
namespace DB
{
size_t getTestTagsLength(const String & multiline_query)
{
const String & text = multiline_query;
size_t pos = 0;
bool first_line = true;
while (true)
{
size_t line_start = pos;
/// Skip spaces.
while ((pos != text.length()) && (text[pos] == ' ' || text[pos] == '\t'))
++pos;
/// Skip comment "--".
static constexpr const char comment[] = "--";
if (text.compare(pos, strlen(comment), comment) != 0)
return line_start;
pos += strlen(comment);
/// Skip the prefix "Tags:" if it's the first line.
if (first_line)
{
while ((pos != text.length()) && (text[pos] == ' ' || text[pos] == '\t'))
++pos;
static constexpr const char tags_prefix[] = "Tags:";
if (text.compare(pos, strlen(tags_prefix), tags_prefix) != 0)
return 0;
pos += strlen(tags_prefix);
first_line = false;
}
/// Skip end-of-line.
size_t eol_pos = text.find_first_of("\r\n", pos);
if (eol_pos == String::npos)
return text.length();
bool two_chars_eol = (eol_pos + 1 < text.length()) && ((text[eol_pos + 1] == '\r') || (text[eol_pos + 1] == '\n')) && (text[eol_pos + 1] != text[eol_pos]);
size_t eol_length = two_chars_eol ? 2 : 1;
pos = eol_pos + eol_length;
}
}
}

View File

@ -0,0 +1,18 @@
#pragma once
#include <Core/Types.h>
namespace DB
{
/// Returns the length of a text looking like
/// -- Tags: x, y, z
/// -- Tag x: explanation of tag x
/// -- Tag y: explanation of tag y
/// -- Tag z: explanation of tag z
///
/// at the beginning of a multiline query.
/// If there are no test tags in the multiline query the function returns 0.
size_t getTestTagsLength(const String & multiline_query);
}

View File

@ -78,6 +78,8 @@
#include <Server/PostgreSQLHandlerFactory.h>
#include <Server/ProtocolServerAdapter.h>
#include <Server/HTTP/HTTPServer.h>
#include <Interpreters/AsynchronousInsertQueue.h>
#include <Compression/CompressionCodecEncrypted.h>
#include <filesystem>
#if !defined(ARCADIA_BUILD)
@ -251,7 +253,6 @@ namespace ErrorCodes
extern const int SUPPORT_IS_DISABLED;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int INCORRECT_DATA;
extern const int INVALID_CONFIG_PARAMETER;
extern const int SYSTEM_ERROR;
extern const int FAILED_TO_GETPWUID;
@ -456,40 +457,6 @@ void checkForUsersNotInMainConfig(
}
}
static void loadEncryptionKey(const std::string & key_command [[maybe_unused]], Poco::Logger * log)
{
#if USE_BASE64 && USE_SSL && USE_INTERNAL_SSL_LIBRARY
auto process = ShellCommand::execute(key_command);
std::string b64_key;
readStringUntilEOF(b64_key, process->out);
process->wait();
// turbob64 doesn't like whitespace characters in input. Strip
// them before decoding.
std::erase_if(b64_key, [](char c)
{
return c == ' ' || c == '\t' || c == '\r' || c == '\n';
});
std::vector<char> buf(b64_key.size());
const size_t key_size = tb64dec(reinterpret_cast<const unsigned char *>(b64_key.data()), b64_key.size(),
reinterpret_cast<unsigned char *>(buf.data()));
if (!key_size)
throw Exception("Failed to decode encryption key", ErrorCodes::INCORRECT_DATA);
else if (key_size < 16)
LOG_WARNING(log, "The encryption key should be at least 16 octets long.");
const std::string_view key = std::string_view(buf.data(), key_size);
CompressionCodecEncrypted::setMasterKey(key);
#else
LOG_WARNING(log, "Server was built without Base64 or SSL support. Encryption is disabled.");
#endif
}
[[noreturn]] void forceShutdown()
{
#if defined(THREAD_SANITIZER) && defined(OS_LINUX)
@ -904,6 +871,8 @@ if (ThreadFuzzer::instance().isEffective())
global_context->updateStorageConfiguration(*config);
global_context->updateInterserverCredentials(*config);
CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "encryption_codecs");
},
/* already_loaded = */ false); /// Reload it right now (initial loading)
@ -944,6 +913,13 @@ if (ThreadFuzzer::instance().isEffective())
global_context->setDefaultProfiles(config());
const Settings & settings = global_context->getSettingsRef();
if (settings.async_insert_threads)
global_context->setAsynchronousInsertQueue(std::make_shared<AsynchronousInsertQueue>(
global_context,
settings.async_insert_threads,
settings.async_insert_max_data_size,
AsynchronousInsertQueue::Timeout{.busy = settings.async_insert_busy_timeout, .stale = settings.async_insert_stale_timeout}));
/// Size of cache for marks (index of MergeTree family of tables). It is mandatory.
size_t mark_cache_size = config().getUInt64("mark_cache_size");
if (!mark_cache_size)
@ -976,9 +952,9 @@ if (ThreadFuzzer::instance().isEffective())
global_context->getMergeTreeSettings().sanityCheck(settings);
global_context->getReplicatedMergeTreeSettings().sanityCheck(settings);
/// Set up encryption.
if (config().has("encryption.key_command"))
loadEncryptionKey(config().getString("encryption.key_command"), log);
/// try set up encryption. There are some errors in config, error will be printed and server wouldn't start.
CompressionCodecEncrypted::Configuration::instance().load(config(), "encryption_codecs");
Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0);

View File

@ -194,11 +194,11 @@
<!-- The following file is used only if ssl_require_client_auth=1 -->
<ssl_ca_cert_file>/path/to/ssl_ca_cert_file</ssl_ca_cert_file>
<!-- Default compression algorithm (applied if client doesn't specify another algorithm).
<!-- Default compression algorithm (applied if client doesn't specify another algorithm, see result_compression in QueryInfo).
Supported algorithms: none, deflate, gzip, stream_gzip -->
<compression>deflate</compression>
<!-- Default compression level (applied if client doesn't specify another level).
<!-- Default compression level (applied if client doesn't specify another level, see result_compression in QueryInfo).
Supported levels: none, low, medium, high -->
<compression_level>medium</compression_level>
@ -1026,10 +1026,28 @@
defined, or encryption codecs will be disabled otherwise. The
command is executed through /bin/sh and is expected to write
a Base64-encoded key to the stdout. -->
<encryption>
<!-- <key_command>/usr/bin/systemd-ask-password &#45;&#45;id="clickhouse-server" &#45;&#45;timeout=0 "Enter the ClickHouse encryption passphrase:" | base64</key_command> -->
<!-- <key_command><![CDATA[IFS=; echo -n >/dev/tty "Enter the ClickHouse encryption passphrase: "; stty=`stty -F /dev/tty -g`; stty -F /dev/tty -echo; read k </dev/tty; stty -F /dev/tty "$stty"; echo -n $k | base64]]></key_command> -->
</encryption>
<encryption_codecs>
<!-- aes_128_gcm_siv -->
<!-- Example of getting hex key from env -->
<!-- the code should use this key and throw an exception if its length is not 16 bytes -->
<!--key_hex from_env="..."></key_hex -->
<!-- Example of multiple hex keys. They can be imported from env or be written down in config-->
<!-- the code should use these keys and throw an exception if their length is not 16 bytes -->
<!-- key_hex id="0">...</key_hex -->
<!-- key_hex id="1" from_env=".."></key_hex -->
<!-- key_hex id="2">...</key_hex -->
<!-- current_key_id>2</current_key_id -->
<!-- Example of getting hex key from config -->
<!-- the code should use this key and throw an exception if its length is not 16 bytes -->
<!-- key>...</key -->
<!-- example of adding nonce -->
<!-- nonce>...</nonce -->
<!-- /aes_128_gcm_siv -->
</encryption_codecs>
<!-- Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster.
Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. -->

View File

@ -15,8 +15,7 @@
namespace fs = std::filesystem;
#define UUID_PATTERN "[\\w]{8}-[\\w]{4}-[\\w]{4}-[\\w]{4}-[\\w]{12}"
#define EXTRACT_UUID_PATTERN fmt::format(".*/({})/.*", UUID_PATTERN)
#define EXTRACT_PATH_PATTERN ".*\\/store/(.*)"
namespace DB
@ -32,39 +31,102 @@ namespace ErrorCodes
* If test-mode option is added, files will be put by given url via PUT request.
*/
void processTableFiles(const fs::path & path, const String & files_prefix, String uuid,
WriteBuffer & metadata_buf, std::function<std::shared_ptr<WriteBuffer>(const String &)> create_dst_buf)
void processFile(const fs::path & file_path, const fs::path & dst_path, bool test_mode, WriteBuffer & metadata_buf)
{
fs::directory_iterator dir_end;
auto process_file = [&](const String & file_name, const String & file_path)
String remote_path;
RE2::FullMatch(file_path.string(), EXTRACT_PATH_PATTERN, &remote_path);
bool is_directory = fs::is_directory(file_path);
writeText(file_path.filename().string(), metadata_buf);
writeChar('\t', metadata_buf);
writeBoolText(is_directory, metadata_buf);
if (!is_directory)
{
auto remote_file_name = files_prefix + "-" + uuid + "-" + file_name;
writeText(remote_file_name, metadata_buf);
writeChar('\t', metadata_buf);
writeIntText(fs::file_size(file_path), metadata_buf);
writeChar('\n', metadata_buf);
}
writeChar('\n', metadata_buf);
auto src_buf = createReadBufferFromFileBase(file_path, {}, fs::file_size(file_path));
auto dst_buf = create_dst_buf(remote_file_name);
if (is_directory)
return;
copyData(*src_buf, *dst_buf);
dst_buf->next();
dst_buf->finalize();
};
auto dst_file_path = fs::path(dst_path) / remote_path;
for (fs::directory_iterator dir_it(path); dir_it != dir_end; ++dir_it)
auto src_buf = createReadBufferFromFileBase(file_path, {}, fs::file_size(file_path));
std::shared_ptr<WriteBuffer> dst_buf;
/// test mode for integration tests.
if (test_mode)
dst_buf = std::make_shared<WriteBufferFromHTTP>(Poco::URI(dst_file_path), Poco::Net::HTTPRequest::HTTP_PUT);
else
dst_buf = std::make_shared<WriteBufferFromFile>(dst_file_path);
copyData(*src_buf, *dst_buf);
dst_buf->next();
dst_buf->finalize();
};
void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_mode)
{
std::cerr << "Data path: " << data_path << ", destination path: " << dst_path << std::endl;
String prefix;
RE2::FullMatch(data_path.string(), EXTRACT_PATH_PATTERN, &prefix);
std::shared_ptr<WriteBuffer> root_meta;
if (test_mode)
{
dst_path /= "store";
auto files_root = dst_path / prefix;
root_meta = std::make_shared<WriteBufferFromHTTP>(Poco::URI(files_root / ".index"), Poco::Net::HTTPRequest::HTTP_PUT);
}
else
{
dst_path = fs::canonical(dst_path);
auto files_root = dst_path / prefix;
fs::create_directories(files_root);
root_meta = std::make_shared<WriteBufferFromFile>(files_root / ".index");
}
fs::directory_iterator dir_end;
for (fs::directory_iterator dir_it(data_path); dir_it != dir_end; ++dir_it)
{
if (dir_it->is_directory())
{
processFile(dir_it->path(), dst_path, test_mode, *root_meta);
String directory_prefix;
RE2::FullMatch(dir_it->path().string(), EXTRACT_PATH_PATTERN, &directory_prefix);
std::shared_ptr<WriteBuffer> directory_meta;
if (test_mode)
{
auto files_root = dst_path / prefix;
directory_meta = std::make_shared<WriteBufferFromHTTP>(Poco::URI(dst_path / directory_prefix / ".index"), Poco::Net::HTTPRequest::HTTP_PUT);
}
else
{
dst_path = fs::canonical(dst_path);
auto files_root = dst_path / prefix;
fs::create_directories(dst_path / directory_prefix);
directory_meta = std::make_shared<WriteBufferFromFile>(dst_path / directory_prefix / ".index");
}
fs::directory_iterator files_end;
for (fs::directory_iterator file_it(dir_it->path()); file_it != files_end; ++file_it)
process_file(dir_it->path().filename().string() + "-" + file_it->path().filename().string(), file_it->path());
processFile(file_it->path(), dst_path, test_mode, *directory_meta);
directory_meta->next();
directory_meta->finalize();
}
else
{
process_file(dir_it->path().filename(), dir_it->path());
processFile(dir_it->path(), dst_path, test_mode, *root_meta);
}
}
root_meta->next();
root_meta->finalize();
}
}
@ -80,8 +142,7 @@ try
("metadata-path", po::value<std::string>(), "Metadata path (select data_paths from system.tables where name='table_name'")
("test-mode", "Use test mode, which will put data on given url via PUT")
("url", po::value<std::string>(), "Web server url for test mode")
("output-dir", po::value<std::string>(), "Directory to put files in non-test mode")
("files-prefix", po::value<std::string>(), "Prefix for stored files");
("output-dir", po::value<std::string>(), "Directory to put files in non-test mode");
po::parsed_options parsed = po::command_line_parser(argc, argv).options(description).run();
po::variables_map options;
@ -94,18 +155,13 @@ try
exit(0);
}
String url, metadata_path, files_prefix;
String metadata_path;
if (options.count("metadata-path"))
metadata_path = options["metadata-path"].as<std::string>();
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No metadata-path option passed");
if (options.count("files-prefix"))
files_prefix = options["files-prefix"].as<std::string>();
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No files-prefix option passed");
fs::path fs_path = fs::weakly_canonical(metadata_path);
if (!fs::exists(fs_path))
{
@ -113,27 +169,14 @@ try
return 1;
}
String uuid;
if (!RE2::Extract(metadata_path, EXTRACT_UUID_PATTERN, "\\1", &uuid))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot extract uuid for: {}", metadata_path);
std::shared_ptr<WriteBuffer> metadata_buf;
std::function<std::shared_ptr<WriteBuffer>(const String &)> create_dst_buf;
String root_path;
if (options.count("test-mode"))
auto test_mode = options.contains("test-mode");
if (test_mode)
{
if (options.count("url"))
url = options["url"].as<std::string>();
root_path = options["url"].as<std::string>();
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No url option passed for test mode");
metadata_buf = std::make_shared<WriteBufferFromHTTP>(Poco::URI(fs::path(url) / (".index-" + uuid)), Poco::Net::HTTPRequest::HTTP_PUT);
create_dst_buf = [&](const String & remote_file_name)
{
return std::make_shared<WriteBufferFromHTTP>(Poco::URI(fs::path(url) / remote_file_name), Poco::Net::HTTPRequest::HTTP_PUT);
};
}
else
{
@ -141,17 +184,9 @@ try
root_path = options["output-dir"].as<std::string>();
else
root_path = fs::current_path();
metadata_buf = std::make_shared<WriteBufferFromFile>(fs::path(root_path) / (".index-" + uuid));
create_dst_buf = [&](const String & remote_file_name)
{
return std::make_shared<WriteBufferFromFile>(fs::path(root_path) / remote_file_name);
};
}
processTableFiles(fs_path, files_prefix, uuid, *metadata_buf, create_dst_buf);
metadata_buf->next();
metadata_buf->finalize();
processTableFiles(fs_path, root_path, test_mode);
return 0;
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <cstring>
#include <memory>
#include <experimental/type_traits>
#include <type_traits>
@ -123,6 +124,10 @@ struct AggregateFunctionSumData
if constexpr (std::is_floating_point_v<T>)
{
/// For floating point we use a similar trick as above, except that now we reinterpret the floating point number as an unsigned
/// integer of the same size and use a mask instead (0 to discard, 0xFF..FF to keep)
static_assert(sizeof(Value) == 4 || sizeof(Value) == 8);
typedef typename std::conditional<sizeof(Value) == 4, UInt32, UInt64>::type equivalent_integer;
constexpr size_t unroll_count = 128 / sizeof(T);
T partial_sums[unroll_count]{};
@ -132,10 +137,12 @@ struct AggregateFunctionSumData
{
for (size_t i = 0; i < unroll_count; ++i)
{
if (!condition_map[i] == add_if_zero)
{
Impl::add(partial_sums[i], ptr[i]);
}
equivalent_integer value;
std::memcpy(&value, &ptr[i], sizeof(Value));
value &= (!condition_map[i] != add_if_zero) - 1;
Value d;
std::memcpy(&d, &value, sizeof(Value));
Impl::add(partial_sums[i], d);
}
ptr += unroll_count;
condition_map += unroll_count;

View File

@ -49,6 +49,7 @@ SRCS(
AggregateFunctionSimpleLinearRegression.cpp
AggregateFunctionSimpleState.cpp
AggregateFunctionSingleValueOrNull.cpp
AggregateFunctionSparkbar.cpp
AggregateFunctionState.cpp
AggregateFunctionStatistics.cpp
AggregateFunctionStatisticsSimple.cpp

View File

@ -23,6 +23,6 @@ UInt64 BackupEntryConcat::getSize() const
std::unique_ptr<ReadBuffer> BackupEntryConcat::getReadBuffer() const
{
return std::make_unique<ConcatReadBuffer>(first_source->getReadBuffer(), second_source->getReadBuffer());
return std::make_unique<ConcatReadBuffer>(*first_source->getReadBuffer(), *second_source->getReadBuffer());
}
}

View File

@ -83,9 +83,8 @@ namespace
/// Replaces elements of types TEMPORARY_TABLE or ALL_TEMPORARY_TABLES with elements of type TABLE or DATABASE.
void replaceTemporaryTablesWithTemporaryDatabase(Elements & elements)
{
for (size_t i = 0; i != elements.size(); ++i)
for (auto & element : elements)
{
auto & element = elements[i];
switch (element.type)
{
case ElementType::TEMPORARY_TABLE:

View File

@ -27,8 +27,8 @@ HedgedConnectionsFactory::HedgedConnectionsFactory(
: pool(pool_), settings(settings_), timeouts(timeouts_), table_to_check(table_to_check_), log(&Poco::Logger::get("HedgedConnectionsFactory"))
{
shuffled_pools = pool->getShuffledPools(settings);
for (size_t i = 0; i != shuffled_pools.size(); ++i)
replicas.emplace_back(ConnectionEstablisherAsync(shuffled_pools[i].pool, &timeouts, settings, log, table_to_check.get()));
for (auto shuffled_pool : shuffled_pools)
replicas.emplace_back(ConnectionEstablisherAsync(shuffled_pool.pool, &timeouts, settings, log, table_to_check.get()));
max_tries
= (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES});

View File

@ -249,8 +249,8 @@ MaskInfo extractInvertedMask(
void inverseMask(PaddedPODArray<UInt8> & mask, MaskInfo & mask_info)
{
for (size_t i = 0; i != mask.size(); ++i)
mask[i] = !mask[i];
for (auto & byte : mask)
byte = !byte;
std::swap(mask_info.has_ones, mask_info.has_zeros);
}

View File

@ -77,7 +77,7 @@ template <typename T>
inline typename std::enable_if<(sizeof(T) > sizeof(DB::UInt64)), DB::UInt64>::type
intHashCRC32(const T & x, DB::UInt64 updated_value)
{
auto * begin = reinterpret_cast<const char *>(&x);
const auto * begin = reinterpret_cast<const char *>(&x);
for (size_t i = 0; i < sizeof(T); i += sizeof(UInt64))
{
updated_value = intHashCRC32(unalignedLoad<DB::UInt64>(begin), updated_value);

View File

@ -15,7 +15,7 @@
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
thread_local bool _memory_tracker_always_throw_logical_error_on_allocation = false;
thread_local bool memory_tracker_always_throw_logical_error_on_allocation = false;
#endif
namespace
@ -173,9 +173,9 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded)
}
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
if (unlikely(_memory_tracker_always_throw_logical_error_on_allocation))
if (unlikely(memory_tracker_always_throw_logical_error_on_allocation))
{
_memory_tracker_always_throw_logical_error_on_allocation = false;
memory_tracker_always_throw_logical_error_on_allocation = false;
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Memory tracker: allocations not allowed.");
}
#endif

View File

@ -15,11 +15,11 @@
/// DENY_ALLOCATIONS_IN_SCOPE in the inner scope. In Release builds these macros do nothing.
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
#include <common/scope_guard.h>
extern thread_local bool _memory_tracker_always_throw_logical_error_on_allocation;
extern thread_local bool memory_tracker_always_throw_logical_error_on_allocation;
#define ALLOCATIONS_IN_SCOPE_IMPL_CONCAT(n, val) \
bool _allocations_flag_prev_val##n = _memory_tracker_always_throw_logical_error_on_allocation; \
_memory_tracker_always_throw_logical_error_on_allocation = val; \
SCOPE_EXIT({ _memory_tracker_always_throw_logical_error_on_allocation = _allocations_flag_prev_val##n; })
bool _allocations_flag_prev_val##n = memory_tracker_always_throw_logical_error_on_allocation; \
memory_tracker_always_throw_logical_error_on_allocation = val; \
SCOPE_EXIT({ memory_tracker_always_throw_logical_error_on_allocation = _allocations_flag_prev_val##n; })
#define ALLOCATIONS_IN_SCOPE_IMPL(n, val) ALLOCATIONS_IN_SCOPE_IMPL_CONCAT(n, val)
#define DENY_ALLOCATIONS_IN_SCOPE ALLOCATIONS_IN_SCOPE_IMPL(__LINE__, true)
#define ALLOW_ALLOCATIONS_IN_SCOPE ALLOCATIONS_IN_SCOPE_IMPL(__LINE__, false)

View File

@ -74,7 +74,6 @@ private:
using GroupsContainer = std::list<Group>;
using OwnerQueryIds = std::unordered_map<String, size_t>;
private:
mutable std::mutex internal_state_mtx;
GroupsContainer readers_queue;
@ -85,7 +84,6 @@ private:
/// or writers_queue.end() otherwise
OwnerQueryIds owner_queries;
private:
RWLockImpl() = default;
void unlock(GroupsContainer::iterator group_it, const String & query_id) noexcept;
void dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it) noexcept;

View File

@ -31,6 +31,10 @@
M(int, pthread_mutex_unlock, pthread_mutex_t * arg)
#endif
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
namespace DB
{

View File

@ -221,7 +221,6 @@ MainThreadStatus & MainThreadStatus::getInstance()
return thread_status;
}
MainThreadStatus::MainThreadStatus()
: ThreadStatus()
{
main_thread = current_thread;
}

View File

@ -30,6 +30,8 @@ struct ZooKeeperResponse : virtual Response
XID xid = 0;
int64_t zxid = 0;
ZooKeeperResponse() = default;
ZooKeeperResponse(const ZooKeeperResponse &) = default;
virtual ~ZooKeeperResponse() override = default;
virtual void readImpl(ReadBuffer &) = 0;
virtual void writeImpl(WriteBuffer &) const = 0;

View File

@ -1,3 +1,7 @@
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
#if defined (OS_LINUX)
# include <sched.h>
#endif

View File

@ -43,7 +43,6 @@ SRCS(
Epoll.cpp
ErrorCodes.cpp
Exception.cpp
ExternalLoaderStatus.cpp
FieldVisitorDump.cpp
FieldVisitorHash.cpp
FieldVisitorSum.cpp

View File

@ -1,3 +1,7 @@
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>

View File

@ -1,215 +1,553 @@
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#endif
#include "Common/Exception.h"
#include "common/types.h"
#include "IO/VarInt.h"
#include <Compression/CompressionFactory.h>
#if USE_SSL && USE_INTERNAL_SSL_LIBRARY
#include <Compression/CompressionCodecEncrypted.h>
#include <Poco/Logger.h>
#include <common/logger_useful.h>
#include <Common/ErrorCodes.h>
// This depends on BoringSSL-specific API, notably <openssl/aead.h>.
#if USE_SSL && USE_INTERNAL_SSL_LIBRARY
#include <Parsers/ASTLiteral.h>
#include <cassert>
#include <openssl/digest.h> // Y_IGNORE
#include <openssl/err.h>
#include <openssl/hkdf.h> // Y_IGNORE
#include <string_view>
#include <boost/algorithm/hex.hpp>
#include <openssl/aead.h> // Y_IGNORE
#endif
// Common part for both parts (with SSL and without)
namespace DB
{
namespace ErrorCodes
namespace ErrorCodes
{
extern const int OPENSSL_ERROR;
}
namespace
{
/// Get string name for method. Return empty string for undefined Method
String getMethodName(EncryptionMethod Method)
{
if (Method == AES_128_GCM_SIV)
{
extern const int ILLEGAL_CODEC_PARAMETER;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int OPENSSL_ERROR;
return "AES_128_GCM_SIV";
}
void CompressionCodecEncrypted::setMasterKey(const std::string_view & master_key)
else if (Method == AES_256_GCM_SIV)
{
keys.emplace(master_key);
return "AES_256_GCM_SIV";
}
CompressionCodecEncrypted::KeyHolder::KeyHolder(const std::string_view & master_key)
else
{
// Derive a key from it.
keygen_key = deriveKey(master_key);
// EVP_AEAD_CTX is not stateful so we can create an
// instance now.
EVP_AEAD_CTX_zero(&ctx);
const int ok = EVP_AEAD_CTX_init(&ctx, EVP_aead_aes_128_gcm(),
reinterpret_cast<const uint8_t*>(keygen_key.data()), keygen_key.size(),
16 /* tag size */, nullptr);
if (!ok)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
return "";
}
}
CompressionCodecEncrypted::KeyHolder::~KeyHolder()
/// Get method code (used for codec, to understand which one we are using)
uint8_t getMethodCode(EncryptionMethod Method)
{
if (Method == AES_128_GCM_SIV)
{
EVP_AEAD_CTX_cleanup(&ctx);
return uint8_t(CompressionMethodByte::AES_128_GCM_SIV);
}
const CompressionCodecEncrypted::KeyHolder & CompressionCodecEncrypted::getKeys()
else if (Method == AES_256_GCM_SIV)
{
if (keys)
return *keys;
else
throw Exception("There is no configuration for encryption in the server config",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
return uint8_t(CompressionMethodByte::AES_256_GCM_SIV);
}
CompressionCodecEncrypted::CompressionCodecEncrypted(const std::string_view & cipher)
else
{
setCodecDescription("Encrypted", {std::make_shared<ASTLiteral>(cipher)});
throw Exception("Wrong encryption Method. Got " + getMethodName(Method), ErrorCodes::BAD_ARGUMENTS);
}
}
uint8_t CompressionCodecEncrypted::getMethodByte() const
} // end of namespace
} // end of namespace DB
#if USE_SSL && USE_INTERNAL_SSL_LIBRARY
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int INCORRECT_DATA;
}
namespace
{
constexpr size_t tag_size = 16; /// AES-GCM-SIV always uses a tag of 16 bytes length
constexpr size_t key_id_max_size = 8; /// Max size of varint.
constexpr size_t nonce_max_size = 13; /// Nonce size and one byte to show if nonce in in text
/// Get encryption/decryption algorithms.
auto getMethod(EncryptionMethod Method)
{
if (Method == AES_128_GCM_SIV)
{
return static_cast<uint8_t>(CompressionMethodByte::Encrypted);
return EVP_aead_aes_128_gcm_siv;
}
void CompressionCodecEncrypted::updateHash(SipHash & hash) const
else if (Method == AES_256_GCM_SIV)
{
getCodecDesc()->updateTreeHash(hash);
return EVP_aead_aes_256_gcm_siv;
}
UInt32 CompressionCodecEncrypted::getMaxCompressedDataSize(UInt32 uncompressed_size) const
else
{
// The GCM mode is a stream cipher. No paddings are
// involved. There will be a tag at the end of ciphertext (16
// octets).
return uncompressed_size + 16;
throw Exception("Wrong encryption Method. Got " + getMethodName(Method), ErrorCodes::BAD_ARGUMENTS);
}
}
UInt32 CompressionCodecEncrypted::doCompressData(const char * source, UInt32 source_size, char * dest) const
/// Find out key size for each algorithm
UInt64 methodKeySize(EncryptionMethod Method)
{
if (Method == AES_128_GCM_SIV)
{
// Generate an IV out of the data block and the key-generation
// key. It is completely deterministic, but does not leak any
// information about the data block except for equivalence of
// identical blocks (under the same master key). The IV will
// be used as an authentication tag. The ciphertext and the
// tag will be written directly in the dest buffer.
const std::string_view plaintext = std::string_view(source, source_size);
encrypt(plaintext, dest);
return source_size + 16;
return 16;
}
void CompressionCodecEncrypted::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size [[maybe_unused]]) const
else if (Method == AES_256_GCM_SIV)
{
// Extract the IV from the encrypted data block. Decrypt the
// block with the extracted IV, and compare the tag. Throw an
// exception if tags don't match.
const std::string_view ciphertext_and_tag = std::string_view(source, source_size);
assert(ciphertext_and_tag.size() == uncompressed_size + 16);
decrypt(ciphertext_and_tag, dest);
return 32;
}
std::string CompressionCodecEncrypted::lastErrorString()
else
{
std::array<char, 1024> buffer{};
ERR_error_string_n(ERR_get_error(), buffer.data(), buffer.size());
return std::string(buffer.data());
throw Exception("Wrong encryption Method. Got " + getMethodName(Method), ErrorCodes::BAD_ARGUMENTS);
}
}
std::string CompressionCodecEncrypted::deriveKey(const std::string_view & master_key)
std::string lastErrorString()
{
std::array<char, 1024> buffer;
ERR_error_string_n(ERR_get_error(), buffer.data(), buffer.size());
return std::string(buffer.data());
}
/// Encrypt plaintext with particular algorithm and put result into ciphertext_and_tag.
/// This function get key and nonce and encrypt text with their help.
/// If something went wrong (can't init context or can't encrypt data) it throws exception.
/// It returns length of encrypted text.
size_t encrypt(const std::string_view & plaintext, char * ciphertext_and_tag, EncryptionMethod method, const String & key, const String & nonce)
{
/// Init context for encryption, using key.
EVP_AEAD_CTX encrypt_ctx;
EVP_AEAD_CTX_zero(&encrypt_ctx);
const int ok_init = EVP_AEAD_CTX_init(&encrypt_ctx, getMethod(method)(),
reinterpret_cast<const uint8_t*>(key.data()), key.size(),
16 /* tag size */, nullptr);
if (!ok_init)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
/// encrypt data using context and given nonce.
size_t out_len;
const int ok_open = EVP_AEAD_CTX_seal(&encrypt_ctx,
reinterpret_cast<uint8_t *>(ciphertext_and_tag),
&out_len, plaintext.size() + 16,
reinterpret_cast<const uint8_t *>(nonce.data()), nonce.size(),
reinterpret_cast<const uint8_t *>(plaintext.data()), plaintext.size(),
nullptr, 0);
if (!ok_open)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
return out_len;
}
/// Encrypt plaintext with particular algorithm and put result into ciphertext_and_tag.
/// This function get key and nonce and encrypt text with their help.
/// If something went wrong (can't init context or can't encrypt data) it throws exception.
/// It returns length of encrypted text.
size_t decrypt(const std::string_view & ciphertext, char * plaintext, EncryptionMethod method, const String & key, const String & nonce)
{
/// Init context for decryption with given key.
EVP_AEAD_CTX decrypt_ctx;
EVP_AEAD_CTX_zero(&decrypt_ctx);
const int ok_init = EVP_AEAD_CTX_init(&decrypt_ctx, getMethod(method)(),
reinterpret_cast<const uint8_t*>(key.data()), key.size(),
16 /* tag size */, nullptr);
if (!ok_init)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
/// decrypt data using given nonce
size_t out_len;
const int ok_open = EVP_AEAD_CTX_open(&decrypt_ctx,
reinterpret_cast<uint8_t *>(plaintext),
&out_len, ciphertext.size(),
reinterpret_cast<const uint8_t *>(nonce.data()), nonce.size(),
reinterpret_cast<const uint8_t *>(ciphertext.data()), ciphertext.size(),
nullptr, 0);
if (!ok_open)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
return out_len;
}
/// Register codec in factory
void registerEncryptionCodec(CompressionCodecFactory & factory, EncryptionMethod Method)
{
const auto method_code = getMethodCode(Method); /// Codec need to know its code
factory.registerCompressionCodec(getMethodName(Method), method_code, [&, Method](const ASTPtr & arguments) -> CompressionCodecPtr
{
/// No salt: derive keys in a deterministic manner.
std::string_view salt(""); // NOLINT
std::string_view info("Codec Encrypted('AES-128-GCM-SIV') key generation key");
std::array<char, 32> result;
const int ok = HKDF(reinterpret_cast<uint8_t *>(result.data()), result.size(),
EVP_sha256(),
reinterpret_cast<const uint8_t *>(master_key.data()), master_key.size(),
reinterpret_cast<const uint8_t *>(salt.data()), salt.size(),
reinterpret_cast<const uint8_t *>(info.data()), info.size());
if (!ok)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
return std::string(result.data(), 16);
}
void CompressionCodecEncrypted::encrypt(const std::string_view & plaintext, char * ciphertext_and_tag)
{
// Fixed nonce. Yes this is unrecommended, but we have to live
// with it.
std::string_view nonce("\0\0\0\0\0\0\0\0\0\0\0\0", 12);
size_t out_len;
const int ok = EVP_AEAD_CTX_seal(&getKeys().ctx,
reinterpret_cast<uint8_t *>(ciphertext_and_tag),
&out_len, plaintext.size() + 16,
reinterpret_cast<const uint8_t *>(nonce.data()), nonce.size(),
reinterpret_cast<const uint8_t *>(plaintext.data()), plaintext.size(),
nullptr, 0);
if (!ok)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
assert(out_len == plaintext.size() + 16);
}
void CompressionCodecEncrypted::decrypt(const std::string_view & ciphertext, char * plaintext)
{
std::string_view nonce("\0\0\0\0\0\0\0\0\0\0\0\0", 12);
size_t out_len;
const int ok = EVP_AEAD_CTX_open(&getKeys().ctx,
reinterpret_cast<uint8_t *>(plaintext),
&out_len, ciphertext.size(),
reinterpret_cast<const uint8_t *>(nonce.data()), nonce.size(),
reinterpret_cast<const uint8_t *>(ciphertext.data()), ciphertext.size(),
nullptr, 0);
if (!ok)
throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR);
assert(out_len == ciphertext.size() - 16);
}
void registerCodecEncrypted(CompressionCodecFactory & factory)
{
const auto method_code = uint8_t(CompressionMethodByte::Encrypted);
factory.registerCompressionCodec("Encrypted", method_code, [&](const ASTPtr & arguments) -> CompressionCodecPtr
if (arguments)
{
if (arguments)
{
if (arguments->children.size() != 1)
throw Exception("Codec Encrypted() must have 1 parameter, given " +
std::to_string(arguments->children.size()),
ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
if (!arguments->children.empty())
throw Exception("Codec " + getMethodName(Method) + " must not have parameters, given " +
std::to_string(arguments->children.size()),
ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
}
return std::make_shared<CompressionCodecEncrypted>(Method);
});
}
const auto children = arguments->children;
const auto * literal = children[0]->as<ASTLiteral>();
if (!literal)
throw Exception("Wrong argument for codec Encrypted(). Expected a string literal",
ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE);
const String cipher = literal->value.safeGet<String>();
if (cipher == "AES-128-GCM-SIV")
return std::make_shared<CompressionCodecEncrypted>(cipher);
else
throw Exception("Cipher '" + cipher + "' is not supported",
ErrorCodes::ILLEGAL_CODEC_PARAMETER);
}
else
{
/* The factory is asking us to construct the codec
* only from the method code. How can that be
* possible? For now we only support a single cipher
* so it's not really a problem, but if we were to
* support more ciphers it would be catastrophic. */
return std::make_shared<CompressionCodecEncrypted>("AES-128-GCM-SIV");
}
});
String unhexKey(const String & hex)
{
try
{
return boost::algorithm::unhex(hex);
}
catch (const std::exception &)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read key_hex, check for valid characters [0-9a-fA-F] and length");
}
}
/// Firstly, write a byte, which shows if the nonce will be put in text (if it was defined in config)
/// Secondly, write nonce in text (this step depends from first step)
/// return new position to write
inline char* writeNonce(const String& nonce, char* dest)
{
/// If nonce consists of nul bytes, it shouldn't be in dest. Zero byte is the only byte that should be written.
/// Otherwise, 1 is written and data from nonce is copied
if (nonce != String("\0\0\0\0\0\0\0\0\0\0\0\0", 12))
{
*dest = 1;
++dest;
size_t copied_symbols = nonce.copy(dest, nonce.size());
if (copied_symbols != nonce.size())
throw Exception(ErrorCodes::INCORRECT_DATA, "Can't copy nonce into destination. Count of copied symbols {}, need to copy {}", copied_symbols, nonce.size());
dest += copied_symbols;
return dest;
}
else
{
*dest = 0;
return ++dest;
}
}
/// Firstly, read a byte, which shows if the nonce will be put in text (if it was defined in config)
/// Secondly, read nonce in text (this step depends from first step)
/// return new position to read
inline const char* readNonce(String& nonce, const char* source)
{
/// If first is zero byte: move source and set zero-bytes nonce
if (!*source)
{
nonce = {"\0\0\0\0\0\0\0\0\0\0\0\0", 12};
return ++source;
}
/// Move to next byte. Nonce will begin from there
++source;
/// Otherwise, use data from source in nonce
nonce = {source, 12};
source += 12;
return source;
}
}
CompressionCodecEncrypted::Configuration & CompressionCodecEncrypted::Configuration::instance()
{
static CompressionCodecEncrypted::Configuration ret;
return ret;
}
void CompressionCodecEncrypted::Configuration::loadImpl(
const Poco::Util::AbstractConfiguration & config, const String & config_prefix, EncryptionMethod method, std::unique_ptr<Params> & new_params)
{
// if method is not smaller than MAX_ENCRYPTION_METHOD it is incorrect
if (method >= MAX_ENCRYPTION_METHOD)
throw Exception("Wrong argument for loading configurations.", ErrorCodes::BAD_ARGUMENTS);
/// Scan all keys in config and add them into storage. If key is in hex, transform it.
/// Remember key ID for each key, because it will be used in encryption/decryption
Strings config_keys;
config.keys(config_prefix, config_keys);
for (const std::string & config_key : config_keys)
{
String key;
UInt64 key_id;
if ((config_key == "key") || config_key.starts_with("key["))
{
key = config.getString(config_prefix + "." + config_key, "");
key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0);
}
else if ((config_key == "key_hex") || config_key.starts_with("key_hex["))
{
key = unhexKey(config.getString(config_prefix + "." + config_key, ""));
key_id = config.getUInt64(config_prefix + "." + config_key + "[@id]", 0);
}
else
continue;
/// For each key its id should be unique.
if (new_params->keys_storage[method].contains(key_id))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Multiple keys have the same ID {}", key_id);
/// Check size of key. Its length depends on encryption algorithm.
if (key.size() != methodKeySize(method))
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Got an encryption key with unexpected size {}, the size should be {}",
key.size(), methodKeySize(method));
new_params->keys_storage[method][key_id] = key;
}
/// Check that we have at least one key for this method (otherwise it is incorrect to use it).
if (new_params->keys_storage[method].empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No keys, an encryption needs keys to work");
/// Try to find which key will be used for encryption. If there is no current_key,
/// first key will be used for encryption (its index equals to zero).
new_params->current_key_id[method] = config.getUInt64(config_prefix + ".current_key_id", 0);
/// Check that we have current key. Otherwise config is incorrect.
if (!new_params->keys_storage[method].contains(new_params->current_key_id[method]))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", new_params->current_key_id[method]);
/// Read nonce (in hex or in string). Its length should be 12 bytes.
if (config.has(config_prefix + ".nonce_hex"))
new_params->nonce[method] = unhexKey(config.getString(config_prefix + ".nonce_hex"));
else
new_params->nonce[method] = config.getString(config_prefix + ".nonce", "");
if (new_params->nonce[method].size() != 12 && !new_params->nonce[method].empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Got nonce with unexpected size {}, the size should be 12", new_params->nonce[method].size());
}
bool CompressionCodecEncrypted::Configuration::tryLoad(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
/// Try to create new parameters and fill them from config.
/// If there will be some errors, print their message to notify user that
/// something went wrong and new parameters are not available
try
{
load(config, config_prefix);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
return false;
}
return true;
}
void CompressionCodecEncrypted::Configuration::load(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
/// Try to create new parameters and fill them from config.
/// If there will be some errors, throw error
std::unique_ptr<Params> new_params(new Params);
if (config.has(config_prefix + ".aes_128_gcm_siv"))
{
loadImpl(config, config_prefix + ".aes_128_gcm_siv", AES_128_GCM_SIV, new_params);
}
if (config.has(config_prefix + ".aes_256_gcm_siv"))
{
loadImpl(config, config_prefix + ".aes_256_gcm_siv", AES_256_GCM_SIV, new_params);
}
params.set(std::move(new_params));
}
void CompressionCodecEncrypted::Configuration::getCurrentKeyAndNonce(EncryptionMethod method, UInt64 & current_key_id, String &current_key, String & nonce) const
{
/// It parameters were not set, throw exception
if (!params.get())
throw Exception("Empty params in CompressionCodecEncrypted configuration", ErrorCodes::BAD_ARGUMENTS);
/// Save parameters in variable, because they can always change.
/// As this function not atomic, we should be certain that we get information from one particular version for correct work.
const auto current_params = params.get();
current_key_id = current_params->current_key_id[method];
/// As parameters can be created empty, we need to check that this key is available.
if (current_params->keys_storage[method].contains(current_key_id))
current_key = current_params->keys_storage[method].at(current_key_id);
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no current_key {} in config. Please, put it in config and reload.", current_key_id);
/// If there is no nonce in config, we need to generate particular one,
/// because all encryptions should have nonce and random nonce generation will lead to cases
/// when nonce after config reload (nonce is not defined in config) will differ from previously generated one.
/// This will lead to data loss.
nonce = current_params->nonce[method];
if (nonce.empty())
nonce = {"\0\0\0\0\0\0\0\0\0\0\0\0", 12};
}
String CompressionCodecEncrypted::Configuration::getKey(EncryptionMethod method, const UInt64 & key_id) const
{
String key;
/// See description of previous finction, logic is the same.
if (!params.get())
throw Exception("Empty params in CompressionCodecEncrypted configuration", ErrorCodes::BAD_ARGUMENTS);
const auto current_params = params.get();
/// check if there is current key in storage
if (current_params->keys_storage[method].contains(key_id))
key = current_params->keys_storage[method].at(key_id);
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no key {} in config", key_id);
return key;
}
CompressionCodecEncrypted::CompressionCodecEncrypted(EncryptionMethod Method): encryption_method(Method)
{
setCodecDescription(getMethodName(encryption_method));
}
uint8_t CompressionCodecEncrypted::getMethodByte() const
{
return getMethodCode(encryption_method);
}
void CompressionCodecEncrypted::updateHash(SipHash & hash) const
{
getCodecDesc()->updateTreeHash(hash);
}
UInt32 CompressionCodecEncrypted::getMaxCompressedDataSize(UInt32 uncompressed_size) const
{
// The GCM mode is a stream cipher. No paddings are
// involved. There will be a tag at the end of ciphertext (16
// octets). Also it has not more than 8 bytes for key_id in the beginning
// KeyID is followed by byte, that shows if nonce was set in config (and also will be put into data)
// and 12 bytes nonce or this byte will be equal to zero and no nonce will follow it.
return uncompressed_size + tag_size + key_id_max_size + nonce_max_size;
}
UInt32 CompressionCodecEncrypted::doCompressData(const char * source, UInt32 source_size, char * dest) const
{
// Generate an IV out of the data block and the key-generation
// key. It is completely deterministic, but does not leak any
// information about the data block except for equivalence of
// identical blocks (under the same key).
const std::string_view plaintext = std::string_view(source, source_size);
/// Get key and nonce for encryption
UInt64 current_key_id;
String current_key, nonce;
Configuration::instance().getCurrentKeyAndNonce(encryption_method, current_key_id, current_key, nonce);
/// Write current key id to support multiple keys.
/// (key id in the beginning will help to decrypt data after changing current key)
char* ciphertext_with_nonce = writeVarUInt(current_key_id, dest);
size_t keyid_size = ciphertext_with_nonce - dest;
/// write nonce in data. This will help to read data even after changing nonce in config
/// If there were no nonce in data, one zero byte will be written
char* ciphertext = writeNonce(nonce, ciphertext_with_nonce);
UInt64 nonce_size = ciphertext - ciphertext_with_nonce;
// The IV will be used as an authentication tag. The ciphertext and the
// tag will be written directly in the dest buffer.
size_t out_len = encrypt(plaintext, ciphertext, encryption_method, current_key, nonce);
/// Length of encrypted text should be equal to text length plus tag_size (which was added by algorithm).
if (out_len != source_size + tag_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't encrypt data, length after encryption {} is wrong, expected {}", out_len, source_size + tag_size);
return out_len + keyid_size + nonce_size;
}
void CompressionCodecEncrypted::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const
{
/// The key is needed for decrypting. That's why it is read at the beginning of process.
UInt64 key_id;
const char * ciphertext_with_nonce = readVarUInt(key_id, source, source_size);
/// Size of text should be decreased by key_size, because key_size bytes were not participating in encryption process.
size_t keyid_size = ciphertext_with_nonce - source;
String nonce;
String key = Configuration::instance().getKey(encryption_method, key_id);
/// try to read nonce from file (if it was set while encrypting)
const char * ciphertext = readNonce(nonce, ciphertext_with_nonce);
/// Size of text should be decreased by nonce_size, because nonce_size bytes were not participating in encryption process.
UInt64 nonce_size = ciphertext - ciphertext_with_nonce;
/// Count text size (nonce and key_id was read from source)
size_t ciphertext_size = source_size - keyid_size - nonce_size;
if (ciphertext_size != uncompressed_size + tag_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't decrypt data, uncompressed_size {} is wrong, expected {}", uncompressed_size, ciphertext_size - tag_size);
size_t out_len = decrypt({ciphertext, ciphertext_size}, dest, encryption_method, key, nonce);
if (out_len != ciphertext_size - tag_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't decrypt data, out length after decryption {} is wrong, expected {}", out_len, ciphertext_size - tag_size);
}
}
#else /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */
namespace DB
{
void registerCodecEncrypted(CompressionCodecFactory &)
{
}
namespace
{
/// Register codec in factory
void registerEncryptionCodec(CompressionCodecFactory & factory, EncryptionMethod Method)
{
auto throw_no_ssl = [](const ASTPtr &) -> CompressionCodecPtr { throw Exception(ErrorCodes::OPENSSL_ERROR, "Server was built without SSL support. Encryption is disabled."); };
const auto method_code = getMethodCode(Method); /// Codec need to know its code
factory.registerCompressionCodec(getMethodName(Method), method_code, throw_no_ssl);
}
}
CompressionCodecEncrypted::Configuration & CompressionCodecEncrypted::Configuration::instance()
{
static CompressionCodecEncrypted::Configuration ret;
return ret;
}
/// if encryption is disabled.
bool CompressionCodecEncrypted::Configuration::tryLoad(const Poco::Util::AbstractConfiguration & config [[maybe_unused]], const String & config_prefix [[maybe_unused]])
{
return false;
}
/// if encryption is disabled, print warning about this.
void CompressionCodecEncrypted::Configuration::load(const Poco::Util::AbstractConfiguration & config [[maybe_unused]], const String & config_prefix [[maybe_unused]])
{
LOG_WARNING(&Poco::Logger::get("CompressionCodecEncrypted"), "Server was built without SSL support. Encryption is disabled.");
}
}
#endif /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */
namespace DB
{
/// Register codecs for all algorithms
void registerCodecEncrypted(CompressionCodecFactory & factory)
{
registerEncryptionCodec(factory, AES_128_GCM_SIV);
registerEncryptionCodec(factory, AES_256_GCM_SIV);
}
}

View File

@ -1,109 +1,138 @@
#pragma once
// This depends on BoringSSL-specific API, notably <openssl/aead.h>.
#include <Common/config.h>
#if USE_SSL && USE_INTERNAL_SSL_LIBRARY && !defined(ARCADIA_BUILD)
#if !defined(ARCADIA_BUILD)
#include <string_view>
#include <unordered_map>
#include <common/types.h>
#include <Compression/ICompressionCodec.h>
#include <boost/noncopyable.hpp>
#include <openssl/aead.h> // Y_IGNORE
#include <optional>
#include <Poco/Util/LayeredConfiguration.h>
#include <Common/MultiVersion.h>
namespace DB
{
/** This codec encrypts and decrypts blocks with AES-128 in
* GCM-SIV mode (RFC-8452), which is the only cipher currently
* supported. Although it is implemented as a compression codec
* it doesn't actually compress data. In fact encrypted data will
* no longer be compressible in any meaningful way. This means if
* you want to apply both compression and encryption to your
* columns, you need to put this codec at the end of the chain
* like "column Int32 Codec(Delta, LZ4,
* Encrypted('AES-128-GCM-SIV'))".
*
* The key is obtained by executing a command specified in the
* configuration file at startup, and if it doesn't specify a
* command the codec refuses to process any data. The command is
* expected to write a Base64-encoded key of any length, and we
* apply HKDF-SHA-256 to derive a 128-bit key-generation key
* (only the first half of the result is used). We then encrypt
* blocks in AES-128-GCM-SIV with a universally fixed nonce (12
* repeated NUL characters).
*
* This construct has a weakness due to the nonce being fixed at
* all times: when the same data block is encrypted twice, the
* resulting ciphertext will be exactly the same. We have to live
* with this weakness because ciphertext must be deterministic,
* as otherwise our engines like ReplicatedMergeTree cannot
* deduplicate data blocks.
*/
class CompressionCodecEncrypted : public ICompressionCodec
/// Now we have two algorithms.
enum EncryptionMethod
{
AES_128_GCM_SIV,
AES_256_GCM_SIV,
MAX_ENCRYPTION_METHOD
};
/** This codec encrypts and decrypts blocks with AES-128 in
* GCM-SIV mode (RFC-8452), which is the only cipher currently
* supported. Although it is implemented as a compression codec
* it doesn't actually compress data. In fact encrypted data will
* no longer be compressible in any meaningful way. This means if
* you want to apply both compression and encryption to your
* columns, you need to put this codec at the end of the chain
* like "column Int32 Codec(Delta, LZ4,
* AES_128_GCM_SIV)".
*
* The key is obtained by executing a command specified in the
* configuration file at startup, and if it doesn't specify a
* command the codec refuses to process any data. The command is
* expected to write a Base64-encoded key of any length, and we
* apply HKDF-SHA-256 to derive a 128-bit key-generation key
* (only the first half of the result is used). We then encrypt
* blocks in AES-128-GCM-SIV with a universally fixed nonce (12
* repeated NUL characters).
*
* This construct has a weakness due to the nonce being fixed at
* all times: when the same data block is encrypted twice, the
* resulting ciphertext will be exactly the same. We have to live
* with this weakness because ciphertext must be deterministic,
* as otherwise our engines like ReplicatedMergeTree cannot
* deduplicate data blocks.
*/
class CompressionCodecEncrypted : public ICompressionCodec
{
public:
/** If a key is available, the server is supposed to
* invoke this static method at the startup. The codec will
* refuse to compress or decompress any data until that. The
* key can be an arbitrary octet string, but it is
* recommended that the key is at least 16 octets long.
*
* Note that the key is currently not guarded by a
* mutex. This method should be invoked no more than once.
*/
explicit CompressionCodecEncrypted(EncryptionMethod Method);
/**
* This is utility class. It holds information about encryption configuration.
*/
class Configuration
{
public:
/** If a master key is available, the server is supposed to
* invoke this static method at the startup. The codec will
* refuse to compress or decompress any data until that. The
* key can be an arbitrary octet string, but it is
* recommended that the key is at least 16 octets long.
*
* Note that the master key is currently not guarded by a
* mutex. This method should be invoked no more than once.
*/
static void setMasterKey(const std::string_view & master_key);
/// Configuration should be singleton. Instance method
static Configuration & instance();
explicit CompressionCodecEncrypted(const std::string_view & cipher);
/// Try to load data from config.
bool tryLoad(const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
/// Load data and throw exception if something went wrong.
void load(const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
bool isCompression() const override
{
return false;
}
bool isGenericCompression() const override
{
return false;
}
bool isPostProcessing() const override
{
return true;
}
protected:
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
/// Get current key and nonce (they will be set in variables, which are pass in this function).
/// All data sets at the same time to prevent situations,
/// when config changes and key and nonce are read from different versions
/// If nonce is empty, it will return 12 null bytes.
void getCurrentKeyAndNonce(EncryptionMethod method, UInt64 & current_key_id, String & current_key, String & nonce) const;
/// Same as getCurrentKeyAndNonce. It is used to get key. (need for correct decryption, that is why nonce is not necessary)
String getKey(EncryptionMethod method, const UInt64 & key_id) const;
private:
static std::string lastErrorString();
static std::string deriveKey(const std::string_view & master_key);
static void encrypt(const std::string_view & plaintext, char * ciphertext_and_tag);
static void decrypt(const std::string_view & ciphertext_and_tag, char * plaintext);
/** A private class that holds keys derived from the master
* key.
*/
struct KeyHolder : private boost::noncopyable
/// struct Params consists of:
/// 1) hash-table of keys and their ids
/// 2) current key for encryption
/// 3) nonce for encryption
/// All this parameters have MAX_ENCRYPTION_METHOD count of versions,
/// because all algorithms can be described in config and used for different tables.
struct Params
{
explicit KeyHolder(const std::string_view & master_key);
~KeyHolder();
std::string keygen_key;
EVP_AEAD_CTX ctx;
std::unordered_map<UInt64, String> keys_storage[MAX_ENCRYPTION_METHOD];
UInt64 current_key_id[MAX_ENCRYPTION_METHOD] = {0, 0};
String nonce[MAX_ENCRYPTION_METHOD];
};
static const KeyHolder & getKeys();
// used to read data from config and create Params
static void loadImpl(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, EncryptionMethod method, std::unique_ptr<Params>& new_params);
static inline std::optional<KeyHolder> keys;
MultiVersion<Params> params;
};
inline CompressionCodecPtr getCompressionCodecEncrypted(const std::string_view & master_key)
uint8_t getMethodByte() const override;
void updateHash(SipHash & hash) const override;
bool isCompression() const override
{
return std::make_shared<CompressionCodecEncrypted>(master_key);
return false;
}
bool isGenericCompression() const override
{
return false;
}
bool isEncryption() const override
{
return true;
}
protected:
UInt32 getMaxCompressedDataSize(UInt32 uncompressed_size) const override;
/// Encrypt data with chosen method.
/// Throws exception if encryption is impossible or size of encrypted text is incorrect
UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override;
/// Decrypt data with chosen method
/// Throws exception if decryption is impossible or size of decrypted text is incorrect
void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override;
private:
EncryptionMethod encryption_method;
};
}
#endif /* USE_SSL && USE_INTERNAL_SSL_LIBRARY */
#endif /* NOT Arcadia_build */

View File

@ -1,3 +1,7 @@
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionInfo.h>
#include <Compression/CompressionFactory.h>

View File

@ -62,7 +62,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
bool is_compression = false;
bool has_none = false;
std::optional<size_t> generic_compression_codec_pos;
std::set<size_t> post_processing_codecs;
std::set<size_t> encryption_codecs;
bool can_substitute_codec_arguments = true;
for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i)
@ -141,8 +141,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
if (!generic_compression_codec_pos && result_codec->isGenericCompression())
generic_compression_codec_pos = i;
if (result_codec->isPostProcessing())
post_processing_codecs.insert(i);
if (result_codec->isEncryption())
encryption_codecs.insert(i);
}
String codec_description = queryToString(codecs_descriptions);
@ -157,8 +157,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// Allow to explicitly specify single NONE codec if user don't want any compression.
/// But applying other transformations solely without compression (e.g. Delta) does not make sense.
/// It's okay to apply post-processing codecs solely without anything else.
if (!is_compression && !has_none && post_processing_codecs.size() != codecs_descriptions->children.size())
/// It's okay to apply encryption codecs solely without anything else.
if (!is_compression && !has_none && encryption_codecs.size() != codecs_descriptions->children.size())
throw Exception(
"Compression codec " + codec_description
+ " does not compress anything."
@ -168,10 +168,10 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).",
ErrorCodes::BAD_ARGUMENTS);
/// It does not make sense to apply any non-post-processing codecs
/// after post-processing one.
if (!post_processing_codecs.empty() &&
*post_processing_codecs.begin() != codecs_descriptions->children.size() - post_processing_codecs.size())
/// It does not make sense to apply any non-encryption codecs
/// after encryption one.
if (!encryption_codecs.empty() &&
*encryption_codecs.begin() != codecs_descriptions->children.size() - encryption_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any non-post-processing codecs after"
" post-processing ones. (Note: you can enable setting 'allow_suspicious_codecs'"
@ -180,7 +180,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(
/// It does not make sense to apply any transformations after generic compression algorithm
/// So, generic compression can be only one and only at the end.
if (generic_compression_codec_pos &&
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - post_processing_codecs.size())
*generic_compression_codec_pos != codecs_descriptions->children.size() - 1 - encryption_codecs.size())
throw Exception("The combination of compression codecs " + codec_description + " is meaningless,"
" because it does not make sense to apply any transformations after generic compression algorithm."
" (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS);

View File

@ -35,15 +35,16 @@ namespace DB
enum class CompressionMethodByte : uint8_t
{
NONE = 0x02,
LZ4 = 0x82,
ZSTD = 0x90,
Multiple = 0x91,
Delta = 0x92,
T64 = 0x93,
DoubleDelta = 0x94,
Gorilla = 0x95,
Encrypted = 0x96,
NONE = 0x02,
LZ4 = 0x82,
ZSTD = 0x90,
Multiple = 0x91,
Delta = 0x92,
T64 = 0x93,
DoubleDelta = 0x94,
Gorilla = 0x95,
AES_128_GCM_SIV = 0x96,
AES_256_GCM_SIV = 0x97
};
}

View File

@ -76,7 +76,7 @@ public:
virtual bool isGenericCompression() const = 0;
/// If it is a post-processing codec such as encryption. Usually it does not make sense to apply non-post-processing codecs after this.
virtual bool isPostProcessing() const { return false; }
virtual bool isEncryption() const { return false; }
/// It is a codec available only for evaluation purposes and not meant to be used in production.
/// It will not be allowed to use unless the user will turn off the safety switch.

View File

@ -1,49 +1,301 @@
#include <cstddef>
#include <cstdint>
#include <iostream>
#include <memory>
#include <string>
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionCodecEncrypted.h>
#include <IO/BufferWithOwnMemory.h>
#include <Poco/DOM/AutoPtr.h>
#include <Poco/DOM/Document.h>
#include <Poco/DOM/Element.h>
#include <Poco/DOM/Text.h>
#include <Poco/NumericString.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include "Common/Exception.h"
namespace DB
inline DB::CompressionCodecPtr getCompressionCodecEncrypted(DB::EncryptionMethod Method)
{
CompressionCodecPtr getCompressionCodecEncrypted(const std::string_view & master_key);
return std::make_shared<DB::CompressionCodecEncrypted>(Method);
}
constexpr size_t key_size = 20;
struct AuxiliaryRandomData
namespace
{
char key[key_size];
size_t decompressed_size;
using namespace Poco;
using namespace Poco::XML;
/*
Fuzzing data consists of:
first byte:
1) length of nonce is in data (for correct work with wrong data from user)
2) length of key is in data (for correct work with wrong data from user)
3) is 128 turn on
4) multiple keys for 128
5) is 256 turn on
6) multiple keys for 256
7) nonce is set
8) current_key is set
read_key() will cosist of following steps:
if (2):
read 4 byte to know size
if multiple_keys:
read id
else:
size is chosen correctly according to algorithm
read size_bytes as a key
next bytes will have information in this order:
if (3):
if (4):
read count
for i in range(count):
read_key()
else:
read_key()
if (7):
read_nonce (simillar to read_key)
if (8):
set current_key
same for AES_256_GCM_SIV with (5) and (6) instead of (3) and (4)
This class read data and generate xml documentation.
*/
class XMLGenerator
{
public:
XMLGenerator(const uint8_t * data, size_t& size);
/// Try to generate config from input data using algorithm, which is described before class declaration
void generate();
/// Size of part, which was used on generating config
size_t keySize() const;
/// Get config
const Poco::AutoPtr<Poco::Util::XMLConfiguration>& getResult() const;
/// If something happened in generator, it will be true
bool hasError() const;
private:
/// generate algorithm section with key and nonce
bool generateAlgorithmKeys(AutoPtr<Poco::XML::Element>& document_root, std::string name,
uint8_t mask_for_algo, uint8_t mask_for_multiple_keys);
/// move on count bytes stream and increase counter
/// returns false if some errors occuried
bool next(ssize_t count=1);
/// Create a key from data
ssize_t generateKey(std::string name, bool multiple=false);
const uint8_t * data;
size_t start_size;
size_t keys_size;
AutoPtr<Poco::XML::Document> xml_document;
AutoPtr<Poco::XML::Element> algo;
AutoPtr<Poco::Util::XMLConfiguration> conf;
uint8_t first_byte;
bool error;
};
XMLGenerator::XMLGenerator(const uint8_t * Data, size_t& Size): data(Data), start_size(Size),
conf(new Poco::Util::XMLConfiguration()), error(false) {}
size_t XMLGenerator::keySize() const { return keys_size; }
const Poco::AutoPtr<Poco::Util::XMLConfiguration>& XMLGenerator::getResult() const { return conf; }
bool XMLGenerator::hasError() const { return error; }
bool XMLGenerator::next(ssize_t count)
{
/// If negative step - something went wrong
if (count == -1)
{
error = true;
return false;
}
/// move data and increase counter
keys_size += count;
/// If get after eof
if (keys_size >= start_size)
{
error = true;
return false;
}
data += count;
return true;
}
/*
<Key>key</key>
or
<key id=..>key</key>
*/
ssize_t XMLGenerator::generateKey(std::string name, bool multiple)
{
/// set traditional key size for algorithms
uint64_t size = 0;
if (name == "aes_128_gcm_siv")
size = 16;
if (name == "aes_256_gcm_siv")
size = 32;
/// try to read size from data
if (first_byte & 0x40)
{
size = *(reinterpret_cast<const uint64_t*>(data));
if (!next(8))
return -1;
}
/// if it is not defined, leave
if (!size)
return -1;
AutoPtr<Poco::XML::Element> key_holder;
if (multiple)
{
/// multiple keys have ids.
uint64_t id = *(reinterpret_cast<const uint64_t*>(data));
if (!next(8))
return -1;
key_holder = xml_document->createElement("key[id=" + std::to_string(id) + "]");
}
else
{
key_holder = xml_document->createElement("key");
}
AutoPtr<Text> key(xml_document->createTextNode(std::string(data, data + size)));
key_holder->appendChild(key);
algo->appendChild(key_holder);
if (!next(size))
return -1;
return size;
}
bool XMLGenerator::generateAlgorithmKeys(
AutoPtr<Poco::XML::Element>& document_root, std::string name, uint8_t mask_for_algo, uint8_t mask_for_multiple_keys)
{
/// check if algorithm is enabled, then add multiple keys or single key
if (first_byte & mask_for_algo)
{
algo = xml_document->createElement(name);
document_root->appendChild(algo);
if (first_byte & mask_for_multiple_keys)
{
uint64_t count = *(reinterpret_cast<const uint64_t*>(data));
if (!next(8))
return false;
for (size_t i = 0; i < count; ++i)
{
if (!next(generateKey(name)))
return false;
}
}
else
{
if (!next(generateKey(name)))
return false;
}
}
/// add nonce
if (first_byte & 0x02)
{
uint64_t nonce_size = 12;
if (first_byte & 0x80)
{
nonce_size = *(reinterpret_cast<const uint64_t*>(data));
if (!next(8))
return false;
}
AutoPtr<Poco::XML::Element> nonce_holder(xml_document->createElement("nonce"));
AutoPtr<Text> nonce(xml_document->createTextNode(std::string(data, data + nonce_size)));
nonce_holder->appendChild(nonce);
algo->appendChild(nonce_holder);
}
/// add current key id
if (first_byte & 0x01)
{
uint64_t current_key = *(reinterpret_cast<const uint64_t*>(data));
if (!next(8))
return false;
AutoPtr<Poco::XML::Element> cur_key_holder(xml_document->createElement("nonce"));
AutoPtr<Text> cur_key(xml_document->createTextNode(std::to_string(current_key)));
cur_key_holder->appendChild(cur_key);
algo->appendChild(cur_key_holder);
}
return true;
}
void XMLGenerator::generate()
{
AutoPtr<Poco::XML::Element> document_root(xml_document->createElement("encryption_codecs"));
xml_document->appendChild(document_root);
/// read first byte for parsing
first_byte = *data;
if (!next())
return;
if (!generateAlgorithmKeys(document_root, "aes_128_gmc_siv", 0x20, 0x10))
return;
if (!generateAlgorithmKeys(document_root, "aes_256_gmc_siv", 0x08, 0x04))
return;
conf->load(xml_document);
}
}
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
try
{
if (size < sizeof(AuxiliaryRandomData))
XMLGenerator generator(data, size);
generator.generate();
if (generator.hasError())
return 0;
const auto * p = reinterpret_cast<const AuxiliaryRandomData *>(data);
auto config = generator.getResult();
auto codec_128 = getCompressionCodecEncrypted(DB::AES_128_GCM_SIV);
auto codec_256 = getCompressionCodecEncrypted(DB::AES_256_GCM_SIV);
DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "");
std::string key = std::string(p->key, key_size);
auto codec = DB::getCompressionCodecEncrypted(key);
size_t data_size = size - generator.keySize();
size_t output_buffer_size = p->decompressed_size % 65536;
size -= sizeof(AuxiliaryRandomData);
data += sizeof(AuxiliaryRandomData) / sizeof(uint8_t);
std::string input = std::string(reinterpret_cast<const char*>(data), size);
fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, size, output_buffer_size);
if (output_buffer_size < size)
return 0;
std::string input = std::string(reinterpret_cast<const char*>(data), data_size);
fmt::print(stderr, "Using input {} of size {}, output size is {}. \n", input, data_size, input.size() - 31);
DB::Memory<> memory;
memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer());
codec->doDecompressData(reinterpret_cast<const char *>(data), size, memory.data(), output_buffer_size);
memory.resize(input.size() + codec_128->getAdditionalSizeAtTheEndOfBuffer());
codec_128->doDecompressData(input.data(), input.size(), memory.data(), input.size() - 31);
memory.resize(input.size() + codec_128->getAdditionalSizeAtTheEndOfBuffer());
codec_256->doDecompressData(input.data(), input.size(), memory.data(), input.size() - 31);
return 0;
}
catch (...)

View File

@ -466,6 +466,7 @@ class IColumn;
M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \
M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \
M(Bool, insert_null_as_default, true, "Insert DEFAULT values instead of NULL in INSERT SELECT (UNION ALL)", 0) \
M(Bool, describe_include_subcolumns, false, "If true, subcolumns of all table columns will be included into result of DESCRIBE query", 0) \
\
M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \
M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \
@ -600,6 +601,15 @@ class IColumn;
M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \
M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \
M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \
\
M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \
M(Bool, async_insert_mode, false, "Insert query is processed almost instantly, but an actual data queued for later asynchronous insertion", 0) \
M(Bool, wait_for_async_insert, true, "If true wait for processing of asynchronous insertion", 0) \
M(Seconds, wait_for_async_insert_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "Timeout for waiting for processing asynchronous insertion", 0) \
M(UInt64, async_insert_max_data_size, 1000000, "Maximum size in bytes of unparsed data collected per query before being inserted", 0) \
M(Milliseconds, async_insert_busy_timeout, 200, "Maximum time to wait before dumping collected data per query since the first data appeared", 0) \
M(Milliseconds, async_insert_stale_timeout, 0, "Maximum time to wait before dumping collected data per query since the last data appeared. Zero means no timeout at all", 0) \
\
M(Bool, cross_to_inner_join_rewrite, true, "Use inner join instead of comma/cross join if possible", 0) \
\
M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \

View File

@ -9,11 +9,10 @@ namespace DB
/// Proxy class which counts number of written block, rows, bytes
class CountingBlockOutputStream : public IBlockOutputStream
class CountingBlockOutputStream final : public IBlockOutputStream
{
public:
CountingBlockOutputStream(const BlockOutputStreamPtr & stream_)
: stream(stream_) {}
explicit CountingBlockOutputStream(const BlockOutputStreamPtr & stream_) : stream(stream_) {}
void setProgressCallback(const ProgressCallback & callback)
{

View File

@ -37,7 +37,7 @@ class IBlockInputStream : public TypePromotion<IBlockInputStream>
public:
IBlockInputStream() { info.parent = this; }
virtual ~IBlockInputStream() {}
virtual ~IBlockInputStream() = default;
IBlockInputStream(const IBlockInputStream &) = delete;
IBlockInputStream & operator=(const IBlockInputStream &) = delete;

View File

@ -21,7 +21,7 @@ struct Progress;
class IBlockOutputStream : private boost::noncopyable
{
public:
IBlockOutputStream() {}
IBlockOutputStream() = default;
/** Get data structure of the stream in a form of "header" block (it is also called "sample block").
* Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values.

View File

@ -87,23 +87,38 @@ ColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, const IColumn &
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName());
}
Names IDataType::getSubcolumnNames() const
void IDataType::forEachSubcolumn(const SubcolumnCallback & callback) const
{
NameSet res;
getDefaultSerialization()->enumerateStreams([&res, this](const ISerialization::SubstreamPath & substream_path)
NameSet set;
getDefaultSerialization()->enumerateStreams([&, this](const ISerialization::SubstreamPath & substream_path)
{
ISerialization::SubstreamPath new_path;
/// Iterate over path to try to get intermediate subcolumns for complex nested types.
for (const auto & elem : substream_path)
{
new_path.push_back(elem);
auto subcolumn_name = ISerialization::getSubcolumnNameForStream(new_path);
if (!subcolumn_name.empty() && tryGetSubcolumnType(subcolumn_name))
res.insert(subcolumn_name);
auto name = ISerialization::getSubcolumnNameForStream(new_path);
auto type = tryGetSubcolumnType(name);
/// Subcolumn names may repeat among several substream paths.
if (!name.empty() && type && !set.count(name))
{
callback(name, type, substream_path);
set.insert(name);
}
}
});
}
return Names(std::make_move_iterator(res.begin()), std::make_move_iterator(res.end()));
Names IDataType::getSubcolumnNames() const
{
Names res;
forEachSubcolumn([&](const auto & name, const auto &, const auto &)
{
res.push_back(name);
});
return res;
}
void IDataType::insertDefaultInto(IColumn & column) const

View File

@ -80,6 +80,9 @@ public:
virtual DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const;
DataTypePtr getSubcolumnType(const String & subcolumn_name) const;
virtual ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const;
using SubcolumnCallback = std::function<void(const String &, const DataTypePtr &, const ISerialization::SubstreamPath &)>;
void forEachSubcolumn(const SubcolumnCallback & callback) const;
Names getSubcolumnNames() const;
/// Returns default serialization of data type.

View File

@ -181,9 +181,9 @@ Names getPrimaryKeyColumns(const ASTExpressionList * primary_key)
Names result;
const auto & children = primary_key->children;
for (size_t index = 0; index != children.size(); ++index)
for (const auto & child : children)
{
const ASTIdentifier * key_part = children[index]->as<const ASTIdentifier>();
const ASTIdentifier * key_part = child->as<const ASTIdentifier>();
result.push_back(key_part->name());
}
return result;
@ -402,9 +402,9 @@ void buildConfigurationFromFunctionWithKeyValueArguments(
ContextPtr context)
{
const auto & children = ast_expr_list->children;
for (size_t i = 0; i != children.size(); ++i)
for (const auto & child : children)
{
const ASTPair * pair = children[i]->as<const ASTPair>();
const ASTPair * pair = child->as<const ASTPair>();
AutoPtr<Element> current_xml_element(doc->createElement(pair->first));
root->appendChild(current_xml_element);

View File

@ -1,32 +1,21 @@
#include "DiskWebServer.h"
#include <common/logger_useful.h>
#include <Common/escapeForFileName.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Disks/IDiskRemote.h>
#include <Disks/ReadIndirectBufferFromRemoteFS.h>
#include <Disks/ReadIndirectBufferFromWebServer.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/SeekAvoidingReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Disks/ReadIndirectBufferFromRemoteFS.h>
#include <Disks/IDiskRemote.h>
#include <Access/AccessControlManager.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Poco/Exception.h>
#include <re2/re2.h>
#define UUID_PATTERN "[\\w]{8}-[\\w]{4}-[\\w]{4}-[\\w]{4}-[\\w]{12}"
#define EXTRACT_UUID_PATTERN fmt::format(".*/({})/.*", UUID_PATTERN)
#define DIRECTORY_FILE_PATTERN(prefix) fmt::format("{}-({})-(\\w+)-(.*)", prefix, UUID_PATTERN)
#define ROOT_FILE_PATTERN(prefix) fmt::format("{}-({})-(\\w+\\.\\w+)", prefix, UUID_PATTERN)
#define MATCH_DIRECTORY_FILE_PATTERN fmt::format(".*/({})/(\\w+)/(.*)", UUID_PATTERN)
#define MATCH_DIRECTORY_PATTERN fmt::format(".*/({})/(\\w+)/", UUID_PATTERN)
#define MATCH_ROOT_FILE_PATTERN fmt::format(".*/({})/(\\w+\\.\\w+)", UUID_PATTERN)
namespace DB
{
@ -34,87 +23,85 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int FILE_DOESNT_EXIST;
extern const int DIRECTORY_DOESNT_EXIST;
extern const int NETWORK_ERROR;
extern const int NOT_IMPLEMENTED;
}
void DiskWebServer::Metadata::initialize(const String & uri_with_path, const String & files_prefix, const String & table_uuid, ContextPtr context) const
void DiskWebServer::initialize(const String & uri_path) const
{
ReadWriteBufferFromHTTP metadata_buf(Poco::URI(fs::path(uri_with_path) / (".index-" + table_uuid)),
Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(),
ConnectionTimeouts::getHTTPTimeouts(context));
String uuid, directory, file, remote_file_name;
size_t file_size;
while (!metadata_buf.eof())
std::vector<String> directories_to_load;
LOG_TRACE(log, "Loading metadata for directory: {}", uri_path);
try
{
readText(remote_file_name, metadata_buf);
assertChar('\t', metadata_buf);
readIntText(file_size, metadata_buf);
assertChar('\n', metadata_buf);
LOG_DEBUG(&Poco::Logger::get("DiskWeb"), "Read file: {}, size: {}", remote_file_name, file_size);
ReadWriteBufferFromHTTP metadata_buf(Poco::URI(fs::path(uri_path) / ".index"),
Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(),
ConnectionTimeouts::getHTTPTimeouts(getContext()));
String file_name;
FileData file_data;
/*
* URI/ {prefix}-{uuid}-all_x_x_x-{file}
* ...
* {prefix}-{uuid}-format_version.txt
* {prefix}-{uuid}-detached-{file}
* ...
**/
if (RE2::FullMatch(remote_file_name, DIRECTORY_FILE_PATTERN(files_prefix), &uuid, &directory, &file))
String dir_name = fs::path(uri_path.substr(url.size())) / "";
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Adding directory: {}", dir_name);
while (!metadata_buf.eof())
{
if (uuid != table_uuid)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected uuid: {}, expected: {}", uuid, table_uuid);
readText(file_name, metadata_buf);
assertChar('\t', metadata_buf);
tables_data[uuid][directory].emplace(std::make_pair(file, file_size));
}
else if (RE2::FullMatch(remote_file_name, ROOT_FILE_PATTERN(files_prefix), &uuid, &file))
{
if (uuid != table_uuid)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected uuid: {}, expected: {}", uuid, table_uuid);
bool is_directory;
readBoolText(is_directory, metadata_buf);
if (!is_directory)
{
assertChar('\t', metadata_buf);
readIntText(file_data.size, metadata_buf);
}
assertChar('\n', metadata_buf);
tables_data[uuid][file].emplace(std::make_pair(file, file_size));
file_data.type = is_directory ? FileType::Directory : FileType::File;
String file_path = fs::path(uri_path) / file_name;
if (file_data.type == FileType::Directory)
{
directories_to_load.push_back(file_path);
// file_path = fs::path(file_path) / "";
}
file_path = file_path.substr(url.size());
files.emplace(std::make_pair(file_path, file_data));
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Adding file: {}, size: {}", file_path, file_data.size);
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected file: {}", remote_file_name);
files.emplace(std::make_pair(dir_name, FileData({ .type = FileType::Directory })));
}
catch (Exception & e)
{
e.addMessage("while loading disk metadata");
throw;
}
for (const auto & directory_path : directories_to_load)
initialize(directory_path);
}
template <typename Directory>
class DiskWebDirectoryIterator final : public IDiskDirectoryIterator
class DiskWebServerDirectoryIterator final : public IDiskDirectoryIterator
{
public:
DiskWebDirectoryIterator(Directory & directory_, const String & directory_root_)
: directory(directory_), iter(directory.begin()), directory_root(directory_root_)
{
}
explicit DiskWebServerDirectoryIterator(std::vector<fs::path> && dir_file_paths_)
: dir_file_paths(std::move(dir_file_paths_)), iter(dir_file_paths.begin()) {}
void next() override { ++iter; }
bool isValid() const override
{
return iter != directory.end();
}
bool isValid() const override { return iter != dir_file_paths.end(); }
String path() const override
{
return fs::path(directory_root) / name();
}
String path() const override { return iter->string(); }
String name() const override
{
return iter->first;
}
String name() const override { return iter->filename(); }
private:
Directory & directory;
typename Directory::iterator iter;
const String directory_root;
std::vector<fs::path> dir_file_paths;
std::vector<fs::path>::iterator iter;
};
@ -125,204 +112,149 @@ public:
const String & uri_,
RemoteMetadata metadata_,
ContextPtr context_,
size_t max_read_tries_,
size_t buf_size_)
: ReadIndirectBufferFromRemoteFS<ReadIndirectBufferFromWebServer>(metadata_)
, uri(uri_)
, context(context_)
, max_read_tries(max_read_tries_)
, buf_size(buf_size_)
{
}
std::unique_ptr<ReadIndirectBufferFromWebServer> createReadBuffer(const String & path) override
{
return std::make_unique<ReadIndirectBufferFromWebServer>(fs::path(uri) / path, context, max_read_tries, buf_size);
return std::make_unique<ReadIndirectBufferFromWebServer>(fs::path(uri) / path, context, buf_size);
}
private:
String uri;
ContextPtr context;
size_t max_read_tries;
size_t buf_size;
};
class WriteBufferFromNothing : public WriteBufferFromFile
{
public:
WriteBufferFromNothing() : WriteBufferFromFile("/dev/null") {}
void sync() override {}
};
DiskWebServer::DiskWebServer(
const String & disk_name_,
const String & uri_,
const String & metadata_path_,
const String & url_,
ContextPtr context_,
SettingsPtr settings_)
size_t min_bytes_for_seek_)
: WithContext(context_->getGlobalContext())
, log(&Poco::Logger::get("DiskWeb"))
, uri(uri_)
, url(url_)
, name(disk_name_)
, metadata_path(metadata_path_)
, settings(std::move(settings_))
, min_bytes_for_seek(min_bytes_for_seek_)
{
}
String DiskWebServer::getFileName(const String & path) const
bool DiskWebServer::exists(const String & path) const
{
String result;
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Checking existence of path: {}", path);
if (RE2::FullMatch(path, MATCH_DIRECTORY_FILE_PATTERN)
&& RE2::Extract(path, MATCH_DIRECTORY_FILE_PATTERN, fmt::format(R"({}-\1-\2-\3)", settings->files_prefix), &result))
return result;
if (RE2::FullMatch(path, MATCH_ROOT_FILE_PATTERN)
&& RE2::Extract(path, MATCH_ROOT_FILE_PATTERN, fmt::format(R"({}-\1-\2)", settings->files_prefix), &result))
return result;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected file: {}", path);
}
bool DiskWebServer::findFileInMetadata(const String & path, File & file_info) const
{
String table_uuid, directory_name, file_name;
if (RE2::FullMatch(path, MATCH_DIRECTORY_FILE_PATTERN, &table_uuid, &directory_name, &file_name)
|| RE2::FullMatch(path, MATCH_ROOT_FILE_PATTERN, &table_uuid, &file_name)
|| RE2::FullMatch(path, MATCH_DIRECTORY_PATTERN, &table_uuid, &directory_name))
{
if (directory_name.empty())
directory_name = file_name;
if (!metadata.tables_data.count(table_uuid))
return false;
if (!metadata.tables_data[table_uuid].count(directory_name))
return false;
if (file_name.empty())
return true;
const auto & files = metadata.tables_data[table_uuid][directory_name];
auto file = files.find(file_name);
if (file == files.end())
return false;
file_info = File(file->first, file->second);
if (files.find(path) != files.end())
return true;
if (path.ends_with(MergeTreeData::FORMAT_VERSION_FILE_NAME) && files.find(fs::path(path).parent_path() / "") == files.end())
{
try
{
initialize(fs::path(url) / fs::path(path).parent_path());
return files.find(path) != files.end();
}
catch (...)
{
const auto message = getCurrentExceptionMessage(false);
bool can_throw = CurrentThread::isInitialized() && CurrentThread::get().getQueryContext();
if (can_throw)
throw Exception(ErrorCodes::NETWORK_ERROR, "Cannot load disk metadata. Error: {}", message);
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Cannot load disk metadata. Error: {}", message);
return false;
}
}
return false;
}
bool DiskWebServer::exists(const String & path) const
{
LOG_DEBUG(log, "Checking existence of file: {}", path);
File file;
return findFileInMetadata(path, file);
}
std::unique_ptr<ReadBufferFromFileBase> DiskWebServer::readFile(const String & path, const ReadSettings & read_settings, size_t) const
{
LOG_TRACE(log, "Read from path: {}", path);
auto iter = files.find(path);
if (iter == files.end())
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File path {} does not exist", path);
File file;
if (!findFileInMetadata(path, file))
throw Exception(ErrorCodes::LOGICAL_ERROR, "File {} not found", path);
auto fs_path = fs::path(url) / path;
auto remote_path = fs_path.parent_path() / (escapeForFileName(fs_path.stem()) + fs_path.extension().string());
remote_path = remote_path.string().substr(url.size());
auto file_name = escapeForFileName(fs::path(path).stem()) + fs::path(path).extension().string();
auto remote_path = fs::path(path).parent_path() / file_name;
LOG_DEBUG(log, "Read from file by path: {}", remote_path.string());
RemoteMetadata meta(path, remote_path);
meta.remote_fs_objects.emplace_back(std::make_pair(remote_path, iter->second.size));
RemoteMetadata meta(uri, remote_path);
meta.remote_fs_objects.emplace_back(std::make_pair(getFileName(remote_path), file.size));
auto reader = std::make_unique<ReadBufferFromWebServer>(uri, meta, getContext(), settings->max_read_tries, read_settings.remote_fs_buffer_size);
return std::make_unique<SeekAvoidingReadBuffer>(std::move(reader), settings->min_bytes_for_seek);
}
std::unique_ptr<WriteBufferFromFileBase> DiskWebServer::writeFile(const String & path, size_t, WriteMode)
{
if (path.ends_with("format_version.txt"))
return std::make_unique<WriteBufferFromNothing>();
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk {} is read-only", getName());
auto reader = std::make_unique<ReadBufferFromWebServer>(url, meta, getContext(), read_settings.remote_fs_buffer_size);
return std::make_unique<SeekAvoidingReadBuffer>(std::move(reader), min_bytes_for_seek);
}
DiskDirectoryIteratorPtr DiskWebServer::iterateDirectory(const String & path)
{
LOG_DEBUG(log, "Iterate directory: {}", path);
String uuid;
if (RE2::FullMatch(path, ".*/store/"))
return std::make_unique<DiskWebDirectoryIterator<UUIDDirectoryListing>>(metadata.tables_data, path);
if (!RE2::Extract(path, EXTRACT_UUID_PATTERN, "\\1", &uuid))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot extract uuid for: {}", path);
/// Do not throw if it is not a query, but disk load.
bool can_throw = CurrentThread::isInitialized() && CurrentThread::get().getQueryContext();
try
std::vector<fs::path> dir_file_paths;
if (files.find(path) == files.end())
{
if (!metadata.tables_data.count(uuid))
metadata.initialize(uri, settings->files_prefix, uuid, getContext());
}
catch (const Poco::Exception &)
{
const auto message = getCurrentExceptionMessage(false);
if (can_throw)
try
{
throw Exception(ErrorCodes::NETWORK_ERROR, "Cannot load disk metadata. Error: {}", message);
initialize(fs::path(url) / path);
}
catch (...)
{
const auto message = getCurrentExceptionMessage(false);
bool can_throw = CurrentThread::isInitialized() && CurrentThread::get().getQueryContext();
if (can_throw)
throw Exception(ErrorCodes::NETWORK_ERROR, "Cannot load disk metadata. Error: {}", message);
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Cannot load disk metadata. Error: {}", message);
/// Empty iterator.
return std::make_unique<DiskWebDirectoryIterator<RootDirectoryListing>>(metadata.tables_data[""], path);
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Cannot load disk metadata. Error: {}", message);
return std::make_unique<DiskWebServerDirectoryIterator>(std::move(dir_file_paths));
}
}
String directory_name;
if (RE2::FullMatch(path, MATCH_DIRECTORY_PATTERN, &uuid, &directory_name))
{
if (metadata.tables_data[uuid].contains(directory_name))
return std::make_unique<DiskWebDirectoryIterator<DirectoryListing>>(metadata.tables_data[uuid][directory_name], path);
if (can_throw)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Directory {} does not exist. (uuid: {})", directory_name, uuid);
return std::make_unique<DiskWebDirectoryIterator<RootDirectoryListing>>(metadata.tables_data[""], path); /// Empty directory.
}
if (files.find(path) == files.end())
throw Exception("Directory '" + path + "' does not exist", ErrorCodes::DIRECTORY_DOESNT_EXIST);
return std::make_unique<DiskWebDirectoryIterator<RootDirectoryListing>>(metadata.tables_data[uuid], path);
for (const auto & file : files)
if (parentPath(file.first) == path)
dir_file_paths.emplace_back(file.first);
LOG_TRACE(log, "Iterate directory {} with {} files", path, dir_file_paths.size());
return std::make_unique<DiskWebServerDirectoryIterator>(std::move(dir_file_paths));
}
size_t DiskWebServer::getFileSize(const String & path) const
{
File file;
if (!findFileInMetadata(path, file))
throw Exception(ErrorCodes::LOGICAL_ERROR, "File {} not found", path);
return file.size;
auto iter = files.find(path);
if (iter == files.end())
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File path {} does not exist", path);
return iter->second.size;
}
bool DiskWebServer::isFile(const String & path) const
{
return RE2::FullMatch(path, ".*/\\w+.\\w+");
auto iter = files.find(path);
if (iter == files.end())
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File path {} does not exist", path);
return iter->second.type == FileType::File;
}
bool DiskWebServer::isDirectory(const String & path) const
{
return RE2::FullMatch(path, ".*/\\w+");
auto iter = files.find(path);
if (iter == files.end())
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File path {} does not exist", path);
return iter->second.type == FileType::Directory;
}
@ -337,15 +269,16 @@ void registerDiskWebServer(DiskFactory & factory)
String uri{config.getString(config_prefix + ".endpoint")};
if (!uri.ends_with('/'))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "URI must end with '/', but '{}' doesn't.", uri);
try
{
Poco::URI poco_uri(uri);
}
catch (const Poco::Exception & e)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad URI: `{}`. Error: {}", uri, e.what());
}
auto settings = std::make_unique<DiskWebServerSettings>(
context->getGlobalContext()->getSettingsRef().http_max_single_read_retries,
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
config.getString(config_prefix + ".files_prefix", disk_name));
String metadata_path = fs::path(context->getPath()) / "disks" / disk_name / "";
return std::make_shared<DiskWebServer>(disk_name, uri, metadata_path, context, std::move(settings));
return std::make_shared<DiskWebServer>(disk_name, uri, context, config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024));
};
factory.registerDiskType("web", creator);

View File

@ -13,31 +13,14 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
struct DiskWebServerSettings
{
/// Number of read attempts before throw that network is unreachable.
size_t max_read_tries;
/// Passed to SeekAvoidingReadBuffer.
size_t min_bytes_for_seek;
String files_prefix;
DiskWebServerSettings(size_t max_read_tries_, size_t min_bytes_for_seek_, String files_prefix_)
: max_read_tries(max_read_tries_) , min_bytes_for_seek(min_bytes_for_seek_), files_prefix(files_prefix_) {}
};
/*
* Quick ready test - you can try this disk, by using these queries (disk has two tables) and this endpoint:
*
* ATTACH TABLE contributors UUID 'a563f7d8-fb00-4d50-a563-f7d8fb007d50' (good_person_name String) engine=MergeTree() order by good_person_name settings storage_policy='web';
* ATTACH TABLE test UUID '11c7a2f9-a949-4c88-91c7-a2f9a949ec88' (a Int32) engine=MergeTree() order by a settings storage_policy='web';
* Quick ready test: ATTACH TABLE test_hits UUID '1ae36516-d62d-4218-9ae3-6516d62da218' ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS storage_policy='web';
*
* <storage_configuration>
* <disks>
* <web>
* <type>web</type>
* <endpoint>https://clickhouse-datasets.s3.yandex.net/kssenii-static-files-disk-test/kssenii-disk-tests/test1/</endpoint>
* <files_prefix>data</files_prefix>
* <endpoint>https://clickhouse-datasets.s3.yandex.net/disk-with-static-files-tests/test-hits/</endpoint>
* </web>
* </disks>
* <policies>
@ -51,64 +34,29 @@ struct DiskWebServerSettings
* </policies>
* </storage_configuration>
*
* If query fails with `DB:Exception Unreachable URL` -- may help to adjust settings: http_connection_timeout, http_receive_timeout, keep_alive_timeout.
*
* To get files for upload run:
* clickhouse static-files-disk-uploader --metadata-path <path> --output-dir <dir> --files-prefix data
* clickhouse static-files-disk-uploader --metadata-path <path> --output-dir <dir>
* (--metadata-path can be found in query: `select data_paths from system.tables where name='<table_name>';`)
*
* When loading files by <endpoint> they must be loaded into <endpoint>/store/ path, but config must conrain only <endpoint>.
*
* If url is not reachable on disk load when server is starting up tables, then all errors are caught.
* If in this case there were errors, tables can be reloaded (become visible) via detach table table_name -> attach table table_name.
* If metadata was successfully loaded at server startup, then tables are available straight away.
**/
class DiskWebServer : public IDisk, WithContext
{
using SettingsPtr = std::unique_ptr<DiskWebServerSettings>;
public:
DiskWebServer(const String & disk_name_,
const String & files_root_path_uri_,
const String & metadata_path_,
const String & url_,
ContextPtr context,
SettingsPtr settings_);
struct File
{
String name;
size_t size;
File(const String & name_ = "", const size_t size_ = 0) : name(name_), size(size_) {}
};
using Directory = std::unordered_map<String, size_t>;
/* Each root directory contains either directories like
* all_x_x_x/{file}, detached/, etc, or root files like format_version.txt.
*/
using RootDirectory = std::unordered_map<String, Directory>;
/* Each table is attached via ATTACH TABLE table UUID <uuid> <def>.
* Then there is a mapping: {table uuid} -> {root directory}
*/
using TableDirectories = std::unordered_map<String, RootDirectory>;
struct Metadata
{
/// Fetch meta only when required.
mutable TableDirectories tables_data;
Metadata() = default;
void initialize(const String & uri_with_path, const String & files_prefix, const String & uuid, ContextPtr context) const;
};
using UUIDDirectoryListing = std::unordered_map<String, RootDirectory>;
using RootDirectoryListing = std::unordered_map<String, Directory>;
using DirectoryListing = std::unordered_map<String, size_t>;
bool findFileInMetadata(const String & path, File & file_info) const;
size_t min_bytes_for_seek_);
bool supportZeroCopyReplication() const override { return false; }
String getFileName(const String & path) const;
DiskType getType() const override { return DiskType::WebServer; }
bool isRemote() const override { return true; }
@ -121,7 +69,7 @@ public:
const String & getName() const final override { return name; }
const String & getPath() const final override { return metadata_path; }
const String & getPath() const final override { return url; }
bool isReadOnly() const override { return true; }
@ -151,7 +99,10 @@ public:
/// Write and modification part
std::unique_ptr<WriteBufferFromFileBase> writeFile(const String &, size_t, WriteMode) override;
std::unique_ptr<WriteBufferFromFileBase> writeFile(const String &, size_t, WriteMode) override
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk {} is read-only", getName());
}
void moveFile(const String &, const String &) override
{
@ -224,13 +175,28 @@ public:
void createHardLink(const String &, const String &) override {}
private:
void initialize(const String & uri_path) const;
enum class FileType
{
File,
Directory
};
struct FileData
{
FileType type;
size_t size;
};
using Files = std::unordered_map<String, FileData>; /// file path -> file data
mutable Files files;
Poco::Logger * log;
String uri, name;
const String metadata_path;
SettingsPtr settings;
String url;
String name;
Metadata metadata;
size_t min_bytes_for_seek;
};
}

View File

@ -1,9 +1,12 @@
#include "ReadIndirectBufferFromWebServer.h"
#include <common/logger_useful.h>
#include <common/sleep.h>
#include <Core/Types.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <thread>
@ -17,17 +20,17 @@ namespace ErrorCodes
extern const int NETWORK_ERROR;
}
static const auto WAIT_MS = 10;
static const auto WAIT_THRESHOLD_MS = 10000;
ReadIndirectBufferFromWebServer::ReadIndirectBufferFromWebServer(const String & url_,
ContextPtr context_,
size_t max_read_tries_,
size_t buf_size_)
: BufferWithOwnMemory<SeekableReadBuffer>(buf_size_)
, log(&Poco::Logger::get("ReadIndirectBufferFromWebServer"))
, context(context_)
, url(url_)
, buf_size(buf_size_)
, max_read_tries(max_read_tries_)
{
}
@ -38,13 +41,20 @@ std::unique_ptr<ReadBuffer> ReadIndirectBufferFromWebServer::initialize()
ReadWriteBufferFromHTTP::HTTPHeaderEntries headers;
headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-", offset)));
const auto & settings = context->getSettingsRef();
LOG_DEBUG(log, "Reading from offset: {}", offset);
const auto & config = context->getConfigRef();
Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 20), 0};
return std::make_unique<ReadWriteBufferFromHTTP>(
uri,
Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(),
ConnectionTimeouts::getHTTPTimeouts(context),
ConnectionTimeouts(std::max(Poco::Timespan(settings.http_connection_timeout.totalSeconds(), 0), Poco::Timespan(20, 0)),
settings.http_send_timeout,
std::max(Poco::Timespan(settings.http_receive_timeout.totalSeconds(), 0), Poco::Timespan(20, 0)),
settings.tcp_keep_alive_timeout,
http_keep_alive_timeout),
0,
Poco::Net::HTTPBasicCredentials{},
buf_size,
@ -55,6 +65,7 @@ std::unique_ptr<ReadBuffer> ReadIndirectBufferFromWebServer::initialize()
bool ReadIndirectBufferFromWebServer::nextImpl()
{
bool next_result = false, successful_read = false;
UInt16 milliseconds_to_wait = WAIT_MS;
if (impl)
{
@ -62,40 +73,38 @@ bool ReadIndirectBufferFromWebServer::nextImpl()
impl->position() = position();
assert(!impl->hasPendingData());
}
else
WriteBufferFromOwnString error_msg;
while (milliseconds_to_wait < WAIT_THRESHOLD_MS)
{
try
{
impl = initialize();
}
catch (const Poco::Exception & e)
{
throw Exception(ErrorCodes::NETWORK_ERROR, "Unreachable url: {}. Error: {}", url, e.what());
}
if (!impl)
{
impl = initialize();
next_result = impl->hasPendingData();
if (next_result)
break;
}
next_result = impl->hasPendingData();
}
for (size_t try_num = 0; (try_num < max_read_tries) && !next_result; ++try_num)
{
try
{
next_result = impl->next();
successful_read = true;
break;
}
catch (const Exception & e)
catch (const Poco::Exception & e)
{
LOG_WARNING(log, "Read attempt {}/{} failed from {}. ({})", try_num, max_read_tries, url, e.message());
LOG_WARNING(log, "Read attempt failed for url: {}. Error: {}", url, e.what());
error_msg << fmt::format("Error: {}\n", e.what());
sleepForMilliseconds(milliseconds_to_wait);
milliseconds_to_wait *= 2;
impl.reset();
impl = initialize();
next_result = impl->hasPendingData();
}
}
if (!successful_read)
throw Exception(ErrorCodes::NETWORK_ERROR, "All read attempts ({}) failed for uri: {}", max_read_tries, url);
throw Exception(ErrorCodes::NETWORK_ERROR,
"All read attempts failed for url: {}. Reason:\n{}", url, error_msg.str());
if (next_result)
{

View File

@ -18,7 +18,6 @@ class ReadIndirectBufferFromWebServer : public BufferWithOwnMemory<SeekableReadB
public:
explicit ReadIndirectBufferFromWebServer(const String & url_,
ContextPtr context_,
size_t max_read_tries_,
size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE);
bool nextImpl() override;
@ -34,7 +33,7 @@ private:
ContextPtr context;
const String url;
size_t buf_size, max_read_tries;
size_t buf_size;
std::unique_ptr<ReadBuffer> impl;

View File

@ -121,15 +121,14 @@ void registerFormats()
registerInputFormatProcessorRawBLOB(factory);
registerOutputFormatProcessorRawBLOB(factory);
#if !defined(ARCADIA_BUILD)
registerInputFormatProcessorORC(factory);
registerOutputFormatProcessorORC(factory);
registerInputFormatProcessorParquet(factory);
registerOutputFormatProcessorParquet(factory);
#if !defined(ARCADIA_BUILD)
registerInputFormatProcessorAvro(factory);
registerOutputFormatProcessorAvro(factory);
#endif
registerInputFormatProcessorArrow(factory);
registerOutputFormatProcessorArrow(factory);

View File

@ -72,6 +72,7 @@ struct DivideIntegralImpl
{
using ResultType = typename NumberTraits::ResultOfIntegerDivision<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -126,6 +127,7 @@ struct ModuloImpl
using IntegerBType = typename NumberTraits::ToInteger<B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -15,11 +15,13 @@
#include <DataTypes/DataTypeInterval.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/Native.h>
#include <DataTypes/NumberTraits.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnAggregateFunction.h>
#include "Core/DecimalFunctions.h"
@ -192,6 +194,7 @@ struct BinaryOperation
{
using ResultType = OpResultType;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <OpCase op_case>
static void NO_INLINE process(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size)
@ -208,6 +211,71 @@ struct BinaryOperation
static ResultType process(A a, B b) { return Op::template apply<ResultType>(a, b); }
};
template <typename B, typename Op>
struct StringIntegerOperationImpl
{
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = true;
template <OpCase op_case>
static void NO_INLINE processFixedString(const UInt8 * __restrict in_vec, const UInt64 n, const B * __restrict b, ColumnFixedString::Chars & out_vec, size_t size)
{
size_t prev_offset = 0;
out_vec.reserve(n * size);
for (size_t i = 0; i < size; ++i)
{
if constexpr (op_case == OpCase::LeftConstant)
{
Op::apply(&in_vec[0], &in_vec[n], b[i], out_vec);
}
else
{
size_t new_offset = prev_offset + n;
if constexpr (op_case == OpCase::Vector)
{
Op::apply(&in_vec[prev_offset], &in_vec[new_offset], b[i], out_vec);
}
else
{
Op::apply(&in_vec[prev_offset], &in_vec[new_offset], b[0], out_vec);
}
prev_offset = new_offset;
}
}
}
template <OpCase op_case>
static void NO_INLINE processString(const UInt8 * __restrict in_vec, const UInt64 * __restrict in_offsets, const B * __restrict b, ColumnString::Chars & out_vec, ColumnString::Offsets & out_offsets, size_t size)
{
size_t prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
if constexpr (op_case == OpCase::LeftConstant)
{
Op::apply(&in_vec[0], &in_vec[in_offsets[0] - 1], b[i], out_vec, out_offsets);
}
else
{
size_t new_offset = in_offsets[i];
if constexpr (op_case == OpCase::Vector)
{
Op::apply(&in_vec[prev_offset], &in_vec[new_offset - 1], b[i], out_vec, out_offsets);
}
else
{
Op::apply(&in_vec[prev_offset], &in_vec[new_offset - 1], b[0], out_vec, out_offsets);
}
prev_offset = new_offset;
}
}
}
};
template <typename Op>
struct FixedStringOperationImpl
{
@ -514,7 +582,7 @@ class FunctionBinaryArithmetic : public IFunction
DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256,
DataTypeDecimal32, DataTypeDecimal64, DataTypeDecimal128, DataTypeDecimal256,
DataTypeDate, DataTypeDateTime,
DataTypeFixedString>;
DataTypeFixedString, DataTypeString>;
using Floats = TypeList<DataTypeFloat32, DataTypeFloat64>;
@ -953,19 +1021,33 @@ public:
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> ||
std::is_same_v<DataTypeFixedString, RightDataType>)
if constexpr ((std::is_same_v<DataTypeFixedString, LeftDataType> || std::is_same_v<DataTypeString, LeftDataType>) ||
(std::is_same_v<DataTypeFixedString, RightDataType> || std::is_same_v<DataTypeString, RightDataType>))
{
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
return false;
else if constexpr (std::is_same_v<LeftDataType, RightDataType>)
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> &&
std::is_same_v<DataTypeFixedString, RightDataType>)
{
if (left.getN() == right.getN())
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
return false;
else
{
type_res = std::make_shared<LeftDataType>(left.getN());
return true;
if (left.getN() == right.getN())
{
type_res = std::make_shared<LeftDataType>(left.getN());
return true;
}
}
}
if constexpr (!Op<LeftDataType, RightDataType>::allow_string_integer)
return false;
else if constexpr (!IsIntegral<RightDataType>)
return false;
else if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType>)
type_res = std::make_shared<LeftDataType>(left.getN());
else
type_res = std::make_shared<DataTypeString>();
return true;
}
else
{
@ -1102,6 +1184,106 @@ public:
return nullptr;
}
template <typename LeftColumnType, typename A, typename B>
ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A & left, const B & right) const
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
const auto * const col_left_raw = arguments[0].column.get();
const auto * const col_right_raw = arguments[1].column.get();
using T1 = typename RightDataType::FieldType;
using ColVecT1 = ColumnVector<T1>;
const ColVecT1 * const col_right = checkAndGetColumn<ColVecT1>(col_right_raw);
const ColumnConst * const col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw);
using OpImpl = StringIntegerOperationImpl<T1, Op<LeftDataType, T1>>;
const ColumnConst * const col_left_const = checkAndGetColumnConst<LeftColumnType>(col_left_raw);
const auto * col_left = col_left_const ? checkAndGetColumn<LeftColumnType>(col_left_const->getDataColumn())
: checkAndGetColumn<LeftColumnType>(col_left_raw);
if (!col_left)
return nullptr;
const typename LeftColumnType::Chars & in_vec = col_left->getChars();
typename LeftColumnType::MutablePtr col_res;
if constexpr (std::is_same_v<LeftDataType, DataTypeFixedString>)
col_res = LeftColumnType::create(col_left->getN());
else
col_res = LeftColumnType::create();
typename LeftColumnType::Chars & out_vec = col_res->getChars();
if (col_left_const && col_right_const)
{
const T1 value = col_right_const->template getValue<T1>();
if constexpr (std::is_same_v<LeftDataType, DataTypeFixedString>)
{
OpImpl::template processFixedString<OpCase::Vector>(in_vec.data(), col_left->getN(), &value, out_vec, 1);
}
else
{
ColumnString::Offsets & out_offsets = col_res->getOffsets();
OpImpl::template processString<OpCase::Vector>(in_vec.data(), col_left->getOffsets().data(), &value, out_vec, out_offsets, 1);
}
return ColumnConst::create(std::move(col_res), col_left->size());
}
else if (!col_left_const && !col_right_const && col_right)
{
if constexpr (std::is_same_v<LeftDataType, DataTypeFixedString>)
{
OpImpl::template processFixedString<OpCase::Vector>(in_vec.data(), col_left->getN(), col_right->getData().data(), out_vec, col_left->size());
}
else
{
ColumnString::Offsets & out_offsets = col_res->getOffsets();
out_offsets.reserve(col_left->size());
OpImpl::template processString<OpCase::Vector>(
in_vec.data(), col_left->getOffsets().data(), col_right->getData().data(), out_vec, out_offsets, col_left->size());
}
}
else if (col_left_const && col_right)
{
if constexpr (std::is_same_v<LeftDataType, DataTypeFixedString>)
{
OpImpl::template processFixedString<OpCase::LeftConstant>(
in_vec.data(), col_left->getN(), col_right->getData().data(), out_vec, col_right->size());
}
else
{
ColumnString::Offsets & out_offsets = col_res->getOffsets();
out_offsets.reserve(col_right->size());
OpImpl::template processString<OpCase::LeftConstant>(
in_vec.data(), col_left->getOffsets().data(), col_right->getData().data(), out_vec, out_offsets, col_right->size());
}
}
else if (col_right_const)
{
const T1 value = col_right_const->template getValue<T1>();
if constexpr (std::is_same_v<LeftDataType, DataTypeFixedString>)
{
OpImpl::template processFixedString<OpCase::RightConstant>(in_vec.data(), col_left->getN(), &value, out_vec, col_left->size());
}
else
{
ColumnString::Offsets & out_offsets = col_res->getOffsets();
out_offsets.reserve(col_left->size());
OpImpl::template processString<OpCase::RightConstant>(
in_vec.data(), col_left->getOffsets().data(), &value, out_vec, out_offsets, col_left->size());
}
}
else
return nullptr;
return col_res;
}
template <typename A, typename B>
ColumnPtr executeNumeric(const ColumnsWithTypeAndName & arguments, const A & left, const B & right) const
{
@ -1228,13 +1410,28 @@ public:
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> ||
std::is_same_v<DataTypeFixedString, RightDataType>)
if constexpr ((std::is_same_v<DataTypeFixedString, LeftDataType> || std::is_same_v<DataTypeString, LeftDataType>) ||
(std::is_same_v<DataTypeFixedString, RightDataType> || std::is_same_v<DataTypeString, RightDataType>))
{
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> &&
std::is_same_v<DataTypeFixedString, RightDataType>)
{
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
return false;
else
return (res = executeFixedString(arguments)) != nullptr;
}
if constexpr (!Op<LeftDataType, RightDataType>::allow_string_integer)
return false;
else
return (res = executeFixedString(arguments)) != nullptr;
else if constexpr (!IsIntegral<RightDataType>)
return false;
else if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType>)
{
return (res = executeStringInteger<ColumnFixedString>(arguments, left, right)) != nullptr;
}
else if constexpr (std::is_same_v<DataTypeString, LeftDataType>)
return (res = executeStringInteger<ColumnString>(arguments, left, right)) != nullptr;
}
else
return (res = executeNumeric(arguments, left, right)) != nullptr;
@ -1264,7 +1461,7 @@ public:
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> || std::is_same_v<DataTypeFixedString, RightDataType>)
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> || std::is_same_v<DataTypeFixedString, RightDataType> || std::is_same_v<DataTypeString, LeftDataType> || std::is_same_v<DataTypeString, RightDataType>)
return false;
else
{
@ -1284,7 +1481,7 @@ public:
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (!std::is_same_v<DataTypeFixedString, LeftDataType> && !std::is_same_v<DataTypeFixedString, RightDataType>)
if constexpr (!std::is_same_v<DataTypeFixedString, LeftDataType> && !std::is_same_v<DataTypeFixedString, RightDataType> && !std::is_same_v<DataTypeString, LeftDataType> && !std::is_same_v<DataTypeString, RightDataType>)
{
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
using OpSpec = Op<typename LeftDataType::FieldType, typename RightDataType::FieldType>;

View File

@ -1,3 +1,7 @@
#if defined(__clang__) && __clang_major__ >= 13
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnFixedString.h>

View File

@ -25,6 +25,7 @@ struct GCDLCMImpl
{
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -106,7 +106,7 @@ String toString(TargetArch arch);
/* Clang shows warning when there aren't any objects to apply pragma.
* To prevent this warning we define this function inside every macros with pragmas.
*/
# define DUMMY_FUNCTION_DEFINITION [[maybe_unused]] void __dummy_function_definition();
# define DUMMY_FUNCTION_DEFINITION [[maybe_unused]] void _dummy_function_definition();
#else
# define BEGIN_AVX512F_SPECIFIC_CODE \
_Pragma("GCC push_options") \

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