CI: Move out scripts from dockers

This commit is contained in:
Max Kainov 2024-08-09 14:22:33 +00:00 committed by Max Kainov
parent bc00f274aa
commit 9b55180184
28 changed files with 94 additions and 134 deletions

View File

@ -47,8 +47,7 @@
"docker/test/stateful": {
"name": "clickhouse/stateful-test",
"dependent": [
"docker/test/stress",
"docker/test/upgrade"
"docker/test/stress"
]
},
"docker/test/unit": {
@ -59,10 +58,6 @@
"name": "clickhouse/stress-test",
"dependent": []
},
"docker/test/upgrade": {
"name": "clickhouse/upgrade-check",
"dependent": []
},
"docker/test/integration/runner": {
"name": "clickhouse/integration-tests-runner",
"dependent": []

View File

@ -93,6 +93,3 @@ RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
ENV COMMIT_SHA=''
ENV PULL_REQUEST_NUMBER=''
ENV COPY_CLICKHOUSE_BINARY_TO_OUTPUT=0
COPY run.sh /
CMD ["/bin/bash", "/run.sh"]

View File

@ -10,7 +10,3 @@ RUN apt-get update -y \
npm \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
COPY create.sql /
COPY run.sh /
CMD ["/bin/bash", "/run.sh"]

View File

@ -1 +0,0 @@
../stateless/setup_minio.sh

View File

@ -85,18 +85,6 @@ RUN curl -L --no-verbose -O 'https://archive.apache.org/dist/hadoop/common/hadoo
ENV MINIO_ROOT_USER="clickhouse"
ENV MINIO_ROOT_PASSWORD="clickhouse"
ENV EXPORT_S3_STORAGE_POLICIES=1
ENV CLICKHOUSE_GRPC_CLIENT="/usr/share/clickhouse-utils/grpc-client/clickhouse-grpc-client.py"
RUN npm install -g azurite@3.30.0 \
&& npm install -g tslib && npm install -g node
COPY run.sh /
COPY setup_minio.sh /
COPY setup_hdfs_minicluster.sh /
COPY attach_gdb.lib /
COPY utils.lib /
# We store stress_tests.lib in stateless image to avoid duplication of this file in stress and upgrade tests
COPY stress_tests.lib /
CMD ["/bin/bash", "/run.sh"]

View File

@ -22,8 +22,5 @@ RUN apt-get update -y \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
COPY run.sh /
ENV EXPORT_S3_STORAGE_POLICIES=1
CMD ["/bin/bash", "/run.sh"]

View File

@ -1,29 +0,0 @@
# rebuild in #33610
# docker build -t clickhouse/upgrade-check .
ARG FROM_TAG=latest
FROM clickhouse/stateful-test:$FROM_TAG
RUN apt-get update -y \
&& env DEBIAN_FRONTEND=noninteractive \
apt-get install --yes --no-install-recommends \
bash \
tzdata \
parallel \
expect \
python3 \
python3-lxml \
python3-termcolor \
python3-requests \
curl \
sudo \
openssl \
netcat-openbsd \
brotli \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
COPY run.sh /
ENV EXPORT_S3_STORAGE_POLICIES=1
CMD ["/bin/bash", "/run.sh"]

View File

@ -56,7 +56,5 @@ RUN apt-get update \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
COPY process_functional_tests_result.py /
COPY --from=clickhouse/cctools:0d6b90a7a490 /opt/gdb /opt/gdb
ENV PATH="/opt/gdb/bin:${PATH}"

View File

@ -16,3 +16,4 @@ no_implicit_reexport = True
strict_equality = True
extra_checks = True
ignore_missing_imports = True
logging-fstring-interpolation = False

View File

@ -535,7 +535,10 @@ class CI:
JobNames.FAST_TEST: JobConfig(
pr_only=True,
digest=DigestConfig(
include_paths=["./tests/queries/0_stateless/"],
include_paths=[
"./tests/queries/0_stateless/",
"./tests/docker_scripts/",
],
exclude_files=[".md"],
docker=["clickhouse/fasttest"],
),

View File

@ -415,6 +415,7 @@ class CommonJobConfigs:
"./tests/clickhouse-test",
"./tests/config",
"./tests/*.txt",
"./tests/docker_scripts/",
],
exclude_files=[".md"],
docker=["clickhouse/stateless-test"],
@ -431,6 +432,7 @@ class CommonJobConfigs:
"./tests/clickhouse-test",
"./tests/config",
"./tests/*.txt",
"./tests/docker_scripts/",
],
exclude_files=[".md"],
docker=["clickhouse/stateful-test"],
@ -448,6 +450,7 @@ class CommonJobConfigs:
"./tests/clickhouse-test",
"./tests/config",
"./tests/*.txt",
"./tests/docker_scripts/",
],
exclude_files=[".md"],
docker=["clickhouse/stress-test"],
@ -459,9 +462,9 @@ class CommonJobConfigs:
UPGRADE_TEST = JobConfig(
job_name_keyword="upgrade",
digest=DigestConfig(
include_paths=["./tests/ci/upgrade_check.py"],
include_paths=["./tests/ci/upgrade_check.py", "./tests/docker_scripts/"],
exclude_files=[".md"],
docker=["clickhouse/upgrade-check"],
docker=["clickhouse/stress-test"],
),
run_command="upgrade_check.py",
runner_type=Runners.STRESS_TESTER,

View File

@ -93,7 +93,7 @@ def process_single_image(
results = [] # type: TestResults
for ver in versions:
stopwatch = Stopwatch()
for i in range(5):
for i in range(2):
success, build_log = build_and_push_one_image(
image, ver, additional_cache, push, from_tag
)

View File

@ -31,15 +31,14 @@ def get_fasttest_cmd(
"--security-opt seccomp=unconfined " # required to issue io_uring sys-calls
"--network=host " # required to get access to IAM credentials
f"-e FASTTEST_WORKSPACE=/fasttest-workspace -e FASTTEST_OUTPUT=/test_output "
f"-e FASTTEST_SOURCE=/ClickHouse "
f"-e FASTTEST_SOURCE=/repo "
f"-e FASTTEST_CMAKE_FLAGS='-DCOMPILER_CACHE=sccache' "
f"-e PULL_REQUEST_NUMBER={pr_number} -e COMMIT_SHA={commit_sha} "
f"-e COPY_CLICKHOUSE_BINARY_TO_OUTPUT=1 "
f"-e SCCACHE_BUCKET={S3_BUILDS_BUCKET} -e SCCACHE_S3_KEY_PREFIX=ccache/sccache "
"-e stage=clone_submodules "
f"--volume={workspace}:/fasttest-workspace --volume={repo_path}:/ClickHouse "
f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt "
f"--volume={output_path}:/test_output {image}"
f"--volume={workspace}:/fasttest-workspace --volume={repo_path}:/repo "
f"--volume={output_path}:/test_output {image} /repo/tests/docker_scripts/fasttest_runner.sh"
)

View File

@ -119,24 +119,24 @@ def get_run_command(
envs += [f"-e {e}" for e in additional_envs]
env_str = " ".join(envs)
volume_with_broken_test = (
f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt "
if "analyzer" not in check_name
else ""
)
if "stateful" in check_name.lower():
run_script = "/repo/tests/docker_scripts/stateful_runner.sh"
elif "stateless" in check_name.lower():
run_script = "/repo/tests/docker_scripts/stateless_runner.sh"
else:
assert False
return (
f"docker run --rm --name func-tester --volume={builds_path}:/package_folder "
# For dmesg and sysctl
"--privileged "
f"{ci_logs_args} "
f"--volume={repo_path}/tests:/usr/share/clickhouse-test "
f"--volume={repo_path}/utils/grpc-client:/usr/share/clickhouse-utils/grpc-client "
f"{volume_with_broken_test}"
f"--volume={repo_path}:/repo "
f"--volume={result_path}:/test_output "
f"--volume={server_log_path}:/var/log/clickhouse-server "
"--security-opt seccomp=unconfined " # required to issue io_uring sys-calls
f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}"
f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image} {run_script}"
)

View File

@ -57,10 +57,16 @@ def get_run_command(
additional_envs: List[str],
ci_logs_args: str,
image: DockerImage,
upgrade_check: bool,
) -> str:
envs = [f"-e {e}" for e in additional_envs]
env_str = " ".join(envs)
if upgrade_check:
run_script = "/repo/tests/docker_scripts/upgrade_runner.sh"
else:
run_script = "/repo/tests/docker_scripts/stress_runner.sh"
cmd = (
"docker run --cap-add=SYS_PTRACE "
# For dmesg and sysctl
@ -70,8 +76,8 @@ def get_run_command(
f"{ci_logs_args}"
f"--volume={build_path}:/package_folder "
f"--volume={result_path}:/test_output "
f"--volume={repo_tests_path}:/usr/share/clickhouse-test "
f"--volume={server_log_path}:/var/log/clickhouse-server {env_str} {image} "
f"--volume={repo_tests_path}/..:/repo "
f"--volume={server_log_path}:/var/log/clickhouse-server {env_str} {image} {run_script}"
)
return cmd
@ -128,7 +134,7 @@ def process_results(
return state, description, test_results, additional_files
def run_stress_test(docker_image_name: str) -> None:
def run_stress_test(upgrade_check: bool = False) -> None:
logging.basicConfig(level=logging.INFO)
for handler in logging.root.handlers:
# pylint: disable=protected-access
@ -148,7 +154,7 @@ def run_stress_test(docker_image_name: str) -> None:
pr_info = PRInfo()
docker_image = pull_image(get_docker_image(docker_image_name))
docker_image = pull_image(get_docker_image("clickhouse/stress-test"))
packages_path = temp_path / "packages"
packages_path.mkdir(parents=True, exist_ok=True)
@ -177,6 +183,7 @@ def run_stress_test(docker_image_name: str) -> None:
additional_envs,
ci_logs_args,
docker_image,
upgrade_check,
)
logging.info("Going to run stress test: %s", run_command)
@ -208,4 +215,4 @@ def run_stress_test(docker_image_name: str) -> None:
if __name__ == "__main__":
run_stress_test("clickhouse/stress-test")
run_stress_test()

View File

@ -1,4 +1,4 @@
import stress_check
if __name__ == "__main__":
stress_check.run_stress_test("clickhouse/upgrade-check")
stress_check.run_stress_test(upgrade_check=True)

View File

@ -1,7 +1,7 @@
#!/bin/bash
# shellcheck source=./utils.lib
source /utils.lib
source /repo/tests/docker_scripts/utils.lib
function attach_gdb_to_clickhouse()
{

View File

@ -325,7 +325,7 @@ case "$stage" in
;&
"run_tests")
run_tests ||:
/process_functional_tests_result.py --in-results-dir "$FASTTEST_OUTPUT/" \
/repo/tests/docker_scripts/process_functional_tests_result.py --in-results-dir "$FASTTEST_OUTPUT/" \
--out-results-file "$FASTTEST_OUTPUT/test_results.tsv" \
--out-status-file "$FASTTEST_OUTPUT/check_status.tsv" || echo -e "failure\tCannot parse results" > "$FASTTEST_OUTPUT/check_status.tsv"
;;

View File

@ -32,7 +32,7 @@ def process_test_log(log_path, broken_tests):
success_finish = False
test_results = []
test_end = True
with open(log_path, "r") as test_file:
with open(log_path, "r", encoding="utf-8") as test_file:
for line in test_file:
original_line = line
line = line.strip()
@ -150,7 +150,7 @@ def process_result(result_path, broken_tests):
if result_path and os.path.exists(result_path):
(
total,
_total,
skipped,
unknown,
failed,
@ -191,11 +191,11 @@ def process_result(result_path, broken_tests):
else:
description = ""
description += "fail: {}, passed: {}".format(failed, success)
description += f"fail: {failed}, passed: {success}"
if skipped != 0:
description += ", skipped: {}".format(skipped)
description += f", skipped: {skipped}"
if unknown != 0:
description += ", unknown: {}".format(unknown)
description += f", unknown: {unknown}"
else:
state = "failure"
description = "Output log doesn't exist"
@ -205,10 +205,10 @@ def process_result(result_path, broken_tests):
def write_results(results_file, status_file, results, status):
with open(results_file, "w") as f:
with open(results_file, "w", encoding="utf-8") as f:
out = csv.writer(f, delimiter="\t")
out.writerows(results)
with open(status_file, "w") as f:
with open(status_file, "w", encoding="utf-8") as f:
out = csv.writer(f, delimiter="\t")
out.writerow(status)
@ -221,15 +221,15 @@ if __name__ == "__main__":
parser.add_argument("--in-results-dir", default="/test_output/")
parser.add_argument("--out-results-file", default="/test_output/test_results.tsv")
parser.add_argument("--out-status-file", default="/test_output/check_status.tsv")
parser.add_argument("--broken-tests", default="/analyzer_tech_debt.txt")
parser.add_argument("--broken-tests", default="/repo/tests/analyzer_tech_debt.txt")
args = parser.parse_args()
broken_tests = list()
broken_tests = []
if os.path.exists(args.broken_tests):
logging.info(f"File {args.broken_tests} with broken tests found")
with open(args.broken_tests) as f:
print(f"File {args.broken_tests} with broken tests found")
with open(args.broken_tests, encoding="utf-8") as f:
broken_tests = f.read().splitlines()
logging.info(f"Broken tests in the list: {len(broken_tests)}")
print(f"Broken tests in the list: {len(broken_tests)}")
state, description, test_results = process_result(args.in_results_dir, broken_tests)
logging.info("Result parsed")

View File

@ -5,7 +5,7 @@ set -e -x -a -u
ls -lha
cd hadoop-3.3.1
cd /hadoop-3.3.1
export JAVA_HOME=/usr
mkdir -p target/test/data

View File

@ -143,7 +143,7 @@ main() {
fi
start_minio
setup_minio "$1"
upload_data "${query_dir}" "${2:-/usr/share/clickhouse-test}"
upload_data "${query_dir}" "${2:-/repo/tests/}"
setup_aws_credentials
}

View File

@ -14,17 +14,17 @@ dpkg -i package_folder/clickhouse-common-static-dbg_*.deb
dpkg -i package_folder/clickhouse-server_*.deb
dpkg -i package_folder/clickhouse-client_*.deb
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
ln -s /repo/tests/clickhouse-test /usr/bin/clickhouse-test
# shellcheck disable=SC1091
source /utils.lib
source /repo/tests/docker_scripts/utils.lib
# install test configs
/usr/share/clickhouse-test/config/install.sh
/repo/tests/config/install.sh
azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent --inMemoryPersistence &
./setup_minio.sh stateful
/repo/tests/docker_scripts/setup_minio.sh stateful
./mc admin trace clickminio > /test_output/minio.log &
MC_ADMIN_PID=$!
@ -105,7 +105,7 @@ setup_logs_replication
clickhouse-client --query "SHOW DATABASES"
clickhouse-client --query "CREATE DATABASE datasets"
clickhouse-client --multiquery < create.sql
clickhouse-client --multiquery < /repo/tests/docker_scripts/create.sql
clickhouse-client --query "SHOW TABLES FROM datasets"
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
@ -237,6 +237,7 @@ function run_tests()
--hung-check
--print-time
--capture-client-stacktrace
--queries "/repo/tests/queries"
"${ADDITIONAL_OPTIONS[@]}"
"$SKIP_TESTS_OPTION"
)
@ -259,7 +260,7 @@ ls -la ./
echo "Files in root directory"
ls -la /
/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv
/repo/tests/docker_scripts/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv
sudo clickhouse stop ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then

View File

@ -1,10 +1,13 @@
#!/bin/bash
# fail on errors, verbose and export all env variables
set -e -x -a
# shellcheck disable=SC1091
source /setup_export_logs.sh
# shellcheck source=../stateless/stress_tests.lib
source /stress_tests.lib
source /repo/tests/docker_scripts/stress_tests.lib
# Avoid overlaps with previous runs
dmesg --clear
@ -39,20 +42,22 @@ if [[ -z "$BUGFIX_VALIDATE_CHECK" ]]; then
chc --version || exit 1
fi
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
ln -sf /repo/tests/clickhouse-test /usr/bin/clickhouse-test
export CLICKHOUSE_GRPC_CLIENT="/repo/utils/grpc-client/clickhouse-grpc-client.py"
# shellcheck disable=SC1091
source /attach_gdb.lib
source /repo/tests/docker_scripts/attach_gdb.lib
# shellcheck disable=SC1091
source /utils.lib
source /repo/tests/docker_scripts/utils.lib
# install test configs
/usr/share/clickhouse-test/config/install.sh
/repo/tests/config/install.sh
./setup_minio.sh stateless
/repo/tests/docker_scripts/setup_minio.sh stateless
./setup_hdfs_minicluster.sh
/repo/tests/docker_scripts/setup_hdfs_minicluster.sh
config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml
@ -316,6 +321,7 @@ function run_tests()
--print-time
--no-drop-if-fail
--capture-client-stacktrace
--queries "/repo/tests/queries"
--test-runs "$NUM_TRIES"
"${ADDITIONAL_OPTIONS[@]}"
)
@ -341,7 +347,7 @@ ls -la ./
echo "Files in root directory"
ls -la /
/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv
/repo/tests/docker_scripts/process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv
clickhouse-client -q "system flush logs" ||:

View File

@ -3,26 +3,25 @@
# shellcheck disable=SC2086
# shellcheck disable=SC2024
set -x
# Avoid overlaps with previous runs
dmesg --clear
# shellcheck disable=SC1091
source /setup_export_logs.sh
set -x
# we mount tests folder from repo to /usr/share
ln -s /usr/share/clickhouse-test/ci/stress.py /usr/bin/stress
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
ln -s /repo/tests/clickhouse-test/ci/stress.py /usr/bin/stress
ln -s /repo/tests/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
# Stress tests and upgrade check uses similar code that was placed
# in a separate bash library. See tests/ci/stress_tests.lib
# shellcheck source=../stateless/attach_gdb.lib
source /attach_gdb.lib
source /repo/tests/docker_scripts/attach_gdb.lib
# shellcheck source=../stateless/stress_tests.lib
source /stress_tests.lib
source /repo/tests/docker_scripts/stress_tests.lib
# shellcheck disable=SC1091
source /utils.lib
source /repo/tests/docker_scripts/utils.lib
install_packages package_folder
@ -55,7 +54,7 @@ export ZOOKEEPER_FAULT_INJECTION=1
# available for dump via clickhouse-local
configure
./setup_minio.sh stateless # to have a proper environment
/repo/tests/docker_scripts/setup_minio.sh stateless # to have a proper environment
config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml
@ -64,7 +63,7 @@ start_server
setup_logs_replication
clickhouse-client --query "CREATE DATABASE datasets"
clickhouse-client --multiquery < create.sql
clickhouse-client --multiquery < /repo/tests/docker_scripts/create.sql
clickhouse-client --query "SHOW TABLES FROM datasets"
clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test"
@ -267,7 +266,7 @@ fi
start_server
stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \
python3 /repo/tests/ci/stress.py --hung-check --drop-databases --output-folder /test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \
&& echo -e "Test script exit code$OK" >> /test_output/test_results.tsv \
|| echo -e "Test script failed$FAIL script exit code: $?" >> /test_output/test_results.tsv

View File

@ -42,7 +42,7 @@ function configure()
# install test configs
export USE_DATABASE_ORDINARY=1
export EXPORT_S3_STORAGE_POLICIES=1
/usr/share/clickhouse-test/config/install.sh
/repo/tests/config/install.sh
# avoid too slow startup
sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \

View File

@ -9,20 +9,20 @@ dmesg --clear
set -x
# we mount tests folder from repo to /usr/share
ln -s /usr/share/clickhouse-test/ci/stress.py /usr/bin/stress
ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test
ln -s /usr/share/clickhouse-test/ci/download_release_packages.py /usr/bin/download_release_packages
ln -s /usr/share/clickhouse-test/ci/get_previous_release_tag.py /usr/bin/get_previous_release_tag
ln -s /repo/tests/ci/stress.py /usr/bin/stress
ln -s /repo/tests/clickhouse-test /usr/bin/clickhouse-test
ln -s /repo/tests/ci/download_release_packages.py /usr/bin/download_release_packages
ln -s /repo/tests/ci/get_previous_release_tag.py /usr/bin/get_previous_release_tag
# Stress tests and upgrade check uses similar code that was placed
# in a separate bash library. See tests/ci/stress_tests.lib
# shellcheck source=../stateless/attach_gdb.lib
source /attach_gdb.lib
source /repo/tests/docker_scripts/attach_gdb.lib
# shellcheck source=../stateless/stress_tests.lib
source /stress_tests.lib
source /repo/tests/docker_scripts/stress_tests.lib
azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log &
./setup_minio.sh stateless # to have a proper environment
/repo/tests/docker_scripts/setup_minio.sh stateless # to have a proper environment
echo "Get previous release tag"
# shellcheck disable=SC2016