Merge pull request #71081 from ClickHouse/ci_paktika_integration_4

CI: Functional Tests with praktika
This commit is contained in:
Max Kainov 2024-11-15 10:46:01 +00:00 committed by GitHub
commit 3b0b42cbbb
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
43 changed files with 2044 additions and 892 deletions

0
ci/__init__.py Normal file
View File

View File

@ -0,0 +1,14 @@
ARG FROM_TAG=latest
FROM clickhouse/stateless-test:$FROM_TAG
USER root
RUN apt-get update -y \
&& env DEBIAN_FRONTEND=noninteractive \
apt-get install --yes --no-install-recommends \
nodejs \
npm \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \
USER clickhouse

View File

@ -0,0 +1,117 @@
# 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"
RUN mkdir /etc/clickhouse-server /etc/clickhouse-keeper /etc/clickhouse-client && chmod 777 /etc/clickhouse-* \
&& mkdir -p /var/lib/clickhouse /var/log/clickhouse-server && chmod 777 /var/log/clickhouse-server /var/lib/clickhouse
RUN addgroup --gid 1001 clickhouse && adduser --uid 1001 --gid 1001 --disabled-password clickhouse
# 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 \
xz-utils \
&& 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 \
&& chmod 777 /hadoop-3.3.1
RUN npm install -g azurite@3.30.0 \
&& npm install -g tslib && npm install -g node
USER clickhouse

View File

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

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

View File

@ -13,11 +13,30 @@ 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_sanitizer>")
parser.add_argument("--param", help="Optional custom job start stage", default=None)
parser.add_argument(
"--build-type",
help="Type: <amd|arm>,<debug|release>,<asan|msan|..>",
)
parser.add_argument(
"--param",
help="Optional user-defined job start stage (for local run)",
default=None,
)
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_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()
@ -33,23 +52,41 @@ def main():
stages.pop(0)
stages.insert(0, stage)
cmake_build_type = "Release"
sanitizer = ""
build_type = args.build_type
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 "debug" in args.BUILD_TYPE.lower():
CACHE_TYPE = "sccache"
BUILD_TYPE = "RelWithDebInfo"
SANITIZER = ""
AUX_DEFS = " -DENABLE_TESTS=0 "
if "debug" in build_type:
print("Build type set: debug")
cmake_build_type = "Debug"
if "asan" in args.BUILD_TYPE.lower():
BUILD_TYPE = "Debug"
AUX_DEFS = " -DENABLE_TESTS=1 "
elif "release" in build_type:
print("Build type set: release")
AUX_DEFS = (
" -DENABLE_TESTS=0 -DSPLIT_DEBUG_SYMBOLS=ON -DBUILD_STANDALONE_KEEPER=1 "
)
elif "asan" in build_type:
print("Sanitizer set: address")
sanitizer = "address"
SANITIZER = "address"
else:
assert False
# if Environment.is_local_run():
# build_cache_type = "disabled"
# else:
build_cache_type = "sccache"
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
@ -69,12 +106,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,
)
@ -95,7 +127,7 @@ def main():
Shell.check(f"ls -l {build_dir}/programs/")
res = results[-1].is_ok()
Result.create_from(results=results, stopwatch=stop_watch).finish_job_accordingly()
Result.create_from(results=results, stopwatch=stop_watch).complete_job()
if __name__ == "__main__":

View File

@ -379,4 +379,4 @@ if __name__ == "__main__":
)
)
Result.create_from(results=results, stopwatch=stop_watch).finish_job_accordingly()
Result.create_from(results=results, stopwatch=stop_watch).complete_job()

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"
@ -322,15 +215,17 @@ def main():
)
if res and JobStages.TEST in stages:
stop_watch_ = Utils.Stopwatch()
step_name = "Tests"
print(step_name)
res = res and CH.run_fast_test()
if res:
results.append(FTResultsProcessor(wd=Settings.OUTPUT_DIR).run())
results[-1].set_timing(stopwatch=stop_watch_)
CH.terminate()
Result.create_from(results=results, stopwatch=stop_watch).finish_job_accordingly()
Result.create_from(results=results, stopwatch=stop_watch).complete_job()
if __name__ == "__main__":

View File

@ -0,0 +1,170 @@
import argparse
import os
import time
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 JobStages(metaclass=MetaClasses.WithIter):
INSTALL_CLICKHOUSE = "install"
START = "start"
TEST = "test"
def parse_args():
parser = argparse.ArgumentParser(description="ClickHouse Build Job")
parser.add_argument(
"--ch-path", help="Path to clickhouse binary", default=f"{Settings.INPUT_DIR}"
)
parser.add_argument(
"--test-options",
help="Comma separated option(s): parallel|non-parallel|BATCH_NUM/BTATCH_TOT|..",
default="",
)
parser.add_argument("--param", help="Optional job start stage", default=None)
parser.add_argument("--test", help="Optional test name pattern", default="")
return parser.parse_args()
def run_test(
no_parallel: bool, no_sequiential: bool, batch_num: int, batch_total: int, test=""
):
test_output_file = f"{Settings.OUTPUT_DIR}/test_result.txt"
test_command = f"clickhouse-test --jobs 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless \
--hung-check --print-time \
--capture-client-stacktrace --queries ./tests/queries -- '{test}' \
| 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(test_command, verbose=True)
def main():
args = parse_args()
test_options = args.test_options.split(",")
no_parallel = "non-parallel" in test_options
no_sequential = "parallel" in test_options
batch_num, total_batches = 0, 0
for to in test_options:
if "/" in to:
batch_num, total_batches = map(int, to.split("/"))
# 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,
# strict=True
# )
ch_path = args.ch_path
assert Path(
ch_path + "/clickhouse"
).is_file(), f"clickhouse binary not found under [{ch_path}]"
stop_watch = Utils.Stopwatch()
stages = list(JobStages)
logs_to_attach = []
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}]")
while stage in stages:
stages.pop(0)
stages.insert(0, stage)
res = True
results = []
Utils.add_to_PATH(f"{ch_path}:tests")
if res and JobStages.INSTALL_CLICKHOUSE in stages:
commands = [
f"rm -rf /tmp/praktika/var/log/clickhouse-server/clickhouse-server.*",
f"chmod +x {ch_path}/clickhouse",
f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-server",
f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-client",
f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-compressor",
f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-local",
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",
# clickhouse benchmark segfaults with --config-path, so provide client config by its default location
f"cp {Settings.TEMP_DIR}/etc/clickhouse-client/* /etc/clickhouse-client/",
# 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"for file in /tmp/praktika/etc/clickhouse-server/config.d/*.xml; do [ -f $file ] && echo Change config $file && sed -i 's|>/var/log|>{Settings.TEMP_DIR}/var/log|g; s|>/etc/|>{Settings.TEMP_DIR}/etc/|g' $(readlink -f $file); done",
f"for file in /tmp/praktika/etc/clickhouse-server/*.xml; do [ -f $file ] && echo Change config $file && sed -i 's|>/var/log|>{Settings.TEMP_DIR}/var/log|g; s|>/etc/|>{Settings.TEMP_DIR}/etc/|g' $(readlink -f $file); done",
f"for file in /tmp/praktika/etc/clickhouse-server/config.d/*.xml; do [ -f $file ] && echo Change config $file && sed -i 's|<path>local_disk|<path>{Settings.TEMP_DIR}/local_disk|g' $(readlink -f $file); done",
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)
minio_log = "/tmp/praktika/output/minio.log"
res = res and CH.start_minio(test_type="stateful", log_file_path=minio_log)
logs_to_attach += [minio_log]
time.sleep(10)
Shell.check("ps -ef | grep minio", verbose=True)
res = res and Shell.check(
"aws s3 ls s3://test --endpoint-url http://localhost:11111/", verbose=True
)
res = res and CH.start()
res = res and CH.wait_ready()
if res:
print("ch started")
logs_to_attach += [
"/tmp/praktika/var/log/clickhouse-server/clickhouse-server.log",
"/tmp/praktika/var/log/clickhouse-server/clickhouse-server.err.log",
]
results.append(
Result.create_from(
name=step_name,
status=res,
stopwatch=stop_watch_,
)
)
res = results[-1].is_ok()
if res and JobStages.TEST in stages:
stop_watch_ = Utils.Stopwatch()
step_name = "Tests"
print(step_name)
# assert Shell.check("clickhouse-client -q \"insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')\"", verbose=True)
run_test(
no_parallel=no_parallel,
no_sequiential=no_sequential,
batch_num=batch_num,
batch_total=total_batches,
test=args.test,
)
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, files=logs_to_attach if not res else []
).complete_job()
if __name__ == "__main__":
main()

View File

@ -0,0 +1,183 @@
import argparse
import os
import time
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 JobStages(metaclass=MetaClasses.WithIter):
INSTALL_CLICKHOUSE = "install"
START = "start"
TEST = "test"
def parse_args():
parser = argparse.ArgumentParser(description="ClickHouse Build Job")
parser.add_argument(
"--ch-path", help="Path to clickhouse binary", default=f"{Settings.INPUT_DIR}"
)
parser.add_argument(
"--test-options",
help="Comma separated option(s): parallel|non-parallel|BATCH_NUM/BTATCH_TOT|..",
default="",
)
parser.add_argument("--param", help="Optional job start stage", default=None)
parser.add_argument("--test", help="Optional test name pattern", default="")
return parser.parse_args()
def run_stateless_test(
no_parallel: bool, no_sequiential: bool, batch_num: int, batch_total: int, test=""
):
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 -- '{test}' | 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()
test_options = args.test_options.split(",")
no_parallel = "non-parallel" in test_options
no_sequential = "parallel" in test_options
batch_num, total_batches = 0, 0
for to in test_options:
if "/" in to:
batch_num, total_batches = map(int, to.split("/"))
# 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,
# strict=True
# )
ch_path = args.ch_path
assert Path(
ch_path + "/clickhouse"
).is_file(), f"clickhouse binary not found under [{ch_path}]"
stop_watch = Utils.Stopwatch()
stages = list(JobStages)
logs_to_attach = []
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}]")
while stage in stages:
stages.pop(0)
stages.insert(0, stage)
res = True
results = []
Utils.add_to_PATH(f"{ch_path}:tests")
if res and JobStages.INSTALL_CLICKHOUSE in stages:
commands = [
f"rm -rf /tmp/praktika/var/log/clickhouse-server/clickhouse-server.*",
f"chmod +x {ch_path}/clickhouse",
f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-server",
f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-client",
f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-compressor",
f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-local",
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/",
# TODO: find a way to work with Azure secret so it's ok for local tests as well, for now keep azure disabled
f"./tests/config/install.sh {Settings.TEMP_DIR}/etc/clickhouse-server {Settings.TEMP_DIR}/etc/clickhouse-client --s3-storage --no-azure",
# clickhouse benchmark segfaults with --config-path, so provide client config by its default location
f"cp {Settings.TEMP_DIR}/etc/clickhouse-client/* /etc/clickhouse-client/",
# 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"for file in /tmp/praktika/etc/clickhouse-server/config.d/*.xml; do [ -f $file ] && echo Change config $file && sed -i 's|>/var/log|>{Settings.TEMP_DIR}/var/log|g; s|>/etc/|>{Settings.TEMP_DIR}/etc/|g' $(readlink -f $file); done",
f"for file in /tmp/praktika/etc/clickhouse-server/*.xml; do [ -f $file ] && echo Change config $file && sed -i 's|>/var/log|>{Settings.TEMP_DIR}/var/log|g; s|>/etc/|>{Settings.TEMP_DIR}/etc/|g' $(readlink -f $file); done",
f"for file in /tmp/praktika/etc/clickhouse-server/config.d/*.xml; do [ -f $file ] && echo Change config $file && sed -i 's|<path>local_disk|<path>{Settings.TEMP_DIR}/local_disk|g' $(readlink -f $file); done",
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)
hdfs_log = "/tmp/praktika/output/hdfs_mini.log"
minio_log = "/tmp/praktika/output/minio.log"
res = res and CH.start_hdfs(log_file_path=hdfs_log)
res = res and CH.start_minio(test_type="stateful", log_file_path=minio_log)
logs_to_attach += [minio_log, hdfs_log]
time.sleep(10)
Shell.check("ps -ef | grep minio", verbose=True)
Shell.check("ps -ef | grep hdfs", verbose=True)
res = res and Shell.check(
"aws s3 ls s3://test --endpoint-url http://localhost:11111/", verbose=True
)
res = res and CH.start()
res = res and CH.wait_ready()
if res:
print("ch started")
logs_to_attach += [
"/tmp/praktika/var/log/clickhouse-server/clickhouse-server.log",
"/tmp/praktika/var/log/clickhouse-server/clickhouse-server.err.log",
]
results.append(
Result.create_from(
name=step_name,
status=res,
stopwatch=stop_watch_,
)
)
res = results[-1].is_ok()
if res and JobStages.TEST in stages:
stop_watch_ = Utils.Stopwatch()
step_name = "Tests"
print(step_name)
assert Shell.check(
"clickhouse-client -q \"insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')\"",
verbose=True,
)
run_stateless_test(
no_parallel=no_parallel,
no_sequiential=no_sequential,
batch_num=batch_num,
batch_total=total_batches,
test=args.test,
)
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, files=logs_to_attach if not res else []
).complete_job()
if __name__ == "__main__":
main()

View File

View File

@ -0,0 +1,142 @@
import subprocess
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 = ""
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_hdfs(self, log_file_path):
command = ["./ci/jobs/scripts/functional_tests/setup_hdfs_minicluster.sh"]
with open(log_file_path, "w") as log_file:
process = subprocess.Popen(
command, stdout=log_file, stderr=subprocess.STDOUT
)
print(
f"Started setup_hdfs_minicluster.sh asynchronously with PID {process.pid}"
)
return True
def start_minio(self, test_type, log_file_path):
command = [
"./ci/jobs/scripts/functional_tests/setup_minio.sh",
test_type,
"./tests",
]
with open(log_file_path, "w") as log_file:
process = subprocess.Popen(
command, stdout=log_file, stderr=subprocess.STDOUT
)
print(f"Started setup_minio.sh asynchronously with PID {process.pid}")
return True
def start(self):
print("Starting ClickHouse server")
Shell.check(f"rm {self.pid_file}")
self.proc = subprocess.Popen(self.command, stderr=subprocess.STDOUT, shell=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.")
if self.minio_proc:
Utils.terminate_process_group(self.minio_proc.pid)

View File

@ -0,0 +1,19 @@
#!/bin/bash
# shellcheck disable=SC2024
set -e -x -a -u
ls -lha
cd /hadoop-3.3.1
export JAVA_HOME=/usr
mkdir -p target/test/data
bin/mapred minicluster -format -nomr -nnport 12222 &
while ! nc -z localhost 12222; do
sleep 1
done
lsof -i :12222

View File

@ -0,0 +1,162 @@
#!/bin/bash
set -euxf -o pipefail
export MINIO_ROOT_USER=${MINIO_ROOT_USER:-clickhouse}
export MINIO_ROOT_PASSWORD=${MINIO_ROOT_PASSWORD:-clickhouse}
TEST_DIR=${2:-/repo/tests/}
if [ -d "$TEMP_DIR" ]; then
TEST_DIR=$(readlink -f $TEST_DIR)
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
}
check_arg() {
local query_dir
if [ ! $# -eq 1 ]; then
if [ ! $# -eq 2 ]; then
echo "ERROR: need either one or two arguments, <stateful|stateless> <test_path> (default path: /usr/share/clickhouse-test)"
usage
fi
fi
case "$1" in
stateless)
query_dir="0_stateless"
;;
stateful)
query_dir="1_stateful"
;;
*)
echo "unknown test type ${test_type}"
usage
;;
esac
echo ${query_dir}
}
find_arch() {
local arch
case $(uname -m) in
x86_64)
arch="amd64"
;;
aarch64)
arch="arm64"
;;
*)
echo "unknown architecture $(uname -m)";
exit 1
;;
esac
echo ${arch}
}
find_os() {
local os
os=$(uname -s | tr '[:upper:]' '[:lower:]')
echo "${os}"
}
download_minio() {
local os
local arch
local minio_server_version=${MINIO_SERVER_VERSION:-2024-08-03T04-33-23Z}
local minio_client_version=${MINIO_CLIENT_VERSION:-2024-07-31T15-58-33Z}
os=$(find_os)
arch=$(find_arch)
wget "https://dl.min.io/server/minio/release/${os}-${arch}/archive/minio.RELEASE.${minio_server_version}" -O ./minio
wget "https://dl.min.io/client/mc/release/${os}-${arch}/archive/mc.RELEASE.${minio_client_version}" -O ./mc
chmod +x ./mc ./minio
}
start_minio() {
pwd
mkdir -p ./minio_data
minio --version
nohup minio server --address ":11111" ./minio_data &
wait_for_it
lsof -i :11111
sleep 5
}
setup_minio() {
local test_type=$1
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
echo "Create @test bucket in minio"
mc anonymous set public clickminio/test
fi
}
# uploads data to minio, by default after unpacking all tests
# will be in /usr/share/clickhouse-test/queries
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/
fi
}
setup_aws_credentials() {
local minio_root_user=${MINIO_ROOT_USER:-clickhouse}
local minio_root_password=${MINIO_ROOT_PASSWORD:-clickhouse}
mkdir -p ~/.aws
cat <<EOT >> ~/.aws/credentials
[default]
aws_access_key_id=${minio_root_user}
aws_secret_access_key=${minio_root_password}
EOT
}
wait_for_it() {
local counter=0
local max_counter=60
local url="http://localhost:11111"
local params=(
--silent
--verbose
)
while ! curl "${params[@]}" "${url}" 2>&1 | grep AccessDenied
do
if [[ ${counter} == "${max_counter}" ]]; then
echo "failed to setup minio"
exit 0
fi
echo "trying to connect to minio"
sleep 1
counter=$((counter + 1))
done
}
main() {
local query_dir
query_dir=$(check_arg "$@")
if ! (minio --version && mc --version); then
download_minio
fi
start_minio
setup_minio "$1"
upload_data "${query_dir}" "$TEST_DIR"
setup_aws_credentials
}
main "$@"

View File

@ -1,7 +1,6 @@
import dataclasses
from typing import List
from praktika.environment import Environment
from praktika.result import Result
OK_SIGN = "[ OK "
@ -233,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
@ -250,10 +251,11 @@ class FTResultsProcessor:
# test_results.sort(key=test_result_comparator)
return Result.create_from(
name=Environment.JOB_NAME,
name="Tests",
results=test_results,
status=state,
files=[self.tests_output_file],
info=info,
with_info_from_results=False,
)

View File

@ -37,6 +37,30 @@ def create_parser():
type=str,
default=None,
)
run_parser.add_argument(
"--test",
help="Custom parameter to pass into a job script, it's up to job script how to use it, for local test",
type=str,
default="",
)
run_parser.add_argument(
"--pr",
help="PR number. Optional parameter for local run. Set if you want an required artifact to be uploaded from CI run in that PR",
type=int,
default=None,
)
run_parser.add_argument(
"--sha",
help="Commit sha. Optional parameter for local run. Set if you want an required artifact to be uploaded from CI run on that sha, head sha will be used if not set",
type=str,
default=None,
)
run_parser.add_argument(
"--branch",
help="Commit sha. Optional parameter for local run. Set if you want an required artifact to be uploaded from CI run on that branch, main branch name will be used if not set",
type=str,
default=None,
)
run_parser.add_argument(
"--ci",
help="When not set - dummy env will be generated, for local test",
@ -85,9 +109,13 @@ if __name__ == "__main__":
workflow=workflow,
job=job,
docker=args.docker,
dummy_env=not args.ci,
local_run=not args.ci,
no_docker=args.no_docker,
param=args.param,
test=args.test,
pr=args.pr,
branch=args.branch,
sha=args.sha,
)
else:
parser.print_help()

View File

@ -6,7 +6,7 @@ from types import SimpleNamespace
from typing import Any, Dict, List, Type
from praktika import Workflow
from praktika._settings import _Settings
from praktika.settings import Settings
from praktika.utils import MetaClasses, T
@ -30,13 +30,12 @@ class _Environment(MetaClasses.Serializable):
INSTANCE_ID: str
INSTANCE_LIFE_CYCLE: str
LOCAL_RUN: bool = False
PARAMETER: Any = None
REPORT_INFO: List[str] = dataclasses.field(default_factory=list)
name = "environment"
@classmethod
def file_name_static(cls, _name=""):
return f"{_Settings.TEMP_DIR}/{cls.name}.json"
return f"{Settings.TEMP_DIR}/{cls.name}.json"
@classmethod
def from_dict(cls: Type[T], obj: Dict[str, Any]) -> T:
@ -67,12 +66,12 @@ class _Environment(MetaClasses.Serializable):
@staticmethod
def get_needs_statuses():
if Path(_Settings.WORKFLOW_STATUS_FILE).is_file():
with open(_Settings.WORKFLOW_STATUS_FILE, "r", encoding="utf8") as f:
if Path(Settings.WORKFLOW_STATUS_FILE).is_file():
with open(Settings.WORKFLOW_STATUS_FILE, "r", encoding="utf8") as f:
return json.load(f)
else:
print(
f"ERROR: Status file [{_Settings.WORKFLOW_STATUS_FILE}] does not exist"
f"ERROR: Status file [{Settings.WORKFLOW_STATUS_FILE}] does not exist"
)
raise RuntimeError()
@ -159,7 +158,8 @@ class _Environment(MetaClasses.Serializable):
@classmethod
def get_s3_prefix_static(cls, pr_number, branch, sha, latest=False):
prefix = ""
if pr_number > 0:
assert sha or latest
if pr_number and pr_number > 0:
prefix += f"{pr_number}"
else:
prefix += f"{branch}"
@ -171,18 +171,15 @@ class _Environment(MetaClasses.Serializable):
# TODO: find a better place for the function. This file should not import praktika.settings
# as it's requires reading users config, that's why imports nested inside the function
def get_report_url(self):
def get_report_url(self, settings, latest=False):
import urllib
from praktika.settings import Settings
from praktika.utils import Utils
path = Settings.HTML_S3_PATH
for bucket, endpoint in Settings.S3_BUCKET_TO_HTTP_ENDPOINT.items():
path = settings.HTML_S3_PATH
for bucket, endpoint in settings.S3_BUCKET_TO_HTTP_ENDPOINT.items():
if bucket in path:
path = path.replace(bucket, endpoint)
break
REPORT_URL = f"https://{path}/{Path(Settings.HTML_PAGE_FILE).name}?PR={self.PR_NUMBER}&sha={self.SHA}&name_0={urllib.parse.quote(self.WORKFLOW_NAME, safe='')}&name_1={urllib.parse.quote(self.JOB_NAME, safe='')}"
REPORT_URL = f"https://{path}/{Path(settings.HTML_PAGE_FILE).name}?PR={self.PR_NUMBER}&sha={'latest' if latest else self.SHA}&name_0={urllib.parse.quote(self.WORKFLOW_NAME, safe='')}&name_1={urllib.parse.quote(self.JOB_NAME, safe='')}"
return REPORT_URL
def is_local_run(self):

View File

@ -1,124 +0,0 @@
import dataclasses
from pathlib import Path
from typing import Dict, Iterable, List, Optional
@dataclasses.dataclass
class _Settings:
######################################
# Pipeline generation settings #
######################################
CI_PATH = "./ci"
WORKFLOW_PATH_PREFIX: str = "./.github/workflows"
WORKFLOWS_DIRECTORY: str = f"{CI_PATH}/workflows"
SETTINGS_DIRECTORY: str = f"{CI_PATH}/settings"
CI_CONFIG_JOB_NAME = "Config Workflow"
DOCKER_BUILD_JOB_NAME = "Docker Builds"
FINISH_WORKFLOW_JOB_NAME = "Finish Workflow"
READY_FOR_MERGE_STATUS_NAME = "Ready for Merge"
CI_CONFIG_RUNS_ON: Optional[List[str]] = None
DOCKER_BUILD_RUNS_ON: Optional[List[str]] = None
VALIDATE_FILE_PATHS: bool = True
######################################
# Runtime Settings #
######################################
MAX_RETRIES_S3 = 3
MAX_RETRIES_GH = 3
######################################
# S3 (artifact storage) settings #
######################################
S3_ARTIFACT_PATH: str = ""
######################################
# CI workspace settings #
######################################
TEMP_DIR: str = "/tmp/praktika"
OUTPUT_DIR: str = f"{TEMP_DIR}/output"
INPUT_DIR: str = f"{TEMP_DIR}/input"
PYTHON_INTERPRETER: str = "python3"
PYTHON_PACKET_MANAGER: str = "pip3"
PYTHON_VERSION: str = "3.9"
INSTALL_PYTHON_FOR_NATIVE_JOBS: bool = False
INSTALL_PYTHON_REQS_FOR_NATIVE_JOBS: str = "./ci/requirements.txt"
ENVIRONMENT_VAR_FILE: str = f"{TEMP_DIR}/environment.json"
RUN_LOG: str = f"{TEMP_DIR}/praktika_run.log"
SECRET_GH_APP_ID: str = "GH_APP_ID"
SECRET_GH_APP_PEM_KEY: str = "GH_APP_PEM_KEY"
ENV_SETUP_SCRIPT: str = "/tmp/praktika_setup_env.sh"
WORKFLOW_STATUS_FILE: str = f"{TEMP_DIR}/workflow_status.json"
######################################
# CI Cache settings #
######################################
CACHE_VERSION: int = 1
CACHE_DIGEST_LEN: int = 20
CACHE_S3_PATH: str = ""
CACHE_LOCAL_PATH: str = f"{TEMP_DIR}/ci_cache"
######################################
# Report settings #
######################################
HTML_S3_PATH: str = ""
HTML_PAGE_FILE: str = "./praktika/json.html"
TEXT_CONTENT_EXTENSIONS: Iterable[str] = frozenset([".txt", ".log"])
S3_BUCKET_TO_HTTP_ENDPOINT: Optional[Dict[str, str]] = None
DOCKERHUB_USERNAME: str = ""
DOCKERHUB_SECRET: str = ""
DOCKER_WD: str = "/wd"
######################################
# CI DB Settings #
######################################
SECRET_CI_DB_URL: str = "CI_DB_URL"
SECRET_CI_DB_PASSWORD: str = "CI_DB_PASSWORD"
CI_DB_DB_NAME = ""
CI_DB_TABLE_NAME = ""
CI_DB_INSERT_TIMEOUT_SEC = 5
_USER_DEFINED_SETTINGS = [
"S3_ARTIFACT_PATH",
"CACHE_S3_PATH",
"HTML_S3_PATH",
"S3_BUCKET_TO_HTTP_ENDPOINT",
"TEXT_CONTENT_EXTENSIONS",
"TEMP_DIR",
"OUTPUT_DIR",
"INPUT_DIR",
"CI_CONFIG_RUNS_ON",
"DOCKER_BUILD_RUNS_ON",
"CI_CONFIG_JOB_NAME",
"PYTHON_INTERPRETER",
"PYTHON_VERSION",
"PYTHON_PACKET_MANAGER",
"INSTALL_PYTHON_FOR_NATIVE_JOBS",
"INSTALL_PYTHON_REQS_FOR_NATIVE_JOBS",
"MAX_RETRIES_S3",
"MAX_RETRIES_GH",
"VALIDATE_FILE_PATHS",
"DOCKERHUB_USERNAME",
"DOCKERHUB_SECRET",
"READY_FOR_MERGE_STATUS_NAME",
"SECRET_CI_DB_URL",
"SECRET_CI_DB_PASSWORD",
"CI_DB_DB_NAME",
"CI_DB_TABLE_NAME",
"CI_DB_INSERT_TIMEOUT_SEC",
"SECRET_GH_APP_PEM_KEY",
"SECRET_GH_APP_ID",
]
class GHRunners:
ubuntu = "ubuntu-latest"
if __name__ == "__main__":
for setting in _USER_DEFINED_SETTINGS:
print(_Settings().__getattribute__(setting))
# print(dataclasses.asdict(_Settings()))

View File

@ -52,7 +52,7 @@ class CIDB:
check_status=result.status,
check_duration_ms=int(result.duration * 1000),
check_start_time=Utils.timestamp_to_str(result.start_time),
report_url=env.get_report_url(),
report_url=env.get_report_url(settings=Settings),
pull_request_url=env.CHANGE_URL,
base_ref=env.BASE_BRANCH,
base_repo=env.REPOSITORY,

View File

@ -23,7 +23,7 @@ class Digest:
hash_string = hash_obj.hexdigest()
return hash_string
def calc_job_digest(self, job_config: Job.Config):
def calc_job_digest(self, job_config: Job.Config, docker_digests):
config = job_config.digest_config
if not config:
return "f" * Settings.CACHE_DIGEST_LEN
@ -31,32 +31,32 @@ class Digest:
cache_key = self._hash_digest_config(config)
if cache_key in self.digest_cache:
return self.digest_cache[cache_key]
included_files = Utils.traverse_paths(
job_config.digest_config.include_paths,
job_config.digest_config.exclude_paths,
sorted=True,
)
print(
f"calc digest for job [{job_config.name}]: hash_key [{cache_key}], include [{len(included_files)}] files"
)
# Sort files to ensure consistent hash calculation
included_files.sort()
# Calculate MD5 hash
res = ""
if not included_files:
res = "f" * Settings.CACHE_DIGEST_LEN
print(f"NOTE: empty digest config [{config}] - return dummy digest")
print(
f"calc digest for job [{job_config.name}]: hash_key [{cache_key}] - from cache"
)
digest = self.digest_cache[cache_key]
else:
included_files = Utils.traverse_paths(
job_config.digest_config.include_paths,
job_config.digest_config.exclude_paths,
sorted=True,
)
print(
f"calc digest for job [{job_config.name}]: hash_key [{cache_key}], include [{len(included_files)}] files"
)
hash_md5 = hashlib.md5()
for file_path in included_files:
res = self._calc_file_digest(file_path, hash_md5)
assert res
self.digest_cache[cache_key] = res
return res
for i, file_path in enumerate(included_files):
hash_md5 = self._calc_file_digest(file_path, hash_md5)
digest = hash_md5.hexdigest()[: Settings.CACHE_DIGEST_LEN]
self.digest_cache[cache_key] = digest
if job_config.run_in_docker:
# respect docker digest in the job digest
docker_digest = docker_digests[job_config.run_in_docker.split("+")[0]]
digest = "-".join([docker_digest, digest])
return digest
def calc_docker_digest(
self,
@ -103,10 +103,10 @@ class Digest:
print(
f"WARNING: No valid file resolved by link {file_path} -> {resolved_path} - skipping digest calculation"
)
return hash_md5.hexdigest()[: Settings.CACHE_DIGEST_LEN]
return hash_md5
with open(resolved_path, "rb") as f:
for chunk in iter(lambda: f.read(4096), b""):
hash_md5.update(chunk)
return hash_md5.hexdigest()[: Settings.CACHE_DIGEST_LEN]
return hash_md5

View File

@ -1,3 +0,0 @@
from praktika._environment import _Environment
Environment = _Environment.get()

View File

@ -18,7 +18,7 @@ class GH:
ret_code, out, err = Shell.get_res_stdout_stderr(command, verbose=True)
res = ret_code == 0
if not res and "Validation Failed" in err:
print("ERROR: GH command validation error")
print(f"ERROR: GH command validation error.")
break
if not res and "Bad credentials" in err:
print("ERROR: GH credentials/auth failure")

View File

@ -1,6 +1,5 @@
from praktika._environment import _Environment
from praktika.cache import Cache
from praktika.mangle import _get_workflows
from praktika.runtime import RunConfig
from praktika.settings import Settings
from praktika.utils import Utils
@ -8,11 +7,10 @@ from praktika.utils import Utils
class CacheRunnerHooks:
@classmethod
def configure(cls, _workflow):
workflow_config = RunConfig.from_fs(_workflow.name)
def configure(cls, workflow):
workflow_config = RunConfig.from_fs(workflow.name)
docker_digests = workflow_config.digest_dockers
cache = Cache()
assert _Environment.get().WORKFLOW_NAME
workflow = _get_workflows(name=_Environment.get().WORKFLOW_NAME)[0]
print(f"Workflow Configure, workflow [{workflow.name}]")
assert (
workflow.enable_cache
@ -20,11 +18,13 @@ class CacheRunnerHooks:
artifact_digest_map = {}
job_digest_map = {}
for job in workflow.jobs:
digest = cache.digest.calc_job_digest(
job_config=job, docker_digests=docker_digests
)
if not job.digest_config:
print(
f"NOTE: job [{job.name}] has no Config.digest_config - skip cache check, always run"
)
digest = cache.digest.calc_job_digest(job_config=job)
job_digest_map[job.name] = digest
if job.provides:
# assign the job digest also to the artifacts it provides
@ -50,7 +50,6 @@ class CacheRunnerHooks:
), f"BUG, Workflow with enabled cache must have job digests after configuration, wf [{workflow.name}]"
print("Check remote cache")
job_to_cache_record = {}
for job_name, job_digest in workflow_config.digest_jobs.items():
record = cache.fetch_success(job_name=job_name, job_digest=job_digest)
if record:
@ -60,7 +59,7 @@ class CacheRunnerHooks:
)
workflow_config.cache_success.append(job_name)
workflow_config.cache_success_base64.append(Utils.to_base64(job_name))
job_to_cache_record[job_name] = record
workflow_config.cache_jobs[job_name] = record
print("Check artifacts to reuse")
for job in workflow.jobs:
@ -68,7 +67,7 @@ class CacheRunnerHooks:
if job.provides:
for artifact_name in job.provides:
workflow_config.cache_artifacts[artifact_name] = (
job_to_cache_record[job.name]
workflow_config.cache_jobs[job.name]
)
print(f"Write config to GH's job output")

View File

@ -1,63 +1,125 @@
import dataclasses
import json
import urllib.parse
from pathlib import Path
from typing import List
from praktika._environment import _Environment
from praktika.gh import GH
from praktika.parser import WorkflowConfigParser
from praktika.result import Result, ResultInfo
from praktika.result import Result, ResultInfo, _ResultS3
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
job_cache_records = RunConfig.from_fs(_workflow.name).cache_jobs
else:
skip_jobs = []
@ -67,36 +129,22 @@ class HtmlRunnerHooks:
if job.name not in skip_jobs:
result = Result.generate_pending(job.name)
else:
result = Result.generate_skipped(job.name)
result = Result.generate_skipped(job.name, job_cache_records[job.name])
results.append(result)
summary_result = Result.generate_pending(_workflow.name, results=results)
summary_result.aux_links.append(env.CHANGE_URL)
summary_result.aux_links.append(env.RUN_URL)
summary_result.links.append(env.CHANGE_URL)
summary_result.links.append(env.RUN_URL)
summary_result.start_time = Utils.timestamp()
page_url = "/".join(
["https:/", Settings.HTML_S3_PATH, str(Path(Settings.HTML_PAGE_FILE).name)]
)
for bucket, endpoint in Settings.S3_BUCKET_TO_HTTP_ENDPOINT.items():
page_url = page_url.replace(bucket, endpoint)
# TODO: add support for non-PRs (use branch?)
page_url += f"?PR={env.PR_NUMBER}&sha=latest&name_0={urllib.parse.quote(env.WORKFLOW_NAME, safe='')}"
summary_result.html_link = page_url
# clean the previous latest results in PR if any
if env.PR_NUMBER:
S3.clean_latest_result()
S3.copy_result_to_s3(
summary_result,
unlock=False,
)
assert _ResultS3.copy_result_to_s3_with_version(summary_result, version=0)
page_url = env.get_report_url(settings=Settings)
print(f"CI Status page url [{page_url}]")
res1 = GH.post_commit_status(
name=_workflow.name,
status=Result.Status.PENDING,
description="",
url=page_url,
url=env.get_report_url(settings=Settings, latest=True),
)
res2 = GH.post_pr_comment(
comment_body=f"Workflow [[{_workflow.name}]({page_url})], commit [{_Environment.get().SHA[:8]}]",
@ -106,23 +154,15 @@ 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):
result = Result.from_fs(_job.name)
S3.copy_result_from_s3(
Result.file_name_static(_workflow.name),
)
workflow_result = Result.from_fs(_workflow.name)
workflow_result.update_sub_result(result)
S3.copy_result_to_s3(
workflow_result,
unlock=True,
_ResultS3.update_workflow_results(
workflow_name=_workflow.name, new_sub_results=result
)
@classmethod
@ -132,14 +172,13 @@ class HtmlRunnerHooks:
@classmethod
def post_run(cls, _workflow, _job, info_errors):
result = Result.from_fs(_job.name)
env = _Environment.get()
S3.copy_result_from_s3(
Result.file_name_static(_workflow.name),
lock=True,
)
workflow_result = Result.from_fs(_workflow.name)
print(f"Workflow info [{workflow_result.info}], info_errors [{info_errors}]")
_ResultS3.upload_result_files_to_s3(result)
_ResultS3.copy_result_to_s3(result)
env = _Environment.get()
new_sub_results = [result]
new_result_info = ""
env_info = env.REPORT_INFO
if env_info:
print(
@ -151,14 +190,8 @@ class HtmlRunnerHooks:
info_str = f"{_job.name}:\n"
info_str += "\n".join(info_errors)
print("Update workflow results with new info")
workflow_result.set_info(info_str)
new_result_info = info_str
old_status = workflow_result.status
S3.upload_result_files_to_s3(result)
workflow_result.update_sub_result(result)
skipped_job_results = []
if not result.is_ok():
print(
"Current job failed - find dependee jobs in the workflow and set their statuses to skipped"
@ -171,7 +204,7 @@ class HtmlRunnerHooks:
print(
f"NOTE: Set job [{dependee_job.name}] status to [{Result.Status.SKIPPED}] due to current failure"
)
skipped_job_results.append(
new_sub_results.append(
Result(
name=dependee_job.name,
status=Result.Status.SKIPPED,
@ -179,20 +212,18 @@ class HtmlRunnerHooks:
+ f" [{_job.name}]",
)
)
for skipped_job_result in skipped_job_results:
workflow_result.update_sub_result(skipped_job_result)
S3.copy_result_to_s3(
workflow_result,
unlock=True,
updated_status = _ResultS3.update_workflow_results(
new_info=new_result_info,
new_sub_results=new_sub_results,
workflow_name=_workflow.name,
)
if workflow_result.status != old_status:
print(
f"Update GH commit status [{result.name}]: [{old_status} -> {workflow_result.status}], link [{workflow_result.html_link}]"
)
if updated_status:
print(f"Update GH commit status [{result.name}]: [{updated_status}]")
GH.post_commit_status(
name=workflow_result.name,
status=GH.convert_to_gh_status(workflow_result.status),
name=_workflow.name,
status=GH.convert_to_gh_status(updated_status),
description="",
url=workflow_result.html_link,
url=env.get_report_url(settings=Settings, latest=True),
)

View File

@ -52,30 +52,58 @@ 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_
obj.command = obj.command.format(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 +112,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

@ -89,15 +89,27 @@
letter-spacing: -0.5px;
}
.dropdown-value {
width: 100px;
font-weight: normal;
font-family: inherit;
background-color: transparent;
color: inherit;
/*border: none;*/
/*outline: none;*/
/*cursor: pointer;*/
}
#result-container {
background-color: var(--tile-background);
margin-left: calc(var(--status-width) + 20px);
padding: 20px;
padding: 0;
box-sizing: border-box;
text-align: center;
font-size: 18px;
font-weight: normal;
flex-grow: 1;
margin-bottom: 40px;
}
#footer {
@ -189,10 +201,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 {
@ -282,6 +291,12 @@
}
}
function updateUrlParameter(paramName, paramValue) {
const url = new URL(window.location.href);
url.searchParams.set(paramName, paramValue);
window.location.href = url.toString();
}
// Attach the toggle function to the click event of the icon
document.getElementById('theme-toggle').addEventListener('click', toggleTheme);
@ -291,14 +306,14 @@
const monthNames = ["Jan", "Feb", "Mar", "Apr", "May", "Jun",
"Jul", "Aug", "Sep", "Oct", "Nov", "Dec"];
const month = monthNames[date.getMonth()];
const year = date.getFullYear();
//const year = date.getFullYear();
const hours = String(date.getHours()).padStart(2, '0');
const minutes = String(date.getMinutes()).padStart(2, '0');
const seconds = String(date.getSeconds()).padStart(2, '0');
//const milliseconds = String(date.getMilliseconds()).padStart(2, '0');
return showDate
? `${day}-${month}-${year} ${hours}:${minutes}:${seconds}`
? `${day}'${month} ${hours}:${minutes}:${seconds}`
: `${hours}:${minutes}:${seconds}`;
}
@ -328,7 +343,7 @@
const milliseconds = Math.floor((duration % 1) * 1000);
const formattedSeconds = String(seconds);
const formattedMilliseconds = String(milliseconds).padStart(3, '0');
const formattedMilliseconds = String(milliseconds).padStart(2, '0').slice(-2);
return `${formattedSeconds}.${formattedMilliseconds}`;
}
@ -346,8 +361,7 @@
return 'status-other';
}
function addKeyValueToStatus(key, value) {
function addKeyValueToStatus(key, value, options = null) {
const statusContainer = document.getElementById('status-container');
let keyValuePair = document.createElement('div');
@ -357,12 +371,40 @@
keyElement.className = 'json-key';
keyElement.textContent = key + ':';
const valueElement = document.createElement('div');
valueElement.className = 'json-value';
valueElement.textContent = value;
let valueElement;
keyValuePair.appendChild(keyElement)
keyValuePair.appendChild(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);
});
} 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);
statusContainer.appendChild(keyValuePair);
}
@ -486,12 +528,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) {
@ -500,16 +542,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);
@ -561,8 +601,7 @@
td.classList.add('time-column');
td.textContent = value ? formatDuration(value) : '';
} else if (column === 'info') {
// For info and other columns, just display the value
td.textContent = value || '';
td.textContent = value.includes('\n') ? '↵' : (value || '');
td.classList.add('info-column');
}
@ -573,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();
@ -630,19 +663,20 @@
let targetData = navigatePath(data, nameParams);
let nest_level = nameParams.length;
if (targetData) {
infoElement.style.display = 'none';
// Add footer links from top-level Result
if (Array.isArray(data.links) && data.links.length > 0) {
data.links.forEach(link => {
const a = document.createElement('a');
a.href = link;
a.textContent = link.split('/').pop();
a.target = '_blank';
footerRight.appendChild(a);
});
}
// Handle footer links if present
if (Array.isArray(data.aux_links) && data.aux_links.length > 0) {
data.aux_links.forEach(link => {
const a = document.createElement('a');
a.href = link;
a.textContent = link.split('/').pop();
a.target = '_blank';
footerRight.appendChild(a);
});
}
if (targetData) {
//infoElement.style.display = 'none';
infoElement.innerHTML = (targetData.info || '').replace(/\n/g, '<br>');
addStatusToStatus(targetData.status, targetData.start_time, targetData.duration)
@ -721,22 +755,62 @@
}
});
if (PR) {
addKeyValueToStatus("PR", PR)
} else {
console.error("TODO")
}
addKeyValueToStatus("sha", sha);
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) {
const shaToLoad = (sha === 'latest') ? commitsArray[commitsArray.length - 1] : sha;
loadResultsJSON(PR, shaToLoad, nameParams);
} else {
document.getElementById('title').textContent = 'Error: Missing required URL parameters: PR, sha, or name_0';
}
});
}
window.onload = init;

View File

@ -1,11 +1,10 @@
import copy
import importlib.util
from pathlib import Path
from typing import Any, Dict
from praktika import Job
from praktika._settings import _USER_DEFINED_SETTINGS, _Settings
from praktika.utils import ContextManager, Utils
from praktika.settings import Settings
from praktika.utils import Utils
def _get_workflows(name=None, file=None):
@ -14,35 +13,34 @@ def _get_workflows(name=None, file=None):
"""
res = []
with ContextManager.cd():
directory = Path(_Settings.WORKFLOWS_DIRECTORY)
for py_file in directory.glob("*.py"):
if file and file not in str(py_file):
continue
module_name = py_file.name.removeprefix(".py")
spec = importlib.util.spec_from_file_location(
module_name, f"{_Settings.WORKFLOWS_DIRECTORY}/{module_name}"
)
assert spec
foo = importlib.util.module_from_spec(spec)
assert spec.loader
spec.loader.exec_module(foo)
try:
for workflow in foo.WORKFLOWS:
if name:
if name == workflow.name:
print(f"Read workflow [{name}] config from [{module_name}]")
res = [workflow]
break
else:
continue
directory = Path(Settings.WORKFLOWS_DIRECTORY)
for py_file in directory.glob("*.py"):
if file and file not in str(py_file):
continue
module_name = py_file.name.removeprefix(".py")
spec = importlib.util.spec_from_file_location(
module_name, f"{Settings.WORKFLOWS_DIRECTORY}/{module_name}"
)
assert spec
foo = importlib.util.module_from_spec(spec)
assert spec.loader
spec.loader.exec_module(foo)
try:
for workflow in foo.WORKFLOWS:
if name:
if name == workflow.name:
print(f"Read workflow [{name}] config from [{module_name}]")
res = [workflow]
break
else:
res += foo.WORKFLOWS
print(f"Read workflow configs from [{module_name}]")
except Exception as e:
print(
f"WARNING: Failed to add WORKFLOWS config from [{module_name}], exception [{e}]"
)
continue
else:
res += foo.WORKFLOWS
print(f"Read workflow configs from [{module_name}]")
except Exception as e:
print(
f"WARNING: Failed to add WORKFLOWS config from [{module_name}], exception [{e}]"
)
if not res:
Utils.raise_with_error(f"Failed to find workflow [{name or file}]")
@ -58,7 +56,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]
@ -108,30 +105,3 @@ def _update_workflow_with_native_jobs(workflow):
for job in workflow.jobs:
aux_job.requires.append(job.name)
workflow.jobs.append(aux_job)
def _get_user_settings() -> Dict[str, Any]:
"""
Gets user's settings
"""
res = {} # type: Dict[str, Any]
directory = Path(_Settings.SETTINGS_DIRECTORY)
for py_file in directory.glob("*.py"):
module_name = py_file.name.removeprefix(".py")
spec = importlib.util.spec_from_file_location(
module_name, f"{_Settings.SETTINGS_DIRECTORY}/{module_name}"
)
assert spec
foo = importlib.util.module_from_spec(spec)
assert spec.loader
spec.loader.exec_module(foo)
for setting in _USER_DEFINED_SETTINGS:
try:
value = getattr(foo, setting)
res[setting] = value
print(f"Apply user defined setting [{setting} = {value}]")
except Exception as e:
pass
return res

View File

@ -10,9 +10,8 @@ from praktika.gh import GH
from praktika.hook_cache import CacheRunnerHooks
from praktika.hook_html import HtmlRunnerHooks
from praktika.mangle import _get_workflows
from praktika.result import Result, ResultInfo
from praktika.result import Result, ResultInfo, _ResultS3
from praktika.runtime import RunConfig
from praktika.s3 import S3
from praktika.settings import Settings
from praktika.utils import Shell, Utils
@ -151,7 +150,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}"
)
@ -225,6 +224,7 @@ def _config_workflow(workflow: Workflow.Config, job_name):
cache_success=[],
cache_success_base64=[],
cache_artifacts={},
cache_jobs={},
).dump()
# checks:
@ -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")
@ -307,9 +310,8 @@ def _finish_workflow(workflow, job_name):
print(env.get_needs_statuses())
print("Check Workflow results")
S3.copy_result_from_s3(
_ResultS3.copy_result_from_s3(
Result.file_name_static(workflow.name),
lock=False,
)
workflow_result = Result.from_fs(workflow.name)
@ -339,10 +341,12 @@ def _finish_workflow(workflow, job_name):
f"NOTE: Result for [{result.name}] has not ok status [{result.status}]"
)
ready_for_merge_status = Result.Status.FAILED
failed_results.append(result.name.split("(", maxsplit=1)[0]) # cut name
failed_results.append(result.name)
if failed_results:
ready_for_merge_description = f"failed: {', '.join(failed_results)}"
ready_for_merge_description = (
f'Failed {len(failed_results)} "Required for Merge" jobs'
)
if not GH.post_commit_status(
name=Settings.READY_FOR_MERGE_STATUS_NAME + f" [{workflow.name}]",
@ -354,14 +358,11 @@ def _finish_workflow(workflow, job_name):
env.add_info(ResultInfo.GH_STATUS_ERROR)
if update_final_report:
S3.copy_result_to_s3(
_ResultS3.copy_result_to_s3(
workflow_result,
unlock=False,
) # no lock - no unlock
)
Result.from_fs(job_name).set_status(Result.Status.SUCCESS).set_info(
ready_for_merge_description
)
Result.from_fs(job_name).set_status(Result.Status.SUCCESS)
if __name__ == "__main__":

View File

@ -1,12 +1,13 @@
import dataclasses
import datetime
import sys
from collections.abc import Container
from pathlib import Path
from typing import Any, Dict, List, Optional
from typing import Any, Dict, List, Optional, Union
from praktika._environment import _Environment
from praktika._settings import _Settings
from praktika.cache import Cache
from praktika.s3 import S3
from praktika.settings import Settings
from praktika.utils import ContextManager, MetaClasses, Shell, Utils
@ -27,10 +28,6 @@ class Result(MetaClasses.Serializable):
files (List[str]): A list of file paths or names related to the result.
links (List[str]): A list of URLs related to the result (e.g., links to reports or resources).
info (str): Additional information about the result. Free-form text.
# TODO: rename
aux_links (List[str]): A list of auxiliary links that provide additional context for the result.
# TODO: remove
html_link (str): A direct link to an HTML representation of the result (e.g., a detailed report page).
Inner Class:
Status: Defines possible statuses for the task, such as "success", "failure", etc.
@ -52,8 +49,6 @@ class Result(MetaClasses.Serializable):
files: List[str] = dataclasses.field(default_factory=list)
links: List[str] = dataclasses.field(default_factory=list)
info: str = ""
aux_links: List[str] = dataclasses.field(default_factory=list)
html_link: str = ""
@staticmethod
def create_from(
@ -62,14 +57,15 @@ class Result(MetaClasses.Serializable):
stopwatch: Utils.Stopwatch = None,
status="",
files=None,
info="",
info: Union[List[str], str] = "",
with_info_from_results=True,
):
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 +74,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 +108,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)
@ -155,7 +151,7 @@ class Result(MetaClasses.Serializable):
@classmethod
def file_name_static(cls, name):
return f"{_Settings.TEMP_DIR}/result_{Utils.normalize_string(name)}.json"
return f"{Settings.TEMP_DIR}/result_{Utils.normalize_string(name)}.json"
@classmethod
def from_dict(cls, obj: Dict[str, Any]) -> "Result":
@ -180,6 +176,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):
@ -233,7 +234,7 @@ class Result(MetaClasses.Serializable):
)
@classmethod
def generate_skipped(cls, name, results=None):
def generate_skipped(cls, name, cache_record: Cache.CacheRecord, results=None):
return Result(
name=name,
status=Result.Status.SKIPPED,
@ -242,7 +243,7 @@ class Result(MetaClasses.Serializable):
results=results or [],
files=[],
links=[],
info="from cache",
info=f"from cache: sha [{cache_record.sha}], pr/branch [{cache_record.pr_number or cache_record.branch}]",
)
@classmethod
@ -276,7 +277,7 @@ class Result(MetaClasses.Serializable):
# Set log file path if logging is enabled
log_file = (
f"{_Settings.TEMP_DIR}/{Utils.normalize_string(name)}.log"
f"{Settings.TEMP_DIR}/{Utils.normalize_string(name)}.log"
if with_log
else None
)
@ -318,18 +319,35 @@ class Result(MetaClasses.Serializable):
files=[log_file] if log_file else None,
)
def finish_job_accordingly(self):
def complete_job(self):
self.dump()
if not self.is_ok():
print("ERROR: Job Failed")
for result in self.results:
if not result.is_ok():
print("Failed checks:")
print(" | ", result)
print(self.to_stdout_formatted())
sys.exit(1)
else:
print("ok")
def to_stdout_formatted(self, indent="", res=""):
if self.is_ok():
return res
res += f"{indent}Task [{self.name}] failed.\n"
fail_info = ""
sub_indent = indent + " "
if not self.results:
if not self.is_ok():
fail_info += f"{sub_indent}{self.name}:\n"
for line in self.info.splitlines():
fail_info += f"{sub_indent}{sub_indent}{line}\n"
return res + fail_info
for sub_result in self.results:
res = sub_result.to_stdout_formatted(sub_indent, res)
return res
class ResultInfo:
SETUP_ENV_JOB_FAILED = (
@ -352,3 +370,202 @@ class ResultInfo:
)
S3_ERROR = "S3 call failure"
class _ResultS3:
@classmethod
def copy_result_to_s3(cls, result, unlock=False):
result.dump()
env = _Environment.get()
s3_path = f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix()}"
s3_path_full = f"{s3_path}/{Path(result.file_name()).name}"
url = S3.copy_file_to_s3(s3_path=s3_path, local_path=result.file_name())
# if unlock:
# if not cls.unlock(s3_path_full):
# print(f"ERROR: File [{s3_path_full}] unlock failure")
# assert False # TODO: investigate
return url
@classmethod
def copy_result_from_s3(cls, local_path, lock=False):
env = _Environment.get()
file_name = Path(local_path).name
s3_path = f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix()}/{file_name}"
# if lock:
# cls.lock(s3_path)
if not S3.copy_file_from_s3(s3_path=s3_path, local_path=local_path):
print(f"ERROR: failed to cp file [{s3_path}] from s3")
raise
@classmethod
def copy_result_from_s3_with_version(cls, local_path):
env = _Environment.get()
file_name = Path(local_path).name
local_dir = Path(local_path).parent
file_name_pattern = f"{file_name}_*"
for file_path in local_dir.glob(file_name_pattern):
file_path.unlink()
s3_path = f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix()}/"
if not S3.copy_file_from_s3_matching_pattern(
s3_path=s3_path, local_path=local_dir, include=file_name_pattern
):
print(f"ERROR: failed to cp file [{s3_path}] from s3")
raise
result_files = []
for file_path in local_dir.glob(file_name_pattern):
result_files.append(file_path)
assert result_files, "No result files found"
result_files.sort()
version = int(result_files[-1].name.split("_")[-1])
Shell.check(f"cp {result_files[-1]} {local_path}", strict=True, verbose=True)
return version
@classmethod
def copy_result_to_s3_with_version(cls, result, version):
result.dump()
filename = Path(result.file_name()).name
file_name_versioned = f"{filename}_{str(version).zfill(3)}"
env = _Environment.get()
s3_path_versioned = (
f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix()}/{file_name_versioned}"
)
s3_path = f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix()}/"
if version == 0:
S3.clean_s3_directory(s3_path=s3_path)
if not S3.put(
s3_path=s3_path_versioned,
local_path=result.file_name(),
if_none_matched=True,
):
print("Failed to put versioned Result")
return False
if not S3.put(s3_path=s3_path, local_path=result.file_name()):
print("Failed to put non-versioned Result")
return True
# @classmethod
# def lock(cls, s3_path, level=0):
# env = _Environment.get()
# s3_path_lock = s3_path + f".lock"
# file_path_lock = f"{Settings.TEMP_DIR}/{Path(s3_path_lock).name}"
# assert Shell.check(
# f"echo '''{env.JOB_NAME}''' > {file_path_lock}", verbose=True
# ), "Never"
#
# i = 20
# meta = S3.head_object(s3_path_lock)
# while meta:
# locked_by_job = meta.get("Metadata", {"job": ""}).get("job", "")
# if locked_by_job:
# decoded_bytes = base64.b64decode(locked_by_job)
# locked_by_job = decoded_bytes.decode("utf-8")
# print(
# f"WARNING: Failed to acquire lock, meta [{meta}], job [{locked_by_job}] - wait"
# )
# i -= 5
# if i < 0:
# info = f"ERROR: lock acquire failure - unlock forcefully"
# print(info)
# env.add_info(info)
# break
# time.sleep(5)
#
# metadata = {"job": Utils.to_base64(env.JOB_NAME)}
# S3.put(
# s3_path=s3_path_lock,
# local_path=file_path_lock,
# metadata=metadata,
# if_none_matched=True,
# )
# time.sleep(1)
# obj = S3.head_object(s3_path_lock)
# if not obj or not obj.has_tags(tags=metadata):
# print(f"WARNING: locked by another job [{obj}]")
# env.add_info("S3 lock file failure")
# cls.lock(s3_path, level=level + 1)
# print("INFO: lock acquired")
#
# @classmethod
# def unlock(cls, s3_path):
# s3_path_lock = s3_path + ".lock"
# env = _Environment.get()
# obj = S3.head_object(s3_path_lock)
# if not obj:
# print("ERROR: lock file is removed")
# assert False # investigate
# elif not obj.has_tags({"job": Utils.to_base64(env.JOB_NAME)}):
# print("ERROR: lock file was acquired by another job")
# assert False # investigate
#
# if not S3.delete(s3_path_lock):
# print(f"ERROR: File [{s3_path_lock}] delete failure")
# print("INFO: lock released")
# return True
@classmethod
def upload_result_files_to_s3(cls, result):
if result.results:
for result_ in result.results:
cls.upload_result_files_to_s3(result_)
for file in result.files:
if not Path(file).is_file():
print(f"ERROR: Invalid file [{file}] in [{result.name}] - skip upload")
result.info += f"\nWARNING: Result file [{file}] was not found"
file_link = S3._upload_file_to_s3(file, upload_to_s3=False)
else:
is_text = False
for text_file_suffix in Settings.TEXT_CONTENT_EXTENSIONS:
if file.endswith(text_file_suffix):
print(
f"File [{file}] matches Settings.TEXT_CONTENT_EXTENSIONS [{Settings.TEXT_CONTENT_EXTENSIONS}] - add text attribute for s3 object"
)
is_text = True
break
file_link = S3._upload_file_to_s3(
file,
upload_to_s3=True,
text=is_text,
s3_subprefix=Utils.normalize_string(result.name),
)
result.links.append(file_link)
if result.files:
print(
f"Result files [{result.files}] uploaded to s3 [{result.links[-len(result.files):]}] - clean files list"
)
result.files = []
result.dump()
@classmethod
def update_workflow_results(cls, workflow_name, new_info="", new_sub_results=None):
assert new_info or new_sub_results
attempt = 1
prev_status = ""
new_status = ""
done = False
while attempt < 10:
version = cls.copy_result_from_s3_with_version(
Result.file_name_static(workflow_name)
)
workflow_result = Result.from_fs(workflow_name)
prev_status = workflow_result.status
if new_info:
workflow_result.set_info(new_info)
if new_sub_results:
if isinstance(new_sub_results, Result):
new_sub_results = [new_sub_results]
for result_ in new_sub_results:
workflow_result.update_sub_result(result_)
new_status = workflow_result.status
if cls.copy_result_to_s3_with_version(workflow_result, version=version + 1):
done = True
break
print(f"Attempt [{attempt}] to upload workflow result failed")
attempt += 1
assert done
if prev_status != new_status:
return new_status
else:
return None

View File

@ -19,7 +19,7 @@ from praktika.utils import Shell, TeePopen, Utils
class Runner:
@staticmethod
def generate_dummy_environment(workflow, job):
def generate_local_run_environment(workflow, job, pr=None, branch=None, sha=None):
print("WARNING: Generate dummy env for local test")
Shell.check(
f"mkdir -p {Settings.TEMP_DIR} {Settings.INPUT_DIR} {Settings.OUTPUT_DIR}"
@ -28,9 +28,9 @@ class Runner:
WORKFLOW_NAME=workflow.name,
JOB_NAME=job.name,
REPOSITORY="",
BRANCH="",
SHA="",
PR_NUMBER=-1,
BRANCH=branch or Settings.MAIN_BRANCH if not pr else "",
SHA=sha or Shell.get_output("git rev-parse HEAD"),
PR_NUMBER=pr or -1,
EVENT_TYPE="",
JOB_OUTPUT_STREAM="",
EVENT_FILE_PATH="",
@ -52,6 +52,7 @@ class Runner:
cache_success=[],
cache_success_base64=[],
cache_artifacts={},
cache_jobs={},
)
for docker in workflow.dockers:
workflow_config.digest_dockers[docker.name] = Digest().calc_docker_digest(
@ -80,13 +81,12 @@ class Runner:
print("Read GH Environment")
env = _Environment.from_env()
env.JOB_NAME = job.name
env.PARAMETER = job.parameter
env.dump()
print(env)
return 0
def _pre_run(self, workflow, job):
def _pre_run(self, workflow, job, local_run=False):
env = _Environment.get()
result = Result(
@ -96,9 +96,10 @@ class Runner:
)
result.dump()
if workflow.enable_report and job.name != Settings.CI_CONFIG_JOB_NAME:
print("Update Job and Workflow Report")
HtmlRunnerHooks.pre_run(workflow, job)
if not local_run:
if workflow.enable_report and job.name != Settings.CI_CONFIG_JOB_NAME:
print("Update Job and Workflow Report")
HtmlRunnerHooks.pre_run(workflow, job)
print("Download required artifacts")
required_artifacts = []
@ -123,28 +124,48 @@ class Runner:
return 0
def _run(self, workflow, job, docker="", no_docker=False, param=None):
def _run(self, workflow, job, docker="", no_docker=False, param=None, test=""):
# re-set envs for local run
env = _Environment.get()
env.JOB_NAME = job.name
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:
# TODO: add support for any image, including not from ci config (e.g. ubuntu:latest)
docker_tag = RunConfig.from_fs(workflow.name).digest_dockers[
job.run_in_docker
]
docker = docker or f"{job.run_in_docker}:{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}"
job.run_in_docker, docker_settings = (
job.run_in_docker.split("+")[0],
job.run_in_docker.split("+")[1:],
)
from_root = "root" in docker_settings
settings = [s for s in docker_settings if s.startswith("--")]
if ":" in job.run_in_docker:
docker_name, docker_tag = job.run_in_docker.split(":")
print(
f"WARNING: Job [{job.name}] use custom docker image with a tag - praktika won't control docker version"
)
else:
docker_name, docker_tag = (
job.run_in_docker,
RunConfig.from_fs(workflow.name).digest_dockers[job.run_in_docker],
)
docker = docker or f"{docker_name}:{docker_tag}"
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} {' '.join(settings)} {docker} {job.command}"
else:
cmd = job.command
python_path = os.getenv("PYTHONPATH", ":")
os.environ["PYTHONPATH"] = f".:{python_path}"
if param:
print(f"Custom --param [{param}] will be passed to job's script")
cmd += f" --param {param}"
if test:
print(f"Custom --test [{test}] will be passed to job's script")
cmd += f" --test {test}"
print(f"--- Run command [{cmd}]")
with TeePopen(cmd, timeout=job.timeout) as process:
@ -219,13 +240,10 @@ 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:
# provide job info to workflow level
info_errors.append(result.info)
if run_exit_code == 0:
providing_artifacts = []
if job.provides and workflow.artifacts:
@ -285,14 +303,24 @@ class Runner:
return True
def run(
self, workflow, job, docker="", dummy_env=False, no_docker=False, param=None
self,
workflow,
job,
docker="",
local_run=False,
no_docker=False,
param=None,
test="",
pr=None,
sha=None,
branch=None,
):
res = True
setup_env_code = -10
prerun_code = -10
run_code = -10
if res and not dummy_env:
if res and not local_run:
print(
f"\n\n=== Setup env script [{job.name}], workflow [{workflow.name}] ==="
)
@ -309,13 +337,15 @@ class Runner:
traceback.print_exc()
print(f"=== Setup env finished ===\n\n")
else:
self.generate_dummy_environment(workflow, job)
self.generate_local_run_environment(
workflow, job, pr=pr, branch=branch, sha=sha
)
if res and not dummy_env:
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:
prerun_code = self._pre_run(workflow, job)
prerun_code = self._pre_run(workflow, job, local_run=local_run)
res = prerun_code == 0
if not res:
print(f"ERROR: Pre-run failed with exit code [{prerun_code}]")
@ -329,7 +359,12 @@ class Runner:
print(f"=== Run script [{job.name}], workflow [{workflow.name}] ===")
try:
run_code = self._run(
workflow, job, docker=docker, no_docker=no_docker, param=param
workflow,
job,
docker=docker,
no_docker=no_docker,
param=param,
test=test,
)
res = run_code == 0
if not res:
@ -339,7 +374,7 @@ class Runner:
traceback.print_exc()
print(f"=== Run scrip finished ===\n\n")
if not dummy_env:
if not local_run:
print(f"=== Post run script [{job.name}], workflow [{workflow.name}] ===")
self._post_run(workflow, job, setup_env_code, prerun_code, run_code)
print(f"=== Post run scrip finished ===")

View File

@ -15,17 +15,23 @@ class RunConfig(MetaClasses.Serializable):
# there are might be issue with special characters in job names if used directly in yaml syntax - create base64 encoded list to avoid this
cache_success_base64: List[str]
cache_artifacts: Dict[str, Cache.CacheRecord]
cache_jobs: Dict[str, Cache.CacheRecord]
sha: str
@classmethod
def from_dict(cls, obj):
cache_artifacts = obj["cache_artifacts"]
cache_jobs = obj["cache_jobs"]
cache_artifacts_deserialized = {}
cache_jobs_deserialized = {}
for artifact_name, cache_artifact in cache_artifacts.items():
cache_artifacts_deserialized[artifact_name] = Cache.CacheRecord.from_dict(
cache_artifact
)
obj["cache_artifacts"] = cache_artifacts_deserialized
for job_name, cache_jobs in cache_jobs.items():
cache_jobs_deserialized[job_name] = Cache.CacheRecord.from_dict(cache_jobs)
obj["cache_jobs"] = cache_artifacts_deserialized
return RunConfig(**obj)
@classmethod

View File

@ -1,12 +1,11 @@
import dataclasses
import json
import time
from pathlib import Path
from typing import Dict
from praktika._environment import _Environment
from praktika.settings import Settings
from praktika.utils import Shell, Utils
from praktika.utils import Shell
class S3:
@ -52,23 +51,22 @@ 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
return f"https://{s3_full_path}".replace(bucket, endpoint)
@classmethod
def put(cls, s3_path, local_path, text=False, metadata=None):
def put(cls, s3_path, local_path, text=False, metadata=None, if_none_matched=False):
assert Path(local_path).exists(), f"Path [{local_path}] does not exist"
assert Path(s3_path), f"Invalid S3 Path [{s3_path}]"
assert Path(
local_path
).is_file(), f"Path [{local_path}] is not file. Only files are supported"
file_name = Path(local_path).name
s3_full_path = s3_path
if not s3_full_path.endswith(file_name):
s3_full_path = f"{s3_path}/{Path(local_path).name}"
if s3_full_path.endswith("/"):
s3_full_path = f"{s3_path}{Path(local_path).name}"
s3_full_path = str(s3_full_path).removeprefix("s3://")
bucket, key = s3_full_path.split("/", maxsplit=1)
@ -76,6 +74,8 @@ class S3:
command = (
f"aws s3api put-object --bucket {bucket} --key {key} --body {local_path}"
)
if if_none_matched:
command += f' --if-none-match "*"'
if metadata:
for k, v in metadata.items():
command += f" --metadata {k}={v}"
@ -84,7 +84,7 @@ class S3:
if text:
cmd += " --content-type text/plain"
res = cls.run_command_with_retries(command)
assert res
return res
@classmethod
def run_command_with_retries(cls, command, retries=Settings.MAX_RETRIES_S3):
@ -101,6 +101,14 @@ class S3:
elif "does not exist" in stderr:
print("ERROR: requested file does not exist")
break
elif "Unknown options" in stderr:
print("ERROR: Invalid AWS CLI command or CLI client version:")
print(f" | awc error: {stderr}")
break
elif "PreconditionFailed" in stderr:
print("ERROR: AWS API Call Precondition Failed")
print(f" | awc error: {stderr}")
break
if ret_code != 0:
print(
f"ERROR: aws s3 cp failed, stdout/stderr err: [{stderr}], out [{stdout}]"
@ -108,13 +116,6 @@ class S3:
res = ret_code == 0
return res
@classmethod
def get_link(cls, s3_path, local_path):
s3_full_path = f"{s3_path}/{Path(local_path).name}"
bucket = s3_path.split("/")[0]
endpoint = Settings.S3_BUCKET_TO_HTTP_ENDPOINT[bucket]
return f"https://{s3_full_path}".replace(bucket, endpoint)
@classmethod
def copy_file_from_s3(cls, s3_path, local_path):
assert Path(s3_path), f"Invalid S3 Path [{s3_path}]"
@ -128,6 +129,19 @@ class S3:
res = cls.run_command_with_retries(cmd)
return res
@classmethod
def copy_file_from_s3_matching_pattern(
cls, s3_path, local_path, include, exclude="*"
):
assert Path(s3_path), f"Invalid S3 Path [{s3_path}]"
assert Path(
local_path
).is_dir(), f"Path [{local_path}] does not exist or not a directory"
assert s3_path.endswith("/"), f"s3 path is invalid [{s3_path}]"
cmd = f'aws s3 cp s3://{s3_path} {local_path} --exclude "{exclude}" --include "{include}" --recursive'
res = cls.run_command_with_retries(cmd)
return res
@classmethod
def head_object(cls, s3_path):
s3_path = str(s3_path).removeprefix("s3://")
@ -148,103 +162,6 @@ class S3:
verbose=True,
)
# TODO: apparently should be placed into separate file to be used only inside praktika
# keeping this module clean from importing Settings, Environment and etc, making it easy for use externally
@classmethod
def copy_result_to_s3(cls, result, unlock=True):
result.dump()
env = _Environment.get()
s3_path = f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix()}"
s3_path_full = f"{s3_path}/{Path(result.file_name()).name}"
url = S3.copy_file_to_s3(s3_path=s3_path, local_path=result.file_name())
if env.PR_NUMBER:
print("Duplicate Result for latest commit alias in PR")
s3_path = f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix(latest=True)}"
url = S3.copy_file_to_s3(s3_path=s3_path, local_path=result.file_name())
if unlock:
if not cls.unlock(s3_path_full):
print(f"ERROR: File [{s3_path_full}] unlock failure")
assert False # TODO: investigate
return url
@classmethod
def copy_result_from_s3(cls, local_path, lock=True):
env = _Environment.get()
file_name = Path(local_path).name
s3_path = f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix()}/{file_name}"
if lock:
cls.lock(s3_path)
if not S3.copy_file_from_s3(s3_path=s3_path, local_path=local_path):
print(f"ERROR: failed to cp file [{s3_path}] from s3")
raise
@classmethod
def lock(cls, s3_path, level=0):
assert level < 3, "Never"
env = _Environment.get()
s3_path_lock = s3_path + f".lock"
file_path_lock = f"{Settings.TEMP_DIR}/{Path(s3_path_lock).name}"
assert Shell.check(
f"echo '''{env.JOB_NAME}''' > {file_path_lock}", verbose=True
), "Never"
i = 20
meta = S3.head_object(s3_path_lock)
while meta:
print(f"WARNING: Failed to acquire lock, meta [{meta}] - wait")
i -= 5
if i < 0:
info = f"ERROR: lock acquire failure - unlock forcefully"
print(info)
env.add_info(info)
break
time.sleep(5)
metadata = {"job": Utils.to_base64(env.JOB_NAME)}
S3.put(
s3_path=s3_path_lock,
local_path=file_path_lock,
metadata=metadata,
)
time.sleep(1)
obj = S3.head_object(s3_path_lock)
if not obj or not obj.has_tags(tags=metadata):
print(f"WARNING: locked by another job [{obj}]")
env.add_info("S3 lock file failure")
cls.lock(s3_path, level=level + 1)
print("INFO: lock acquired")
@classmethod
def unlock(cls, s3_path):
s3_path_lock = s3_path + ".lock"
env = _Environment.get()
obj = S3.head_object(s3_path_lock)
if not obj:
print("ERROR: lock file is removed")
assert False # investigate
elif not obj.has_tags({"job": Utils.to_base64(env.JOB_NAME)}):
print("ERROR: lock file was acquired by another job")
assert False # investigate
if not S3.delete(s3_path_lock):
print(f"ERROR: File [{s3_path_lock}] delete failure")
print("INFO: lock released")
return True
@classmethod
def get_result_link(cls, result):
env = _Environment.get()
s3_path = f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix(latest=True if env.PR_NUMBER else False)}"
return S3.get_link(s3_path=s3_path, local_path=result.file_name())
@classmethod
def clean_latest_result(cls):
env = _Environment.get()
env.SHA = "latest"
assert env.PR_NUMBER
s3_path = f"{Settings.HTML_S3_PATH}/{env.get_s3_prefix()}"
S3.clean_s3_directory(s3_path=s3_path)
@classmethod
def _upload_file_to_s3(
cls, local_file_path, upload_to_s3: bool, text: bool = False, s3_subprefix=""
@ -260,36 +177,3 @@ class S3:
)
return html_link
return f"file://{Path(local_file_path).absolute()}"
@classmethod
def upload_result_files_to_s3(cls, result):
if result.results:
for result_ in result.results:
cls.upload_result_files_to_s3(result_)
for file in result.files:
if not Path(file).is_file():
print(f"ERROR: Invalid file [{file}] in [{result.name}] - skip upload")
result.info += f"\nWARNING: Result file [{file}] was not found"
file_link = cls._upload_file_to_s3(file, upload_to_s3=False)
else:
is_text = False
for text_file_suffix in Settings.TEXT_CONTENT_EXTENSIONS:
if file.endswith(text_file_suffix):
print(
f"File [{file}] matches Settings.TEXT_CONTENT_EXTENSIONS [{Settings.TEXT_CONTENT_EXTENSIONS}] - add text attribute for s3 object"
)
is_text = True
break
file_link = cls._upload_file_to_s3(
file,
upload_to_s3=True,
text=is_text,
s3_subprefix=Utils.normalize_string(result.name),
)
result.links.append(file_link)
if result.files:
print(
f"Result files [{result.files}] uploaded to s3 [{result.links[-len(result.files):]}] - clean files list"
)
result.files = []
result.dump()

View File

@ -1,8 +1,152 @@
from praktika._settings import _Settings
from praktika.mangle import _get_user_settings
import dataclasses
import importlib.util
from pathlib import Path
from typing import Dict, Iterable, List, Optional
Settings = _Settings()
user_settings = _get_user_settings()
for setting, value in user_settings.items():
Settings.__setattr__(setting, value)
@dataclasses.dataclass
class _Settings:
######################################
# Pipeline generation settings #
######################################
MAIN_BRANCH = "main"
CI_PATH = "./ci"
WORKFLOW_PATH_PREFIX: str = "./.github/workflows"
WORKFLOWS_DIRECTORY: str = f"{CI_PATH}/workflows"
SETTINGS_DIRECTORY: str = f"{CI_PATH}/settings"
CI_CONFIG_JOB_NAME = "Config Workflow"
DOCKER_BUILD_JOB_NAME = "Docker Builds"
FINISH_WORKFLOW_JOB_NAME = "Finish Workflow"
READY_FOR_MERGE_STATUS_NAME = "Ready for Merge"
CI_CONFIG_RUNS_ON: Optional[List[str]] = None
DOCKER_BUILD_RUNS_ON: Optional[List[str]] = None
VALIDATE_FILE_PATHS: bool = True
######################################
# Runtime Settings #
######################################
MAX_RETRIES_S3 = 3
MAX_RETRIES_GH = 3
######################################
# S3 (artifact storage) settings #
######################################
S3_ARTIFACT_PATH: str = ""
######################################
# CI workspace settings #
######################################
TEMP_DIR: str = "/tmp/praktika"
OUTPUT_DIR: str = f"{TEMP_DIR}/output"
INPUT_DIR: str = f"{TEMP_DIR}/input"
PYTHON_INTERPRETER: str = "python3"
PYTHON_PACKET_MANAGER: str = "pip3"
PYTHON_VERSION: str = "3.9"
INSTALL_PYTHON_FOR_NATIVE_JOBS: bool = False
INSTALL_PYTHON_REQS_FOR_NATIVE_JOBS: str = "./ci/requirements.txt"
ENVIRONMENT_VAR_FILE: str = f"{TEMP_DIR}/environment.json"
RUN_LOG: str = f"{TEMP_DIR}/praktika_run.log"
SECRET_GH_APP_ID: str = "GH_APP_ID"
SECRET_GH_APP_PEM_KEY: str = "GH_APP_PEM_KEY"
ENV_SETUP_SCRIPT: str = "/tmp/praktika_setup_env.sh"
WORKFLOW_STATUS_FILE: str = f"{TEMP_DIR}/workflow_status.json"
######################################
# CI Cache settings #
######################################
CACHE_VERSION: int = 1
CACHE_DIGEST_LEN: int = 20
CACHE_S3_PATH: str = ""
CACHE_LOCAL_PATH: str = f"{TEMP_DIR}/ci_cache"
######################################
# Report settings #
######################################
HTML_S3_PATH: str = ""
HTML_PAGE_FILE: str = "./praktika/json.html"
TEXT_CONTENT_EXTENSIONS: Iterable[str] = frozenset([".txt", ".log"])
S3_BUCKET_TO_HTTP_ENDPOINT: Optional[Dict[str, str]] = None
DOCKERHUB_USERNAME: str = ""
DOCKERHUB_SECRET: str = ""
DOCKER_WD: str = "/wd"
######################################
# CI DB Settings #
######################################
SECRET_CI_DB_URL: str = "CI_DB_URL"
SECRET_CI_DB_PASSWORD: str = "CI_DB_PASSWORD"
CI_DB_DB_NAME = ""
CI_DB_TABLE_NAME = ""
CI_DB_INSERT_TIMEOUT_SEC = 5
DISABLE_MERGE_COMMIT = True
_USER_DEFINED_SETTINGS = [
"S3_ARTIFACT_PATH",
"CACHE_S3_PATH",
"HTML_S3_PATH",
"S3_BUCKET_TO_HTTP_ENDPOINT",
"TEXT_CONTENT_EXTENSIONS",
"TEMP_DIR",
"OUTPUT_DIR",
"INPUT_DIR",
"CI_CONFIG_RUNS_ON",
"DOCKER_BUILD_RUNS_ON",
"CI_CONFIG_JOB_NAME",
"PYTHON_INTERPRETER",
"PYTHON_VERSION",
"PYTHON_PACKET_MANAGER",
"INSTALL_PYTHON_FOR_NATIVE_JOBS",
"INSTALL_PYTHON_REQS_FOR_NATIVE_JOBS",
"MAX_RETRIES_S3",
"MAX_RETRIES_GH",
"VALIDATE_FILE_PATHS",
"DOCKERHUB_USERNAME",
"DOCKERHUB_SECRET",
"READY_FOR_MERGE_STATUS_NAME",
"SECRET_CI_DB_URL",
"SECRET_CI_DB_PASSWORD",
"CI_DB_DB_NAME",
"CI_DB_TABLE_NAME",
"CI_DB_INSERT_TIMEOUT_SEC",
"SECRET_GH_APP_PEM_KEY",
"SECRET_GH_APP_ID",
"MAIN_BRANCH",
"DISABLE_MERGE_COMMIT",
]
def _get_settings() -> _Settings:
res = _Settings()
directory = Path(_Settings.SETTINGS_DIRECTORY)
for py_file in directory.glob("*.py"):
module_name = py_file.name.removeprefix(".py")
spec = importlib.util.spec_from_file_location(
module_name, f"{_Settings.SETTINGS_DIRECTORY}/{module_name}"
)
assert spec
foo = importlib.util.module_from_spec(spec)
assert spec.loader
spec.loader.exec_module(foo)
for setting in _USER_DEFINED_SETTINGS:
try:
value = getattr(foo, setting)
res.__setattr__(setting, value)
# print(f"- read user defined setting [{setting} = {value}]")
except Exception as e:
# print(f"Exception while read user settings: {e}")
pass
return res
class GHRunners:
ubuntu = "ubuntu-latest"
Settings = _get_settings()

View File

@ -17,8 +17,6 @@ from threading import Thread
from types import SimpleNamespace
from typing import Any, Dict, Iterator, List, Optional, Type, TypeVar, Union
from praktika._settings import _Settings
T = TypeVar("T", bound="Serializable")
@ -81,25 +79,26 @@ class MetaClasses:
class ContextManager:
@staticmethod
@contextmanager
def cd(to: Optional[Union[Path, str]] = None) -> Iterator[None]:
def cd(to: Optional[Union[Path, str]]) -> Iterator[None]:
"""
changes current working directory to @path or `git root` if @path is None
:param to:
:return:
"""
if not to:
try:
to = Shell.get_output_or_raise("git rev-parse --show-toplevel")
except:
pass
if not to:
if Path(_Settings.DOCKER_WD).is_dir():
to = _Settings.DOCKER_WD
if not to:
assert False, "FIX IT"
assert to
# if not to:
# try:
# to = Shell.get_output_or_raise("git rev-parse --show-toplevel")
# except:
# pass
# if not to:
# if Path(_Settings.DOCKER_WD).is_dir():
# to = _Settings.DOCKER_WD
# if not to:
# assert False, "FIX IT"
# assert to
old_pwd = os.getcwd()
os.chdir(to)
if to:
os.chdir(to)
try:
yield
finally:

View File

@ -4,10 +4,8 @@ from itertools import chain
from pathlib import Path
from praktika import Workflow
from praktika._settings import GHRunners
from praktika.mangle import _get_workflows
from praktika.settings import Settings
from praktika.utils import ContextManager
from praktika.settings import GHRunners, Settings
class Validator:
@ -119,61 +117,56 @@ class Validator:
def validate_file_paths_in_run_command(cls, workflow: Workflow.Config) -> None:
if not Settings.VALIDATE_FILE_PATHS:
return
with ContextManager.cd():
for job in workflow.jobs:
run_command = job.command
command_parts = run_command.split(" ")
for part in command_parts:
if ">" in part:
return
if "/" in part:
assert (
Path(part).is_file() or Path(part).is_dir()
), f"Apparently run command [{run_command}] for job [{job}] has invalid path [{part}]. Setting to disable check: VALIDATE_FILE_PATHS"
for job in workflow.jobs:
run_command = job.command
command_parts = run_command.split(" ")
for part in command_parts:
if ">" in part:
return
if "/" in part:
assert (
Path(part).is_file() or Path(part).is_dir()
), f"Apparently run command [{run_command}] for job [{job}] has invalid path [{part}]. Setting to disable check: VALIDATE_FILE_PATHS"
@classmethod
def validate_file_paths_in_digest_configs(cls, workflow: Workflow.Config) -> None:
if not Settings.VALIDATE_FILE_PATHS:
return
with ContextManager.cd():
for job in workflow.jobs:
if not job.digest_config:
continue
for include_path in chain(
job.digest_config.include_paths, job.digest_config.exclude_paths
):
if "*" in include_path:
assert glob.glob(
include_path, recursive=True
), f"Apparently file glob [{include_path}] in job [{job.name}] digest_config [{job.digest_config}] invalid, workflow [{workflow.name}]. Setting to disable check: VALIDATE_FILE_PATHS"
else:
assert (
Path(include_path).is_file() or Path(include_path).is_dir()
), f"Apparently file path [{include_path}] in job [{job.name}] digest_config [{job.digest_config}] invalid, workflow [{workflow.name}]. Setting to disable check: VALIDATE_FILE_PATHS"
for job in workflow.jobs:
if not job.digest_config:
continue
for include_path in chain(
job.digest_config.include_paths, job.digest_config.exclude_paths
):
if "*" in include_path:
assert glob.glob(
include_path, recursive=True
), f"Apparently file glob [{include_path}] in job [{job.name}] digest_config [{job.digest_config}] invalid, workflow [{workflow.name}]. Setting to disable check: VALIDATE_FILE_PATHS"
else:
assert (
Path(include_path).is_file() or Path(include_path).is_dir()
), f"Apparently file path [{include_path}] in job [{job.name}] digest_config [{job.digest_config}] invalid, workflow [{workflow.name}]. Setting to disable check: VALIDATE_FILE_PATHS"
@classmethod
def validate_requirements_txt_files(cls, workflow: Workflow.Config) -> None:
with ContextManager.cd():
for job in workflow.jobs:
if job.job_requirements:
if job.job_requirements.python_requirements_txt:
path = Path(job.job_requirements.python_requirements_txt)
message = f"File with py requirement [{path}] does not exist"
if job.name in (
Settings.DOCKER_BUILD_JOB_NAME,
Settings.CI_CONFIG_JOB_NAME,
Settings.FINISH_WORKFLOW_JOB_NAME,
):
message += '\n If all requirements already installed on your runners - add setting INSTALL_PYTHON_REQS_FOR_NATIVE_JOBS""'
message += "\n If requirements needs to be installed - add requirements file (Settings.INSTALL_PYTHON_REQS_FOR_NATIVE_JOBS):"
message += "\n echo jwt==1.3.1 > ./ci/requirements.txt"
message += (
"\n echo requests==2.32.3 >> ./ci/requirements.txt"
)
message += "\n echo https://clickhouse-builds.s3.amazonaws.com/packages/praktika-0.1-py3-none-any.whl >> ./ci/requirements.txt"
cls.evaluate_check(
path.is_file(), message, job.name, workflow.name
for job in workflow.jobs:
if job.job_requirements:
if job.job_requirements.python_requirements_txt:
path = Path(job.job_requirements.python_requirements_txt)
message = f"File with py requirement [{path}] does not exist"
if job.name in (
Settings.DOCKER_BUILD_JOB_NAME,
Settings.CI_CONFIG_JOB_NAME,
Settings.FINISH_WORKFLOW_JOB_NAME,
):
message += '\n If all requirements already installed on your runners - add setting INSTALL_PYTHON_REQS_FOR_NATIVE_JOBS""'
message += "\n If requirements needs to be installed - add requirements file (Settings.INSTALL_PYTHON_REQS_FOR_NATIVE_JOBS):"
message += "\n echo jwt==1.3.1 > ./ci/requirements.txt"
message += (
"\n echo requests==2.32.3 >> ./ci/requirements.txt"
)
message += "\n echo https://clickhouse-builds.s3.amazonaws.com/packages/praktika-0.1-py3-none-any.whl >> ./ci/requirements.txt"
cls.evaluate_check(path.is_file(), message, job.name, workflow.name)
@classmethod
def validate_dockers(cls, workflow: Workflow.Config):

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,8 @@ jobs:
steps:
- name: Checkout code
uses: actions/checkout@v4
with:
ref: ${{{{ github.head_ref }}}}
{JOB_ADDONS}
- name: Prepare env script
run: |
@ -102,7 +104,11 @@ jobs:
run: |
. /tmp/praktika_setup_env.sh
set -o pipefail
{PYTHON} -m praktika run --job '''{JOB_NAME}''' --workflow "{WORKFLOW_NAME}" --ci |& tee {RUN_LOG}
if command -v ts &> /dev/null; then
python3 -m praktika run --job '''{JOB_NAME}''' --workflow "{WORKFLOW_NAME}" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log
else
python3 -m praktika run --job '''{JOB_NAME}''' --workflow "{WORKFLOW_NAME}" --ci |& tee /tmp/praktika/praktika_run.log
fi
{UPLOADS_GITHUB}\
"""
@ -184,12 +190,10 @@ jobs:
False
), f"Workflow event not yet supported [{workflow_config.event}]"
with ContextManager.cd():
with open(self._get_workflow_file_name(workflow_config.name), "w") as f:
f.write(yaml_workflow_str)
with open(self._get_workflow_file_name(workflow_config.name), "w") as f:
f.write(yaml_workflow_str)
with ContextManager.cd():
Shell.check("git add ./.github/workflows/*.yaml")
Shell.check("git add ./.github/workflows/*.yaml")
class PullRequestPushYamlGen:

View File

@ -7,24 +7,33 @@ S3_BUCKET_HTTP_ENDPOINT = "clickhouse-builds.s3.amazonaws.com"
class RunnerLabels:
CI_SERVICES = "ci_services"
CI_SERVICES_EBS = "ci_services_ebs"
BUILDER = "builder"
BUILDER_AMD = "builder"
BUILDER_ARM = "builder-aarch64"
FUNC_TESTER_AMD = "func-tester"
FUNC_TESTER_ARM = "func-tester-aarch64"
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,18 +127,18 @@ 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/stateful-test",
# path="./ci/docker/test/stateful",
# platforms=Docker.Platforms.arm_amd,
# depends_on=["clickhouse/stateless-test"],
# ),
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/stateful-test",
platforms=Docker.Platforms.arm_amd,
depends_on=["clickhouse/stateless-test"],
),
# Docker.Config(
# name="clickhouse/stress-test",
# path="./ci/docker/test/stress",
@ -230,4 +239,6 @@ DOCKERS = [
class JobNames:
STYLE_CHECK = "Style Check"
FAST_TEST = "Fast test"
BUILD_AMD_DEBUG = "Build amd64 debug"
BUILD = "Build"
STATELESS = "Stateless tests"
STATEFUL = "Stateful tests"

View File

@ -4,6 +4,8 @@ from ci.settings.definitions import (
RunnerLabels,
)
MAIN_BRANCH = "master"
S3_ARTIFACT_PATH = f"{S3_BUCKET_NAME}/artifacts"
CI_CONFIG_RUNS_ON = [RunnerLabels.CI_SERVICES]
DOCKER_BUILD_RUNS_ON = [RunnerLabels.CI_SERVICES_EBS]

View File

@ -1,5 +1,3 @@
from typing import List
from praktika import Artifact, Job, Workflow
from praktika.settings import Settings
@ -13,7 +11,10 @@ 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"
CH_ARM_RELEASE = "CH_ARM_RELEASE"
CH_ARM_ASAN = "CH_ARM_ASAN"
style_check_job = Job.Config(
@ -25,7 +26,7 @@ style_check_job = Job.Config(
fast_test_job = Job.Config(
name=JobNames.FAST_TEST,
runs_on=[RunnerLabels.BUILDER],
runs_on=[RunnerLabels.BUILDER_AMD],
command="python3 ./ci/jobs/fast_test.py",
run_in_docker="clickhouse/fasttest",
digest_config=Job.CacheDigestConfig(
@ -37,11 +38,13 @@ fast_test_job = Job.Config(
),
)
job_build_amd_debug = Job.Config(
name=JobNames.BUILD_AMD_DEBUG,
runs_on=[RunnerLabels.BUILDER],
command="python3 ./ci/jobs/build_clickhouse.py amd_debug",
build_jobs = Job.Config(
name=JobNames.BUILD,
runs_on=["...from params..."],
requires=[JobNames.FAST_TEST],
command="python3 ./ci/jobs/build_clickhouse.py --build-type {PARAMETER}",
run_in_docker="clickhouse/fasttest",
timeout=3600 * 2,
digest_config=Job.CacheDigestConfig(
include_paths=[
"./src",
@ -54,9 +57,85 @@ job_build_amd_debug = Job.Config(
"./docker/packager/packager",
"./rust",
"./tests/ci/version_helper.py",
"./ci/jobs/build_clickhouse.py",
],
),
provides=[ArtifactNames.ch_debug_binary],
).parametrize(
parameter=["amd_debug", "amd_release", "arm_release", "arm_asan"],
provides=[
[ArtifactNames.CH_AMD_DEBUG],
[ArtifactNames.CH_AMD_RELEASE],
[ArtifactNames.CH_ARM_RELEASE],
[ArtifactNames.CH_ARM_ASAN],
],
runs_on=[
[RunnerLabels.BUILDER_AMD],
[RunnerLabels.BUILDER_AMD],
[RunnerLabels.BUILDER_ARM],
[RunnerLabels.BUILDER_ARM],
],
)
stateless_tests_jobs = Job.Config(
name=JobNames.STATELESS,
runs_on=[RunnerLabels.BUILDER_AMD],
command="python3 ./ci/jobs/functional_stateless_tests.py --test-options {PARAMETER}",
# many tests expect to see "/var/lib/clickhouse" in various output lines - add mount for now, consider creating this dir in docker file
run_in_docker="clickhouse/stateless-test+--security-opt seccomp=unconfined",
digest_config=Job.CacheDigestConfig(
include_paths=[
"./ci/jobs/functional_stateless_tests.py",
],
),
).parametrize(
parameter=[
"amd_debug,parallel",
"amd_debug,non-parallel",
"amd_release,parallel",
"amd_release,non-parallel",
"arm_asan,parallel",
"arm_asan,non-parallel",
],
runs_on=[
[RunnerLabels.BUILDER_AMD],
[RunnerLabels.FUNC_TESTER_AMD],
[RunnerLabels.BUILDER_AMD],
[RunnerLabels.FUNC_TESTER_AMD],
[RunnerLabels.BUILDER_ARM],
[RunnerLabels.FUNC_TESTER_ARM],
],
requires=[
[ArtifactNames.CH_AMD_DEBUG],
[ArtifactNames.CH_AMD_DEBUG],
[ArtifactNames.CH_AMD_RELEASE],
[ArtifactNames.CH_AMD_RELEASE],
[ArtifactNames.CH_ARM_ASAN],
[ArtifactNames.CH_ARM_ASAN],
],
)
stateful_tests_jobs = Job.Config(
name=JobNames.STATEFUL,
runs_on=[RunnerLabels.BUILDER_AMD],
command="python3 ./ci/jobs/functional_stateful_tests.py --test-options {PARAMETER}",
# many tests expect to see "/var/lib/clickhouse"
# some tests expect to see "/var/log/clickhouse"
run_in_docker="clickhouse/stateless-test+--security-opt seccomp=unconfined",
digest_config=Job.CacheDigestConfig(
include_paths=[
"./ci/jobs/functional_stateful_tests.py",
],
),
).parametrize(
parameter=[
"amd_debug,parallel",
],
runs_on=[
[RunnerLabels.BUILDER_AMD],
],
requires=[
[ArtifactNames.CH_AMD_DEBUG],
],
)
workflow = Workflow.Config(
@ -66,14 +145,31 @@ workflow = Workflow.Config(
jobs=[
style_check_job,
fast_test_job,
job_build_amd_debug,
*build_jobs,
*stateless_tests_jobs,
*stateful_tests_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",
),
Artifact.Config(
name=ArtifactNames.CH_ARM_RELEASE,
type=Artifact.Type.S3,
path=f"{Settings.TEMP_DIR}/build/programs/clickhouse",
),
Artifact.Config(
name=ArtifactNames.CH_ARM_ASAN,
type=Artifact.Type.S3,
path=f"{Settings.TEMP_DIR}/build/programs/clickhouse",
),
],
dockers=DOCKERS,
secrets=SECRETS,
@ -84,11 +180,14 @@ workflow = Workflow.Config(
WORKFLOWS = [
workflow,
] # 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", dummy_env=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

@ -2154,9 +2154,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):
@ -2620,14 +2620,14 @@ def run_tests_process(*args, **kwargs):
def do_run_tests(jobs, test_suite: TestSuite):
if jobs > 1 and len(test_suite.parallel_tests) > 0:
print(
"Found",
len(test_suite.parallel_tests),
"parallel tests and",
len(test_suite.sequential_tests),
"sequential tests",
)
print(
"Found",
len(test_suite.parallel_tests),
"parallel tests and",
len(test_suite.sequential_tests),
"sequential tests",
)
if test_suite.parallel_tests:
tests_n = len(test_suite.parallel_tests)
jobs = min(jobs, tests_n)
@ -2640,6 +2640,7 @@ def do_run_tests(jobs, test_suite: TestSuite):
# It makes it more difficult to detect real flaky tests,
# because the distribution and the amount
# of failures will be nearly the same for all tests from the group.
# TODO: add shuffle for sequential tests
random.shuffle(test_suite.parallel_tests)
batch_size = len(test_suite.parallel_tests) // jobs
@ -2685,6 +2686,7 @@ def do_run_tests(jobs, test_suite: TestSuite):
if not p.is_alive():
processes.remove(p)
if test_suite.sequential_tests:
run_tests_array(
(
test_suite.sequential_tests,
@ -2694,17 +2696,7 @@ def do_run_tests(jobs, test_suite: TestSuite):
)
)
return len(test_suite.sequential_tests) + len(test_suite.parallel_tests)
num_tests = len(test_suite.all_tests)
run_tests_array(
(
test_suite.all_tests,
num_tests,
test_suite,
False,
)
)
return num_tests
return len(test_suite.sequential_tests) + len(test_suite.parallel_tests)
def is_test_from_dir(suite_dir, case):
@ -3291,7 +3283,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"
@ -3340,7 +3335,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

@ -6,6 +6,7 @@
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name> <!-- For tests only-->
</invalidCertificateHandler>

View File

@ -9,6 +9,20 @@ 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 )"
FAST_TEST=0
NO_AZURE=0
while [[ "$#" -gt 0 ]]; do
case $1 in
--fast-test) FAST_TEST=1 ;;
--s3-storage) EXPORT_S3_STORAGE_POLICIES=1 ;;
--no-azure) NO_AZURE=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 +86,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,8 +198,10 @@ 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
ln -sf $SRC_PATH/config.d/azure_storage_conf.xml $DEST_SERVER_PATH/config.d/
if [[ "$EXPORT_S3_STORAGE_POLICIES" == "1" ]]; then
if [[ "$NO_AZURE" != "1" ]]; then
ln -sf $SRC_PATH/config.d/azure_storage_conf.xml $DEST_SERVER_PATH/config.d/
fi
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_02963.xml $DEST_SERVER_PATH/config.d/
@ -195,7 +210,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