diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index f25611e8666..284d94b5c5a 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -895,6 +895,48 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinS390X: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_s390x + EOF + - name: Download changed images + uses: actions/download-artifact@v3 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + submodules: true + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v3 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" ############################################################################################ ##################################### Docker images ####################################### ############################################################################################ @@ -978,6 +1020,7 @@ jobs: - BuilderBinFreeBSD - BuilderBinPPC64 - BuilderBinRISCV64 + - BuilderBinS390X - BuilderBinAmd64Compat - BuilderBinAarch64V80Compat - BuilderBinClangTidy diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 25927954809..cf9f025585d 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -955,6 +955,47 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinS390X: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_s390x + EOF + - name: Download changed images + uses: actions/download-artifact@v3 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + submodules: true + - name: Build + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v3 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" ############################################################################################ ##################################### Docker images ####################################### ############################################################################################ @@ -1037,6 +1078,7 @@ jobs: - BuilderBinFreeBSD - BuilderBinPPC64 - BuilderBinRISCV64 + - BuilderBinS390X - BuilderBinAmd64Compat - BuilderBinAarch64V80Compat - BuilderBinClangTidy @@ -5185,3 +5227,39 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" +############################################################################################## +##################################### SQL TEST ############################################### +############################################################################################## + SQLTest: + needs: [BuilderDebRelease] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/sqltest + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=SQLTest + REPO_COPY=${{runner.temp}}/sqltest/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: SQLTest + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 sqltest.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" diff --git a/base/base/EnumReflection.h b/base/base/EnumReflection.h index 0d1f8ae0a40..e3208f16a75 100644 --- a/base/base/EnumReflection.h +++ b/base/base/EnumReflection.h @@ -3,6 +3,7 @@ #include #include + template concept is_enum = std::is_enum_v; namespace detail diff --git a/base/poco/Data/include/Poco/Data/TypeHandler.h b/base/poco/Data/include/Poco/Data/TypeHandler.h index 34f88e986f7..e7633de7018 100644 --- a/base/poco/Data/include/Poco/Data/TypeHandler.h +++ b/base/poco/Data/include/Poco/Data/TypeHandler.h @@ -97,7 +97,7 @@ namespace Data /// /// static void extract(std::size_t pos, Person& obj, const Person& defVal, AbstractExtractor::Ptr pExt) /// { - /// // defVal is the default person we should use if we encunter NULL entries, so we take the individual fields + /// // defVal is the default person we should use if we encounter NULL entries, so we take the individual fields /// // as defaults. You can do more complex checking, ie return defVal if only one single entry of the fields is null etc... /// poco_assert_dbg (!pExt.isNull()); /// std::string lastName; diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h index 65917ac9dd4..c19eecf5c73 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h @@ -146,7 +146,7 @@ namespace Net std::string cipherList; /// Specifies the supported ciphers in OpenSSL notation. - /// Defaults to "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH". + /// Defaults to "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH". std::string dhParamsFile; /// Specifies a file containing Diffie-Hellman parameters. @@ -172,7 +172,7 @@ namespace Net VerificationMode verificationMode = VERIFY_RELAXED, int verificationDepth = 9, bool loadDefaultCAs = false, - const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); + const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH"); /// Creates a Context. /// /// * usage specifies whether the context is used by a client or server. @@ -200,7 +200,7 @@ namespace Net VerificationMode verificationMode = VERIFY_RELAXED, int verificationDepth = 9, bool loadDefaultCAs = false, - const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); + const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH"); /// Creates a Context. /// /// * usage specifies whether the context is used by a client or server. diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h index 21a1ed685e5..e4037c87927 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h @@ -76,7 +76,7 @@ namespace Net /// none|relaxed|strict|once /// 1..9 /// true|false - /// ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH + /// ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH /// true|false /// /// KeyFileHandler diff --git a/base/poco/NetSSL_OpenSSL/src/Context.cpp b/base/poco/NetSSL_OpenSSL/src/Context.cpp index ca220c40a33..d0bab902b89 100644 --- a/base/poco/NetSSL_OpenSSL/src/Context.cpp +++ b/base/poco/NetSSL_OpenSSL/src/Context.cpp @@ -41,7 +41,7 @@ Context::Params::Params(): verificationMode(VERIFY_RELAXED), verificationDepth(9), loadDefaultCAs(false), - cipherList("ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH") + cipherList("ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH") { } diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index b85d4253b89..945eb9affa4 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,6 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/target.cmake b/cmake/target.cmake index ffab08f1103..204a67d4357 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -47,7 +47,7 @@ if (CMAKE_CROSSCOMPILING) set (ENABLE_RUST OFF CACHE INTERNAL "") elseif (ARCH_S390X) set (ENABLE_GRPC OFF CACHE INTERNAL "") - set (ENABLE_SENTRY OFF CACHE INTERNAL "") + set (ENABLE_RUST OFF CACHE INTERNAL "") endif () elseif (OS_FREEBSD) # FIXME: broken dependencies diff --git a/docker/README.md b/docker/README.md index bae02201add..c1bb3b49f00 100644 --- a/docker/README.md +++ b/docker/README.md @@ -2,4 +2,4 @@ This directory contain Dockerfiles for `clickhouse-server`. They are updated in each release. -Also there is bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. +Also, there is a bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. diff --git a/docker/images.json b/docker/images.json index 9e4d19d30aa..d895e2da2f0 100644 --- a/docker/images.json +++ b/docker/images.json @@ -125,6 +125,7 @@ "docker/test/keeper-jepsen", "docker/test/server-jepsen", "docker/test/sqllogic", + "docker/test/sqltest", "docker/test/stateless" ] }, @@ -155,13 +156,16 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": [ - ] + "dependent": [] }, "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] }, + "docker/test/sqltest": { + "name": "clickhouse/sqltest", + "dependent": [] + }, "docker/test/integration/nginx_dav": { "name": "clickhouse/nginx-dav", "dependent": [] diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 897bcd24d04..940daad9c61 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -80,6 +80,14 @@ RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ # Download toolchain and SDK for Darwin RUN curl -sL -O https://github.com/phracker/MacOSX-SDKs/releases/download/11.3/MacOSX11.0.sdk.tar.xz +# Download and install mold 2.0 for s390x build +RUN curl -Lo /tmp/mold.tar.gz "https://github.com/rui314/mold/releases/download/v2.0.0/mold-2.0.0-x86_64-linux.tar.gz" \ + && mkdir /tmp/mold \ + && tar -xzf /tmp/mold.tar.gz -C /tmp/mold \ + && cp -r /tmp/mold/mold*/* /usr \ + && rm -rf /tmp/mold \ + && rm /tmp/mold.tar.gz + # Architecture of the image when BuildKit/buildx is used ARG TARGETARCH ARG NFPM_VERSION=2.20.0 diff --git a/docker/packager/packager b/docker/packager/packager index 2d50d7efd4b..1d0ccae8a69 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -143,6 +143,7 @@ def parse_env_variables( FREEBSD_SUFFIX = "-freebsd" PPC_SUFFIX = "-ppc64le" RISCV_SUFFIX = "-riscv64" + S390X_SUFFIX = "-s390x" AMD64_COMPAT_SUFFIX = "-amd64-compat" result = [] @@ -156,6 +157,7 @@ def parse_env_variables( is_cross_arm_v80compat = compiler.endswith(ARM_V80COMPAT_SUFFIX) is_cross_ppc = compiler.endswith(PPC_SUFFIX) is_cross_riscv = compiler.endswith(RISCV_SUFFIX) + is_cross_s390x = compiler.endswith(S390X_SUFFIX) is_cross_freebsd = compiler.endswith(FREEBSD_SUFFIX) is_amd64_compat = compiler.endswith(AMD64_COMPAT_SUFFIX) @@ -217,6 +219,11 @@ def parse_env_variables( cmake_flags.append( "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-riscv64.cmake" ) + elif is_cross_s390x: + cc = compiler[: -len(S390X_SUFFIX)] + cmake_flags.append( + "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-s390x.cmake" + ) elif is_amd64_compat: cc = compiler[: -len(AMD64_COMPAT_SUFFIX)] result.append("DEB_ARCH=amd64") @@ -380,6 +387,7 @@ def parse_args() -> argparse.Namespace: "clang-16-aarch64-v80compat", "clang-16-ppc64le", "clang-16-riscv64", + "clang-16-s390x", "clang-16-amd64-compat", "clang-16-freebsd", ), diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index 47edbbb14a5..b55baa0e0fc 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -35,4 +35,7 @@ ENV LC_ALL en_US.UTF-8 ENV TZ=Europe/Amsterdam RUN ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone +# This script is used to setup realtime export of server logs from the CI into external ClickHouse cluster: +COPY setup_export_logs.sh / + CMD sleep 1 diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh new file mode 100755 index 00000000000..12fae855b03 --- /dev/null +++ b/docker/test/base/setup_export_logs.sh @@ -0,0 +1,61 @@ +#!/bin/bash + +# This script sets up export of system log tables to a remote server. +# Remote tables are created if not exist, and augmented with extra columns, +# and their names will contain a hash of the table structure, +# which allows exporting tables from servers of different versions. + +# Pre-configured destination cluster, where to export the data +CLUSTER=${CLUSTER:=system_logs_export} + +EXTRA_COLUMNS=${EXTRA_COLUMNS:="pull_request_number UInt32, commit_sha String, check_start_time DateTime, check_name LowCardinality(String), instance_type LowCardinality(String), "} +EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:="0 AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type"} +EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:="check_name, "} + +CONNECTION_PARAMETERS=${CONNECTION_PARAMETERS:=""} + +# Create all configured system logs: +clickhouse-client --query "SYSTEM FLUSH LOGS" + +# For each system log table: +clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table +do + # Calculate hash of its structure: + hash=$(clickhouse-client --query " + SELECT sipHash64(groupArray((name, type))) + FROM (SELECT name, type FROM system.columns + WHERE database = 'system' AND table = '$table' + ORDER BY position) + ") + + # 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"'/; + s/ORDER BY \(/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"'/; + s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; + /^TTL /d + ') + + echo "Creating destination table ${table}_${hash}" >&2 + + echo "$statement" | clickhouse-client $CONNECTION_PARAMETERS + + echo "Creating table system.${table}_sender" >&2 + + # Create Distributed table and materialized view to watch on the original table: + clickhouse-client --query " + CREATE TABLE system.${table}_sender + ENGINE = Distributed(${CLUSTER}, default, ${table}_${hash}) + EMPTY AS + SELECT ${EXTRA_COLUMNS_EXPRESSION}, * + FROM system.${table} + " + + echo "Creating materialized view system.${table}_watcher" >&2 + + clickhouse-client --query " + CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS + SELECT ${EXTRA_COLUMNS_EXPRESSION}, * + FROM system.${table} + " +done diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 60e6199aaa4..477bbdb6919 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -148,6 +148,7 @@ function clone_submodules contrib/liburing contrib/libfiu contrib/incbin + contrib/yaml-cpp ) git submodule sync @@ -170,6 +171,7 @@ function run_cmake "-DENABLE_SIMDJSON=1" "-DENABLE_JEMALLOC=1" "-DENABLE_LIBURING=1" + "-DENABLE_YAML_CPP=1" ) export CCACHE_DIR="$FASTTEST_WORKSPACE/ccache" diff --git a/docker/test/integration/runner/compose/docker_compose_coredns.yml b/docker/test/integration/runner/compose/docker_compose_coredns.yml index b329d4e0a46..e4736e04846 100644 --- a/docker/test/integration/runner/compose/docker_compose_coredns.yml +++ b/docker/test/integration/runner/compose/docker_compose_coredns.yml @@ -2,7 +2,7 @@ version: "2.3" services: coredns: - image: coredns/coredns:latest + image: coredns/coredns:1.9.3 # :latest broke this test restart: always volumes: - ${COREDNS_CONFIG_DIR}/example.com:/example.com diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 4ef42ed377d..db828741b0d 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -1,4 +1,5 @@ #!/bin/bash + set -exu trap "exit" INT TERM diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile new file mode 100644 index 00000000000..437677f4fd1 --- /dev/null +++ b/docker/test/sqltest/Dockerfile @@ -0,0 +1,30 @@ +# docker build -t clickhouse/sqltest . +ARG FROM_TAG=latest +FROM clickhouse/test-base:$FROM_TAG + +RUN apt-get update --yes \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get install --yes --no-install-recommends \ + wget \ + git \ + python3 \ + python3-dev \ + python3-pip \ + sudo \ + && apt-get clean + +RUN pip3 install \ + pyyaml \ + clickhouse-driver + +ARG sqltest_repo="https://github.com/elliotchance/sqltest/" + +RUN git clone ${sqltest_repo} + +ENV TZ=UTC +ENV MAX_RUN_TIME=900 +RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +COPY run.sh / +COPY test.py / +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh new file mode 100755 index 00000000000..cba1c1dab1f --- /dev/null +++ b/docker/test/sqltest/run.sh @@ -0,0 +1,51 @@ +#!/bin/bash +# shellcheck disable=SC2015 + +set -x +set -e +set -u +set -o pipefail + +BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-16_debug_none_unsplitted_disable_False_binary"} +BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"} + +function wget_with_retry +{ + for _ in 1 2 3 4; do + if wget -nv -nd -c "$1";then + return 0 + else + sleep 0.5 + fi + done + return 1 +} + +wget_with_retry "$BINARY_URL_TO_DOWNLOAD" +chmod +x clickhouse +./clickhouse install --noninteractive + +echo " +users: + default: + access_management: 1" > /etc/clickhouse-server/users.d/access_management.yaml + +clickhouse start + +# Wait for start +for _ in {1..100} +do + clickhouse-client --query "SELECT 1" && break ||: + sleep 1 +done + +# Run the test +pushd sqltest/standards/2016/ +/test.py +mv report.html test.log /workspace +popd + +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.err.log + +mv /var/log/clickhouse-server/clickhouse-server.log.zst /var/log/clickhouse-server/clickhouse-server.err.log.zst /workspace diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py new file mode 100755 index 00000000000..5807ca79b02 --- /dev/null +++ b/docker/test/sqltest/test.py @@ -0,0 +1,148 @@ +#!/usr/bin/env python3 + +import os +import yaml +import html +import random +import string +from clickhouse_driver import Client + + +client = Client(host="localhost", port=9000) +settings = { + "default_table_engine": "Memory", + "union_default_mode": "DISTINCT", + "calculate_text_stack_trace": 0, +} + +database_name = "sqltest_" + "".join( + random.choice(string.ascii_lowercase) for _ in range(10) +) + +client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings=settings) +client.execute(f"CREATE DATABASE {database_name}", settings=settings) + +client = Client(host="localhost", port=9000, database=database_name) + +summary = {"success": 0, "total": 0, "results": {}} + +log_file = open("test.log", "w") +report_html_file = open("report.html", "w") + +with open("features.yml", "r") as file: + yaml_content = yaml.safe_load(file) + + for category in yaml_content: + log_file.write(category.capitalize() + " features:\n") + summary["results"][category] = {"success": 0, "total": 0, "results": {}} + + for test in yaml_content[category]: + log_file.write(test + ": " + yaml_content[category][test] + "\n") + summary["results"][category]["results"][test] = { + "success": 0, + "total": 0, + "description": yaml_content[category][test], + } + + test_path = test[0] + "/" + test + ".tests.yml" + if os.path.exists(test_path): + with open(test_path, "r") as test_file: + test_yaml_content = yaml.load_all(test_file, Loader=yaml.FullLoader) + + for test_case in test_yaml_content: + queries = test_case["sql"] + if not isinstance(queries, list): + queries = [queries] + + for query in queries: + # Example: E011-01 + test_group = "" + if "-" in test: + test_group = test.split("-", 1)[0] + summary["results"][category]["results"][test_group][ + "total" + ] += 1 + summary["results"][category]["results"][test]["total"] += 1 + summary["results"][category]["total"] += 1 + summary["total"] += 1 + + log_file.write(query + "\n") + + try: + result = client.execute(query, settings=settings) + log_file.write(str(result) + "\n") + + if test_group: + summary["results"][category]["results"][test_group][ + "success" + ] += 1 + summary["results"][category]["results"][test][ + "success" + ] += 1 + summary["results"][category]["success"] += 1 + summary["success"] += 1 + + except Exception as e: + log_file.write(f"Error occurred: {str(e)}\n") + +client.execute(f"DROP DATABASE {database_name}", settings=settings) + + +def enable_color(ratio): + if ratio == 0: + return "" + elif ratio < 0.5: + return "" + elif ratio < 1: + return "" + else: + return "" + + +reset_color = "" + + +def print_ratio(indent, name, success, total, description): + report_html_file.write( + "{}{}: {}{} / {} ({:.1%}){}{}\n".format( + " " * indent, + name.capitalize(), + enable_color(success / total), + success, + total, + success / total, + reset_color, + f" - " + html.escape(description) if description else "", + ) + ) + + +report_html_file.write( + "
\n"
+)
+
+print_ratio(0, "Total", summary["success"], summary["total"], "")
+
+for category in summary["results"]:
+    cat_summary = summary["results"][category]
+
+    if cat_summary["total"] == 0:
+        continue
+
+    print_ratio(2, category, cat_summary["success"], cat_summary["total"], "")
+
+    for test in summary["results"][category]["results"]:
+        test_summary = summary["results"][category]["results"][test]
+
+        if test_summary["total"] == 0:
+            continue
+
+        print_ratio(
+            6 if "-" in test else 4,
+            test,
+            test_summary["success"],
+            test_summary["total"],
+            test_summary["description"],
+        )
+
+report_html_file.write("
\n") diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index c973b6c6ec6..300d613e3b0 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -20,6 +20,22 @@ ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & ./setup_minio.sh stateful +# Setup a cluster for logs export to ClickHouse Cloud +# Note: these variables are provided to the Docker run command by the Python script in tests/ci +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] +then + echo " + remote_servers: + system_logs_export: + shard: + replica: + secure: 1 + user: ci + host: '${CLICKHOUSE_CI_LOGS_HOST}' + password: '${CLICKHOUSE_CI_LOGS_PASSWORD}' + " > /etc/clickhouse-server/config.d/system_logs_export.yaml +fi + function start() { if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then @@ -65,6 +81,22 @@ function start() } start + +# Initialize export of system logs to ClickHouse Cloud +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] +then + export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" + # TODO: Check if the password will appear in the logs. + export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}" + + ./setup_export_logs.sh + + # Unset variables after use + export CONNECTION_PARAMETERS='' + export CLICKHOUSE_CI_LOGS_HOST='' + export CLICKHOUSE_CI_LOGS_PASSWORD='' +fi + # shellcheck disable=SC2086 # No quotes because I want to split it into words. /s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 86ca7924f74..ab670209a69 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -87,4 +87,5 @@ RUN npm install -g azurite \ COPY run.sh / COPY setup_minio.sh / COPY setup_hdfs_minicluster.sh / + CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 3694fb7c2f6..015ba7d7423 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -36,6 +36,22 @@ fi ./setup_minio.sh stateless ./setup_hdfs_minicluster.sh +# Setup a cluster for logs export to ClickHouse Cloud +# Note: these variables are provided to the Docker run command by the Python script in tests/ci +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] +then + echo " + remote_servers: + system_logs_export: + shard: + replica: + secure: 1 + user: ci + host: '${CLICKHOUSE_CI_LOGS_HOST}' + password: '${CLICKHOUSE_CI_LOGS_PASSWORD}' + " > /etc/clickhouse-server/config.d/system_logs_export.yaml +fi + # For flaky check we also enable thread fuzzer if [ "$NUM_TRIES" -gt "1" ]; then export THREAD_FUZZER_CPU_TIME_PERIOD_US=1000 @@ -92,7 +108,28 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) fi -sleep 5 + +# Wait for the server to start, but not for too long. +for _ in {1..100} +do + clickhouse-client --query "SELECT 1" && break + sleep 1 +done + +# Initialize export of system logs to ClickHouse Cloud +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] +then + export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" + # TODO: Check if the password will appear in the logs. + export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}" + + ./setup_export_logs.sh + + # Unset variables after use + export CONNECTION_PARAMETERS='' + export CLICKHOUSE_CI_LOGS_HOST='' + export CLICKHOUSE_CI_LOGS_PASSWORD='' +fi attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 6148ea44da7..72d124d63a6 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -51,8 +51,39 @@ configure azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & ./setup_minio.sh stateless # to have a proper environment +# Setup a cluster for logs export to ClickHouse Cloud +# Note: these variables are provided to the Docker run command by the Python script in tests/ci +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] +then + echo " +remote_servers: + system_logs_export: + shard: + replica: + secure: 1 + user: ci + host: '${CLICKHOUSE_CI_LOGS_HOST}' + password: '${CLICKHOUSE_CI_LOGS_PASSWORD}' +" > /etc/clickhouse-server/config.d/system_logs_export.yaml +fi + start +# Initialize export of system logs to ClickHouse Cloud +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] +then + export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" + # TODO: Check if the password will appear in the logs. + export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}" + + ./setup_export_logs.sh + + # Unset variables after use + export CONNECTION_PARAMETERS='' + export CLICKHOUSE_CI_LOGS_HOST='' + export CLICKHOUSE_CI_LOGS_PASSWORD='' +fi + # shellcheck disable=SC2086 # No quotes because I want to split it into words. /s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 8779dd1a544..36afb94433a 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -323,9 +323,9 @@ clickhouse-client clickhouse://192.168.1.15,192.168.1.25 `clickhouse-client` uses the first existing file of the following: - Defined in the `--config-file` parameter. -- `./clickhouse-client.xml` -- `~/.clickhouse-client/config.xml` -- `/etc/clickhouse-client/config.xml` +- `./clickhouse-client.xml`, `.yaml`, `.yml` +- `~/.clickhouse-client/config.xml`, `.yaml`, `.yml` +- `/etc/clickhouse-client/config.xml`, `.yaml`, `.yml` Example of a config file: @@ -342,6 +342,17 @@ Example of a config file: ``` +Or the same config in a YAML format: + +```yaml +user: username +password: 'password' +secure: true +openSSL: + client: + caConfig: '/etc/ssl/cert.pem' +``` + ### Query ID Format {#query-id-format} In interactive mode `clickhouse-client` shows query ID for every query. By default, the ID is formatted like this: diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 02f52b6f8bf..6ed72152c1e 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -169,7 +169,6 @@ host = '127.0.0.1', port = 3306, database = 'test', connection_pool_size = 8, -on_duplicate_clause = 1, replace_query = 1 ``` @@ -185,7 +184,6 @@ replace_query = 1 3306 test 8 - 1 1 diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 5187ccce789..a7637082496 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1640,7 +1640,7 @@ Keys for server/client settings: - verificationMode (default: relaxed) – The method for checking the node’s certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: `none`, `relaxed`, `strict`, `once`. - verificationDepth (default: 9) – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile (default: true) – Wether built-in CA certificates for OpenSSL will be used. ClickHouse assumes that builtin CA certificates are in the file `/etc/ssl/cert.pem` (resp. the directory `/etc/ssl/certs`) or in file (resp. directory) specified by the environment variable `SSL_CERT_FILE` (resp. `SSL_CERT_DIR`). -- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`) - Supported OpenSSL encryptions. +- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`) - Supported OpenSSL encryptions. - cacheSessions (default: false) – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. Acceptable values: `true`, `false`. - sessionIdContext (default: `${application.name}`) – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. This parameter is always recommended since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: `${application.name}`. - sessionCacheSize (default: [1024\*20](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1978)) – The maximum number of sessions that the server caches. A value of 0 means unlimited sessions. diff --git a/docs/en/sql-reference/data-types/tuple.md b/docs/en/sql-reference/data-types/tuple.md index dfe0eda2e21..3add754e6b6 100644 --- a/docs/en/sql-reference/data-types/tuple.md +++ b/docs/en/sql-reference/data-types/tuple.md @@ -4,7 +4,7 @@ sidebar_position: 54 sidebar_label: Tuple(T1, T2, ...) --- -# Tuple(t1, T2, …) +# Tuple(T1, T2, …) A tuple of elements, each having an individual [type](../../sql-reference/data-types/index.md#data_types). Tuple must contain at least one element. diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index 6c3d80683db..c95ff5758f4 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -2476,52 +2476,3 @@ Dictionary updates (other than loading at first use) do not block queries. Durin We recommend periodically updating the dictionaries with the geobase. During an update, generate new files and write them to a separate location. When everything is ready, rename them to the files used by the server. There are also functions for working with OS identifiers and search engines, but they shouldn’t be used. - -## Embedded Dictionaries - - - -ClickHouse contains a built-in feature for working with a geobase. - -This allows you to: - -- Use a region’s ID to get its name in the desired language. -- Use a region’s ID to get the ID of a city, area, federal district, country, or continent. -- Check whether a region is part of another region. -- Get a chain of parent regions. - -All the functions support “translocality,” the ability to simultaneously use different perspectives on region ownership. For more information, see the section “Functions for working with web analytics dictionaries”. - -The internal dictionaries are disabled in the default package. -To enable them, uncomment the parameters `path_to_regions_hierarchy_file` and `path_to_regions_names_files` in the server configuration file. - -The geobase is loaded from text files. - -Place the `regions_hierarchy*.txt` files into the `path_to_regions_hierarchy_file` directory. This configuration parameter must contain the path to the `regions_hierarchy.txt` file (the default regional hierarchy), and the other files (`regions_hierarchy_ua.txt`) must be located in the same directory. - -Put the `regions_names_*.txt` files in the `path_to_regions_names_files` directory. - -You can also create these files yourself. The file format is as follows: - -`regions_hierarchy*.txt`: TabSeparated (no header), columns: - -- region ID (`UInt32`) -- parent region ID (`UInt32`) -- region type (`UInt8`): 1 - continent, 3 - country, 4 - federal district, 5 - region, 6 - city; other types do not have values -- population (`UInt32`) — optional column - -`regions_names_*.txt`: TabSeparated (no header), columns: - -- region ID (`UInt32`) -- region name (`String`) — Can’t contain tabs or line feeds, even escaped ones. - -A flat array is used for storing in RAM. For this reason, IDs shouldn’t be more than a million. - -Dictionaries can be updated without restarting the server. However, the set of available dictionaries is not updated. -For updates, the file modification times are checked. If a file has changed, the dictionary is updated. -The interval to check for changes is configured in the `builtin_dictionaries_reload_interval` parameter. -Dictionary updates (other than loading at first use) do not block queries. During updates, queries use the old versions of dictionaries. If an error occurs during an update, the error is written to the server log, and queries continue using the old version of dictionaries. - -We recommend periodically updating the dictionaries with the geobase. During an update, generate new files and write them to a separate location. When everything is ready, rename them to the files used by the server. - -There are also functions for working with OS identifiers and search engines, but they shouldn’t be used. diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 44d385312d0..f66994ed5a6 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -183,9 +183,8 @@ arrayConcat(arrays) **Arguments** - `arrays` – Arbitrary number of arguments of [Array](../../sql-reference/data-types/array.md) type. - **Example** - +**Example** ``` sql SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 7ed2deaeda6..88e4ac03fdb 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -559,6 +559,29 @@ Result: └────────────────────────────┘ ``` +## tupleConcat + +Combines tuples passed as arguments. + +``` sql +tupleConcat(tuples) +``` + +**Arguments** + +- `tuples` – Arbitrary number of arguments of [Tuple](../../sql-reference/data-types/tuple.md) type. + +**Example** + +``` sql +SELECT tupleConcat((1, 2), (3, 4), (true, false)) AS res +``` + +``` text +┌─res──────────────────┐ +│ (1,2,3,4,true,false) │ +└──────────────────────┘ +``` ## Distance functions diff --git a/docs/ru/operations/named-collections.md b/docs/ru/operations/named-collections.md index ba6b47116ad..48ee7c9f15d 100644 --- a/docs/ru/operations/named-collections.md +++ b/docs/ru/operations/named-collections.md @@ -88,7 +88,6 @@ SELECT * FROM s3_engine_table LIMIT 3; 3306 test 8 - 1 1 diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 81a696bcfc1..7b026244624 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1106,7 +1106,7 @@ ClickHouse использует потоки из глобального пул - verificationMode - Способ проверки сертификатов узла. Подробности находятся в описании класса [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h). Допустимые значения: `none`, `relaxed`, `strict`, `once`. - verificationDepth - Максимальная длина верификационной цепи. Верификация завершится ошибкой, если длина цепи сертификатов превысит установленное значение. - loadDefaultCAFile - Признак того, что будут использоваться встроенные CA-сертификаты для OpenSSL. Допустимые значения: `true`, `false`. \| -- cipherList - Поддерживаемые OpenSSL-шифры. Например, `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. +- cipherList - Поддерживаемые OpenSSL-шифры. Например, `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`. - cacheSessions - Включение/выключение кеширования сессии. Использовать обязательно вместе с `sessionIdContext`. Допустимые значения: `true`, `false`. - sessionIdContext - Уникальный набор произвольных символов, которые сервер добавляет к каждому сгенерированному идентификатору. Длина строки не должна превышать `SSL_MAX_SSL_SESSION_ID_LENGTH`. Рекомендуется к использованию всегда, поскольку позволяет избежать проблем как в случае, если сервер кеширует сессию, так и если клиент затребовал кеширование. По умолчанию `${application.name}`. - sessionCacheSize - Максимальное количество сессий, которые кэширует сервер. По умолчанию - 1024\*20. 0 - неограниченное количество сессий. diff --git a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md index eedc913cf82..3f42f3f8da4 100644 --- a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md +++ b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md @@ -346,9 +346,7 @@ UserID.bin,URL.bin,和EventTime.bin是UserID - 我们将主键列(UserID, URL)中的一些列值标记为橙色。 - 这些橙色标记的列值是每个颗粒中每个主键列的最小值。这里的例外是最后一个颗粒(上图中的颗粒1082),最后一个颗粒我们标记的是最大的值。 - - 正如我们将在下面看到的,这些橙色标记的列值将是表主索引中的条目。 + 这些橙色标记的列值是每个颗粒中第一行的主键列值。正如我们将在下面看到的,这些橙色标记的列值将是表主索引中的条目。 - 我们从0开始对行进行编号,以便与ClickHouse内部行编号方案对齐,该方案也用于记录消息。 ::: @@ -1071,13 +1069,6 @@ ClickHouse服务器日志文件中相应的跟踪日志确认了ClickHouse正在 ## 通过projections使用联合主键索引 -Projections目前是一个实验性的功能,因此我们需要告诉ClickHouse: - -```sql -SET optimize_use_projections = 1; -``` - - 在原表上创建projection: ```sql ALTER TABLE hits_UserID_URL @@ -1096,10 +1087,12 @@ ALTER TABLE hits_UserID_URL :::note - 该projection正在创建一个隐藏表,该表的行顺序和主索引基于该projection的给定order BY子句 -- 我们使用MATERIALIZE关键字,以便立即用源表hits_UserID_URL的所有887万行导入隐藏表 +- `SHOW TABLES` 语句查询是不会列出这个隐藏表的 +- 我们使用`MATERIALIZE`关键字,以便立即用源表hits_UserID_URL的所有887万行导入隐藏表 - 如果在源表hits_UserID_URL中插入了新行,那么这些行也会自动插入到隐藏表中 - 查询总是(从语法上)针对源表hits_UserID_URL,但是如果隐藏表的行顺序和主索引允许更有效地执行查询,那么将使用该隐藏表 -- 实际上,隐式创建的隐藏表的行顺序和主索引与我们显式创建的辅助表相同: +- 请注意,投影(projections)不会使 `ORDER BY` 查询语句的效率更高,即使 `ORDER BY` 匹配上了 projection 的 `ORDER BY` 语句(请参阅:https://github.com/ClickHouse/ClickHouse/issues/47333) +- 实际上,隐式创建的隐藏表的行顺序和主索引与我们显式创建的辅助表相同: @@ -1163,7 +1156,7 @@ ClickHouse服务器日志文件中跟踪日志确认了ClickHouse正在对索引 ``` -## 移除无效的主键列 +## 小结 带有联合主键(UserID, URL)的表的主索引对于加快UserID的查询过滤非常有用。但是,尽管URL列是联合主键的一部分,但该索引在加速URL查询过滤方面并没有提供显著的帮助。 @@ -1176,4 +1169,12 @@ ClickHouse服务器日志文件中跟踪日志确认了ClickHouse正在对索引 但是,如果复合主键中的键列在基数上有很大的差异,那么查询按基数升序对主键列进行排序是有益的。 -主键键列之间的基数差越大,主键键列的顺序越重要。我们将在以后的文章中对此进行演示。请继续关注。 +主键键列之间的基数差得越大,主键中的列的顺序越重要。我们将在下一章节对此进行演示。 + +# 高效地为键列排序 + +TODO + +# 高效地识别单行 + +TODO diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index f6106d8734e..8e2cb389f04 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -455,7 +455,7 @@ SSL客户端/服务器配置。 - verificationMode – The method for checking the node’s certificates. Details are in the description of the [A.背景](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) 同学们 可能的值: `none`, `relaxed`, `strict`, `once`. - verificationDepth – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile – Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `true`, `false`. \| -- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. +- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`. - cacheSessions – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. 可接受的值: `true`, `false`. - sessionIdContext – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. 始终建议使用此参数,因为如果服务器缓存会话,以及客户端请求缓存,它有助于避免出现问题。 默认值: `${application.name}`. - sessionCacheSize – The maximum number of sessions that the server caches. Default value: 1024\*20. 0 – Unlimited sessions. diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 466a0c194f7..c5acd10f791 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -1,8 +1,6 @@ #include #include -#include #include -#include #include #include #include @@ -18,9 +16,7 @@ #include #include #include -#include #include -#include #include #include #include @@ -38,8 +34,6 @@ #include -namespace fs = std::filesystem; - /** A tool for evaluating ClickHouse performance. * The tool emulates a case with fixed amount of simultaneously executing queries. */ @@ -79,7 +73,6 @@ public: bool randomize_, size_t max_iterations_, double max_time_, - const String & json_path_, size_t confidence_, const String & query_id_, const String & query_to_execute_, @@ -98,7 +91,6 @@ public: cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_), - json_path(json_path_), confidence(confidence_), query_id(query_id_), query_to_execute(query_to_execute_), @@ -165,9 +157,6 @@ public: int main(const std::vector &) override { - if (!json_path.empty() && fs::exists(json_path)) /// Clear file with previous results - fs::remove(json_path); - readQueries(); runBenchmark(); return 0; @@ -197,7 +186,6 @@ private: bool cumulative; size_t max_iterations; double max_time; - String json_path; size_t confidence; String query_id; String query_to_execute; @@ -226,26 +214,23 @@ private: size_t read_bytes = 0; size_t result_rows = 0; size_t result_bytes = 0; - double work_time = 0; using Sampler = ReservoirSampler; Sampler sampler {1 << 16}; - void add(double seconds, size_t read_rows_inc, size_t read_bytes_inc, size_t result_rows_inc, size_t result_bytes_inc) + void add(double duration, size_t read_rows_inc, size_t read_bytes_inc, size_t result_rows_inc, size_t result_bytes_inc) { ++queries; - work_time += seconds; read_rows += read_rows_inc; read_bytes += read_bytes_inc; result_rows += result_rows_inc; result_bytes += result_bytes_inc; - sampler.insert(seconds); + sampler.insert(duration); } void clear() { queries = 0; - work_time = 0; read_rows = 0; read_bytes = 0; result_rows = 0; @@ -331,10 +316,13 @@ private: return false; } - if (delay > 0 && delay_watch.elapsedSeconds() > delay) + double seconds = delay_watch.elapsedSeconds(); + if (delay > 0 && seconds > delay) { printNumberOfQueriesExecuted(queries_executed); - cumulative ? report(comparison_info_total) : report(comparison_info_per_interval); + cumulative + ? report(comparison_info_total, total_watch.elapsedSeconds()) + : report(comparison_info_per_interval, seconds); delay_watch.restart(); } } @@ -350,16 +338,7 @@ private: try { for (size_t i = 0; i < concurrency; ++i) - { - EntryPtrs connection_entries; - connection_entries.reserve(connections.size()); - - for (const auto & connection : connections) - connection_entries.emplace_back(std::make_shared( - connection->get(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings)))); - - pool.scheduleOrThrowOnError([this, connection_entries]() mutable { thread(connection_entries); }); - } + pool.scheduleOrThrowOnError([this]() mutable { thread(); }); } catch (...) { @@ -389,21 +368,18 @@ private: pool.wait(); total_watch.stop(); - if (!json_path.empty()) - reportJSON(comparison_info_total, json_path); - printNumberOfQueriesExecuted(queries_executed); - report(comparison_info_total); + report(comparison_info_total, total_watch.elapsedSeconds()); } - void thread(EntryPtrs & connection_entries) + void thread() { Query query; /// Randomly choosing connection index pcg64 generator(randomSeed()); - std::uniform_int_distribution distribution(0, connection_entries.size() - 1); + std::uniform_int_distribution distribution(0, connections.size() - 1); /// In these threads we do not accept INT signal. sigset_t sig_set; @@ -423,15 +399,13 @@ private: extracted = queue.tryPop(query, 100); if (shutdown || (max_iterations && queries_executed == max_iterations)) - { return; - } } const auto connection_index = distribution(generator); try { - execute(connection_entries, query, connection_index); + execute(query, connection_index); consecutive_errors = 0; } catch (...) @@ -460,17 +434,18 @@ private: } } - void execute(EntryPtrs & connection_entries, Query & query, size_t connection_index) + void execute(Query & query, size_t connection_index) { Stopwatch watch; - Connection & connection = **connection_entries[connection_index]; + ConnectionPool::Entry entry = connections[connection_index]->get( + ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings)); if (reconnect) - connection.disconnect(); + entry->disconnect(); RemoteQueryExecutor executor( - connection, query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage); + *entry, query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage); if (!query_id.empty()) executor.setQueryId(query_id); @@ -485,19 +460,19 @@ private: executor.finish(); - double seconds = (display_client_side_time || progress.elapsed_ns == 0) + double duration = (display_client_side_time || progress.elapsed_ns == 0) ? watch.elapsedSeconds() : progress.elapsed_ns / 1e9; std::lock_guard lock(mutex); size_t info_index = round_robin ? 0 : connection_index; - comparison_info_per_interval[info_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); - comparison_info_total[info_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes); - t_test.add(info_index, seconds); + comparison_info_per_interval[info_index]->add(duration, progress.read_rows, progress.read_bytes, info.rows, info.bytes); + comparison_info_total[info_index]->add(duration, progress.read_rows, progress.read_bytes, info.rows, info.bytes); + t_test.add(info_index, duration); } - void report(MultiStats & infos) + void report(MultiStats & infos, double seconds) { std::lock_guard lock(mutex); @@ -510,8 +485,6 @@ private: if (0 == info->queries) return; - double seconds = info->work_time / concurrency; - std::string connection_description = connections[i]->getDescription(); if (round_robin) { @@ -525,10 +498,10 @@ private: } std::cerr << connection_description << ", " - << "queries " << info->queries << ", "; + << "queries: " << info->queries << ", "; if (info->errors) { - std::cerr << "errors " << info->errors << ", "; + std::cerr << "errors: " << info->errors << ", "; } std::cerr << "QPS: " << (info->queries / seconds) << ", " @@ -567,62 +540,6 @@ private: } } - void reportJSON(MultiStats & infos, const std::string & filename) - { - WriteBufferFromFile json_out(filename); - - std::lock_guard lock(mutex); - - auto print_key_value = [&](auto key, auto value, bool with_comma = true) - { - json_out << double_quote << key << ": " << value << (with_comma ? ",\n" : "\n"); - }; - - auto print_percentile = [&json_out](Stats & info, auto percent, bool with_comma = true) - { - json_out << "\"" << percent << "\": " << info.sampler.quantileNearest(percent / 100.0) << (with_comma ? ",\n" : "\n"); - }; - - json_out << "{\n"; - - for (size_t i = 0; i < infos.size(); ++i) - { - const auto & info = infos[i]; - - json_out << double_quote << connections[i]->getDescription() << ": {\n"; - json_out << double_quote << "statistics" << ": {\n"; - - double seconds = info->work_time / concurrency; - - print_key_value("QPS", info->queries.load() / seconds); - print_key_value("RPS", info->read_rows / seconds); - print_key_value("MiBPS", info->read_bytes / seconds / 1048576); - print_key_value("RPS_result", info->result_rows / seconds); - print_key_value("MiBPS_result", info->result_bytes / seconds / 1048576); - print_key_value("num_queries", info->queries.load()); - print_key_value("num_errors", info->errors, false); - - json_out << "},\n"; - json_out << double_quote << "query_time_percentiles" << ": {\n"; - - if (info->queries != 0) - { - for (int percent = 0; percent <= 90; percent += 10) - print_percentile(*info, percent); - - print_percentile(*info, 95); - print_percentile(*info, 99); - print_percentile(*info, 99.9); - print_percentile(*info, 99.99, false); - } - - json_out << "}\n"; - json_out << (i == infos.size() - 1 ? "}\n" : "},\n"); - } - - json_out << "}\n"; - } - public: ~Benchmark() override @@ -675,7 +592,6 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("iterations,i", value()->default_value(0), "amount of queries to be executed") ("timelimit,t", value()->default_value(0.), "stop launch of queries after specified time limit") ("randomize,r", "randomize order of execution") - ("json", value()->default_value(""), "write final report to specified file in JSON format") ("host,h", value()->multitoken(), "list of hosts") ("port", value()->multitoken(), "list of ports") ("roundrobin", "Instead of comparing queries for different --host/--port just pick one random --host/--port for every query and send query to it.") @@ -739,7 +655,6 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options.count("randomize"), options["iterations"].as(), options["timelimit"].as(), - options["json"].as(), options["confidence"].as(), options["query_id"].as(), options["query"].as(), diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index efe7121cace..556eca808f6 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -608,6 +608,8 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t ss << "ALTER TABLE " << getQuotedTable(original_table) << ((partition_name == "'all'") ? " DROP PARTITION ID " : " DROP PARTITION ") << partition_name; UInt64 num_shards_drop_partition = executeQueryOnCluster(task_table.cluster_push, ss.str(), task_cluster->settings_push, ClusterExecutionMode::ON_EACH_SHARD); + if (num_shards_drop_partition != task_table.cluster_push->getShardCount()) + return TaskStatus::Error; LOG_INFO(log, "Drop partition {} in original table {} have been executed successfully on {} shards of {}", partition_name, getQuotedTable(original_table), num_shards_drop_partition, task_table.cluster_push->getShardCount()); diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 43c66a32302..d7d61bbcd3b 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -163,13 +163,15 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { ASTPtr res = parseQueryAndMovePosition( parser, pos, end, "query", multiple, cmd_settings.max_query_size, cmd_settings.max_parser_depth); - /// For insert query with data(INSERT INTO ... VALUES ...), will lead to format fail, - /// should throw exception early and make exception message more readable. + + /// For insert query with data(INSERT INTO ... VALUES ...), that will lead to the formatting failure, + /// we should throw an exception early, and make exception message more readable. if (const auto * insert_query = res->as(); insert_query && insert_query->data) { throw Exception(DB::ErrorCodes::INVALID_FORMAT_INSERT_QUERY_WITH_DATA, "Can't format ASTInsertQuery with data, since data will be lost"); } + if (!quiet) { if (!backslash) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 485476efd5e..afad5be6b26 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1655,6 +1655,9 @@ try database_catalog.initializeAndLoadTemporaryDatabase(); loadMetadataSystem(global_context); maybeConvertSystemDatabase(global_context); + /// This has to be done before the initialization of system logs, + /// otherwise there is a race condition between the system database initialization + /// and creation of new tables in the database. startupSystemTables(); /// After attaching system databases we can initialize system log. global_context->initializeSystemLogs(); diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index e277c49e39d..c3aeec15cdd 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -550,12 +550,12 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg return access_denied(ErrorCodes::ACCESS_DENIED, "{}: Not enough privileges. " "The required privileges have been granted, but without grant option. " - "To execute this query it's necessary to have grant {} WITH GRANT OPTION", + "To execute this query, it's necessary to have the grant {} WITH GRANT OPTION", AccessRightsElement{flags, args...}.toStringWithoutOptions()); } return access_denied(ErrorCodes::ACCESS_DENIED, - "{}: Not enough privileges. To execute this query it's necessary to have grant {}", + "{}: Not enough privileges. To execute this query, it's necessary to have the grant {}", AccessRightsElement{flags, args...}.toStringWithoutOptions() + (grant_option ? " WITH GRANT OPTION" : "")); } @@ -756,11 +756,11 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const show_error(ErrorCodes::ACCESS_DENIED, "Not enough privileges. " "Role {} is granted, but without ADMIN option. " - "To execute this query it's necessary to have the role {} granted with ADMIN option.", + "To execute this query, it's necessary to have the role {} granted with ADMIN option.", backQuote(*role_name), backQuoteIfNeed(*role_name)); else show_error(ErrorCodes::ACCESS_DENIED, "Not enough privileges. " - "To execute this query it's necessary to have the role {} granted with ADMIN option.", + "To execute this query, it's necessary to have the role {} granted with ADMIN option.", backQuoteIfNeed(*role_name)); } diff --git a/src/AggregateFunctions/AggregateFunctionForEach.h b/src/AggregateFunctions/AggregateFunctionForEach.h index 480b4cc690e..ec59fd2e616 100644 --- a/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/src/AggregateFunctions/AggregateFunctionForEach.h @@ -240,7 +240,7 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const AggregateFunctionForEachData & state = data(place); - writeBinary(state.dynamic_array_size, buf); + writeBinaryLittleEndian(state.dynamic_array_size, buf); const char * nested_state = state.array_of_aggregate_datas; for (size_t i = 0; i < state.dynamic_array_size; ++i) @@ -255,7 +255,7 @@ public: AggregateFunctionForEachData & state = data(place); size_t new_size = 0; - readBinary(new_size, buf); + readBinaryLittleEndian(new_size, buf); ensureAggregateData(place, new_size, *arena); diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index ca6d31a716d..3e501b29414 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -108,7 +108,14 @@ private: inline size_t buf_size() const { return 1ULL << size_degree; } /// NOLINT inline size_t max_fill() const { return 1ULL << (size_degree - 1); } /// NOLINT inline size_t mask() const { return buf_size() - 1; } - inline size_t place(HashValue x) const { return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); } + + inline size_t place(HashValue x) const + { + if constexpr (std::endian::native == std::endian::little) + return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); + else + return (std::byteswap(x) >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); + } /// The value is divided by 2 ^ skip_degree inline bool good(HashValue hash) const diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp index 9f137124be0..cd5bd2b83fa 100644 --- a/src/Common/Config/configReadClient.cpp +++ b/src/Common/Config/configReadClient.cpp @@ -5,36 +5,57 @@ #include #include + namespace fs = std::filesystem; namespace DB { -bool safeFsExists(const String & path) -{ - std::error_code ec; - return fs::exists(path, ec); -} - bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::string & home_path) { std::string config_path; - if (config.has("config-file")) - config_path = config.getString("config-file"); - else if (safeFsExists("./clickhouse-client.xml")) - config_path = "./clickhouse-client.xml"; - else if (!home_path.empty() && safeFsExists(home_path + "/.clickhouse-client/config.xml")) - config_path = home_path + "/.clickhouse-client/config.xml"; - else if (safeFsExists("/etc/clickhouse-client/config.xml")) - config_path = "/etc/clickhouse-client/config.xml"; - if (!config_path.empty()) + bool found = false; + if (config.has("config-file")) + { + found = true; + config_path = config.getString("config-file"); + } + else + { + std::vector names; + names.emplace_back("./clickhouse-client"); + if (!home_path.empty()) + names.emplace_back(home_path + "/.clickhouse-client/config"); + names.emplace_back("/etc/clickhouse-client/config"); + + for (const auto & name : names) + { + for (const auto & extension : {".xml", ".yaml", ".yml"}) + { + config_path = name + extension; + + std::error_code ec; + if (fs::exists(config_path, ec)) + { + found = true; + break; + } + } + if (found) + break; + } + } + + if (found) { ConfigProcessor config_processor(config_path); auto loaded_config = config_processor.loadConfig(); config.add(loaded_config.configuration); return true; } + return false; } + } diff --git a/src/Common/LoggingFormatStringHelpers.h b/src/Common/LoggingFormatStringHelpers.h index 6dffd202807..06320509c17 100644 --- a/src/Common/LoggingFormatStringHelpers.h +++ b/src/Common/LoggingFormatStringHelpers.h @@ -1,4 +1,5 @@ #pragma once + #include #include #include @@ -6,6 +7,8 @@ #include #include #include +#include + struct PreformattedMessage; consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs); diff --git a/src/Common/NamedCollections/NamedCollectionConfiguration.cpp b/src/Common/NamedCollections/NamedCollectionConfiguration.cpp index 1c42b001ceb..d9bf2349a5a 100644 --- a/src/Common/NamedCollections/NamedCollectionConfiguration.cpp +++ b/src/Common/NamedCollections/NamedCollectionConfiguration.cpp @@ -3,6 +3,8 @@ #include #include #include +#include + namespace DB { diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 28ccc91b014..1f5832e7aac 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -33,13 +33,17 @@ namespace Poco { + class Logger; + namespace Util { class AbstractConfiguration; } + } + namespace DB { diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 90dcb2e5f8e..1fcf663a744 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -11,8 +11,11 @@ #include +using namespace std::literals; + namespace DB { + namespace ErrorCodes { extern const int CANNOT_RESTORE_FROM_FIELD_DUMP; @@ -582,34 +585,41 @@ String toString(const Field & x) x); } -String fieldTypeToString(Field::Types::Which type) +std::string_view fieldTypeToString(Field::Types::Which type) { switch (type) { - case Field::Types::Which::Null: return "Null"; - case Field::Types::Which::Array: return "Array"; - case Field::Types::Which::Tuple: return "Tuple"; - case Field::Types::Which::Map: return "Map"; - case Field::Types::Which::Object: return "Object"; - case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState"; - case Field::Types::Which::Bool: return "Bool"; - case Field::Types::Which::String: return "String"; - case Field::Types::Which::Decimal32: return "Decimal32"; - case Field::Types::Which::Decimal64: return "Decimal64"; - case Field::Types::Which::Decimal128: return "Decimal128"; - case Field::Types::Which::Decimal256: return "Decimal256"; - case Field::Types::Which::Float64: return "Float64"; - case Field::Types::Which::Int64: return "Int64"; - case Field::Types::Which::Int128: return "Int128"; - case Field::Types::Which::Int256: return "Int256"; - case Field::Types::Which::UInt64: return "UInt64"; - case Field::Types::Which::UInt128: return "UInt128"; - case Field::Types::Which::UInt256: return "UInt256"; - case Field::Types::Which::UUID: return "UUID"; - case Field::Types::Which::IPv4: return "IPv4"; - case Field::Types::Which::IPv6: return "IPv6"; - case Field::Types::Which::CustomType: return "CustomType"; + case Field::Types::Which::Null: return "Null"sv; + case Field::Types::Which::Array: return "Array"sv; + case Field::Types::Which::Tuple: return "Tuple"sv; + case Field::Types::Which::Map: return "Map"sv; + case Field::Types::Which::Object: return "Object"sv; + case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState"sv; + case Field::Types::Which::Bool: return "Bool"sv; + case Field::Types::Which::String: return "String"sv; + case Field::Types::Which::Decimal32: return "Decimal32"sv; + case Field::Types::Which::Decimal64: return "Decimal64"sv; + case Field::Types::Which::Decimal128: return "Decimal128"sv; + case Field::Types::Which::Decimal256: return "Decimal256"sv; + case Field::Types::Which::Float64: return "Float64"sv; + case Field::Types::Which::Int64: return "Int64"sv; + case Field::Types::Which::Int128: return "Int128"sv; + case Field::Types::Which::Int256: return "Int256"sv; + case Field::Types::Which::UInt64: return "UInt64"sv; + case Field::Types::Which::UInt128: return "UInt128"sv; + case Field::Types::Which::UInt256: return "UInt256"sv; + case Field::Types::Which::UUID: return "UUID"sv; + case Field::Types::Which::IPv4: return "IPv4"sv; + case Field::Types::Which::IPv6: return "IPv6"sv; + case Field::Types::Which::CustomType: return "CustomType"sv; } } +/// Keep in mind, that "magic_enum" is very expensive for compiler, that's why we don't use it. +std::string_view Field::getTypeName() const +{ + return fieldTypeToString(which); +} + + } diff --git a/src/Core/Field.h b/src/Core/Field.h index ef1bd9a895d..12542ca0bf1 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -15,8 +15,7 @@ #include #include #include -#include -#include + namespace DB { @@ -449,7 +448,7 @@ public: Types::Which getType() const { return which; } - constexpr std::string_view getTypeName() const { return magic_enum::enum_name(which); } + std::string_view getTypeName() const; bool isNull() const { return which == Types::Null; } template @@ -1005,7 +1004,7 @@ void writeFieldText(const Field & x, WriteBuffer & buf); String toString(const Field & x); -String fieldTypeToString(Field::Types::Which type); +std::string_view fieldTypeToString(Field::Types::Which type); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6c3d339b4be..714ac17a15d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -816,7 +816,6 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \ /* moved to config.xml: see also src/Core/ServerSettings.h */ \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_buffer_flush_schedule_pool_size, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_pool_size, 16) \ diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 32fffd3af06..db14f91512f 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -10,6 +10,8 @@ #include #include #include +#include + namespace DB { diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 37a0f2db3e2..1a79c23d955 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -72,12 +72,6 @@ void applySettingsQuirks(Settings & settings, Poco::Logger * log) } } -#if defined(THREAD_SANITIZER) - settings.use_hedged_requests.value = false; - if (log) - LOG_WARNING(log, "use_hedged_requests has been disabled for the build with Thread Sanitizer, because they are using fibers, leading to a failed assertion inside TSan"); -#endif - if (!queryProfilerWorks()) { if (settings.query_profiler_real_time_period_ns) diff --git a/src/Core/examples/CMakeLists.txt b/src/Core/examples/CMakeLists.txt index 868173e0e31..2326eada96d 100644 --- a/src/Core/examples/CMakeLists.txt +++ b/src/Core/examples/CMakeLists.txt @@ -9,6 +9,3 @@ target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io) clickhouse_add_executable (mysql_protocol mysql_protocol.cpp) target_link_libraries (mysql_protocol PRIVATE dbms) - -clickhouse_add_executable (coro coro.cpp) -target_link_libraries (coro PRIVATE clickhouse_common_io) diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp deleted file mode 100644 index fbccc261e9d..00000000000 --- a/src/Core/examples/coro.cpp +++ /dev/null @@ -1,194 +0,0 @@ -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#if defined(__clang__) -#include - -namespace std // NOLINT(cert-dcl58-cpp) -{ - using namespace experimental::coroutines_v1; // NOLINT(cert-dcl58-cpp) -} - -#if __has_warning("-Wdeprecated-experimental-coroutine") -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wdeprecated-experimental-coroutine" -#endif - -#else -#include -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wzero-as-null-pointer-constant" -#endif - - -template -struct suspend_value // NOLINT(readability-identifier-naming) -{ - constexpr bool await_ready() const noexcept { return true; } // NOLINT(readability-identifier-naming) - constexpr void await_suspend(std::coroutine_handle<>) const noexcept {} // NOLINT(readability-identifier-naming) - constexpr T await_resume() const noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " ret " << val << std::endl; - return val; - } - - T val; -}; - -template -struct Task -{ - struct promise_type // NOLINT(readability-identifier-naming) - { - using coro_handle = std::coroutine_handle; - auto get_return_object() { return coro_handle::from_promise(*this); } // NOLINT(readability-identifier-naming) - auto initial_suspend() { return std::suspend_never(); } // NOLINT(readability-identifier-naming) - auto final_suspend() noexcept { return suspend_value{*r->value}; } // NOLINT(readability-identifier-naming) - //void return_void() {} - void return_value(T value_) { r->value = value_; } // NOLINT(readability-identifier-naming) - void unhandled_exception() // NOLINT(readability-identifier-naming) - { - DB::tryLogCurrentException("Logger"); - r->exception = std::current_exception(); // NOLINT(bugprone-throw-keyword-missing) - } - - explicit promise_type(std::string tag_) : tag(tag_) {} - ~promise_type() { std::cout << "~promise_type " << tag << std::endl; } - std::string tag; - coro_handle next; - Task * r = nullptr; - }; - - using coro_handle = std::coroutine_handle; - - bool await_ready() const noexcept { return false; } // NOLINT(readability-identifier-naming) - void await_suspend(coro_handle g) noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " await_suspend " << my.promise().tag << std::endl; - std::cout << " g tag " << g.promise().tag << std::endl; - g.promise().next = my; - } - T await_resume() noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " await_res " << my.promise().tag << std::endl; - return *value; - } - - Task(coro_handle handle) : my(handle), tag(handle.promise().tag) // NOLINT(google-explicit-constructor) - { - assert(handle); - my.promise().r = this; - std::cout << " Task " << tag << std::endl; - } - Task(Task &) = delete; - Task(Task &&rhs) noexcept : my(rhs.my), tag(rhs.tag) - { - rhs.my = {}; - std::cout << " Task&& " << tag << std::endl; - } - static bool resumeImpl(Task *r) - { - if (r->value) - return false; - - auto & next = r->my.promise().next; - - if (next) - { - if (resumeImpl(next.promise().r)) - return true; - next = {}; - } - - if (!r->value) - { - r->my.resume(); - if (r->exception) - std::rethrow_exception(r->exception); - } - return !r->value; - } - - bool resume() - { - return resumeImpl(this); - } - - T res() - { - return *value; - } - - ~Task() - { - std::cout << " ~Task " << tag << std::endl; - } - -private: - coro_handle my; - std::string tag; - std::optional value; - std::exception_ptr exception; -}; - -Task boo([[maybe_unused]] std::string tag) -{ - std::cout << "x" << std::endl; - co_await std::suspend_always(); - std::cout << StackTrace().toString(); - std::cout << "y" << std::endl; - co_return 1; -} - -Task bar([[maybe_unused]] std::string tag) -{ - std::cout << "a" << std::endl; - int res1 = co_await boo("boo1"); - std::cout << "b " << res1 << std::endl; - int res2 = co_await boo("boo2"); - if (res2 == 1) - throw DB::Exception(1, "hello"); - std::cout << "c " << res2 << std::endl; - co_return res1 + res2; // 1 + 1 = 2 -} - -Task foo([[maybe_unused]] std::string tag) -{ - std::cout << "Hello" << std::endl; - auto res1 = co_await bar("bar1"); - std::cout << "Coro " << res1 << std::endl; - auto res2 = co_await bar("bar2"); - std::cout << "World " << res2 << std::endl; - co_return res1 * res2; // 2 * 2 = 4 -} - -int main() -{ - Poco::AutoPtr app_channel(new Poco::ConsoleChannel(std::cerr)); - Poco::Logger::root().setChannel(app_channel); - Poco::Logger::root().setLevel("trace"); - - LOG_INFO(&Poco::Logger::get(""), "Starting"); - - try - { - auto t = foo("foo"); - std::cout << ".. started" << std::endl; - while (t.resume()) - std::cout << ".. yielded" << std::endl; - std::cout << ".. done: " << t.res() << std::endl; - } - catch (DB::Exception & e) - { - std::cout << "Got exception " << e.what() << std::endl; - std::cout << e.getStackTraceString() << std::endl; - } -} diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index cedcca870dd..e01c1aea0e9 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -129,7 +129,7 @@ namespace for (size_t i = offset; i < end; ++i) { ColumnArray::Offset current_offset = offset_values[i]; - writeIntBinary(current_offset - prev_offset, ostr); + writeBinaryLittleEndian(current_offset - prev_offset, ostr); prev_offset = current_offset; } } @@ -145,7 +145,7 @@ namespace while (i < initial_size + limit && !istr.eof()) { ColumnArray::Offset current_size = 0; - readIntBinary(current_size, istr); + readBinaryLittleEndian(current_size, istr); if (unlikely(current_size > MAX_ARRAY_SIZE)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size is too large: {}", current_size); diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 8cabaec753d..0294a1c8a67 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -106,28 +106,28 @@ void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & { /// ColumnVector::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 typename ColumnVector::ValueType x = static_cast::ValueType>(field.get()); - writeBinary(x, ostr); + writeBinaryLittleEndian(x, ostr); } template void SerializationNumber::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); field = NearestFieldType(x); } template void SerializationNumber::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeBinary(assert_cast &>(column).getData()[row_num], ostr); + writeBinaryLittleEndian(assert_cast &>(column).getData()[row_num], ostr); } template void SerializationNumber::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); assert_cast &>(column).getData().push_back(x); } diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 0f65069db35..bbab279c0e2 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -471,8 +471,16 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & { String link = path_to_table_symlinks + escapeForFileName(table_name); fs::path data = fs::canonical(getContext()->getPath()) / actual_data_path; - if (!if_data_path_exist || fs::exists(data)) - fs::create_directory_symlink(data, link); + + /// If it already points where needed. + std::error_code ec; + if (fs::equivalent(data, link, ec)) + return; + + if (if_data_path_exist && !fs::exists(data)) + return; + + fs::create_directory_symlink(data, link); } catch (...) { diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 86a5ce29d9c..d724c35c383 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -755,6 +755,9 @@ class FunctionBinaryArithmetic : public IFunction static constexpr bool is_multiply = IsOperation::multiply; static constexpr bool is_division = IsOperation::division; static constexpr bool is_bit_hamming_distance = IsOperation::bit_hamming_distance; + static constexpr bool is_modulo = IsOperation::modulo; + static constexpr bool is_div_int = IsOperation::div_int; + static constexpr bool is_div_int_or_zero = IsOperation::div_int_or_zero; ContextPtr context; bool check_decimal_overflow = true; @@ -964,13 +967,28 @@ class FunctionBinaryArithmetic : public IFunction "argument of numeric type cannot be first", name); std::string function_name; - if (is_multiply) + if constexpr (is_multiply) { function_name = "tupleMultiplyByNumber"; } - else + else // is_division { - function_name = "tupleDivideByNumber"; + if constexpr (is_modulo) + { + function_name = "tupleModuloByNumber"; + } + else if constexpr (is_div_int) + { + function_name = "tupleIntDivByNumber"; + } + else if constexpr (is_div_int_or_zero) + { + function_name = "tupleIntDivOrZeroByNumber"; + } + else + { + function_name = "tupleDivideByNumber"; + } } return FunctionFactory::instance().get(function_name, context); diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index db6529da73c..1b2e2eb3bd6 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -62,14 +62,13 @@ namespace ErrorCodes */ -class FunctionDictHelper : WithContext +class FunctionDictHelper { public: - explicit FunctionDictHelper(ContextPtr context_) : WithContext(context_) {} + explicit FunctionDictHelper(ContextPtr context_) : current_context(context_) {} std::shared_ptr getDictionary(const String & dictionary_name) { - auto current_context = getContext(); auto dict = current_context->getExternalDictionariesLoader().getDictionary(dictionary_name, current_context); if (!access_checked) @@ -132,10 +131,12 @@ public: DictionaryStructure getDictionaryStructure(const String & dictionary_name) const { - return getContext()->getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, getContext()); + return current_context->getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, current_context); } private: + ContextPtr current_context; + /// Access cannot be not granted, since in this case checkAccess() will throw and access_checked will not be updated. std::atomic access_checked = false; diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index ab6b6399486..a4ff49859cc 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -13,6 +13,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; } /** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. @@ -39,9 +40,6 @@ struct HasTokenImpl if (start_pos != nullptr) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' does not support start_pos argument", name); - if (pattern.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); - if (haystack_offsets.empty()) return; @@ -49,7 +47,7 @@ struct HasTokenImpl const UInt8 * const end = haystack_data.data() + haystack_data.size(); const UInt8 * pos = begin; - if (!std::none_of(pattern.begin(), pattern.end(), isTokenSeparator)) + if (const auto has_separator = std::any_of(pattern.cbegin(), pattern.cend(), isTokenSeparator); has_separator || pattern.empty()) { if (res_null) { @@ -57,8 +55,12 @@ struct HasTokenImpl std::ranges::fill(res_null->getData(), true); return; } - else + else if (has_separator) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters"); + else if (pattern.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected internal state"); } size_t pattern_size = pattern.size(); diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index 0c54901579e..8ea53c865ce 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -60,7 +60,7 @@ struct IsOperation static constexpr bool bit_hamming_distance = IsSameOperation::value; - static constexpr bool division = div_floating || div_int || div_int_or_zero; + static constexpr bool division = div_floating || div_int || div_int_or_zero || modulo; static constexpr bool allow_decimal = plus || minus || multiply || division || least || greatest; }; diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 8288d872f18..9eb222d8c09 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -208,6 +208,10 @@ public: { return FunctionFactory::instance().getImpl("mapConcat", context)->build(arguments); } + else if (isTuple(arguments.at(0).type)) + { + return FunctionFactory::instance().getImpl("tupleConcat", context)->build(arguments); + } else return std::make_unique( FunctionConcat::create(context), collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index e03701327b1..16326dd5a44 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -764,9 +764,8 @@ namespace } /// Note: Doesn't check the duplicates in the `from` array. - - WhichDataType which(from_type); - if (isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) + /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 + if (WhichDataType which(from_type); isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) { cache.table_num_to_idx = std::make_unique(); auto & table = *cache.table_num_to_idx; @@ -774,10 +773,13 @@ namespace { if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i])) { - /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 - StringRef ref = cache.from_column->getDataAt(i); UInt64 key = 0; - memcpy(&key, ref.data, ref.size); + auto * dst = reinterpret_cast(&key); + const auto ref = cache.from_column->getDataAt(i); + if constexpr (std::endian::native == std::endian::big) + dst += sizeof(key) - ref.size; + + memcpy(dst, ref.data, ref.size); table[key] = i; } } diff --git a/src/Functions/tupleConcat.cpp b/src/Functions/tupleConcat.cpp new file mode 100644 index 00000000000..0556f4181e6 --- /dev/null +++ b/src/Functions/tupleConcat.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; +} + +/// tupleConcat(tup1, ...) - concatenate tuples. +class FunctionTupleConcat : public IFunction +{ +public: + static constexpr auto name = "tupleConcat"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} requires at least one argument.", + getName()); + + DataTypes tuple_arg_types; + + for (const auto arg_idx : collections::range(0, arguments.size())) + { + const auto * arg = arguments[arg_idx].get(); + if (!isTuple(arg)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument {} of function {}", + arg->getName(), + arg_idx + 1, + getName()); + + const auto * type = checkAndGetDataType(arg); + for (const auto & elem : type->getElements()) + tuple_arg_types.push_back(elem); + } + + return std::make_shared(tuple_arg_types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + const size_t num_arguments = arguments.size(); + Columns columns; + + for (size_t i = 0; i < num_arguments; i++) + { + const DataTypeTuple * arg_type = checkAndGetDataType(arguments[i].type.get()); + + if (!arg_type) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument {} of function {}", + arguments[i].type->getName(), + i + 1, + getName()); + + ColumnPtr arg_col = arguments[i].column->convertToFullColumnIfConst(); + const ColumnTuple * tuple_col = checkAndGetColumn(arg_col.get()); + + if (!tuple_col) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of argument of function {}", + arguments[i].column->getName(), + getName()); + + for (const auto & inner_col : tuple_col->getColumns()) + columns.push_back(inner_col); + } + + return ColumnTuple::create(columns); + } +}; + +REGISTER_FUNCTION(TupleConcat) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index d53d39e2f3b..35ba49e4545 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -23,6 +23,9 @@ struct PlusName { static constexpr auto name = "plus"; }; struct MinusName { static constexpr auto name = "minus"; }; struct MultiplyName { static constexpr auto name = "multiply"; }; struct DivideName { static constexpr auto name = "divide"; }; +struct ModuloName { static constexpr auto name = "modulo"; }; +struct IntDivName { static constexpr auto name = "intDiv"; }; +struct IntDivOrZeroName { static constexpr auto name = "intDivOrZero"; }; struct L1Label { static constexpr auto name = "1"; }; struct L2Label { static constexpr auto name = "2"; }; @@ -141,6 +144,12 @@ using FunctionTupleMultiply = FunctionTupleOperator; using FunctionTupleDivide = FunctionTupleOperator; +using FunctionTupleModulo = FunctionTupleOperator; + +using FunctionTupleIntDiv = FunctionTupleOperator; + +using FunctionTupleIntDivOrZero = FunctionTupleOperator; + class FunctionTupleNegate : public ITupleFunction { public: @@ -297,6 +306,12 @@ using FunctionTupleMultiplyByNumber = FunctionTupleOperatorByNumber; +using FunctionTupleModuloByNumber = FunctionTupleOperatorByNumber; + +using FunctionTupleIntDivByNumber = FunctionTupleOperatorByNumber; + +using FunctionTupleIntDivOrZeroByNumber = FunctionTupleOperatorByNumber; + class FunctionDotProduct : public ITupleFunction { public: @@ -1563,6 +1578,9 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerAlias("vectorDifference", FunctionTupleMinus::name, FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(FunctionDocumentation @@ -1626,6 +1644,9 @@ If the types of the first interval (or the interval in the tuple) and the second factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerAlias("scalarProduct", TupleOrArrayFunctionDotProduct::name, FunctionFactory::CaseInsensitive); diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 84550d3f7c5..d59bc4fd115 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -14,26 +14,26 @@ # include # include -# include # include -# include # include +# include namespace DB { + namespace ErrorCodes { extern const int AWS_ERROR; } -} -namespace DB::S3 +namespace S3 { namespace { + bool areCredentialsEmptyOrExpired(const Aws::Auth::AWSCredentials & credentials, uint64_t expiration_window_seconds) { if (credentials.IsEmpty()) @@ -569,4 +569,6 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( } +} + #endif diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index aa4c9b17e48..d092c7b8ea5 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -304,9 +304,10 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer & /** Will escape quote_character and a list of special characters('\b', '\f', '\n', '\r', '\t', '\0', '\\'). * - when escape_quote_with_quote is true, use backslash to escape list of special characters, * and use quote_character to escape quote_character. such as: 'hello''world' - * - otherwise use backslash to escape list of special characters and quote_character + * otherwise use backslash to escape list of special characters and quote_character + * - when escape_backslash_with_backslash is true, backslash is escaped with another backslash */ -template +template void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & buf) { const char * pos = begin; @@ -360,7 +361,8 @@ void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & b writeChar('0', buf); break; case '\\': - writeChar('\\', buf); + if constexpr (escape_backslash_with_backslash) + writeChar('\\', buf); writeChar('\\', buf); break; default: @@ -466,6 +468,13 @@ inline void writeQuotedString(std::string_view ref, WriteBuffer & buf) writeAnyQuotedString<'\''>(ref.data(), ref.data() + ref.size(), buf); } +inline void writeQuotedStringPostgreSQL(std::string_view ref, WriteBuffer & buf) +{ + writeChar('\'', buf); + writeAnyEscapedString<'\'', true, false>(ref.data(), ref.data() + ref.size(), buf); + writeChar('\'', buf); +} + inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf) { writeAnyQuotedString<'"'>(s, buf); diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 77474d68795..1a8268b9b1b 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -139,7 +139,7 @@ namespace /// For example, to execute /// GRANT ALL ON mydb.* TO role1 /// REVOKE ALL ON *.* FROM role1 - /// the current user needs to have grants only on the 'mydb' database. + /// the current user needs to have the grants only on the 'mydb' database. AccessRights all_granted_access; for (const auto & id : grantees_from_query) { diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 1283879971d..85e0c5a0ae7 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -36,6 +36,7 @@ public: explicit ConcurrentHashJoin(ContextPtr context_, std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_ = false); ~ConcurrentHashJoin() override = default; + std::string getName() const override { return "ConcurrentHashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & block, bool check_limits) override; void checkTypesOfKeys(const Block & block) const override; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 24fe52099e6..e27412e9515 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3180,7 +3180,12 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptrsystem_logs = std::make_unique(getGlobalContext(), getConfigRef()); } @@ -3381,6 +3386,16 @@ std::shared_ptr Context::getAsynchronousInsertLog() const return shared->system_logs->asynchronous_insert_log; } +std::vector Context::getSystemLogs() const +{ + auto lock = getLock(); + + if (!shared->system_logs) + return {}; + + return shared->system_logs->logs; +} + CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index d6c16b69f6c..cfb04a11485 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -86,6 +86,7 @@ struct Progress; struct FileProgress; class Clusters; class QueryCache; +class ISystemLog; class QueryLog; class QueryThreadLog; class QueryViewsLog; @@ -1023,6 +1024,8 @@ public: std::shared_ptr getFilesystemReadPrefetchesLog() const; std::shared_ptr getAsynchronousInsertLog() const; + std::vector getSystemLogs() const; + /// Returns an object used to log operations with parts if it possible. /// Provide table name to make required checks. std::shared_ptr getPartLog(const String & part_database) const; diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index e55ac278705..5f664314818 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -30,6 +30,7 @@ public: std::shared_ptr storage_, const Block & right_sample_block_with_storage_column_names_); + std::string getName() const override { return "DirectKeyValueJoin"; } virtual const TableJoin & getTableJoin() const override { return *table_join; } virtual bool addBlockToJoin(const Block &, bool) override; diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index a6b53a51c04..3fc9f8920ed 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -28,6 +28,7 @@ public: LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join"); } + std::string getName() const override { return "FullSortingMergeJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & /* block */, bool /* check_limits */) override diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index ce519892b0e..44949440467 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -60,6 +60,7 @@ public: ~GraceHashJoin() override; + std::string getName() const override { return "GraceHashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } void initialize(const Block & sample_block) override; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 56dea98c1f1..9f55945816c 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -151,6 +151,7 @@ public: ~HashJoin() override; + std::string getName() const override { return "HashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } /** Add block of data from right hand of JOIN to the map. diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 97b119bd795..493a5dd2126 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -48,6 +48,8 @@ class IJoin public: virtual ~IJoin() = default; + virtual std::string getName() const = 0; + virtual const TableJoin & getTableJoin() const = 0; /// Add block of data from right hand of JOIN. diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 82fe2f57a0e..590b7fe37b8 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -295,7 +295,7 @@ BlockIO InterpreterKillQueryQuery::execute() if (res_columns[0]->empty() && access_denied) throw Exception(ErrorCodes::ACCESS_DENIED, "Not allowed to kill mutation. " - "To execute this query it's necessary to have the grant {}", required_access_rights.toString()); + "To execute this query, it's necessary to have the grant {}", required_access_rights.toString()); res_io.pipeline = QueryPipeline(Pipe(std::make_shared(header.cloneWithColumns(std::move(res_columns))))); @@ -359,7 +359,7 @@ BlockIO InterpreterKillQueryQuery::execute() if (res_columns[0]->empty() && access_denied) throw Exception(ErrorCodes::ACCESS_DENIED, "Not allowed to kill move partition. " - "To execute this query it's necessary to have the grant {}", required_access_rights.toString()); + "To execute this query, it's necessary to have the grant {}", required_access_rights.toString()); res_io.pipeline = QueryPipeline(Pipe(std::make_shared(header.cloneWithColumns(std::move(res_columns))))); diff --git a/src/Interpreters/InterpreterRenameQuery.h b/src/Interpreters/InterpreterRenameQuery.h index 25c707d9962..79a6c0ddef4 100644 --- a/src/Interpreters/InterpreterRenameQuery.h +++ b/src/Interpreters/InterpreterRenameQuery.h @@ -10,7 +10,7 @@ namespace DB class AccessRightsElements; class DDLGuard; -/// To avoid deadlocks, we must acquire locks for tables in same order in any different RENAMES. +/// To avoid deadlocks, we must acquire locks for tables in same order in any different RENAMEs. struct UniqueTableName { String database_name; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 20fca1b1e76..fb04cc77955 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -299,7 +299,7 @@ void checkAccessRightsForSelect( } throw Exception( ErrorCodes::ACCESS_DENIED, - "{}: Not enough privileges. To execute this query it's necessary to have grant SELECT for at least one column on {}", + "{}: Not enough privileges. To execute this query, it's necessary to have the grant SELECT for at least one column on {}", context->getUserName(), table_id.getFullTableName()); } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 9c77b40f803..c0341330846 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -103,40 +103,34 @@ namespace ActionLocks namespace { -ExecutionStatus getOverallExecutionStatusOfCommands() +/// Sequentially tries to execute all commands and throws exception with info about failed commands +void executeCommandsAndThrowIfError(std::vector> commands) { - return ExecutionStatus(0); -} + ExecutionStatus result(0); + for (auto & command : commands) + { + try + { + command(); + } + catch (...) + { + ExecutionStatus current_result = ExecutionStatus::fromCurrentException(); -/// Consequently tries to execute all commands and generates final exception message for failed commands -template -ExecutionStatus getOverallExecutionStatusOfCommands(Callable && command, Callables && ... commands) -{ - ExecutionStatus status_head(0); - try - { - command(); - } - catch (...) - { - status_head = ExecutionStatus::fromCurrentException(); + if (result.code == 0) + result.code = current_result.code; + + if (!current_result.message.empty()) + { + if (!result.message.empty()) + result.message += '\n'; + result.message += current_result.message; + } + } } - ExecutionStatus status_tail = getOverallExecutionStatusOfCommands(std::forward(commands)...); - - auto res_status = status_head.code != 0 ? status_head.code : status_tail.code; - auto res_message = status_head.message + (status_tail.message.empty() ? "" : ("\n" + status_tail.message)); - - return ExecutionStatus(res_status, res_message); -} - -/// Consequently tries to execute all commands and throws exception with info about failed commands -template -void executeCommandsAndThrowIfError(Callables && ... commands) -{ - auto status = getOverallExecutionStatusOfCommands(std::forward(commands)...); - if (status.code != 0) - throw Exception::createDeprecated(status.message, status.code); + if (result.code != 0) + throw Exception::createDeprecated(result.message, result.code); } @@ -425,10 +419,10 @@ BlockIO InterpreterSystemQuery::execute() case Type::RELOAD_DICTIONARIES: { getContext()->checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY); - executeCommandsAndThrowIfError( + executeCommandsAndThrowIfError({ [&] { system_context->getExternalDictionariesLoader().reloadAllTriedToLoad(); }, [&] { system_context->getEmbeddedDictionaries().reload(); } - ); + }); ExternalDictionariesLoader::resetAll(); break; } @@ -557,23 +551,14 @@ BlockIO InterpreterSystemQuery::execute() case Type::FLUSH_LOGS: { getContext()->checkAccess(AccessType::SYSTEM_FLUSH_LOGS); - executeCommandsAndThrowIfError( - [&] { if (auto query_log = getContext()->getQueryLog()) query_log->flush(true); }, - [&] { if (auto part_log = getContext()->getPartLog("")) part_log->flush(true); }, - [&] { if (auto query_thread_log = getContext()->getQueryThreadLog()) query_thread_log->flush(true); }, - [&] { if (auto trace_log = getContext()->getTraceLog()) trace_log->flush(true); }, - [&] { if (auto text_log = getContext()->getTextLog()) text_log->flush(true); }, - [&] { if (auto metric_log = getContext()->getMetricLog()) metric_log->flush(true); }, - [&] { if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) asynchronous_metric_log->flush(true); }, - [&] { if (auto opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); }, - [&] { if (auto query_views_log = getContext()->getQueryViewsLog()) query_views_log->flush(true); }, - [&] { if (auto zookeeper_log = getContext()->getZooKeeperLog()) zookeeper_log->flush(true); }, - [&] { if (auto session_log = getContext()->getSessionLog()) session_log->flush(true); }, - [&] { if (auto transactions_info_log = getContext()->getTransactionsInfoLog()) transactions_info_log->flush(true); }, - [&] { if (auto processors_profile_log = getContext()->getProcessorsProfileLog()) processors_profile_log->flush(true); }, - [&] { if (auto cache_log = getContext()->getFilesystemCacheLog()) cache_log->flush(true); }, - [&] { if (auto asynchronous_insert_log = getContext()->getAsynchronousInsertLog()) asynchronous_insert_log->flush(true); } - ); + + auto logs = getContext()->getSystemLogs(); + std::vector> commands; + commands.reserve(logs.size()); + for (auto * system_log : logs) + commands.emplace_back([system_log] { system_log->flush(true); }); + + executeCommandsAndThrowIfError(commands); break; } case Type::STOP_LISTEN: diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index fb5066b2d04..1d2ebc6b456 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -18,6 +18,7 @@ class JoinSwitcher : public IJoin public: JoinSwitcher(std::shared_ptr table_join_, const Block & right_sample_block_); + std::string getName() const override { return "JoinSwitcher"; } const TableJoin & getTableJoin() const override { return *table_join; } /// Add block of data from right hand of JOIN into current join object. diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 03a661c5b8a..98fae1d419f 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -22,6 +22,7 @@ class MergeJoin : public IJoin public: MergeJoin(std::shared_ptr table_join_, const Block & right_sample_block); + std::string getName() const override { return "PartialMergeJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & block, bool check_limits) override; void checkTypesOfKeys(const Block & block) const override; diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index de2a779b740..f8bd70afdb6 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -520,6 +520,8 @@ ContextMutablePtr Session::makeSessionContext() {}, session_context->getSettingsRef().max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -582,6 +584,8 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -655,24 +659,38 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - if (!notified_session_log_about_login) - { - if (auto session_log = getSessionLog()) - { - session_log->addLoginSuccess( - auth_id, - named_session ? std::optional(named_session->key.second) : std::nullopt, - *query_context, - user); - - notified_session_log_about_login = true; - } - } + /// Interserver does not create session context + recordLoginSucess(query_context); return query_context; } +void Session::recordLoginSucess(ContextPtr login_context) const +{ + if (notified_session_log_about_login) + return; + + if (!login_context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session or query context must be created"); + + if (auto session_log = getSessionLog()) + { + const auto & settings = login_context->getSettingsRef(); + const auto access = login_context->getAccess(); + + session_log->addLoginSuccess(auth_id, + named_session ? named_session->key.second : "", + settings, + access, + getClientInfo(), + user); + } + + notified_session_log_about_login = true; +} + + void Session::releaseSessionID() { if (!named_session) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 51c0e3c71fa..81ef987b428 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,6 +97,8 @@ public: private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; + void recordLoginSucess(ContextPtr login_context) const; + mutable bool notified_session_log_about_login = false; const UUID auth_id; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 0a8a7fc18c5..b847eaf9824 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -199,12 +199,13 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length()); } -void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user) +void SessionLog::addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user) { - const auto access = login_context.getAccess(); - const auto & settings = login_context.getSettingsRef(); - const auto & client_info = login_context.getClientInfo(); - DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; @@ -215,8 +216,7 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional ses } log_entry.external_auth_server = login_user ? login_user->auth_data.getLDAPServerName() : ""; - if (session_id) - log_entry.session_id = *session_id; + log_entry.session_id = session_id; if (const auto roles_info = access->getRolesInfo()) log_entry.roles = roles_info->getCurrentRolesNames(); diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 1282ac09c4d..8757bc12270 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -20,6 +20,7 @@ enum SessionLogElementType : int8_t class ContextAccess; struct User; using UserPtr = std::shared_ptr; +using ContextAccessPtr = std::shared_ptr; /** A struct which will be inserted as row into session_log table. * @@ -72,7 +73,13 @@ class SessionLog : public SystemLog using SystemLog::SystemLog; public: - void addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user); + void addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user); + void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const std::optional & user, const Exception & reason); void addLogOut(const UUID & auth_id, const UserPtr & login_user, const ClientInfo & client_info); }; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 12f3b9744cb..23a5a96584c 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -129,6 +129,7 @@ std::shared_ptr createSystemLog( "Creating {}.{} from {}", default_database_name, default_table_name, config_prefix); SystemLogSettings log_settings; + log_settings.queue_settings.database = config.getString(config_prefix + ".database", default_database_name); log_settings.queue_settings.table = config.getString(config_prefix + ".table", default_table_name); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 16e641b2718..af3b027b2db 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -330,7 +330,7 @@ public: const ColumnsWithTypeAndName & right_sample_columns); void setAsofInequality(ASOFJoinInequality inequality) { asof_inequality = inequality; } - ASOFJoinInequality getAsofInequality() { return asof_inequality; } + ASOFJoinInequality getAsofInequality() const { return asof_inequality; } ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index 5c76f6f33bf..4a9a3d8df5b 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -93,7 +93,7 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const { - /// 100 - just arbitrary value. + /// 100 - just arbitrary value. constexpr auto min_elements_for_hashing = 100; /// Special case for very large arrays. Instead of listing all elements, will use hash of them. @@ -118,9 +118,31 @@ void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const } } +/// Use different rules for escaping backslashes and quotes +class FieldVisitorToStringPostgreSQL : public StaticVisitor +{ +public: + template + String operator() (const T & x) const { return visitor(x); } + +private: + FieldVisitorToString visitor; +}; + +template<> +String FieldVisitorToStringPostgreSQL::operator() (const String & x) const +{ + WriteBufferFromOwnString wb; + writeQuotedStringPostgreSQL(x, wb); + return wb.str(); +} + void ASTLiteral::formatImplWithoutAlias(const FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << applyVisitor(FieldVisitorToString(), value); + if (settings.literal_escaping_style == LiteralEscapingStyle::Regular) + settings.ostr << applyVisitor(FieldVisitorToString(), value); + else + settings.ostr << applyVisitor(FieldVisitorToStringPostgreSQL(), value); } } diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index 0cfdc3762a1..90d9ede7337 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -73,11 +73,11 @@ void ASTProjectionSelectQuery::formatImpl(const FormatSettings & s, FormatState if (orderBy()) { - /// Let's convert the ASTFunction into ASTExpressionList, which generates consistent format + /// Let's convert tuple ASTFunction into ASTExpressionList, which generates consistent format /// between GROUP BY and ORDER BY projection definition. s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : ""); ASTPtr order_by; - if (auto * func = orderBy()->as()) + if (auto * func = orderBy()->as(); func && func->name == "tuple") order_by = func->arguments; else { diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 2b097025b07..80a05598adc 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -197,6 +198,7 @@ public: IdentifierQuotingStyle identifier_quoting_style; bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. + LiteralEscapingStyle literal_escaping_style; explicit FormatSettings( WriteBuffer & ostr_, @@ -204,7 +206,8 @@ public: bool hilite_ = false, bool always_quote_identifiers_ = false, IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks, - bool show_secrets_ = true) + bool show_secrets_ = true, + LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular) : ostr(ostr_) , one_line(one_line_) , hilite(hilite_) @@ -212,6 +215,7 @@ public: , identifier_quoting_style(identifier_quoting_style_) , show_secrets(show_secrets_) , nl_or_ws(one_line ? ' ' : '\n') + , literal_escaping_style(literal_escaping_style_) { } @@ -223,6 +227,7 @@ public: , identifier_quoting_style(other.identifier_quoting_style) , show_secrets(other.show_secrets) , nl_or_ws(other.nl_or_ws) + , literal_escaping_style(other.literal_escaping_style) { } diff --git a/src/Parsers/LiteralEscapingStyle.h b/src/Parsers/LiteralEscapingStyle.h new file mode 100644 index 00000000000..10d4d84a85d --- /dev/null +++ b/src/Parsers/LiteralEscapingStyle.h @@ -0,0 +1,14 @@ +#pragma once + + +namespace DB +{ + +/// Method to escape single quotes. +enum class LiteralEscapingStyle +{ + Regular, /// Escape backslashes with backslash (\\) and quotes with backslash (\') + PostgreSQL, /// Do not escape backslashes (\), escape quotes with quote ('') +}; + +} diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index e411574bd65..261f234081c 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -4,7 +4,7 @@ clickhouse_add_executable(lexer lexer.cpp ${SRCS}) target_link_libraries(lexer PRIVATE clickhouse_parsers) clickhouse_add_executable(select_parser select_parser.cpp ${SRCS} "../../Server/ServerType.cpp") -target_link_libraries(select_parser PRIVATE clickhouse_parsers) +target_link_libraries(select_parser PRIVATE dbms) clickhouse_add_executable(create_parser create_parser.cpp ${SRCS} "../../Server/ServerType.cpp") -target_link_libraries(create_parser PRIVATE clickhouse_parsers) +target_link_libraries(create_parser PRIVATE dbms) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c118fccded4..56a48ce8328 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -101,7 +101,7 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names, } throw Exception(ErrorCodes::ACCESS_DENIED, - "{}: Not enough privileges. To execute this query it's necessary to have grant SELECT for at least one column on {}", + "{}: Not enough privileges. To execute this query, it's necessary to have the grant SELECT for at least one column on {}", query_context->getUserName(), storage_id.getFullTableName()); } diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 33fa7955e0d..63a5eeb51d2 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -2,6 +2,9 @@ #include #include #include +#include +#include +#include #include namespace DB @@ -62,6 +65,36 @@ void JoinStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } +void JoinStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + + const auto & table_join = join->getTableJoin(); + settings.out << prefix << "Type: " << toString(table_join.kind()) << '\n'; + settings.out << prefix << "Strictness: " << toString(table_join.strictness()) << '\n'; + settings.out << prefix << "Algorithm: " << join->getName() << '\n'; + + if (table_join.strictness() == JoinStrictness::Asof) + settings.out << prefix << "ASOF inequality: " << toString(table_join.getAsofInequality()) << '\n'; + + if (!table_join.getClauses().empty()) + settings.out << prefix << "Clauses: " << table_join.formatClauses(table_join.getClauses(), true /*short_format*/) << '\n'; +} + +void JoinStep::describeActions(JSONBuilder::JSONMap & map) const +{ + const auto & table_join = join->getTableJoin(); + map.add("Type", toString(table_join.kind())); + map.add("Strictness", toString(table_join.strictness())); + map.add("Algorithm", join->getName()); + + if (table_join.strictness() == JoinStrictness::Asof) + map.add("ASOF inequality", toString(table_join.getAsofInequality())); + + if (!table_join.getClauses().empty()) + map.add("Clauses", table_join.formatClauses(table_join.getClauses(), true /*short_format*/)); +} + void JoinStep::updateInputStream(const DataStream & new_input_stream_, size_t idx) { if (idx == 0) diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index e7185f36588..369ee9bec8b 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -27,6 +27,9 @@ public: void describePipeline(FormatSettings & settings) const override; + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + const JoinPtr & getJoin() const { return join; } bool allowPushDownToRight() const; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index ff5690a3b07..10d5e7a0242 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -561,8 +561,7 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } - auto client_info = session->getClientInfo(); - auto context = session->makeQueryContext(std::move(client_info)); + auto context = session->makeQueryContext(); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. if (params.has("client_protocol_version")) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0cfcd815cce..db0a7b34d7e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3166,6 +3166,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } } + if (command.type == AlterCommand::MODIFY_QUERY) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "ALTER MODIFY QUERY is not supported by MergeTree engines family"); + if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned) { throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 5efb7286685..466dbb45a4d 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -226,7 +226,7 @@ static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet return false; } -void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const +void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context, std::set & pk_positions) const { auto function_node_optional = node.toFunctionNodeOrNull(); @@ -237,7 +237,7 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree for (size_t i = 0; i < arguments_size; ++i) { auto argument = function_node_optional->getArgumentAt(i); - analyzeImpl(res, argument, where_optimizer_context); + analyzeImpl(res, argument, where_optimizer_context, pk_positions); } } else @@ -270,6 +270,7 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree cond.good = cond.viable; /// Find min position in PK of any column that is used in this condition. cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + pk_positions.emplace(cond.min_position_in_primary_key); } res.emplace_back(std::move(cond)); @@ -281,7 +282,29 @@ MergeTreeWhereOptimizer::Conditions MergeTreeWhereOptimizer::analyze(const RPNBu const WhereOptimizerContext & where_optimizer_context) const { Conditions res; - analyzeImpl(res, node, where_optimizer_context); + std::set pk_positions; + analyzeImpl(res, node, where_optimizer_context, pk_positions); + + /// E.g., if the primary key is (a, b, c) but the condition is a = 1 and c = 1, + /// we should only put (a = 1) to the tail of PREWHERE, + /// and treat (c = 1) as a normal column. + if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) + { + Int64 min_valid_pk_pos = -1; + for (auto pk_pos : pk_positions) + { + if (pk_pos != min_valid_pk_pos + 1) + break; + min_valid_pk_pos = pk_pos; + } + for (auto & cond : res) + { + if (cond.min_position_in_primary_key > min_valid_pk_pos) + cond.min_position_in_primary_key = std::numeric_limits::max() - 1; + } + LOG_TRACE(log, "The min valid primary key position for moving to the tail of PREWHERE is {}", min_valid_pk_pos); + } + return res; } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index fb5e84b67c6..dd9dc803f35 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -108,7 +108,7 @@ private: std::optional optimizeImpl(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const; - void analyzeImpl(Conditions & res, const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const; + void analyzeImpl(Conditions & res, const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context, std::set & pk_positions) const; /// Transform conjunctions chain in WHERE expression to Conditions list. Conditions analyze(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 9c05afd5284..eb154d0d943 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -996,8 +996,11 @@ void StorageBuffer::reschedule() std::unique_lock lock(buffer.tryLock()); if (lock.owns_lock()) { - min_first_write_time = buffer.first_write_time; - rows += buffer.data.rows(); + if (buffer.data) + { + min_first_write_time = std::min(min_first_write_time, buffer.first_write_time); + rows += buffer.data.rows(); + } } } diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index b0a220eb1d2..76a439eabaf 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -104,6 +104,7 @@ Pipe StorageMySQL::read( column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::BackticksMySQL, + LiteralEscapingStyle::Regular, remote_database_name, remote_table_name, context_); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 3551ee36819..7961c44e844 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -122,7 +122,7 @@ Pipe StoragePostgreSQL::read( query_info_, column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), - IdentifierQuotingStyle::DoubleQuotes, remote_table_schema, remote_table_name, context_); + IdentifierQuotingStyle::DoubleQuotes, LiteralEscapingStyle::PostgreSQL, remote_table_schema, remote_table_name, context_); LOG_TRACE(log, "Query: {}", query); Block sample_block; diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index d5ae6f2383f..d5db5763da9 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -91,6 +91,7 @@ Pipe StorageSQLite::read( column_names, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "", remote_table_name, context_); diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index b532d1c91f0..1715cde9d1e 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -79,6 +79,7 @@ std::function StorageXDBC::getReadPOSTDataCallback( column_names, columns_description.getOrdinary(), bridge_helper->getIdentifierQuotingStyle(), + LiteralEscapingStyle::Regular, remote_database_name, remote_table_name, local_context); diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 5c1442ece11..749a154c19d 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -127,7 +127,8 @@ static void checkOld( std::string transformed_query = transformQueryForExternalDatabase( query_info, query_info.syntax_analyzer_result->requiredSourceColumns(), - state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "test", "table", state.context); EXPECT_EQ(transformed_query, expected) << query; } @@ -180,7 +181,8 @@ static void checkNewAnalyzer( query_info.table_expression = findTableExpression(query_node->getJoinTree(), "table"); std::string transformed_query = transformQueryForExternalDatabase( - query_info, column_names, state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + query_info, column_names, state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "test", "table", state.context); EXPECT_EQ(transformed_query, expected) << query; } diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 375510e62bf..84a696a1e9c 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -258,6 +258,7 @@ String transformQueryForExternalDatabaseImpl( Names used_columns, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context) @@ -337,7 +338,8 @@ String transformQueryForExternalDatabaseImpl( IAST::FormatSettings settings( out, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ identifier_quoting_style != IdentifierQuotingStyle::None, - /*identifier_quoting_style*/ identifier_quoting_style); + /*identifier_quoting_style*/ identifier_quoting_style, /*show_secrets_*/ true, + /*literal_escaping_style*/ literal_escaping_style); select->format(settings); @@ -351,6 +353,7 @@ String transformQueryForExternalDatabase( const Names & column_names, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context) @@ -375,6 +378,7 @@ String transformQueryForExternalDatabase( column_names, available_columns, identifier_quoting_style, + literal_escaping_style, database, table, context); @@ -386,6 +390,7 @@ String transformQueryForExternalDatabase( query_info.syntax_analyzer_result->requiredSourceColumns(), available_columns, identifier_quoting_style, + literal_escaping_style, database, table, context); diff --git a/src/Storages/transformQueryForExternalDatabase.h b/src/Storages/transformQueryForExternalDatabase.h index 0f2b0a5822f..fb6af21907e 100644 --- a/src/Storages/transformQueryForExternalDatabase.h +++ b/src/Storages/transformQueryForExternalDatabase.h @@ -31,6 +31,7 @@ String transformQueryForExternalDatabase( const Names & column_names, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context); diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 514aaf7e2ac..74b875c23be 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -79,7 +79,7 @@ def main(): build_url = url break else: - raise Exception("Cannot binary clickhouse among build results") + raise Exception("Cannot find the clickhouse binary among build results") logging.info("Got build url %s", build_url) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 979f7c35c2c..a2f3a3ab401 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -177,6 +177,11 @@ CI_CONFIG = CiConfig( package_type="binary", static_binary_name="riscv64", ), + "binary_s390x": BuildConfig( + compiler="clang-16-s390x", + package_type="binary", + static_binary_name="s390x", + ), }, builds_report_config={ "ClickHouse build check": [ @@ -198,6 +203,7 @@ CI_CONFIG = CiConfig( "binary_darwin_aarch64", "binary_ppc64le", "binary_riscv64", + "binary_s390x", "binary_amd64_compat", ], }, @@ -270,6 +276,7 @@ CI_CONFIG = CiConfig( "SQLancer (release)": TestConfig("package_release"), "SQLancer (debug)": TestConfig("package_debug"), "Sqllogic test (release)": TestConfig("package_release"), + "SQLTest": TestConfig("package_release"), }, ) CI_CONFIG.validate() @@ -283,96 +290,6 @@ REQUIRED_CHECKS = [ "Fast test", "Stateful tests (release)", "Stateless tests (release)", - "Stateless tests (debug) [1/5]", - "Stateless tests (debug) [2/5]", - "Stateless tests (debug) [3/5]", - "Stateless tests (debug) [4/5]", - "Stateless tests (debug) [5/5]", - "AST fuzzer (asan)", - "AST fuzzer (msan)", - "AST fuzzer (tsan)", - "AST fuzzer (ubsan)", - "AST fuzzer (debug)", - "Compatibility check (aarch64)", - "Compatibility check (amd64)", - "Install packages (amd64)", - "Install packages (arm64)", - "Integration tests (asan) [1/6]", - "Integration tests (asan) [2/6]", - "Integration tests (asan) [3/6]", - "Integration tests (asan) [4/6]", - "Integration tests (asan) [5/6]", - "Integration tests (asan) [6/6]", - "Integration tests (release) [1/4]", - "Integration tests (release) [2/4]", - "Integration tests (release) [3/4]", - "Integration tests (release) [4/4]", - "Integration tests (tsan) [1/6]", - "Integration tests (tsan) [2/6]", - "Integration tests (tsan) [3/6]", - "Integration tests (tsan) [4/6]", - "Integration tests (tsan) [5/6]", - "Integration tests (tsan) [6/6]", - "Integration tests flaky check (asan)", - "Stateful tests (aarch64)", - "Stateful tests (asan)", - "Stateful tests (asan, ParallelReplicas)", - "Stateful tests (debug)", - "Stateful tests (debug, ParallelReplicas)", - "Stateful tests (msan)", - "Stateful tests (msan, ParallelReplicas)", - "Stateful tests (release, ParallelReplicas)", - "Stateful tests (tsan)", - "Stateful tests (tsan, ParallelReplicas)", - "Stateful tests (ubsan)", - "Stateful tests (ubsan, ParallelReplicas)", - "Stateless tests (aarch64)", - "Stateless tests (asan) [1/4]", - "Stateless tests (asan) [2/4]", - "Stateless tests (asan) [3/4]", - "Stateless tests (asan) [4/4]", - "Stateless tests (debug) [1/5]", - "Stateless tests (debug) [2/5]", - "Stateless tests (debug) [3/5]", - "Stateless tests (debug) [4/5]", - "Stateless tests (debug) [5/5]", - "Stateless tests (debug, s3 storage) [1/6]", - "Stateless tests (debug, s3 storage) [2/6]", - "Stateless tests (debug, s3 storage) [3/6]", - "Stateless tests (debug, s3 storage) [4/6]", - "Stateless tests (debug, s3 storage) [5/6]", - "Stateless tests (debug, s3 storage) [6/6]", - "Stateless tests (msan) [1/6]", - "Stateless tests (msan) [2/6]", - "Stateless tests (msan) [3/6]", - "Stateless tests (msan) [4/6]", - "Stateless tests (msan) [5/6]", - "Stateless tests (msan) [6/6]", - "Stateless tests (release, DatabaseReplicated) [1/4]", - "Stateless tests (release, DatabaseReplicated) [2/4]", - "Stateless tests (release, DatabaseReplicated) [3/4]", - "Stateless tests (release, DatabaseReplicated) [4/4]", - "Stateless tests (release, s3 storage) [1/2]", - "Stateless tests (release, s3 storage) [2/2]", - "Stateless tests (release, wide parts enabled)", - "Stateless tests (tsan) [1/5]", - "Stateless tests (tsan) [2/5]", - "Stateless tests (tsan) [3/5]", - "Stateless tests (tsan) [4/5]", - "Stateless tests (tsan) [5/5]", - "Stateless tests (tsan, s3 storage) [1/5]", - "Stateless tests (tsan, s3 storage) [2/5]", - "Stateless tests (tsan, s3 storage) [3/5]", - "Stateless tests (tsan, s3 storage) [4/5]", - "Stateless tests (tsan, s3 storage) [5/5]", - "Stateless tests (ubsan) [1/2]", - "Stateless tests (ubsan) [2/2]", - "Stress test (asan)", - "Stress test (debug)", - "Stress test (msan)", - "Stress test (tsan)", - "Stress test (ubsan)", - "Upgrade check (asan)", "Style Check", "Unit tests (asan)", "Unit tests (msan)", diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index b165cda37e8..39132549bac 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -87,7 +87,7 @@ class ClickHouseHelper: ) if response.status_code >= 500: - # A retriable error + # A retryable error time.sleep(1) continue diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 1cbd4cf9222..14db348ef36 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -40,6 +40,12 @@ class TestDockerImageCheck(unittest.TestCase): [ di.DockerImage("docker/test/base", "clickhouse/test-base", False), di.DockerImage("docker/docs/builder", "clickhouse/docs-builder", True), + di.DockerImage( + "docker/test/sqltest", + "clickhouse/sqltest", + False, + "clickhouse/test-base", # type: ignore + ), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index b773d1eddd9..d06da94d0f0 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -17,6 +17,7 @@ from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, prepare_tests_results_for_clickhouse, + get_instance_type, ) from commit_status_helper import ( NotSet, @@ -73,9 +74,11 @@ def get_image_name(check_name): def get_run_command( + pr_info, + check_start_time, check_name, builds_path, - repo_tests_path, + repo_path, result_path, server_log_path, kill_timeout, @@ -105,16 +108,28 @@ def get_run_command( envs += [f"-e {e}" for e in additional_envs] + instance_type = get_instance_type() + + envs += [ + "-e CLICKHOUSE_CI_LOGS_HOST", + "-e CLICKHOUSE_CI_LOGS_PASSWORD", + f"-e PULL_REQUEST_NUMBER='{pr_info.number}'", + f"-e COMMIT_SHA='{pr_info.sha}'", + f"-e CHECK_START_TIME='{check_start_time}'", + f"-e CHECK_NAME='{check_name}'", + f"-e INSTANCE_TYPE='{instance_type}'", + ] + env_str = " ".join(envs) volume_with_broken_test = ( - f"--volume={repo_tests_path}/analyzer_tech_debt.txt:/analyzer_tech_debt.txt" + f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt" if "analyzer" in check_name else "" ) return ( f"docker run --volume={builds_path}:/package_folder " - f"--volume={repo_tests_path}:/usr/share/clickhouse-test " + f"--volume={repo_path}/tests:/usr/share/clickhouse-test " f"{volume_with_broken_test} " f"--volume={result_path}:/test_output --volume={server_log_path}:/var/log/clickhouse-server " f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" @@ -306,8 +321,6 @@ def main(): image_name = get_image_name(check_name) docker_image = get_image_with_version(reports_path, image_name) - repo_tests_path = os.path.join(repo_path, "tests") - packages_path = os.path.join(temp_path, "packages") if not os.path.exists(packages_path): os.makedirs(packages_path) @@ -334,9 +347,11 @@ def main(): additional_envs.append("GLOBAL_TAGS=no-random-settings") run_command = get_run_command( + pr_info, + stopwatch.start_time_str, check_name, packages_path, - repo_tests_path, + repo_path, result_path, server_log_path, kill_timeout, @@ -368,6 +383,22 @@ def main(): ch_helper = ClickHouseHelper() + # Cleanup run log from the credentials of CI logs database. + # Note: a malicious user can still print them by splitting the value into parts. + # But we will be warned when a malicious user modifies CI script. + # Although they can also print them from inside tests. + # Nevertheless, the credentials of the CI logs have limited scope + # and does not provide access to sensitive info. + + ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST", "CLICKHOUSE_CI_LOGS_HOST") + ci_logs_password = os.getenv( + "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" + ) + subprocess.check_call( + f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", + shell=True, + ) + report_url = upload_results( s3_helper, pr_info.number, diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py new file mode 100644 index 00000000000..be22a1c9312 --- /dev/null +++ b/tests/ci/sqltest.py @@ -0,0 +1,155 @@ +#!/usr/bin/env python3 + +import logging +import subprocess +import os +import sys + +from github import Github + +from build_download_helper import get_build_name_for_check, read_build_urls +from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse +from commit_status_helper import ( + RerunHelper, + get_commit, + post_commit_status, +) +from docker_pull_helper import get_image_with_version +from env_helper import ( + GITHUB_RUN_URL, + REPORTS_PATH, + TEMP_PATH, +) +from get_robot_token import get_best_robot_token +from pr_info import PRInfo +from report import TestResult +from s3_helper import S3Helper +from stopwatch import Stopwatch + +IMAGE_NAME = "clickhouse/sqltest" + + +def get_run_command(pr_number, sha, download_url, workspace_path, image): + return ( + f"docker run " + # For sysctl + "--privileged " + "--network=host " + f"--volume={workspace_path}:/workspace " + "--cap-add syslog --cap-add sys_admin --cap-add=SYS_PTRACE " + f'-e PR_TO_TEST={pr_number} -e SHA_TO_TEST={sha} -e BINARY_URL_TO_DOWNLOAD="{download_url}" ' + f"{image}" + ) + + +def main(): + logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + + temp_path = TEMP_PATH + reports_path = REPORTS_PATH + + check_name = sys.argv[1] + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + pr_info = PRInfo() + + gh = Github(get_best_robot_token(), per_page=100) + commit = get_commit(gh, pr_info.sha) + + rerun_helper = RerunHelper(commit, check_name) + if rerun_helper.is_already_finished_by_status(): + logging.info("Check is already finished according to github status, exiting") + sys.exit(0) + + docker_image = get_image_with_version(reports_path, IMAGE_NAME) + + build_name = get_build_name_for_check(check_name) + print(build_name) + urls = read_build_urls(build_name, reports_path) + if not urls: + raise Exception("No build URLs found") + + for url in urls: + if url.endswith("/clickhouse"): + build_url = url + break + else: + raise Exception("Cannot find the clickhouse binary among build results") + + logging.info("Got build url %s", build_url) + + workspace_path = os.path.join(temp_path, "workspace") + if not os.path.exists(workspace_path): + os.makedirs(workspace_path) + + run_command = get_run_command( + pr_info.number, pr_info.sha, build_url, workspace_path, docker_image + ) + logging.info("Going to run %s", run_command) + + run_log_path = os.path.join(temp_path, "run.log") + with open(run_log_path, "w", encoding="utf-8") as log: + with subprocess.Popen( + run_command, shell=True, stderr=log, stdout=log + ) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") + + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + + check_name_lower = ( + check_name.lower().replace("(", "").replace(")", "").replace(" ", "") + ) + s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" + paths = { + "run.log": run_log_path, + "server.log.zst": os.path.join(workspace_path, "server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), + "report.html": os.path.join(workspace_path, "report.html"), + "test.log": os.path.join(workspace_path, "test.log"), + } + + s3_helper = S3Helper() + for f in paths: + try: + paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + f) + except Exception as ex: + logging.info("Exception uploading file %s text %s", f, ex) + paths[f] = "" + + report_url = GITHUB_RUN_URL + if paths["report.html"]: + report_url = paths["report.html"] + + status = "success" + description = "See the report" + test_result = TestResult(description, "OK") + + ch_helper = ClickHouseHelper() + + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + [test_result], + status, + stopwatch.duration_seconds, + stopwatch.start_time_str, + report_url, + check_name, + ) + + ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) + + logging.info("Result: '%s', '%s', '%s'", status, description, report_url) + print(f"::notice ::Report url: {report_url}") + post_commit_status(commit, status, report_url, description, check_name, pr_info) + + +if __name__ == "__main__": + main() diff --git a/tests/ci/stress.py b/tests/ci/stress.py index eb829cf519c..2c566144f2c 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -302,7 +302,7 @@ if __name__ == "__main__": have_long_running_queries = prepare_for_hung_check(args.drop_databases) except Exception as ex: have_long_running_queries = True - logging.error("Failed to prepare for hung check %s", str(ex)) + logging.error("Failed to prepare for hung check: %s", str(ex)) logging.info("Checking if some queries hung") cmd = " ".join( [ diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 895eb318bc4..42d372efb5d 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -14,6 +14,7 @@ from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, prepare_tests_results_for_clickhouse, + get_instance_type, ) from commit_status_helper import RerunHelper, get_commit, post_commit_status from docker_pull_helper import get_image_with_version @@ -28,12 +29,34 @@ from upload_result_helper import upload_results def get_run_command( - build_path, result_folder, repo_tests_path, server_log_folder, image + pr_info, + check_start_time, + check_name, + build_path, + result_folder, + repo_tests_path, + server_log_folder, + image, ): + instance_type = get_instance_type() + + envs = [ + # a static link, don't use S3_URL or S3_DOWNLOAD + "-e S3_URL='https://s3.amazonaws.com/clickhouse-datasets'", + "-e CLICKHOUSE_CI_LOGS_HOST", + "-e CLICKHOUSE_CI_LOGS_PASSWORD", + f"-e PULL_REQUEST_NUMBER='{pr_info.number}'", + f"-e COMMIT_SHA='{pr_info.sha}'", + f"-e CHECK_START_TIME='{check_start_time}'", + f"-e CHECK_NAME='{check_name}'", + f"-e INSTANCE_TYPE='{instance_type}'", + ] + + env_str = " ".join(envs) + cmd = ( "docker run --cap-add=SYS_PTRACE " - # a static link, don't use S3_URL or S3_DOWNLOAD - "-e S3_URL='https://s3.amazonaws.com/clickhouse-datasets' " + f"{env_str} " # For dmesg and sysctl "--privileged " f"--volume={build_path}:/package_folder " @@ -149,9 +172,16 @@ def run_stress_test(docker_image_name): run_log_path = os.path.join(temp_path, "run.log") run_command = get_run_command( - packages_path, result_path, repo_tests_path, server_log_path, docker_image + pr_info, + stopwatch.start_time_str, + check_name, + packages_path, + result_path, + repo_tests_path, + server_log_path, + docker_image, ) - logging.info("Going to run func tests: %s", run_command) + logging.info("Going to run stress test: %s", run_command) with TeePopen(run_command, run_log_path, timeout=60 * 150) as process: retcode = process.wait() @@ -168,6 +198,22 @@ def run_stress_test(docker_image_name): ) ch_helper = ClickHouseHelper() + # Cleanup run log from the credentials of CI logs database. + # Note: a malicious user can still print them by splitting the value into parts. + # But we will be warned when a malicious user modifies CI script. + # Although they can also print them from inside tests. + # Nevertheless, the credentials of the CI logs have limited scope + # and does not provide access to sensitive info. + + ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST", "CLICKHOUSE_CI_LOGS_HOST") + ci_logs_password = os.getenv( + "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" + ) + subprocess.check_call( + f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", + shell=True, + ) + report_url = upload_results( s3_helper, pr_info.number, diff --git a/tests/ci/tests/docker_images_for_tests.json b/tests/ci/tests/docker_images_for_tests.json index 0d40d43c33f..70db8760561 100644 --- a/tests/ci/tests/docker_images_for_tests.json +++ b/tests/ci/tests/docker_images_for_tests.json @@ -119,7 +119,8 @@ "docker/test/stateless", "docker/test/integration/base", "docker/test/fuzzer", - "docker/test/keeper-jepsen" + "docker/test/keeper-jepsen", + "docker/test/sqltest" ] }, "docker/test/integration/kerberized_hadoop": { @@ -153,5 +154,9 @@ "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/sqltest": { + "name": "clickhouse/sqltest", + "dependent": [] } } diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index be4d71502d2..5069468110c 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -22,7 +22,7 @@ def test_access_rights_for_function(): instance.query("CREATE USER A") instance.query("CREATE USER B") assert ( - "it's necessary to have grant CREATE FUNCTION ON *.*" + "it's necessary to have the grant CREATE FUNCTION ON *.*" in instance.query_and_get_error(create_function_query, user="A") ) @@ -32,7 +32,7 @@ def test_access_rights_for_function(): assert instance.query("SELECT MySum(1, 2)") == "3\n" assert ( - "it's necessary to have grant DROP FUNCTION ON *.*" + "it's necessary to have the grant DROP FUNCTION ON *.*" in instance.query_and_get_error("DROP FUNCTION MySum", user="B") ) @@ -44,7 +44,7 @@ def test_access_rights_for_function(): instance.query("REVOKE CREATE FUNCTION ON *.* FROM A") assert ( - "it's necessary to have grant CREATE FUNCTION ON *.*" + "it's necessary to have the grant CREATE FUNCTION ON *.*" in instance.query_and_get_error(create_function_query, user="A") ) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index c19cca4126a..a6ba0c8918f 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -882,7 +882,7 @@ def test_required_privileges(): instance.query("CREATE USER u1") backup_name = new_backup_name() - expected_error = "necessary to have grant BACKUP ON test.table" + expected_error = "necessary to have the grant BACKUP ON test.table" assert expected_error in instance.query_and_get_error( f"BACKUP TABLE test.table TO {backup_name}", user="u1" ) @@ -890,12 +890,12 @@ def test_required_privileges(): instance.query("GRANT BACKUP ON test.table TO u1") instance.query(f"BACKUP TABLE test.table TO {backup_name}", user="u1") - expected_error = "necessary to have grant INSERT, CREATE TABLE ON test.table" + expected_error = "necessary to have the grant INSERT, CREATE TABLE ON test.table" assert expected_error in instance.query_and_get_error( f"RESTORE TABLE test.table FROM {backup_name}", user="u1" ) - expected_error = "necessary to have grant INSERT, CREATE TABLE ON test.table2" + expected_error = "necessary to have the grant INSERT, CREATE TABLE ON test.table2" assert expected_error in instance.query_and_get_error( f"RESTORE TABLE test.table AS test.table2 FROM {backup_name}", user="u1" ) @@ -907,7 +907,7 @@ def test_required_privileges(): instance.query("DROP TABLE test.table") - expected_error = "necessary to have grant INSERT, CREATE TABLE ON test.table" + expected_error = "necessary to have the grant INSERT, CREATE TABLE ON test.table" assert expected_error in instance.query_and_get_error( f"RESTORE ALL FROM {backup_name}", user="u1" ) @@ -1014,14 +1014,14 @@ def test_system_users_required_privileges(): backup_name = new_backup_name() - expected_error = "necessary to have grant BACKUP ON system.users" + expected_error = "necessary to have the grant BACKUP ON system.users" assert expected_error in instance.query_and_get_error( f"BACKUP TABLE system.users, TABLE system.roles TO {backup_name}", user="u2" ) instance.query("GRANT BACKUP ON system.users TO u2") - expected_error = "necessary to have grant BACKUP ON system.roles" + expected_error = "necessary to have the grant BACKUP ON system.roles" assert expected_error in instance.query_and_get_error( f"BACKUP TABLE system.users, TABLE system.roles TO {backup_name}", user="u2" ) @@ -1035,7 +1035,7 @@ def test_system_users_required_privileges(): instance.query("DROP ROLE r1") expected_error = ( - "necessary to have grant CREATE USER, CREATE ROLE, ROLE ADMIN ON *.*" + "necessary to have the grant CREATE USER, CREATE ROLE, ROLE ADMIN ON *.*" ) assert expected_error in instance.query_and_get_error( f"RESTORE ALL FROM {backup_name}", user="u2" @@ -1043,7 +1043,7 @@ def test_system_users_required_privileges(): instance.query("GRANT CREATE USER, CREATE ROLE, ROLE ADMIN ON *.* TO u2") - expected_error = "necessary to have grant SELECT ON test.* WITH GRANT OPTION" + expected_error = "necessary to have the grant SELECT ON test.* WITH GRANT OPTION" assert expected_error in instance.query_and_get_error( f"RESTORE ALL FROM {backup_name}", user="u2" ) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 39496b8a5c8..a9ebbeb66c2 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -561,7 +561,7 @@ def test_required_privileges(): node1.query("GRANT CLUSTER ON *.* TO u1") backup_name = new_backup_name() - expected_error = "necessary to have grant BACKUP ON default.tbl" + expected_error = "necessary to have the grant BACKUP ON default.tbl" assert expected_error in node1.query_and_get_error( f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}", user="u1" ) @@ -571,7 +571,7 @@ def test_required_privileges(): node1.query(f"DROP TABLE tbl ON CLUSTER 'cluster' SYNC") - expected_error = "necessary to have grant INSERT, CREATE TABLE ON default.tbl2" + expected_error = "necessary to have the grant INSERT, CREATE TABLE ON default.tbl2" assert expected_error in node1.query_and_get_error( f"RESTORE TABLE tbl AS tbl2 ON CLUSTER 'cluster' FROM {backup_name}", user="u1" ) @@ -587,7 +587,7 @@ def test_required_privileges(): node1.query(f"DROP TABLE tbl2 ON CLUSTER 'cluster' SYNC") node1.query("REVOKE ALL FROM u1") - expected_error = "necessary to have grant INSERT, CREATE TABLE ON default.tbl" + expected_error = "necessary to have the grant INSERT, CREATE TABLE ON default.tbl" assert expected_error in node1.query_and_get_error( f"RESTORE ALL ON CLUSTER 'cluster' FROM {backup_name}", user="u1" ) @@ -607,7 +607,7 @@ def test_system_users(): node1.query("CREATE USER u2 SETTINGS allow_backup=false") node1.query("GRANT CLUSTER ON *.* TO u2") - expected_error = "necessary to have grant BACKUP ON system.users" + expected_error = "necessary to have the grant BACKUP ON system.users" assert expected_error in node1.query_and_get_error( f"BACKUP TABLE system.users ON CLUSTER 'cluster' TO {backup_name}", user="u2" ) @@ -619,14 +619,16 @@ def test_system_users(): node1.query("DROP USER u1") - expected_error = "necessary to have grant CREATE USER ON *.*" + expected_error = "necessary to have the grant CREATE USER ON *.*" assert expected_error in node1.query_and_get_error( f"RESTORE TABLE system.users ON CLUSTER 'cluster' FROM {backup_name}", user="u2" ) node1.query("GRANT CREATE USER ON *.* TO u2") - expected_error = "necessary to have grant SELECT ON default.tbl WITH GRANT OPTION" + expected_error = ( + "necessary to have the grant SELECT ON default.tbl WITH GRANT OPTION" + ) assert expected_error in node1.query_and_get_error( f"RESTORE TABLE system.users ON CLUSTER 'cluster' FROM {backup_name}", user="u2" ) diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index 5c3f06a9d9d..c9f20333654 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -5,6 +5,7 @@ import time import concurrent from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV, assert_eq_with_retry +import re cluster = ClickHouseCluster(__file__) @@ -110,6 +111,73 @@ def create_and_fill_table(): nodes[i].query(f"INSERT INTO tbl SELECT number FROM numbers(40000000)") +def wait_for_fail_backup(node, backup_id, backup_name): + expected_errors = [ + "Concurrent backups not supported", + f"Backup {backup_name} already exists", + ] + status = node.query( + f"SELECT status FROM system.backups WHERE id == '{backup_id}'" + ).rstrip("\n") + # It is possible that the second backup was picked up first, and then the async backup + if status == "BACKUP_FAILED": + error = node.query( + f"SELECT error FROM system.backups WHERE id == '{backup_id}'" + ).rstrip("\n") + assert any([expected_error in error for expected_error in expected_errors]) + return + elif status == "CREATING_BACKUP": + assert_eq_with_retry( + node, + f"SELECT status FROM system.backups WHERE id = '{backup_id}'", + "BACKUP_FAILED", + sleep_time=2, + retry_count=50, + ) + error = node.query( + f"SELECT error FROM system.backups WHERE id == '{backup_id}'" + ).rstrip("\n") + assert re.search(f"Backup {backup_name} already exists", error) + return + else: + assert False, "Concurrent backups both passed, when one is expected to fail" + + +def wait_for_fail_restore(node, restore_id): + expected_errors = [ + "Concurrent restores not supported", + "Cannot restore the table default.tbl because it already contains some data", + ] + status = node.query( + f"SELECT status FROM system.backups WHERE id == '{restore_id}'" + ).rstrip("\n") + # It is possible that the second backup was picked up first, and then the async backup + if status == "RESTORE_FAILED": + error = node.query( + f"SELECT error FROM system.backups WHERE id == '{restore_id}'" + ).rstrip("\n") + assert any([expected_error in error for expected_error in expected_errors]) + return + elif status == "RESTORING": + assert_eq_with_retry( + node, + f"SELECT status FROM system.backups WHERE id = '{backup_id}'", + "RESTORE_FAILED", + sleep_time=2, + retry_count=50, + ) + error = node.query( + f"SELECT error FROM system.backups WHERE id == '{backup_id}'" + ).rstrip("\n") + assert re.search( + "Cannot restore the table default.tbl because it already contains some data", + error, + ) + return + else: + assert False, "Concurrent restores both passed, when one is expected to fail" + + # All the tests have concurrent backup/restores with same backup names # The same works with different backup names too. Since concurrency # check comes before backup name check, separate tests are not added for different names @@ -137,33 +205,13 @@ def test_concurrent_backups_on_same_node(): f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" ) - if not error: - status = ( - nodes[0] - .query(f"SELECT status FROM system.backups WHERE id == '{id}'") - .rstrip("\n") - ) - # It is possible that the second backup was picked up first, and then the async backup - if status == "BACKUP_FAILED": - return - elif status == "CREATING_BACKUP": - assert_eq_with_retry( - nodes[0], - f"SELECT status FROM system.backups WHERE id = '{id}'", - "BACKUP_FAILED", - sleep_time=2, - retry_count=50, - ) - return - else: - raise Exception( - "Concurrent backups both passed, when one is expected to fail" - ) - expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", ] + if not error: + wait_for_fail_backup(nodes[0], id, backup_name) + assert any([expected_error in error for expected_error in expected_errors]) assert_eq_with_retry( @@ -207,33 +255,14 @@ def test_concurrent_backups_on_different_nodes(): f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" ) - if not error: - status = ( - nodes[1] - .query(f"SELECT status FROM system.backups WHERE id == '{id}'") - .rstrip("\n") - ) - # It is possible that the second backup was picked up first, and then the async backup - if status == "BACKUP_FAILED": - return - elif status == "CREATING_BACKUP": - assert_eq_with_retry( - nodes[1], - f"SELECT status FROM system.backups WHERE id = '{id}'", - "BACKUP_FAILED", - sleep_time=2, - retry_count=50, - ) - return - else: - raise Exception( - "Concurrent backups both passed, when one is expected to fail" - ) - expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", ] + + if not error: + wait_for_fail_backup(nodes[1], id, backup_name) + assert any([expected_error in error for expected_error in expected_errors]) assert_eq_with_retry( @@ -276,33 +305,14 @@ def test_concurrent_restores_on_same_node(): f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" ) - if not error: - status = ( - nodes[0] - .query(f"SELECT status FROM system.backups WHERE id == '{restore_id}'") - .rstrip("\n") - ) - # It is possible that the second backup was picked up first, and then the async backup - if status == "RESTORE_FAILED": - return - elif status == "RESTORING": - assert_eq_with_retry( - nodes[0], - f"SELECT status FROM system.backups WHERE id == '{restore_id}'", - "RESTORE_FAILED", - sleep_time=2, - retry_count=50, - ) - return - else: - raise Exception( - "Concurrent restores both passed, when one is expected to fail" - ) - expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", ] + + if not error: + wait_for_fail_restore(nodes[0], restore_id) + assert any([expected_error in error for expected_error in expected_errors]) assert_eq_with_retry( @@ -345,33 +355,14 @@ def test_concurrent_restores_on_different_node(): f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" ) - if not error: - status = ( - nodes[0] - .query(f"SELECT status FROM system.backups WHERE id == '{restore_id}'") - .rstrip("\n") - ) - # It is possible that the second backup was picked up first, and then the async backup - if status == "RESTORE_FAILED": - return - elif status == "RESTORING": - assert_eq_with_retry( - nodes[0], - f"SELECT status FROM system.backups WHERE id == '{restore_id}'", - "RESTORE_FAILED", - sleep_time=2, - retry_count=50, - ) - return - else: - raise Exception( - "Concurrent restores both passed, when one is expected to fail" - ) - expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", ] + + if not error: + wait_for_fail_restore(nodes[0], restore_id) + assert any([expected_error in error for expected_error in expected_errors]) assert_eq_with_retry( diff --git a/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py b/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py index 5d760c9fc2c..894464fd813 100644 --- a/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py +++ b/tests/integration/test_disabled_access_control_improvements/test_select_from_system_tables.py @@ -43,7 +43,7 @@ def test_system_db(): assert node.query("SELECT count() FROM system.tables WHERE name='table2'") == "1\n" assert node.query("SELECT count()>0 FROM system.settings", user="another") == "1\n" - expected_error = "necessary to have grant SHOW USERS ON *.*" + expected_error = "necessary to have the grant SHOW USERS ON *.*" assert expected_error in node.query_and_get_error( "SELECT count()>0 FROM system.users", user="another" ) @@ -62,7 +62,7 @@ def test_system_db(): ) assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n" - expected_error = "necessary to have grant SHOW USERS ON *.*" + expected_error = "necessary to have the grant SHOW USERS ON *.*" assert expected_error in node.query_and_get_error( "SELECT count()>0 FROM system.users", user="sqluser" ) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index 0bbaa69691e..35f4d81b66a 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -449,7 +449,7 @@ def test_introspection(): ] ) - expected_error = "necessary to have grant SHOW USERS" + expected_error = "necessary to have the grant SHOW USERS" assert expected_error in instance.query_and_get_error("SHOW GRANTS FOR B", user="A") expected_access1 = ( @@ -556,7 +556,7 @@ def test_grant_with_replace_option(): ) expected_error = ( - "it's necessary to have grant INSERT ON test.table WITH GRANT OPTION" + "it's necessary to have the grant INSERT ON test.table WITH GRANT OPTION" ) assert expected_error in instance.query_and_get_error( "GRANT INSERT ON test.table TO B WITH REPLACE OPTION", user="A" @@ -568,7 +568,7 @@ def test_grant_with_replace_option(): instance.query("GRANT INSERT ON test.table TO A WITH GRANT OPTION") expected_error = ( - "it's necessary to have grant SELECT ON test.table WITH GRANT OPTION" + "it's necessary to have the grant SELECT ON test.table WITH GRANT OPTION" ) assert expected_error in instance.query_and_get_error( "GRANT INSERT ON test.table TO B WITH REPLACE OPTION", user="A" diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference b/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py similarity index 100% rename from tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference rename to tests/integration/test_host_regexp_multiple_ptr_records/__init__.py diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml new file mode 100644 index 00000000000..6ca61fb3c44 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml @@ -0,0 +1,11 @@ + + + + + + test1\.example\.com$ + + default + + + \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml new file mode 100644 index 00000000000..9c27c612f63 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml @@ -0,0 +1,5 @@ + + :: + 0.0.0.0 + 1 + diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile new file mode 100644 index 00000000000..3edf37dafa5 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile @@ -0,0 +1,8 @@ +. { + hosts /example.com { + reload "20ms" + fallthrough + } + forward . 127.0.0.11 + log +} diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com new file mode 100644 index 00000000000..6c6e4cbee2e --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com @@ -0,0 +1 @@ +filled in runtime, but needs to exist in order to be volume mapped in docker diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py new file mode 100644 index 00000000000..82ae0b6c527 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/test.py @@ -0,0 +1,109 @@ +import pytest +import socket +from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check +from time import sleep +import os + +DOCKER_COMPOSE_PATH = get_docker_compose_path() +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +cluster = ClickHouseCluster(__file__) + +ch_server = cluster.add_instance( + "clickhouse-server", + with_coredns=True, + main_configs=["configs/listen_host.xml"], + user_configs=["configs/host_regexp.xml"], + ipv6_address="2001:3984:3989::1:1111", +) + +client = cluster.add_instance( + "clickhouse-client", + ipv6_address="2001:3984:3989::1:1112", +) + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def check_ptr_record(ip, hostname): + try: + host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) + if hostname.lower() == host.lower(): + return True + except socket.herror: + pass + return False + + +def setup_dns_server(ip): + domains_string = "test3.example.com test2.example.com test1.example.com" + example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' + run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) + + # DNS server takes time to reload the configuration. + for try_num in range(10): + if all(check_ptr_record(ip, host) for host in domains_string.split()): + break + sleep(1) + + +def setup_ch_server(dns_server_ip): + ch_server.exec_in_container( + (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) + ) + ch_server.exec_in_container( + (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) + ) + ch_server.query("SYSTEM DROP DNS CACHE") + + +def build_endpoint_v4(ip): + return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'" + + +def build_endpoint_v6(ip): + return build_endpoint_v4(f"[{ip}]") + + +def test_host_regexp_multiple_ptr_v4_fails_with_wrong_resolution(started_cluster): + server_ip = cluster.get_instance_ip("clickhouse-server") + random_ip = "9.9.9.9" + dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) + + setup_dns_server(random_ip) + setup_ch_server(dns_server_ip) + + endpoint = build_endpoint_v4(server_ip) + + assert "1\n" != client.exec_in_container(["bash", "-c", f"curl {endpoint}"]) + + +def test_host_regexp_multiple_ptr_v4(started_cluster): + server_ip = cluster.get_instance_ip("clickhouse-server") + client_ip = cluster.get_instance_ip("clickhouse-client") + dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) + + setup_dns_server(client_ip) + setup_ch_server(dns_server_ip) + + endpoint = build_endpoint_v4(server_ip) + + assert "1\n" == client.exec_in_container(["bash", "-c", f"curl {endpoint}"]) + + +def test_host_regexp_multiple_ptr_v6(started_cluster): + setup_dns_server(client.ipv6_address) + setup_ch_server(cluster.get_instance_global_ipv6(cluster.coredns_host)) + + endpoint = build_endpoint_v6(ch_server.ipv6_address) + + assert "1\n" == client.exec_in_container(["bash", "-c", f"curl -6 {endpoint}"]) diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/__init__.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/config.xml b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/config.xml new file mode 100644 index 00000000000..42a1f962705 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/config.xml @@ -0,0 +1,4 @@ + + 1 + 250 + diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/host_regexp.xml b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/host_regexp.xml new file mode 100644 index 00000000000..9329c8dbde2 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/host_regexp.xml @@ -0,0 +1,11 @@ + + + + + + test1\.example\.com$ + + default + + + diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/listen_host.xml b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/listen_host.xml new file mode 100644 index 00000000000..9c27c612f63 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/listen_host.xml @@ -0,0 +1,5 @@ + + :: + 0.0.0.0 + 1 + diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile new file mode 100644 index 00000000000..3edf37dafa5 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile @@ -0,0 +1,8 @@ +. { + hosts /example.com { + reload "20ms" + fallthrough + } + forward . 127.0.0.11 + log +} diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/example.com b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/example.com new file mode 100644 index 00000000000..9beb415c290 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/example.com @@ -0,0 +1 @@ +filled in runtime, but needs to exist in order to be volume mapped in docker \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py new file mode 100644 index 00000000000..fe69d72c1c7 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py @@ -0,0 +1,62 @@ +import pycurl +import threading +from io import BytesIO +import sys + +client_ip = sys.argv[1] +server_ip = sys.argv[2] + +mutex = threading.Lock() +success_counter = 0 +number_of_threads = 100 +number_of_iterations = 100 + + +def perform_request(): + buffer = BytesIO() + crl = pycurl.Curl() + crl.setopt(pycurl.INTERFACE, client_ip) + crl.setopt(crl.WRITEDATA, buffer) + crl.setopt(crl.URL, f"http://{server_ip}:8123/?query=select+1&user=test_dns") + + crl.perform() + + # End curl session + crl.close() + + str_response = buffer.getvalue().decode("iso-8859-1") + expected_response = "1\n" + + mutex.acquire() + + global success_counter + + if str_response == expected_response: + success_counter += 1 + + mutex.release() + + +def perform_multiple_requests(n): + for request_number in range(n): + perform_request() + + +threads = [] + + +for i in range(number_of_threads): + thread = threading.Thread( + target=perform_multiple_requests, args=(number_of_iterations,) + ) + thread.start() + threads.append(thread) + +for thread in threads: + thread.join() + + +if success_counter == number_of_threads * number_of_iterations: + exit(0) + +exit(1) diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py new file mode 100644 index 00000000000..d73e8813e79 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py @@ -0,0 +1,88 @@ +import pytest +import socket +from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check +from time import sleep +import os + +DOCKER_COMPOSE_PATH = get_docker_compose_path() +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +cluster = ClickHouseCluster(__file__) + +ch_server = cluster.add_instance( + "clickhouse-server", + with_coredns=True, + main_configs=["configs/config.xml", "configs/listen_host.xml"], + user_configs=["configs/host_regexp.xml"], +) + +client = cluster.add_instance( + "clickhouse-client", +) + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def check_ptr_record(ip, hostname): + try: + host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) + if hostname.lower() == host.lower(): + return True + except socket.herror: + pass + return False + + +def setup_dns_server(ip): + domains_string = "test3.example.com test2.example.com test1.example.com" + example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' + run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) + + # DNS server takes time to reload the configuration. + for try_num in range(10): + if all(check_ptr_record(ip, host) for host in domains_string.split()): + break + sleep(1) + + +def setup_ch_server(dns_server_ip): + ch_server.exec_in_container( + (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) + ) + ch_server.exec_in_container( + (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) + ) + ch_server.query("SYSTEM DROP DNS CACHE") + + +def build_endpoint_v4(ip): + return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'" + + +def build_endpoint_v6(ip): + return build_endpoint_v4(f"[{ip}]") + + +def test_host_regexp_multiple_ptr_v4(started_cluster): + server_ip = cluster.get_instance_ip("clickhouse-server") + client_ip = cluster.get_instance_ip("clickhouse-client") + dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) + + setup_dns_server(client_ip) + setup_ch_server(dns_server_ip) + + current_dir = os.path.dirname(__file__) + client.copy_file_to_container( + os.path.join(current_dir, "scripts", "stress_test.py"), "stress_test.py" + ) + + client.exec_in_container(["python3", f"stress_test.py", client_ip, server_ip]) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 5f5657fad54..cbb8c94c701 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -344,7 +344,7 @@ def test_granular_access_create_alter_drop_query(cluster): ) assert ( - "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant CREATE NAMED COLLECTION" + "DB::Exception: kek: Not enough privileges. To execute this query, it's necessary to have the grant CREATE NAMED COLLECTION" in node.query_and_get_error( "CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'", user="kek" ) @@ -370,7 +370,7 @@ def test_granular_access_create_alter_drop_query(cluster): ) assert ( - "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" + "DB::Exception: kek: Not enough privileges. To execute this query, it's necessary to have the grant ALTER NAMED COLLECTION" in node.query_and_get_error( "ALTER NAMED COLLECTION collection2 SET key1=2", user="kek" ) @@ -385,14 +385,14 @@ def test_granular_access_create_alter_drop_query(cluster): ) node.query("REVOKE alter named collection ON collection2 FROM kek") assert ( - "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant ALTER NAMED COLLECTION" + "DB::Exception: kek: Not enough privileges. To execute this query, it's necessary to have the grant ALTER NAMED COLLECTION" in node.query_and_get_error( "ALTER NAMED COLLECTION collection2 SET key1=3", user="kek" ) ) assert ( - "DB::Exception: kek: Not enough privileges. To execute this query it's necessary to have grant DROP NAMED COLLECTION" + "DB::Exception: kek: Not enough privileges. To execute this query, it's necessary to have the grant DROP NAMED COLLECTION" in node.query_and_get_error("DROP NAMED COLLECTION collection2", user="kek") ) node.query("GRANT drop named collection ON collection2 TO kek") diff --git a/tests/integration/test_peak_memory_usage/__init__.py b/tests/integration/test_peak_memory_usage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_peak_memory_usage/configs/remote_servers.xml b/tests/integration/test_peak_memory_usage/configs/remote_servers.xml new file mode 100644 index 00000000000..b137758489e --- /dev/null +++ b/tests/integration/test_peak_memory_usage/configs/remote_servers.xml @@ -0,0 +1,20 @@ + + + + + 1 + + shard_1 + 9000 + + + + 3 + + shard_2 + 9000 + + + + + diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py new file mode 100644 index 00000000000..a1313461482 --- /dev/null +++ b/tests/integration/test_peak_memory_usage/test.py @@ -0,0 +1,122 @@ +import pytest +import tempfile +import re + +from helpers.cluster import ClickHouseCluster +from helpers.uclient import client, prompt + +cluster = ClickHouseCluster(__file__) + +shard_1 = cluster.add_instance( + "shard_1", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + macros={ + "shard": "shard_1", + }, +) +shard_2 = cluster.add_instance( + "shard_2", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + macros={ + "shard": "shard_2", + }, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + shard_1.query( + "CREATE TABLE fixed_numbers ON CLUSTER 'cluster' (" + "number UInt64" + ") ENGINE=MergeTree()" + "ORDER BY number" + ) + + shard_1.query( + "CREATE TABLE fixed_numbers_2 ON CLUSTER 'cluster' (" + "number UInt64" + ") ENGINE=Memory ()" + ) + + shard_1.query( + "CREATE TABLE distributed_fixed_numbers (number UInt64) ENGINE=Distributed('cluster', 'default', 'fixed_numbers')" + ) + shard_1.query("INSERT INTO fixed_numbers SELECT number FROM numbers(0, 100)") + + shard_2.query("INSERT INTO fixed_numbers SELECT number FROM numbers(100, 200)") + + shard_1.query("INSERT INTO fixed_numbers_2 SELECT number FROM numbers(0, 10)") + + shard_2.query( + "INSERT INTO fixed_numbers_2 SELECT number FROM numbers(0, 120000)" + ) + + yield cluster + finally: + cluster.shutdown() + + +def get_memory_usage_from_client_output_and_close(client_output): + client_output.seek(0) + peek_memory_usage_str_found = False + for line in client_output: + print(f"'{line}'\n") + if not peek_memory_usage_str_found: + peek_memory_usage_str_found = "Peak memory usage" in line + + if peek_memory_usage_str_found: + search_obj = re.search(r"[+-]?[0-9]+\.[0-9]+", line) + if search_obj: + client_output.close() + print(f"peak_memory_usage {search_obj.group()}") + return search_obj.group() + + print(f"peak_memory_usage not found") + client_output.close() + return "" + + +def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): + client_output = tempfile.TemporaryFile(mode="w+t") + command_text = ( + f"{started_cluster.get_client_cmd()} --host {shard_1.ip_address} --port 9000" + ) + with client(name="client1>", log=client_output, command=command_text) as client1: + client1.expect(prompt) + client1.send( + "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number", + ) + client1.expect("Peak memory usage", timeout=60) + client1.expect(prompt) + + peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) + assert peak_memory_usage + assert shard_2.contains_in_log( + f"Peak memory usage (for query): {peak_memory_usage}" + ) + + +def test_clickhouse_client_max_peak_memory_single_node(started_cluster): + client_output = tempfile.TemporaryFile(mode="w+t") + + command_text = ( + f"{started_cluster.get_client_cmd()} --host {shard_1.ip_address} --port 9000" + ) + with client(name="client1>", log=client_output, command=command_text) as client1: + client1.expect(prompt) + client1.send( + "SELECT COUNT(*) FROM (SELECT number FROM numbers(1,300000) INTERSECT SELECT number FROM numbers(10000,1200000))" + ) + client1.expect("Peak memory usage", timeout=60) + client1.expect(prompt) + + peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) + assert peak_memory_usage + assert shard_1.contains_in_log( + f"Peak memory usage (for query): {peak_memory_usage}" + ) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index 4dbd530dd17..1276a6079f0 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -147,12 +147,16 @@ def test_drop_replica_and_achieve_quorum(started_cluster): @pytest.mark.parametrize(("add_new_data"), [False, True]) def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): - zero.query( - "DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition ON CLUSTER cluster" + # use different table names for easier disambiguation in logs between runs (you may also check uuid though, but not always convenient) + table_name = ( + "test_quorum_insert_with_drop_partition_new_data" + if add_new_data + else "test_quorum_insert_with_drop_partition" ) + zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( - "CREATE TABLE test_quorum_insert_with_drop_partition ON CLUSTER cluster " + f"CREATE TABLE {table_name} ON CLUSTER cluster " "(a Int8, d Date) " "Engine = ReplicatedMergeTree " "PARTITION BY d ORDER BY a " @@ -161,78 +165,74 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): print("Create Replicated table with three replicas") zero.query(create_query) - print("Stop fetches for test_quorum_insert_with_drop_partition at first replica.") - first.query("SYSTEM STOP FETCHES test_quorum_insert_with_drop_partition") + print(f"Stop fetches for {table_name} at first replica.") + first.query(f"SYSTEM STOP FETCHES {table_name}") print("Insert with quorum. (zero and second)") - zero.query( - "INSERT INTO test_quorum_insert_with_drop_partition(a,d) VALUES(1, '2011-01-01')" - ) + zero.query(f"INSERT INTO {table_name}(a,d) VALUES(1, '2011-01-01')") print("Drop partition.") - zero.query( - "ALTER TABLE test_quorum_insert_with_drop_partition DROP PARTITION '2011-01-01'" - ) + zero.query(f"ALTER TABLE {table_name} DROP PARTITION '2011-01-01'") if add_new_data: print("Insert to deleted partition") - zero.query( - "INSERT INTO test_quorum_insert_with_drop_partition(a,d) VALUES(2, '2011-01-01')" - ) + zero.query(f"INSERT INTO {table_name}(a,d) VALUES(2, '2011-01-01')") - print("Resume fetches for test_quorum_insert_with_drop_partition at first replica.") - first.query("SYSTEM START FETCHES test_quorum_insert_with_drop_partition") + print(f"Resume fetches for {table_name} at first replica.") + first.query(f"SYSTEM START FETCHES {table_name}") print("Sync first replica with others.") - first.query("SYSTEM SYNC REPLICA test_quorum_insert_with_drop_partition") + first.query(f"SYSTEM SYNC REPLICA {table_name}") assert "20110101" not in first.query( - """ - WITH (SELECT toString(uuid) FROM system.tables WHERE name = 'test_quorum_insert_with_drop_partition') AS uuid, + f""" + WITH (SELECT toString(uuid) FROM system.tables WHERE name = '{table_name}') AS uuid, '/clickhouse/tables/' || uuid || '/0/quorum/last_part' AS p SELECT * FROM system.zookeeper WHERE path = p FORMAT Vertical """ ) + # Sync second replica not to have `REPLICA_IS_NOT_IN_QUORUM` error + second.query(f"SYSTEM SYNC REPLICA {table_name}") + print("Select from updated partition.") if add_new_data: + assert TSV("2\t2011-01-01\n") == TSV(zero.query(f"SELECT * FROM {table_name}")) assert TSV("2\t2011-01-01\n") == TSV( - zero.query("SELECT * FROM test_quorum_insert_with_drop_partition") - ) - assert TSV("2\t2011-01-01\n") == TSV( - second.query("SELECT * FROM test_quorum_insert_with_drop_partition") + second.query(f"SELECT * FROM {table_name}") ) else: - assert TSV("") == TSV( - zero.query("SELECT * FROM test_quorum_insert_with_drop_partition") - ) - assert TSV("") == TSV( - second.query("SELECT * FROM test_quorum_insert_with_drop_partition") - ) + assert TSV("") == TSV(zero.query(f"SELECT * FROM {table_name}")) + assert TSV("") == TSV(second.query(f"SELECT * FROM {table_name}")) - zero.query( - "DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition ON CLUSTER cluster" - ) + zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") @pytest.mark.parametrize(("add_new_data"), [False, True]) def test_insert_quorum_with_move_partition(started_cluster, add_new_data): - zero.query( - "DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source ON CLUSTER cluster" + # use different table names for easier disambiguation in logs between runs (you may also check uuid though, but not always convenient) + source_table_name = ( + "test_insert_quorum_with_move_partition_source_new_data" + if add_new_data + else "test_insert_quorum_with_move_partition_source" ) - zero.query( - "DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination ON CLUSTER cluster" + destination_table_name = ( + "test_insert_quorum_with_move_partition_destination_new_data" + if add_new_data + else "test_insert_quorum_with_move_partition_destination" ) + zero.query(f"DROP TABLE IF EXISTS {source_table_name} ON CLUSTER cluster") + zero.query(f"DROP TABLE IF EXISTS {destination_table_name} ON CLUSTER cluster") create_source = ( - "CREATE TABLE test_insert_quorum_with_move_partition_source ON CLUSTER cluster " + f"CREATE TABLE {source_table_name} ON CLUSTER cluster " "(a Int8, d Date) " "Engine = ReplicatedMergeTree " "PARTITION BY d ORDER BY a " ) create_destination = ( - "CREATE TABLE test_insert_quorum_with_move_partition_destination ON CLUSTER cluster " + f"CREATE TABLE {destination_table_name} ON CLUSTER cluster " "(a Int8, d Date) " "Engine = ReplicatedMergeTree " "PARTITION BY d ORDER BY a " @@ -244,65 +244,52 @@ def test_insert_quorum_with_move_partition(started_cluster, add_new_data): print("Create destination Replicated table with three replicas") zero.query(create_destination) - print( - "Stop fetches for test_insert_quorum_with_move_partition_source at first replica." - ) - first.query("SYSTEM STOP FETCHES test_insert_quorum_with_move_partition_source") + print(f"Stop fetches for {source_table_name} at first replica.") + first.query(f"SYSTEM STOP FETCHES {source_table_name}") print("Insert with quorum. (zero and second)") - zero.query( - "INSERT INTO test_insert_quorum_with_move_partition_source(a,d) VALUES(1, '2011-01-01')" - ) + zero.query(f"INSERT INTO {source_table_name}(a,d) VALUES(1, '2011-01-01')") print("Drop partition.") zero.query( - "ALTER TABLE test_insert_quorum_with_move_partition_source MOVE PARTITION '2011-01-01' TO TABLE test_insert_quorum_with_move_partition_destination" + f"ALTER TABLE {source_table_name} MOVE PARTITION '2011-01-01' TO TABLE {destination_table_name}" ) if add_new_data: print("Insert to deleted partition") - zero.query( - "INSERT INTO test_insert_quorum_with_move_partition_source(a,d) VALUES(2, '2011-01-01')" - ) + zero.query(f"INSERT INTO {source_table_name}(a,d) VALUES(2, '2011-01-01')") - print( - "Resume fetches for test_insert_quorum_with_move_partition_source at first replica." - ) - first.query("SYSTEM START FETCHES test_insert_quorum_with_move_partition_source") + print(f"Resume fetches for {source_table_name} at first replica.") + first.query(f"SYSTEM START FETCHES {source_table_name}") print("Sync first replica with others.") - first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_move_partition_source") + first.query(f"SYSTEM SYNC REPLICA {source_table_name}") assert "20110101" not in first.query( - """ - WITH (SELECT toString(uuid) FROM system.tables WHERE name = 'test_insert_quorum_with_move_partition_source') AS uuid, + f""" + WITH (SELECT toString(uuid) FROM system.tables WHERE name = '{source_table_name}') AS uuid, '/clickhouse/tables/' || uuid || '/0/quorum/last_part' AS p SELECT * FROM system.zookeeper WHERE path = p FORMAT Vertical """ ) + # Sync second replica not to have `REPLICA_IS_NOT_IN_QUORUM` error + second.query(f"SYSTEM SYNC REPLICA {source_table_name}") + print("Select from updated partition.") if add_new_data: assert TSV("2\t2011-01-01\n") == TSV( - zero.query("SELECT * FROM test_insert_quorum_with_move_partition_source") + zero.query(f"SELECT * FROM {source_table_name}") ) assert TSV("2\t2011-01-01\n") == TSV( - second.query("SELECT * FROM test_insert_quorum_with_move_partition_source") + second.query(f"SELECT * FROM {source_table_name}") ) else: - assert TSV("") == TSV( - zero.query("SELECT * FROM test_insert_quorum_with_move_partition_source") - ) - assert TSV("") == TSV( - second.query("SELECT * FROM test_insert_quorum_with_move_partition_source") - ) + assert TSV("") == TSV(zero.query(f"SELECT * FROM {source_table_name}")) + assert TSV("") == TSV(second.query(f"SELECT * FROM {source_table_name}")) - zero.query( - "DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source ON CLUSTER cluster" - ) - zero.query( - "DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination ON CLUSTER cluster" - ) + zero.query(f"DROP TABLE IF EXISTS {source_table_name} ON CLUSTER cluster") + zero.query(f"DROP TABLE IF EXISTS {destination_table_name} ON CLUSTER cluster") def test_insert_quorum_with_ttl(started_cluster): diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 1235f7d34df..d0a04f40b69 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1226,7 +1226,7 @@ def test_force_synchronous_settings(started_cluster): def select_func(): dummy_node.query( - "SELECT sleepEachRow(1) FROM test_force_synchronous_settings.t" + "SELECT sleepEachRow(1) FROM test_force_synchronous_settings.t SETTINGS function_sleep_max_microseconds_per_block = 0" ) select_thread = threading.Thread(target=select_func) diff --git a/tests/integration/test_reverse_dns_query/__init__.py b/tests/integration/test_reverse_dns_query/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_reverse_dns_query/configs/config.xml b/tests/integration/test_reverse_dns_query/configs/config.xml new file mode 100644 index 00000000000..5ce55afa2a7 --- /dev/null +++ b/tests/integration/test_reverse_dns_query/configs/config.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_reverse_dns_query/configs/listen_host.xml b/tests/integration/test_reverse_dns_query/configs/listen_host.xml new file mode 100644 index 00000000000..9c27c612f63 --- /dev/null +++ b/tests/integration/test_reverse_dns_query/configs/listen_host.xml @@ -0,0 +1,5 @@ + + :: + 0.0.0.0 + 1 + diff --git a/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml b/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml new file mode 100644 index 00000000000..35d0a07c6a6 --- /dev/null +++ b/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_reverse_dns_query/coredns_config/Corefile b/tests/integration/test_reverse_dns_query/coredns_config/Corefile new file mode 100644 index 00000000000..3edf37dafa5 --- /dev/null +++ b/tests/integration/test_reverse_dns_query/coredns_config/Corefile @@ -0,0 +1,8 @@ +. { + hosts /example.com { + reload "20ms" + fallthrough + } + forward . 127.0.0.11 + log +} diff --git a/tests/integration/test_reverse_dns_query/coredns_config/example.com b/tests/integration/test_reverse_dns_query/coredns_config/example.com new file mode 100644 index 00000000000..6c6e4cbee2e --- /dev/null +++ b/tests/integration/test_reverse_dns_query/coredns_config/example.com @@ -0,0 +1 @@ +filled in runtime, but needs to exist in order to be volume mapped in docker diff --git a/tests/integration/test_reverse_dns_query/test.py b/tests/integration/test_reverse_dns_query/test.py new file mode 100644 index 00000000000..00c3956f74f --- /dev/null +++ b/tests/integration/test_reverse_dns_query/test.py @@ -0,0 +1,74 @@ +import pytest +import socket +from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check +from time import sleep +import os + +DOCKER_COMPOSE_PATH = get_docker_compose_path() +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +cluster = ClickHouseCluster(__file__) + +ch_server = cluster.add_instance( + "clickhouse-server", + with_coredns=True, + main_configs=[ + "configs/config.xml", + "configs/reverse_dns_function.xml", + "configs/listen_host.xml", + ], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def check_ptr_record(ip, hostname): + try: + host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) + if hostname.lower() == host.lower(): + return True + except socket.herror: + pass + return False + + +def setup_dns_server(ip): + domains_string = "test.example.com" + example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' + run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) + + # DNS server takes time to reload the configuration. + for try_num in range(10): + if all(check_ptr_record(ip, host) for host in domains_string.split()): + break + sleep(1) + + +def setup_ch_server(dns_server_ip): + ch_server.exec_in_container( + (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) + ) + ch_server.exec_in_container( + (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) + ) + ch_server.query("SYSTEM DROP DNS CACHE") + + +def test_reverse_dns_query(started_cluster): + dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) + random_ipv6 = "4ae8:fa0f:ee1d:68c5:0b76:1b79:7ae6:1549" # https://commentpicker.com/ip-address-generator.php + setup_dns_server(random_ipv6) + setup_ch_server(dns_server_ip) + + for _ in range(0, 200): + response = ch_server.query(f"select reverseDNSQuery('{random_ipv6}')") + assert response == "['test.example.com']\n" diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index acf544c98aa..d88a8f2a243 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -636,7 +636,9 @@ def test_grant_create_row_policy(): assert node.query("SHOW POLICIES") == "" node.query("CREATE USER X") - expected_error = "necessary to have grant CREATE ROW POLICY ON mydb.filtered_table1" + expected_error = ( + "necessary to have the grant CREATE ROW POLICY ON mydb.filtered_table1" + ) assert expected_error in node.query_and_get_error( "CREATE POLICY pA ON mydb.filtered_table1 FOR SELECT USING a + + + + + 10 + + + diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml new file mode 100644 index 00000000000..7bb7fa875e4 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml @@ -0,0 +1,39 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + + +
+ default +
+ + s3_disk + +
+
+ + +
+ s3_disk +
+
+
+
+
+ + + true + 1.0 + + + true +
diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py new file mode 100644 index 00000000000..04bff4a44fb --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/test.py @@ -0,0 +1,94 @@ +#!/usr/bin/env python3 +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) +node3 = cluster.add_instance( + "node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_ttl_move_and_s3(started_cluster): + for i, node in enumerate([node1, node2, node3]): + node.query( + """ + CREATE TABLE s3_test_with_ttl (date DateTime, id UInt32, value String) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/s3_test', '{}') + ORDER BY id + PARTITION BY id + TTL date TO DISK 's3_disk' + SETTINGS storage_policy='s3_and_default', temporary_directories_lifetime=1 + """.format( + i + ) + ) + + node1.query("SYSTEM STOP MOVES s3_test_with_ttl") + + node2.query("SYSTEM STOP MOVES s3_test_with_ttl") + + for i in range(30): + if i % 2 == 0: + node = node1 + else: + node = node2 + + node.query( + f"INSERT INTO s3_test_with_ttl SELECT now() + 5, {i}, randomPrintableASCII(1048570)" + ) + + node1.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + node2.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + node3.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + + assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + + node1.query("SYSTEM START MOVES s3_test_with_ttl") + node2.query("SYSTEM START MOVES s3_test_with_ttl") + + assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + + for attempt in reversed(range(5)): + time.sleep(5) + + print( + node1.query( + "SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical" + ) + ) + + minio = cluster.minio_client + objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True) + counter = 0 + for obj in objects: + print(f"Objectname: {obj.object_name}, metadata: {obj.metadata}") + counter += 1 + + print(f"Total objects: {counter}") + + if counter == 330: + break + + print(f"Attempts remaining: {attempt}") + + assert counter == 330 diff --git a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py new file mode 100644 index 00000000000..fb9f3eb67b9 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py @@ -0,0 +1,46 @@ +#!/usr/bin/env python3 +import time + +import pytest +from helpers.cluster import ClickHouseCluster + + +single_node_cluster = ClickHouseCluster(__file__) +small_node = single_node_cluster.add_instance( + "small_node", + main_configs=["configs/s3.xml"], + user_configs=["configs/max_delayed_streams.xml"], + with_minio=True, +) + + +@pytest.fixture(scope="module") +def started_single_node_cluster(): + try: + single_node_cluster.start() + + yield single_node_cluster + finally: + single_node_cluster.shutdown() + + +def test_vertical_merge_memory_usage(started_single_node_cluster): + if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): + pytest.skip("Disabled for debug and sanitizers. Too slow.") + + small_node.query( + "create table tvm2 (c0 UInt64, c1 UInt64, c2 UInt64, c3 UInt64, c4 UInt64, c5 UInt64, c6 UInt64, c7 UInt64, c8 UInt64, c9 UInt64, c10 UInt64, c11 UInt64, c12 UInt64, c13 UInt64, c14 UInt64, c15 UInt64, c16 UInt64, c17 UInt64, c18 UInt64, c19 UInt64, c20 UInt64, c21 UInt64, c22 UInt64, c23 UInt64, c24 UInt64, c25 UInt64, c26 UInt64, c27 UInt64, c28 UInt64, c29 UInt64, c30 UInt64, c31 UInt64, c32 UInt64, c33 UInt64, c34 UInt64, c35 UInt64, c36 UInt64, c37 UInt64, c38 UInt64, c39 UInt64, c40 UInt64, c41 UInt64, c42 UInt64, c43 UInt64, c44 UInt64, c45 UInt64, c46 UInt64, c47 UInt64, c48 UInt64, c49 UInt64, c50 UInt64, c51 UInt64, c52 UInt64, c53 UInt64, c54 UInt64, c55 UInt64, c56 UInt64, c57 UInt64, c58 UInt64, c59 UInt64, c60 UInt64, c61 UInt64, c62 UInt64, c63 UInt64, c64 UInt64, c65 UInt64, c66 UInt64, c67 UInt64, c68 UInt64, c69 UInt64, c70 UInt64, c71 UInt64, c72 UInt64, c73 UInt64, c74 UInt64, c75 UInt64, c76 UInt64, c77 UInt64, c78 UInt64, c79 UInt64, c80 UInt64, c81 UInt64, c82 UInt64, c83 UInt64, c84 UInt64, c85 UInt64, c86 UInt64, c87 UInt64, c88 UInt64, c89 UInt64, c90 UInt64, c91 UInt64, c92 UInt64, c93 UInt64, c94 UInt64, c95 UInt64, c96 UInt64, c97 UInt64, c98 UInt64, c99 UInt64, c100 UInt64, c101 UInt64, c102 UInt64, c103 UInt64, c104 UInt64, c105 UInt64, c106 UInt64, c107 UInt64, c108 UInt64, c109 UInt64, c110 UInt64, c111 UInt64, c112 UInt64, c113 UInt64, c114 UInt64, c115 UInt64, c116 UInt64, c117 UInt64, c118 UInt64, c119 UInt64, c120 UInt64, c121 UInt64, c122 UInt64, c123 UInt64, c124 UInt64, c125 UInt64, c126 UInt64, c127 UInt64, c128 UInt64, c129 UInt64, c130 UInt64, c131 UInt64, c132 UInt64, c133 UInt64, c134 UInt64, c135 UInt64, c136 UInt64, c137 UInt64, c138 UInt64, c139 UInt64, c140 UInt64, c141 UInt64, c142 UInt64, c143 UInt64, c144 UInt64, c145 UInt64, c146 UInt64, c147 UInt64, c148 UInt64, c149 UInt64, c150 UInt64, c151 UInt64, c152 UInt64, c153 UInt64, c154 UInt64, c155 UInt64, c156 UInt64, c157 UInt64, c158 UInt64, c159 UInt64, c160 UInt64, c161 UInt64, c162 UInt64, c163 UInt64, c164 UInt64, c165 UInt64, c166 UInt64, c167 UInt64, c168 UInt64, c169 UInt64, c170 UInt64, c171 UInt64, c172 UInt64, c173 UInt64, c174 UInt64, c175 UInt64, c176 UInt64, c177 UInt64, c178 UInt64, c179 UInt64, c180 UInt64, c181 UInt64, c182 UInt64, c183 UInt64, c184 UInt64, c185 UInt64, c186 UInt64, c187 UInt64, c188 UInt64, c189 UInt64, c190 UInt64, c191 UInt64, c192 UInt64, c193 UInt64, c194 UInt64, c195 UInt64, c196 UInt64, c197 UInt64, c198 UInt64, c199 UInt64, c200 UInt64, c201 UInt64, c202 UInt64, c203 UInt64, c204 UInt64, c205 UInt64, c206 UInt64, c207 UInt64, c208 UInt64, c209 UInt64, c210 UInt64, c211 UInt64, c212 UInt64, c213 UInt64, c214 UInt64, c215 UInt64, c216 UInt64, c217 UInt64, c218 UInt64, c219 UInt64, c220 UInt64, c221 UInt64, c222 UInt64, c223 UInt64, c224 UInt64, c225 UInt64, c226 UInt64, c227 UInt64, c228 UInt64, c229 UInt64, c230 UInt64, c231 UInt64, c232 UInt64, c233 UInt64, c234 UInt64, c235 UInt64, c236 UInt64, c237 UInt64, c238 UInt64, c239 UInt64, c240 UInt64, c241 UInt64, c242 UInt64, c243 UInt64, c244 UInt64, c245 UInt64, c246 UInt64, c247 UInt64, c248 UInt64, c249 UInt64, c250 UInt64, c251 UInt64, c252 UInt64, c253 UInt64, c254 UInt64, c255 UInt64, c256 UInt64, c257 UInt64, c258 UInt64, c259 UInt64, c260 UInt64, c261 UInt64, c262 UInt64, c263 UInt64, c264 UInt64, c265 UInt64, c266 UInt64, c267 UInt64, c268 UInt64, c269 UInt64, c270 UInt64, c271 UInt64, c272 UInt64, c273 UInt64, c274 UInt64, c275 UInt64, c276 UInt64, c277 UInt64, c278 UInt64, c279 UInt64, c280 UInt64, c281 UInt64, c282 UInt64, c283 UInt64, c284 UInt64, c285 UInt64, c286 UInt64, c287 UInt64, c288 UInt64, c289 UInt64, c290 UInt64, c291 UInt64, c292 UInt64, c293 UInt64, c294 UInt64, c295 UInt64, c296 UInt64, c297 UInt64, c298 UInt64, c299 UInt64) engine = MergeTree order by tuple() settings min_rows_for_wide_part = 10, min_bytes_for_wide_part=0, storage_policy = 's3', vertical_merge_algorithm_min_rows_to_activate=1" + ) + + small_node.query( + "insert into tvm2 select number + 0, number + 1, number + 2, number + 3, number + 4, number + 5, number + 6, number + 7, number + 8, number + 9, number + 10, number + 11, number + 12, number + 13, number + 14, number + 15, number + 16, number + 17, number + 18, number + 19, number + 20, number + 21, number + 22, number + 23, number + 24, number + 25, number + 26, number + 27, number + 28, number + 29, number + 30, number + 31, number + 32, number + 33, number + 34, number + 35, number + 36, number + 37, number + 38, number + 39, number + 40, number + 41, number + 42, number + 43, number + 44, number + 45, number + 46, number + 47, number + 48, number + 49, number + 50, number + 51, number + 52, number + 53, number + 54, number + 55, number + 56, number + 57, number + 58, number + 59, number + 60, number + 61, number + 62, number + 63, number + 64, number + 65, number + 66, number + 67, number + 68, number + 69, number + 70, number + 71, number + 72, number + 73, number + 74, number + 75, number + 76, number + 77, number + 78, number + 79, number + 80, number + 81, number + 82, number + 83, number + 84, number + 85, number + 86, number + 87, number + 88, number + 89, number + 90, number + 91, number + 92, number + 93, number + 94, number + 95, number + 96, number + 97, number + 98, number + 99, number + 100, number + 101, number + 102, number + 103, number + 104, number + 105, number + 106, number + 107, number + 108, number + 109, number + 110, number + 111, number + 112, number + 113, number + 114, number + 115, number + 116, number + 117, number + 118, number + 119, number + 120, number + 121, number + 122, number + 123, number + 124, number + 125, number + 126, number + 127, number + 128, number + 129, number + 130, number + 131, number + 132, number + 133, number + 134, number + 135, number + 136, number + 137, number + 138, number + 139, number + 140, number + 141, number + 142, number + 143, number + 144, number + 145, number + 146, number + 147, number + 148, number + 149, number + 150, number + 151, number + 152, number + 153, number + 154, number + 155, number + 156, number + 157, number + 158, number + 159, number + 160, number + 161, number + 162, number + 163, number + 164, number + 165, number + 166, number + 167, number + 168, number + 169, number + 170, number + 171, number + 172, number + 173, number + 174, number + 175, number + 176, number + 177, number + 178, number + 179, number + 180, number + 181, number + 182, number + 183, number + 184, number + 185, number + 186, number + 187, number + 188, number + 189, number + 190, number + 191, number + 192, number + 193, number + 194, number + 195, number + 196, number + 197, number + 198, number + 199, number + 200, number + 201, number + 202, number + 203, number + 204, number + 205, number + 206, number + 207, number + 208, number + 209, number + 210, number + 211, number + 212, number + 213, number + 214, number + 215, number + 216, number + 217, number + 218, number + 219, number + 220, number + 221, number + 222, number + 223, number + 224, number + 225, number + 226, number + 227, number + 228, number + 229, number + 230, number + 231, number + 232, number + 233, number + 234, number + 235, number + 236, number + 237, number + 238, number + 239, number + 240, number + 241, number + 242, number + 243, number + 244, number + 245, number + 246, number + 247, number + 248, number + 249, number + 250, number + 251, number + 252, number + 253, number + 254, number + 255, number + 256, number + 257, number + 258, number + 259, number + 260, number + 261, number + 262, number + 263, number + 264, number + 265, number + 266, number + 267, number + 268, number + 269, number + 270, number + 271, number + 272, number + 273, number + 274, number + 275, number + 276, number + 277, number + 278, number + 279, number + 280, number + 281, number + 282, number + 283, number + 284, number + 285, number + 286, number + 287, number + 288, number + 289, number + 290, number + 291, number + 292, number + 293, number + 294, number + 295, number + 296, number + 297, number + 298, number + 299 from numbers(20)" + ) + small_node.query("optimize table tvm2 final") + small_node.query("system flush logs") + + # Should be about 25M + res = small_node.query( + "select formatReadableSize(peak_memory_usage), * from system.part_log where table = 'tvm2' and database = currentDatabase() and event_date >= today() - 1 and event_type = 'MergeParts' and peak_memory_usage > 100000000 format Vertical" + ) + + assert res == "" diff --git a/tests/integration/test_select_access_rights/test_from_system_tables.py b/tests/integration/test_select_access_rights/test_from_system_tables.py index cd29183561a..a4dfa662601 100644 --- a/tests/integration/test_select_access_rights/test_from_system_tables.py +++ b/tests/integration/test_select_access_rights/test_from_system_tables.py @@ -44,14 +44,14 @@ def test_system_db(): assert node.query("SELECT count()>0 FROM system.settings", user="another") == "1\n" expected_error = ( - "necessary to have grant SELECT for at least one column on system.users" + "necessary to have the grant SELECT for at least one column on system.users" ) assert expected_error in node.query_and_get_error( "SELECT count()>0 FROM system.users", user="another" ) expected_error = ( - "necessary to have grant SELECT for at least one column on system.clusters" + "necessary to have the grant SELECT for at least one column on system.clusters" ) assert expected_error in node.query_and_get_error( "SELECT count()>0 FROM system.clusters", user="another" @@ -72,14 +72,14 @@ def test_system_db(): assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n" expected_error = ( - "necessary to have grant SELECT for at least one column on system.users" + "necessary to have the grant SELECT for at least one column on system.users" ) assert expected_error in node.query_and_get_error( "SELECT count()>0 FROM system.users", user="sqluser" ) expected_error = ( - "necessary to have grant SELECT for at least one column on system.clusters" + "necessary to have the grant SELECT for at least one column on system.clusters" ) assert node.query_and_get_error( "SELECT count()>0 FROM system.clusters", user="sqluser" @@ -138,7 +138,7 @@ def test_information_schema(): ) expected_error = ( - "necessary to have grant SELECT(table_name) ON information_schema.tables" + "necessary to have the grant SELECT(table_name) ON information_schema.tables" ) assert expected_error in node.query_and_get_error( "SELECT count() FROM information_schema.tables WHERE table_name='table1'", diff --git a/tests/integration/test_select_access_rights/test_main.py b/tests/integration/test_select_access_rights/test_main.py index 76940cdadb4..eedecc2d30c 100644 --- a/tests/integration/test_select_access_rights/test_main.py +++ b/tests/integration/test_select_access_rights/test_main.py @@ -30,7 +30,7 @@ def test_select_single_column(): select_query = "SELECT a FROM table1" assert ( - "it's necessary to have grant SELECT(a) ON default.table1" + "it's necessary to have the grant SELECT(a) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -39,7 +39,7 @@ def test_select_single_column(): instance.query("REVOKE SELECT(a) ON default.table1 FROM A") assert ( - "it's necessary to have grant SELECT(a) ON default.table1" + "it's necessary to have the grant SELECT(a) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -51,7 +51,7 @@ def test_select_single_column_with_table_grant(): select_query = "SELECT a FROM table1" assert ( - "it's necessary to have grant SELECT(a) ON default.table1" + "it's necessary to have the grant SELECT(a) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -60,7 +60,7 @@ def test_select_single_column_with_table_grant(): instance.query("REVOKE SELECT(a) ON default.table1 FROM A") assert ( - "it's necessary to have grant SELECT(a) ON default.table1" + "it's necessary to have the grant SELECT(a) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -72,19 +72,19 @@ def test_select_all_columns(): select_query = "SELECT * FROM table1" assert ( - "it's necessary to have grant SELECT(d, a, b) ON default.table1" + "it's necessary to have the grant SELECT(d, a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) instance.query("GRANT SELECT(d) ON default.table1 TO A") assert ( - "it's necessary to have grant SELECT(d, a, b) ON default.table1" + "it's necessary to have the grant SELECT(d, a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) instance.query("GRANT SELECT(a) ON default.table1 TO A") assert ( - "it's necessary to have grant SELECT(d, a, b) ON default.table1" + "it's necessary to have the grant SELECT(d, a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -99,7 +99,7 @@ def test_select_all_columns_with_table_grant(): select_query = "SELECT * FROM table1" assert ( - "it's necessary to have grant SELECT(d, a, b) ON default.table1" + "it's necessary to have the grant SELECT(d, a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -114,7 +114,7 @@ def test_alias(): select_query = "SELECT x, y, x + y AS s FROM table1" assert ( - "it's necessary to have grant SELECT(x, y) ON default.table1" + "it's necessary to have the grant SELECT(x, y) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -129,7 +129,7 @@ def test_alias_columns(): select_query = "SELECT * FROM table1" assert ( - "it's necessary to have grant SELECT(x, y) ON default.table1" + "it's necessary to have the grant SELECT(x, y) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -138,7 +138,7 @@ def test_alias_columns(): select_query = "SELECT s FROM table1" assert ( - "it's necessary to have grant SELECT(s) ON default.table1" + "it's necessary to have the grant SELECT(s) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -156,7 +156,7 @@ def test_materialized_columns(): select_query = "SELECT * FROM table1" assert ( - "it's necessary to have grant SELECT(x, y) ON default.table1" + "it's necessary to have the grant SELECT(x, y) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -165,7 +165,7 @@ def test_materialized_columns(): select_query = "SELECT p FROM table1" assert ( - "it's necessary to have grant SELECT(p) ON default.table1" + "it's necessary to have the grant SELECT(p) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -186,13 +186,13 @@ def test_select_join(): select_query = "SELECT * FROM table1 JOIN table2 USING(d)" assert ( - "it's necessary to have grant SELECT(d, x, y) ON default.table2" + "it's necessary to have the grant SELECT(d, x, y) ON default.table2" in instance.query_and_get_error(select_query, user="A") ) instance.query("GRANT SELECT(d, x, y) ON default.table2 TO A") assert ( - "it's necessary to have grant SELECT(d, a, b) ON default.table1" + "it's necessary to have the grant SELECT(d, a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -201,7 +201,7 @@ def test_select_join(): instance.query("REVOKE SELECT ON default.table2 FROM A") assert ( - "it's necessary to have grant SELECT(d, x, y) ON default.table2" + "it's necessary to have the grant SELECT(d, x, y) ON default.table2" in instance.query_and_get_error(select_query, user="A") ) @@ -216,13 +216,13 @@ def test_select_union(): select_query = "SELECT * FROM table1 UNION ALL SELECT * FROM table2" assert ( - "it's necessary to have grant SELECT(a, b) ON default.table1" + "it's necessary to have the grant SELECT(a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) instance.query("GRANT SELECT(a, b) ON default.table1 TO A") assert ( - "it's necessary to have grant SELECT(a, b) ON default.table2" + "it's necessary to have the grant SELECT(a, b) ON default.table2" in instance.query_and_get_error(select_query, user="A") ) @@ -231,7 +231,7 @@ def test_select_union(): instance.query("REVOKE SELECT ON default.table1 FROM A") assert ( - "it's necessary to have grant SELECT(a, b) ON default.table1" + "it's necessary to have the grant SELECT(a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -243,7 +243,7 @@ def test_select_count(): select_query = "SELECT count() FROM table1" assert ( - "it's necessary to have grant SELECT for at least one column on default.table1" + "it's necessary to have the grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -252,7 +252,7 @@ def test_select_count(): instance.query("REVOKE SELECT(x) ON default.table1 FROM A") assert ( - "it's necessary to have grant SELECT for at least one column on default.table1" + "it's necessary to have the grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -261,7 +261,7 @@ def test_select_count(): instance.query("REVOKE SELECT(y) ON default.table1 FROM A") assert ( - "it's necessary to have grant SELECT for at least one column on default.table1" + "it's necessary to have the grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -279,7 +279,7 @@ def test_select_where(): select_query = "SELECT a FROM table1 WHERE b = 0" assert ( - "it's necessary to have grant SELECT(a, b) ON default.table1" + "it's necessary to have the grant SELECT(a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -288,7 +288,7 @@ def test_select_where(): instance.query("REVOKE SELECT ON default.table1 FROM A") assert ( - "it's necessary to have grant SELECT(a, b) ON default.table1" + "it's necessary to have the grant SELECT(a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -306,7 +306,7 @@ def test_select_prewhere(): select_query = "SELECT a FROM table1 PREWHERE b = 0" assert ( - "it's necessary to have grant SELECT(a, b) ON default.table1" + "it's necessary to have the grant SELECT(a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -315,7 +315,7 @@ def test_select_prewhere(): instance.query("REVOKE SELECT ON default.table1 FROM A") assert ( - "it's necessary to have grant SELECT(a, b) ON default.table1" + "it's necessary to have the grant SELECT(a, b) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) @@ -334,11 +334,11 @@ def test_select_with_row_policy(): select_query = "SELECT a FROM table1" select_query2 = "SELECT count() FROM table1" assert ( - "it's necessary to have grant SELECT(a) ON default.table1" + "it's necessary to have the grant SELECT(a) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) assert ( - "it's necessary to have grant SELECT for at least one column on default.table1" + "it's necessary to have the grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query2, user="A") ) @@ -348,10 +348,10 @@ def test_select_with_row_policy(): instance.query("REVOKE SELECT(a) ON default.table1 FROM A") assert ( - "it's necessary to have grant SELECT(a) ON default.table1" + "it's necessary to have the grant SELECT(a) ON default.table1" in instance.query_and_get_error(select_query, user="A") ) assert ( - "it's necessary to have grant SELECT for at least one column on default.table1" + "it's necessary to have the grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query2, user="A") ) diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 8770cc82064..5e40b534cee 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -589,10 +589,10 @@ def test_function_current_profiles(): def test_allow_ddl(): - assert "it's necessary to have grant" in instance.query_and_get_error( + assert "it's necessary to have the grant" in instance.query_and_get_error( "CREATE TABLE tbl(a Int32) ENGINE=Log", user="robin" ) - assert "it's necessary to have grant" in instance.query_and_get_error( + assert "it's necessary to have the grant" in instance.query_and_get_error( "GRANT CREATE ON tbl TO robin", user="robin" ) assert "DDL queries are prohibited" in instance.query_and_get_error( @@ -615,10 +615,10 @@ def test_allow_introspection(): assert "Introspection functions are disabled" in instance.query_and_get_error( "SELECT demangle('a')" ) - assert "it's necessary to have grant" in instance.query_and_get_error( + assert "it's necessary to have the grant" in instance.query_and_get_error( "SELECT demangle('a')", user="robin" ) - assert "it's necessary to have grant" in instance.query_and_get_error( + assert "it's necessary to have the grant" in instance.query_and_get_error( "SELECT demangle('a')", user="robin", settings={"allow_introspection_functions": 1}, @@ -659,7 +659,7 @@ def test_allow_introspection(): "REVOKE demangle ON *.* FROM robin", settings={"allow_introspection_functions": 1}, ) - assert "it's necessary to have grant" in instance.query_and_get_error( + assert "it's necessary to have the grant" in instance.query_and_get_error( "SELECT demangle('a')", user="robin" ) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 686eb1ea751..d4f8fab3a82 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -726,6 +726,22 @@ def test_auto_close_connection(started_cluster): assert count == 2 +def test_literal_escaping(started_cluster): + cursor = started_cluster.postgres_conn.cursor() + cursor.execute(f"DROP TABLE IF EXISTS escaping") + cursor.execute(f"CREATE TABLE escaping(text varchar(255))") + node1.query( + "CREATE TABLE default.escaping (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'escaping', 'postgres', 'mysecretpassword')" + ) + node1.query("SELECT * FROM escaping WHERE text = ''''") # ' -> '' + node1.query("SELECT * FROM escaping WHERE text = '\\''") # ' -> '' + node1.query("SELECT * FROM escaping WHERE text = '\\\\\\''") # \' -> \'' + node1.query("SELECT * FROM escaping WHERE text = '\\\\\\''") # \' -> \'' + node1.query("SELECT * FROM escaping WHERE text like '%a''a%'") # %a'a% -> %a''a% + node1.query("SELECT * FROM escaping WHERE text like '%a\\'a%'") # %a'a% -> %a''a% + cursor.execute(f"DROP TABLE escaping") + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index e3eb3ae455b..9fb794648cb 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -944,7 +944,7 @@ def test_predefined_connection_configuration(started_cluster): f"CREATE TABLE {name} (id UInt32) ENGINE = S3(s3_conf1, format='CSV')" ) assert ( - "To execute this query it's necessary to have grant NAMED COLLECTION ON s3_conf1" + "To execute this query, it's necessary to have the grant NAMED COLLECTION ON s3_conf1" in error ) error = instance.query_and_get_error( @@ -952,7 +952,7 @@ def test_predefined_connection_configuration(started_cluster): user="user", ) assert ( - "To execute this query it's necessary to have grant NAMED COLLECTION ON s3_conf1" + "To execute this query, it's necessary to have the grant NAMED COLLECTION ON s3_conf1" in error ) @@ -973,12 +973,12 @@ def test_predefined_connection_configuration(started_cluster): error = instance.query_and_get_error("SELECT * FROM s3(no_collection)") assert ( - "To execute this query it's necessary to have grant NAMED COLLECTION ON no_collection" + "To execute this query, it's necessary to have the grant NAMED COLLECTION ON no_collection" in error ) error = instance.query_and_get_error("SELECT * FROM s3(no_collection)", user="user") assert ( - "To execute this query it's necessary to have grant NAMED COLLECTION ON no_collection" + "To execute this query, it's necessary to have the grant NAMED COLLECTION ON no_collection" in error ) instance = started_cluster.instances["dummy"] # has named collection access diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index 7f359078967..7ff7a871413 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -68,13 +68,13 @@ def test_url_cluster_with_named_collection(): def test_table_function_url_access_rights(): node1.query("CREATE USER OR REPLACE u1") - expected_error = "necessary to have grant CREATE TEMPORARY TABLE, URL ON *.*" + expected_error = "necessary to have the grant CREATE TEMPORARY TABLE, URL ON *.*" assert expected_error in node1.query_and_get_error( f"SELECT * FROM url('http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", user="u1", ) - expected_error = "necessary to have grant CREATE TEMPORARY TABLE, URL ON *.*" + expected_error = "necessary to have the grant CREATE TEMPORARY TABLE, URL ON *.*" assert expected_error in node1.query_and_get_error( f"SELECT * FROM url('http://nginx:80/test_1', 'TSV')", user="u1" ) @@ -89,7 +89,7 @@ def test_table_function_url_access_rights(): user="u1", ) == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) - expected_error = "necessary to have grant URL ON *.*" + expected_error = "necessary to have the grant URL ON *.*" assert expected_error in node1.query_and_get_error( f"DESCRIBE TABLE url('http://nginx:80/test_1', 'TSV')", user="u1" ) diff --git a/tests/integration/test_table_functions_access_rights/test.py b/tests/integration/test_table_functions_access_rights/test.py index 09a05122c07..b1d1a291bc5 100644 --- a/tests/integration/test_table_functions_access_rights/test.py +++ b/tests/integration/test_table_functions_access_rights/test.py @@ -40,7 +40,7 @@ def test_merge(): instance.query("CREATE USER A") assert ( - "it's necessary to have grant CREATE TEMPORARY TABLE ON *.*" + "it's necessary to have the grant CREATE TEMPORARY TABLE ON *.*" in instance.query_and_get_error(select_query, user="A") ) @@ -62,7 +62,7 @@ def test_merge(): instance.query("GRANT SELECT ON default.table1 TO A") instance.query("GRANT INSERT ON default.table2 TO A") assert ( - "it's necessary to have grant SELECT ON default.table2" + "it's necessary to have the grant SELECT ON default.table2" in instance.query_and_get_error(select_query, user="A") ) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 62640394a85..a27bb472ea8 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -117,7 +117,8 @@ def assert_took(took, should_took): # we need to decrease the lower limit because the server limits could # be enforced by throttling some server background IO instead of query IO # and we have no control over it - assert took >= should_took[0] * 0.85 and took < should_took[1] + # Note that throttler does not apply any restrictions on upper bound, so we can only tell how much time required "at least", not "at most" + assert took >= should_took * 0.85 @pytest.mark.parametrize( @@ -132,7 +133,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_local_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds @@ -142,7 +143,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7, 14), + 7, id="user_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -152,7 +153,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_local_throttling", ), # @@ -164,7 +165,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_remote_to_local_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds @@ -174,7 +175,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7, 14), + 7, id="user_remote_to_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -184,7 +185,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_to_local_throttling", ), # @@ -196,7 +197,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_remote_to_remote_throttling", ), # No throttling for S3-to-S3, uses native copy @@ -206,7 +207,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (0, 3), + 0, id="user_remote_to_remote_throttling", ), # No throttling for S3-to-S3, uses native copy @@ -216,7 +217,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (0, 3), + 0, id="server_remote_to_remote_throttling", ), # @@ -233,7 +234,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_local_to_remote_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds, but for S3Client it is 2x more @@ -243,7 +244,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7 * 3, 7 * 4 - 1), + 7 * 3, id="user_local_to_remote_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds, but for S3Client it is 2x more @@ -253,7 +254,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3 * 3, 3 * 5), + 3 * 3, id="server_local_to_remote_throttling", ), ], @@ -290,7 +291,7 @@ def test_backup_throttling_override(): }, ) # reading 1e6*8 bytes with 500Ki default bandwith should take (8-0.5)/0.5=15 seconds - assert_took(took, (15, 20)) + assert_took(took, 15) @pytest.mark.parametrize( @@ -299,14 +300,14 @@ def test_backup_throttling_override(): # # Local # - pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + pytest.param("default", None, None, None, 0, id="no_local_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "default", "user", "max_local_read_bandwidth", "1M", - (7, 14), + 7, id="user_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -315,20 +316,20 @@ def test_backup_throttling_override(): "server", "max_local_read_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_local_throttling", ), # # Remote # - pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", "user", "max_remote_read_network_bandwidth", "1M", - (7, 14), + 7, id="user_remote_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -337,7 +338,7 @@ def test_backup_throttling_override(): "server", "max_remote_read_network_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_throttling", ), ], @@ -361,14 +362,14 @@ def test_read_throttling(policy, mode, setting, value, should_took): # # Local # - pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + pytest.param("default", None, None, None, 0, id="no_local_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "default", "user", "max_local_write_bandwidth", "1M", - (7, 14), + 7, id="local_user_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -377,29 +378,29 @@ def test_read_throttling(policy, mode, setting, value, should_took): "server", "max_local_write_bandwidth_for_server", "2M", - (3, 7), + 3, id="local_server_throttling", ), # # Remote # - pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), - # writeing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), + # writing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", "user", "max_remote_write_network_bandwidth", "1M", - (7, 14), + 7, id="user_remote_throttling", ), - # writeing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # writing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds pytest.param( "s3", "server", "max_remote_write_network_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_throttling", ), ], diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index a993c330d2a..eb8e9826eff 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -36,7 +36,9 @@ create temporary table known_short_messages (s String) as select * from (select 'Database {} does not exist', 'Dictionary ({}) not found', 'Unknown table function {}', 'Unknown format {}', 'Unknown explain kind ''{}''', 'Unknown setting {}', 'Unknown input format {}', 'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', -'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}' +'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}', +'Processed: {}%', 'Creating {}: {}', 'Table {}.{} doesn''t exist', 'Invalid cache key hex: {}', +'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. diff --git a/tests/queries/0_stateless/00906_low_cardinality_cache.sql b/tests/queries/0_stateless/00906_low_cardinality_cache.sql index cd2ceabcf6d..55eacd0db44 100644 --- a/tests/queries/0_stateless/00906_low_cardinality_cache.sql +++ b/tests/queries/0_stateless/00906_low_cardinality_cache.sql @@ -1,5 +1,5 @@ drop table if exists lc_00906; -create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b; +create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into lc_00906 select '0123456789' from numbers(100000000); select count(), b from lc_00906 group by b; drop table if exists lc_00906; diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh index fe757acfd06..8e28995980f 100755 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sh @@ -16,14 +16,14 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY hop(now('US/Samoa'), INTERVAL '5' SECOND, INTERVAL '5' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY hop(now('US/Samoa'), INTERVAL '10' SECOND, INTERVAL '10' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1); EOF for _ in {1..100}; do $CLICKHOUSE_CLIENT "${opts[@]}" --query="SELECT count(*) FROM dst" | grep -q "1" && echo 'OK' && break - sleep .5 + sleep .2 done $CLICKHOUSE_CLIENT "${opts[@]}" --query="SELECT count FROM dst" diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index 47316a6a805..3ba59f9a424 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -19,7 +19,7 @@ function run_selects() thread_num=$1 readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name != 'zookeeper' and name != 'models' - AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num") + AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num AND name NOT LIKE '%\\_sender' AND name NOT LIKE '%\\_watcher'") for t in "${tables_arr[@]}" do diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index f2348c29146..d4884cbf457 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -242,10 +242,10 @@ wait ||: wait_for_queries_to_finish 40 -$CLICKHOUSE_CLIENT -q "SELECT type, count(n) = countDistinct(n) FROM merge(currentDatabase(), '') GROUP BY type ORDER BY type" -$CLICKHOUSE_CLIENT -q "SELECT DISTINCT arraySort(groupArrayIf(n, type=1)) = arraySort(groupArrayIf(n, type=2)) FROM merge(currentDatabase(), '') GROUP BY _table ORDER BY _table" -$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM merge(currentDatabase(), '') WHERE type=4" -$CLICKHOUSE_CLIENT -q "SELECT type, count(n) == max(n), sum(n) == max(n)*(max(n)+1)/2 FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type" +$CLICKHOUSE_CLIENT --implicit_transaction=1 --throw_on_unsupported_query_inside_transaction=0 -q "SELECT type, count(n) = countDistinct(n) FROM merge(currentDatabase(), '') GROUP BY type ORDER BY type" +$CLICKHOUSE_CLIENT --implicit_transaction=1 --throw_on_unsupported_query_inside_transaction=0 -q "SELECT DISTINCT arraySort(groupArrayIf(n, type=1)) = arraySort(groupArrayIf(n, type=2)) FROM merge(currentDatabase(), '') GROUP BY _table ORDER BY _table" +$CLICKHOUSE_CLIENT --implicit_transaction=1 --throw_on_unsupported_query_inside_transaction=0 -q "SELECT count(n), sum(n) FROM merge(currentDatabase(), '') WHERE type=4" +$CLICKHOUSE_CLIENT --implicit_transaction=1 --throw_on_unsupported_query_inside_transaction=0 -q "SELECT type, count(n) == max(n), sum(n) == max(n)*(max(n)+1)/2 FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type" $CLICKHOUSE_CLIENT --query "DROP TABLE src"; $CLICKHOUSE_CLIENT --query "DROP TABLE dst"; diff --git a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh index 32ad78dead6..0d2016952d4 100755 --- a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh @@ -113,10 +113,10 @@ kill -TERM $PID_2 wait wait_for_queries_to_finish -$CLICKHOUSE_CLIENT -q "SELECT type, count(n) = countDistinct(n) FROM merge(currentDatabase(), '') GROUP BY type ORDER BY type" -$CLICKHOUSE_CLIENT -q "SELECT DISTINCT arraySort(groupArrayIf(n, type=1)) = arraySort(groupArrayIf(n, type=2)) FROM merge(currentDatabase(), '') GROUP BY _table ORDER BY _table" -$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM merge(currentDatabase(), '') WHERE type=4" -$CLICKHOUSE_CLIENT -q "SELECT type, count(n) == max(n), sum(n) == max(n)*(max(n)+1)/2 FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type" +$CLICKHOUSE_CLIENT --implicit_transaction=1 --throw_on_unsupported_query_inside_transaction=0 -q "SELECT type, count(n) = countDistinct(n) FROM merge(currentDatabase(), '') GROUP BY type ORDER BY type" +$CLICKHOUSE_CLIENT --implicit_transaction=1 --throw_on_unsupported_query_inside_transaction=0 -q "SELECT DISTINCT arraySort(groupArrayIf(n, type=1)) = arraySort(groupArrayIf(n, type=2)) FROM merge(currentDatabase(), '') GROUP BY _table ORDER BY _table" +$CLICKHOUSE_CLIENT --implicit_transaction=1 --throw_on_unsupported_query_inside_transaction=0 -q "SELECT count(n), sum(n) FROM merge(currentDatabase(), '') WHERE type=4" +$CLICKHOUSE_CLIENT --implicit_transaction=1 --throw_on_unsupported_query_inside_transaction=0 -q "SELECT type, count(n) == max(n), sum(n) == max(n)*(max(n)+1)/2 FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type" $CLICKHOUSE_CLIENT --query "DROP TABLE src"; diff --git a/tests/queries/0_stateless/01293_system_distribution_queue.sql b/tests/queries/0_stateless/01293_system_distribution_queue.sql index 9997f18f61d..f14c0d64a7d 100644 --- a/tests/queries/0_stateless/01293_system_distribution_queue.sql +++ b/tests/queries/0_stateless/01293_system_distribution_queue.sql @@ -8,7 +8,7 @@ create table null_01293 (key Int) engine=Null(); create table dist_01293 as null_01293 engine=Distributed(test_cluster_two_shards, currentDatabase(), null_01293, key); -- no rows, since no active monitor -select * from system.distribution_queue; +select * from system.distribution_queue where database = currentDatabase(); select 'INSERT'; system stop distributed sends dist_01293; diff --git a/tests/queries/0_stateless/01306_benchmark_json.reference b/tests/queries/0_stateless/01306_benchmark_json.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/01306_benchmark_json.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/01306_benchmark_json.sh b/tests/queries/0_stateless/01306_benchmark_json.sh deleted file mode 100755 index ae361826250..00000000000 --- a/tests/queries/0_stateless/01306_benchmark_json.sh +++ /dev/null @@ -1,11 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -set -e - -$CLICKHOUSE_BENCHMARK --iterations 10 --json "${CLICKHOUSE_TMP}"/out.json <<< "SELECT 1" 2>/dev/null && cat "${CLICKHOUSE_TMP}"/out.json | - $CLICKHOUSE_LOCAL --input-format JSONAsString --structure "s String" --query "SELECT isValidJSON(s) FROM table" diff --git a/tests/queries/0_stateless/01460_DistributedFilesToInsert.reference b/tests/queries/0_stateless/01460_DistributedFilesToInsert.reference deleted file mode 100644 index 7a123a39978..00000000000 --- a/tests/queries/0_stateless/01460_DistributedFilesToInsert.reference +++ /dev/null @@ -1,12 +0,0 @@ -INSERT -0 -0 -STOP/START DISTRIBUTED SENDS -1 -0 -FLUSH DISTRIBUTED -1 -0 -DROP TABLE -1 -0 diff --git a/tests/queries/0_stateless/01460_DistributedFilesToInsert.sql b/tests/queries/0_stateless/01460_DistributedFilesToInsert.sql deleted file mode 100644 index 9b7a98f30bc..00000000000 --- a/tests/queries/0_stateless/01460_DistributedFilesToInsert.sql +++ /dev/null @@ -1,45 +0,0 @@ --- Tags: no-parallel, no-fasttest --- Tag no-fasttest: Look at DistributedFilesToInsert, so cannot run in parallel. - --- otherwise SYSTEM STOP DISTRIBUTED SENDS does not makes any effect (for localhost) --- (i.e. no .bin files and hence no sending is required) -set prefer_localhost_replica=0; - -drop table if exists data_01460; -drop table if exists dist_01460; - -create table data_01460 as system.one engine=Null(); -create table dist_01460 as data_01460 engine=Distributed(test_shard_localhost, currentDatabase(), data_01460) settings monitor_sleep_time_ms=50; - -select 'INSERT'; -select value from system.metrics where metric = 'DistributedFilesToInsert'; -insert into dist_01460 select * from system.one; -select sleep(1) format Null; -- monitor_sleep_time_ms -select value from system.metrics where metric = 'DistributedFilesToInsert'; - -select 'STOP/START DISTRIBUTED SENDS'; -system stop distributed sends dist_01460; -insert into dist_01460 select * from system.one; -select sleep(1) format Null; -- monitor_sleep_time_ms -select value from system.metrics where metric = 'DistributedFilesToInsert'; -system start distributed sends dist_01460; -select sleep(1) format Null; -- monitor_sleep_time_ms -select value from system.metrics where metric = 'DistributedFilesToInsert'; - -select 'FLUSH DISTRIBUTED'; -system stop distributed sends dist_01460; -insert into dist_01460 select * from system.one; -select sleep(1) format Null; -- monitor_sleep_time_ms -select value from system.metrics where metric = 'DistributedFilesToInsert'; -system flush distributed dist_01460; -select value from system.metrics where metric = 'DistributedFilesToInsert'; - -select 'DROP TABLE'; -system stop distributed sends dist_01460; -insert into dist_01460 select * from system.one; -select sleep(1) format Null; -- monitor_sleep_time_ms -select value from system.metrics where metric = 'DistributedFilesToInsert'; -drop table dist_01460; -select value from system.metrics where metric = 'DistributedFilesToInsert'; - -drop table data_01460; diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index be42a656c66..54ca55d2068 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -168,19 +168,23 @@ Filter 3 > one condition of filter is pushed down before LEFT JOIN Join +Join Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before LEFT JOIN Join +Join Filter column: notEquals(number_0, 1_UInt8) 0 0 3 3 > one condition of filter is pushed down before INNER JOIN Join +Join Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before INNER JOIN Join +Join Filter column: notEquals(number_0, 1_UInt8) 3 3 > filter is pushed down before UNION diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.reference b/tests/queries/0_stateless/01683_text_log_deadlock.reference index 76de47c4a46..4cf61460252 100644 --- a/tests/queries/0_stateless/01683_text_log_deadlock.reference +++ b/tests/queries/0_stateless/01683_text_log_deadlock.reference @@ -1 +1 @@ -queries 25000 +queries: 25000 diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.sh b/tests/queries/0_stateless/01683_text_log_deadlock.sh index debb7a0017f..af7f348a6a2 100755 --- a/tests/queries/0_stateless/01683_text_log_deadlock.sh +++ b/tests/queries/0_stateless/01683_text_log_deadlock.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_BENCHMARK --secure -i 25000 -c 32 --query 'SELECT 1' 2>&1 | grep -oF 'queries 25000' +$CLICKHOUSE_BENCHMARK --secure -i 25000 -c 32 --query 'SELECT 1' 2>&1 | grep -oF 'queries: 25000' diff --git a/tests/queries/0_stateless/01710_normal_projection_format.reference b/tests/queries/0_stateless/01710_normal_projection_format.reference new file mode 100644 index 00000000000..318c7253724 --- /dev/null +++ b/tests/queries/0_stateless/01710_normal_projection_format.reference @@ -0,0 +1 @@ +CREATE TABLE default.test\n(\n `uuid` FixedString(16),\n `id` Int32,\n `ns` FixedString(16),\n `dt` DateTime64(6),\n PROJECTION mtlog_proj_source_reference\n (\n SELECT *\n ORDER BY substring(ns, 1, 5)\n )\n)\nENGINE = MergeTree\nORDER BY (id, dt, uuid)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01710_normal_projection_format.sql b/tests/queries/0_stateless/01710_normal_projection_format.sql new file mode 100644 index 00000000000..6210be166b3 --- /dev/null +++ b/tests/queries/0_stateless/01710_normal_projection_format.sql @@ -0,0 +1,17 @@ +DROP TABLE if exists test; + +CREATE TABLE test +( + uuid FixedString(16), + id int, + ns FixedString(16), + dt DateTime64(6), +) +ENGINE = MergeTree +ORDER BY (id, dt, uuid); + +ALTER TABLE test ADD PROJECTION mtlog_proj_source_reference (SELECT * ORDER BY substring(ns, 1, 5)); + +SHOW CREATE test; + +drop table test; diff --git a/tests/queries/0_stateless/01955_clickhouse_benchmark_connection_hang.reference b/tests/queries/0_stateless/01955_clickhouse_benchmark_connection_hang.reference deleted file mode 100644 index 9e388b62601..00000000000 --- a/tests/queries/0_stateless/01955_clickhouse_benchmark_connection_hang.reference +++ /dev/null @@ -1,3 +0,0 @@ -Loaded 1 queries. -I/O error: Too many open files -70 diff --git a/tests/queries/0_stateless/01955_clickhouse_benchmark_connection_hang.sh b/tests/queries/0_stateless/01955_clickhouse_benchmark_connection_hang.sh deleted file mode 100755 index 7740ffcce7b..00000000000 --- a/tests/queries/0_stateless/01955_clickhouse_benchmark_connection_hang.sh +++ /dev/null @@ -1,72 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-ubsan -# Tag no-ubsan: Limits RLIMIT_NOFILE, see comment in the test - -# shellcheck disable=SC2086 - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# NOTE: Tests with limit for number of opened files cannot be run under UBsan. -# -# UBsan needs to create pipe each time it need to check the type: -# -# pipe() -# __sanitizer::IsAccessibleMemoryRange(unsigned long, unsigned long) -# __ubsan::checkDynamicType(void*, void*, unsigned long) + 271 -# HandleDynamicTypeCacheMiss(__ubsan::DynamicTypeCacheMissData*, unsigned long, unsigned long, __ubsan::ReportOptions) + 34 -# __ubsan_handle_dynamic_type_cache_miss_abort + 58 -# -# Obviously it will fail if RLIMIT_NOFILE exceeded (like in this test), and the UBsan will falsely report [1]: -# -# 01955_clickhouse_benchmark_connection_hang: [ FAIL ] 1.56 sec. - result differs with reference: -# --- /usr/share/clickhouse-test/queries/0_stateless/01955_clickhouse_benchmark_connection_hang.reference 2021-07-21 11:14:58.000000000 +0300 -# +++ /tmp/clickhouse-test/0_stateless/01955_clickhouse_benchmark_connection_hang.stdout 2021-07-21 11:53:45.684050372 +0300 -# @@ -1,3 +1,22 @@ -# Loaded 1 queries. -# -I/O error: Too many open files -# -70 -# +../contrib/libcxx/include/memory:3212:19: runtime error: member call on address 0x00002939d5c0 which does not point to an object of type 'std::__1::__shared_weak_count' -# +0x00002939d5c0: note: object has invalid vptr -# + -# +==558==WARNING: Can't create a socket pair to start external symbolizer (errno: 24) -# +==558==WARNING: Can't create a socket pair to start external symbolizer (errno: 24) -# +==558==WARNING: Can't create a socket pair to start external symbolizer (errno: 24) -# +==558==WARNING: Can't create a socket pair to start external symbolizer (errno: 24) -# +==558==WARNING: Can't create a socket pair to start external symbolizer (errno: 24) -# +==558==WARNING: Failed to use and restart external symbolizer! -# + #0 0xfe86b57 (/usr/bin/clickhouse+0xfe86b57) -# + #1 0xfe83fd7 (/usr/bin/clickhouse+0xfe83fd7) -# + #2 0xfe89af4 (/usr/bin/clickhouse+0xfe89af4) -# + #3 0xfe81fa9 (/usr/bin/clickhouse+0xfe81fa9) -# + #4 0x1f377609 (/usr/bin/clickhouse+0x1f377609) -# + #5 0xfe7e2a1 (/usr/bin/clickhouse+0xfe7e2a1) -# + #6 0xfce1003 (/usr/bin/clickhouse+0xfce1003) -# + #7 0x7f3345bd30b2 (/lib/x86_64-linux-gnu/libc.so.6+0x270b2) -# + #8 0xfcbf0ed (/usr/bin/clickhouse+0xfcbf0ed) -# + -# +SUMMARY: UndefinedBehaviorSanitizer: undefined-behavior ../contrib/libcxx/include/memory:3212:19 in -# +1 -# -# Stacktrace from lldb: -# -# thread #1, name = 'clickhouse-benc', stop reason = Dynamic type mismatch -# * frame #0: 0x000000000fffc070 clickhouse`__ubsan_on_report -# frame #1: 0x000000000fff6511 clickhouse`__ubsan::Diag::~Diag() + 209 -# frame #2: 0x000000000fffcb11 clickhouse`HandleDynamicTypeCacheMiss(__ubsan::DynamicTypeCacheMissData*, unsigned long, unsigned long, __ubsan::ReportOptions) + 609 -# frame #3: 0x000000000fffcf2a clickhouse`__ubsan_handle_dynamic_type_cache_miss_abort + 58 -# frame #4: 0x00000000101a33f8 clickhouse`std::__1::shared_ptr::PoolEntryHelper>::~shared_ptr(this=) + 152 at memory:3212 -# frame #5: 0x00000000101a267a clickhouse`PoolBase::Entry::~Entry(this=) + 26 at PoolBase.h:67 -# frame #6: 0x00000000101a0878 clickhouse`DB::ConnectionPool::get(this=, timeouts=0x00007fffffffc278, settings=, force_connected=true) + 664 at ConnectionPool.h:93 -# frame #7: 0x00000000101a6395 clickhouse`DB::Benchmark::runBenchmark(this=) + 981 at Benchmark.cpp:309 -# frame #8: 0x000000001019e84a clickhouse`DB::Benchmark::main(this=0x00007fffffffd8c8, (null)=) + 586 at Benchmark.cpp:128 -# frame #9: 0x000000001f5d028a clickhouse`Poco::Util::Application::run(this=0x00007fffffffd8c8) + 42 at Application.cpp:334 -# frame #10: 0x000000001019ab42 clickhouse`mainEntryClickHouseBenchmark(argc=, argv=) + 6978 at Benchmark.cpp:655 -# frame #11: 0x000000000fffdfc4 clickhouse`main(argc_=, argv_=) + 356 at main.cpp:366 -# frame #12: 0x00007ffff7de6d0a libc.so.6`__libc_start_main(main=(clickhouse`main at main.cpp:339), argc=7, argv=0x00007fffffffe1e8, init=, fini=, rtld_fini=, stack_end=0x00007fffffffe1d8) + 234 at libc-start.c:308 -# frame #13: 0x000000000ffdc0aa clickhouse`_start + 42 - -# Limit number of files to 50, and we will get EMFILE for some of socket() -prlimit --nofile=50 $CLICKHOUSE_BENCHMARK --iterations 1 --concurrency 50 --query 'select 1' 2>&1 -echo $? diff --git a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.reference b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.reference index d43b658d5d4..f78c46e54f4 100644 --- a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.reference +++ b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.reference @@ -1,3 +1,3 @@ with_on_cluster_02250_ON_CLUSTER_grant_default without_on_cluster_02250_ON_CLUSTER_grant_default -Not enough privileges. To execute this query it's necessary to have grant CLUSTER ON *.*. (ACCESS_DENIED) +Not enough privileges. To execute this query, it's necessary to have the grant CLUSTER ON *.*. (ACCESS_DENIED) diff --git a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh index 6129d0d39c6..66417e9694a 100755 --- a/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh +++ b/tests/queries/0_stateless/02250_ON_CLUSTER_grant.sh @@ -27,5 +27,5 @@ echo "with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME" $CLICKHOUSE_CLIENT --user "with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME" -q "CREATE DATABASE IF NOT EXISTS db_with_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME ON CLUSTER test_shard_localhost" >/dev/null echo "without_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME" $CLICKHOUSE_CLIENT --user "without_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME" -q "CREATE DATABASE IF NOT EXISTS db_without_on_cluster_$CLICKHOUSE_TEST_UNIQUE_NAME ON CLUSTER test_shard_localhost" |& { - grep -m1 -F -o "Not enough privileges. To execute this query it's necessary to have grant CLUSTER ON *.*. (ACCESS_DENIED)" + grep -m1 -F -o "Not enough privileges. To execute this query, it's necessary to have the grant CLUSTER ON *.*. (ACCESS_DENIED)" } diff --git a/tests/queries/0_stateless/02263_format_insert_settings.reference b/tests/queries/0_stateless/02263_format_insert_settings.reference index 721e7960875..e2d1ec3980e 100644 --- a/tests/queries/0_stateless/02263_format_insert_settings.reference +++ b/tests/queries/0_stateless/02263_format_insert_settings.reference @@ -1,6 +1,6 @@ -insert into foo settings max_threads=1 +[multi] insert into foo settings max_threads=1 Syntax error (query): failed at position 40 (end of query): -insert into foo format tsv settings max_threads=1 +[multi] insert into foo format tsv settings max_threads=1 Can't format ASTInsertQuery with data, since data will be lost. [multi] insert into foo format tsv settings max_threads=1 INSERT INTO foo @@ -8,7 +8,7 @@ SETTINGS max_threads = 1 FORMAT tsv [oneline] insert into foo format tsv settings max_threads=1 INSERT INTO foo SETTINGS max_threads = 1 FORMAT tsv -insert into foo settings max_threads=1 format tsv settings max_threads=1 +[multi] insert into foo settings max_threads=1 format tsv settings max_threads=1 You have SETTINGS before and after FORMAT Cannot parse input: expected '\n' before: 'settings max_threads=1 1' 1 diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index efb3d39ab6c..8b156ffec83 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -8,7 +8,7 @@ function run_format() { local q="$1" && shift - echo "$q" + echo "[multi] $q" $CLICKHOUSE_FORMAT "$@" <<<"$q" } function run_format_both() @@ -22,20 +22,20 @@ function run_format_both() } # NOTE: that those queries may work slow, due to stack trace obtaining -run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):") +run_format 'insert into foo settings max_threads=1' |& grep --max-count 2 --only-matching -e "Syntax error (query): failed at position .* (end of query):" -e '^\[.*$' # compatibility -run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.") +run_format 'insert into foo format tsv settings max_threads=1' |& grep --max-count 2 --only-matching -e "Can't format ASTInsertQuery with data, since data will be lost." -e '^\[.*$' run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert -run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert |& grep --max-count 2 --only-matching -e "You have SETTINGS before and after FORMAT" -e '^\[.*$' # and via server (since this is a separate code path) $CLICKHOUSE_CLIENT -q 'drop table if exists data_02263' $CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()' -$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' 2> >(grep -m1 -F -o "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'") +$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' |& grep --max-count 1 -F --only-matching "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'" $CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 format TSV settings max_threads=1 1' $CLICKHOUSE_CLIENT -q 'select * from data_02263' -$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' |& grep --max-count 1 -F --only-matching "You have SETTINGS before and after FORMAT" $CLICKHOUSE_CLIENT -q 'drop table data_02263' run_format_both 'insert into foo values' diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 61a2e4e9f02..dec8f17874f 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -881,11 +881,18 @@ tumble tumbleEnd tumbleStart tuple +tupleConcat tupleDivide tupleDivideByNumber tupleElement tupleHammingDistance +tupleIntDiv +tupleIntDivByNumber +tupleIntDivOrZero +tupleIntDivOrZeroByNumber tupleMinus +tupleModulo +tupleModuloByNumber tupleMultiply tupleMultiplyByNumber tupleNegate diff --git a/tests/queries/0_stateless/02428_delete_with_settings.sql b/tests/queries/0_stateless/02428_delete_with_settings.sql index 071a3f74184..618c08608fc 100644 --- a/tests/queries/0_stateless/02428_delete_with_settings.sql +++ b/tests/queries/0_stateless/02428_delete_with_settings.sql @@ -1,5 +1,5 @@ drop table if exists test; -create table test (id Int32, key String) engine=MergeTree() order by tuple(); +create table test (id Int32, key String) engine=MergeTree() order by tuple() settings index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into test select number, toString(number) from numbers(1000000); delete from test where id % 2 = 0 SETTINGS mutations_sync=0; select count() from test; diff --git a/tests/queries/0_stateless/02561_temporary_table_grants.sh b/tests/queries/0_stateless/02561_temporary_table_grants.sh index 6e0c96786e8..33784f1d536 100755 --- a/tests/queries/0_stateless/02561_temporary_table_grants.sh +++ b/tests/queries/0_stateless/02561_temporary_table_grants.sh @@ -10,24 +10,24 @@ user=user_$CLICKHOUSE_TEST_UNIQUE_NAME $CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS $user" $CLICKHOUSE_CLIENT --query "CREATE USER $user IDENTIFIED WITH PLAINTEXT_PASSWORD BY 'hello'" -$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_memory_02561(name String)" 2>&1 | grep -F "Not enough privileges. To execute this query it's necessary to have grant CREATE TEMPORARY TABLE" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_memory_02561(name String)" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant CREATE TEMPORARY TABLE" > /dev/null && echo "OK" $CLICKHOUSE_CLIENT --query "GRANT CREATE TEMPORARY TABLE ON *.* TO $user" $CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_memory_02561(name String)" -$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_merge_tree_02561(name String) ENGINE = MergeTree() ORDER BY name" 2>&1 | grep -F "Not enough privileges. To execute this query it's necessary to have grant CREATE ARBITRARY TEMPORARY TABLE" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_merge_tree_02561(name String) ENGINE = MergeTree() ORDER BY name" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant CREATE ARBITRARY TEMPORARY TABLE" > /dev/null && echo "OK" $CLICKHOUSE_CLIENT --query "GRANT CREATE ARBITRARY TEMPORARY TABLE ON *.* TO $user" $CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_merge_tree_02561(name String) ENGINE = MergeTree() ORDER BY name" -$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_file_02561(name String) ENGINE = File(TabSeparated)" 2>&1 | grep -F "Not enough privileges. To execute this query it's necessary to have grant FILE" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_file_02561(name String) ENGINE = File(TabSeparated)" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant FILE" > /dev/null && echo "OK" $CLICKHOUSE_CLIENT --query "GRANT FILE ON *.* TO $user" $CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_file_02561(name String) ENGINE = File(TabSeparated)" -$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_url_02561(name String) ENGINE = URL('http://127.0.0.1:8123?query=select+12', 'RawBLOB')" 2>&1 | grep -F "Not enough privileges. To execute this query it's necessary to have grant URL" > /dev/null && echo "OK" +$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_url_02561(name String) ENGINE = URL('http://127.0.0.1:8123?query=select+12', 'RawBLOB')" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant URL" > /dev/null && echo "OK" $CLICKHOUSE_CLIENT --query "GRANT URL ON *.* TO $user" diff --git a/tests/queries/0_stateless/02681_undrop_query.reference b/tests/queries/0_stateless/02681_undrop_query.reference deleted file mode 100644 index 5df6c392eb9..00000000000 --- a/tests/queries/0_stateless/02681_undrop_query.reference +++ /dev/null @@ -1,32 +0,0 @@ -test MergeTree undrop -02681_undrop_mergetree -1 -2 -3 -test detach -UPDATE num = 2 WHERE id = 1 -test MergeTree with cluster -02681_undrop_uuid_on_cluster -1 -2 -3 -test MergeTree without uuid on cluster -02681_undrop_no_uuid_on_cluster -1 -2 -3 -test ReplicatedMergeTree undrop -02681_undrop_replicatedmergetree -1 -2 -3 -test Log undrop -02681_undrop_log -1 -2 -3 -test Distributed undrop -02681_undrop_distributed -test MergeTree drop and undrop multiple times -02681_undrop_multiple -3 diff --git a/tests/queries/0_stateless/02681_undrop_query.sql b/tests/queries/0_stateless/02681_undrop_query.sql deleted file mode 100644 index 39ca1548d53..00000000000 --- a/tests/queries/0_stateless/02681_undrop_query.sql +++ /dev/null @@ -1,90 +0,0 @@ --- Tags: no-ordinary-database, no-replicated-database, distributed, zookeeper - -set database_atomic_wait_for_drop_and_detach_synchronously = 0; -set allow_experimental_undrop_table_query = 1; - -select 'test MergeTree undrop'; -drop table if exists 02681_undrop_mergetree sync; -create table 02681_undrop_mergetree (id Int32) Engine=MergeTree() order by id; -insert into 02681_undrop_mergetree values (1),(2),(3); -drop table 02681_undrop_mergetree; -select table from system.dropped_tables where table = '02681_undrop_mergetree' limit 1; -undrop table 02681_undrop_mergetree; -select * from 02681_undrop_mergetree order by id; -drop table 02681_undrop_mergetree sync; - -select 'test detach'; -drop table if exists 02681_undrop_detach sync; -create table 02681_undrop_detach (id Int32, num Int32) Engine=MergeTree() order by id; -insert into 02681_undrop_detach values (1, 1); -detach table 02681_undrop_detach; -undrop table 02681_undrop_detach; -- { serverError 57 } -attach table 02681_undrop_detach; -alter table 02681_undrop_detach update num = 2 where id = 1; -select command from system.mutations where table='02681_undrop_detach' and database=currentDatabase() limit 1; -drop table 02681_undrop_detach sync; - -select 'test MergeTree with cluster'; -drop table if exists 02681_undrop_uuid_on_cluster on cluster test_shard_localhost sync format Null; -create table 02681_undrop_uuid_on_cluster on cluster test_shard_localhost (id Int32) Engine=MergeTree() order by id format Null; -insert into 02681_undrop_uuid_on_cluster values (1),(2),(3); -drop table 02681_undrop_uuid_on_cluster on cluster test_shard_localhost format Null; -select table from system.dropped_tables where table = '02681_undrop_uuid_on_cluster' limit 1; -undrop table 02681_undrop_uuid_on_cluster on cluster test_shard_localhost format Null; -select * from 02681_undrop_uuid_on_cluster order by id; -drop table 02681_undrop_uuid_on_cluster sync; - -select 'test MergeTree without uuid on cluster'; -drop table if exists 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost sync format Null; -create table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost (id Int32) Engine=MergeTree() order by id format Null; -insert into 02681_undrop_no_uuid_on_cluster values (1),(2),(3); -drop table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost format Null; -select table from system.dropped_tables where table = '02681_undrop_no_uuid_on_cluster' limit 1; -undrop table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost format Null; -select * from 02681_undrop_no_uuid_on_cluster order by id; -drop table 02681_undrop_no_uuid_on_cluster on cluster test_shard_localhost sync format Null; - -select 'test ReplicatedMergeTree undrop'; -drop table if exists 02681_undrop_replicatedmergetree sync; -create table 02681_undrop_replicatedmergetree (id Int32) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/02681_undrop_replicatedmergetree', 'test_undrop') order by id; -insert into 02681_undrop_replicatedmergetree values (1),(2),(3); -drop table 02681_undrop_replicatedmergetree; -select table from system.dropped_tables where table = '02681_undrop_replicatedmergetree' limit 1; -undrop table 02681_undrop_replicatedmergetree; -select * from 02681_undrop_replicatedmergetree order by id; -drop table 02681_undrop_replicatedmergetree sync; - -select 'test Log undrop'; -drop table if exists 02681_undrop_log sync; -create table 02681_undrop_log (id Int32) Engine=Log(); -insert into 02681_undrop_log values (1),(2),(3); -drop table 02681_undrop_log; -select table from system.dropped_tables where table = '02681_undrop_log' limit 1; -undrop table 02681_undrop_log; -select * from 02681_undrop_log order by id; -drop table 02681_undrop_log sync; - -select 'test Distributed undrop'; -drop table if exists 02681_undrop_distributed sync; -create table 02681_undrop_distributed (id Int32) Engine = Distributed(test_shard_localhost, currentDatabase(), 02681_undrop, rand()); -drop table 02681_undrop_distributed; -select table from system.dropped_tables where table = '02681_undrop_distributed' limit 1; -undrop table 02681_undrop_distributed; -drop table 02681_undrop_distributed sync; - -select 'test MergeTree drop and undrop multiple times'; -drop table if exists 02681_undrop_multiple sync; -create table 02681_undrop_multiple (id Int32) Engine=MergeTree() order by id; -insert into 02681_undrop_multiple values (1); -drop table 02681_undrop_multiple; -create table 02681_undrop_multiple (id Int32) Engine=MergeTree() order by id; -insert into 02681_undrop_multiple values (2); -drop table 02681_undrop_multiple; -create table 02681_undrop_multiple (id Int32) Engine=MergeTree() order by id; -insert into 02681_undrop_multiple values (3); -drop table 02681_undrop_multiple; -select table from system.dropped_tables where table = '02681_undrop_multiple' limit 1; -undrop table 02681_undrop_multiple; -select * from 02681_undrop_multiple order by id; -undrop table 02681_undrop_multiple; -- { serverError 57 } -drop table 02681_undrop_multiple sync; diff --git a/tests/queries/0_stateless/02681_undrop_query_uuid.reference b/tests/queries/0_stateless/02681_undrop_query_uuid.reference deleted file mode 100644 index beae016401b..00000000000 --- a/tests/queries/0_stateless/02681_undrop_query_uuid.reference +++ /dev/null @@ -1,6 +0,0 @@ -test MergeTree with uuid -02681_undrop_uuid -OK -1 -2 -3 diff --git a/tests/queries/0_stateless/02681_undrop_query_uuid.sh b/tests/queries/0_stateless/02681_undrop_query_uuid.sh deleted file mode 100755 index a93f30ef459..00000000000 --- a/tests/queries/0_stateless/02681_undrop_query_uuid.sh +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-ordinary-database, no-replicated-database - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -echo 'test MergeTree with uuid' -${CLICKHOUSE_CLIENT} -q "drop table if exists 02681_undrop_uuid sync;" -uuid=$(${CLICKHOUSE_CLIENT} --query "SELECT generateUUIDv4()") -uuid2=$(${CLICKHOUSE_CLIENT} --query "SELECT generateUUIDv4()") -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -q "create table 02681_undrop_uuid UUID '$uuid' on cluster test_shard_localhost (id Int32) Engine=MergeTree() order by id;" -${CLICKHOUSE_CLIENT} -q "insert into 02681_undrop_uuid values (1),(2),(3);" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -q "drop table 02681_undrop_uuid on cluster test_shard_localhost settings database_atomic_wait_for_drop_and_detach_synchronously = 0;" -${CLICKHOUSE_CLIENT} -q "select table from system.dropped_tables where table = '02681_undrop_uuid' limit 1;" -${CLICKHOUSE_CLIENT} -q "undrop table 02681_undrop_uuid UUID '$uuid2' settings allow_experimental_undrop_table_query = 1;" 2>&1| grep -Faq "UNKNOWN_TABLE" && echo OK -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -q "undrop table 02681_undrop_uuid UUID '$uuid' on cluster test_shard_localhost settings allow_experimental_undrop_table_query = 1;" -${CLICKHOUSE_CLIENT} -q "select * from 02681_undrop_uuid order by id;" -${CLICKHOUSE_CLIENT} -q "drop table 02681_undrop_uuid sync;" diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql deleted file mode 100644 index 244ba4e959a..00000000000 --- a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql +++ /dev/null @@ -1,2 +0,0 @@ -SET allow_experimental_query_cache = 0; -SET allow_experimental_query_cache = 1; diff --git a/tests/queries/0_stateless/02816_has_token_empty.reference b/tests/queries/0_stateless/02816_has_token_empty.reference index aa47d0d46d4..8435d77c5fe 100644 --- a/tests/queries/0_stateless/02816_has_token_empty.reference +++ b/tests/queries/0_stateless/02816_has_token_empty.reference @@ -1,2 +1,6 @@ 0 +\N +\N 0 +\N +\N diff --git a/tests/queries/0_stateless/02816_has_token_empty.sql b/tests/queries/0_stateless/02816_has_token_empty.sql index e5d6156debd..3e00959126b 100644 --- a/tests/queries/0_stateless/02816_has_token_empty.sql +++ b/tests/queries/0_stateless/02816_has_token_empty.sql @@ -2,6 +2,10 @@ SELECT hasTokenCaseInsensitive('K(G', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('Hello', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('', 'Hello'); +SELECT hasTokenCaseInsensitiveOrNull('Hello', ''); +SELECT hasTokenCaseInsensitiveOrNull('', ''); SELECT hasToken('Hello', ''); -- { serverError BAD_ARGUMENTS } SELECT hasToken('', 'Hello'); SELECT hasToken('', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenOrNull('', ''); +SELECT hasTokenOrNull('Hello', ''); diff --git a/tests/queries/0_stateless/02833_tuple_concat.reference b/tests/queries/0_stateless/02833_tuple_concat.reference new file mode 100644 index 00000000000..2c865f13ffc --- /dev/null +++ b/tests/queries/0_stateless/02833_tuple_concat.reference @@ -0,0 +1,6 @@ +(1,'y',2,'n') +(1,'y',2,'n',3,'n') +(1,2,3,'a','b','c','2020-10-08','2020-11-08') 1 2 3 a b c 2020-10-08 2020-11-08 +(1,2,1,2) 1 2 1 2 +(1,2,3,4) 1 2 3 4 +(3,4,1,2) 3 4 1 2 diff --git a/tests/queries/0_stateless/02833_tuple_concat.sql b/tests/queries/0_stateless/02833_tuple_concat.sql new file mode 100644 index 00000000000..df43e08d595 --- /dev/null +++ b/tests/queries/0_stateless/02833_tuple_concat.sql @@ -0,0 +1,23 @@ +SELECT tupleConcat(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tupleConcat((1, 'y'), 1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT tupleConcat((1, 'y'), (2, 'n')); +SELECT tupleConcat((1, 'y'), (2, 'n'), (3, 'n')); + +WITH (1,2,3) || ('a','b','c') || ('2020-10-08'::Date, '2020-11-08'::Date) AS t +SELECT t, t.1, t.2, t.3, t.4, t.5, t.6, t.7, t.8; + +DROP TABLE IF EXISTS t_02833; +CREATE TABLE t_02833 (tup Tuple(a UInt64, b UInt64)) ENGINE=Log; +INSERT INTO t_02833 VALUES ((1, 2)); + +WITH (tup || tup) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +WITH (tup || (3, 4)) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +WITH ((3, 4) || tup) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +DROP TABLE t_02833; diff --git a/tests/queries/0_stateless/02834_alter_exception.reference b/tests/queries/0_stateless/02834_alter_exception.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02834_alter_exception.sql b/tests/queries/0_stateless/02834_alter_exception.sql new file mode 100644 index 00000000000..d42f40fcbf7 --- /dev/null +++ b/tests/queries/0_stateless/02834_alter_exception.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS alter_02834; +CREATE TABLE alter_02834 (a UInt64) ENGINE=MergeTree() ORDER BY a; +ALTER TABLE alter_02834 MODIFY QUERY SELECT a FROM alter_02834; -- { serverError NOT_IMPLEMENTED } +DROP TABLE alter_02834; diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.reference b/tests/queries/0_stateless/02834_client_yaml_configs.reference new file mode 100644 index 00000000000..b2eddb19e52 --- /dev/null +++ b/tests/queries/0_stateless/02834_client_yaml_configs.reference @@ -0,0 +1,3 @@ +31337 +31338 +31339 diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh new file mode 100755 index 00000000000..66d3df8829e --- /dev/null +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-random-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +pushd "${CLICKHOUSE_TMP}" > /dev/null || exit + +echo "max_block_size: 31337" > clickhouse-client.yaml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.yaml + +echo "max_block_size: 31338" > clickhouse-client.yml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.yml + +echo "31339" > clickhouse-client.xml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.xml + +popd > /dev/null || exit diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference new file mode 100644 index 00000000000..0cc2e802682 --- /dev/null +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -0,0 +1,116 @@ +Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Header: id UInt64 + value_1 String + rhs.id UInt64 + rhs.value_1 String +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1_1 String : 1 + INPUT : 2 -> value_1_3 String : 2 + INPUT : 3 -> id_2 UInt64 : 3 + ALIAS id_0 :: 0 -> id UInt64 : 4 + ALIAS value_1_1 :: 1 -> value_1 String : 0 + ALIAS value_1_3 :: 2 -> rhs.value_1 String : 1 + ALIAS id_2 :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 + Join (JOIN FillRightFirst) + Header: id_0 UInt64 + value_1_1 String + value_1_3 String + id_2 UInt64 + Type: INNER + Strictness: ALL + Algorithm: HashJoin + Clauses: [(id_0) = (id_2)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_0 UInt64 + value_1_1 String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + ALIAS id :: 0 -> id_0 UInt64 : 2 + ALIAS value_1 :: 1 -> value_1_1 String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value_1 String + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_2 UInt64 + value_1_3 String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + ALIAS id :: 0 -> id_2 UInt64 : 2 + ALIAS value_1 :: 1 -> value_1_3 String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value_1 String + ReadType: Default + Parts: 1 + Granules: 1 +-- +Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Header: id UInt64 + value_1 String + rhs.id UInt64 + rhs.value_1 String +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1_1 String : 1 + INPUT :: 2 -> value_2_4 UInt64 : 2 + INPUT : 3 -> value_1_3 String : 3 + INPUT :: 4 -> value_2_5 UInt64 : 4 + INPUT : 5 -> id_2 UInt64 : 5 + ALIAS id_0 :: 0 -> id UInt64 : 6 + ALIAS value_1_1 :: 1 -> value_1 String : 0 + ALIAS value_1_3 :: 3 -> rhs.value_1 String : 1 + ALIAS id_2 :: 5 -> rhs.id UInt64 : 3 +Positions: 6 0 3 1 + Join (JOIN FillRightFirst) + Header: id_0 UInt64 + value_1_1 String + value_2_4 UInt64 + value_1_3 String + value_2_5 UInt64 + id_2 UInt64 + Type: INNER + Strictness: ASOF + Algorithm: HashJoin + ASOF inequality: LESS + Clauses: [(id_0, value_2_4) = (id_2, value_2_5)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_0 UInt64 + value_1_1 String + value_2_4 UInt64 + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 + ALIAS id :: 0 -> id_0 UInt64 : 3 + ALIAS value_1 :: 1 -> value_1_1 String : 0 + ALIAS value_2 :: 2 -> value_2_4 UInt64 : 1 + Positions: 3 0 1 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value_1 String + value_2 UInt64 + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_2 UInt64 + value_1_3 String + value_2_5 UInt64 + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 + ALIAS id :: 0 -> id_2 UInt64 : 3 + ALIAS value_1 :: 1 -> value_1_3 String : 0 + ALIAS value_2 :: 2 -> value_2_5 UInt64 : 1 + Positions: 3 0 1 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value_1 String + value_2 UInt64 + ReadType: Default + Parts: 1 + Granules: 1 diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql new file mode 100644 index 00000000000..d0475fa14b6 --- /dev/null +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -0,0 +1,31 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table_1; +CREATE TABLE test_table_1 +( + id UInt64, + value_1 String, + value_2 UInt64 +) ENGINE=MergeTree ORDER BY id; + +DROP TABLE IF EXISTS test_table_2; +CREATE TABLE test_table_2 +( + id UInt64, + value_1 String, + value_2 UInt64 +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table_1 VALUES (0, 'Value', 0); +INSERT INTO test_table_2 VALUES (0, 'Value', 0); + +EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 +FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 +FROM test_table_1 AS lhs ASOF JOIN test_table_2 AS rhs ON lhs.id = rhs.id AND lhs.value_2 < rhs.value_2; + +DROP TABLE test_table_1; +DROP TABLE test_table_2; diff --git a/tests/queries/0_stateless/02841_tuple_modulo.reference b/tests/queries/0_stateless/02841_tuple_modulo.reference new file mode 100644 index 00000000000..6e6f07d0683 --- /dev/null +++ b/tests/queries/0_stateless/02841_tuple_modulo.reference @@ -0,0 +1,4 @@ +(1,0) +(2,2) +(2,2) +(0,0) diff --git a/tests/queries/0_stateless/02841_tuple_modulo.sql b/tests/queries/0_stateless/02841_tuple_modulo.sql new file mode 100644 index 00000000000..56bacf87967 --- /dev/null +++ b/tests/queries/0_stateless/02841_tuple_modulo.sql @@ -0,0 +1,4 @@ +SELECT (5,4) % 2; +SELECT intDiv((5,4), 2); +SELECT intDivOrZero((5,4), 2); +SELECT intDivOrZero((5,4), 0); diff --git a/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference b/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference new file mode 100644 index 00000000000..6e04d969e67 --- /dev/null +++ b/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference @@ -0,0 +1,20 @@ +SELECT count() +FROM t_02848_mt1 +PREWHERE notEmpty(v) AND (k = 3) +1 +SELECT count() +FROM t_02848_mt2 +PREWHERE (d LIKE \'%es%\') AND (c < 20) AND (b = \'3\') AND (a = 3) +1 +SELECT count() +FROM t_02848_mt2 +PREWHERE (d LIKE \'%es%\') AND (c < 20) AND (c > 0) AND (a = 3) +1 +SELECT count() +FROM t_02848_mt2 +PREWHERE (d LIKE \'%es%\') AND (b = \'3\') AND (c < 20) +1 +SELECT count() +FROM t_02848_mt2 +PREWHERE (d LIKE \'%es%\') AND (b = \'3\') AND (a = 3) +1 diff --git a/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.sql b/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.sql new file mode 100644 index 00000000000..bc9d7e5664e --- /dev/null +++ b/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.sql @@ -0,0 +1,34 @@ +SET optimize_move_to_prewhere = 1; +SET enable_multiple_prewhere_read_steps = 1; + +DROP TABLE IF EXISTS t_02848_mt1; +DROP TABLE IF EXISTS t_02848_mt2; + +CREATE TABLE t_02848_mt1 (k UInt32, v String) ENGINE = MergeTree ORDER BY k SETTINGS min_bytes_for_wide_part=0; + +INSERT INTO t_02848_mt1 SELECT number, toString(number) FROM numbers(100); + +EXPLAIN SYNTAX SELECT count() FROM t_02848_mt1 WHERE k = 3 AND notEmpty(v); +SELECT count() FROM t_02848_mt1 WHERE k = 3 AND notEmpty(v); + +CREATE TABLE t_02848_mt2 (a UInt32, b String, c Int32, d String) ENGINE = MergeTree ORDER BY (a,b,c) SETTINGS min_bytes_for_wide_part=0; + +INSERT INTO t_02848_mt2 SELECT number, toString(number), number, 'aaaabbbbccccddddtestxxxyyy' FROM numbers(100); + +-- the estimated column sizes are: {a: 428, b: 318, c: 428, d: 73} +-- it's not correct but let's fix it in the future. + +EXPLAIN SYNTAX SELECT count() FROM t_02848_mt2 WHERE a = 3 AND b == '3' AND c < 20 AND d like '%es%'; +SELECT count() FROM t_02848_mt2 WHERE a = 3 AND b == '3' AND c < 20 AND d like '%es%'; + +EXPLAIN SYNTAX SELECT count() FROM t_02848_mt2 WHERE a = 3 AND c < 20 AND c > 0 AND d like '%es%'; +SELECT count() FROM t_02848_mt2 WHERE a = 3 AND c < 20 AND c > 0 AND d like '%es%'; + +EXPLAIN SYNTAX SELECT count() FROM t_02848_mt2 WHERE b == '3' AND c < 20 AND d like '%es%'; +SELECT count() FROM t_02848_mt2 WHERE b == '3' AND c < 20 AND d like '%es%'; + +EXPLAIN SYNTAX SELECT count() FROM t_02848_mt2 WHERE a = 3 AND b == '3' AND d like '%es%'; +SELECT count() FROM t_02848_mt2 WHERE a = 3 AND b == '3' AND d like '%es%'; + +DROP TABLE t_02848_mt1; +DROP TABLE t_02848_mt2; diff --git a/tests/queries/0_stateless/02843_context_has_expired.reference b/tests/queries/0_stateless/02843_context_has_expired.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02843_context_has_expired.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02843_context_has_expired.sql b/tests/queries/0_stateless/02843_context_has_expired.sql new file mode 100644 index 00000000000..ccef3458ed7 --- /dev/null +++ b/tests/queries/0_stateless/02843_context_has_expired.sql @@ -0,0 +1,23 @@ +DROP DICTIONARY IF EXISTS dict; +DROP TABLE IF EXISTS source; + +CREATE TABLE source +( + id UInt64, + value String +) +ENGINE=Memory; + +CREATE DICTIONARY dict +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'source')) +LAYOUT(DIRECT()); + +SELECT 1 IN (SELECT dictGet('dict', 'value', materialize('1'))); + +DROP DICTIONARY dict; +DROP TABLE source; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 778e554c40d..812908931ef 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2434,6 +2434,7 @@ tsv tui tumbleEnd tumbleStart +tupleConcat tupleDivide tupleDivideByNumber tupleElement