From 8e3bd222f00324667a1d41fb62f50adc91045b6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:42:32 +0200 Subject: [PATCH 01/23] Add SQLTest --- .github/workflows/pull_request.yml | 36 +++++++ docker/images.json | 7 +- docker/test/sqltest/Dockerfile | 29 ++++++ docker/test/sqltest/run.sh | 41 ++++++++ docker/test/sqltest/test.py | 118 ++++++++++++++++++++++ tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/sqltest.py | 157 +++++++++++++++++++++++++++++ tests/ci/tests/docker_images.json | 4 + 8 files changed, 391 insertions(+), 3 deletions(-) create mode 100644 docker/test/sqltest/Dockerfile create mode 100755 docker/test/sqltest/run.sh create mode 100644 docker/test/sqltest/test.py create mode 100644 tests/ci/sqltest.py diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index dd834959578..b3283c3afbe 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5182,3 +5182,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/docker/images.json b/docker/images.json index e8fc329a640..663fa21a6c9 100644 --- a/docker/images.json +++ b/docker/images.json @@ -155,11 +155,14 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": [ - ] + "dependent": [] }, "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/sqltest": { + "name": "clickhouse/sqltest", + "dependent": [] } } diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile new file mode 100644 index 00000000000..c58907a98a8 --- /dev/null +++ b/docker/test/sqltest/Dockerfile @@ -0,0 +1,29 @@ +# 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 / +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..ef9ecaec33b --- /dev/null +++ b/docker/test/sqltest/run.sh @@ -0,0 +1,41 @@ +#!/bin/bash + +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 +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 +popd + +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.err.log diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py new file mode 100644 index 00000000000..7b38fdc52ad --- /dev/null +++ b/docker/test/sqltest/test.py @@ -0,0 +1,118 @@ +#!/usr/bin/env python3 + +import os +import yaml +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:") + summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} + + for test in yaml_content[category]: + log_file.write(test + ": " + yaml_content[category][test]) + 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) + + try: + result = client.execute(query, settings = settings) + log_file.write(result) + + 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)}") + +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%}){}{}
".format( + ' ' * indent, + name, + enable_color(success / total), + success, + total, + success / total, + reset_color, + f" - {description}" if description else '')) + + +report_html_file.write("

") + +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(4, test, test_summary['success'], test_summary['total'], test_summary['description']) + +report_html_file.write("

") 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/sqltest.py b/tests/ci/sqltest.py new file mode 100644 index 00000000000..5c20cc4849b --- /dev/null +++ b/tests/ci/sqltest.py @@ -0,0 +1,157 @@ +#!/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, + format_description, + 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, + "main.log": os.path.join(workspace_path, "main.log"), + "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/tests/docker_images.json b/tests/ci/tests/docker_images.json index 0d40d43c33f..0637058e184 100644 --- a/tests/ci/tests/docker_images.json +++ b/tests/ci/tests/docker_images.json @@ -153,5 +153,9 @@ "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/sqltest": { + "name": "clickhouse/sqltest", + "dependent": [] } } From df5abcf685a141d163e4aefc213521fc64b613ca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:45:43 +0200 Subject: [PATCH 02/23] Update Docker --- docker/test/sqltest/Dockerfile | 1 + docker/test/sqltest/test.py | 0 2 files changed, 1 insertion(+) mode change 100644 => 100755 docker/test/sqltest/test.py diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile index c58907a98a8..437677f4fd1 100644 --- a/docker/test/sqltest/Dockerfile +++ b/docker/test/sqltest/Dockerfile @@ -26,4 +26,5 @@ 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/test.py b/docker/test/sqltest/test.py old mode 100644 new mode 100755 From 174166b23382488f1a6fc12cbf121b1a1da7f49b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:47:41 +0200 Subject: [PATCH 03/23] Fix error --- docker/test/sqltest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index ef9ecaec33b..b26d4d89f8b 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -34,7 +34,7 @@ done # Run the test pushd sqltest/standards/2016/ -./test.py +/test.py popd zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log From fcf4a6ea887f957b8d911b1611932ecc59260b7a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:48:48 +0200 Subject: [PATCH 04/23] Fix error --- docker/test/sqltest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index b26d4d89f8b..24032874403 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -35,6 +35,7 @@ done # Run the test pushd sqltest/standards/2016/ /test.py +mv report.html test.log / popd zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log From 2255307a477ab86dc7740537a31e4791f3e769a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:49:47 +0200 Subject: [PATCH 05/23] Improvement --- docker/test/sqltest/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index 7b38fdc52ad..945446358c3 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -95,7 +95,7 @@ def print_ratio(indent, name, success, total, description): f" - {description}" if description else '')) -report_html_file.write("

") +report_html_file.write("

")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -115,4 +115,4 @@ for category in summary['results']:
 
         print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
-report_html_file.write("

") +report_html_file.write("
") From 50dac21483b9003ad546da6d87340124bf108d69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:54:09 +0200 Subject: [PATCH 06/23] Improvement --- docker/test/sqltest/test.py | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index 945446358c3..ae2d6406c78 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -2,6 +2,7 @@ import os import yaml +import html import random import string from clickhouse_driver import Client @@ -26,11 +27,11 @@ with open('features.yml', 'r') as file: yaml_content = yaml.safe_load(file) for category in yaml_content: - log_file.write(category.capitalize() + " features:") + 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]) + 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" @@ -54,11 +55,11 @@ with open('features.yml', 'r') as file: summary['results'][category]['total'] += 1 summary['total'] += 1 - log_file.write(query) + log_file.write(query + "\n") try: result = client.execute(query, settings = settings) - log_file.write(result) + log_file.write(result + "\n") if test_group: summary['results'][category]['results'][test_group]['success'] += 1 @@ -67,7 +68,7 @@ with open('features.yml', 'r') as file: summary['success'] += 1 except Exception as e: - log_file.write(f"Error occurred: {str(e)}") + log_file.write(f"Error occurred: {str(e)}\n") client.execute(f"DROP DATABASE {database_name}", settings = settings) @@ -84,7 +85,7 @@ def enable_color(ratio): reset_color = "
" def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
".format( + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
\n".format( ' ' * indent, name, enable_color(success / total), @@ -92,10 +93,10 @@ def print_ratio(indent, name, success, total, description): total, success / total, reset_color, - f" - {description}" if description else '')) + f" - " + html.escape(description) if description else '')) -report_html_file.write("
")
+report_html_file.write("
\n")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -115,4 +116,4 @@ for category in summary['results']:
 
         print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
-report_html_file.write("
") +report_html_file.write("
\n") From 76677e009b8cf9a501fde1f7273fa5d546f4ffa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:00:50 +0200 Subject: [PATCH 07/23] Improvement --- docker/test/sqltest/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index ae2d6406c78..cf2f976e05a 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -59,7 +59,7 @@ with open('features.yml', 'r') as file: try: result = client.execute(query, settings = settings) - log_file.write(result + "\n") + log_file.write(str(result) + "\n") if test_group: summary['results'][category]['results'][test_group]['success'] += 1 From fe5972953703edb8a7e13c6066e5af369359b82c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:01:00 +0200 Subject: [PATCH 08/23] Improvement --- docker/test/sqltest/run.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 24032874403..03678971f60 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -23,6 +23,12 @@ function wget_with_retry 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 From 9b10e0a5bc2b7e3f84614d1aa3fa84c60de76929 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:09:19 +0200 Subject: [PATCH 09/23] Improvement --- docker/test/sqltest/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index cf2f976e05a..ff321d3e5df 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -85,9 +85,9 @@ def enable_color(ratio): reset_color = "
" def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
\n".format( + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}\n".format( ' ' * indent, - name, + name.capitalize(), enable_color(success / total), success, total, @@ -96,7 +96,7 @@ def print_ratio(indent, name, success, total, description): f" - " + html.escape(description) if description else '')) -report_html_file.write("
\n")
+report_html_file.write("
\n")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -114,6 +114,6 @@ for category in summary['results']:
         if test_summary['total'] == 0:
             continue
 
-        print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
+        print_ratio(6 if '-' in test else 4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
 report_html_file.write("
\n") From 891c01e6ab30c2d4377a118cb4c449134a56e17a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 02:37:02 +0000 Subject: [PATCH 10/23] Automatic style fix --- docker/test/sqltest/test.py | 119 ++++++++++++++++++++++-------------- 1 file changed, 74 insertions(+), 45 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index ff321d3e5df..5807ca79b02 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -8,69 +8,85 @@ 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} +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)) +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.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) +client = Client(host="localhost", port=9000, database=database_name) -summary = {'success': 0, 'total': 0, 'results': {}} +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: +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': {}} + 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]} + 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) + 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']; + 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 = "" + 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 + 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) + 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 + 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) +client.execute(f"DROP DATABASE {database_name}", settings=settings) + def enable_color(ratio): if ratio == 0: @@ -82,38 +98,51 @@ def enable_color(ratio): 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( + "{}{}: {}{} / {} ({:.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")
+report_html_file.write(
+    "
\n"
+)
 
-print_ratio(0, 'Total', summary['success'], summary['total'], '')
+print_ratio(0, "Total", summary["success"], summary["total"], "")
 
-for category in summary['results']:
-    cat_summary = summary['results'][category]
+for category in summary["results"]:
+    cat_summary = summary["results"][category]
 
-    if cat_summary['total'] == 0:
+    if cat_summary["total"] == 0:
         continue
 
-    print_ratio(2, category, cat_summary['success'], cat_summary['total'], '')
+    print_ratio(2, category, cat_summary["success"], cat_summary["total"], "")
 
-    for test in summary['results'][category]['results']:
-        test_summary = summary['results'][category]['results'][test]
+    for test in summary["results"][category]["results"]:
+        test_summary = summary["results"][category]["results"][test]
 
-        if test_summary['total'] == 0:
+        if test_summary["total"] == 0:
             continue
 
-        print_ratio(6 if '-' in test else 4, test, test_summary['success'], test_summary['total'], test_summary['description'])
+        print_ratio(
+            6 if "-" in test else 4,
+            test,
+            test_summary["success"],
+            test_summary["total"],
+            test_summary["description"],
+        )
 
 report_html_file.write("
\n") From 29094a22cf4cc787cc50f5d494b29dcf511cc765 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 20:00:00 +0200 Subject: [PATCH 11/23] Fix Docker --- docker/images.json | 1 + tests/ci/tests/docker_images.json | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/images.json b/docker/images.json index 663fa21a6c9..8e19401ba72 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" ] }, diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images.json index 0637058e184..70db8760561 100644 --- a/tests/ci/tests/docker_images.json +++ b/tests/ci/tests/docker_images.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": { From f458108c44f2d9f0c92725f05d3959658c97ee6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 22:15:23 +0200 Subject: [PATCH 12/23] Fix style --- docker/test/sqllogic/run.sh | 2 ++ tests/ci/sqltest.py | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 444252837a3..3b900c097e2 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -1,4 +1,6 @@ #!/bin/bash +# shellcheck disable=SC2015 + set -exu trap "exit" INT TERM diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 5c20cc4849b..b752d4e4aee 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -11,7 +11,6 @@ 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, - format_description, get_commit, post_commit_status, ) From fa44f84377bc097cbe0c25852603de9be536f75e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jul 2023 04:50:12 +0200 Subject: [PATCH 13/23] Fix style --- docker/test/sqllogic/run.sh | 1 - docker/test/sqltest/run.sh | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 3b900c097e2..5be44fc148c 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -1,5 +1,4 @@ #!/bin/bash -# shellcheck disable=SC2015 set -exu trap "exit" INT TERM diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 03678971f60..42aeef9df15 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -1,4 +1,5 @@ #!/bin/bash +# shellcheck disable=SC2015 set -x set -e From 3398355e4649beaa2b0a3180aa687c364c245e81 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 02:04:30 +0200 Subject: [PATCH 14/23] Add something with unclear purpose --- docker/README.md | 2 +- tests/ci/docker_test.py | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/README.md b/docker/README.md index ec52ddd143e..76fdbf0f3f0 100644 --- a/docker/README.md +++ b/docker/README.md @@ -2,4 +2,4 @@ This directory contain Dockerfiles for `clickhouse-client` and `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/tests/ci/docker_test.py b/tests/ci/docker_test.py index d5d27f73694..61319041b9f 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -40,6 +40,7 @@ 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", True), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", From c3c8ea9c27669a548e2404e2ed6c2e8f3c734dc9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 17:30:55 +0200 Subject: [PATCH 15/23] Do something --- tests/ci/docker_test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 61319041b9f..8542de412d2 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -40,7 +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", True), + di.DockerImage( + "docker/test/sqltest", + "clickhouse/sqltest", + True, + "clickhouse/test-base", # type: ignore + ), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", From 8646eefc9cfeedc840c14bc7571c0c6490679a4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:05:24 +0300 Subject: [PATCH 16/23] Update docker_test.py --- tests/ci/docker_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 8542de412d2..bdbee92d2d3 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -43,7 +43,7 @@ class TestDockerImageCheck(unittest.TestCase): di.DockerImage( "docker/test/sqltest", "clickhouse/sqltest", - True, + False, "clickhouse/test-base", # type: ignore ), di.DockerImage( From 0641dfd7e640bfd08d8533d41cb8ada524dd7b3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 22:33:04 +0200 Subject: [PATCH 17/23] Add missing modification --- tests/ci/ci_config.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 517e40fd2d6..be8364f8874 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -402,6 +402,9 @@ CI_CONFIG = { "Sqllogic test (release)": { "required_build": "package_release", }, + "SQLTest": { + "required_build": "package_release", + }, }, } # type: dict From b4f0d0bc2edb5865128e16dd90db00d25df8747e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 21:05:39 +0300 Subject: [PATCH 18/23] Update pull_request.yml --- .github/workflows/pull_request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 1d515633f61..6ec8c003491 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5194,7 +5194,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/sqltest REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME="SQLTest" + CHECK_NAME=SQLTest REPO_COPY=${{runner.temp}}/sqltest/ClickHouse EOF - name: Download json reports From 5ec6a4695f2911758e8d64ee52e34149c4940112 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 23:22:15 +0200 Subject: [PATCH 19/23] Fix log paths --- tests/ci/sqltest.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index b752d4e4aee..7385716850f 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,11 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "main.log": os.path.join(workspace_path, "main.log"), - "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"), + "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/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() From 619af0c2c66255e8c8aa4f8fd62b96c94fe35ee3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 2 Aug 2023 21:46:23 +0000 Subject: [PATCH 20/23] Automatic style fix --- tests/ci/sqltest.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 7385716850f..498448a7928 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,8 +110,12 @@ def main(): 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, "/var/log/clickhouse-server/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), + "server.log.zst": os.path.join( + workspace_path, "/var/log/clickhouse-server/server.log.zst" + ), + "server.err.log.zst": os.path.join( + workspace_path, "/var/log/clickhouse-server/server.err.log.zst" + ), "report.html": os.path.join(workspace_path, "/report.html"), "test.log": os.path.join(workspace_path, "/test.log"), } From 65e8ee8fb7d30b0765c8bfd3d58d9def64e002bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 15:28:18 +0200 Subject: [PATCH 21/23] Fix paths --- docker/test/sqltest/run.sh | 4 +++- tests/ci/sqltest.py | 8 ++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 42aeef9df15..cba1c1dab1f 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -42,8 +42,10 @@ done # Run the test pushd sqltest/standards/2016/ /test.py -mv report.html test.log / +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/tests/ci/sqltest.py b/tests/ci/sqltest.py index 7385716850f..d76baf22bb3 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,10 +110,10 @@ def main(): 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, "/var/log/clickhouse-server/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), - "report.html": os.path.join(workspace_path, "/report.html"), - "test.log": os.path.join(workspace_path, "/test.log"), + "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), + "report.html": os.path.join(workspace_path, "/workspace/report.html"), + "test.log": os.path.join(workspace_path, "/workspace/test.log"), } s3_helper = S3Helper() From a96b0457e04543876b35ca1a5223ea6007bbf9bb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 13:43:45 +0000 Subject: [PATCH 22/23] Automatic style fix --- tests/ci/sqltest.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index d76baf22bb3..69af21f1761 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -111,7 +111,9 @@ def main(): paths = { "run.log": run_log_path, "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), + "server.err.log.zst": os.path.join( + workspace_path, "/workspace/server.err.log.zst" + ), "report.html": os.path.join(workspace_path, "/workspace/report.html"), "test.log": os.path.join(workspace_path, "/workspace/test.log"), } From 388feb953d97b244b162c433e2983cbe4e2d73f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 19:28:59 +0200 Subject: [PATCH 23/23] Fix paths --- tests/ci/sqltest.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index d76baf22bb3..be22a1c9312 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,10 +110,10 @@ def main(): 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, "/workspace/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), - "report.html": os.path.join(workspace_path, "/workspace/report.html"), - "test.log": os.path.join(workspace_path, "/workspace/test.log"), + "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()