CI: Stateless Tests with praktika

This commit is contained in:
Max Kainov 2024-10-29 21:09:03 +00:00
parent aeffae571c
commit e198b20509
29 changed files with 955 additions and 305 deletions

View File

@ -30,6 +30,9 @@ jobs:
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |
@ -68,6 +71,9 @@ jobs:
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |
@ -106,6 +112,9 @@ jobs:
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |
@ -144,6 +153,9 @@ jobs:
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |
@ -172,16 +184,19 @@ jobs:
python3 -m praktika run --job '''Fast test''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log
fi
build_amd64_debug:
build_amd_debug:
runs-on: [builder]
needs: [config_workflow, docker_builds]
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgYW1kNjQgZGVidWc=') }}
name: "Build amd64 debug"
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9kZWJ1Zyk=') }}
name: "Build (amd_debug)"
outputs:
data: ${{ steps.run.outputs.DATA }}
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |
@ -205,21 +220,24 @@ jobs:
. /tmp/praktika_setup_env.sh
set -o pipefail
if command -v ts &> /dev/null; then
python3 -m praktika run --job '''Build amd64 debug''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log
python3 -m praktika run --job '''Build (amd_debug)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log
else
python3 -m praktika run --job '''Build amd64 debug''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log
python3 -m praktika run --job '''Build (amd_debug)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log
fi
stateless_tests_amd_debug:
build_amd_release:
runs-on: [builder]
needs: [config_workflow, docker_builds, build_amd64_debug]
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWQsIGRlYnVnKQ==') }}
name: "Stateless tests (amd, debug)"
needs: [config_workflow, docker_builds]
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9yZWxlYXNlKQ==') }}
name: "Build (amd_release)"
outputs:
data: ${{ steps.run.outputs.DATA }}
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |
@ -243,14 +261,137 @@ jobs:
. /tmp/praktika_setup_env.sh
set -o pipefail
if command -v ts &> /dev/null; then
python3 -m praktika run --job '''Stateless tests (amd, debug)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log
python3 -m praktika run --job '''Build (amd_release)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log
else
python3 -m praktika run --job '''Stateless tests (amd, debug)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log
python3 -m praktika run --job '''Build (amd_release)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log
fi
stateless_tests_amd_debug_parallel_1_2:
runs-on: [builder]
needs: [config_workflow, docker_builds, build_amd_debug]
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWQsIGRlYnVnKSAocGFyYWxsZWwgMS8yKQ==') }}
name: "Stateless tests (amd, debug) (parallel 1/2)"
outputs:
data: ${{ steps.run.outputs.DATA }}
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |
cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF'
export PYTHONPATH=./ci:.
cat > /tmp/praktika/workflow_config_pr.json << 'EOF'
${{ needs.config_workflow.outputs.data }}
EOF
cat > /tmp/praktika/workflow_status.json << 'EOF'
${{ toJson(needs) }}
EOF
ENV_SETUP_SCRIPT_EOF
rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika
mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output
- name: Run
id: run
run: |
. /tmp/praktika_setup_env.sh
set -o pipefail
if command -v ts &> /dev/null; then
python3 -m praktika run --job '''Stateless tests (amd, debug) (parallel 1/2)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log
else
python3 -m praktika run --job '''Stateless tests (amd, debug) (parallel 1/2)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log
fi
stateless_tests_amd_debug_parallel_2_2:
runs-on: [builder]
needs: [config_workflow, docker_builds, build_amd_debug]
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWQsIGRlYnVnKSAocGFyYWxsZWwgMi8yKQ==') }}
name: "Stateless tests (amd, debug) (parallel 2/2)"
outputs:
data: ${{ steps.run.outputs.DATA }}
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |
cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF'
export PYTHONPATH=./ci:.
cat > /tmp/praktika/workflow_config_pr.json << 'EOF'
${{ needs.config_workflow.outputs.data }}
EOF
cat > /tmp/praktika/workflow_status.json << 'EOF'
${{ toJson(needs) }}
EOF
ENV_SETUP_SCRIPT_EOF
rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika
mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output
- name: Run
id: run
run: |
. /tmp/praktika_setup_env.sh
set -o pipefail
if command -v ts &> /dev/null; then
python3 -m praktika run --job '''Stateless tests (amd, debug) (parallel 2/2)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log
else
python3 -m praktika run --job '''Stateless tests (amd, debug) (parallel 2/2)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log
fi
stateless_tests_amd_debug_non_parallel:
runs-on: [style-checker]
needs: [config_workflow, docker_builds, build_amd_debug]
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWQsIGRlYnVnKSAobm9uLXBhcmFsbGVsKQ==') }}
name: "Stateless tests (amd, debug) (non-parallel)"
outputs:
data: ${{ steps.run.outputs.DATA }}
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |
cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF'
export PYTHONPATH=./ci:.
cat > /tmp/praktika/workflow_config_pr.json << 'EOF'
${{ needs.config_workflow.outputs.data }}
EOF
cat > /tmp/praktika/workflow_status.json << 'EOF'
${{ toJson(needs) }}
EOF
ENV_SETUP_SCRIPT_EOF
rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika
mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output
- name: Run
id: run
run: |
. /tmp/praktika_setup_env.sh
set -o pipefail
if command -v ts &> /dev/null; then
python3 -m praktika run --job '''Stateless tests (amd, debug) (non-parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log
else
python3 -m praktika run --job '''Stateless tests (amd, debug) (non-parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log
fi
finish_workflow:
runs-on: [ci_services]
needs: [config_workflow, docker_builds, style_check, fast_test, build_amd64_debug, stateless_tests_amd_debug]
needs: [config_workflow, docker_builds, style_check, fast_test, build_amd_debug, build_amd_release, stateless_tests_amd_debug_parallel_1_2, stateless_tests_amd_debug_parallel_2_2, stateless_tests_amd_debug_non_parallel]
if: ${{ !cancelled() }}
name: "Finish Workflow"
outputs:
@ -258,6 +399,9 @@ jobs:
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script
run: |

0
ci/__init__.py Normal file
View File

View File

@ -0,0 +1,107 @@
# docker build -t clickhouse/stateless-test .
FROM ubuntu:22.04
# ARG for quick switch to a given ubuntu mirror
ARG apt_archive="http://archive.ubuntu.com"
RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list
ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.6.20200320/clickhouse-odbc-1.1.6-Linux.tar.gz"
# moreutils - provides ts fo FT
# expect, bzip2 - requried by FT
# bsdmainutils - provides hexdump for FT
# golang version 1.13 on Ubuntu 20 is enough for tests
RUN apt-get update -y \
&& env DEBIAN_FRONTEND=noninteractive \
apt-get install --yes --no-install-recommends \
awscli \
brotli \
lz4 \
expect \
moreutils \
bzip2 \
bsdmainutils \
golang \
lsof \
mysql-client=8.0* \
ncdu \
netcat-openbsd \
nodejs \
npm \
odbcinst \
openjdk-11-jre-headless \
openssl \
postgresql-client \
python3 \
python3-pip \
qemu-user-static \
sqlite3 \
sudo \
tree \
unixodbc \
rustc \
cargo \
zstd \
file \
jq \
pv \
zip \
unzip \
p7zip-full \
curl \
wget \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
ARG PROTOC_VERSION=25.1
RUN curl -OL https://github.com/protocolbuffers/protobuf/releases/download/v${PROTOC_VERSION}/protoc-${PROTOC_VERSION}-linux-x86_64.zip \
&& unzip protoc-${PROTOC_VERSION}-linux-x86_64.zip -d /usr/local \
&& rm protoc-${PROTOC_VERSION}-linux-x86_64.zip
COPY requirements.txt /
RUN pip3 install --no-cache-dir -r /requirements.txt
RUN mkdir -p /tmp/clickhouse-odbc-tmp \
&& cd /tmp/clickhouse-odbc-tmp \
&& curl -L ${odbc_driver_url} | tar --strip-components=1 -xz clickhouse-odbc-1.1.6-Linux \
&& mkdir /usr/local/lib64 -p \
&& cp /tmp/clickhouse-odbc-tmp/lib64/*.so /usr/local/lib64/ \
&& odbcinst -i -d -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbcinst.ini.sample \
&& odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \
&& sed -i 's"=libclickhouseodbc"=/usr/local/lib64/libclickhouseodbc"' /etc/odbcinst.ini \
&& rm -rf /tmp/clickhouse-odbc-tmp
ENV TZ=Europe/Amsterdam
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
ENV NUM_TRIES=1
# Unrelated to vars in setup_minio.sh, but should be the same there
# to have the same binaries for local running scenario
ARG MINIO_SERVER_VERSION=2024-08-03T04-33-23Z
ARG MINIO_CLIENT_VERSION=2024-07-31T15-58-33Z
ARG TARGETARCH
# Download Minio-related binaries
RUN arch=${TARGETARCH:-amd64} \
&& curl -L "https://dl.min.io/server/minio/release/linux-${arch}/archive/minio.RELEASE.${MINIO_SERVER_VERSION}" -o /minio \
&& curl -L "https://dl.min.io/client/mc/release/linux-${arch}/archive/mc.RELEASE.${MINIO_CLIENT_VERSION}" -o /mc \
&& chmod +x /mc /minio
ENV MINIO_ROOT_USER="clickhouse"
ENV MINIO_ROOT_PASSWORD="clickhouse"
# for minio to work without root
RUN chmod 777 /home
ENV HOME="/home"
ENV TEMP_DIR="/tmp/praktika"
ENV PATH="/wd/tests:/tmp/praktika/input:$PATH"
RUN curl -L --no-verbose -O 'https://archive.apache.org/dist/hadoop/common/hadoop-3.3.1/hadoop-3.3.1.tar.gz' \
&& tar -xvf hadoop-3.3.1.tar.gz \
&& rm -rf hadoop-3.3.1.tar.gz
RUN npm install -g azurite@3.30.0 \
&& npm install -g tslib && npm install -g node

View File

@ -0,0 +1,5 @@
Jinja2==3.1.3
numpy==1.26.4
requests==2.32.3
pandas==1.5.3
scipy==1.12.0

0
ci/jobs/__init__.py Normal file
View File

View File

@ -1,5 +1,6 @@
import argparse
from praktika.param import get_param
from praktika.result import Result
from praktika.settings import Settings
from praktika.utils import MetaClasses, Shell, Utils
@ -14,7 +15,9 @@ class JobStages(metaclass=MetaClasses.WithIter):
def parse_args():
parser = argparse.ArgumentParser(description="ClickHouse Build Job")
parser.add_argument(
"BUILD_TYPE", help="Type: <amd|arm>_<debug|release>_<asan|msan|..>"
"--build-type",
help="Type: <amd|arm>_<debug|release>_<asan|msan|..>",
default=None,
)
parser.add_argument(
"--param",
@ -24,6 +27,18 @@ def parse_args():
return parser.parse_args()
CMAKE_CMD = """cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA \
-DCMAKE_BUILD_TYPE={BUILD_TYPE} \
-DSANITIZE={SANITIZER} \
-DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DENABLE_TESTS=0 \
-DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr \
-DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON \
{AUX_DEFS} \
-DCMAKE_C_COMPILER=clang-18 -DCMAKE_CXX_COMPILER=clang++-18 \
-DCOMPILER_CACHE={CACHE_TYPE} \
-DENABLE_BUILD_PROFILING=1 {DIR}"""
def main():
args = parse_args()
@ -42,20 +57,45 @@ def main():
cmake_build_type = "Release"
sanitizer = ""
if "debug" in args.BUILD_TYPE.lower():
print("Build type set: debug")
cmake_build_type = "Debug"
if args.build_type and get_param():
assert (
False
), "Build type must provided via job parameter (CI case) or via --build-type input argument not both"
if "asan" in args.BUILD_TYPE.lower():
print("Sanitizer set: address")
sanitizer = "address"
build_type = args.build_type or get_param()
assert (
build_type
), "build_type must be provided either as input argument or as a parameter of parametrized job in CI"
build_type = build_type.lower()
# if Environment.is_local_run():
# build_cache_type = "disabled"
# else:
build_cache_type = "sccache"
CACHE_TYPE = "sccache"
if "debug" in build_type:
print("Build type set: debug")
BUILD_TYPE = "Debug"
AUX_DEFS = " -DSPLIT_DEBUG_SYMBOLS=ON -DBUILD_STANDALONE_KEEPER=1 "
elif "release" in build_type:
print("Build type set: release")
BUILD_TYPE = "None"
AUX_DEFS = " -DENABLE_TESTS=1 "
if "asan" in build_type:
print("Sanitizer set: address")
SANITIZER = "address"
else:
SANITIZER = ""
cmake_cmd = CMAKE_CMD.format(
BUILD_TYPE=BUILD_TYPE,
CACHE_TYPE=CACHE_TYPE,
SANITIZER=SANITIZER,
AUX_DEFS=AUX_DEFS,
DIR=Utils.cwd(),
)
current_directory = Utils.cwd()
build_dir = f"{Settings.TEMP_DIR}/build"
res = True
@ -75,12 +115,7 @@ def main():
results.append(
Result.create_from_command_execution(
name="Cmake configuration",
command=f"cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE={cmake_build_type} \
-DSANITIZE={sanitizer} -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DENABLE_TESTS=0 \
-DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr \
-DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON \
-DCMAKE_C_COMPILER=clang-18 -DCMAKE_CXX_COMPILER=clang++-18 -DCOMPILER_CACHE={build_cache_type} -DENABLE_TESTS=1 \
-DENABLE_BUILD_PROFILING=1 {current_directory}",
command=cmake_cmd,
workdir=build_dir,
with_log=True,
)

View File

@ -1,120 +1,13 @@
import argparse
import threading
from pathlib import Path
from praktika.result import Result
from praktika.settings import Settings
from praktika.utils import MetaClasses, Shell, Utils
from ci.jobs.scripts.clickhouse_proc import ClickHouseProc
from ci.jobs.scripts.functional_tests_results import FTResultsProcessor
class ClickHouseProc:
def __init__(self):
self.ch_config_dir = f"{Settings.TEMP_DIR}/etc/clickhouse-server"
self.pid_file = f"{self.ch_config_dir}/clickhouse-server.pid"
self.config_file = f"{self.ch_config_dir}/config.xml"
self.user_files_path = f"{self.ch_config_dir}/user_files"
self.test_output_file = f"{Settings.OUTPUT_DIR}/test_result.txt"
self.command = f"clickhouse-server --config-file {self.config_file} --pid-file {self.pid_file} -- --path {self.ch_config_dir} --user_files_path {self.user_files_path} --top_level_domains_path {self.ch_config_dir}/top_level_domains --keeper_server.storage_path {self.ch_config_dir}/coordination"
self.proc = None
self.pid = 0
nproc = int(Utils.cpu_count() / 2)
self.fast_test_command = f"clickhouse-test --hung-check --fast-tests-only --no-random-settings --no-random-merge-tree-settings --no-long --testname --shard --zookeeper --check-zookeeper-session --order random --print-time --report-logs-stats --jobs {nproc} -- '' | ts '%Y-%m-%d %H:%M:%S' \
| tee -a \"{self.test_output_file}\""
# TODO: store info in case of failure
self.info = ""
self.info_file = ""
Utils.set_env("CLICKHOUSE_CONFIG_DIR", self.ch_config_dir)
Utils.set_env("CLICKHOUSE_CONFIG", self.config_file)
Utils.set_env("CLICKHOUSE_USER_FILES", self.user_files_path)
Utils.set_env("CLICKHOUSE_SCHEMA_FILES", f"{self.ch_config_dir}/format_schemas")
def start(self):
print("Starting ClickHouse server")
Shell.check(f"rm {self.pid_file}")
def run_clickhouse():
self.proc = Shell.run_async(
self.command, verbose=True, suppress_output=True
)
thread = threading.Thread(target=run_clickhouse)
thread.daemon = True # Allow program to exit even if thread is still running
thread.start()
# self.proc = Shell.run_async(self.command, verbose=True)
started = False
try:
for _ in range(5):
pid = Shell.get_output(f"cat {self.pid_file}").strip()
if not pid:
Utils.sleep(1)
continue
started = True
print(f"Got pid from fs [{pid}]")
_ = int(pid)
break
except Exception:
pass
if not started:
stdout = self.proc.stdout.read().strip() if self.proc.stdout else ""
stderr = self.proc.stderr.read().strip() if self.proc.stderr else ""
Utils.print_formatted_error("Failed to start ClickHouse", stdout, stderr)
return False
print(f"ClickHouse server started successfully, pid [{pid}]")
return True
def wait_ready(self):
res, out, err = 0, "", ""
attempts = 30
delay = 2
for attempt in range(attempts):
res, out, err = Shell.get_res_stdout_stderr(
'clickhouse-client --query "select 1"', verbose=True
)
if out.strip() == "1":
print("Server ready")
break
else:
print(f"Server not ready, wait")
Utils.sleep(delay)
else:
Utils.print_formatted_error(
f"Server not ready after [{attempts*delay}s]", out, err
)
return False
return True
def run_fast_test(self):
if Path(self.test_output_file).exists():
Path(self.test_output_file).unlink()
exit_code = Shell.run(self.fast_test_command)
return exit_code == 0
def terminate(self):
print("Terminate ClickHouse process")
timeout = 10
if self.proc:
Utils.terminate_process_group(self.proc.pid)
self.proc.terminate()
try:
self.proc.wait(timeout=10)
print(f"Process {self.proc.pid} terminated gracefully.")
except Exception:
print(
f"Process {self.proc.pid} did not terminate in {timeout} seconds, killing it..."
)
Utils.terminate_process_group(self.proc.pid, force=True)
self.proc.wait() # Wait for the process to be fully killed
print(f"Process {self.proc} was killed.")
def clone_submodules():
submodules_to_update = [
"contrib/sysroot",
@ -240,7 +133,7 @@ def main():
Shell.check(f"rm -rf {build_dir} && mkdir -p {build_dir}")
results.append(
Result.create_from_command_execution(
name="Checkout Submodules for Minimal Build",
name="Checkout Submodules",
command=clone_submodules,
)
)
@ -295,8 +188,8 @@ def main():
if res and JobStages.CONFIG in stages:
commands = [
f"rm -rf {Settings.TEMP_DIR}/etc/ && mkdir -p {Settings.TEMP_DIR}/etc/clickhouse-client {Settings.TEMP_DIR}/etc/clickhouse-server",
f"cp {current_directory}/programs/server/config.xml {current_directory}/programs/server/users.xml {Settings.TEMP_DIR}/etc/clickhouse-server/",
f"{current_directory}/tests/config/install.sh {Settings.TEMP_DIR}/etc/clickhouse-server {Settings.TEMP_DIR}/etc/clickhouse-client",
f"cp ./programs/server/config.xml ./programs/server/users.xml {Settings.TEMP_DIR}/etc/clickhouse-server/",
f"./tests/config/install.sh {Settings.TEMP_DIR}/etc/clickhouse-server {Settings.TEMP_DIR}/etc/clickhouse-client --fast-test",
# f"cp -a {current_directory}/programs/server/config.d/log_to_console.xml {Settings.TEMP_DIR}/etc/clickhouse-server/config.d/",
f"rm -f {Settings.TEMP_DIR}/etc/clickhouse-server/config.d/secure_ports.xml",
update_path_ch_config,
@ -310,7 +203,7 @@ def main():
)
res = results[-1].is_ok()
CH = ClickHouseProc()
CH = ClickHouseProc(fast_test=True)
if res and JobStages.TEST in stages:
stop_watch_ = Utils.Stopwatch()
step_name = "Start ClickHouse Server"

View File

@ -1,31 +1,78 @@
import argparse
import os
from pathlib import Path
from praktika.param import get_param
from praktika.result import Result
from praktika.settings import Settings
from praktika.utils import MetaClasses, Shell, Utils
from ci.jobs.scripts.clickhouse_proc import ClickHouseProc
from ci.jobs.scripts.functional_tests_results import FTResultsProcessor
from ci.settings.definitions import azure_secret
class JobStages(metaclass=MetaClasses.WithIter):
CHECKOUT_SUBMODULES = "checkout"
CMAKE = "cmake"
BUILD = "build"
INSTALL_CLICKHOUSE = "install"
START = "start"
TEST = "test"
def parse_args():
parser = argparse.ArgumentParser(description="ClickHouse Build Job")
parser.add_argument("BUILD_TYPE", help="Type: <amd|arm_debug|release_sanitizer>")
parser.add_argument(
"BUILD_TYPE", help="Type: <amd|arm>_<debug|release>_<asan|tsan|..>"
)
parser.add_argument("--param", help="Optional custom job start stage", default=None)
return parser.parse_args()
def run_stateless_test(
no_parallel: bool, no_sequiential: bool, batch_num: int, batch_total: int
):
assert not (no_parallel and no_sequiential)
test_output_file = f"{Settings.OUTPUT_DIR}/test_result.txt"
aux = ""
nproc = int(Utils.cpu_count() / 2)
if batch_num and batch_total:
aux = f"--run-by-hash-total {batch_total} --run-by-hash-num {batch_num-1}"
statless_test_command = f"clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
--no-drop-if-fail --capture-client-stacktrace --queries /repo/tests/queries --test-runs 1 --hung-check \
{'--no-parallel' if no_parallel else ''} {'--no-sequential' if no_sequiential else ''} \
--print-time --jobs {nproc} --report-coverage --report-logs-stats {aux} \
--queries ./tests/queries -- '' | ts '%Y-%m-%d %H:%M:%S' \
| tee -a \"{test_output_file}\""
if Path(test_output_file).exists():
Path(test_output_file).unlink()
Shell.run(statless_test_command, verbose=True)
def main():
args = parse_args()
params = get_param().split(" ")
parallel_or_sequential = None
no_parallel = False
no_sequential = False
if params:
parallel_or_sequential = params[0]
if len(params) > 1:
batch_num, total_batches = map(int, params[1].split("/"))
else:
batch_num, total_batches = 0, 0
if parallel_or_sequential:
no_parallel = parallel_or_sequential == "non-parallel"
no_sequential = parallel_or_sequential == "parallel"
os.environ["AZURE_CONNECTION_STRING"] = Shell.get_output(
f"aws ssm get-parameter --region us-east-1 --name azure_connection_string --with-decryption --output text --query Parameter.Value",
verbose=True,
)
stop_watch = Utils.Stopwatch()
stages = list(JobStages)
stage = args.param or JobStages.CHECKOUT_SUBMODULES
stage = args.param or JobStages.INSTALL_CLICKHOUSE
if stage:
assert stage in JobStages, f"--param must be one of [{list(JobStages)}]"
print(f"Job will start from stage [{stage}]")
@ -36,9 +83,65 @@ def main():
res = True
results = []
if res and JobStages.CHECKOUT_SUBMODULES in stages:
info = Shell.get_output(f"ls -l {Settings.INPUT_DIR}")
results.append(Result(name="TEST", status=Result.Status.SUCCESS, info=info))
Utils.add_to_PATH(f"{Settings.INPUT_DIR}:tests")
if res and JobStages.INSTALL_CLICKHOUSE in stages:
commands = [
f"chmod +x {Settings.INPUT_DIR}/clickhouse",
f"ln -sf {Settings.INPUT_DIR}/clickhouse {Settings.INPUT_DIR}/clickhouse-server",
f"ln -sf {Settings.INPUT_DIR}/clickhouse {Settings.INPUT_DIR}/clickhouse-client",
f"rm -rf {Settings.TEMP_DIR}/etc/ && mkdir -p {Settings.TEMP_DIR}/etc/clickhouse-client {Settings.TEMP_DIR}/etc/clickhouse-server",
f"cp programs/server/config.xml programs/server/users.xml {Settings.TEMP_DIR}/etc/clickhouse-server/",
f"./tests/config/install.sh {Settings.TEMP_DIR}/etc/clickhouse-server {Settings.TEMP_DIR}/etc/clickhouse-client --s3-storage",
# update_path_ch_config,
f"sed -i 's|>/var/|>{Settings.TEMP_DIR}/var/|g; s|>/etc/|>{Settings.TEMP_DIR}/etc/|g' {Settings.TEMP_DIR}/etc/clickhouse-server/config.xml",
f"sed -i 's|>/etc/|>{Settings.TEMP_DIR}/etc/|g' {Settings.TEMP_DIR}/etc/clickhouse-server/config.d/ssl_certs.xml",
f"clickhouse-server --version",
]
results.append(
Result.create_from_command_execution(
name="Install ClickHouse", command=commands, with_log=True
)
)
res = results[-1].is_ok()
CH = ClickHouseProc()
if res and JobStages.START in stages:
stop_watch_ = Utils.Stopwatch()
step_name = "Start ClickHouse Server"
print(step_name)
res = res and CH.start_minio()
res = res and CH.start()
res = res and CH.wait_ready()
results.append(
Result.create_from(
name=step_name,
status=res,
stopwatch=stop_watch_,
files=(
[
"/tmp/praktika/var/log/clickhouse-server/clickhouse-server.log",
"/tmp/praktika/var/log/clickhouse-server/clickhouse-server.err.log",
]
if not res
else []
),
)
)
res = results[-1].is_ok()
if res and JobStages.TEST in stages:
stop_watch_ = Utils.Stopwatch()
step_name = "Tests"
print(step_name)
run_stateless_test(
no_parallel=no_parallel,
no_sequiential=no_sequential,
batch_num=batch_num,
batch_total=total_batches,
)
results.append(FTResultsProcessor(wd=Settings.OUTPUT_DIR).run())
results[-1].set_timing(stopwatch=stop_watch_)
res = results[-1].is_ok()
Result.create_from(results=results, stopwatch=stop_watch).complete_job()

View File

View File

@ -0,0 +1,144 @@
import threading
import time
from pathlib import Path
from praktika.settings import Settings
from praktika.utils import Shell, Utils
class ClickHouseProc:
BACKUPS_XML = """
<clickhouse>
<backups>
<type>local</type>
<path>{CH_RUNTIME_DIR}/var/lib/clickhouse/disks/backups/</path>
</backups>
</clickhouse>
"""
def __init__(self, fast_test=False):
self.ch_config_dir = f"{Settings.TEMP_DIR}/etc/clickhouse-server"
self.pid_file = f"{self.ch_config_dir}/clickhouse-server.pid"
self.config_file = f"{self.ch_config_dir}/config.xml"
self.user_files_path = f"{self.ch_config_dir}/user_files"
self.test_output_file = f"{Settings.OUTPUT_DIR}/test_result.txt"
self.command = f"clickhouse-server --config-file {self.config_file} --pid-file {self.pid_file} -- --path {self.ch_config_dir} --user_files_path {self.user_files_path} --top_level_domains_path {self.ch_config_dir}/top_level_domains --keeper_server.storage_path {self.ch_config_dir}/coordination"
self.proc = None
self.pid = 0
nproc = int(Utils.cpu_count() / 2)
self.fast_test_command = f"clickhouse-test --hung-check --fast-tests-only --no-random-settings --no-random-merge-tree-settings --no-long --testname --shard --zookeeper --check-zookeeper-session --order random --print-time --report-logs-stats --jobs {nproc} -- '' | ts '%Y-%m-%d %H:%M:%S' \
| tee -a \"{self.test_output_file}\""
# TODO: store info in case of failure
self.info = ""
self.info_file = ""
self.minio_cmd = f"tests/docker_scripts/setup_minio.sh stateless 2>&1 > {Settings.OUTPUT_DIR}/minio.log"
Utils.set_env("CLICKHOUSE_CONFIG_DIR", self.ch_config_dir)
Utils.set_env("CLICKHOUSE_CONFIG", self.config_file)
Utils.set_env("CLICKHOUSE_USER_FILES", self.user_files_path)
Utils.set_env("CLICKHOUSE_SCHEMA_FILES", f"{self.ch_config_dir}/format_schemas")
if not fast_test:
with open(f"{self.ch_config_dir}/config.d/backups.xml", "w") as file:
file.write(self.BACKUPS_XML)
self.minio_proc = None
def start_minio(self):
print("Starting minio")
def run_minio():
self.minio_proc = Shell.run_async(
self.minio_cmd, verbose=True, suppress_output=True
)
thread = threading.Thread(target=run_minio)
thread.daemon = True # Allow program to exit even if thread is still running
thread.start()
time.sleep(5)
return thread.is_alive()
def start(self):
print("Starting ClickHouse server")
Shell.check(f"rm {self.pid_file}")
def run_clickhouse():
self.proc = Shell.run_async(
self.command, verbose=True, suppress_output=False
)
thread = threading.Thread(target=run_clickhouse)
thread.daemon = True # Allow program to exit even if thread is still running
thread.start()
started = False
try:
for _ in range(5):
pid = Shell.get_output(f"cat {self.pid_file}").strip()
if not pid:
Utils.sleep(1)
continue
started = True
print(f"Got pid from fs [{pid}]")
_ = int(pid)
break
except Exception:
pass
if not started:
stdout = self.proc.stdout.read().strip() if self.proc.stdout else ""
stderr = self.proc.stderr.read().strip() if self.proc.stderr else ""
Utils.print_formatted_error("Failed to start ClickHouse", stdout, stderr)
return False
print(f"ClickHouse server started successfully, pid [{pid}]")
return True
def wait_ready(self):
res, out, err = 0, "", ""
attempts = 30
delay = 2
for attempt in range(attempts):
res, out, err = Shell.get_res_stdout_stderr(
'clickhouse-client --query "select 1"', verbose=True
)
if out.strip() == "1":
print("Server ready")
break
else:
print(f"Server not ready, wait")
Utils.sleep(delay)
else:
Utils.print_formatted_error(
f"Server not ready after [{attempts*delay}s]", out, err
)
return False
return True
def run_fast_test(self):
if Path(self.test_output_file).exists():
Path(self.test_output_file).unlink()
exit_code = Shell.run(self.fast_test_command)
return exit_code == 0
def terminate(self):
print("Terminate ClickHouse process")
timeout = 10
if self.proc:
Utils.terminate_process_group(self.proc.pid)
self.proc.terminate()
try:
self.proc.wait(timeout=10)
print(f"Process {self.proc.pid} terminated gracefully.")
except Exception:
print(
f"Process {self.proc.pid} did not terminate in {timeout} seconds, killing it..."
)
Utils.terminate_process_group(self.proc.pid, force=True)
self.proc.wait() # Wait for the process to be fully killed
print(f"Process {self.proc} was killed.")
if self.minio_proc:
Utils.terminate_process_group(self.minio_proc.pid)

View File

@ -232,6 +232,8 @@ class FTResultsProcessor:
else:
pass
info = f"Total: {s.total - s.skipped}, Failed: {s.failed}"
# TODO: !!!
# def test_result_comparator(item):
# # sort by status then by check name
@ -253,6 +255,7 @@ class FTResultsProcessor:
results=test_results,
status=state,
files=[self.tests_output_file],
info=info,
with_info_from_results=False,
)

View File

@ -80,6 +80,8 @@ class _Settings:
CI_DB_TABLE_NAME = ""
CI_DB_INSERT_TIMEOUT_SEC = 5
DISABLE_MERGE_COMMIT = True
_USER_DEFINED_SETTINGS = [
"S3_ARTIFACT_PATH",
@ -112,6 +114,7 @@ _USER_DEFINED_SETTINGS = [
"SECRET_GH_APP_PEM_KEY",
"SECRET_GH_APP_ID",
"MAIN_BRANCH",
"DISABLE_MERGE_COMMIT",
]

View File

@ -11,50 +11,112 @@ from praktika.result import Result, ResultInfo
from praktika.runtime import RunConfig
from praktika.s3 import S3
from praktika.settings import Settings
from praktika.utils import Shell, Utils
from praktika.utils import Utils
@dataclasses.dataclass
class GitCommit:
date: str
message: str
# date: str
# message: str
sha: str
@staticmethod
def from_json(json_data: str) -> List["GitCommit"]:
def from_json(file) -> List["GitCommit"]:
commits = []
json_data = None
try:
data = json.loads(json_data)
with open(file, "r", encoding="utf-8") as f:
json_data = json.load(f)
commits = [
GitCommit(
message=commit["messageHeadline"],
sha=commit["oid"],
date=commit["committedDate"],
# message=commit["messageHeadline"],
sha=commit["sha"],
# date=commit["committedDate"],
)
for commit in data.get("commits", [])
for commit in json_data
]
except Exception as e:
print(
f"ERROR: Failed to deserialize commit's data: [{json_data}], ex: [{e}]"
f"ERROR: Failed to deserialize commit's data [{json_data}], ex: [{e}]"
)
return commits
@classmethod
def update_s3_data(cls):
env = _Environment.get()
sha = env.SHA
if not sha:
print("WARNING: Failed to retrieve commit sha")
return
commits = cls.pull_from_s3()
for commit in commits:
if sha == commit.sha:
print(
f"INFO: Sha already present in commits data [{sha}] - skip data update"
)
return
commits.append(GitCommit(sha=sha))
cls.push_to_s3(commits)
return
@classmethod
def dump(cls, commits):
commits_ = []
for commit in commits:
commits_.append(dataclasses.asdict(commit))
with open(cls.file_name(), "w", encoding="utf8") as f:
json.dump(commits_, f)
@classmethod
def pull_from_s3(cls):
local_path = Path(cls.file_name())
file_name = local_path.name
env = _Environment.get()
s3_path = f"{Settings.HTML_S3_PATH}/{cls.get_s3_prefix(pr_number=env.PR_NUMBER, branch=env.BRANCH)}/{file_name}"
if not S3.copy_file_from_s3(s3_path=s3_path, local_path=local_path):
print(f"WARNING: failed to cp file [{s3_path}] from s3")
return []
return cls.from_json(local_path)
@classmethod
def push_to_s3(cls, commits):
print(f"INFO: push commits data to s3, commits num [{len(commits)}]")
cls.dump(commits)
local_path = Path(cls.file_name())
file_name = local_path.name
env = _Environment.get()
s3_path = f"{Settings.HTML_S3_PATH}/{cls.get_s3_prefix(pr_number=env.PR_NUMBER, branch=env.BRANCH)}/{file_name}"
if not S3.copy_file_to_s3(s3_path=s3_path, local_path=local_path, text=True):
print(f"WARNING: failed to cp file [{local_path}] to s3")
@classmethod
def get_s3_prefix(cls, pr_number, branch):
prefix = ""
assert pr_number or branch
if pr_number and pr_number > 0:
prefix += f"{pr_number}"
else:
prefix += f"{branch}"
return prefix
@classmethod
def file_name(cls):
return f"{Settings.TEMP_DIR}/commits.json"
# def _get_pr_commits(pr_number):
# res = []
# if not pr_number:
# return res
# output = Shell.get_output(f"gh pr view {pr_number} --json commits")
# if output:
# res = GitCommit.from_json(output)
# return res
class HtmlRunnerHooks:
@classmethod
def configure(cls, _workflow):
def _get_pr_commits(pr_number):
res = []
if not pr_number:
return res
output = Shell.get_output(f"gh pr view {pr_number} --json commits")
if output:
res = GitCommit.from_json(output)
return res
# generate pending Results for all jobs in the workflow
if _workflow.enable_cache:
skip_jobs = RunConfig.from_fs(_workflow.name).cache_success
@ -106,11 +168,9 @@ class HtmlRunnerHooks:
Utils.raise_with_error(
"Failed to set both GH commit status and PR comment with Workflow Status, cannot proceed"
)
if env.PR_NUMBER:
commits = _get_pr_commits(env.PR_NUMBER)
# TODO: upload commits data to s3 to visualise it on a report page
print(commits)
# TODO: enable for branch, add commit number limiting
GitCommit.update_s3_data()
@classmethod
def pre_run(cls, _workflow, _job):

View File

@ -52,30 +52,57 @@ class Job:
self,
parameter: Optional[List[Any]] = None,
runs_on: Optional[List[List[str]]] = None,
provides: Optional[List[List[str]]] = None,
requires: Optional[List[List[str]]] = None,
timeout: Optional[List[int]] = None,
):
assert (
parameter or runs_on
), "Either :parameter or :runs_on must be non empty list for parametrisation"
if runs_on:
assert isinstance(runs_on, list) and isinstance(runs_on[0], list)
if not parameter:
parameter = [None] * len(runs_on)
if not runs_on:
runs_on = [None] * len(parameter)
if not timeout:
timeout = [None] * len(parameter)
if not provides:
provides = [None] * len(parameter)
if not requires:
requires = [None] * len(parameter)
assert (
len(parameter) == len(runs_on) == len(timeout)
), "Parametrization lists must be of the same size"
len(parameter)
== len(runs_on)
== len(timeout)
== len(provides)
== len(requires)
), f"Parametrization lists must be of the same size [{len(parameter)}, {len(runs_on)}, {len(timeout)}, {len(provides)}, {len(requires)}]"
res = []
for parameter_, runs_on_, timeout_ in zip(parameter, runs_on, timeout):
for parameter_, runs_on_, timeout_, provides_, requires_ in zip(
parameter, runs_on, timeout, provides, requires
):
obj = copy.deepcopy(self)
assert (
not obj.provides
), "Job.Config.provides must be empty for parametrized jobs"
if parameter_:
obj.parameter = parameter_
if runs_on_:
obj.runs_on = runs_on_
if timeout_:
obj.timeout = timeout_
if provides_:
assert (
not obj.provides
), "Job.Config.provides must be empty for parametrized jobs"
obj.provides = provides_
if requires_:
assert (
not obj.requires
), "Job.Config.requires and parametrize(requires=...) are both set"
obj.requires = requires_
obj.name = obj.get_job_name_with_parameter()
res.append(obj)
return res
@ -84,13 +111,16 @@ class Job:
name, parameter, runs_on = self.name, self.parameter, self.runs_on
res = name
name_params = []
if isinstance(parameter, list) or isinstance(parameter, dict):
name_params.append(json.dumps(parameter))
elif parameter is not None:
name_params.append(parameter)
if runs_on:
if parameter:
if isinstance(parameter, list) or isinstance(parameter, dict):
name_params.append(json.dumps(parameter))
else:
name_params.append(parameter)
elif runs_on:
assert isinstance(runs_on, list)
name_params.append(json.dumps(runs_on))
else:
assert False
if name_params:
name_params = [str(param) for param in name_params]
res += f" ({', '.join(name_params)})"

View File

@ -200,10 +200,7 @@
}
th.name-column, td.name-column {
max-width: 400px; /* Set the maximum width for the column */
white-space: nowrap; /* Prevent text from wrapping */
overflow: hidden; /* Hide the overflowed text */
text-overflow: ellipsis; /* Show ellipsis (...) for overflowed text */
min-width: 350px;
}
th.status-column, td.status-column {
@ -364,7 +361,6 @@
}
function addKeyValueToStatus(key, value, options = null) {
const statusContainer = document.getElementById('status-container');
let keyValuePair = document.createElement('div');
@ -374,27 +370,40 @@
keyElement.className = 'json-key';
keyElement.textContent = key + ':';
let valueElement
if (value) {
valueElement = document.createElement('div');
valueElement.className = 'json-value';
valueElement.textContent = value;
} else if (options) {
let valueElement;
if (options) {
// Create dropdown if options are provided
valueElement = document.createElement('select');
valueElement.className = 'dropdown-value';
options.forEach(optionValue => {
const option = document.createElement('option');
option.value = optionValue;
option.textContent = optionValue.slice(0, 10);
// Set the initially selected option
if (optionValue === value) {
option.selected = true;
}
valueElement.appendChild(option);
});
// Update the URL parameter when the selected value changes
valueElement.addEventListener('change', (event) => {
const selectedValue = event.target.value;
updateUrlParameter(key, selectedValue);
});
options.forEach(optionValue => {
const option = document.createElement('option');
option.value = optionValue;
option.textContent = optionValue;
valueElement.appendChild(option);
});
} else {
// Create a simple text display if no options are provided
valueElement = document.createElement('div');
valueElement.className = 'json-value';
valueElement.textContent = value || 'N/A'; // Display 'N/A' if value is null
}
keyValuePair.appendChild(keyElement)
keyValuePair.appendChild(valueElement)
keyValuePair.appendChild(keyElement);
keyValuePair.appendChild(valueElement);
statusContainer.appendChild(keyValuePair);
}
@ -518,12 +527,12 @@
const columns = ['name', 'status', 'start_time', 'duration', 'info'];
const columnSymbols = {
name: '📂',
status: '⏯️',
name: '🗂️',
status: '🧾',
start_time: '🕒',
duration: '⏳',
info: '',
files: '📄'
info: '📝',
files: '📎'
};
function createResultsTable(results, nest_level) {
@ -532,16 +541,14 @@
const thead = document.createElement('thead');
const tbody = document.createElement('tbody');
// Get the current URL parameters
const currentUrl = new URL(window.location.href);
// Create table headers based on the fixed columns
const headerRow = document.createElement('tr');
columns.forEach(column => {
const th = document.createElement('th');
th.textContent = th.textContent = columnSymbols[column] || column;
th.textContent = columnSymbols[column] || column;
th.style.cursor = 'pointer'; // Make headers clickable
th.addEventListener('click', () => sortTable(results, column, tbody, nest_level)); // Add click event to sort the table
th.setAttribute('data-sort-direction', 'asc'); // Default sort direction
th.addEventListener('click', () => sortTable(results, column, columnSymbols[column] || column, tbody, nest_level, columns)); // Add click event to sort the table
headerRow.appendChild(th);
});
thead.appendChild(headerRow);
@ -605,39 +612,33 @@
});
}
function sortTable(results, key, tbody, nest_level) {
function sortTable(results, column, key, tbody, nest_level, columns) {
// Find the table header element for the given key
let th = null;
const tableHeaders = document.querySelectorAll('th'); // Select all table headers
tableHeaders.forEach(header => {
if (header.textContent.trim().toLowerCase() === key.toLowerCase()) {
th = header;
}
});
const tableHeaders = document.querySelectorAll('th');
let th = Array.from(tableHeaders).find(header => header.textContent === key);
if (!th) {
console.error(`No table header found for key: ${key}`);
return;
}
// Determine the current sort direction
let ascending = th.getAttribute('data-sort-direction') === 'asc' ? false : true;
const ascending = th.getAttribute('data-sort-direction') === 'asc';
th.setAttribute('data-sort-direction', ascending ? 'desc' : 'asc');
// Toggle the sort direction for the next click
th.setAttribute('data-sort-direction', ascending ? 'asc' : 'desc');
// Sort the results array by the given key
results.sort((a, b) => {
if (a[key] < b[key]) return ascending ? -1 : 1;
if (a[key] > b[key]) return ascending ? 1 : -1;
if (a[column] < b[column]) return ascending ? -1 : 1;
if (a[column] > b[column]) return ascending ? 1 : -1;
return 0;
});
// Clear the existing rows in tbody
tbody.innerHTML = '';
// Re-populate the table with sorted data
populateTableRows(tbody, results, columns, nest_level);
}
function loadJSON(PR, sha, nameParams) {
function loadResultsJSON(PR, sha, nameParams) {
const infoElement = document.getElementById('info-container');
let lastModifiedTime = null;
const task = nameParams[0].toLowerCase();
@ -753,22 +754,61 @@
}
});
if (PR) {
addKeyValueToStatus("PR", PR)
} else {
console.error("TODO")
}
addKeyValueToStatus("sha", null, [sha, 'lala']);
if (nameParams[1]) {
addKeyValueToStatus("job", nameParams[1]);
}
addKeyValueToStatus("workflow", nameParams[0]);
let path_commits_json = '';
let commitsArray = [];
if (PR && sha && root_name) {
loadJSON(PR, sha, nameParams);
if (PR) {
addKeyValueToStatus("PR", PR);
const baseUrl = window.location.origin + window.location.pathname.replace('/json.html', '');
path_commits_json = `${baseUrl}/${encodeURIComponent(PR)}/commits.json`;
} else {
document.getElementById('title').textContent = 'Error: Missing required URL parameters: PR, sha, or name_0';
// Placeholder for a different path when PR is missing
console.error("PR parameter is missing. Setting alternate commits path.");
path_commits_json = '/path/to/alternative/commits.json';
}
function loadCommitsArray(path) {
return fetch(path, { cache: "no-cache" })
.then(response => {
if (!response.ok) {
console.error(`HTTP error! status: ${response.status}`)
return [];
}
return response.json();
})
.then(data => {
if (Array.isArray(data) && data.every(item => typeof item === 'object' && item.hasOwnProperty('sha'))) {
return data.map(item => item.sha);
} else {
throw new Error('Invalid data format: expected array of objects with a "sha" key');
}
})
.catch(error => {
console.error('Error loading commits JSON:', error);
return []; // Return an empty array if an error occurs
});
}
loadCommitsArray(path_commits_json)
.then(data => {
commitsArray = data;
})
.finally(() => {
// Proceed with the rest of the initialization
addKeyValueToStatus("sha", sha || "latest", commitsArray.concat(["latest"]));
if (nameParams[1]) {
addKeyValueToStatus("job", nameParams[1]);
}
addKeyValueToStatus("workflow", nameParams[0]);
// Check if all required parameters are present to load JSON
if (PR && sha && root_name) {
loadResultsJSON(PR, sha, nameParams);
} else {
document.getElementById('title').textContent = 'Error: Missing required URL parameters: PR, sha, or name_0';
}
});
}
window.onload = init;

View File

@ -58,7 +58,6 @@ def _update_workflow_artifacts(workflow):
artifact_job = {}
for job in workflow.jobs:
for artifact_name in job.provides:
assert artifact_name not in artifact_job
artifact_job[artifact_name] = job.name
for artifact in workflow.artifacts:
artifact._provided_by = artifact_job[artifact.name]

View File

@ -151,7 +151,7 @@ def _config_workflow(workflow: Workflow.Config, job_name):
status = Result.Status.ERROR
print("ERROR: ", info)
else:
Shell.check(f"{Settings.PYTHON_INTERPRETER} -m praktika --generate")
assert Shell.check(f"{Settings.PYTHON_INTERPRETER} -m praktika yaml")
exit_code, output, err = Shell.get_res_stdout_stderr(
f"git diff-index HEAD -- {Settings.WORKFLOW_PATH_PREFIX}"
)
@ -250,6 +250,9 @@ def _config_workflow(workflow: Workflow.Config, job_name):
info_lines.append(job_name + ": " + info)
results.append(result_)
if workflow.enable_merge_commit:
assert False, "NOT implemented"
# config:
if workflow.dockers:
print("Calculate docker's digests")

8
ci/praktika/param.py Normal file
View File

@ -0,0 +1,8 @@
from praktika._environment import _Environment
# TODO: find better place and/or right storage for parameter
def get_param():
env = _Environment.get()
assert env.PARAMETER
return env.PARAMETER

View File

@ -1,7 +1,6 @@
import dataclasses
import datetime
import sys
from collections.abc import Container
from pathlib import Path
from typing import Any, Dict, List, Optional
@ -68,8 +67,9 @@ class Result(MetaClasses.Serializable):
if isinstance(status, bool):
status = Result.Status.SUCCESS if status else Result.Status.FAILED
if not results and not status:
print("ERROR: Either .results or .status must be provided")
raise
Utils.raise_with_error(
f"Either .results ({results}) or .status ({status}) must be provided"
)
if not name:
name = _Environment.get().JOB_NAME
if not name:
@ -78,10 +78,10 @@ class Result(MetaClasses.Serializable):
result_status = status or Result.Status.SUCCESS
infos = []
if info:
if isinstance(info, Container):
infos += info
if isinstance(info, str):
infos += [info]
else:
infos.append(info)
infos += info
if results and not status:
for result in results:
if result.status not in (Result.Status.SUCCESS, Result.Status.FAILED):
@ -112,7 +112,7 @@ class Result(MetaClasses.Serializable):
return self.status not in (Result.Status.PENDING, Result.Status.RUNNING)
def is_running(self):
return self.status not in (Result.Status.RUNNING,)
return self.status in (Result.Status.RUNNING,)
def is_ok(self):
return self.status in (Result.Status.SKIPPED, Result.Status.SUCCESS)
@ -180,6 +180,11 @@ class Result(MetaClasses.Serializable):
)
return self
def set_timing(self, stopwatch: Utils.Stopwatch):
self.start_time = stopwatch.start_time
self.duration = stopwatch.duration
return self
def update_sub_result(self, result: "Result"):
assert self.results, "BUG?"
for i, result_ in enumerate(self.results):

View File

@ -125,15 +125,24 @@ class Runner:
return 0
def _run(self, workflow, job, docker="", no_docker=False, param=None):
# re-set envs for local run
env = _Environment.get()
env.JOB_NAME = job.name
env.PARAMETER = job.parameter
env.dump()
if param:
if not isinstance(param, str):
Utils.raise_with_error(
f"Custom param for local tests must be of type str, got [{type(param)}]"
)
env = _Environment.get()
env.dump()
if job.run_in_docker and not no_docker:
job.run_in_docker, docker_settings = (
job.run_in_docker.split("+")[0],
job.run_in_docker.split("+")[1:],
)
from_root = "root" in docker_settings
if ":" in job.run_in_docker:
docker_name, docker_tag = job.run_in_docker.split(":")
print(
@ -145,7 +154,7 @@ class Runner:
RunConfig.from_fs(workflow.name).digest_dockers[job.run_in_docker],
)
docker = docker or f"{docker_name}:{docker_tag}"
cmd = f"docker run --rm --user \"$(id -u):$(id -g)\" -e PYTHONPATH='{Settings.DOCKER_WD}:{Settings.DOCKER_WD}/ci' --volume ./:{Settings.DOCKER_WD} --volume {Settings.TEMP_DIR}:{Settings.TEMP_DIR} --workdir={Settings.DOCKER_WD} {docker} {job.command}"
cmd = f"docker run --rm --name praktika {'--user $(id -u):$(id -g)' if not from_root else ''} -e PYTHONPATH='{Settings.DOCKER_WD}:{Settings.DOCKER_WD}/ci' --volume ./:{Settings.DOCKER_WD} --volume {Settings.TEMP_DIR}:{Settings.TEMP_DIR} --workdir={Settings.DOCKER_WD} {docker} {job.command}"
else:
cmd = job.command
@ -226,7 +235,8 @@ class Runner:
print(info)
result.set_info(info).set_status(Result.Status.ERROR).dump()
result.set_files(files=[Settings.RUN_LOG])
if not result.is_ok():
result.set_files(files=[Settings.RUN_LOG])
result.update_duration().dump()
if result.info and result.status != Result.Status.SUCCESS:
@ -329,7 +339,7 @@ class Runner:
workflow, job, pr=pr, branch=branch, sha=sha
)
if res:
if res and (not local_run or pr or sha or branch):
res = False
print(f"=== Pre run script [{job.name}], workflow [{workflow.name}] ===")
try:

View File

@ -52,7 +52,7 @@ class S3:
cmd += " --content-type text/plain"
res = cls.run_command_with_retries(cmd)
if not res:
raise
raise RuntimeError()
bucket = s3_path.split("/")[0]
endpoint = Settings.S3_BUCKET_TO_HTTP_ENDPOINT[bucket]
assert endpoint

View File

@ -31,6 +31,7 @@ class Workflow:
enable_report: bool = False
enable_merge_ready_status: bool = False
enable_cidb: bool = False
enable_merge_commit: bool = False
def is_event_pull_request(self):
return self.event == Workflow.Event.PULL_REQUEST

View File

@ -80,6 +80,9 @@ jobs:
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{{{ github.event.pull_reguest.head.sha }}}}
{JOB_ADDONS}
- name: Prepare env script
run: |

View File

@ -8,23 +8,30 @@ class RunnerLabels:
CI_SERVICES = "ci_services"
CI_SERVICES_EBS = "ci_services_ebs"
BUILDER = "builder"
STYLE_CHECKER = "style-checker"
BASE_BRANCH = "master"
azure_secret = Secret.Config(
name="azure_connection_string",
type=Secret.Type.AWS_SSM_VAR,
)
SECRETS = [
Secret.Config(
name="dockerhub_robot_password",
type=Secret.Type.AWS_SSM_VAR,
),
Secret.Config(
name="woolenwolf_gh_app.clickhouse-app-id",
type=Secret.Type.AWS_SSM_SECRET,
),
Secret.Config(
name="woolenwolf_gh_app.clickhouse-app-key",
type=Secret.Type.AWS_SSM_SECRET,
),
azure_secret,
# Secret.Config(
# name="woolenwolf_gh_app.clickhouse-app-id",
# type=Secret.Type.AWS_SSM_SECRET,
# ),
# Secret.Config(
# name="woolenwolf_gh_app.clickhouse-app-key",
# type=Secret.Type.AWS_SSM_SECRET,
# ),
]
DOCKERS = [
@ -118,12 +125,12 @@ DOCKERS = [
# platforms=Docker.Platforms.arm_amd,
# depends_on=["clickhouse/test-base"],
# ),
# Docker.Config(
# name="clickhouse/stateless-test",
# path="./ci/docker/test/stateless",
# platforms=Docker.Platforms.arm_amd,
# depends_on=["clickhouse/test-base"],
# ),
Docker.Config(
name="clickhouse/stateless-test",
path="./ci/docker/stateless-test",
platforms=Docker.Platforms.arm_amd,
depends_on=[],
),
# Docker.Config(
# name="clickhouse/stateful-test",
# path="./ci/docker/test/stateful",
@ -230,5 +237,6 @@ DOCKERS = [
class JobNames:
STYLE_CHECK = "Style Check"
FAST_TEST = "Fast test"
BUILD_AMD_DEBUG = "Build amd64 debug"
BUILD = "Build"
BUILD_AMD_DEBUG = "Build (amd, debug)"
STATELESS_TESTS = "Stateless tests (amd, debug)"

View File

@ -13,7 +13,8 @@ from ci.settings.definitions import (
class ArtifactNames:
ch_debug_binary = "clickhouse_debug_binary"
CH_AMD_DEBUG = "CH_AMD_DEBUG"
CH_AMD_RELEASE = "CH_AMD_RELEASE"
style_check_job = Job.Config(
@ -37,10 +38,10 @@ fast_test_job = Job.Config(
),
)
job_build_amd_debug = Job.Config(
name=JobNames.BUILD_AMD_DEBUG,
amd_build_jobs = Job.Config(
name=JobNames.BUILD,
runs_on=[RunnerLabels.BUILDER],
command="python3 ./ci/jobs/build_clickhouse.py amd_debug",
command="python3 ./ci/jobs/build_clickhouse.py",
run_in_docker="clickhouse/fasttest",
digest_config=Job.CacheDigestConfig(
include_paths=[
@ -56,20 +57,30 @@ job_build_amd_debug = Job.Config(
"./tests/ci/version_helper.py",
],
),
provides=[ArtifactNames.ch_debug_binary],
).parametrize(
parameter=["amd_debug", "amd_release"],
provides=[[ArtifactNames.CH_AMD_DEBUG], [ArtifactNames.CH_AMD_RELEASE]],
)
stateless_tests_job = Job.Config(
statless_batch_num = 2
stateless_tests_amd_debug_jobs = Job.Config(
name=JobNames.STATELESS_TESTS,
runs_on=[RunnerLabels.BUILDER],
command="python3 ./ci/jobs/functional_stateless_tests.py amd_debug",
run_in_docker="clickhouse/fasttest:latest",
run_in_docker="clickhouse/stateless-test",
digest_config=Job.CacheDigestConfig(
include_paths=[
"./ci/jobs/functional_stateless_tests.py",
],
),
requires=[ArtifactNames.ch_debug_binary],
requires=[ArtifactNames.CH_AMD_DEBUG],
).parametrize(
parameter=[
f"parallel {i+1}/{statless_batch_num}" for i in range(statless_batch_num)
]
+ ["non-parallel"],
runs_on=[[RunnerLabels.BUILDER] for _ in range(statless_batch_num)]
+ [[RunnerLabels.STYLE_CHECKER]],
)
workflow = Workflow.Config(
@ -79,15 +90,20 @@ workflow = Workflow.Config(
jobs=[
style_check_job,
fast_test_job,
job_build_amd_debug,
stateless_tests_job,
*amd_build_jobs,
*stateless_tests_amd_debug_jobs,
],
artifacts=[
Artifact.Config(
name=ArtifactNames.ch_debug_binary,
name=ArtifactNames.CH_AMD_DEBUG,
type=Artifact.Type.S3,
path=f"{Settings.TEMP_DIR}/build/programs/clickhouse",
)
),
Artifact.Config(
name=ArtifactNames.CH_AMD_RELEASE,
type=Artifact.Type.S3,
path=f"{Settings.TEMP_DIR}/build/programs/clickhouse",
),
],
dockers=DOCKERS,
secrets=SECRETS,
@ -101,8 +117,11 @@ WORKFLOWS = [
] # type: List[Workflow.Config]
if __name__ == "__main__":
# local job test inside praktika environment
from praktika.runner import Runner
Runner().run(workflow, fast_test_job, docker="fasttest", local_run=True)
# if __name__ == "__main__":
# # local job test inside praktika environment
# from praktika.runner import Runner
# from praktika.digest import Digest
#
# print(Digest().calc_job_digest(amd_debug_build_job))
#
# Runner().run(workflow, fast_test_job, docker="fasttest", local_run=True)

View File

@ -2153,9 +2153,9 @@ class TestSuite:
self.sequential_tests = []
self.parallel_tests = []
for test_name in self.all_tests:
if self.is_sequential_test(test_name):
if self.is_sequential_test(test_name) and not args.no_sequential:
self.sequential_tests.append(test_name)
else:
elif not args.no_parallel:
self.parallel_tests.append(test_name)
def is_sequential_test(self, test_name):
@ -3290,7 +3290,10 @@ def parse_args():
help='Replace random database name with "default" in stderr',
)
parser.add_argument(
"--parallel", default="1/1", help="One parallel test run number/total"
"--no-sequential", action="store_true", help="Not run no-parallel"
)
parser.add_argument(
"--no-parallel", action="store_true", help="Run only no-parallel"
)
parser.add_argument(
"-j", "--jobs", default=1, nargs="?", type=int, help="Run all tests in parallel"
@ -3339,7 +3342,7 @@ def parse_args():
parser.add_argument(
"--sequential",
nargs="+",
help="Run these tests sequentially even if --parallel specified",
help="Run all tests sequentially",
)
parser.add_argument(
"--no-long", action="store_true", dest="no_long", help="Do not run long tests"

View File

@ -1,8 +1,8 @@
<clickhouse>
<openSSL>
<server>
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
<certificateFile>/tmp/praktika/etc/clickhouse-server/server.crt</certificateFile>
<privateKeyFile>/tmp/praktika/etc/clickhouse-server/server.key</privateKeyFile>
</server>
</openSSL>
</clickhouse>

View File

@ -9,6 +9,21 @@ DEST_SERVER_PATH="${1:-/etc/clickhouse-server}"
DEST_CLIENT_PATH="${2:-/etc/clickhouse-client}"
SRC_PATH="$( cd "$(dirname "$0")" >/dev/null 2>&1 ; pwd -P )"
shift # DEST_SERVER_PATH
shift # DEST_CLIENT_PATH
FAST_TEST=0
S3_STORAGE=0
while [[ "$#" -gt 0 ]]; do
case $1 in
--fast-test) FAST_TEST=1 ;;
--s3-storage) S3_STORAGE=1 ;;
*) echo "Unknown option: $1" ; exit 1 ;;
esac
shift
done
echo "Going to install test configs from $SRC_PATH into $DEST_SERVER_PATH"
mkdir -p $DEST_SERVER_PATH/config.d/
@ -72,9 +87,8 @@ ln -sf $SRC_PATH/config.d/serverwide_trace_collector.xml $DEST_SERVER_PATH/confi
ln -sf $SRC_PATH/config.d/rocksdb.xml $DEST_SERVER_PATH/config.d/
# Not supported with fasttest.
if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ]
then
ln -sf $SRC_PATH/config.d/legacy_geobase.xml $DEST_SERVER_PATH/config.d/
if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] || [ "$FAST_TEST" != "1" ]; then
ln -sf "$SRC_PATH/config.d/legacy_geobase.xml" "$DEST_SERVER_PATH/config.d/"
fi
ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/
@ -185,7 +199,7 @@ elif [[ "$USE_AZURE_STORAGE_FOR_MERGE_TREE" == "1" ]]; then
ln -sf $SRC_PATH/config.d/azure_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/
fi
if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then
if [[ "$EXPORT_S3_STORAGE_POLICIES" == "1" ]] || [[ "$S3_STORAGE" = "1" ]]; then
ln -sf $SRC_PATH/config.d/azure_storage_conf.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/storage_conf.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/storage_conf_02944.xml $DEST_SERVER_PATH/config.d/
@ -195,7 +209,7 @@ if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then
ln -sf $SRC_PATH/users.d/s3_cache_new.xml $DEST_SERVER_PATH/users.d/
fi
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
if [[ "$USE_DATABASE_REPLICATED" == "1" ]]; then
ln -sf $SRC_PATH/users.d/database_replicated.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/config.d/database_replicated.xml $DEST_SERVER_PATH/config.d/
rm /etc/clickhouse-server/config.d/zookeeper.xml

View File

@ -5,6 +5,12 @@ set -euxf -o pipefail
export MINIO_ROOT_USER=${MINIO_ROOT_USER:-clickhouse}
export MINIO_ROOT_PASSWORD=${MINIO_ROOT_PASSWORD:-clickhouse}
if [ -d "$TEMP_DIR" ]; then
cd "$TEMP_DIR"
# add / for minio mc in docker
PATH="/:.:$PATH"
fi
usage() {
echo $"Usage: $0 <stateful|stateless> <test_path> (default path: /usr/share/clickhouse-test)"
exit 1
@ -70,9 +76,10 @@ download_minio() {
}
start_minio() {
pwd
mkdir -p ./minio_data
./minio --version
./minio server --address ":11111" ./minio_data &
minio --version
minio server --address ":11111" ./minio_data &
wait_for_it
lsof -i :11111
sleep 5
@ -80,12 +87,14 @@ start_minio() {
setup_minio() {
local test_type=$1
./mc alias set clickminio http://localhost:11111 clickhouse clickhouse
./mc admin user add clickminio test testtest
./mc admin policy attach clickminio readwrite --user=test
./mc mb --ignore-existing clickminio/test
echo "setup_minio(), test_type=$test_type"
mc alias set clickminio http://localhost:11111 clickhouse clickhouse
mc admin user add clickminio test testtest
mc admin policy attach clickminio readwrite --user=test ||:
mc mb --ignore-existing clickminio/test
if [ "$test_type" = "stateless" ]; then
./mc anonymous set public clickminio/test
echo "Create @test bucket in minio"
mc anonymous set public clickminio/test
fi
}
@ -95,12 +104,13 @@ upload_data() {
local query_dir=$1
local test_path=$2
local data_path=${test_path}/queries/${query_dir}/data_minio
echo "upload_data() data_path=$data_path"
# iterating over globs will cause redundant file variable to be
# a path to a file, not a filename
# shellcheck disable=SC2045
if [ -d "${data_path}" ]; then
./mc cp --recursive "${data_path}"/ clickminio/test/
mc cp --recursive "${data_path}"/ clickminio/test/
fi
}
@ -138,7 +148,7 @@ wait_for_it() {
main() {
local query_dir
query_dir=$(check_arg "$@")
if [ ! -f ./minio ]; then
if ! (minio --version && mc --version); then
download_minio
fi
start_minio