Merge remote-tracking branch 'rschu1ze/master' into cgroup-observing-memory-tracker

This commit is contained in:
Robert Schulze 2024-02-14 08:56:22 +00:00
commit 7b7ea6d477
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
187 changed files with 2497 additions and 1474 deletions

View File

@ -97,6 +97,14 @@ jobs:
build_name: package_release
checkout_depth: 0
data: ${{ needs.RunConfig.outputs.data }}
BuilderDebReleaseCoverage:
needs: [RunConfig, BuildDockers]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_build.yml
with:
build_name: package_release_coverage
checkout_depth: 0
data: ${{ needs.RunConfig.outputs.data }}
BuilderDebAarch64:
needs: [RunConfig, BuildDockers]
if: ${{ !failure() && !cancelled() }}
@ -277,6 +285,7 @@ jobs:
- BuilderDebDebug
- BuilderDebMsan
- BuilderDebRelease
- BuilderDebReleaseCoverage
- BuilderDebTsan
- BuilderDebUBsan
uses: ./.github/workflows/reusable_test.yml
@ -365,6 +374,14 @@ jobs:
test_name: Stateless tests (release)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatelessTestCoverage:
needs: [RunConfig, BuilderDebReleaseCoverage]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_test.yml
with:
test_name: Stateless tests (coverage)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatelessTestReleaseDatabaseReplicated:
needs: [RunConfig, BuilderDebRelease]
if: ${{ !failure() && !cancelled() }}
@ -465,6 +482,14 @@ jobs:
test_name: Stateful tests (release)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatefulTestCoverage:
needs: [RunConfig, BuilderDebReleaseCoverage]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_test.yml
with:
test_name: Stateful tests (coverage)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatefulTestAarch64:
needs: [RunConfig, BuilderDebAarch64]
if: ${{ !failure() && !cancelled() }}

View File

@ -147,6 +147,14 @@ jobs:
build_name: package_release
checkout_depth: 0
data: ${{ needs.RunConfig.outputs.data }}
BuilderDebReleaseCoverage:
needs: [RunConfig, FastTest]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_build.yml
with:
build_name: package_release_coverage
checkout_depth: 0
data: ${{ needs.RunConfig.outputs.data }}
BuilderDebAarch64:
needs: [RunConfig, FastTest]
if: ${{ !failure() && !cancelled() }}
@ -309,6 +317,7 @@ jobs:
- BuilderDebDebug
- BuilderDebMsan
- BuilderDebRelease
- BuilderDebReleaseCoverage
- BuilderDebTsan
- BuilderDebUBsan
uses: ./.github/workflows/reusable_test.yml
@ -382,6 +391,14 @@ jobs:
test_name: Stateless tests (release)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatelessTestCoverage:
needs: [RunConfig, BuilderDebReleaseCoverage]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_test.yml
with:
test_name: Stateless tests (coverage)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatelessTestReleaseDatabaseReplicated:
needs: [RunConfig, BuilderDebRelease]
if: ${{ !failure() && !cancelled() }}
@ -509,6 +526,14 @@ jobs:
test_name: Stateful tests (release)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatefulTestCoverage:
needs: [RunConfig, BuilderDebReleaseCoverage]
if: ${{ !failure() && !cancelled() }}
uses: ./.github/workflows/reusable_test.yml
with:
test_name: Stateful tests (coverage)
runner_type: func-tester
data: ${{ needs.RunConfig.outputs.data }}
FunctionalStatefulTestAarch64:
needs: [RunConfig, BuilderDebAarch64]
if: ${{ !failure() && !cancelled() }}

View File

@ -355,7 +355,7 @@ if (COMPILER_CLANG)
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-absolute-paths")
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-absolute-paths")
if (NOT ENABLE_TESTS AND NOT SANITIZE AND OS_LINUX)
if (NOT ENABLE_TESTS AND NOT SANITIZE AND NOT SANITIZE_COVERAGE AND OS_LINUX)
# https://clang.llvm.org/docs/ThinLTO.html
# Applies to clang and linux only.
# Disabled when building with tests or sanitizers.
@ -553,7 +553,7 @@ if (ENABLE_RUST)
endif()
endif()
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND NOT SANITIZE AND OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64))
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND NOT SANITIZE AND NOT SANITIZE_COVERAGE AND OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64))
set(CHECK_LARGE_OBJECT_SIZES_DEFAULT ON)
else ()
set(CHECK_LARGE_OBJECT_SIZES_DEFAULT OFF)

View File

@ -1,4 +1,5 @@
#include "coverage.h"
#include <sys/mman.h>
#pragma GCC diagnostic ignored "-Wreserved-identifier"
@ -52,11 +53,21 @@ namespace
uint32_t * guards_start = nullptr;
uint32_t * guards_end = nullptr;
uintptr_t * coverage_array = nullptr;
uintptr_t * current_coverage_array = nullptr;
uintptr_t * cumulative_coverage_array = nullptr;
size_t coverage_array_size = 0;
uintptr_t * all_addresses_array = nullptr;
size_t all_addresses_array_size = 0;
uintptr_t * allocate(size_t size)
{
/// Note: mmap return zero-initialized memory, and we count on that.
void * map = mmap(nullptr, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == map)
return nullptr;
return static_cast<uintptr_t*>(map);
}
}
extern "C"
@ -79,7 +90,8 @@ void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop)
coverage_array_size = stop - start;
/// Note: we will leak this.
coverage_array = static_cast<uintptr_t*>(malloc(sizeof(uintptr_t) * coverage_array_size));
current_coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size);
cumulative_coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size);
resetCoverage();
}
@ -92,8 +104,8 @@ void __sanitizer_cov_pcs_init(const uintptr_t * pcs_begin, const uintptr_t * pcs
return;
pc_table_initialized = true;
all_addresses_array = static_cast<uintptr_t*>(malloc(sizeof(uintptr_t) * coverage_array_size));
all_addresses_array_size = pcs_end - pcs_begin;
all_addresses_array = allocate(sizeof(uintptr_t) * all_addresses_array_size);
/// They are not a real pointers, but also contain a flag in the most significant bit,
/// in which we are not interested for now. Reset it.
@ -115,17 +127,24 @@ void __sanitizer_cov_trace_pc_guard(uint32_t * guard)
/// The values of `*guard` are as you set them in
/// __sanitizer_cov_trace_pc_guard_init and so you can make them consecutive
/// and use them to dereference an array or a bit vector.
void * pc = __builtin_return_address(0);
intptr_t pc = reinterpret_cast<uintptr_t>(__builtin_return_address(0));
coverage_array[guard - guards_start] = reinterpret_cast<uintptr_t>(pc);
current_coverage_array[guard - guards_start] = pc;
cumulative_coverage_array[guard - guards_start] = pc;
}
}
__attribute__((no_sanitize("coverage")))
std::span<const uintptr_t> getCoverage()
std::span<const uintptr_t> getCurrentCoverage()
{
return {coverage_array, coverage_array_size};
return {current_coverage_array, coverage_array_size};
}
__attribute__((no_sanitize("coverage")))
std::span<const uintptr_t> getCumulativeCoverage()
{
return {cumulative_coverage_array, coverage_array_size};
}
__attribute__((no_sanitize("coverage")))
@ -137,7 +156,7 @@ std::span<const uintptr_t> getAllInstrumentedAddresses()
__attribute__((no_sanitize("coverage")))
void resetCoverage()
{
memset(coverage_array, 0, coverage_array_size * sizeof(*coverage_array));
memset(current_coverage_array, 0, coverage_array_size * sizeof(*current_coverage_array));
/// The guard defines whether the __sanitizer_cov_trace_pc_guard should be called.
/// For example, you can unset it after first invocation to prevent excessive work.

View File

@ -15,7 +15,10 @@ void dumpCoverageReportIfPossible();
/// Get accumulated unique program addresses of the instrumented parts of the code,
/// seen so far after program startup or after previous reset.
/// The returned span will be represented as a sparse map, containing mostly zeros, which you should filter away.
std::span<const uintptr_t> getCoverage();
std::span<const uintptr_t> getCurrentCoverage();
/// Similar but not being reset.
std::span<const uintptr_t> getCumulativeCoverage();
/// Get all instrumented addresses that could be in the coverage.
std::span<const uintptr_t> getAllInstrumentedAddresses();

View File

@ -63,14 +63,14 @@ endif()
option(WITH_COVERAGE "Instrumentation for code coverage with default implementation" OFF)
if (WITH_COVERAGE)
message (INFORMATION "Enabled instrumentation for code coverage")
message (STATUS "Enabled instrumentation for code coverage")
set(COVERAGE_FLAGS "-fprofile-instr-generate -fcoverage-mapping")
endif()
option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF)
if (SANITIZE_COVERAGE)
message (INFORMATION "Enabled instrumentation for code coverage")
message (STATUS "Enabled instrumentation for code coverage")
# We set this define for whole build to indicate that at least some parts are compiled with coverage.
# And to expose it in system.build_options.

View File

@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="24.1.2.5"
ARG VERSION="24.1.4.19"
ARG PACKAGES="clickhouse-keeper"
ARG DIRECT_DOWNLOAD_URLS=""

View File

@ -115,12 +115,17 @@ def run_docker_image_with_env(
subprocess.check_call(cmd, shell=True)
def is_release_build(debug_build: bool, package_type: str, sanitizer: str) -> bool:
return not debug_build and package_type == "deb" and sanitizer == ""
def is_release_build(
debug_build: bool, package_type: str, sanitizer: str, coverage: bool
) -> bool:
return (
not debug_build and package_type == "deb" and sanitizer == "" and not coverage
)
def parse_env_variables(
debug_build: bool,
coverage: bool,
compiler: str,
sanitizer: str,
package_type: str,
@ -261,7 +266,7 @@ def parse_env_variables(
build_target = (
f"{build_target} clickhouse-odbc-bridge clickhouse-library-bridge"
)
if is_release_build(debug_build, package_type, sanitizer):
if is_release_build(debug_build, package_type, sanitizer, coverage):
cmake_flags.append("-DSPLIT_DEBUG_SYMBOLS=ON")
result.append("WITH_PERFORMANCE=1")
if is_cross_arm:
@ -287,6 +292,9 @@ def parse_env_variables(
else:
result.append("BUILD_TYPE=None")
if coverage:
cmake_flags.append("-DSANITIZE_COVERAGE=1 -DBUILD_STANDALONE_KEEPER=0")
if not cache:
cmake_flags.append("-DCOMPILER_CACHE=disabled")
@ -415,6 +423,11 @@ def parse_args() -> argparse.Namespace:
choices=("address", "thread", "memory", "undefined", ""),
default="",
)
parser.add_argument(
"--coverage",
action="store_true",
help="enable granular coverage with introspection",
)
parser.add_argument("--clang-tidy", action="store_true")
parser.add_argument(
@ -507,6 +520,7 @@ def main() -> None:
env_prepared = parse_env_variables(
args.debug_build,
args.coverage,
args.compiler,
args.sanitizer,
args.package_type,

View File

@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \
# lts / testing / prestable / etc
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}"
ARG VERSION="24.1.2.5"
ARG VERSION="24.1.4.19"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
ARG DIRECT_DOWNLOAD_URLS=""

View File

@ -27,7 +27,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list
ARG REPO_CHANNEL="stable"
ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main"
ARG VERSION="24.1.2.5"
ARG VERSION="24.1.4.19"
ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static"
# set non-empty deb_location_url url to create a docker image

View File

@ -118,13 +118,19 @@ if [ -n "$CLICKHOUSE_USER" ] && [ "$CLICKHOUSE_USER" != "default" ] || [ -n "$CL
EOT
fi
CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS="${CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS:-}"
# checking $DATA_DIR for initialization
if [ -d "${DATA_DIR%/}/data" ]; then
DATABASE_ALREADY_EXISTS='true'
fi
# only run initialization on an empty data directory
if [ -z "${DATABASE_ALREADY_EXISTS}" ]; then
# run initialization if flag CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS is not empty or data directory is empty
if [[ -n "${CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS}" || -z "${DATABASE_ALREADY_EXISTS}" ]]; then
RUN_INITDB_SCRIPTS='true'
fi
if [ -z "${RUN_INITDB_SCRIPTS}" ]; then
if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then
# port is needed to check if clickhouse-server is ready for connections
HTTP_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=http_port --try)"

View File

@ -17,16 +17,20 @@ CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export}
EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, INDEX ix_pr (pull_request_number) TYPE set(100), INDEX ix_commit (commit_sha) TYPE set(100), INDEX ix_check_time (check_start_time) TYPE minmax, "}
EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, toLowCardinality('') AS check_name, toLowCardinality('') AS instance_type, '' AS instance_id"}
EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "}
EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name"}
# trace_log needs more columns for symbolization
EXTRA_COLUMNS_TRACE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), lines Array(LowCardinality(String)), "
EXTRA_COLUMNS_EXPRESSION_TRACE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> demangle(addressToSymbol(x)), trace)::Array(LowCardinality(String)) AS symbols, arrayMap(x -> addressToLine(x), trace)::Array(LowCardinality(String)) AS lines"
# coverage_log needs more columns for symbolization, but only symbol names (the line numbers are too heavy to calculate)
EXTRA_COLUMNS_COVERAGE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), "
EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> demangle(addressToSymbol(x)), coverage)::Array(LowCardinality(String)) AS symbols"
function __set_connection_args
{
# It's impossible to use generous $CONNECTION_ARGS string, it's unsafe from word splitting perspective.
# It's impossible to use a generic $CONNECTION_ARGS string, it's unsafe from word splitting perspective.
# That's why we must stick to the generated option
CONNECTION_ARGS=(
--receive_timeout=45 --send_timeout=45 --secure
@ -129,6 +133,19 @@ function setup_logs_replication
debug_or_sanitizer_build=$(clickhouse-client -q "WITH ((SELECT value FROM system.build_options WHERE name='BUILD_TYPE') AS build, (SELECT value FROM system.build_options WHERE name='CXX_FLAGS') as flags) SELECT build='Debug' OR flags LIKE '%fsanitize%'")
echo "Build is debug or sanitizer: $debug_or_sanitizer_build"
# We will pre-create a table system.coverage_log.
# It is normally created by clickhouse-test rather than the server,
# so we will create it in advance to make it be picked up by the next commands:
clickhouse-client --query "
CREATE TABLE IF NOT EXISTS system.coverage_log
(
time DateTime COMMENT 'The time of test run',
test_name String COMMENT 'The name of the test',
coverage Array(UInt64) COMMENT 'An array of addresses of the code (a subset of addresses instrumented for coverage) that were encountered during the test run'
) ENGINE = Null COMMENT 'Contains information about per-test coverage from the CI, but used only for exporting to the CI cluster'
"
# For each system log table:
echo 'Create %_log tables'
clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table
@ -139,11 +156,16 @@ function setup_logs_replication
# Do not try to resolve stack traces in case of debug/sanitizers
# build, since it is too slow (flushing of trace_log can take ~1min
# with such MV attached)
if [[ "$debug_or_sanitizer_build" = 1 ]]; then
if [[ "$debug_or_sanitizer_build" = 1 ]]
then
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}"
else
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_TRACE_LOG}"
fi
elif [[ "$table" = "coverage_log" ]]
then
EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS_COVERAGE_LOG}"
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG}"
else
EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS}"
EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}"
@ -160,7 +182,7 @@ function setup_logs_replication
# Create the destination table with adapted name and structure:
statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e '
s/^\($/('"$EXTRA_COLUMNS_FOR_TABLE"'/;
s/ORDER BY \(/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"'/;
s/^ORDER BY (([^\(].+?)|\((.+?)\))$/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \2\3)/;
s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/;
/^TTL /d
')

View File

@ -62,46 +62,47 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \
# kazoo 2.10.0 is broken
# https://s3.amazonaws.com/clickhouse-test-reports/59337/524625a1d2f4cc608a3f1059e3df2c30f353a649/integration_tests__asan__analyzer__[5_6].html
RUN python3 -m pip install --no-cache-dir \
PyMySQL \
aerospike==11.1.0 \
asyncio \
PyMySQL==1.1.0 \
asyncio==3.4.3 \
avro==1.10.2 \
azure-storage-blob \
boto3 \
cassandra-driver \
azure-storage-blob==12.19.0 \
boto3==1.34.24 \
cassandra-driver==3.29.0 \
confluent-kafka==2.3.0 \
delta-spark==2.3.0 \
dict2xml \
dicttoxml \
dict2xml==1.7.4 \
dicttoxml==1.7.16 \
docker==6.1.3 \
docker-compose==1.29.2 \
grpcio \
grpcio-tools \
kafka-python \
grpcio==1.60.0 \
grpcio-tools==1.60.0 \
kafka-python==2.0.2 \
lz4==4.3.3 \
minio==7.2.3 \
nats-py==2.6.0 \
protobuf==4.25.2 \
kazoo==2.9.0 \
lz4 \
minio \
nats-py \
protobuf \
psycopg2-binary==2.9.6 \
pyhdfs \
pyhdfs==0.3.1 \
pymongo==3.11.0 \
pyspark==3.3.2 \
pytest \
pytest==7.4.4 \
pytest-order==1.0.0 \
pytest-random \
pytest-repeat \
pytest-timeout \
pytest-xdist \
pytz \
pytest-random==0.2 \
pytest-repeat==0.9.3 \
pytest-timeout==2.2.0 \
pytest-xdist==3.5.0 \
pytest-reportlog==0.4.0 \
pytz==2023.3.post1 \
pyyaml==5.3.1 \
redis \
requests-kerberos \
redis==5.0.1 \
requests-kerberos==0.14.0 \
tzlocal==2.1 \
retry \
bs4 \
lxml \
urllib3
retry==0.9.2 \
bs4==0.0.2 \
lxml==5.1.0 \
urllib3==2.0.7
# bs4, lxml are for cloud tests, do not delete
# Hudi supports only spark 3.3.*, not 3.4

View File

@ -74,7 +74,6 @@ RUN arch=${TARGETARCH:-amd64} \
&& wget "https://dl.min.io/client/mc/release/linux-${arch}/archive/mc.RELEASE.${MINIO_CLIENT_VERSION}" -O ./mc \
&& chmod +x ./mc ./minio
RUN wget --no-verbose 'https://archive.apache.org/dist/hadoop/common/hadoop-3.3.1/hadoop-3.3.1.tar.gz' \
&& tar -xvf hadoop-3.3.1.tar.gz \
&& rm -rf hadoop-3.3.1.tar.gz

View File

@ -185,11 +185,15 @@ function run_tests()
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--replicated-database')
# Too many tests fail for DatabaseReplicated in parallel.
ADDITIONAL_OPTIONS+=('--jobs')
ADDITIONAL_OPTIONS+=('2')
elif [[ 1 == $(clickhouse-client --query "SELECT value LIKE '%SANITIZE_COVERAGE%' FROM system.build_options WHERE name = 'CXX_FLAGS'") ]]; then
# Coverage on a per-test basis could only be collected sequentially.
# Do not set the --jobs parameter.
echo "Running tests with coverage collection."
else
# Too many tests fail for DatabaseReplicated in parallel. All other
# configurations are OK.
# All other configurations are OK.
ADDITIONAL_OPTIONS+=('--jobs')
ADDITIONAL_OPTIONS+=('8')
fi

View File

@ -5,7 +5,6 @@ FROM ubuntu:22.04
ARG apt_archive="http://archive.ubuntu.com"
RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list
# 15.0.2
ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=17
RUN apt-get update \
@ -30,8 +29,7 @@ RUN apt-get update \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
# Install cmake 3.20+ for rust support
# Install cmake 3.20+ for Rust support
# Used https://askubuntu.com/a/1157132 as reference
RUN curl -s https://apt.kitware.com/keys/kitware-archive-latest.asc | \
gpg --dearmor - > /etc/apt/trusted.gpg.d/kitware.gpg && \
@ -65,8 +63,7 @@ RUN apt-get update \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
# This symlink required by gcc to find lld compiler
# This symlink is required by gcc to find the lld linker
RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld
# for external_symbolizer_path
RUN ln -s /usr/bin/llvm-symbolizer-${LLVM_VERSION} /usr/bin/llvm-symbolizer
@ -111,5 +108,4 @@ RUN arch=${TARGETARCH:-amd64} \
&& mv "/tmp/sccache-$SCCACHE_VERSION-$rarch-unknown-linux-musl/sccache" /usr/bin \
&& rm "/tmp/sccache-$SCCACHE_VERSION-$rarch-unknown-linux-musl" -r
COPY process_functional_tests_result.py /

View File

@ -0,0 +1,21 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v23.12.4.15-stable (4233d111d20) FIXME as compared to v23.12.3.40-stable (a594704ae75)
#### Bug Fix (user-visible misbehavior in an official stable release)
* Fix incorrect result of arrayElement / map[] on empty value [#59594](https://github.com/ClickHouse/ClickHouse/pull/59594) ([Raúl Marín](https://github.com/Algunenano)).
* Fix crash in topK when merging empty states [#59603](https://github.com/ClickHouse/ClickHouse/pull/59603) ([Raúl Marín](https://github.com/Algunenano)).
* Fix distributed table with a constant sharding key [#59606](https://github.com/ClickHouse/ClickHouse/pull/59606) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix leftPad / rightPad function with FixedString input [#59739](https://github.com/ClickHouse/ClickHouse/pull/59739) ([Raúl Marín](https://github.com/Algunenano)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Fix 02720_row_policy_column_with_dots [#59453](https://github.com/ClickHouse/ClickHouse/pull/59453) ([Duc Canh Le](https://github.com/canhld94)).
* Pin python dependencies in stateless tests [#59663](https://github.com/ClickHouse/ClickHouse/pull/59663) ([Raúl Marín](https://github.com/Algunenano)).

View File

@ -0,0 +1,34 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v24.1.3.31-stable (135b08cbd28) FIXME as compared to v24.1.2.5-stable (b2605dd4a5a)
#### Improvement
* Backported in [#59569](https://github.com/ClickHouse/ClickHouse/issues/59569): Now dashboard understands both compressed and uncompressed state of URL's #hash (backward compatibility). Continuation of [#59124](https://github.com/ClickHouse/ClickHouse/issues/59124) . [#59548](https://github.com/ClickHouse/ClickHouse/pull/59548) ([Amos Bird](https://github.com/amosbird)).
* Backported in [#59776](https://github.com/ClickHouse/ClickHouse/issues/59776): Added settings `split_parts_ranges_into_intersecting_and_non_intersecting_final` and `split_intersecting_parts_ranges_into_layers_final`. This settings are needed to disable optimizations for queries with `FINAL` and needed for debug only. [#59705](https://github.com/ClickHouse/ClickHouse/pull/59705) ([Maksim Kita](https://github.com/kitaisreal)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Fix `ASTAlterCommand::formatImpl` in case of column specific settings… [#59445](https://github.com/ClickHouse/ClickHouse/pull/59445) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)).
* Make MAX use the same rules as permutation for complex types [#59498](https://github.com/ClickHouse/ClickHouse/pull/59498) ([Raúl Marín](https://github.com/Algunenano)).
* Fix corner case when passing `update_insert_deduplication_token_in_dependent_materialized_views` [#59544](https://github.com/ClickHouse/ClickHouse/pull/59544) ([Jordi Villar](https://github.com/jrdi)).
* Fix incorrect result of arrayElement / map[] on empty value [#59594](https://github.com/ClickHouse/ClickHouse/pull/59594) ([Raúl Marín](https://github.com/Algunenano)).
* Fix crash in topK when merging empty states [#59603](https://github.com/ClickHouse/ClickHouse/pull/59603) ([Raúl Marín](https://github.com/Algunenano)).
* Maintain function alias in RewriteSumFunctionWithSumAndCountVisitor [#59658](https://github.com/ClickHouse/ClickHouse/pull/59658) ([Raúl Marín](https://github.com/Algunenano)).
* Fix leftPad / rightPad function with FixedString input [#59739](https://github.com/ClickHouse/ClickHouse/pull/59739) ([Raúl Marín](https://github.com/Algunenano)).
#### NO CL ENTRY
* NO CL ENTRY: 'Revert "Backport [#59650](https://github.com/ClickHouse/ClickHouse/issues/59650) to 24.1: MergeTree FINAL optimization diagnostics and settings"'. [#59701](https://github.com/ClickHouse/ClickHouse/pull/59701) ([Raúl Marín](https://github.com/Algunenano)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Fix 02720_row_policy_column_with_dots [#59453](https://github.com/ClickHouse/ClickHouse/pull/59453) ([Duc Canh Le](https://github.com/canhld94)).
* Refactoring of dashboard state encoding [#59554](https://github.com/ClickHouse/ClickHouse/pull/59554) ([Sergei Trifonov](https://github.com/serxa)).
* MergeTree FINAL optimization diagnostics and settings [#59650](https://github.com/ClickHouse/ClickHouse/pull/59650) ([Maksim Kita](https://github.com/kitaisreal)).
* Pin python dependencies in stateless tests [#59663](https://github.com/ClickHouse/ClickHouse/pull/59663) ([Raúl Marín](https://github.com/Algunenano)).

View File

@ -0,0 +1,28 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v24.1.4.19-stable (334aecf7e4b) FIXME as compared to v24.1.3.31-stable (135b08cbd28)
#### Improvement
* Backported in [#59826](https://github.com/ClickHouse/ClickHouse/issues/59826): In case when `merge_max_block_size_bytes` is small enough and tables contain wide rows (strings or tuples) background merges may stuck in an endless loop. This behaviour is fixed. Follow-up for https://github.com/ClickHouse/ClickHouse/pull/59340. [#59812](https://github.com/ClickHouse/ClickHouse/pull/59812) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
#### Build/Testing/Packaging Improvement
* Backported in [#59885](https://github.com/ClickHouse/ClickHouse/issues/59885): If you want to run initdb scripts every time when ClickHouse container is starting you shoud initialize environment varible CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS. [#59808](https://github.com/ClickHouse/ClickHouse/pull/59808) ([Alexander Nikolaev](https://github.com/AlexNik)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Fix digest calculation in Keeper [#59439](https://github.com/ClickHouse/ClickHouse/pull/59439) ([Antonio Andelic](https://github.com/antonio2368)).
* Fix distributed table with a constant sharding key [#59606](https://github.com/ClickHouse/ClickHouse/pull/59606) ([Vitaly Baranov](https://github.com/vitlibar)).
* Fix query start time on non initial queries [#59662](https://github.com/ClickHouse/ClickHouse/pull/59662) ([Raúl Marín](https://github.com/Algunenano)).
* Fix parsing of partition expressions surrounded by parens [#59901](https://github.com/ClickHouse/ClickHouse/pull/59901) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Temporarily remove a feature that doesn't work [#59688](https://github.com/ClickHouse/ClickHouse/pull/59688) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Make ZooKeeper actually sequentialy consistent [#59735](https://github.com/ClickHouse/ClickHouse/pull/59735) ([Alexander Tokmakov](https://github.com/tavplubix)).
* Fix special build reports in release branches [#59797](https://github.com/ClickHouse/ClickHouse/pull/59797) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).

View File

@ -6,6 +6,12 @@ sidebar_label: Memory
# Memory Table Engine
:::note
When using the Memory table engine on ClickHouse Cloud, data is not replicated across all nodes (by design). To guarantee that all queries are routed to the same node and that the Memory table engine works as expected, you can do one of the following:
- Execute all operations in the same session
- Use a client that uses TCP or the native interface (which enables support for sticky connections) such as [clickhouse-client](/en/interfaces/cli)
:::
The Memory engine stores data in RAM, in uncompressed form. Data is stored in exactly the same form as it is received when read. In other words, reading from this table is completely free.
Concurrent data access is synchronized. Locks are short: read and write operations do not block each other.
Indexes are not supported. Reading is parallelized.

View File

@ -31,6 +31,10 @@ This reduces maintenance effort and avoids redundancy.
## Configuration Settings and Usage
:::note
In ClickHouse Cloud, you must use [query level settings](/en/operations/settings/query-level) to edit query cache settings. Editing [config level settings](/en/operations/configuration-files) is currently not supported.
:::
Setting [use_query_cache](settings/settings.md#use-query-cache) can be used to control whether a specific query or all queries of the
current session should utilize the query cache. For example, the first execution of query

View File

@ -514,6 +514,10 @@ ENGINE = MergeTree ORDER BY x;
## Temporary Tables
:::note
Please note that temporary tables are not replicated. As a result, there is no guarantee that data inserted into a temporary table will be available in other replicas. The primary use case where temporary tables can be useful is for querying or joining small external datasets during a single session.
:::
ClickHouse supports temporary tables which have the following characteristics:
- Temporary tables disappear when the session ends, including if the connection is lost.

View File

@ -130,6 +130,8 @@ if [ -n "$SANITIZER" ]; then
fi
elif [[ $BUILD_TYPE == 'debug' ]]; then
VERSION_POSTFIX+="+debug"
elif [[ $BUILD_TYPE =~ 'coverage' ]]; then
VERSION_POSTFIX+="+coverage"
fi
if [[ "$PKG_ROOT" != "$SOURCE" ]]; then

View File

@ -49,6 +49,12 @@ contents:
dst: /usr/bin/clickhouse-client
- src: root/usr/bin/clickhouse-local
dst: /usr/bin/clickhouse-local
- src: root/usr/bin/ch
dst: /usr/bin/ch
- src: root/usr/bin/chc
dst: /usr/bin/chc
- src: root/usr/bin/chl
dst: /usr/bin/chl
- src: root/usr/bin/clickhouse-obfuscator
dst: /usr/bin/clickhouse-obfuscator
# docs

View File

@ -7,35 +7,16 @@ endif ()
include(${ClickHouse_SOURCE_DIR}/cmake/split_debug_symbols.cmake)
# The `clickhouse` binary is a multi purpose tool that contains multiple execution modes (client, server, etc.),
# each of them may be built and linked as a separate library.
# If you do not know what modes you need, turn this option OFF and enable SERVER and CLIENT only.
# So client/server/... is just a symlink to `clickhouse` binary.
#
# But, there are several components that requires extra libraries, like keeper
# requires NuRaft, that regular binary does not requires, so you can disable
# compilation of this components.
#
# If you do not know what modes you need, turn then all.
option (ENABLE_CLICKHOUSE_ALL "Enable all ClickHouse modes by default" ON)
option (ENABLE_CLICKHOUSE_SERVER "Server mode (main mode)" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_CLIENT "Client mode (interactive tui/shell that connects to the server)"
${ENABLE_CLICKHOUSE_ALL})
# https://clickhouse.com/docs/en/operations/utilities/clickhouse-local/
option (ENABLE_CLICKHOUSE_LOCAL "Local files fast processing mode" ${ENABLE_CLICKHOUSE_ALL})
# https://clickhouse.com/docs/en/operations/utilities/clickhouse-benchmark/
option (ENABLE_CLICKHOUSE_BENCHMARK "Queries benchmarking mode" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG "Configs processor (extract values etc.)" ${ENABLE_CLICKHOUSE_ALL})
# https://clickhouse.com/docs/en/operations/utilities/clickhouse-compressor/
option (ENABLE_CLICKHOUSE_COMPRESSOR "Data compressor and decompressor" ${ENABLE_CLICKHOUSE_ALL})
# https://clickhouse.com/docs/en/operations/utilities/clickhouse-copier/
option (ENABLE_CLICKHOUSE_COPIER "Inter-cluster data copying mode" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_FORMAT "Queries pretty-printer and formatter with syntax highlighting"
${ENABLE_CLICKHOUSE_ALL})
# https://clickhouse.com/docs/en/operations/utilities/clickhouse-obfuscator/
option (ENABLE_CLICKHOUSE_OBFUSCATOR "Table data obfuscator (convert real data to benchmark-ready one)"
${ENABLE_CLICKHOUSE_ALL})
# https://clickhouse.com/docs/en/operations/utilities/odbc-bridge/
# TODO Also needs NANODBC.
if (ENABLE_ODBC AND NOT USE_MUSL)
@ -51,18 +32,12 @@ endif ()
# https://presentations.clickhouse.com/matemarketing_2020/
option (ENABLE_CLICKHOUSE_GIT_IMPORT "A tool to analyze Git repositories" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER "A tool to export table data files to be later put to a static files web server" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_KEEPER "ClickHouse alternative to ZooKeeper" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_KEEPER_CONVERTER "Util allows to convert ZooKeeper logs and snapshots into clickhouse-keeper snapshot" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_KEEPER_CLIENT "ClickHouse Keeper Client" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_SU "A tool similar to 'su'" ${ENABLE_CLICKHOUSE_ALL})
option (ENABLE_CLICKHOUSE_DISKS "A tool to manage disks" ${ENABLE_CLICKHOUSE_ALL})
if (NOT ENABLE_NURAFT)
# RECONFIGURE_MESSAGE_LEVEL should not be used here,
# since ENABLE_NURAFT is set to OFF for FreeBSD and Darwin.
@ -71,27 +46,7 @@ if (NOT ENABLE_NURAFT)
set(ENABLE_CLICKHOUSE_KEEPER_CONVERTER OFF)
endif()
option(ENABLE_CLICKHOUSE_INSTALL "Install ClickHouse without .deb/.rpm/.tgz packages (having the binary only)" ${ENABLE_CLICKHOUSE_ALL})
message(STATUS "ClickHouse modes:")
if (NOT ENABLE_CLICKHOUSE_SERVER)
message(WARNING "ClickHouse server mode is not going to be built.")
else()
message(STATUS "Server mode: ON")
endif()
if (NOT ENABLE_CLICKHOUSE_CLIENT)
message(WARNING "ClickHouse client mode is not going to be built. You won't be able to connect to the server and run tests")
else()
message(STATUS "Client mode: ON")
endif()
if (ENABLE_CLICKHOUSE_LOCAL)
message(STATUS "Local mode: ON")
else()
message(STATUS "Local mode: OFF")
endif()
message(STATUS "ClickHouse extra components:")
if (ENABLE_CLICKHOUSE_SELF_EXTRACTING)
message(STATUS "Self-extracting executable: ON")
@ -99,42 +54,6 @@ else()
message(STATUS "Self-extracting executable: OFF")
endif()
if (ENABLE_CLICKHOUSE_BENCHMARK)
message(STATUS "Benchmark mode: ON")
else()
message(STATUS "Benchmark mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG)
message(STATUS "Extract from config mode: ON")
else()
message(STATUS "Extract from config mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_COMPRESSOR)
message(STATUS "Compressor mode: ON")
else()
message(STATUS "Compressor mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_COPIER)
message(STATUS "Copier mode: ON")
else()
message(STATUS "Copier mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_FORMAT)
message(STATUS "Format mode: ON")
else()
message(STATUS "Format mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_OBFUSCATOR)
message(STATUS "Obfuscator mode: ON")
else()
message(STATUS "Obfuscator mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_ODBC_BRIDGE)
message(STATUS "ODBC bridge mode: ON")
else()
@ -147,18 +66,6 @@ else()
message(STATUS "Library bridge mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_INSTALL)
message(STATUS "ClickHouse install: ON")
else()
message(STATUS "ClickHouse install: OFF")
endif()
if (ENABLE_CLICKHOUSE_GIT_IMPORT)
message(STATUS "ClickHouse git-import: ON")
else()
message(STATUS "ClickHouse git-import: OFF")
endif()
if (ENABLE_CLICKHOUSE_KEEPER)
message(STATUS "ClickHouse keeper mode: ON")
else()
@ -177,19 +84,6 @@ else()
message(STATUS "ClickHouse keeper-client mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_DISKS)
message(STATUS "Clickhouse disks mode: ON")
else()
message(STATUS "ClickHouse disks mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_SU)
message(STATUS "ClickHouse su: ON")
else()
message(STATUS "ClickHouse su: OFF")
endif()
configure_file (config_tools.h.in ${CONFIG_INCLUDE_PATH}/config_tools.h)
macro(clickhouse_target_link_split_lib target name)
@ -272,42 +166,6 @@ endif ()
target_link_libraries (clickhouse PRIVATE clickhouse_common_io string_utils ${HARMFUL_LIB})
target_include_directories (clickhouse PRIVATE ${CMAKE_CURRENT_BINARY_DIR})
if (ENABLE_CLICKHOUSE_SERVER)
clickhouse_target_link_split_lib(clickhouse server)
endif ()
if (ENABLE_CLICKHOUSE_CLIENT)
clickhouse_target_link_split_lib(clickhouse client)
endif ()
if (ENABLE_CLICKHOUSE_LOCAL)
clickhouse_target_link_split_lib(clickhouse local)
endif ()
if (ENABLE_CLICKHOUSE_BENCHMARK)
clickhouse_target_link_split_lib(clickhouse benchmark)
endif ()
if (ENABLE_CLICKHOUSE_COPIER)
clickhouse_target_link_split_lib(clickhouse copier)
endif ()
if (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG)
clickhouse_target_link_split_lib(clickhouse extract-from-config)
endif ()
if (ENABLE_CLICKHOUSE_COMPRESSOR)
clickhouse_target_link_split_lib(clickhouse compressor)
endif ()
if (ENABLE_CLICKHOUSE_FORMAT)
clickhouse_target_link_split_lib(clickhouse format)
endif ()
if (ENABLE_CLICKHOUSE_OBFUSCATOR)
clickhouse_target_link_split_lib(clickhouse obfuscator)
endif ()
if (ENABLE_CLICKHOUSE_GIT_IMPORT)
clickhouse_target_link_split_lib(clickhouse git-import)
endif ()
if (ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER)
clickhouse_target_link_split_lib(clickhouse static-files-disk-uploader)
endif ()
if (ENABLE_CLICKHOUSE_SU)
clickhouse_target_link_split_lib(clickhouse su)
endif ()
if (ENABLE_CLICKHOUSE_KEEPER)
clickhouse_target_link_split_lib(clickhouse keeper)
endif()
@ -317,77 +175,40 @@ endif()
if (ENABLE_CLICKHOUSE_KEEPER_CLIENT)
clickhouse_target_link_split_lib(clickhouse keeper-client)
endif()
if (ENABLE_CLICKHOUSE_INSTALL)
clickhouse_target_link_split_lib(clickhouse install)
endif ()
if (ENABLE_CLICKHOUSE_DISKS)
clickhouse_target_link_split_lib(clickhouse disks)
endif ()
clickhouse_target_link_split_lib(clickhouse install)
set (CLICKHOUSE_BUNDLE)
macro(clickhouse_program_install name lib_name)
clickhouse_target_link_split_lib(clickhouse ${lib_name})
add_custom_target (${name} ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ${name} DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/${name}" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE ${name})
foreach(alias ${ARGN})
message(STATUS "Adding alias ${alias} for ${name}")
add_custom_target (${alias} ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ${alias} DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/${alias}" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE ${alias})
endforeach()
endmacro()
if (ENABLE_CLICKHOUSE_SELF_EXTRACTING)
list(APPEND CLICKHOUSE_BUNDLE self-extracting)
endif ()
if (ENABLE_CLICKHOUSE_SERVER)
add_custom_target (clickhouse-server ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-server DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-server" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-server)
endif ()
if (ENABLE_CLICKHOUSE_CLIENT)
add_custom_target (clickhouse-client ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-client DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-client" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-client)
endif ()
if (ENABLE_CLICKHOUSE_LOCAL)
add_custom_target (clickhouse-local ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-local DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-local" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-local)
endif ()
if (ENABLE_CLICKHOUSE_BENCHMARK)
add_custom_target (clickhouse-benchmark ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-benchmark DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-benchmark" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-benchmark)
endif ()
if (ENABLE_CLICKHOUSE_COPIER)
add_custom_target (clickhouse-copier ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-copier DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-copier" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-copier)
endif ()
if (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG)
add_custom_target (clickhouse-extract-from-config ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-extract-from-config DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-extract-from-config" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-extract-from-config)
endif ()
if (ENABLE_CLICKHOUSE_COMPRESSOR)
add_custom_target (clickhouse-compressor ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-compressor DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-compressor" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-compressor)
endif ()
if (ENABLE_CLICKHOUSE_FORMAT)
add_custom_target (clickhouse-format ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-format DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-format" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-format)
endif ()
if (ENABLE_CLICKHOUSE_OBFUSCATOR)
add_custom_target (clickhouse-obfuscator ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-obfuscator DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-obfuscator" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-obfuscator)
endif ()
if (ENABLE_CLICKHOUSE_GIT_IMPORT)
add_custom_target (clickhouse-git-import ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-git-import DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-git-import" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-git-import)
endif ()
if (ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER)
add_custom_target (clickhouse-static-files-disk-uploader ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-static-files-disk-uploader DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-static-files-disk-uploader" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-static-files-disk-uploader)
endif ()
if (ENABLE_CLICKHOUSE_SU)
add_custom_target (clickhouse-su ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-su DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-su" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-su)
endif ()
clickhouse_program_install(clickhouse-server server)
clickhouse_program_install(clickhouse-client client chc)
clickhouse_program_install(clickhouse-local local chl ch)
clickhouse_program_install(clickhouse-benchmark benchmark)
clickhouse_program_install(clickhouse-copier copier)
clickhouse_program_install(clickhouse-extract-from-config extract-from-config)
clickhouse_program_install(clickhouse-compressor compressor)
clickhouse_program_install(clickhouse-format format)
clickhouse_program_install(clickhouse-obfuscator obfuscator)
clickhouse_program_install(clickhouse-git-import git-import)
clickhouse_program_install(clickhouse-static-files-disk-uploader static-files-disk-uploader)
clickhouse_program_install(clickhouse-disks disks)
clickhouse_program_install(clickhouse-su su)
if (ENABLE_CLICKHOUSE_KEEPER)
if (NOT BUILD_STANDALONE_KEEPER AND CREATE_KEEPER_SYMLINK)
@ -417,11 +238,6 @@ if (ENABLE_CLICKHOUSE_KEEPER_CLIENT)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-keeper-client)
endif ()
if (ENABLE_CLICKHOUSE_DISKS)
add_custom_target (clickhouse-disks ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-disks DEPENDS clickhouse)
install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-disks" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
list(APPEND CLICKHOUSE_BUNDLE clickhouse-disks)
endif ()
add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_BUNDLE})

View File

@ -640,7 +640,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
{
std::cout << "Usage: " << argv[0] << " [options] < queries.txt\n";
std::cout << desc << "\n";
return 1;
std::cout << "\nSee also: https://clickhouse.com/docs/en/operations/utilities/clickhouse-benchmark/\n";
return 0;
}
print_stacktrace = options.count("stacktrace");

View File

@ -1000,6 +1000,7 @@ void Client::printHelpMessage(const OptionsDescription & options_description)
std::cout << options_description.external_description.value() << "\n";
std::cout << options_description.hosts_and_ports_description.value() << "\n";
std::cout << "In addition, --param_name=value can be specified for substitution of parameters for parametrized queries.\n";
std::cout << "\nSee also: https://clickhouse.com/docs/en/integrations/sql-clients/cli\n";
}

View File

@ -100,6 +100,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv)
std::cout << "Usage: " << argv[0] << " [options] < INPUT > OUTPUT" << std::endl;
std::cout << "Usage: " << argv[0] << " [options] INPUT OUTPUT" << std::endl;
std::cout << desc << std::endl;
std::cout << "\nSee also: https://clickhouse.com/docs/en/operations/utilities/clickhouse-compressor/\n";
return 0;
}

View File

@ -2,23 +2,8 @@
#pragma once
#cmakedefine01 ENABLE_CLICKHOUSE_SERVER
#cmakedefine01 ENABLE_CLICKHOUSE_CLIENT
#cmakedefine01 ENABLE_CLICKHOUSE_LOCAL
#cmakedefine01 ENABLE_CLICKHOUSE_BENCHMARK
#cmakedefine01 ENABLE_CLICKHOUSE_PERFORMANCE_TEST
#cmakedefine01 ENABLE_CLICKHOUSE_COPIER
#cmakedefine01 ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG
#cmakedefine01 ENABLE_CLICKHOUSE_COMPRESSOR
#cmakedefine01 ENABLE_CLICKHOUSE_FORMAT
#cmakedefine01 ENABLE_CLICKHOUSE_OBFUSCATOR
#cmakedefine01 ENABLE_CLICKHOUSE_GIT_IMPORT
#cmakedefine01 ENABLE_CLICKHOUSE_INSTALL
#cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE
#cmakedefine01 ENABLE_CLICKHOUSE_LIBRARY_BRIDGE
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CLIENT
#cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CONVERTER
#cmakedefine01 ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER
#cmakedefine01 ENABLE_CLICKHOUSE_SU
#cmakedefine01 ENABLE_CLICKHOUSE_DISKS

View File

@ -78,6 +78,7 @@ void ClusterCopierApp::handleHelp(const std::string &, const std::string &)
help_formatter.setHeader("Copies tables from one cluster to another");
help_formatter.setUsage("--config-file <config-file> --task-path <task-path>");
help_formatter.format(std::cerr);
help_formatter.setFooter("See also: https://clickhouse.com/docs/en/operations/utilities/clickhouse-copier/");
stopOptionsProcessing();
}

View File

@ -11,6 +11,10 @@ set (CLICKHOUSE_DISKS_SOURCES
CommandRemove.cpp
CommandWrite.cpp)
if (CLICKHOUSE_CLOUD)
set (CLICKHOUSE_DISKS_SOURCES ${CLICKHOUSE_DISKS_SOURCES} CommandPackedIO.cpp)
endif ()
set (CLICKHOUSE_DISKS_LINK
PRIVATE
boost::program_options

View File

@ -61,7 +61,6 @@ public:
auto out = disk->writeFile(relative_path_output);
copyData(*in, *out);
out->finalize();
return;
}
else
{

View File

@ -65,6 +65,9 @@ void DisksApp::addOptions(
positional_options_description.add("command_name", 1);
supported_commands = {"list-disks", "list", "move", "remove", "link", "copy", "write", "read", "mkdir"};
#ifdef CLICKHOUSE_CLOUD
supported_commands.insert("packed-io");
#endif
command_descriptions.emplace("list-disks", makeCommandListDisks());
command_descriptions.emplace("list", makeCommandList());
@ -75,6 +78,9 @@ void DisksApp::addOptions(
command_descriptions.emplace("write", makeCommandWrite());
command_descriptions.emplace("read", makeCommandRead());
command_descriptions.emplace("mkdir", makeCommandMkDir());
#ifdef CLICKHOUSE_CLOUD
command_descriptions.emplace("packed-io", makeCommandPackedIO());
#endif
}
void DisksApp::processOptions()
@ -89,6 +95,11 @@ void DisksApp::processOptions()
config().setString("log-level", options["log-level"].as<String>());
}
DisksApp::~DisksApp()
{
global_context->shutdown();
}
void DisksApp::init(std::vector<String> & common_arguments)
{
stopOptionsProcessing();
@ -134,6 +145,7 @@ void DisksApp::parseAndCheckOptions(
.options(options_description_)
.positional(positional_options_description)
.allow_unregistered();
po::parsed_options parsed = parser.run();
po::store(parsed, options);
@ -199,8 +211,8 @@ int DisksApp::main(const std::vector<String> & /*args*/)
po::parsed_options parsed = parser.run();
po::store(parsed, options);
po::notify(options);
args = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional);
args = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional);
command->processOptions(config(), options);
}
else

View File

@ -21,6 +21,7 @@ class DisksApp : public Poco::Util::Application, public Loggers
{
public:
DisksApp() = default;
~DisksApp() override;
void init(std::vector<String> & common_arguments);
@ -52,9 +53,9 @@ protected:
std::vector<String> command_arguments;
std::unordered_set<String> supported_commands;
std::unordered_map<String, CommandPtr> command_descriptions;
po::variables_map options;
};
}

View File

@ -63,3 +63,4 @@ DB::CommandPtr makeCommandRead();
DB::CommandPtr makeCommandRemove();
DB::CommandPtr makeCommandWrite();
DB::CommandPtr makeCommandMkDir();
DB::CommandPtr makeCommandPackedIO();

View File

@ -172,6 +172,7 @@ clickhouse-client --query "INSERT INTO git.commits FORMAT TSV" < commits.tsv
clickhouse-client --query "INSERT INTO git.file_changes FORMAT TSV" < file_changes.tsv
clickhouse-client --query "INSERT INTO git.line_changes FORMAT TSV" < line_changes.tsv
Check out this presentation: https://presentations.clickhouse.com/matemarketing_2020/
)";
namespace po = boost::program_options;

View File

@ -79,10 +79,6 @@ namespace ErrorCodes
}
/// ANSI escape sequence for intense color in terminal.
#define HILITE "\033[1m"
#define END_HILITE "\033[0m"
#if defined(OS_DARWIN)
/// Until createUser() and createGroup() are implemented, only sudo-less installations are supported/default for macOS.
static constexpr auto DEFAULT_CLICKHOUSE_SERVER_USER = "";
@ -216,6 +212,16 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
{
try
{
const char * start_hilite = "";
const char * end_hilite = "";
if (isatty(STDOUT_FILENO))
{
/// ANSI escape sequence for intense color in terminal.
start_hilite = "\033[1m";
end_hilite = "\033[0m";
}
po::options_description desc;
desc.add_options()
("help,h", "produce help message")
@ -236,9 +242,10 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
if (options.count("help"))
{
std::cout << "Install ClickHouse without .deb/.rpm/.tgz packages (having the binary only)\n\n";
std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " install [options]", getuid() != 0) << '\n';
std::cout << desc << '\n';
return 1;
return 0;
}
/// We need to copy binary to the binary directory.
@ -707,7 +714,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
{
fmt::print("Users config file {} already exists, will keep it and extract users info from it.\n", users_config_file.string());
/// Check if password for default user already specified.
/// Check if password for the default user already specified.
ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false);
ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig()));
@ -799,13 +806,13 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
/// Set up password for default user.
if (has_password_for_default_user)
{
fmt::print(HILITE "Password for default user is already specified. To remind or reset, see {} and {}." END_HILITE "\n",
users_config_file.string(), users_d.string());
fmt::print("{}Password for the default user is already specified. To remind or reset, see {} and {}.{}\n",
start_hilite, users_config_file.string(), users_d.string(), end_hilite);
}
else if (!can_ask_password)
{
fmt::print(HILITE "Password for default user is empty string. See {} and {} to change it." END_HILITE "\n",
users_config_file.string(), users_d.string());
fmt::print("{}Password for the default user is an empty string. See {} and {} to change it.{}\n",
start_hilite, users_config_file.string(), users_d.string(), end_hilite);
}
else
{
@ -814,7 +821,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
char buf[1000] = {};
std::string password;
if (auto * result = readpassphrase("Enter password for default user: ", buf, sizeof(buf), 0))
if (auto * result = readpassphrase("Enter password for the default user: ", buf, sizeof(buf), 0))
password = result;
if (!password.empty())
@ -839,7 +846,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
"</clickhouse>\n";
out.sync();
out.finalize();
fmt::print(HILITE "Password for default user is saved in file {}." END_HILITE "\n", password_file);
fmt::print("{}Password for the default user is saved in file {}.{}\n", start_hilite, password_file, end_hilite);
#else
out << "<clickhouse>\n"
" <users>\n"
@ -850,13 +857,13 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
"</clickhouse>\n";
out.sync();
out.finalize();
fmt::print(HILITE "Password for default user is saved in plaintext in file {}." END_HILITE "\n", password_file);
fmt::print("{}Password for the default user is saved in plaintext in file {}.{}\n", start_hilite, password_file, end_hilite);
#endif
has_password_for_default_user = true;
}
else
fmt::print(HILITE "Password for default user is empty string. See {} and {} to change it." END_HILITE "\n",
users_config_file.string(), users_d.string());
fmt::print("{}Password for the default user is an empty string. See {} and {} to change it.{}\n",
start_hilite, users_config_file.string(), users_d.string(), end_hilite);
}
/** Set capabilities for the binary.

View File

@ -1,5 +1,4 @@
#include <iostream>
#include <optional>
#include <boost/program_options.hpp>
#include <Coordination/KeeperSnapshotManager.h>

View File

@ -828,6 +828,7 @@ void LocalServer::printHelpMessage([[maybe_unused]] const OptionsDescription & o
std::cout << options_description.main_description.value() << "\n";
std::cout << getHelpFooter() << "\n";
std::cout << "In addition, --param_name=value can be specified for substitution of parameters for parametrized queries.\n";
std::cout << "\nSee also: https://clickhouse.com/docs/en/operations/utilities/clickhouse-local/\n";
#endif
}

View File

@ -1,6 +1,7 @@
#include <csignal>
#include <csetjmp>
#include <unistd.h>
#include <fcntl.h>
#include <new>
#include <iostream>
@ -19,39 +20,32 @@
#include <Common/IO.h>
#include <base/phdr_cache.h>
#include <base/coverage.h>
/// Universal executable for various clickhouse applications
#if ENABLE_CLICKHOUSE_SERVER
int mainEntryClickHouseServer(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_CLIENT
int mainEntryClickHouseClient(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_LOCAL
int mainEntryClickHouseLocal(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_BENCHMARK
int mainEntryClickHouseBenchmark(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG
int mainEntryClickHouseExtractFromConfig(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_COMPRESSOR
int mainEntryClickHouseCompressor(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_FORMAT
int mainEntryClickHouseFormat(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_COPIER
int mainEntryClickHouseClusterCopier(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_OBFUSCATOR
int mainEntryClickHouseObfuscator(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_GIT_IMPORT
int mainEntryClickHouseGitImport(int argc, char ** argv);
#endif
int mainEntryClickHouseStaticFilesDiskUploader(int argc, char ** argv);
int mainEntryClickHouseSU(int argc, char ** argv);
int mainEntryClickHouseDisks(int argc, char ** argv);
int mainEntryClickHouseHashBinary(int, char **)
{
/// Intentionally without newline. So you can run:
/// objcopy --add-section .clickhouse.hash=<(./clickhouse hash-binary) clickhouse
std::cout << getHashOfLoadedBinaryHex();
return 0;
}
#if ENABLE_CLICKHOUSE_KEEPER
int mainEntryClickHouseKeeper(int argc, char ** argv);
#endif
@ -61,30 +55,13 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv);
#if ENABLE_CLICKHOUSE_KEEPER_CLIENT
int mainEntryClickHouseKeeperClient(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER
int mainEntryClickHouseStaticFilesDiskUploader(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_SU
int mainEntryClickHouseSU(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_INSTALL
// install
int mainEntryClickHouseInstall(int argc, char ** argv);
int mainEntryClickHouseStart(int argc, char ** argv);
int mainEntryClickHouseStop(int argc, char ** argv);
int mainEntryClickHouseStatus(int argc, char ** argv);
int mainEntryClickHouseRestart(int argc, char ** argv);
#endif
#if ENABLE_CLICKHOUSE_DISKS
int mainEntryClickHouseDisks(int argc, char ** argv);
#endif
int mainEntryClickHouseHashBinary(int, char **)
{
/// Intentionally without newline. So you can run:
/// objcopy --add-section .clickhouse.hash=<(./clickhouse hash-binary) clickhouse
std::cout << getHashOfLoadedBinaryHex();
return 0;
}
namespace
{
@ -96,36 +73,22 @@ using MainFunc = int (*)(int, char**);
/// Add an item here to register new application
std::pair<std::string_view, MainFunc> clickhouse_applications[] =
{
#if ENABLE_CLICKHOUSE_LOCAL
{"local", mainEntryClickHouseLocal},
#endif
#if ENABLE_CLICKHOUSE_CLIENT
{"client", mainEntryClickHouseClient},
#endif
#if ENABLE_CLICKHOUSE_BENCHMARK
{"benchmark", mainEntryClickHouseBenchmark},
#endif
#if ENABLE_CLICKHOUSE_SERVER
{"server", mainEntryClickHouseServer},
#endif
#if ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG
{"extract-from-config", mainEntryClickHouseExtractFromConfig},
#endif
#if ENABLE_CLICKHOUSE_COMPRESSOR
{"compressor", mainEntryClickHouseCompressor},
#endif
#if ENABLE_CLICKHOUSE_FORMAT
{"format", mainEntryClickHouseFormat},
#endif
#if ENABLE_CLICKHOUSE_COPIER
{"copier", mainEntryClickHouseClusterCopier},
#endif
#if ENABLE_CLICKHOUSE_OBFUSCATOR
{"obfuscator", mainEntryClickHouseObfuscator},
#endif
#if ENABLE_CLICKHOUSE_GIT_IMPORT
{"git-import", mainEntryClickHouseGitImport},
#endif
{"static-files-disk-uploader", mainEntryClickHouseStaticFilesDiskUploader},
{"su", mainEntryClickHouseSU},
{"hash-binary", mainEntryClickHouseHashBinary},
{"disks", mainEntryClickHouseDisks},
// keeper
#if ENABLE_CLICKHOUSE_KEEPER
{"keeper", mainEntryClickHouseKeeper},
#endif
@ -135,34 +98,20 @@ std::pair<std::string_view, MainFunc> clickhouse_applications[] =
#if ENABLE_CLICKHOUSE_KEEPER_CLIENT
{"keeper-client", mainEntryClickHouseKeeperClient},
#endif
#if ENABLE_CLICKHOUSE_INSTALL
// install
{"install", mainEntryClickHouseInstall},
{"start", mainEntryClickHouseStart},
{"stop", mainEntryClickHouseStop},
{"status", mainEntryClickHouseStatus},
{"restart", mainEntryClickHouseRestart},
#endif
#if ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER
{"static-files-disk-uploader", mainEntryClickHouseStaticFilesDiskUploader},
#endif
#if ENABLE_CLICKHOUSE_SU
{"su", mainEntryClickHouseSU},
#endif
{"hash-binary", mainEntryClickHouseHashBinary},
#if ENABLE_CLICKHOUSE_DISKS
{"disks", mainEntryClickHouseDisks},
#endif
};
/// Add an item here to register a new short name
std::pair<std::string_view, std::string_view> clickhouse_short_names[] =
{
#if ENABLE_CLICKHOUSE_LOCAL
{"chl", "local"},
#endif
#if ENABLE_CLICKHOUSE_CLIENT
{"chc", "client"},
#endif
};
int printHelp(int, char **)
@ -392,6 +341,50 @@ void checkHarmfulEnvironmentVariables(char ** argv)
}
#endif
#if defined(SANITIZE_COVERAGE)
__attribute__((no_sanitize("coverage")))
void dumpCoverage()
{
/// A user can request to dump the coverage information into files at exit.
/// This is useful for non-server applications such as clickhouse-format or clickhouse-client,
/// that cannot introspect it with SQL functions at runtime.
/// The CLICKHOUSE_WRITE_COVERAGE environment variable defines a prefix for a filename 'prefix.pid'
/// containing the list of addresses of covered .
/// The format is even simpler than Clang's "sancov": an array of 64-bit addresses, native byte order, no header.
if (const char * coverage_filename_prefix = getenv("CLICKHOUSE_WRITE_COVERAGE")) // NOLINT(concurrency-mt-unsafe)
{
auto dump = [](const std::string & name, auto span)
{
/// Write only non-zeros.
std::vector<uintptr_t> data;
data.reserve(span.size());
for (auto addr : span)
if (addr)
data.push_back(addr);
int fd = ::open(name.c_str(), O_WRONLY | O_CREAT | O_TRUNC | O_CLOEXEC, 0400);
if (-1 == fd)
{
writeError("Cannot open a file to write the coverage data\n");
}
else
{
if (!writeRetry(fd, reinterpret_cast<const char *>(data.data()), data.size() * sizeof(data[0])))
writeError("Cannot write the coverage data to a file\n");
if (0 != ::close(fd))
writeError("Cannot close the file with coverage data\n");
}
};
dump(fmt::format("{}.{}", coverage_filename_prefix, getpid()), getCumulativeCoverage());
}
}
#endif
}
bool isClickhouseApp(std::string_view app_suffix, std::vector<char *> & argv)
@ -512,6 +505,12 @@ int main(int argc_, char ** argv_)
if (main_func == printHelp && !argv.empty() && (argv.size() == 1 || argv[1][0] == '-'))
main_func = mainEntryClickHouseLocal;
return main_func(static_cast<int>(argv.size()), argv.data());
int exit_code = main_func(static_cast<int>(argv.size()), argv.data());
#if defined(SANITIZE_COVERAGE)
dumpCoverage();
#endif
return exit_code;
}
#endif

View File

@ -558,7 +558,7 @@ static void sanityChecks(Server & server)
{
const char * filename = "/proc/sys/kernel/task_delayacct";
if (readNumber(filename) == 0)
server.context()->addWarningMessage("Delay accounting is not enabled, OSIOWaitMicroseconds will not be gathered. Check " + String(filename));
server.context()->addWarningMessage("Delay accounting is not enabled, OSIOWaitMicroseconds will not be gathered. You can enable it using `echo 1 > " + String(filename) + "` or by using sysctl.");
}
catch (...) // NOLINT(bugprone-empty-catch)
{

View File

@ -937,6 +937,11 @@
</macros>
-->
<!--
<default_replica_path>/clickhouse/tables/{database}/{table}</default_replica_path>
<default_replica_name>{replica}</default_replica_name>
-->
<!-- Replica group name for database Replicated.
The cluster created by Replicated database will consist of replicas in the same group.
DDL queries will only wail for the replicas in the same group.

View File

@ -107,6 +107,7 @@ try
if (argc < 3)
{
std::cout << "A tool similar to 'su'" << std::endl;
std::cout << "Usage: ./clickhouse su user:group ..." << std::endl;
exit(0); // NOLINT(concurrency-mt-unsafe)
}

View File

@ -14,6 +14,11 @@ macro(configure_rustc)
set(RUST_CFLAGS "${RUST_CFLAGS} --sysroot ${CMAKE_SYSROOT}")
endif()
if (CMAKE_OSX_DEPLOYMENT_TARGET)
set(RUST_CXXFLAGS "${RUST_CXXFLAGS} -mmacosx-version-min=${CMAKE_OSX_DEPLOYMENT_TARGET}")
set(RUST_CFLAGS "${RUST_CFLAGS} -mmacosx-version-min=${CMAKE_OSX_DEPLOYMENT_TARGET}")
endif()
if (USE_MUSL)
set(RUST_CXXFLAGS "${RUST_CXXFLAGS} -D_LIBCPP_HAS_MUSL_LIBC=1")
endif ()
@ -25,14 +30,23 @@ macro(configure_rustc)
set(RUSTCWRAPPER "")
endif()
set(RUSTFLAGS "[]")
set(RUSTFLAGS)
if (CMAKE_OSX_DEPLOYMENT_TARGET)
list(APPEND RUSTFLAGS "-C" "link-arg=-mmacosx-version-min=${CMAKE_OSX_DEPLOYMENT_TARGET}")
endif()
set(RUST_CARGO_BUILD_STD "")
# For more info: https://doc.rust-lang.org/beta/unstable-book/compiler-flags/sanitizer.html#memorysanitizer
if (SANITIZE STREQUAL "memory")
set(RUST_CARGO_BUILD_STD "build-std = [\"std\", \"panic_abort\", \"core\", \"alloc\"]")
set(RUSTFLAGS "[\"-Zsanitizer=memory\", \"-Zsanitizer-memory-track-origins\"]")
list(APPEND RUSTFLAGS "-Zsanitizer=memory" "-Zsanitizer-memory-track-origins")
endif()
list(TRANSFORM RUSTFLAGS PREPEND "\"")
list(TRANSFORM RUSTFLAGS APPEND "\"")
list(JOIN RUSTFLAGS "," RUSTFLAGS)
set(RUSTFLAGS "[${RUSTFLAGS}]")
message(STATUS "RUST_CFLAGS: ${RUST_CFLAGS}")
message(STATUS "RUST_CXXFLAGS: ${RUST_CXXFLAGS}")
message(STATUS "RUSTFLAGS: ${RUSTFLAGS}")

View File

@ -156,7 +156,6 @@ public:
void read(ReadBuffer & in)
{
size_t new_size = 0;
auto * const position = in.position();
readVarUInt(new_size, in);
if (new_size > 100'000'000'000)
throw DB::Exception(
@ -174,8 +173,14 @@ public:
}
else
{
in.position() = position; // Rollback position
asSingleLevel().read(in);
asSingleLevel().reserve(new_size);
for (size_t i = 0; i < new_size; ++i)
{
typename SingleLevelSet::Cell x;
x.read(in);
asSingleLevel().insert(x.getValue());
}
}
}

View File

@ -128,7 +128,10 @@ ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const
}
}
if (need_to_add_cast_function)
// Add cast if constant was created as a result of constant folding.
// Constant folding may lead to type transformation and literal on shard
// may have a different type.
if (need_to_add_cast_function || source_expression != nullptr)
{
auto constant_type_name_ast = std::make_shared<ASTLiteral>(constant_value->getType()->getName());
return makeASTFunction("_CAST", std::move(constant_value_ast), std::move(constant_type_name_ast));

View File

@ -2171,21 +2171,45 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_
node_to_replace = &sort_node->getExpression();
auto * constant_node = (*node_to_replace)->as<ConstantNode>();
if (!constant_node || constant_node->getValue().getType() != Field::Types::UInt64)
if (!constant_node
|| (constant_node->getValue().getType() != Field::Types::UInt64 && constant_node->getValue().getType() != Field::Types::Int64))
continue;
UInt64 positional_argument_number = constant_node->getValue().get<UInt64>();
if (positional_argument_number == 0 || positional_argument_number > projection_nodes.size())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
UInt64 pos;
if (constant_node->getValue().getType() == Field::Types::UInt64)
{
pos = constant_node->getValue().get<UInt64>();
}
else // Int64
{
auto value = constant_node->getValue().get<Int64>();
if (value > 0)
pos = value;
else
{
if (static_cast<size_t>(std::abs(value)) > projection_nodes.size())
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Negative positional argument number {} is out of bounds. Expected in range [-{}, -1]. In scope {}",
value,
projection_nodes.size(),
scope.scope_node->formatASTForErrorMessage());
pos = projection_nodes.size() + value + 1;
}
}
if (!pos || pos > projection_nodes.size())
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Positional argument number {} is out of bounds. Expected in range [1, {}]. In scope {}",
positional_argument_number,
pos,
projection_nodes.size(),
scope.scope_node->formatASTForErrorMessage());
--positional_argument_number;
*node_to_replace = projection_nodes[positional_argument_number]->clone();
if (auto it = resolved_expressions.find(projection_nodes[positional_argument_number]);
it != resolved_expressions.end())
--pos;
*node_to_replace = projection_nodes[pos]->clone();
if (auto it = resolved_expressions.find(projection_nodes[pos]); it != resolved_expressions.end())
{
resolved_expressions[*node_to_replace] = it->second;
}

View File

@ -57,9 +57,9 @@ namespace
if (size_t uuid_pos = zookeeper_path_arg.find(table_uuid_str); uuid_pos != String::npos)
zookeeper_path_arg.replace(uuid_pos, table_uuid_str.size(), "{uuid}");
}
const auto & config = data.global_context->getConfigRef();
if ((zookeeper_path_arg == StorageReplicatedMergeTree::getDefaultZooKeeperPath(config))
&& (replica_name_arg == StorageReplicatedMergeTree::getDefaultReplicaName(config))
const auto & server_settings = data.global_context->getServerSettings();
if ((zookeeper_path_arg == server_settings.default_replica_path.value)
&& (replica_name_arg == server_settings.default_replica_name.value)
&& ((engine_args.size() == 2) || !engine_args[2]->as<ASTLiteral>()))
{
engine_args.erase(engine_args.begin(), engine_args.begin() + 2);

View File

@ -203,6 +203,7 @@ void ColumnFixedString::updatePermutation(IColumn::PermutationSortDirection dire
void ColumnFixedString::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
const ColumnFixedString & src_concrete = assert_cast<const ColumnFixedString &>(src);
chassert(this->n == src_concrete.n);
if (start + length > src_concrete.size())
throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Parameters start = {}, length = {} are out of bound "

View File

@ -130,15 +130,21 @@ public:
int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override
{
const ColumnFixedString & rhs = assert_cast<const ColumnFixedString &>(rhs_);
chassert(this->n == rhs.n);
return memcmpSmallAllowOverflow15(chars.data() + p1 * n, rhs.chars.data() + p2 * n, n);
}
void compareColumn(const IColumn & rhs, size_t rhs_row_num,
PaddedPODArray<UInt64> * row_indexes, PaddedPODArray<Int8> & compare_results,
int direction, int nan_direction_hint) const override
void compareColumn(
const IColumn & rhs_,
size_t rhs_row_num,
PaddedPODArray<UInt64> * row_indexes,
PaddedPODArray<Int8> & compare_results,
int direction,
int nan_direction_hint) const override
{
return doCompareColumn<ColumnFixedString>(assert_cast<const ColumnFixedString &>(rhs), rhs_row_num, row_indexes,
compare_results, direction, nan_direction_hint);
const ColumnFixedString & rhs = assert_cast<const ColumnFixedString &>(rhs_);
chassert(this->n == rhs.n);
return doCompareColumn<ColumnFixedString>(rhs, rhs_row_num, row_indexes, compare_results, direction, nan_direction_hint);
}
bool hasEqualValues() const override

View File

@ -175,15 +175,6 @@ String Macros::expand(const String & s) const
return expand(s, info);
}
String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const
{
MacroExpansionInfo info;
info.table_id = table_id;
if (!allow_uuid)
info.table_id.uuid = UUIDHelpers::Nil;
return expand(s, info);
}
Names Macros::expand(const Names & source_names, size_t level) const
{
Names result_names;

View File

@ -57,8 +57,6 @@ public:
String expand(const String & s) const;
String expand(const String & s, const StorageID & table_id, bool allow_uuid) const;
/** Apply expand for the list.
*/

View File

@ -13,6 +13,9 @@
#undef __msan_unpoison_string
#define __msan_unpoison(X, Y) /// NOLINT
/// Given a pointer and **its size**, unpoisons 15 bytes **at the end**
/// See memcmpSmall.h / memcpySmall.h
#define __msan_unpoison_overflow_15(X, Y) /// NOLINT
#define __msan_test_shadow(X, Y) (false) /// NOLINT
#define __msan_print_shadow(X, Y) /// NOLINT
#define __msan_unpoison_string(X) /// NOLINT
@ -24,6 +27,8 @@
# undef __msan_print_shadow
# undef __msan_unpoison_string
# include <sanitizer/msan_interface.h>
# undef __msan_unpoison_overflow_15
# define __msan_unpoison_overflow_15(PTR, PTR_SIZE) __msan_unpoison(&(PTR)[(PTR_SIZE)], 15)
# endif
#endif

View File

@ -8,6 +8,7 @@
M(Query, "Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries.") \
M(SelectQuery, "Same as Query, but only for SELECT queries.") \
M(InsertQuery, "Same as Query, but only for INSERT queries.") \
M(InitialQuery, "Same as Query, but only counts initial queries (see is_initial_query).")\
M(QueriesWithSubqueries, "Count queries with all subqueries") \
M(SelectQueriesWithSubqueries, "Count SELECT queries with all subqueries") \
M(InsertQueriesWithSubqueries, "Count INSERT queries with all subqueries") \

View File

@ -401,6 +401,9 @@ ZooKeeper::ZooKeeper(
keeper_feature_flags.logFlags(log);
ProfileEvents::increment(ProfileEvents::ZooKeeperInit);
/// Avoid stale reads after connecting
sync("/", [](const SyncResponse &){});
}
catch (...)
{

View File

@ -7,6 +7,7 @@
#include <base/simd.h>
#include <Core/Defines.h>
#include <Common/MemorySanitizer.h>
namespace detail
@ -26,9 +27,8 @@ inline int cmp(T a, T b)
/// We can process uninitialized memory in the functions below.
/// Results don't depend on the values inside uninitialized memory but Memory Sanitizer cannot see it.
/// Disable optimized functions if compile with Memory Sanitizer.
#if defined(__AVX512BW__) && defined(__AVX512VL__) && !defined(MEMORY_SANITIZER)
/// Results don't depend on the values inside uninitialized memory
#if defined(__AVX512BW__) && defined(__AVX512VL__)
# include <immintrin.h>
@ -42,6 +42,9 @@ inline int cmp(T a, T b)
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
__msan_unpoison_overflow_15(a, a_size);
__msan_unpoison_overflow_15(b, b_size);
size_t min_size = std::min(a_size, b_size);
for (size_t offset = 0; offset < min_size; offset += 16)
@ -74,6 +77,9 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char
template <typename Char>
inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
__msan_unpoison_overflow_15(a, a_size);
__msan_unpoison_overflow_15(b, b_size);
size_t min_size = std::min(a_size, b_size);
for (size_t offset = 0; offset < min_size; offset += 16)
@ -144,6 +150,9 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size)
{
__msan_unpoison_overflow_15(a, size);
__msan_unpoison_overflow_15(b, size);
for (size_t offset = 0; offset < size; offset += 16)
{
uint16_t mask = _mm_cmp_epi8_mask(
@ -174,6 +183,9 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch
if (a_size != b_size)
return false;
__msan_unpoison_overflow_15(a, a_size);
__msan_unpoison_overflow_15(b, b_size);
for (size_t offset = 0; offset < a_size; offset += 16)
{
uint16_t mask = _mm_cmp_epi8_mask(
@ -246,6 +258,7 @@ inline bool memequal16(const void * a, const void * b)
/** Compare memory region to zero */
inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
{
__msan_unpoison_overflow_15(reinterpret_cast<const char *>(data), size);
const __m128i zero16 = _mm_setzero_si128();
for (size_t offset = 0; offset < size; offset += 16)
@ -263,7 +276,7 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
return true;
}
#elif defined(__SSE2__) && !defined(MEMORY_SANITIZER)
#elif defined(__SSE2__)
# include <emmintrin.h>
@ -277,6 +290,9 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
__msan_unpoison_overflow_15(a, a_size);
__msan_unpoison_overflow_15(b, b_size);
size_t min_size = std::min(a_size, b_size);
for (size_t offset = 0; offset < min_size; offset += 16)
@ -309,6 +325,9 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char
template <typename Char>
inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
__msan_unpoison_overflow_15(a, a_size);
__msan_unpoison_overflow_15(b, b_size);
size_t min_size = std::min(a_size, b_size);
for (size_t offset = 0; offset < min_size; offset += 16)
@ -380,6 +399,9 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size)
{
__msan_unpoison_overflow_15(a, size);
__msan_unpoison_overflow_15(b, size);
for (size_t offset = 0; offset < size; offset += 16)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
@ -410,6 +432,9 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch
if (a_size != b_size)
return false;
__msan_unpoison_overflow_15(a, a_size);
__msan_unpoison_overflow_15(b, b_size);
for (size_t offset = 0; offset < a_size; offset += 16)
{
uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(
@ -483,6 +508,8 @@ inline bool memequal16(const void * a, const void * b)
/** Compare memory region to zero */
inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
{
__msan_unpoison_overflow_15(reinterpret_cast<const char *>(data), size);
const __m128i zero16 = _mm_setzero_si128();
for (size_t offset = 0; offset < size; offset += 16)
@ -509,6 +536,9 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
__msan_unpoison_overflow_15(a, a_size);
__msan_unpoison_overflow_15(b, b_size);
size_t min_size = std::min(a_size, b_size);
for (size_t offset = 0; offset < min_size; offset += 16)
@ -534,6 +564,9 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char
template <typename Char>
inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{
__msan_unpoison_overflow_15(a, a_size);
__msan_unpoison_overflow_15(b, b_size);
size_t min_size = std::min(a_size, b_size);
for (size_t offset = 0; offset < min_size; offset += 16)
@ -599,6 +632,9 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size)
{
__msan_unpoison_overflow_15(a, size);
__msan_unpoison_overflow_15(b, size);
for (size_t offset = 0; offset < size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqq_u8(
@ -625,6 +661,9 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch
if (a_size != b_size)
return false;
__msan_unpoison_overflow_15(a, a_size);
__msan_unpoison_overflow_15(b, b_size);
for (size_t offset = 0; offset < a_size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqq_u8(
@ -683,6 +722,7 @@ inline bool memequal16(const void * a, const void * b)
inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
{
__msan_unpoison_overflow_15(reinterpret_cast<const char *>(data), size);
for (size_t offset = 0; offset < size; offset += 16)
{
uint64_t mask = getNibbleMask(vceqzq_u8(vld1q_u8(reinterpret_cast<const unsigned char *>(data) + offset)));

View File

@ -1,5 +1,7 @@
#pragma once
#include <Common/MemorySanitizer.h>
#include <cstring>
#include <sys/types.h> /// ssize_t
@ -38,6 +40,7 @@ namespace detail
{
inline void memcpySmallAllowReadWriteOverflow15Impl(char * __restrict dst, const char * __restrict src, ssize_t n)
{
__msan_unpoison_overflow_15(src, n);
while (n > 0)
{
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst),
@ -64,6 +67,7 @@ namespace detail
{
inline void memcpySmallAllowReadWriteOverflow15Impl(char * __restrict dst, const char * __restrict src, ssize_t n)
{
__msan_unpoison_overflow_15(src, n);
while (n > 0)
{
vst1q_s8(reinterpret_cast<signed char *>(dst), vld1q_s8(reinterpret_cast<const signed char *>(src)));

View File

@ -16,7 +16,7 @@
#include <Coordination/pathUtils.h>
#include <Coordination/KeeperConstants.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include "Core/Field.h"
#include <Core/Field.h>
#include <Disks/DiskLocal.h>
@ -79,20 +79,20 @@ namespace
writeBinary(false, out);
/// Serialize stat
writeBinary(node.stat.czxid, out);
writeBinary(node.stat.mzxid, out);
writeBinary(node.stat.ctime, out);
writeBinary(node.stat.mtime, out);
writeBinary(node.stat.version, out);
writeBinary(node.stat.cversion, out);
writeBinary(node.stat.aversion, out);
writeBinary(node.stat.ephemeralOwner, out);
writeBinary(node.czxid, out);
writeBinary(node.mzxid, out);
writeBinary(node.ctime(), out);
writeBinary(node.mtime, out);
writeBinary(node.version, out);
writeBinary(node.cversion, out);
writeBinary(node.aversion, out);
writeBinary(node.ephemeralOwner(), out);
if (version < SnapshotVersion::V6)
writeBinary(static_cast<int32_t>(node.getData().size()), out);
writeBinary(node.stat.numChildren, out);
writeBinary(node.stat.pzxid, out);
writeBinary(static_cast<int32_t>(node.data_size), out);
writeBinary(node.numChildren(), out);
writeBinary(node.pzxid, out);
writeBinary(node.seq_num, out);
writeBinary(node.seqNum(), out);
if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5)
writeBinary(node.sizeInBytes(), out);
@ -102,7 +102,7 @@ namespace
{
String new_data;
readBinary(new_data, in);
node.setData(std::move(new_data));
node.setData(new_data);
if (version >= SnapshotVersion::V1)
{
@ -138,22 +138,36 @@ namespace
}
/// Deserialize stat
readBinary(node.stat.czxid, in);
readBinary(node.stat.mzxid, in);
readBinary(node.stat.ctime, in);
readBinary(node.stat.mtime, in);
readBinary(node.stat.version, in);
readBinary(node.stat.cversion, in);
readBinary(node.stat.aversion, in);
readBinary(node.stat.ephemeralOwner, in);
readBinary(node.czxid, in);
readBinary(node.mzxid, in);
int64_t ctime;
readBinary(ctime, in);
node.setCtime(ctime);
readBinary(node.mtime, in);
readBinary(node.version, in);
readBinary(node.cversion, in);
readBinary(node.aversion, in);
int64_t ephemeral_owner = 0;
readBinary(ephemeral_owner, in);
if (ephemeral_owner != 0)
node.setEphemeralOwner(ephemeral_owner);
if (version < SnapshotVersion::V6)
{
int32_t data_length = 0;
readBinary(data_length, in);
}
readBinary(node.stat.numChildren, in);
readBinary(node.stat.pzxid, in);
readBinary(node.seq_num, in);
int32_t num_children = 0;
readBinary(num_children, in);
if (ephemeral_owner == 0)
node.setNumChildren(num_children);
readBinary(node.pzxid, in);
int32_t seq_num = 0;
readBinary(seq_num, in);
if (ephemeral_owner == 0)
node.setSeqNum(seq_num);
if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5)
{
@ -238,7 +252,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
/// Benign race condition possible while taking snapshot: NuRaft decide to create snapshot at some log id
/// and only after some time we lock storage and enable snapshot mode. So snapshot_container_size can be
/// slightly bigger than required.
if (node.stat.mzxid > snapshot.zxid)
if (node.mzxid > snapshot.zxid)
break;
writeBinary(path, out);
@ -363,11 +377,6 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
if (recalculate_digest)
storage.nodes_digest = 0;
const auto is_node_empty = [](const auto & node)
{
return node.getData().empty() && node.stat == KeeperStorage::Node::Stat{};
};
for (size_t nodes_read = 0; nodes_read < snapshot_container_size; ++nodes_read)
{
std::string path;
@ -395,7 +404,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
}
else if (match_result == EXACT)
{
if (!is_node_empty(node))
if (!node.empty())
{
if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT)
{
@ -412,8 +421,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
}
storage.container.insertOrReplace(path, node);
if (node.stat.ephemeralOwner != 0)
storage.ephemerals[node.stat.ephemeralOwner].insert(path);
if (node.isEphemeral())
storage.ephemerals[node.ephemeralOwner()].insert(path);
if (recalculate_digest)
storage.nodes_digest += node.getDigest(path);
@ -433,16 +442,16 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
{
if (itr.key != "/")
{
if (itr.value.stat.numChildren != static_cast<int32_t>(itr.value.getChildren().size()))
if (itr.value.numChildren() != static_cast<int32_t>(itr.value.getChildren().size()))
{
#ifdef NDEBUG
/// TODO (alesapin) remove this, it should be always CORRUPTED_DATA.
LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}", itr.value.stat.numChildren, itr.value.getChildren().size(), itr.key);
" is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
#else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}",
itr.value.stat.numChildren, itr.value.getChildren().size(), itr.key);
itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
#endif
}
}

View File

@ -166,54 +166,132 @@ KeeperStorage::ResponsesForSessions processWatchesImpl(
}
// When this function is updated, update CURRENT_DIGEST_VERSION!!
uint64_t calculateDigest(std::string_view path, std::string_view data, const KeeperStorage::Node::Stat & stat)
uint64_t calculateDigest(std::string_view path, const KeeperStorage::Node & node)
{
SipHash hash;
hash.update(path);
hash.update(data);
auto data = node.getData();
if (!data.empty())
{
chassert(data.data() != nullptr);
hash.update(data);
}
hash.update(stat.czxid);
hash.update(stat.mzxid);
hash.update(stat.ctime);
hash.update(stat.mtime);
hash.update(stat.version);
hash.update(stat.cversion);
hash.update(stat.aversion);
hash.update(stat.ephemeralOwner);
hash.update(stat.numChildren);
hash.update(stat.pzxid);
hash.update(node.czxid);
hash.update(node.mzxid);
hash.update(node.ctime());
hash.update(node.mtime);
hash.update(node.version);
hash.update(node.cversion);
hash.update(node.aversion);
hash.update(node.ephemeralOwner());
hash.update(node.numChildren());
hash.update(node.pzxid);
return hash.get64();
auto digest = hash.get64();
/// 0 means no cached digest
if (digest == 0)
return 1;
return digest;
}
}
KeeperStorage::Node & KeeperStorage::Node::operator=(const Node & other)
{
if (this == &other)
return *this;
czxid = other.czxid;
mzxid = other.mzxid;
pzxid = other.pzxid;
acl_id = other.acl_id;
mtime = other.mtime;
is_ephemeral_and_ctime = other.is_ephemeral_and_ctime;
ephemeral_or_children_data = other.ephemeral_or_children_data;
data_size = other.data_size;
version = other.version;
cversion = other.cversion;
aversion = other.aversion;
if (data_size != 0)
{
data = std::unique_ptr<char[]>(new char[data_size]);
memcpy(data.get(), other.data.get(), data_size);
}
children = other.children;
return *this;
}
KeeperStorage::Node::Node(const Node & other)
{
*this = other;
}
bool KeeperStorage::Node::empty() const
{
return data_size == 0 && mzxid == 0;
}
void KeeperStorage::Node::copyStats(const Coordination::Stat & stat)
{
czxid = stat.czxid;
mzxid = stat.mzxid;
pzxid = stat.pzxid;
mtime = stat.mtime;
setCtime(stat.ctime);
version = stat.version;
cversion = stat.cversion;
aversion = stat.aversion;
if (stat.ephemeralOwner == 0)
{
is_ephemeral_and_ctime.is_ephemeral = false;
setNumChildren(stat.numChildren);
}
else
{
setEphemeralOwner(stat.ephemeralOwner);
}
}
void KeeperStorage::Node::setResponseStat(Coordination::Stat & response_stat) const
{
response_stat.czxid = stat.czxid;
response_stat.mzxid = stat.mzxid;
response_stat.ctime = stat.ctime;
response_stat.mtime = stat.mtime;
response_stat.version = stat.version;
response_stat.cversion = stat.cversion;
response_stat.aversion = stat.aversion;
response_stat.ephemeralOwner = stat.ephemeralOwner;
response_stat.dataLength = static_cast<int32_t>(data.size());
response_stat.numChildren = stat.numChildren;
response_stat.pzxid = stat.pzxid;
response_stat.czxid = czxid;
response_stat.mzxid = mzxid;
response_stat.ctime = ctime();
response_stat.mtime = mtime;
response_stat.version = version;
response_stat.cversion = cversion;
response_stat.aversion = aversion;
response_stat.ephemeralOwner = ephemeralOwner();
response_stat.dataLength = static_cast<int32_t>(data_size);
response_stat.numChildren = numChildren();
response_stat.pzxid = pzxid;
}
uint64_t KeeperStorage::Node::sizeInBytes() const
{
return sizeof(Node) + children.size() * sizeof(StringRef) + data.size();
return sizeof(Node) + children.size() * sizeof(StringRef) + data_size;
}
void KeeperStorage::Node::setData(String new_data)
void KeeperStorage::Node::setData(const String & new_data)
{
data = std::move(new_data);
data_size = static_cast<uint32_t>(new_data.size());
if (data_size != 0)
{
data = std::unique_ptr<char[]>(new char[new_data.size()]);
memcpy(data.get(), new_data.data(), data_size);
}
}
void KeeperStorage::Node::addChild(StringRef child_path)
@ -228,25 +306,41 @@ void KeeperStorage::Node::removeChild(StringRef child_path)
void KeeperStorage::Node::invalidateDigestCache() const
{
has_cached_digest = false;
cached_digest = 0;
}
UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const
{
if (!has_cached_digest)
{
cached_digest = calculateDigest(path, data, stat);
has_cached_digest = true;
}
if (cached_digest == 0)
cached_digest = calculateDigest(path, *this);
return cached_digest;
};
void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other)
{
stat = other.stat;
seq_num = other.seq_num;
setData(other.getData());
czxid = other.czxid;
mzxid = other.mzxid;
pzxid = other.pzxid;
acl_id = other.acl_id; /// 0 -- no ACL by default
mtime = other.mtime;
is_ephemeral_and_ctime = other.is_ephemeral_and_ctime;
ephemeral_or_children_data = other.ephemeral_or_children_data;
data_size = other.data_size;
if (data_size != 0)
{
data = std::unique_ptr<char[]>(new char[data_size]);
memcpy(data.get(), other.data.get(), data_size);
}
version = other.version;
cversion = other.cversion;
aversion = other.aversion;
cached_digest = other.cached_digest;
}
@ -278,13 +372,13 @@ void KeeperStorage::initializeSystemNodes()
// update root and the digest based on it
auto current_root_it = container.find("/");
assert(current_root_it != container.end());
chassert(current_root_it != container.end());
removeDigest(current_root_it->value, "/");
auto updated_root_it = container.updateValue(
"/",
[](auto & node)
[](KeeperStorage::Node & node)
{
++node.stat.numChildren;
node.increaseNumChildren();
node.addChild(getBaseNodeName(keeper_system_path));
}
);
@ -294,7 +388,7 @@ void KeeperStorage::initializeSystemNodes()
// insert child system nodes
for (const auto & [path, data] : keeper_context->getSystemNodesWithData())
{
assert(path.starts_with(keeper_system_path));
chassert(path.starts_with(keeper_system_path));
Node child_system_node;
child_system_node.setData(data);
auto [map_key, _] = container.insert(std::string{path}, child_system_node);
@ -339,7 +433,7 @@ std::shared_ptr<KeeperStorage::Node> KeeperStorage::UncommittedState::tryGetNode
void KeeperStorage::UncommittedState::applyDelta(const Delta & delta)
{
assert(!delta.path.empty());
chassert(!delta.path.empty());
if (!nodes.contains(delta.path))
{
if (auto storage_node = tryGetNodeFromStorage(delta.path))
@ -355,22 +449,22 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta)
if constexpr (std::same_as<DeltaType, CreateNodeDelta>)
{
assert(!node);
chassert(!node);
node = std::make_shared<Node>();
node->stat = operation.stat;
node->copyStats(operation.stat);
node->setData(operation.data);
acls = operation.acls;
last_applied_zxid = delta.zxid;
}
else if constexpr (std::same_as<DeltaType, RemoveNodeDelta>)
{
assert(node);
chassert(node);
node = nullptr;
last_applied_zxid = delta.zxid;
}
else if constexpr (std::same_as<DeltaType, UpdateNodeDelta>)
{
assert(node);
chassert(node);
node->invalidateDigestCache();
operation.update_fn(*node);
last_applied_zxid = delta.zxid;
@ -384,6 +478,40 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta)
delta.operation);
}
bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool is_local, std::function<bool(const AuthID &)> predicate) const
{
const auto check_auth = [&](const auto & auth_ids)
{
for (const auto & auth : auth_ids)
{
using TAuth = std::remove_reference_t<decltype(auth)>;
const AuthID * auth_ptr = nullptr;
if constexpr (std::is_pointer_v<TAuth>)
auth_ptr = auth;
else
auth_ptr = &auth;
if (predicate(*auth_ptr))
return true;
}
return false;
};
if (is_local)
return check_auth(storage.session_and_auth[session_id]);
if (check_auth(storage.session_and_auth[session_id]))
return true;
// check if there are uncommitted
const auto auth_it = session_and_auth.find(session_id);
if (auth_it == session_and_auth.end())
return false;
return check_auth(auth_it->second);
}
void KeeperStorage::UncommittedState::addDelta(Delta new_delta)
{
const auto & added_delta = deltas.emplace_back(std::move(new_delta));
@ -408,7 +536,7 @@ void KeeperStorage::UncommittedState::addDeltas(std::vector<Delta> new_deltas)
void KeeperStorage::UncommittedState::commit(int64_t commit_zxid)
{
assert(deltas.empty() || deltas.front().zxid >= commit_zxid);
chassert(deltas.empty() || deltas.front().zxid >= commit_zxid);
// collect nodes that have no further modification in the current transaction
std::unordered_set<std::string> modified_nodes;
@ -426,7 +554,7 @@ void KeeperStorage::UncommittedState::commit(int64_t commit_zxid)
if (!front_delta.path.empty())
{
auto & path_deltas = deltas_for_path.at(front_delta.path);
assert(path_deltas.front() == &front_delta);
chassert(path_deltas.front() == &front_delta);
path_deltas.pop_front();
if (path_deltas.empty())
{
@ -444,7 +572,7 @@ void KeeperStorage::UncommittedState::commit(int64_t commit_zxid)
else if (auto * add_auth = std::get_if<AddAuthDelta>(&front_delta.operation))
{
auto & uncommitted_auth = session_and_auth[add_auth->session_id];
assert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id);
chassert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id);
uncommitted_auth.pop_front();
if (uncommitted_auth.empty())
session_and_auth.erase(add_auth->session_id);
@ -484,7 +612,7 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid)
if (delta_it->zxid < rollback_zxid)
break;
assert(delta_it->zxid == rollback_zxid);
chassert(delta_it->zxid == rollback_zxid);
if (!delta_it->path.empty())
{
std::visit(
@ -671,7 +799,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid)
if (node_it == container.end())
onStorageInconsistency();
if (operation.version != -1 && operation.version != node_it->value.stat.version)
if (operation.version != -1 && operation.version != node_it->value.version)
onStorageInconsistency();
removeDigest(node_it->value, path);
@ -693,7 +821,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid)
if (node_it == container.end())
onStorageInconsistency();
if (operation.version != -1 && operation.version != node_it->value.stat.aversion)
if (operation.version != -1 && operation.version != node_it->value.aversion)
onStorageInconsistency();
acl_map.removeUsage(node_it->value.acl_id);
@ -738,7 +866,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid)
bool KeeperStorage::createNode(
const std::string & path,
String data,
const KeeperStorage::Node::Stat & stat,
const Coordination::Stat & stat,
Coordination::ACLs node_acls)
{
auto parent_path = parentNodePath(path);
@ -747,7 +875,7 @@ bool KeeperStorage::createNode(
if (node_it == container.end())
return false;
if (node_it->value.stat.ephemeralOwner != 0)
if (node_it->value.isEphemeral())
return false;
if (container.contains(path))
@ -759,8 +887,8 @@ bool KeeperStorage::createNode(
acl_map.addUsage(acl_id);
created_node.acl_id = acl_id;
created_node.stat = stat;
created_node.setData(std::move(data));
created_node.copyStats(stat);
created_node.setData(data);
auto [map_key, _] = container.insert(path, created_node);
/// Take child path from key owned by map.
auto child_path = getBaseNodeName(map_key->getKey());
@ -769,7 +897,7 @@ bool KeeperStorage::createNode(
[child_path](KeeperStorage::Node & parent)
{
parent.addChild(child_path);
chassert(parent.stat.numChildren == static_cast<int32_t>(parent.getChildren().size()));
chassert(parent.numChildren() == static_cast<int32_t>(parent.getChildren().size()));
}
);
@ -783,21 +911,22 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t version)
if (node_it == container.end())
return false;
if (version != -1 && version != node_it->value.stat.version)
if (version != -1 && version != node_it->value.version)
return false;
if (node_it->value.stat.numChildren)
if (node_it->value.numChildren())
return false;
auto prev_node = node_it->value;
acl_map.removeUsage(prev_node.acl_id);
KeeperStorage::Node prev_node;
prev_node.shallowCopy(node_it->value);
acl_map.removeUsage(node_it->value.acl_id);
container.updateValue(
parentNodePath(path),
[child_basename = getBaseNodeName(node_it->key)](KeeperStorage::Node & parent)
{
parent.removeChild(child_basename);
chassert(parent.stat.numChildren == static_cast<int32_t>(parent.getChildren().size()));
chassert(parent.numChildren() == static_cast<int32_t>(parent.getChildren().size()));
}
);
@ -957,7 +1086,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
if (parent_node == nullptr)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
else if (parent_node->stat.ephemeralOwner != 0)
else if (parent_node->isEphemeral())
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}};
std::string path_created = request.path;
@ -966,7 +1095,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
if (request.not_exists)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
auto seq_num = parent_node->seq_num;
auto seq_num = parent_node->seqNum();
std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
seq_num_str.exceptions(std::ios::failbit);
@ -1006,20 +1135,20 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
auto parent_update = [parent_cversion, zxid](KeeperStorage::Node & node)
{
/// Increment sequential number even if node is not sequential
++node.seq_num;
node.increaseSeqNum();
if (parent_cversion == -1)
++node.stat.cversion;
else if (parent_cversion > node.stat.cversion)
node.stat.cversion = parent_cversion;
++node.cversion;
else if (parent_cversion > node.cversion)
node.cversion = parent_cversion;
if (zxid > node.stat.pzxid)
node.stat.pzxid = zxid;
++node.stat.numChildren;
if (zxid > node.pzxid)
node.pzxid = zxid;
node.increaseNumChildren();
};
new_deltas.emplace_back(std::string{parent_path}, zxid, KeeperStorage::UpdateNodeDelta{std::move(parent_update)});
KeeperStorage::Node::Stat stat;
Coordination::Stat stat;
stat.czxid = zxid;
stat.mzxid = zxid;
stat.pzxid = zxid;
@ -1133,7 +1262,8 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
else
{
node_it->value.setResponseStat(response.stat);
response.data = node_it->value.getData();
auto data = node_it->value.getData();
response.data = std::string(data);
response.error = Coordination::Error::ZOK;
}
@ -1190,8 +1320,8 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
{
[zxid](KeeperStorage::Node & parent)
{
if (parent.stat.pzxid < zxid)
parent.stat.pzxid = zxid;
if (parent.pzxid < zxid)
parent.pzxid = zxid;
}
}
);
@ -1205,9 +1335,9 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
update_parent_pzxid();
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
}
else if (request.version != -1 && request.version != node->stat.version)
else if (request.version != -1 && request.version != node->version)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
else if (node->stat.numChildren != 0)
else if (node->numChildren() != 0)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOTEMPTY}};
if (request.restored_from_zookeeper_log)
@ -1218,14 +1348,14 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
zxid,
KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent)
{
++parent.stat.cversion;
--parent.stat.numChildren;
++parent.cversion;
parent.decreaseNumChildren();
}});
new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->stat.ephemeralOwner});
new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->ephemeralOwner()});
if (node->stat.ephemeralOwner != 0)
storage.unregisterEphemeralPath(node->stat.ephemeralOwner, request.path);
if (node->isEphemeral())
storage.unregisterEphemeralPath(node->ephemeralOwner(), request.path);
digest = storage.calculateNodesDigest(digest, new_deltas);
@ -1339,7 +1469,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
auto node = storage.uncommitted_state.getNode(request.path);
if (request.version != -1 && request.version != node->stat.version)
if (request.version != -1 && request.version != node->version)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
new_deltas.emplace_back(
@ -1348,9 +1478,9 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
KeeperStorage::UpdateNodeDelta{
[zxid, data = request.data, time](KeeperStorage::Node & value)
{
value.stat.version++;
value.stat.mzxid = zxid;
value.stat.mtime = time;
value.version++;
value.mzxid = zxid;
value.mtime = time;
value.setData(data);
},
request.version});
@ -1362,7 +1492,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
{
[](KeeperStorage::Node & parent)
{
parent.stat.cversion++;
parent.cversion++;
}
}
);
@ -1464,9 +1594,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
auto list_request_type = ALL;
if (auto * filtered_list = dynamic_cast<Coordination::ZooKeeperFilteredListRequest *>(&request))
{
list_request_type = filtered_list->list_request_type;
}
if (list_request_type == ALL)
return true;
@ -1476,7 +1604,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
if (child_it == container.end())
onStorageInconsistency();
const auto is_ephemeral = child_it->value.stat.ephemeralOwner != 0;
const auto is_ephemeral = child_it->value.isEphemeral();
return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY);
};
@ -1529,7 +1657,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
auto node = storage.uncommitted_state.getNode(request.path);
if (check_not_exists)
{
if (node && (request.version == -1 || request.version == node->stat.version))
if (node && (request.version == -1 || request.version == node->version))
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}};
}
else
@ -1537,7 +1665,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
if (!node)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
if (request.version != -1 && request.version != node->stat.version)
if (request.version != -1 && request.version != node->version)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
}
@ -1573,7 +1701,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
if (check_not_exists)
{
if (node_it != container.end() && (request.version == -1 || request.version == node_it->value.stat.version))
if (node_it != container.end() && (request.version == -1 || request.version == node_it->value.version))
on_error(Coordination::Error::ZNODEEXISTS);
else
response.error = Coordination::Error::ZOK;
@ -1582,7 +1710,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
{
if (node_it == container.end())
on_error(Coordination::Error::ZNONODE);
else if (request.version != -1 && request.version != node_it->value.stat.version)
else if (request.version != -1 && request.version != node_it->value.version)
on_error(Coordination::Error::ZBADVERSION);
else
response.error = Coordination::Error::ZOK;
@ -1635,7 +1763,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr
auto node = uncommitted_state.getNode(request.path);
if (request.version != -1 && request.version != node->stat.aversion)
if (request.version != -1 && request.version != node->aversion)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
@ -1655,7 +1783,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr
zxid,
KeeperStorage::UpdateNodeDelta
{
[](KeeperStorage::Node & n) { ++n.stat.aversion; }
[](KeeperStorage::Node & n) { ++n.aversion; }
}
}
};
@ -1824,7 +1952,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
}
}
assert(request.requests.empty() || operation_type.has_value());
chassert(request.requests.empty() || operation_type.has_value());
}
std::vector<KeeperStorage::Delta>
@ -1873,7 +2001,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
auto & deltas = storage.uncommitted_state.deltas;
// the deltas will have at least SubDeltaEnd or FailedMultiDelta
assert(!deltas.empty());
chassert(!deltas.empty());
if (auto * failed_multi = std::get_if<KeeperStorage::FailedMultiDelta>(&deltas.front().operation))
{
for (size_t i = 0; i < concrete_requests.size(); ++i)
@ -2073,7 +2201,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec
[&](const CreateNodeDelta & create_delta)
{
auto node = std::make_shared<Node>();
node->stat = create_delta.stat;
node->copyStats(create_delta.stat);
node->setData(create_delta.data);
updated_nodes.emplace(delta.path, node);
},
@ -2196,8 +2324,8 @@ void KeeperStorage::preprocessRequest(
{
[ephemeral_path](Node & parent)
{
++parent.stat.cversion;
--parent.stat.numChildren;
++parent.cversion;
parent.decreaseNumChildren();
}
}
);
@ -2300,7 +2428,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(
if (is_local)
{
assert(zk_request->isReadRequest());
chassert(zk_request->isReadRequest());
if (check_acl && !request_processor->checkAuth(*this, session_id, true))
{
response = zk_request->makeResponse();

View File

@ -5,17 +5,15 @@
#include <Coordination/ACLMap.h>
#include <Coordination/SessionExpiryQueue.h>
#include <Coordination/SnapshotableHashTable.h>
#include <IO/WriteBufferFromString.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Coordination/KeeperContext.h>
#include <absl/container/flat_hash_set.h>
namespace DB
{
class KeeperContext;
using KeeperContextPtr = std::shared_ptr<KeeperContext>;
struct KeeperStorageRequestProcessor;
using KeeperStorageRequestProcessorPtr = std::shared_ptr<KeeperStorageRequestProcessor>;
using ResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr &)>;
@ -35,40 +33,113 @@ public:
/// New fields should be added to the struct only if it's really necessary
struct Node
{
/// to reduce size of the Node struct we use a custom Stat without dataLength
struct Stat
{
int64_t czxid{0};
int64_t mzxid{0};
int64_t ctime{0};
int64_t mtime{0};
int32_t version{0};
int32_t cversion{0};
int32_t aversion{0};
int32_t numChildren{0}; /// NOLINT
int64_t ephemeralOwner{0}; /// NOLINT
int64_t pzxid{0};
bool operator==(const Stat &) const = default;
};
int64_t czxid{0};
int64_t mzxid{0};
int64_t pzxid{0};
uint64_t acl_id = 0; /// 0 -- no ACL by default
Stat stat{};
int32_t seq_num = 0;
/// we cannot use `std::optional<uint64_t> because we want to
/// pack the boolean with seq_num above
mutable bool has_cached_digest = false;
int64_t mtime{0};
std::unique_ptr<char[]> data{nullptr};
uint32_t data_size{0};
int32_t version{0};
int32_t cversion{0};
int32_t aversion{0};
mutable uint64_t cached_digest = 0;
Node() = default;
Node & operator=(const Node & other);
Node(const Node & other);
bool empty() const;
bool isEphemeral() const
{
return is_ephemeral_and_ctime.is_ephemeral;
}
int64_t ephemeralOwner() const
{
if (isEphemeral())
return ephemeral_or_children_data.ephemeral_owner;
return 0;
}
void setEphemeralOwner(int64_t ephemeral_owner)
{
is_ephemeral_and_ctime.is_ephemeral = ephemeral_owner != 0;
ephemeral_or_children_data.ephemeral_owner = ephemeral_owner;
}
int32_t numChildren() const
{
if (isEphemeral())
return 0;
return ephemeral_or_children_data.children_info.num_children;
}
void setNumChildren(int32_t num_children)
{
ephemeral_or_children_data.children_info.num_children = num_children;
}
void increaseNumChildren()
{
chassert(!isEphemeral());
++ephemeral_or_children_data.children_info.num_children;
}
void decreaseNumChildren()
{
chassert(!isEphemeral());
--ephemeral_or_children_data.children_info.num_children;
}
int32_t seqNum() const
{
if (isEphemeral())
return 0;
return ephemeral_or_children_data.children_info.seq_num;
}
void setSeqNum(int32_t seq_num)
{
ephemeral_or_children_data.children_info.seq_num = seq_num;
}
void increaseSeqNum()
{
chassert(!isEphemeral());
++ephemeral_or_children_data.children_info.seq_num;
}
int64_t ctime() const
{
return is_ephemeral_and_ctime.ctime;
}
void setCtime(uint64_t ctime)
{
is_ephemeral_and_ctime.ctime = ctime;
}
void copyStats(const Coordination::Stat & stat);
void setResponseStat(Coordination::Stat & response_stat) const;
/// Object memory size
uint64_t sizeInBytes() const;
void setData(String new_data);
void setData(const String & new_data);
const auto & getData() const noexcept { return data; }
std::string_view getData() const noexcept { return {data.get(), data_size}; }
void addChild(StringRef child_path);
@ -87,19 +158,46 @@ public:
// (e.g. we don't need to copy list of children)
void shallowCopy(const Node & other);
private:
String data;
/// as ctime can't be negative because it stores the timestamp when the
/// node was created, we can use the MSB for a bool
struct
{
bool is_ephemeral : 1;
int64_t ctime : 63;
} is_ephemeral_and_ctime{false, 0};
/// ephemeral notes cannot have children so a node can set either
/// ephemeral_owner OR seq_num + num_children
union
{
int64_t ephemeral_owner;
struct
{
int32_t seq_num;
int32_t num_children;
} children_info;
} ephemeral_or_children_data{0};
ChildrenSet children{};
};
#if !defined(ADDRESS_SANITIZER) && !defined(MEMORY_SANITIZER)
static_assert(
sizeof(ListNode<Node>) <= 144,
"std::list node containing ListNode<Node> is > 160 bytes (sizeof(ListNode<Node>) + 16 bytes for pointers) which will increase "
"memory consumption");
#endif
enum DigestVersion : uint8_t
{
NO_DIGEST = 0,
V1 = 1,
V2 = 2, // added system nodes that modify the digest on startup so digest from V0 is invalid
V3 = 3 // fixed bug with casting, removed duplicate czxid usage
V3 = 3, // fixed bug with casting, removed duplicate czxid usage
V4 = 4 // 0 is not a valid digest value
};
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V3;
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V4;
struct ResponseForSession
{
@ -169,7 +267,7 @@ public:
// - quickly commit the changes to the storage
struct CreateNodeDelta
{
KeeperStorage::Node::Stat stat;
Coordination::Stat stat;
Coordination::ACLs acls;
String data;
};
@ -243,39 +341,7 @@ public:
void applyDelta(const Delta & delta);
bool hasACL(int64_t session_id, bool is_local, std::function<bool(const AuthID &)> predicate)
{
const auto check_auth = [&](const auto & auth_ids)
{
for (const auto & auth : auth_ids)
{
using TAuth = std::remove_reference_t<decltype(auth)>;
const AuthID * auth_ptr = nullptr;
if constexpr (std::is_pointer_v<TAuth>)
auth_ptr = auth;
else
auth_ptr = &auth;
if (predicate(*auth_ptr))
return true;
}
return false;
};
if (is_local)
return check_auth(storage.session_and_auth[session_id]);
if (check_auth(storage.session_and_auth[session_id]))
return true;
// check if there are uncommitted
const auto auth_it = session_and_auth.find(session_id);
if (auth_it == session_and_auth.end())
return false;
return check_auth(auth_it->second);
}
bool hasACL(int64_t session_id, bool is_local, std::function<bool(const AuthID &)> predicate) const;
void forEachAuthInSession(int64_t session_id, std::function<void(const AuthID &)> func) const;
@ -334,7 +400,7 @@ public:
bool createNode(
const std::string & path,
String data,
const KeeperStorage::Node::Stat & stat,
const Coordination::Stat & stat,
Coordination::ACLs node_acls);
// Remove node in the storage

View File

@ -2,72 +2,58 @@
#include <base/StringRef.h>
#include <Common/HashTable/HashMap.h>
#include <Common/ArenaUtils.h>
#include <list>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template<typename V>
struct ListNode
{
StringRef key;
V value;
/// |* * ****** |
/// ^ ^ ^
/// active_in_map free_key version
/// (1 byte) (1 byte) (6 bytes)
uint64_t node_metadata = 0;
struct
{
bool active_in_map : 1;
bool free_key : 1;
uint64_t version : 62;
} node_metadata{false, false, 0};
void setInactiveInMap()
{
node_metadata &= ~active_in_map_mask;
node_metadata.active_in_map = false;
}
void setActiveInMap()
{
node_metadata |= active_in_map_mask;
node_metadata.active_in_map = true;
}
bool isActiveInMap()
{
return node_metadata & active_in_map_mask;
return node_metadata.active_in_map;
}
void setFreeKey()
{
node_metadata |= free_key_mask;
node_metadata.free_key = true;
}
bool getFreeKey()
{
return node_metadata & free_key_mask;
return node_metadata.free_key;
}
uint64_t getVersion()
{
return node_metadata & version_mask;
return node_metadata.version;
}
void setVersion(uint64_t version)
{
if (version > version_mask)
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Snapshot version {} is larger than maximum allowed value {}", version, version_mask);
node_metadata &= ~version_mask;
node_metadata |= version;
node_metadata.version = version;
}
static constexpr uint64_t active_in_map_mask = static_cast<uint64_t>(1) << 63;
static constexpr uint64_t free_key_mask = static_cast<uint64_t>(1) << 62;
static constexpr uint64_t version_mask = ~(static_cast<uint64_t>(3) << 62);
};
template <class V>

View File

@ -101,30 +101,37 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP
KeeperStorage::Node node{};
String data;
Coordination::read(data, in);
node.setData(std::move(data));
node.setData(data);
Coordination::read(node.acl_id, in);
/// Deserialize stat
Coordination::read(node.stat.czxid, in);
Coordination::read(node.stat.mzxid, in);
Coordination::read(node.czxid, in);
Coordination::read(node.mzxid, in);
/// For some reason ZXID specified in filename can be smaller
/// then actual zxid from nodes. In this case we will use zxid from nodes.
max_zxid = std::max(max_zxid, node.stat.mzxid);
max_zxid = std::max(max_zxid, node.mzxid);
Coordination::read(node.stat.ctime, in);
Coordination::read(node.stat.mtime, in);
Coordination::read(node.stat.version, in);
Coordination::read(node.stat.cversion, in);
Coordination::read(node.stat.aversion, in);
Coordination::read(node.stat.ephemeralOwner, in);
Coordination::read(node.stat.pzxid, in);
int64_t ctime;
Coordination::read(ctime, in);
node.setCtime(ctime);
Coordination::read(node.mtime, in);
Coordination::read(node.version, in);
Coordination::read(node.cversion, in);
Coordination::read(node.aversion, in);
int64_t ephemeral_owner;
Coordination::read(ephemeral_owner, in);
if (ephemeral_owner != 0)
node.setEphemeralOwner(ephemeral_owner);
Coordination::read(node.pzxid, in);
if (!path.empty())
{
node.seq_num = node.stat.cversion;
if (ephemeral_owner == 0)
node.setSeqNum(node.cversion);
storage.container.insertOrReplace(path, node);
if (node.stat.ephemeralOwner != 0)
storage.ephemerals[node.stat.ephemeralOwner].insert(path);
if (ephemeral_owner != 0)
storage.ephemerals[ephemeral_owner].insert(path);
storage.acl_map.addUsage(node.acl_id);
}
@ -139,7 +146,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP
if (itr.key != "/")
{
auto parent_path = parentNodePath(itr.key);
storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; });
storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); value.increaseNumChildren(); });
}
}

View File

@ -1508,7 +1508,7 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s
using Node = DB::KeeperStorage::Node;
Node node{};
node.setData(data);
node.stat.ephemeralOwner = ephemeral_owner;
node.setEphemeralOwner(ephemeral_owner);
storage.container.insertOrReplace(path, node);
auto child_it = storage.container.find(path);
auto child_path = DB::getBaseNodeName(child_it->key);
@ -1517,7 +1517,7 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s
[&](auto & parent)
{
parent.addChild(child_path);
parent.stat.numChildren++;
parent.increaseNumChildren();
});
}
@ -1530,12 +1530,12 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple)
DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression);
DB::KeeperStorage storage(500, "", keeper_context);
addNode(storage, "/hello", "world", 1);
addNode(storage, "/hello/somepath", "somedata", 3);
addNode(storage, "/hello1", "world", 1);
addNode(storage, "/hello2", "somedata", 3);
storage.session_id_counter = 5;
storage.zxid = 2;
storage.ephemerals[3] = {"/hello"};
storage.ephemerals[1] = {"/hello/somepath"};
storage.ephemerals[3] = {"/hello2"};
storage.ephemerals[1] = {"/hello1"};
storage.getSessionID(130);
storage.getSessionID(130);
@ -1556,13 +1556,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple)
auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf);
EXPECT_EQ(restored_storage->container.size(), 6);
EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2);
EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1);
EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0);
EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 3);
EXPECT_EQ(restored_storage->container.getValue("/hello1").getChildren().size(), 0);
EXPECT_EQ(restored_storage->container.getValue("/hello2").getChildren().size(), 0);
EXPECT_EQ(restored_storage->container.getValue("/").getData(), "");
EXPECT_EQ(restored_storage->container.getValue("/hello").getData(), "world");
EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getData(), "somedata");
EXPECT_EQ(restored_storage->container.getValue("/hello1").getData(), "world");
EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata");
EXPECT_EQ(restored_storage->session_id_counter, 7);
EXPECT_EQ(restored_storage->zxid, 2);
EXPECT_EQ(restored_storage->ephemerals.size(), 2);
@ -2251,12 +2251,12 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions)
DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression);
DB::KeeperStorage storage(500, "", keeper_context);
addNode(storage, "/hello", "world", 1);
addNode(storage, "/hello/somepath", "somedata", 3);
addNode(storage, "/hello1", "world", 1);
addNode(storage, "/hello2", "somedata", 3);
storage.session_id_counter = 5;
storage.zxid = 2;
storage.ephemerals[3] = {"/hello"};
storage.ephemerals[1] = {"/hello/somepath"};
storage.ephemerals[3] = {"/hello2"};
storage.ephemerals[1] = {"/hello1"};
storage.getSessionID(130);
storage.getSessionID(130);
@ -2273,13 +2273,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions)
auto [restored_storage, snapshot_meta, _] = new_manager.deserializeSnapshotFromBuffer(debuf);
EXPECT_EQ(restored_storage->container.size(), 6);
EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2);
EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1);
EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0);
EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 3);
EXPECT_EQ(restored_storage->container.getValue("/hello1").getChildren().size(), 0);
EXPECT_EQ(restored_storage->container.getValue("/hello2").getChildren().size(), 0);
EXPECT_EQ(restored_storage->container.getValue("/").getData(), "");
EXPECT_EQ(restored_storage->container.getValue("/hello").getData(), "world");
EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getData(), "somedata");
EXPECT_EQ(restored_storage->container.getValue("/hello1").getData(), "world");
EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata");
EXPECT_EQ(restored_storage->session_id_counter, 7);
EXPECT_EQ(restored_storage->zxid, 2);
EXPECT_EQ(restored_storage->ephemerals.size(), 2);
@ -2948,7 +2948,7 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest)
create_path("/test_node");
auto node_it = storage.container.find("/test_node");
ASSERT_NE(node_it, storage.container.end());
auto node_version = node_it->value.stat.version;
auto node_version = node_it->value.version;
{
SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS");

View File

@ -118,6 +118,8 @@ namespace DB
M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \
M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \
M(UInt32, max_database_replicated_create_table_thread_pool_size, 1, "The number of threads to create tables during replica recovery in DatabaseReplicated. Zero means number of threads equal number of cores.", 0) \
M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \
M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp

View File

@ -14,6 +14,11 @@ namespace ErrorCodes
}
void ServerUUID::load(const fs::path & server_uuid_file, Poco::Logger * log)
{
server_uuid = loadServerUUID(server_uuid_file, log);
}
UUID loadServerUUID(const fs::path & server_uuid_file, Poco::Logger * log)
{
/// Write a uuid file containing a unique uuid if the file doesn't already exist during server start.
@ -25,8 +30,7 @@ void ServerUUID::load(const fs::path & server_uuid_file, Poco::Logger * log)
ReadBufferFromFile in(server_uuid_file);
readUUIDText(uuid, in);
assertEOF(in);
server_uuid = uuid;
return;
return uuid;
}
catch (...)
{
@ -44,7 +48,7 @@ void ServerUUID::load(const fs::path & server_uuid_file, Poco::Logger * log)
out.write(uuid_str.data(), uuid_str.size());
out.sync();
out.finalize();
server_uuid = new_uuid;
return new_uuid;
}
catch (...)
{

View File

@ -21,4 +21,6 @@ public:
static void load(const fs::path & server_uuid_file, Poco::Logger * log);
};
UUID loadServerUUID(const fs::path & server_uuid_file, Poco::Logger * log);
}

View File

@ -774,7 +774,7 @@ class IColumn;
M(Bool, load_marks_asynchronously, false, "Load MergeTree marks asynchronously", 0) \
M(Bool, enable_filesystem_read_prefetches_log, false, "Log to system.filesystem prefetch_log during query. Should be used only for testing or debugging, not recommended to be turned on by default", 0) \
M(Bool, allow_prefetched_read_pool_for_remote_filesystem, true, "Prefer prefetched threadpool if all parts are on remote filesystem", 0) \
M(Bool, allow_prefetched_read_pool_for_local_filesystem, false, "Prefer prefetched threadpool if all parts are on remote filesystem", 0) \
M(Bool, allow_prefetched_read_pool_for_local_filesystem, false, "Prefer prefetched threadpool if all parts are on local filesystem", 0) \
\
M(UInt64, prefetch_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the prefetch buffer to read from the filesystem.", 0) \
M(UInt64, filesystem_prefetch_step_bytes, 0, "Prefetch step in bytes. Zero means `auto` - approximately the best prefetch step will be auto deduced, but might not be 100% the best. The actual value might be different because of setting filesystem_prefetch_min_bytes_for_single_read_task", 0) \
@ -858,7 +858,7 @@ class IColumn;
M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \
M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \
M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \
M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \
M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \
M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \
M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \
M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \

View File

@ -93,7 +93,8 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"},
{"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"},
{"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"},
{"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}}},
{"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."},
{"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}}},
{"24.1", {{"print_pretty_type_names", false, true, "Better user experience."},
{"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"},
{"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"},

View File

@ -20,6 +20,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int TYPE_MISMATCH;
extern const int LOGICAL_ERROR;
}
DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type)
@ -55,62 +56,61 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type)
ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column)
{
if (!column)
return column;
ColumnPtr res = column;
if (const auto * column_array = typeid_cast<const ColumnArray *>(column.get()))
{
const auto & data = column_array->getDataPtr();
auto data_no_lc = recursiveRemoveLowCardinality(data);
if (data.get() == data_no_lc.get())
return column;
return ColumnArray::create(data_no_lc, column_array->getOffsetsPtr());
if (data.get() != data_no_lc.get())
res = ColumnArray::create(data_no_lc, column_array->getOffsetsPtr());
}
if (const auto * column_const = typeid_cast<const ColumnConst *>(column.get()))
else if (const auto * column_const = typeid_cast<const ColumnConst *>(column.get()))
{
const auto & nested = column_const->getDataColumnPtr();
auto nested_no_lc = recursiveRemoveLowCardinality(nested);
if (nested.get() == nested_no_lc.get())
return column;
return ColumnConst::create(nested_no_lc, column_const->size());
if (nested.get() != nested_no_lc.get())
res = ColumnConst::create(nested_no_lc, column_const->size());
}
if (const auto * column_tuple = typeid_cast<const ColumnTuple *>(column.get()))
else if (const auto * column_tuple = typeid_cast<const ColumnTuple *>(column.get()))
{
auto columns = column_tuple->getColumns();
for (auto & element : columns)
element = recursiveRemoveLowCardinality(element);
return ColumnTuple::create(columns);
res = ColumnTuple::create(columns);
}
if (const auto * column_map = typeid_cast<const ColumnMap *>(column.get()))
else if (const auto * column_map = typeid_cast<const ColumnMap *>(column.get()))
{
const auto & nested = column_map->getNestedColumnPtr();
auto nested_no_lc = recursiveRemoveLowCardinality(nested);
if (nested.get() == nested_no_lc.get())
return column;
return ColumnMap::create(nested_no_lc);
if (nested.get() != nested_no_lc.get())
res = ColumnMap::create(nested_no_lc);
}
/// Special case when column is a lazy argument of short circuit function.
/// We should call recursiveRemoveLowCardinality on the result column
/// when function will be executed.
if (const auto * column_function = typeid_cast<const ColumnFunction *>(column.get()))
else if (const auto * column_function = typeid_cast<const ColumnFunction *>(column.get()))
{
if (!column_function->isShortCircuitArgument())
return column;
return column_function->recursivelyConvertResultToFullColumnIfLowCardinality();
if (column_function->isShortCircuitArgument())
res = column_function->recursivelyConvertResultToFullColumnIfLowCardinality();
}
else if (const auto * column_low_cardinality = typeid_cast<const ColumnLowCardinality *>(column.get()))
{
res = column_low_cardinality->convertToFullColumn();
}
if (const auto * column_low_cardinality = typeid_cast<const ColumnLowCardinality *>(column.get()))
return column_low_cardinality->convertToFullColumn();
if (res != column)
{
/// recursiveRemoveLowCardinality() must not change the size of a passed column!
if (res->size() != column->size())
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"recursiveRemoveLowCardinality() somehow changed the size of column {}. Old size={}, new size={}. It's a bug",
column->getName(), column->size(), res->size());
}
}
return column;
return res;
}
ColumnPtr recursiveLowCardinalityTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type)

View File

@ -3,9 +3,7 @@
#include <memory>
#include <base/chrono_io.h>
#include <Core/Defines.h>
#include <Common/CurrentMetrics.h>
#include <Common/HashTable/Hash.h>
#include <Common/HashTable/HashSet.h>
#include <Common/ProfileEvents.h>
#include <Common/ProfilingScopedRWLock.h>

View File

@ -4,9 +4,9 @@
#include <Disks/DiskFactory.h>
#include <IO/FileEncryptionCommon.h>
#include <IO/ReadBufferFromEncryptedFile.h>
#include <IO/ReadBufferFromFileDecorator.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromEncryptedFile.h>
#include <IO/ReadBufferFromEmptyFile.h>
#include <boost/algorithm/hex.hpp>
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
@ -374,7 +374,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
{
/// File is empty, that's a normal case, see DiskEncrypted::truncateFile().
/// There is no header so we just return `ReadBufferFromString("")`.
return std::make_unique<ReadBufferFromEmptyFile>(wrapped_path);
return std::make_unique<ReadBufferFromFileDecorator>(std::make_unique<ReadBufferFromString>(std::string_view{}), wrapped_path);
}
auto encryption_settings = current_settings.get();
FileEncryption::Header header = readHeader(*buffer);

View File

@ -6,6 +6,7 @@
#include <Common/Exception.h>
#include <boost/algorithm/hex.hpp>
#include <IO/ReadBufferFromEncryptedFile.h>
#include <IO/ReadBufferFromFileDecorator.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromEncryptedFile.h>
#include <Common/quoteString.h>

View File

@ -1,6 +1,6 @@
#include "ReadBufferFromRemoteFSGather.h"
#include <IO/ReadBufferFromFileBase.h>
#include <IO/SeekableReadBuffer.h>
#include <Disks/IO/CachedOnDiskReadBufferFromFile.h>
#include <Disks/ObjectStorages/Cached/CachedObjectStorage.h>
@ -62,7 +62,7 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(
current_object = blobs_to_read.front();
}
std::unique_ptr<ReadBufferFromFileBase> ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object)
SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object)
{
if (current_buf && !with_cache)
{
@ -79,7 +79,7 @@ std::unique_ptr<ReadBufferFromFileBase> ReadBufferFromRemoteFSGather::createImpl
if (with_cache)
{
auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path);
return std::make_unique<CachedOnDiskReadBufferFromFile>(
return std::make_shared<CachedOnDiskReadBufferFromFile>(
object_path,
cache_key,
settings.remote_fs_cache,

View File

@ -53,7 +53,7 @@ public:
bool isContentCached(size_t offset, size_t size) override;
private:
std::unique_ptr<ReadBufferFromFileBase> createImplementationBuffer(const StoredObject & object);
SeekableReadBufferPtr createImplementationBuffer(const StoredObject & object);
bool nextImpl() override;
@ -80,7 +80,7 @@ private:
StoredObject current_object;
size_t current_buf_idx = 0;
std::unique_ptr<ReadBufferFromFileBase> current_buf;
SeekableReadBufferPtr current_buf;
LoggerPtr log;
};

View File

@ -39,7 +39,7 @@ std::unique_ptr<ReadBufferFromFileBase> createReadBufferFromFileBase(
size_t alignment)
{
if (file_size.has_value() && !*file_size)
return std::make_unique<ReadBufferFromEmptyFile>(filename);
return std::make_unique<ReadBufferFromEmptyFile>();
size_t estimated_size = 0;
if (read_hint.has_value())

View File

@ -389,6 +389,7 @@ void DiskObjectStorage::shutdown()
{
LOG_INFO(log, "Shutting down disk {}", name);
object_storage->shutdown();
metadata_storage->shutdown();
LOG_INFO(log, "Disk {} shut down", name);
}
@ -531,7 +532,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
const bool file_can_be_empty = !file_size.has_value() || *file_size == 0;
if (storage_objects.empty() && file_can_be_empty)
return std::make_unique<ReadBufferFromEmptyFile>(path);
return std::make_unique<ReadBufferFromEmptyFile>();
return object_storage->readObjects(
storage_objects,

View File

@ -210,6 +210,11 @@ public:
throwNotImplemented();
}
virtual void shutdown()
{
/// This method is overridden for specific metadata implementations in ClickHouse Cloud.
}
virtual ~IMetadataStorage() = default;
/// ==== More specific methods. Previous were almost general purpose. ====

View File

@ -21,11 +21,14 @@ namespace
enum class Kind
{
Current,
Cumulative,
All
};
/** If ClickHouse is build with coverage instrumentation, returns an array
* of currently accumulated (`coverage`) / all possible (`coverageAll`) unique code addresses.
* of currently accumulated (`coverageCurrent`)
* or accumulated since the startup (`coverageCumulative`)
* or all possible (`coverageAll`) unique code addresses.
*/
class FunctionCoverage : public IFunction
{
@ -64,7 +67,11 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
{
auto coverage_table = kind == Kind::Current ? getCoverage() : getAllInstrumentedAddresses();
auto coverage_table = kind == Kind::Current
? getCurrentCoverage()
: (kind == Kind::Cumulative
? getCumulativeCoverage()
: getAllInstrumentedAddresses());
auto column_addresses = ColumnUInt64::create();
auto & data = column_addresses->getData();
@ -85,8 +92,68 @@ public:
REGISTER_FUNCTION(Coverage)
{
factory.registerFunction("coverage", [](ContextPtr){ return std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionCoverage>(Kind::Current)); });
factory.registerFunction("coverageAll", [](ContextPtr){ return std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionCoverage>(Kind::All)); });
factory.registerFunction("coverageCurrent", [](ContextPtr){ return std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionCoverage>(Kind::Current)); },
FunctionDocumentation
{
.description=R"(
This function is only available if ClickHouse was built with the SANITIZE_COVERAGE=1 option.
It returns an array of unique addresses (a subset of the instrumented points in code) in the code
encountered at runtime after the previous coverage reset (with the `SYSTEM RESET COVERAGE` query) or after server startup.
[example:functions]
The order of array elements is undetermined.
You can use another function, `coverageAll` to find all instrumented addresses in the code to compare and calculate the percentage.
You can process the addresses with the `addressToSymbol` (possibly with `demangle`) and `addressToLine` functions
to calculate symbol-level, file-level, or line-level coverage.
If you run multiple tests sequentially and reset the coverage with the `SYSTEM RESET COVERAGE` query between the tests,
you can obtain a coverage information for every test in isolation, to find which functions are covered by which tests and vise-versa.
By default, every *basic block* in the code is covered, which roughly means - a sequence of instructions without jumps,
e.g. a body of for loop without ifs, or a single branch of if.
See https://clang.llvm.org/docs/SanitizerCoverage.html for more information.
)",
.examples{
{"functions", "SELECT DISTINCT demangle(addressToSymbol(arrayJoin(coverageCurrent())))", ""}},
.categories{"Introspection"}
});
factory.registerFunction("coverageCumulative", [](ContextPtr){ return std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionCoverage>(Kind::Cumulative)); },
FunctionDocumentation
{
.description=R"(
This function is only available if ClickHouse was built with the SANITIZE_COVERAGE=1 option.
It returns an array of unique addresses (a subset of the instrumented points in code) in the code
encountered at runtime after server startup.
In contrast to `coverageCurrent` it cannot be reset with the `SYSTEM RESET COVERAGE`.
See the `coverageCurrent` function for the details.
)",
.categories{"Introspection"}
});
factory.registerFunction("coverageAll", [](ContextPtr){ return std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionCoverage>(Kind::All)); },
FunctionDocumentation
{
.description=R"(
This function is only available if ClickHouse was built with the SANITIZE_COVERAGE=1 option.
It returns an array of all unique addresses in the code instrumented for coverage
- all possible addresses that can appear in the result of the `coverage` function.
You can use this function, and the `coverage` function to compare and calculate the coverage percentage.
See the `coverageCurrent` function for the details.
)",
.categories{"Introspection"}
});
}
}

View File

@ -43,11 +43,11 @@ class ExtractKeyValuePairs : public IFunction
builder.withQuotingCharacter(parsed_arguments.quoting_character.value());
}
bool is_number_of_pairs_unlimited = context->getSettingsRef().extract_kvp_max_pairs_per_row == 0;
bool is_number_of_pairs_unlimited = context->getSettingsRef().extract_key_value_pairs_max_pairs_per_row == 0;
if (!is_number_of_pairs_unlimited)
{
builder.withMaxNumberOfPairs(context->getSettingsRef().extract_kvp_max_pairs_per_row);
builder.withMaxNumberOfPairs(context->getSettingsRef().extract_key_value_pairs_max_pairs_per_row);
}
return builder.build();

View File

@ -228,12 +228,7 @@ public:
off_t getPosition() override
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition is not supported when reading from archive");
}
size_t getFileOffsetOfBufferEnd() const override
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getFileOffsetOfBufferEnd is not supported when reading from archive");
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive");
}
String getFileName() const override { return handle.getFileName(); }

View File

@ -15,7 +15,6 @@ namespace ErrorCodes
extern const int CANNOT_UNPACK_ARCHIVE;
extern const int LOGICAL_ERROR;
extern const int SEEK_POSITION_OUT_OF_BOUND;
extern const int UNSUPPORTED_METHOD;
extern const int CANNOT_SEEK_THROUGH_FILE;
}
@ -253,11 +252,6 @@ public:
checkResult(err);
}
size_t getFileOffsetOfBufferEnd() const override
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getFileOffsetOfBufferEnd is not supported when reading from zip archive");
}
off_t seek(off_t off, int whence) override
{
off_t current_pos = getPosition();

View File

@ -4,7 +4,8 @@
namespace DB
{
BoundedReadBuffer::BoundedReadBuffer(std::unique_ptr<ReadBufferFromFileBase> impl_) : impl(std::move(impl_))
BoundedReadBuffer::BoundedReadBuffer(std::unique_ptr<SeekableReadBuffer> impl_)
: ReadBufferFromFileDecorator(std::move(impl_))
{
}

View File

@ -1,5 +1,5 @@
#pragma once
#include <IO/ReadBufferFromFileBase.h>
#include <IO/ReadBufferFromFileDecorator.h>
namespace DB
@ -7,10 +7,10 @@ namespace DB
/// A buffer which allows to make an underlying buffer as right bounded,
/// e.g. the buffer cannot return data beyond offset specified in `setReadUntilPosition`.
class BoundedReadBuffer : public ReadBufferFromFileBase
class BoundedReadBuffer : public ReadBufferFromFileDecorator
{
public:
explicit BoundedReadBuffer(std::unique_ptr<ReadBufferFromFileBase> impl_);
explicit BoundedReadBuffer(std::unique_ptr<SeekableReadBuffer> impl_);
bool supportsRightBoundedReads() const override { return true; }
@ -23,8 +23,6 @@ public:
off_t seek(off_t off, int whence) override;
size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; }
String getFileName() const override { return impl->getFileName(); }
size_t getFileSize() override { return impl->getFileSize(); }
/// file_offset_of_buffer_end can differ from impl's file_offset_of_buffer_end
/// because of resizing of the tail. => Need to also override getPosition() as
@ -32,8 +30,6 @@ public:
off_t getPosition() override;
private:
std::unique_ptr<ReadBufferFromFileBase> impl;
std::optional<size_t> read_until_position;
/// atomic because can be used in log or exception messages while being updated.
std::atomic<size_t> file_offset_of_buffer_end = 0;

View File

@ -18,6 +18,7 @@ public:
/// Returns adjusted position, i.e. returns `3` if the position in the nested buffer is `start_offset + 3`.
off_t getPosition() override;
off_t seek(off_t off, int whence) override;
private:

View File

@ -92,11 +92,6 @@ size_t MMapReadBufferFromFileDescriptor::getFileSize()
return getSizeFromFileDescriptor(getFD(), getFileName());
}
size_t MMapReadBufferFromFileDescriptor::getFileOffsetOfBufferEnd() const
{
return mapped.getOffset() + mapped.getLength();
}
size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &)
{
if (offset >= mapped.getLength())

View File

@ -36,8 +36,6 @@ public:
std::string getFileName() const override;
size_t getFileOffsetOfBufferEnd() const override;
int getFD() const;
size_t getFileSize() override;

View File

@ -76,9 +76,4 @@ off_t MMapReadBufferFromFileWithCache::seek(off_t offset, int whence)
return new_pos;
}
size_t MMapReadBufferFromFileWithCache::getFileOffsetOfBufferEnd() const
{
return mapped->getOffset() + mapped->getLength();
}
}

View File

@ -19,7 +19,7 @@ public:
off_t getPosition() override;
std::string getFileName() const override;
off_t seek(off_t offset, int whence) override;
size_t getFileOffsetOfBufferEnd() const override;
bool isRegularLocalFile(size_t * /* out_view_offset */) override { return true; }
private:

View File

@ -21,7 +21,7 @@ public:
OpenedFile(const std::string & file_name_, int flags_);
~OpenedFile();
/// Close prematurally.
/// Close prematurely.
void close();
int getFD() const;
@ -40,4 +40,3 @@ private:
};
}

View File

@ -14,18 +14,12 @@ namespace DB
/// - ThreadPoolReader
class ReadBufferFromEmptyFile : public ReadBufferFromFileBase
{
public:
explicit ReadBufferFromEmptyFile(const String & file_name_) : file_name(file_name_) {}
private:
String file_name;
bool nextImpl() override { return false; }
std::string getFileName() const override { return file_name; }
std::string getFileName() const override { return "<empty>"; }
off_t seek(off_t /*off*/, int /*whence*/) override { return 0; }
off_t getPosition() override { return 0; }
size_t getFileSize() override { return 0; }
size_t getFileOffsetOfBufferEnd() const override { return 0; }
};
}

View File

@ -101,18 +101,6 @@ bool ReadBufferFromEncryptedFile::nextImpl()
return true;
}
size_t ReadBufferFromEncryptedFile::getFileSize()
{
size_t size = in->getFileSize();
return size > FileEncryption::Header::kSize ? size - FileEncryption::Header::kSize : size;
}
size_t ReadBufferFromEncryptedFile::getFileOffsetOfBufferEnd() const
{
size_t file_offset = in->getFileOffsetOfBufferEnd();
return file_offset > FileEncryption::Header::kSize ? file_offset - FileEncryption::Header::kSize : file_offset;
}
}
#endif

View File

@ -27,10 +27,10 @@ public:
std::string getFileName() const override { return in->getFileName(); }
void setReadUntilPosition(size_t position) override { in->setReadUntilPosition(position + FileEncryption::Header::kSize); }
void setReadUntilEnd() override { in->setReadUntilEnd(); }
size_t getFileSize() override;
size_t getFileOffsetOfBufferEnd() const override;
size_t getFileSize() override { return in->getFileSize(); }
private:
bool nextImpl() override;

View File

@ -60,12 +60,6 @@ public:
/// file offset and what getPosition() returns.
virtual bool isRegularLocalFile(size_t * /* out_view_offset */ = nullptr) { return false; }
/// NOTE: This method should be thread-safe against seek(), since it can be
/// used in CachedOnDiskReadBufferFromFile from multiple threads (because
/// it first releases the buffer, and then do logging, and so other thread
/// can already call seek() which will lead to data-race).
virtual size_t getFileOffsetOfBufferEnd() const = 0;
protected:
std::optional<size_t> file_size;
ProfileCallback profile_callback;

View File

@ -0,0 +1,60 @@
#include <IO/ReadBufferFromFileDecorator.h>
namespace DB
{
ReadBufferFromFileDecorator::ReadBufferFromFileDecorator(std::unique_ptr<SeekableReadBuffer> impl_)
: ReadBufferFromFileDecorator(std::move(impl_), "")
{
}
ReadBufferFromFileDecorator::ReadBufferFromFileDecorator(std::unique_ptr<SeekableReadBuffer> impl_, const String & file_name_)
: impl(std::move(impl_)), file_name(file_name_)
{
swap(*impl);
}
std::string ReadBufferFromFileDecorator::getFileName() const
{
if (!file_name.empty())
return file_name;
return getFileNameFromReadBuffer(*impl);
}
off_t ReadBufferFromFileDecorator::getPosition()
{
swap(*impl);
auto position = impl->getPosition();
swap(*impl);
return position;
}
off_t ReadBufferFromFileDecorator::seek(off_t off, int whence)
{
swap(*impl);
auto result = impl->seek(off, whence);
swap(*impl);
return result;
}
bool ReadBufferFromFileDecorator::nextImpl()
{
swap(*impl);
auto result = impl->next();
swap(*impl);
return result;
}
size_t ReadBufferFromFileDecorator::getFileSize()
{
return getFileSizeFromReadBuffer(*impl);
}
}

View File

@ -0,0 +1,37 @@
#pragma once
#include <IO/ReadBufferFromFileBase.h>
namespace DB
{
/// Delegates all reads to underlying buffer. Doesn't have own memory.
class ReadBufferFromFileDecorator : public ReadBufferFromFileBase
{
public:
explicit ReadBufferFromFileDecorator(std::unique_ptr<SeekableReadBuffer> impl_);
ReadBufferFromFileDecorator(std::unique_ptr<SeekableReadBuffer> impl_, const String & file_name_);
std::string getFileName() const override;
off_t getPosition() override;
off_t seek(off_t off, int whence) override;
bool nextImpl() override;
bool isWithFileSize() const { return dynamic_cast<const WithFileSize *>(impl.get()) != nullptr; }
const ReadBuffer & getWrappedReadBuffer() const { return *impl; }
ReadBuffer & getWrappedReadBuffer() { return *impl; }
size_t getFileSize() override;
protected:
std::unique_ptr<SeekableReadBuffer> impl;
String file_name;
};
}

View File

@ -20,6 +20,7 @@ public:
: SeekableReadBuffer(const_cast<char *>(str.data()), str.size(), 0) {}
off_t seek(off_t off, int whence) override;
off_t getPosition() override;
};

View File

@ -44,6 +44,12 @@ public:
virtual String getInfoForLog() { return ""; }
/// NOTE: This method should be thread-safe against seek(), since it can be
/// used in CachedOnDiskReadBufferFromFile from multiple threads (because
/// it first releases the buffer, and then do logging, and so other thread
/// can already call seek() which will lead to data-race).
virtual size_t getFileOffsetOfBufferEnd() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFileOffsetOfBufferEnd() not implemented"); }
/// If true, setReadUntilPosition() guarantees that eof will be reported at the given position.
virtual bool supportsRightBoundedReads() const { return false; }

View File

@ -2,6 +2,7 @@
#include <IO/ReadBufferFromFile.h>
#include <IO/CompressedReadBufferWrapper.h>
#include <IO/ParallelReadBuffer.h>
#include <IO/ReadBufferFromFileDecorator.h>
#include <IO/PeekableReadBuffer.h>
namespace DB
@ -16,15 +17,23 @@ template <typename T>
static size_t getFileSize(T & in)
{
if (auto * with_file_size = dynamic_cast<WithFileSize *>(&in))
{
return with_file_size->getFileSize();
}
throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size");
}
size_t getFileSizeFromReadBuffer(ReadBuffer & in)
{
if (auto * compressed = dynamic_cast<CompressedReadBufferWrapper *>(&in))
if (auto * delegate = dynamic_cast<ReadBufferFromFileDecorator *>(&in))
{
return getFileSize(delegate->getWrappedReadBuffer());
}
else if (auto * compressed = dynamic_cast<CompressedReadBufferWrapper *>(&in))
{
return getFileSize(compressed->getWrappedReadBuffer());
}
return getFileSize(in);
}
@ -43,7 +52,11 @@ std::optional<size_t> tryGetFileSizeFromReadBuffer(ReadBuffer & in)
bool isBufferWithFileSize(const ReadBuffer & in)
{
if (const auto * compressed = dynamic_cast<const CompressedReadBufferWrapper *>(&in))
if (const auto * delegate = dynamic_cast<const ReadBufferFromFileDecorator *>(&in))
{
return delegate->isWithFileSize();
}
else if (const auto * compressed = dynamic_cast<const CompressedReadBufferWrapper *>(&in))
{
return isBufferWithFileSize(compressed->getWrappedReadBuffer());
}
@ -53,7 +66,11 @@ bool isBufferWithFileSize(const ReadBuffer & in)
size_t getDataOffsetMaybeCompressed(const ReadBuffer & in)
{
if (const auto * compressed = dynamic_cast<const CompressedReadBufferWrapper *>(&in))
if (const auto * delegate = dynamic_cast<const ReadBufferFromFileDecorator *>(&in))
{
return getDataOffsetMaybeCompressed(delegate->getWrappedReadBuffer());
}
else if (const auto * compressed = dynamic_cast<const CompressedReadBufferWrapper *>(&in))
{
return getDataOffsetMaybeCompressed(compressed->getWrappedReadBuffer());
}

View File

@ -1645,7 +1645,7 @@ void ActionsDAG::mergeNodes(ActionsDAG && second)
}
}
ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split_nodes) const
ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split_nodes, bool create_split_nodes_mapping) const
{
/// Split DAG into two parts.
/// (first_nodes, first_outputs) is a part which will have split_list in result.
@ -1777,15 +1777,6 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
child = child_data.to_second;
}
/// Input from second DAG should also be in the first.
if (copy.type == ActionType::INPUT)
{
auto & input_copy = first_nodes.emplace_back(*cur.node);
assert(cur_data.to_first == nullptr);
cur_data.to_first = &input_copy;
new_inputs.push_back(cur.node);
}
}
else
{
@ -1804,11 +1795,12 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
/// If this node is needed in result, add it as input.
Node input_node;
input_node.type = ActionType::INPUT;
input_node.result_type = node.result_type;
input_node.result_name = node.result_name;
input_node.result_type = cur.node->result_type;
input_node.result_name = cur.node->result_name;
cur_data.to_second = &second_nodes.emplace_back(std::move(input_node));
new_inputs.push_back(cur.node);
if (cur.node->type != ActionType::INPUT)
new_inputs.push_back(cur.node);
}
}
}
@ -1824,7 +1816,13 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
for (const auto * input_node : inputs)
{
const auto & cur = data[input_node];
first_inputs.push_back(cur.to_first);
if (cur.to_first)
{
first_inputs.push_back(cur.to_first);
if (cur.to_second)
first_outputs.push_back(cur.to_first);
}
}
for (const auto * input : new_inputs)
@ -1834,6 +1832,13 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
first_outputs.push_back(cur.to_first);
}
for (const auto * input_node : inputs)
{
const auto & cur = data[input_node];
if (cur.to_second)
second_inputs.push_back(cur.to_second);
}
auto first_actions = std::make_shared<ActionsDAG>();
first_actions->nodes.swap(first_nodes);
first_actions->outputs.swap(first_outputs);
@ -1844,7 +1849,14 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
second_actions->outputs.swap(second_outputs);
second_actions->inputs.swap(second_inputs);
return {std::move(first_actions), std::move(second_actions)};
std::unordered_map<const Node *, const Node *> split_nodes_mapping;
if (create_split_nodes_mapping)
{
for (const auto * node : split_nodes)
split_nodes_mapping[node] = data[node].to_first;
}
return {std::move(first_actions), std::move(second_actions), std::move(split_nodes_mapping)};
}
ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const

View File

@ -327,13 +327,18 @@ public:
/// Merge current nodes with specified dag nodes
void mergeNodes(ActionsDAG && second);
using SplitResult = std::pair<ActionsDAGPtr, ActionsDAGPtr>;
struct SplitResult
{
ActionsDAGPtr first;
ActionsDAGPtr second;
std::unordered_map<const Node *, const Node *> split_nodes_mapping;
};
/// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children.
/// Execution of first then second parts on block is equivalent to execution of initial DAG.
/// First DAG and initial DAG have equal inputs, second DAG and initial DAG has equal outputs.
/// Second DAG inputs may contain less inputs then first DAG (but also include other columns).
SplitResult split(std::unordered_set<const Node *> split_nodes) const;
SplitResult split(std::unordered_set<const Node *> split_nodes, bool create_split_nodes_mapping = false) const;
/// Splits actions into two parts. Returned first half may be swapped with ARRAY JOIN.
SplitResult splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const;

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