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: steps:
- name: Checkout code - name: Checkout code
uses: actions/checkout@v4 uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script - name: Prepare env script
run: | run: |
@ -68,6 +71,9 @@ jobs:
steps: steps:
- name: Checkout code - name: Checkout code
uses: actions/checkout@v4 uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script - name: Prepare env script
run: | run: |
@ -106,6 +112,9 @@ jobs:
steps: steps:
- name: Checkout code - name: Checkout code
uses: actions/checkout@v4 uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script - name: Prepare env script
run: | run: |
@ -144,6 +153,9 @@ jobs:
steps: steps:
- name: Checkout code - name: Checkout code
uses: actions/checkout@v4 uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script - name: Prepare env script
run: | run: |
@ -172,16 +184,19 @@ jobs:
python3 -m praktika run --job '''Fast test''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log python3 -m praktika run --job '''Fast test''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log
fi fi
build_amd64_debug: build_amd_debug:
runs-on: [builder] runs-on: [builder]
needs: [config_workflow, docker_builds] needs: [config_workflow, docker_builds]
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgYW1kNjQgZGVidWc=') }} if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9kZWJ1Zyk=') }}
name: "Build amd64 debug" name: "Build (amd_debug)"
outputs: outputs:
data: ${{ steps.run.outputs.DATA }} data: ${{ steps.run.outputs.DATA }}
steps: steps:
- name: Checkout code - name: Checkout code
uses: actions/checkout@v4 uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script - name: Prepare env script
run: | run: |
@ -205,21 +220,24 @@ jobs:
. /tmp/praktika_setup_env.sh . /tmp/praktika_setup_env.sh
set -o pipefail set -o pipefail
if command -v ts &> /dev/null; then 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 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 fi
stateless_tests_amd_debug: build_amd_release:
runs-on: [builder] runs-on: [builder]
needs: [config_workflow, docker_builds, build_amd64_debug] needs: [config_workflow, docker_builds]
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWQsIGRlYnVnKQ==') }} if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9yZWxlYXNlKQ==') }}
name: "Stateless tests (amd, debug)" name: "Build (amd_release)"
outputs: outputs:
data: ${{ steps.run.outputs.DATA }} data: ${{ steps.run.outputs.DATA }}
steps: steps:
- name: Checkout code - name: Checkout code
uses: actions/checkout@v4 uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script - name: Prepare env script
run: | run: |
@ -243,14 +261,137 @@ jobs:
. /tmp/praktika_setup_env.sh . /tmp/praktika_setup_env.sh
set -o pipefail set -o pipefail
if command -v ts &> /dev/null; then 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 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 fi
finish_workflow: finish_workflow:
runs-on: [ci_services] 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() }} if: ${{ !cancelled() }}
name: "Finish Workflow" name: "Finish Workflow"
outputs: outputs:
@ -258,6 +399,9 @@ jobs:
steps: steps:
- name: Checkout code - name: Checkout code
uses: actions/checkout@v4 uses: actions/checkout@v4
with:
clear-repository: true
ref: ${{ github.event.pull_reguest.head.sha }}
- name: Prepare env script - name: Prepare env script
run: | 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 import argparse
from praktika.param import get_param
from praktika.result import Result from praktika.result import Result
from praktika.settings import Settings from praktika.settings import Settings
from praktika.utils import MetaClasses, Shell, Utils from praktika.utils import MetaClasses, Shell, Utils
@ -14,7 +15,9 @@ class JobStages(metaclass=MetaClasses.WithIter):
def parse_args(): def parse_args():
parser = argparse.ArgumentParser(description="ClickHouse Build Job") parser = argparse.ArgumentParser(description="ClickHouse Build Job")
parser.add_argument( 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( parser.add_argument(
"--param", "--param",
@ -24,6 +27,18 @@ def parse_args():
return parser.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(): def main():
args = parse_args() args = parse_args()
@ -42,20 +57,45 @@ def main():
cmake_build_type = "Release" cmake_build_type = "Release"
sanitizer = "" sanitizer = ""
if "debug" in args.BUILD_TYPE.lower(): if args.build_type and get_param():
print("Build type set: debug") assert (
cmake_build_type = "Debug" 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(): build_type = args.build_type or get_param()
print("Sanitizer set: address") assert (
sanitizer = "address" 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(): # if Environment.is_local_run():
# build_cache_type = "disabled" # build_cache_type = "disabled"
# else: # 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" build_dir = f"{Settings.TEMP_DIR}/build"
res = True res = True
@ -75,12 +115,7 @@ def main():
results.append( results.append(
Result.create_from_command_execution( Result.create_from_command_execution(
name="Cmake configuration", name="Cmake configuration",
command=f"cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE={cmake_build_type} \ command=cmake_cmd,
-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}",
workdir=build_dir, workdir=build_dir,
with_log=True, with_log=True,
) )

View File

@ -1,120 +1,13 @@
import argparse import argparse
import threading
from pathlib import Path
from praktika.result import Result from praktika.result import Result
from praktika.settings import Settings from praktika.settings import Settings
from praktika.utils import MetaClasses, Shell, Utils 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.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(): def clone_submodules():
submodules_to_update = [ submodules_to_update = [
"contrib/sysroot", "contrib/sysroot",
@ -240,7 +133,7 @@ def main():
Shell.check(f"rm -rf {build_dir} && mkdir -p {build_dir}") Shell.check(f"rm -rf {build_dir} && mkdir -p {build_dir}")
results.append( results.append(
Result.create_from_command_execution( Result.create_from_command_execution(
name="Checkout Submodules for Minimal Build", name="Checkout Submodules",
command=clone_submodules, command=clone_submodules,
) )
) )
@ -295,8 +188,8 @@ def main():
if res and JobStages.CONFIG in stages: if res and JobStages.CONFIG in stages:
commands = [ commands = [
f"rm -rf {Settings.TEMP_DIR}/etc/ && mkdir -p {Settings.TEMP_DIR}/etc/clickhouse-client {Settings.TEMP_DIR}/etc/clickhouse-server", 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"cp ./programs/server/config.xml ./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"./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"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", f"rm -f {Settings.TEMP_DIR}/etc/clickhouse-server/config.d/secure_ports.xml",
update_path_ch_config, update_path_ch_config,
@ -310,7 +203,7 @@ def main():
) )
res = results[-1].is_ok() res = results[-1].is_ok()
CH = ClickHouseProc() CH = ClickHouseProc(fast_test=True)
if res and JobStages.TEST in stages: if res and JobStages.TEST in stages:
stop_watch_ = Utils.Stopwatch() stop_watch_ = Utils.Stopwatch()
step_name = "Start ClickHouse Server" step_name = "Start ClickHouse Server"

View File

@ -1,31 +1,78 @@
import argparse import argparse
import os
from pathlib import Path
from praktika.param import get_param
from praktika.result import Result from praktika.result import Result
from praktika.settings import Settings from praktika.settings import Settings
from praktika.utils import MetaClasses, Shell, Utils 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): class JobStages(metaclass=MetaClasses.WithIter):
CHECKOUT_SUBMODULES = "checkout" INSTALL_CLICKHOUSE = "install"
CMAKE = "cmake" START = "start"
BUILD = "build" TEST = "test"
def parse_args(): def parse_args():
parser = argparse.ArgumentParser(description="ClickHouse Build Job") 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) parser.add_argument("--param", help="Optional custom job start stage", default=None)
return parser.parse_args() 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(): def main():
args = parse_args() 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() stop_watch = Utils.Stopwatch()
stages = list(JobStages) stages = list(JobStages)
stage = args.param or JobStages.CHECKOUT_SUBMODULES stage = args.param or JobStages.INSTALL_CLICKHOUSE
if stage: if stage:
assert stage in JobStages, f"--param must be one of [{list(JobStages)}]" assert stage in JobStages, f"--param must be one of [{list(JobStages)}]"
print(f"Job will start from stage [{stage}]") print(f"Job will start from stage [{stage}]")
@ -36,9 +83,65 @@ def main():
res = True res = True
results = [] results = []
if res and JobStages.CHECKOUT_SUBMODULES in stages: Utils.add_to_PATH(f"{Settings.INPUT_DIR}:tests")
info = Shell.get_output(f"ls -l {Settings.INPUT_DIR}")
results.append(Result(name="TEST", status=Result.Status.SUCCESS, info=info)) 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() res = results[-1].is_ok()
Result.create_from(results=results, stopwatch=stop_watch).complete_job() 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: else:
pass pass
info = f"Total: {s.total - s.skipped}, Failed: {s.failed}"
# TODO: !!! # TODO: !!!
# def test_result_comparator(item): # def test_result_comparator(item):
# # sort by status then by check name # # sort by status then by check name
@ -253,6 +255,7 @@ class FTResultsProcessor:
results=test_results, results=test_results,
status=state, status=state,
files=[self.tests_output_file], files=[self.tests_output_file],
info=info,
with_info_from_results=False, with_info_from_results=False,
) )

View File

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

View File

@ -11,50 +11,112 @@ from praktika.result import Result, ResultInfo
from praktika.runtime import RunConfig from praktika.runtime import RunConfig
from praktika.s3 import S3 from praktika.s3 import S3
from praktika.settings import Settings from praktika.settings import Settings
from praktika.utils import Shell, Utils from praktika.utils import Utils
@dataclasses.dataclass @dataclasses.dataclass
class GitCommit: class GitCommit:
date: str # date: str
message: str # message: str
sha: str sha: str
@staticmethod @staticmethod
def from_json(json_data: str) -> List["GitCommit"]: def from_json(file) -> List["GitCommit"]:
commits = [] commits = []
json_data = None
try: try:
data = json.loads(json_data) with open(file, "r", encoding="utf-8") as f:
json_data = json.load(f)
commits = [ commits = [
GitCommit( GitCommit(
message=commit["messageHeadline"], # message=commit["messageHeadline"],
sha=commit["oid"], sha=commit["sha"],
date=commit["committedDate"], # date=commit["committedDate"],
) )
for commit in data.get("commits", []) for commit in json_data
] ]
except Exception as e: except Exception as e:
print( 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 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: class HtmlRunnerHooks:
@classmethod @classmethod
def configure(cls, _workflow): 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 # generate pending Results for all jobs in the workflow
if _workflow.enable_cache: if _workflow.enable_cache:
skip_jobs = RunConfig.from_fs(_workflow.name).cache_success skip_jobs = RunConfig.from_fs(_workflow.name).cache_success
@ -106,11 +168,9 @@ class HtmlRunnerHooks:
Utils.raise_with_error( Utils.raise_with_error(
"Failed to set both GH commit status and PR comment with Workflow Status, cannot proceed" "Failed to set both GH commit status and PR comment with Workflow Status, cannot proceed"
) )
if env.PR_NUMBER: if env.PR_NUMBER:
commits = _get_pr_commits(env.PR_NUMBER) # TODO: enable for branch, add commit number limiting
# TODO: upload commits data to s3 to visualise it on a report page GitCommit.update_s3_data()
print(commits)
@classmethod @classmethod
def pre_run(cls, _workflow, _job): def pre_run(cls, _workflow, _job):

View File

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

View File

@ -200,10 +200,7 @@
} }
th.name-column, td.name-column { th.name-column, td.name-column {
max-width: 400px; /* Set the maximum width for the column */ min-width: 350px;
white-space: nowrap; /* Prevent text from wrapping */
overflow: hidden; /* Hide the overflowed text */
text-overflow: ellipsis; /* Show ellipsis (...) for overflowed text */
} }
th.status-column, td.status-column { th.status-column, td.status-column {
@ -364,7 +361,6 @@
} }
function addKeyValueToStatus(key, value, options = null) { function addKeyValueToStatus(key, value, options = null) {
const statusContainer = document.getElementById('status-container'); const statusContainer = document.getElementById('status-container');
let keyValuePair = document.createElement('div'); let keyValuePair = document.createElement('div');
@ -374,27 +370,40 @@
keyElement.className = 'json-key'; keyElement.className = 'json-key';
keyElement.textContent = key + ':'; keyElement.textContent = key + ':';
let valueElement let valueElement;
if (value) {
valueElement = document.createElement('div'); if (options) {
valueElement.className = 'json-value'; // Create dropdown if options are provided
valueElement.textContent = value;
} else if (options) {
valueElement = document.createElement('select'); valueElement = document.createElement('select');
valueElement.className = 'dropdown-value'; 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) => { valueElement.addEventListener('change', (event) => {
const selectedValue = event.target.value; const selectedValue = event.target.value;
updateUrlParameter(key, selectedValue); updateUrlParameter(key, selectedValue);
}); });
options.forEach(optionValue => { } else {
const option = document.createElement('option'); // Create a simple text display if no options are provided
option.value = optionValue; valueElement = document.createElement('div');
option.textContent = optionValue; valueElement.className = 'json-value';
valueElement.appendChild(option); 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); statusContainer.appendChild(keyValuePair);
} }
@ -518,12 +527,12 @@
const columns = ['name', 'status', 'start_time', 'duration', 'info']; const columns = ['name', 'status', 'start_time', 'duration', 'info'];
const columnSymbols = { const columnSymbols = {
name: '📂', name: '🗂️',
status: '⏯️', status: '🧾',
start_time: '🕒', start_time: '🕒',
duration: '⏳', duration: '⏳',
info: '', info: '📝',
files: '📄' files: '📎'
}; };
function createResultsTable(results, nest_level) { function createResultsTable(results, nest_level) {
@ -532,16 +541,14 @@
const thead = document.createElement('thead'); const thead = document.createElement('thead');
const tbody = document.createElement('tbody'); 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 // Create table headers based on the fixed columns
const headerRow = document.createElement('tr'); const headerRow = document.createElement('tr');
columns.forEach(column => { columns.forEach(column => {
const th = document.createElement('th'); 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.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); headerRow.appendChild(th);
}); });
thead.appendChild(headerRow); 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 // Find the table header element for the given key
let th = null; const tableHeaders = document.querySelectorAll('th');
const tableHeaders = document.querySelectorAll('th'); // Select all table headers let th = Array.from(tableHeaders).find(header => header.textContent === key);
tableHeaders.forEach(header => {
if (header.textContent.trim().toLowerCase() === key.toLowerCase()) {
th = header;
}
});
if (!th) { if (!th) {
console.error(`No table header found for key: ${key}`); console.error(`No table header found for key: ${key}`);
return; return;
} }
// Determine the current sort direction const ascending = th.getAttribute('data-sort-direction') === 'asc';
let ascending = th.getAttribute('data-sort-direction') === 'asc' ? false : true; 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) => { results.sort((a, b) => {
if (a[key] < b[key]) return ascending ? -1 : 1; if (a[column] < b[column]) return ascending ? -1 : 1;
if (a[key] > b[key]) return ascending ? 1 : -1; if (a[column] > b[column]) return ascending ? 1 : -1;
return 0; return 0;
}); });
// Clear the existing rows in tbody
tbody.innerHTML = '';
// Re-populate the table with sorted data // Re-populate the table with sorted data
populateTableRows(tbody, results, columns, nest_level); populateTableRows(tbody, results, columns, nest_level);
} }
function loadJSON(PR, sha, nameParams) { function loadResultsJSON(PR, sha, nameParams) {
const infoElement = document.getElementById('info-container'); const infoElement = document.getElementById('info-container');
let lastModifiedTime = null; let lastModifiedTime = null;
const task = nameParams[0].toLowerCase(); const task = nameParams[0].toLowerCase();
@ -753,22 +754,61 @@
} }
}); });
if (PR) { let path_commits_json = '';
addKeyValueToStatus("PR", PR) let commitsArray = [];
} else {
console.error("TODO")
}
addKeyValueToStatus("sha", null, [sha, 'lala']);
if (nameParams[1]) {
addKeyValueToStatus("job", nameParams[1]);
}
addKeyValueToStatus("workflow", nameParams[0]);
if (PR && sha && root_name) { if (PR) {
loadJSON(PR, sha, nameParams); addKeyValueToStatus("PR", PR);
const baseUrl = window.location.origin + window.location.pathname.replace('/json.html', '');
path_commits_json = `${baseUrl}/${encodeURIComponent(PR)}/commits.json`;
} else { } 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; window.onload = init;

View File

@ -58,7 +58,6 @@ def _update_workflow_artifacts(workflow):
artifact_job = {} artifact_job = {}
for job in workflow.jobs: for job in workflow.jobs:
for artifact_name in job.provides: for artifact_name in job.provides:
assert artifact_name not in artifact_job
artifact_job[artifact_name] = job.name artifact_job[artifact_name] = job.name
for artifact in workflow.artifacts: for artifact in workflow.artifacts:
artifact._provided_by = artifact_job[artifact.name] 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 status = Result.Status.ERROR
print("ERROR: ", info) print("ERROR: ", info)
else: 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( exit_code, output, err = Shell.get_res_stdout_stderr(
f"git diff-index HEAD -- {Settings.WORKFLOW_PATH_PREFIX}" 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) info_lines.append(job_name + ": " + info)
results.append(result_) results.append(result_)
if workflow.enable_merge_commit:
assert False, "NOT implemented"
# config: # config:
if workflow.dockers: if workflow.dockers:
print("Calculate docker's digests") 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 dataclasses
import datetime import datetime
import sys import sys
from collections.abc import Container
from pathlib import Path from pathlib import Path
from typing import Any, Dict, List, Optional from typing import Any, Dict, List, Optional
@ -68,8 +67,9 @@ class Result(MetaClasses.Serializable):
if isinstance(status, bool): if isinstance(status, bool):
status = Result.Status.SUCCESS if status else Result.Status.FAILED status = Result.Status.SUCCESS if status else Result.Status.FAILED
if not results and not status: if not results and not status:
print("ERROR: Either .results or .status must be provided") Utils.raise_with_error(
raise f"Either .results ({results}) or .status ({status}) must be provided"
)
if not name: if not name:
name = _Environment.get().JOB_NAME name = _Environment.get().JOB_NAME
if not name: if not name:
@ -78,10 +78,10 @@ class Result(MetaClasses.Serializable):
result_status = status or Result.Status.SUCCESS result_status = status or Result.Status.SUCCESS
infos = [] infos = []
if info: if info:
if isinstance(info, Container): if isinstance(info, str):
infos += info infos += [info]
else: else:
infos.append(info) infos += info
if results and not status: if results and not status:
for result in results: for result in results:
if result.status not in (Result.Status.SUCCESS, Result.Status.FAILED): 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) return self.status not in (Result.Status.PENDING, Result.Status.RUNNING)
def is_running(self): def is_running(self):
return self.status not in (Result.Status.RUNNING,) return self.status in (Result.Status.RUNNING,)
def is_ok(self): def is_ok(self):
return self.status in (Result.Status.SKIPPED, Result.Status.SUCCESS) return self.status in (Result.Status.SKIPPED, Result.Status.SUCCESS)
@ -180,6 +180,11 @@ class Result(MetaClasses.Serializable):
) )
return self 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"): def update_sub_result(self, result: "Result"):
assert self.results, "BUG?" assert self.results, "BUG?"
for i, result_ in enumerate(self.results): for i, result_ in enumerate(self.results):

View File

@ -125,15 +125,24 @@ class Runner:
return 0 return 0
def _run(self, workflow, job, docker="", no_docker=False, param=None): 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 param:
if not isinstance(param, str): if not isinstance(param, str):
Utils.raise_with_error( Utils.raise_with_error(
f"Custom param for local tests must be of type str, got [{type(param)}]" 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: 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: if ":" in job.run_in_docker:
docker_name, docker_tag = job.run_in_docker.split(":") docker_name, docker_tag = job.run_in_docker.split(":")
print( print(
@ -145,7 +154,7 @@ class Runner:
RunConfig.from_fs(workflow.name).digest_dockers[job.run_in_docker], RunConfig.from_fs(workflow.name).digest_dockers[job.run_in_docker],
) )
docker = docker or f"{docker_name}:{docker_tag}" 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: else:
cmd = job.command cmd = job.command
@ -226,7 +235,8 @@ class Runner:
print(info) print(info)
result.set_info(info).set_status(Result.Status.ERROR).dump() 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() result.update_duration().dump()
if result.info and result.status != Result.Status.SUCCESS: if result.info and result.status != Result.Status.SUCCESS:
@ -329,7 +339,7 @@ class Runner:
workflow, job, pr=pr, branch=branch, sha=sha workflow, job, pr=pr, branch=branch, sha=sha
) )
if res: if res and (not local_run or pr or sha or branch):
res = False res = False
print(f"=== Pre run script [{job.name}], workflow [{workflow.name}] ===") print(f"=== Pre run script [{job.name}], workflow [{workflow.name}] ===")
try: try:

View File

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

View File

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

View File

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

View File

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

View File

@ -13,7 +13,8 @@ from ci.settings.definitions import (
class ArtifactNames: 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( style_check_job = Job.Config(
@ -37,10 +38,10 @@ fast_test_job = Job.Config(
), ),
) )
job_build_amd_debug = Job.Config( amd_build_jobs = Job.Config(
name=JobNames.BUILD_AMD_DEBUG, name=JobNames.BUILD,
runs_on=[RunnerLabels.BUILDER], 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", run_in_docker="clickhouse/fasttest",
digest_config=Job.CacheDigestConfig( digest_config=Job.CacheDigestConfig(
include_paths=[ include_paths=[
@ -56,20 +57,30 @@ job_build_amd_debug = Job.Config(
"./tests/ci/version_helper.py", "./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, name=JobNames.STATELESS_TESTS,
runs_on=[RunnerLabels.BUILDER], runs_on=[RunnerLabels.BUILDER],
command="python3 ./ci/jobs/functional_stateless_tests.py amd_debug", 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( digest_config=Job.CacheDigestConfig(
include_paths=[ include_paths=[
"./ci/jobs/functional_stateless_tests.py", "./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( workflow = Workflow.Config(
@ -79,15 +90,20 @@ workflow = Workflow.Config(
jobs=[ jobs=[
style_check_job, style_check_job,
fast_test_job, fast_test_job,
job_build_amd_debug, *amd_build_jobs,
stateless_tests_job, *stateless_tests_amd_debug_jobs,
], ],
artifacts=[ artifacts=[
Artifact.Config( Artifact.Config(
name=ArtifactNames.ch_debug_binary, name=ArtifactNames.CH_AMD_DEBUG,
type=Artifact.Type.S3, type=Artifact.Type.S3,
path=f"{Settings.TEMP_DIR}/build/programs/clickhouse", 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, dockers=DOCKERS,
secrets=SECRETS, secrets=SECRETS,
@ -101,8 +117,11 @@ WORKFLOWS = [
] # type: List[Workflow.Config] ] # type: List[Workflow.Config]
if __name__ == "__main__": # if __name__ == "__main__":
# local job test inside praktika environment # # local job test inside praktika environment
from praktika.runner import Runner # from praktika.runner import Runner
# from praktika.digest import Digest
Runner().run(workflow, fast_test_job, docker="fasttest", local_run=True) #
# 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.sequential_tests = []
self.parallel_tests = [] self.parallel_tests = []
for test_name in self.all_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) self.sequential_tests.append(test_name)
else: elif not args.no_parallel:
self.parallel_tests.append(test_name) self.parallel_tests.append(test_name)
def is_sequential_test(self, 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', help='Replace random database name with "default" in stderr',
) )
parser.add_argument( 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( parser.add_argument(
"-j", "--jobs", default=1, nargs="?", type=int, help="Run all tests in parallel" "-j", "--jobs", default=1, nargs="?", type=int, help="Run all tests in parallel"
@ -3339,7 +3342,7 @@ def parse_args():
parser.add_argument( parser.add_argument(
"--sequential", "--sequential",
nargs="+", nargs="+",
help="Run these tests sequentially even if --parallel specified", help="Run all tests sequentially",
) )
parser.add_argument( parser.add_argument(
"--no-long", action="store_true", dest="no_long", help="Do not run long tests" "--no-long", action="store_true", dest="no_long", help="Do not run long tests"

View File

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

View File

@ -9,6 +9,21 @@ DEST_SERVER_PATH="${1:-/etc/clickhouse-server}"
DEST_CLIENT_PATH="${2:-/etc/clickhouse-client}" DEST_CLIENT_PATH="${2:-/etc/clickhouse-client}"
SRC_PATH="$( cd "$(dirname "$0")" >/dev/null 2>&1 ; pwd -P )" 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" echo "Going to install test configs from $SRC_PATH into $DEST_SERVER_PATH"
mkdir -p $DEST_SERVER_PATH/config.d/ 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/ ln -sf $SRC_PATH/config.d/rocksdb.xml $DEST_SERVER_PATH/config.d/
# Not supported with fasttest. # Not supported with fasttest.
if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] || [ "$FAST_TEST" != "1" ]; then
then ln -sf "$SRC_PATH/config.d/legacy_geobase.xml" "$DEST_SERVER_PATH/config.d/"
ln -sf $SRC_PATH/config.d/legacy_geobase.xml $DEST_SERVER_PATH/config.d/
fi fi
ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ 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/ ln -sf $SRC_PATH/config.d/azure_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/
fi 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/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.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/storage_conf_02944.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/ ln -sf $SRC_PATH/users.d/s3_cache_new.xml $DEST_SERVER_PATH/users.d/
fi 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/users.d/database_replicated.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/config.d/database_replicated.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_replicated.xml $DEST_SERVER_PATH/config.d/
rm /etc/clickhouse-server/config.d/zookeeper.xml 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_USER=${MINIO_ROOT_USER:-clickhouse}
export MINIO_ROOT_PASSWORD=${MINIO_ROOT_PASSWORD:-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() { usage() {
echo $"Usage: $0 <stateful|stateless> <test_path> (default path: /usr/share/clickhouse-test)" echo $"Usage: $0 <stateful|stateless> <test_path> (default path: /usr/share/clickhouse-test)"
exit 1 exit 1
@ -70,9 +76,10 @@ download_minio() {
} }
start_minio() { start_minio() {
pwd
mkdir -p ./minio_data mkdir -p ./minio_data
./minio --version minio --version
./minio server --address ":11111" ./minio_data & minio server --address ":11111" ./minio_data &
wait_for_it wait_for_it
lsof -i :11111 lsof -i :11111
sleep 5 sleep 5
@ -80,12 +87,14 @@ start_minio() {
setup_minio() { setup_minio() {
local test_type=$1 local test_type=$1
./mc alias set clickminio http://localhost:11111 clickhouse clickhouse echo "setup_minio(), test_type=$test_type"
./mc admin user add clickminio test testtest mc alias set clickminio http://localhost:11111 clickhouse clickhouse
./mc admin policy attach clickminio readwrite --user=test mc admin user add clickminio test testtest
./mc mb --ignore-existing clickminio/test mc admin policy attach clickminio readwrite --user=test ||:
mc mb --ignore-existing clickminio/test
if [ "$test_type" = "stateless" ]; then if [ "$test_type" = "stateless" ]; then
./mc anonymous set public clickminio/test echo "Create @test bucket in minio"
mc anonymous set public clickminio/test
fi fi
} }
@ -95,12 +104,13 @@ upload_data() {
local query_dir=$1 local query_dir=$1
local test_path=$2 local test_path=$2
local data_path=${test_path}/queries/${query_dir}/data_minio 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 # iterating over globs will cause redundant file variable to be
# a path to a file, not a filename # a path to a file, not a filename
# shellcheck disable=SC2045 # shellcheck disable=SC2045
if [ -d "${data_path}" ]; then if [ -d "${data_path}" ]; then
./mc cp --recursive "${data_path}"/ clickminio/test/ mc cp --recursive "${data_path}"/ clickminio/test/
fi fi
} }
@ -138,7 +148,7 @@ wait_for_it() {
main() { main() {
local query_dir local query_dir
query_dir=$(check_arg "$@") query_dir=$(check_arg "$@")
if [ ! -f ./minio ]; then if ! (minio --version && mc --version); then
download_minio download_minio
fi fi
start_minio start_minio