diff --git a/.gitmodules b/.gitmodules index a3b6450032a..a346c23631f 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,6 +1,9 @@ # Please do not use 'branch = ...' tags with submodule entries. Such tags make updating submodules a # little bit more convenient but they do *not* specify the tracked submodule branch. Thus, they are # more confusing than useful. +[submodule "contrib/jwt-cpp"] + path = contrib/jwt-cpp + url = https://github.com/Thalhammer/jwt-cpp [submodule "contrib/zstd"] path = contrib/zstd url = https://github.com/facebook/zstd @@ -348,6 +351,9 @@ [submodule "contrib/idna"] path = contrib/idna url = https://github.com/ada-url/idna.git +[submodule "contrib/google-cloud-cpp"] + path = contrib/google-cloud-cpp + url = https://github.com/ClickHouse/google-cloud-cpp.git [submodule "contrib/rust_vendor"] path = contrib/rust_vendor url = https://github.com/ClickHouse/rust_vendor.git diff --git a/ci/__init__.py b/ci/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/ci/docker/stateful-test/Dockerfile b/ci/docker/stateful-test/Dockerfile new file mode 100644 index 00000000000..e21aec4a48f --- /dev/null +++ b/ci/docker/stateful-test/Dockerfile @@ -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 diff --git a/ci/docker/stateless-test/Dockerfile b/ci/docker/stateless-test/Dockerfile new file mode 100644 index 00000000000..dcfaa5f6267 --- /dev/null +++ b/ci/docker/stateless-test/Dockerfile @@ -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 diff --git a/ci/docker/stateless-test/requirements.txt b/ci/docker/stateless-test/requirements.txt new file mode 100644 index 00000000000..6f64cc08951 --- /dev/null +++ b/ci/docker/stateless-test/requirements.txt @@ -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 diff --git a/ci/jobs/__init__.py b/ci/jobs/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/ci/jobs/build_clickhouse.py b/ci/jobs/build_clickhouse.py index 21ed8091608..ed9fd491fcf 100644 --- a/ci/jobs/build_clickhouse.py +++ b/ci/jobs/build_clickhouse.py @@ -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: ") - parser.add_argument("--param", help="Optional custom job start stage", default=None) + parser.add_argument( + "--build-type", + help="Type: ,,", + ) + 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__": diff --git a/ci/jobs/check_style.py b/ci/jobs/check_style.py index f9cdc76302d..d4b81abc92c 100644 --- a/ci/jobs/check_style.py +++ b/ci/jobs/check_style.py @@ -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() diff --git a/ci/jobs/fast_test.py b/ci/jobs/fast_test.py index 1dcd65b6ed2..03a4c0cd496 100644 --- a/ci/jobs/fast_test.py +++ b/ci/jobs/fast_test.py @@ -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__": diff --git a/ci/jobs/functional_stateful_tests.py b/ci/jobs/functional_stateful_tests.py new file mode 100644 index 00000000000..b5840fcd45d --- /dev/null +++ b/ci/jobs/functional_stateful_tests.py @@ -0,0 +1,171 @@ +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/", + # 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|local_disk|{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() diff --git a/ci/jobs/functional_stateless_tests.py b/ci/jobs/functional_stateless_tests.py new file mode 100644 index 00000000000..0d73312bd9e --- /dev/null +++ b/ci/jobs/functional_stateless_tests.py @@ -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|local_disk|{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() diff --git a/ci/jobs/scripts/__init__.py b/ci/jobs/scripts/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/ci/jobs/scripts/clickhouse_proc.py b/ci/jobs/scripts/clickhouse_proc.py new file mode 100644 index 00000000000..6108563605f --- /dev/null +++ b/ci/jobs/scripts/clickhouse_proc.py @@ -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 = """ + + + local + {CH_RUNTIME_DIR}/var/lib/clickhouse/disks/backups/ + + +""" + + 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) diff --git a/ci/jobs/scripts/functional_tests/setup_hdfs_minicluster.sh b/ci/jobs/scripts/functional_tests/setup_hdfs_minicluster.sh new file mode 100755 index 00000000000..b810b27fe2b --- /dev/null +++ b/ci/jobs/scripts/functional_tests/setup_hdfs_minicluster.sh @@ -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 diff --git a/ci/jobs/scripts/functional_tests/setup_minio.sh b/ci/jobs/scripts/functional_tests/setup_minio.sh new file mode 100755 index 00000000000..88839c39674 --- /dev/null +++ b/ci/jobs/scripts/functional_tests/setup_minio.sh @@ -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 (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, (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 <> ~/.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 "$@" diff --git a/ci/jobs/scripts/functional_tests_results.py b/ci/jobs/scripts/functional_tests_results.py index 5ac9d6b985d..06989fb0a44 100755 --- a/ci/jobs/scripts/functional_tests_results.py +++ b/ci/jobs/scripts/functional_tests_results.py @@ -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, ) diff --git a/ci/praktika/__main__.py b/ci/praktika/__main__.py index 7f472ecd9ae..3dfdc26d69d 100644 --- a/ci/praktika/__main__.py +++ b/ci/praktika/__main__.py @@ -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() diff --git a/ci/praktika/_environment.py b/ci/praktika/_environment.py index ce9c6f5b486..734a4be3176 100644 --- a/ci/praktika/_environment.py +++ b/ci/praktika/_environment.py @@ -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): diff --git a/ci/praktika/_settings.py b/ci/praktika/_settings.py deleted file mode 100644 index 3052d8ef877..00000000000 --- a/ci/praktika/_settings.py +++ /dev/null @@ -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())) diff --git a/ci/praktika/cidb.py b/ci/praktika/cidb.py index 087845ec762..53088c102cd 100644 --- a/ci/praktika/cidb.py +++ b/ci/praktika/cidb.py @@ -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, diff --git a/ci/praktika/digest.py b/ci/praktika/digest.py index 93b62b13dc0..6b7e5eec07b 100644 --- a/ci/praktika/digest.py +++ b/ci/praktika/digest.py @@ -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 diff --git a/ci/praktika/environment.py b/ci/praktika/environment.py deleted file mode 100644 index 8f53aa6230b..00000000000 --- a/ci/praktika/environment.py +++ /dev/null @@ -1,3 +0,0 @@ -from praktika._environment import _Environment - -Environment = _Environment.get() diff --git a/ci/praktika/gh.py b/ci/praktika/gh.py index 77c360a0052..b7e49628ac8 100644 --- a/ci/praktika/gh.py +++ b/ci/praktika/gh.py @@ -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") diff --git a/ci/praktika/hook_cache.py b/ci/praktika/hook_cache.py index b1b5c654f20..e001e936a71 100644 --- a/ci/praktika/hook_cache.py +++ b/ci/praktika/hook_cache.py @@ -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") diff --git a/ci/praktika/hook_html.py b/ci/praktika/hook_html.py index f4bd4435511..e2faefb2fa9 100644 --- a/ci/praktika/hook_html.py +++ b/ci/praktika/hook_html.py @@ -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), ) diff --git a/ci/praktika/job.py b/ci/praktika/job.py index d0d4232cfa2..595a86456e9 100644 --- a/ci/praktika/job.py +++ b/ci/praktika/job.py @@ -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)})" diff --git a/ci/praktika/json.html b/ci/praktika/json.html index 2f8c3e45d0b..b11106719cd 100644 --- a/ci/praktika/json.html +++ b/ci/praktika/json.html @@ -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, '
'); 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; diff --git a/ci/praktika/mangle.py b/ci/praktika/mangle.py index 89fc52cf849..f94b11adad5 100644 --- a/ci/praktika/mangle.py +++ b/ci/praktika/mangle.py @@ -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 diff --git a/ci/praktika/native_jobs.py b/ci/praktika/native_jobs.py index f7fd4ca190b..cff6c851d0e 100644 --- a/ci/praktika/native_jobs.py +++ b/ci/praktika/native_jobs.py @@ -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__": diff --git a/ci/praktika/result.py b/ci/praktika/result.py index 3d3c986d5f9..8164b1d1295 100644 --- a/ci/praktika/result.py +++ b/ci/praktika/result.py @@ -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 diff --git a/ci/praktika/runner.py b/ci/praktika/runner.py index 797a799a74d..38112dd5684 100644 --- a/ci/praktika/runner.py +++ b/ci/praktika/runner.py @@ -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 ===") diff --git a/ci/praktika/runtime.py b/ci/praktika/runtime.py index a87b67c2c79..07c24e0498c 100644 --- a/ci/praktika/runtime.py +++ b/ci/praktika/runtime.py @@ -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 diff --git a/ci/praktika/s3.py b/ci/praktika/s3.py index 8cfb70a9076..82034b57b80 100644 --- a/ci/praktika/s3.py +++ b/ci/praktika/s3.py @@ -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() diff --git a/ci/praktika/settings.py b/ci/praktika/settings.py index 1a4068d9398..b281a95370c 100644 --- a/ci/praktika/settings.py +++ b/ci/praktika/settings.py @@ -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() diff --git a/ci/praktika/utils.py b/ci/praktika/utils.py index b96c78e4fa7..2bcc94f2559 100644 --- a/ci/praktika/utils.py +++ b/ci/praktika/utils.py @@ -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: diff --git a/ci/praktika/validator.py b/ci/praktika/validator.py index 29edc0a27ed..0bb722903e5 100644 --- a/ci/praktika/validator.py +++ b/ci/praktika/validator.py @@ -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): diff --git a/ci/praktika/workflow.py b/ci/praktika/workflow.py index 41e8056f9ef..8c5ec12440f 100644 --- a/ci/praktika/workflow.py +++ b/ci/praktika/workflow.py @@ -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 diff --git a/ci/praktika/yaml_generator.py b/ci/praktika/yaml_generator.py index 00c469fec0c..f56715755e8 100644 --- a/ci/praktika/yaml_generator.py +++ b/ci/praktika/yaml_generator.py @@ -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: diff --git a/ci/settings/definitions.py b/ci/settings/definitions.py index 176e865e6f3..8ebf79231ac 100644 --- a/ci/settings/definitions.py +++ b/ci/settings/definitions.py @@ -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" diff --git a/ci/settings/settings.py b/ci/settings/settings.py index 8d5e7bc3c87..0f3b1efcee0 100644 --- a/ci/settings/settings.py +++ b/ci/settings/settings.py @@ -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] diff --git a/ci/workflows/pull_request.py b/ci/workflows/pull_request.py index 74129177efb..761ab8a6ebc 100644 --- a/ci/workflows/pull_request.py +++ b/ci/workflows/pull_request.py @@ -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) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index fa0f95245f2..fcec9132cb7 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -217,7 +217,11 @@ add_contrib (libssh-cmake libssh) add_contrib (prometheus-protobufs-cmake prometheus-protobufs prometheus-protobufs-gogo) -add_contrib(numactl-cmake numactl) +add_contrib (numactl-cmake numactl) + +add_contrib (google-cloud-cpp-cmake google-cloud-cpp) # requires grpc, protobuf, absl + +add_contrib (jwt-cpp-cmake jwt-cpp) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/contrib/google-cloud-cpp b/contrib/google-cloud-cpp new file mode 160000 index 00000000000..83f30caadb8 --- /dev/null +++ b/contrib/google-cloud-cpp @@ -0,0 +1 @@ +Subproject commit 83f30caadb8613fb5c408d8c2fd545291596b53f diff --git a/contrib/google-cloud-cpp-cmake/CMakeLists.txt b/contrib/google-cloud-cpp-cmake/CMakeLists.txt new file mode 100644 index 00000000000..d4e7a885a39 --- /dev/null +++ b/contrib/google-cloud-cpp-cmake/CMakeLists.txt @@ -0,0 +1,105 @@ +set(ENABLE_GOOGLE_CLOUD_CPP_DEFAULT OFF) + +if(ENABLE_LIBRARIES AND CLICKHOUSE_CLOUD AND OS_LINUX) + set(ENABLE_GOOGLE_CLOUD_CPP_DEFAULT ON) +endif() + +option(ENABLE_GOOGLE_CLOUD_CPP "Enable Google Cloud Cpp" ${ENABLE_GOOGLE_CLOUD_CPP_DEFAULT}) + +if(NOT ENABLE_GOOGLE_CLOUD_CPP) + message(STATUS "Not using Google Cloud Cpp") + return() +endif() + +if(NOT ENABLE_GRPC) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use Google Cloud Cpp without gRPC") +endif() +if (NOT ENABLE_PROTOBUF) + message( ${RECONFIGURE_MESSAGE_LEVEL} "Can't use Google Cloud Cpp without protobuf") +endif() + +# Gather sources and options. +set(GOOGLE_CLOUD_CPP_SOURCES) +set(GOOGLE_CLOUD_CPP_PUBLIC_INCLUDES) +set(GOOGLE_CLOUD_CPP_PRIVATE_INCLUDES) +set(GOOGLE_CLOUD_CPP_PRIVATE_LIBS) + +# Directories. +SET(GOOGLE_CLOUD_CPP_DIR "${ClickHouse_SOURCE_DIR}/contrib/google-cloud-cpp" ) +list(APPEND GOOGLE_CLOUD_CPP_PRIVATE_INCLUDES "${GOOGLE_CLOUD_CPP_DIR}") + +# Set the PROJECT_SOURCE_DIR so that all Google Cloud cmake files work +set(PROJECT_SOURCE_DIR_BAK ${PROJECT_SOURCE_DIR}) +set(PROJECT_SOURCE_DIR ${GOOGLE_CLOUD_CPP_DIR}) + +list(APPEND CMAKE_MODULE_PATH "${GOOGLE_CLOUD_CPP_DIR}/cmake") + +# Building this target results in all protobufs being compiled. +add_custom_target(google-cloud-cpp-protos) + +include("GoogleCloudCppLibrary") + +# Set some variables required for googleapis CMakeLists.txt to work. +set(GOOGLE_CLOUD_CPP_ENABLE_GRPC ON) +set(PROJECT_VERSION "1") +set(PROJECT_VERSION_MAJOR "1") +set(PROTO_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/google-protobuf/src") +set(GOOGLE_CLOUD_CPP_GRPC_PLUGIN_EXECUTABLE $) + +include(GoogleApis.cmake) + +add_library(gRPC::grpc++ ALIAS _ch_contrib_grpc) +add_library(gRPC::grpc ALIAS _ch_contrib_grpc) + +# google-cloud-cpp-kms. +google_cloud_cpp_add_library_protos(kms) + +include(google_cloud_cpp_common.cmake) +include(google_cloud_cpp_grpc_utils.cmake) + +SET(GOOGLE_CLOUD_CPP_KMS_DIR "${GOOGLE_CLOUD_CPP_DIR}/google/cloud/kms") + +file(GLOB GOOGLE_CLOUD_CPP_KMS_SRC + "${GOOGLE_CLOUD_CPP_KMS_DIR}/v1/*.cc" + "${GOOGLE_CLOUD_CPP_KMS_DIR}/v1/internal/*.cc" + "${GOOGLE_CLOUD_CPP_KMS_DIR}/inventory/v1/*.cc" +) + +list(APPEND GOOGLE_CLOUD_CPP_SOURCES ${GOOGLE_CLOUD_CPP_KMS_SRC}) +list(APPEND GOOGLE_CLOUD_CPP_PUBLIC_INCLUDES "${GOOGLE_CLOUD_CPP_DIR}" "${CMAKE_CURRENT_BINARY_DIR}") + +set(GRPC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/grpc") +list(APPEND GOOGLE_CLOUD_CPP_PUBLIC_INCLUDES "${GRPC_INCLUDE_DIR}/include" "${GRPC_INCLUDE_DIR}/spm-cpp-include") + +# Restore the PROJECT_SOURCE_DIR. +set(PROJECT_SOURCE_DIR ${PROJECT_SOURCE_DIR_BAK}) + +# Link against external libraries. +list(APPEND GOOGLE_CLOUD_CPP_PRIVATE_LIBS + google_cloud_cpp_common + google_cloud_cpp_grpc_utils + google_cloud_cpp_kms_protos + google_cloud_cpp_cloud_location_locations_protos + google_cloud_cpp_iam_v1_iam_policy_protos + gRPC::grpc++ + absl::optional +) + +list(APPEND GOOGLE_CLOUD_CPP_PUBLIC_LIBS + absl::optional + gRPC::grpc++ +) + +# Add library. +add_library(_gcloud ${GOOGLE_CLOUD_CPP_SOURCES}) + +target_include_directories(_gcloud SYSTEM PUBLIC ${GOOGLE_CLOUD_CPP_PUBLIC_INCLUDES}) +target_include_directories(_gcloud SYSTEM PRIVATE ${GOOGLE_CLOUD_CPP_PRIVATE_INCLUDES}) +target_link_libraries(_gcloud PRIVATE ${GOOGLE_CLOUD_CPP_PRIVATE_LIBS}) + +# The library is large - avoid bloat. +if (OMIT_HEAVY_DEBUG_SYMBOLS) + target_compile_options(_gcloud PRIVATE -g0) +endif() + +add_library(ch_contrib::google_cloud_cpp ALIAS _gcloud) diff --git a/contrib/google-cloud-cpp-cmake/GoogleApis.cmake b/contrib/google-cloud-cpp-cmake/GoogleApis.cmake new file mode 100644 index 00000000000..070ce2275c4 --- /dev/null +++ b/contrib/google-cloud-cpp-cmake/GoogleApis.cmake @@ -0,0 +1,469 @@ +# ~~~ +# Copyright 2020 Google LLC +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# https://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# ~~~ + +# File copied from google-cloud-cpp/external/googleapis/CMakeLists.txt with minor modifications. + +if (NOT GOOGLE_CLOUD_CPP_ENABLE_GRPC) + return() +endif () + +include(GoogleapisConfig) + +set(GOOGLE_CLOUD_CPP_GOOGLEAPIS_URL + "https://github.com/googleapis/googleapis/archive/${_GOOGLE_CLOUD_CPP_GOOGLEAPIS_COMMIT_SHA}.tar.gz" + "https://storage.googleapis.com/cloud-cpp-community-archive/github.com/googleapis/googleapis/archive/${_GOOGLE_CLOUD_CPP_GOOGLEAPIS_COMMIT_SHA}.tar.gz" +) +set(GOOGLE_CLOUD_CPP_GOOGLEAPIS_URL_HASH + "${_GOOGLE_CLOUD_CPP_GOOGLEAPIS_SHA256}") +if (GOOGLE_CLOUD_CPP_OVERRIDE_GOOGLEAPIS_URL) + set(GOOGLE_CLOUD_CPP_GOOGLEAPIS_URL + ${GOOGLE_CLOUD_CPP_OVERRIDE_GOOGLEAPIS_URL}) +endif () +if (GOOGLE_CLOUD_CPP_OVERRIDE_GOOGLEAPIS_URL_HASH) + set(GOOGLE_CLOUD_CPP_GOOGLEAPIS_URL_HASH + "${GOOGLE_CLOUD_CPP_OVERRIDE_GOOGLEAPIS_URL_HASH}") +endif () + +set(EXTERNAL_GOOGLEAPIS_PROTO_FILES + # cmake-format: sort + "google/api/annotations.proto" + "google/api/auth.proto" + "google/api/backend.proto" + "google/api/billing.proto" + "google/api/client.proto" + "google/api/config_change.proto" + "google/api/consumer.proto" + "google/api/context.proto" + "google/api/control.proto" + "google/api/distribution.proto" + "google/api/documentation.proto" + "google/api/endpoint.proto" + "google/api/error_reason.proto" + "google/api/field_behavior.proto" + "google/api/field_info.proto" + "google/api/http.proto" + "google/api/httpbody.proto" + "google/api/label.proto" + "google/api/launch_stage.proto" + "google/api/log.proto" + "google/api/logging.proto" + "google/api/metric.proto" + "google/api/monitored_resource.proto" + "google/api/monitoring.proto" + "google/api/policy.proto" + "google/api/quota.proto" + "google/api/resource.proto" + "google/api/routing.proto" + "google/api/service.proto" + "google/api/source_info.proto" + "google/api/system_parameter.proto" + "google/api/usage.proto" + "google/api/visibility.proto" + "google/cloud/extended_operations.proto" + "google/cloud/location/locations.proto" + # orgpolicy/v**1** is used *indirectly* by google/cloud/asset, therefore it + # does not appear in protolists/asset.list. In addition, it is not compiled + # by any other library. So, added manually. + "google/cloud/orgpolicy/v1/orgpolicy.proto" + # Some gRPC based authentication is implemented by the IAM Credentials + # service. + "google/iam/credentials/v1/common.proto" + "google/iam/credentials/v1/iamcredentials.proto" + # We expose google::iam::v1::Policy in our google::cloud::IAMUpdater + "google/iam/v1/iam_policy.proto" + "google/iam/v1/options.proto" + "google/iam/v1/policy.proto" + "google/longrunning/operations.proto" + "google/rpc/code.proto" + "google/rpc/context/attribute_context.proto" + "google/rpc/error_details.proto" + "google/rpc/status.proto" + "google/type/calendar_period.proto" + "google/type/color.proto" + "google/type/date.proto" + "google/type/datetime.proto" + "google/type/dayofweek.proto" + "google/type/decimal.proto" + "google/type/expr.proto" + "google/type/fraction.proto" + "google/type/interval.proto" + "google/type/latlng.proto" + "google/type/localized_text.proto" + "google/type/money.proto" + "google/type/month.proto" + "google/type/phone_number.proto" + "google/type/postal_address.proto" + "google/type/quaternion.proto" + "google/type/timeofday.proto") + +include(GoogleCloudCppCommonOptions) + +# Set EXTERNAL_GOOGLEAPIS_SOURCE in the parent directory, as it is used by all +# the generated libraries. The Conan packages (https://conan.io), will need to +# patch this value. Setting the value in a single place makes such patching +# easier. +set(EXTERNAL_GOOGLEAPIS_PREFIX "${PROJECT_BINARY_DIR}/external/googleapis") +set(EXTERNAL_GOOGLEAPIS_SOURCE + "${EXTERNAL_GOOGLEAPIS_PREFIX}/src/googleapis_download" + PARENT_SCOPE) +set(EXTERNAL_GOOGLEAPIS_SOURCE + "${EXTERNAL_GOOGLEAPIS_PREFIX}/src/googleapis_download") + +# Include the functions to compile proto files and maintain proto libraries. +include(CompileProtos) + +set(EXTERNAL_GOOGLEAPIS_BYPRODUCTS) +foreach (proto ${EXTERNAL_GOOGLEAPIS_PROTO_FILES}) + list(APPEND EXTERNAL_GOOGLEAPIS_BYPRODUCTS + "${EXTERNAL_GOOGLEAPIS_SOURCE}/${proto}") +endforeach () + +file(GLOB protolists "protolists/*.list") +foreach (file IN LISTS protolists) + google_cloud_cpp_load_protolist(protos "${file}") + foreach (proto IN LISTS protos) + list(APPEND EXTERNAL_GOOGLEAPIS_BYPRODUCTS "${proto}") + endforeach () +endforeach () + +include(ExternalProject) + +# -- The build needs protobuf files. The original build scripts download them from a remote server (see target 'googleapis_download'). +# This is too unreliable in the context of ClickHouse ... we instead ship the downloaded archive with the ClickHouse source and +# extract it into the build directory directly. + +# Dummy googleapis_download target. This needs to exist because lots of other targets depend on it +# We however trick it a little bit saying this target generates the ${EXTERNAL_GOOGLEAPIS_BYPRODUCTS} BYPRODUCTS when +# actually the following section is the one actually providing such BYPRODUCTS. +externalproject_add( + googleapis_download + EXCLUDE_FROM_ALL ON + PREFIX "${EXTERNAL_GOOGLEAPIS_PREFIX}" + PATCH_COMMAND "" + DOWNLOAD_COMMAND "" + CONFIGURE_COMMAND "" + BUILD_COMMAND "" + INSTALL_COMMAND "" + BUILD_BYPRODUCTS ${EXTERNAL_GOOGLEAPIS_BYPRODUCTS} + LOG_DOWNLOAD OFF) + +# Command that extracts the tarball into the proper dir +# Note: The hash must match the Google Cloud Api version, otherwise funny things will happen. +# Find the right hash in "strip-prefix" in MODULE.bazel in the subrepository +message(STATUS "Extracting googleapis tarball") +set(PB_HASH "e60db19f11f94175ac682c5898cce0f77cc508ea") +set(PB_ARCHIVE "${PB_HASH}.tar.gz") +set(PB_DIR "googleapis-${PB_HASH}") + +file(ARCHIVE_EXTRACT INPUT + "${ClickHouse_SOURCE_DIR}/contrib/google-cloud-cpp-cmake/googleapis/${PB_ARCHIVE}" + DESTINATION + "${EXTERNAL_GOOGLEAPIS_PREFIX}/tmp") + +file(REMOVE_RECURSE "${EXTERNAL_GOOGLEAPIS_SOURCE}") +file(RENAME + "${EXTERNAL_GOOGLEAPIS_PREFIX}/tmp/${PB_DIR}" + "${EXTERNAL_GOOGLEAPIS_SOURCE}" +) + +google_cloud_cpp_find_proto_include_dir(PROTO_INCLUDE_DIR) + +google_cloud_cpp_add_protos_property() + +function (external_googleapis_short_name var proto) + string(REPLACE "google/" "" short_name "${proto}") + string(REPLACE "/" "_" short_name "${short_name}") + string(REPLACE ".proto" "_protos" short_name "${short_name}") + set("${var}" + "${short_name}" + PARENT_SCOPE) +endfunction () + +# Create a single source proto library. +# +# * proto: the filename for the proto source. +# * (optional) ARGN: proto libraries the new library depends on. +function (external_googleapis_add_library proto) + external_googleapis_short_name(short_name "${proto}") + google_cloud_cpp_grpcpp_library( + google_cloud_cpp_${short_name} "${EXTERNAL_GOOGLEAPIS_SOURCE}/${proto}" + PROTO_PATH_DIRECTORIES "${EXTERNAL_GOOGLEAPIS_SOURCE}" + "${PROTO_INCLUDE_DIR}") + + external_googleapis_set_version_and_alias("${short_name}") + + set(public_deps) + foreach (dep_short_name ${ARGN}) + list(APPEND public_deps "google-cloud-cpp::${dep_short_name}") + endforeach () + list(LENGTH public_deps public_deps_length) + if (public_deps_length EQUAL 0) + target_link_libraries("google_cloud_cpp_${short_name}") + else () + target_link_libraries("google_cloud_cpp_${short_name}" + PUBLIC ${public_deps}) + endif () +endfunction () + +function (external_googleapis_set_version_and_alias short_name) + add_dependencies("google_cloud_cpp_${short_name}" googleapis_download) + set_target_properties( + "google_cloud_cpp_${short_name}" + PROPERTIES EXPORT_NAME google-cloud-cpp::${short_name} + VERSION "${PROJECT_VERSION}" + SOVERSION ${PROJECT_VERSION_MAJOR}) + add_library("google-cloud-cpp::${short_name}" ALIAS + "google_cloud_cpp_${short_name}") +endfunction () + +if (GOOGLE_CLOUD_CPP_USE_INSTALLED_COMMON) + return() +endif () + +# Avoid adding new proto libraries to this list as these libraries are always +# installed, regardless of whether or not they are needed. See #8022 for more +# details. +set(external_googleapis_installed_libraries_list + # cmake-format: sort + google_cloud_cpp_cloud_common_common_protos + google_cloud_cpp_iam_credentials_v1_common_protos + google_cloud_cpp_iam_credentials_v1_iamcredentials_protos + google_cloud_cpp_iam_v1_iam_policy_protos + google_cloud_cpp_iam_v1_options_protos + google_cloud_cpp_iam_v1_policy_protos + google_cloud_cpp_longrunning_operations_protos) + +# These proto files cannot be added in the foreach() loop because they have +# dependencies. +set(PROTO_FILES_WITH_DEPENDENCIES + # cmake-format: sort + "google/api/annotations.proto" + "google/api/auth.proto" + "google/api/billing.proto" + "google/api/client.proto" + "google/api/control.proto" + "google/api/distribution.proto" + "google/api/endpoint.proto" + "google/api/log.proto" + "google/api/logging.proto" + "google/api/metric.proto" + "google/api/monitored_resource.proto" + "google/api/monitoring.proto" + "google/api/quota.proto" + "google/api/service.proto" + "google/api/usage.proto" + "google/cloud/location/locations.proto" + "google/rpc/status.proto") + +# For some directories *most* (but not all) the proto files are simple enough +# that the libraries can be generated with a foreach() loop. +foreach (proto IN LISTS EXTERNAL_GOOGLEAPIS_PROTO_FILES) + if (proto MATCHES "^google/api/" + OR proto MATCHES "^google/type" + OR proto MATCHES "^google/rpc/" + OR proto MATCHES "^google/cloud/") + external_googleapis_short_name(short_name "${proto}") + list(APPEND external_googleapis_installed_libraries_list + google_cloud_cpp_${short_name}) + list(FIND PROTO_FILES_WITH_DEPENDENCIES "${proto}" has_dependency) + if (has_dependency EQUAL -1) + external_googleapis_add_library("${proto}") + endif () + endif () +endforeach () + +# Out of order because they have dependencies. +external_googleapis_add_library("google/api/annotations.proto" api_http_protos) +external_googleapis_add_library("google/api/auth.proto" api_annotations_protos) +external_googleapis_add_library("google/api/client.proto" + api_launch_stage_protos) +external_googleapis_add_library("google/api/control.proto" api_policy_protos) +external_googleapis_add_library("google/api/metric.proto" + api_launch_stage_protos api_label_protos) +external_googleapis_add_library("google/api/billing.proto" + api_annotations_protos api_metric_protos) +external_googleapis_add_library("google/api/distribution.proto" + api_annotations_protos) +external_googleapis_add_library("google/api/endpoint.proto" + api_annotations_protos) +external_googleapis_add_library("google/api/log.proto" api_label_protos) +external_googleapis_add_library("google/api/logging.proto" + api_annotations_protos api_label_protos) +external_googleapis_add_library("google/api/monitored_resource.proto" + api_launch_stage_protos api_label_protos) +external_googleapis_add_library("google/api/monitoring.proto" + api_annotations_protos) +external_googleapis_add_library("google/api/quota.proto" api_annotations_protos) +external_googleapis_add_library("google/api/usage.proto" api_annotations_protos + api_visibility_protos) +external_googleapis_add_library( + "google/api/service.proto" + api_annotations_protos + api_auth_protos + api_backend_protos + api_billing_protos + api_client_protos + api_context_protos + api_control_protos + api_documentation_protos + api_endpoint_protos + api_http_protos + api_label_protos + api_log_protos + api_logging_protos + api_metric_protos + api_monitored_resource_protos + api_monitoring_protos + api_quota_protos + api_resource_protos + api_source_info_protos + api_system_parameter_protos + api_usage_protos) + +external_googleapis_add_library("google/cloud/location/locations.proto" + api_annotations_protos api_client_protos) + +external_googleapis_add_library("google/iam/v1/options.proto" + api_annotations_protos) +external_googleapis_add_library("google/iam/v1/policy.proto" + api_annotations_protos type_expr_protos) +external_googleapis_add_library("google/rpc/status.proto" + rpc_error_details_protos) + +external_googleapis_add_library( + "google/longrunning/operations.proto" api_annotations_protos + api_client_protos rpc_status_protos) + +external_googleapis_add_library( + "google/iam/v1/iam_policy.proto" + api_annotations_protos + api_client_protos + api_field_behavior_protos + api_resource_protos + iam_v1_options_protos + iam_v1_policy_protos) + +external_googleapis_add_library("google/iam/credentials/v1/common.proto" + api_field_behavior_protos api_resource_protos) + +external_googleapis_add_library( + "google/iam/credentials/v1/iamcredentials.proto" api_annotations_protos + api_client_protos iam_credentials_v1_common_protos) + +google_cloud_cpp_load_protolist(cloud_common_list "${GOOGLE_CLOUD_CPP_DIR}/external/googleapis/protolists/common.list") +google_cloud_cpp_load_protodeps(cloud_common_deps "${GOOGLE_CLOUD_CPP_DIR}/external/googleapis/protodeps/common.deps") +google_cloud_cpp_grpcpp_library( + google_cloud_cpp_cloud_common_common_protos ${cloud_common_list} + PROTO_PATH_DIRECTORIES "${EXTERNAL_GOOGLEAPIS_SOURCE}" + "${PROTO_INCLUDE_DIR}") +external_googleapis_set_version_and_alias(cloud_common_common_protos) +target_link_libraries(google_cloud_cpp_cloud_common_common_protos + PUBLIC ${cloud_common_deps}) + +# Install the libraries and headers in the locations determined by +# GNUInstallDirs +include(GNUInstallDirs) + +install( + TARGETS ${external_googleapis_installed_libraries_list} + EXPORT googleapis-targets + RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} + COMPONENT google_cloud_cpp_runtime + LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} + COMPONENT google_cloud_cpp_runtime + NAMELINK_COMPONENT google_cloud_cpp_development + ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR} + COMPONENT google_cloud_cpp_development) + +foreach (target ${external_googleapis_installed_libraries_list}) + google_cloud_cpp_install_proto_library_headers("${target}") + google_cloud_cpp_install_proto_library_protos( + "${target}" "${EXTERNAL_GOOGLEAPIS_SOURCE}") +endforeach () + +# Create and install the pkg-config files. +foreach (target ${external_googleapis_installed_libraries_list}) + external_googleapis_install_pc("${target}") +endforeach () + +# Create and install the googleapis pkg-config file for backwards compatibility. +set(GOOGLE_CLOUD_CPP_PC_LIBS "") +google_cloud_cpp_set_pkgconfig_paths() +set(GOOGLE_CLOUD_CPP_PC_NAME "The Google APIS C++ Proto Library") +set(GOOGLE_CLOUD_CPP_PC_DESCRIPTION + "Provides C++ APIs to access Google Cloud Platforms.") +# This list is for backwards compatibility purposes only. DO NOT add new +# libraries to it. +string( + JOIN + " " + GOOGLE_CLOUD_CPP_PC_REQUIRES + "google_cloud_cpp_bigtable_protos" + "google_cloud_cpp_cloud_bigquery_protos" + "google_cloud_cpp_iam_protos" + "google_cloud_cpp_pubsub_protos" + "google_cloud_cpp_storage_protos" + "google_cloud_cpp_logging_protos" + "google_cloud_cpp_iam_v1_iam_policy_protos" + "google_cloud_cpp_iam_v1_options_protos" + "google_cloud_cpp_iam_v1_policy_protos" + "google_cloud_cpp_longrunning_operations_protos" + "google_cloud_cpp_api_auth_protos" + "google_cloud_cpp_api_annotations_protos" + "google_cloud_cpp_api_client_protos" + "google_cloud_cpp_api_field_behavior_protos" + "google_cloud_cpp_api_http_protos" + "google_cloud_cpp_rpc_status_protos" + "google_cloud_cpp_rpc_error_details_protos" + "google_cloud_cpp_type_expr_protos" + "grpc++" + "grpc" + "openssl" + "protobuf" + "zlib" + "libcares") +set(GOOGLE_CLOUD_CPP_PC_LIBS "") +google_cloud_cpp_set_pkgconfig_paths() +configure_file("${PROJECT_SOURCE_DIR}/cmake/templates/config.pc.in" + "googleapis.pc" @ONLY) +install( + FILES "${CMAKE_CURRENT_BINARY_DIR}/googleapis.pc" + DESTINATION "${CMAKE_INSTALL_LIBDIR}/pkgconfig" + COMPONENT google_cloud_cpp_development) + +# Create and install the CMake configuration files. +# include(CMakePackageConfigHelpers) + +# configure_file("${CMAKE_CURRENT_LIST_DIR}/config.cmake.in" +# "google_cloud_cpp_googleapis-config.cmake" @ONLY) +# write_basic_package_version_file( +# "google_cloud_cpp_googleapis-config-version.cmake" +# VERSION ${PROJECT_VERSION} +# COMPATIBILITY ExactVersion) + +# Export the CMake targets to make it easy to create configuration files. +# install( +# EXPORT googleapis-targets +# DESTINATION "${CMAKE_INSTALL_LIBDIR}/cmake/google_cloud_cpp_googleapis" +# COMPONENT google_cloud_cpp_development) +# install( +# FILES +# "${CMAKE_CURRENT_BINARY_DIR}/google_cloud_cpp_googleapis-config.cmake" +# "${CMAKE_CURRENT_BINARY_DIR}/google_cloud_cpp_googleapis-config-version.cmake" +# "${PROJECT_SOURCE_DIR}/cmake/FindgRPC.cmake" +# "${PROJECT_SOURCE_DIR}/cmake/CompileProtos.cmake" +# DESTINATION "${CMAKE_INSTALL_LIBDIR}/cmake/google_cloud_cpp_googleapis" +# COMPONENT google_cloud_cpp_development) diff --git a/contrib/google-cloud-cpp-cmake/google_cloud_cpp_common.cmake b/contrib/google-cloud-cpp-cmake/google_cloud_cpp_common.cmake new file mode 100644 index 00000000000..5f23ce0fafb --- /dev/null +++ b/contrib/google-cloud-cpp-cmake/google_cloud_cpp_common.cmake @@ -0,0 +1,447 @@ +# ~~~ +# Copyright 2022 Google LLC +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# https://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# ~~~ + +# File copied from google-cloud-cpp/google-cloud-cpp/google_cloud_cpp_common.cmake with minor modifications. + +set(GOOGLE_CLOUD_CPP_COMMON_DIR "${GOOGLE_CLOUD_CPP_DIR}/google/cloud") + +# Generate the version information from the CMake values. +# configure_file(${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/version_info.h.in +# ${CMAKE_CURRENT_SOURCE_DIR}/internal/version_info.h) + +# Create the file that captures build information. Having access to the compiler +# and build flags at runtime allows us to print better benchmark results. +string(TOUPPER "${CMAKE_BUILD_TYPE}" GOOGLE_CLOUD_CPP_BUILD_TYPE_UPPER) +configure_file(${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/build_info.cc.in internal/build_info.cc) + +# the client library +add_library( + google_cloud_cpp_common # cmake-format: sort + ${CMAKE_CURRENT_BINARY_DIR}/internal/build_info.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/access_token.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/access_token.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/backoff_policy.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/common_options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/credentials.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/credentials.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/experimental_tag.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/future.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/future_generic.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/future_void.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/idempotency.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/absl_str_cat_quiet.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/absl_str_join_quiet.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/absl_str_replace_quiet.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/algorithm.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/api_client_header.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/api_client_header.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/attributes.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/auth_header_error.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/auth_header_error.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/backoff_policy.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/backoff_policy.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/base64_transforms.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/base64_transforms.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/big_endian.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/build_info.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/call_context.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/clock.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/compiler_info.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/compiler_info.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/compute_engine_util.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/compute_engine_util.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/credentials_impl.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/credentials_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/debug_future_status.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/debug_future_status.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/debug_string.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/debug_string.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/detect_gcp.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/detect_gcp_impl.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/detect_gcp_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/diagnostics_pop.inc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/diagnostics_push.inc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/disable_deprecation_warnings.inc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/disable_msvc_crt_secure_warnings.inc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/error_context.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/error_context.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/filesystem.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/filesystem.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/format_time_point.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/format_time_point.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/future_base.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/future_coroutines.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/future_fwd.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/future_impl.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/future_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/future_then_impl.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/future_then_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/getenv.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/getenv.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/group_options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/invocation_id_generator.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/invocation_id_generator.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/invoke_result.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/ios_flags_saver.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/log_impl.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/log_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/make_status.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/make_status.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/noexcept_action.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/noexcept_action.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/non_constructible.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/opentelemetry.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/opentelemetry.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/opentelemetry_context.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/opentelemetry_context.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/pagination_range.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/parse_rfc3339.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/parse_rfc3339.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/populate_common_options.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/populate_common_options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/port_platform.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/random.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/random.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/retry_info.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/retry_loop_helpers.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/retry_loop_helpers.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/retry_policy_impl.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/retry_policy_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/service_endpoint.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/service_endpoint.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/sha256_hash.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/sha256_hash.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/sha256_hmac.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/sha256_hmac.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/sha256_type.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/status_payload_keys.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/status_payload_keys.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/status_utils.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/status_utils.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/strerror.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/strerror.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/subject_token.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/subject_token.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/throw_delegate.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/throw_delegate.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/timer_queue.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/timer_queue.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/trace_propagator.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/trace_propagator.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/traced_stream_range.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/tuple.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/type_list.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/type_traits.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/url_encode.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/url_encode.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/user_agent_prefix.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/user_agent_prefix.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/utility.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/version_info.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/kms_key_name.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/kms_key_name.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/location.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/location.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/log.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/log.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/no_await_tag.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/opentelemetry_options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/optional.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/options.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/polling_policy.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/project.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/project.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/retry_policy.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/rpc_metadata.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/status.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/status.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/status_or.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/stream_range.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/terminate_handler.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/terminate_handler.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/tracing_options.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/tracing_options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/universe_domain_options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/version.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/version.h) +target_link_libraries( + google_cloud_cpp_common + PUBLIC absl::base + absl::memory + absl::optional + absl::span + absl::str_format + absl::time + absl::variant + Threads::Threads) +if (WIN32) + target_compile_definitions(google_cloud_cpp_common + PRIVATE WIN32_LEAN_AND_MEAN) + target_link_libraries(google_cloud_cpp_common PUBLIC bcrypt) +else () + target_link_libraries(google_cloud_cpp_common PUBLIC OpenSSL::Crypto ch_contrib::re2) +endif () + +google_cloud_cpp_add_common_options(google_cloud_cpp_common) +target_include_directories( + google_cloud_cpp_common PUBLIC $ + $) + +# We're putting generated code into ${PROJECT_BINARY_DIR} (e.g. compiled +# protobufs or build info), so we need it on the include path, however we don't +# want it checked by linters so we mark it as SYSTEM. +target_include_directories(google_cloud_cpp_common SYSTEM + PUBLIC $) +target_compile_options(google_cloud_cpp_common + PUBLIC ${GOOGLE_CLOUD_CPP_EXCEPTIONS_FLAG}) + +set_target_properties( + google_cloud_cpp_common + PROPERTIES EXPORT_NAME "google-cloud-cpp::common" + VERSION ${PROJECT_VERSION} + SOVERSION ${PROJECT_VERSION_MAJOR}) +add_library(google-cloud-cpp::common ALIAS google_cloud_cpp_common) + +#create_bazel_config(google_cloud_cpp_common YEAR 2018) + +# # Export the CMake targets to make it easy to create configuration files. +# install( +# EXPORT google_cloud_cpp_common-targets +# DESTINATION "${CMAKE_INSTALL_LIBDIR}/cmake/google_cloud_cpp_common" +# COMPONENT google_cloud_cpp_development) + +# # Install the libraries and headers in the locations determined by +# # GNUInstallDirs +# install( +# TARGETS google_cloud_cpp_common +# EXPORT google_cloud_cpp_common-targets +# RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} +# COMPONENT google_cloud_cpp_runtime +# LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} +# COMPONENT google_cloud_cpp_runtime +# NAMELINK_COMPONENT google_cloud_cpp_development +# ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR} +# COMPONENT google_cloud_cpp_development) + +#google_cloud_cpp_install_headers(google_cloud_cpp_common include/google/cloud) + +# google_cloud_cpp_add_pkgconfig( +# "common" +# "Google Cloud C++ Client Library Common Components" +# "Common Components used by the Google Cloud C++ Client Libraries." +# "absl_optional" +# "absl_span" +# "absl_strings" +# "absl_time" +# "absl_time_zone" +# "absl_variant" +# "${GOOGLE_CLOUD_CPP_OPENTELEMETRY_API}" +# NON_WIN32_REQUIRES +# openssl +# WIN32_LIBS +# bcrypt) + +# Create and install the CMake configuration files. +# configure_file("config.cmake.in" "google_cloud_cpp_common-config.cmake" @ONLY) +# write_basic_package_version_file( +# "google_cloud_cpp_common-config-version.cmake" +# VERSION ${PROJECT_VERSION} +# COMPATIBILITY ExactVersion) + +# install( +# FILES +# "${CMAKE_CURRENT_BINARY_DIR}/google_cloud_cpp_common-config.cmake" +# "${CMAKE_CURRENT_BINARY_DIR}/google_cloud_cpp_common-config-version.cmake" +# DESTINATION "${CMAKE_INSTALL_LIBDIR}/cmake/google_cloud_cpp_common" +# COMPONENT google_cloud_cpp_development) + +# if (GOOGLE_CLOUD_CPP_WITH_MOCKS) +# # Create a header-only library for the mocks. We use a CMake `INTERFACE` +# # library for these, a regular library would not work on macOS (where the +# # library needs at least one .o file). +# add_library(google_cloud_cpp_mocks INTERFACE) +# set(google_cloud_cpp_mocks_hdrs +# # cmake-format: sort +# mocks/current_options.h mocks/mock_async_streaming_read_write_rpc.h +# mocks/mock_stream_range.h) +# export_list_to_bazel("google_cloud_cpp_mocks.bzl" +# "google_cloud_cpp_mocks_hdrs" YEAR "2022") +# target_link_libraries( +# google_cloud_cpp_mocks INTERFACE google-cloud-cpp::common GTest::gmock +# GTest::gtest) +# set_target_properties(google_cloud_cpp_mocks +# PROPERTIES EXPORT_NAME google-cloud-cpp::mocks) +# target_include_directories( +# google_cloud_cpp_mocks +# INTERFACE $ +# $ +# $) +# target_compile_options(google_cloud_cpp_mocks +# INTERFACE ${GOOGLE_CLOUD_CPP_EXCEPTIONS_FLAG}) +# add_library(google-cloud-cpp::mocks ALIAS google_cloud_cpp_mocks) + +# install( +# FILES ${google_cloud_cpp_mocks_hdrs} +# DESTINATION "include/google/cloud/mocks" +# COMPONENT google_cloud_cpp_development) + +# # Export the CMake targets to make it easy to create configuration files. +# install( +# EXPORT google_cloud_cpp_mocks-targets +# DESTINATION "${CMAKE_INSTALL_LIBDIR}/cmake/google_cloud_cpp_mocks" +# COMPONENT google_cloud_cpp_development) + +# install( +# TARGETS google_cloud_cpp_mocks +# EXPORT google_cloud_cpp_mocks-targets +# COMPONENT google_cloud_cpp_development) + +# google_cloud_cpp_add_pkgconfig( +# "mocks" "Google Cloud C++ Testing Library" +# "Helpers for testing the Google Cloud C++ Client Libraries" +# "google_cloud_cpp_common" "gmock") + +# # Create and install the CMake configuration files. +# configure_file("mocks-config.cmake.in" +# "google_cloud_cpp_mocks-config.cmake" @ONLY) +# write_basic_package_version_file( +# "google_cloud_cpp_mocks-config-version.cmake" +# VERSION ${PROJECT_VERSION} +# COMPATIBILITY ExactVersion) + +# install( +# FILES +# "${CMAKE_CURRENT_BINARY_DIR}/google_cloud_cpp_mocks-config.cmake" +# "${CMAKE_CURRENT_BINARY_DIR}/google_cloud_cpp_mocks-config-version.cmake" +# DESTINATION "${CMAKE_INSTALL_LIBDIR}/cmake/google_cloud_cpp_mocks" +# COMPONENT google_cloud_cpp_development) +# endif () + +# if (BUILD_TESTING) +# include(FindBenchmarkWithWorkarounds) + +# set(google_cloud_cpp_common_unit_tests +# # cmake-format: sort +# access_token_test.cc +# common_options_test.cc +# future_coroutines_test.cc +# future_generic_test.cc +# future_generic_then_test.cc +# future_void_test.cc +# future_void_then_test.cc +# internal/algorithm_test.cc +# internal/api_client_header_test.cc +# internal/backoff_policy_test.cc +# internal/base64_transforms_test.cc +# internal/big_endian_test.cc +# internal/call_context_test.cc +# internal/clock_test.cc +# internal/compiler_info_test.cc +# internal/compute_engine_util_test.cc +# internal/credentials_impl_test.cc +# internal/debug_future_status_test.cc +# internal/debug_string_test.cc +# internal/detect_gcp_test.cc +# internal/error_context_test.cc +# internal/filesystem_test.cc +# internal/format_time_point_test.cc +# internal/future_impl_test.cc +# internal/future_then_impl_test.cc +# internal/group_options_test.cc +# internal/invocation_id_generator_test.cc +# internal/invoke_result_test.cc +# internal/log_impl_test.cc +# internal/make_status_test.cc +# internal/noexcept_action_test.cc +# internal/opentelemetry_context_test.cc +# internal/opentelemetry_test.cc +# internal/pagination_range_test.cc +# internal/parse_rfc3339_test.cc +# internal/populate_common_options_test.cc +# internal/random_test.cc +# internal/retry_loop_helpers_test.cc +# internal/retry_policy_impl_test.cc +# internal/service_endpoint_test.cc +# internal/sha256_hash_test.cc +# internal/sha256_hmac_test.cc +# internal/status_payload_keys_test.cc +# internal/status_utils_test.cc +# internal/strerror_test.cc +# internal/subject_token_test.cc +# internal/throw_delegate_test.cc +# internal/timer_queue_test.cc +# internal/trace_propagator_test.cc +# internal/traced_stream_range_test.cc +# internal/tuple_test.cc +# internal/type_list_test.cc +# internal/url_encode_test.cc +# internal/user_agent_prefix_test.cc +# internal/utility_test.cc +# kms_key_name_test.cc +# location_test.cc +# log_test.cc +# mocks/current_options_test.cc +# mocks/mock_stream_range_test.cc +# options_test.cc +# polling_policy_test.cc +# project_test.cc +# status_or_test.cc +# status_test.cc +# stream_range_test.cc +# terminate_handler_test.cc +# tracing_options_test.cc) + +# # Export the list of unit tests so the Bazel BUILD file can pick it up. +# export_list_to_bazel("google_cloud_cpp_common_unit_tests.bzl" +# "google_cloud_cpp_common_unit_tests" YEAR "2018") + +# foreach (fname ${google_cloud_cpp_common_unit_tests}) +# google_cloud_cpp_add_executable(target "common" "${fname}") +# target_link_libraries( +# ${target} +# PRIVATE google_cloud_cpp_testing +# google-cloud-cpp::common +# google-cloud-cpp::mocks +# absl::variant +# GTest::gmock_main +# GTest::gmock +# GTest::gtest) +# google_cloud_cpp_add_common_options(${target}) +# add_test(NAME ${target} COMMAND ${target}) +# endforeach () + +# set(google_cloud_cpp_common_benchmarks # cmake-format: sort +# options_benchmark.cc) + +# # Export the list of benchmarks to a .bzl file so we do not need to maintain +# # the list in two places. +# export_list_to_bazel("google_cloud_cpp_common_benchmarks.bzl" +# "google_cloud_cpp_common_benchmarks" YEAR "2020") + +# # Generate a target for each benchmark. +# foreach (fname ${google_cloud_cpp_common_benchmarks}) +# google_cloud_cpp_add_executable(target "common" "${fname}") +# add_test(NAME ${target} COMMAND ${target}) +# target_link_libraries(${target} PRIVATE google-cloud-cpp::common +# benchmark::benchmark_main) +# google_cloud_cpp_add_common_options(${target}) +# endforeach () +# endif () + +# if (BUILD_TESTING AND GOOGLE_CLOUD_CPP_ENABLE_CXX_EXCEPTIONS) +# google_cloud_cpp_add_samples_relative("common" "samples/") +# endif () diff --git a/contrib/google-cloud-cpp-cmake/google_cloud_cpp_grpc_utils.cmake b/contrib/google-cloud-cpp-cmake/google_cloud_cpp_grpc_utils.cmake new file mode 100644 index 00000000000..e5538b2eaa8 --- /dev/null +++ b/contrib/google-cloud-cpp-cmake/google_cloud_cpp_grpc_utils.cmake @@ -0,0 +1,350 @@ +# ~~~ +# Copyright 2022 Google LLC +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# https://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# ~~~ + +# File copied from google-cloud-cpp/google-cloud-cpp/google_cloud_cpp_grpc_utils.cmake with minor modifications. + +set(GOOGLE_CLOUD_CPP_COMMON_DIR "${GOOGLE_CLOUD_CPP_DIR}/google/cloud") + +# the library +add_library( + google_cloud_cpp_grpc_utils # cmake-format: sort + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/async_operation.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/async_streaming_read_write_rpc.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/background_threads.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/completion_queue.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/completion_queue.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/connection_options.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/connection_options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/grpc_error_delegate.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/grpc_error_delegate.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/grpc_options.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/grpc_options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/grpc_utils/async_operation.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/grpc_utils/completion_queue.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/grpc_utils/grpc_error_delegate.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/grpc_utils/version.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/iam_updater.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_connection_ready.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_connection_ready.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_long_running_operation.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_polling_loop.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_polling_loop.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_read_stream_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_read_write_stream_auth.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_read_write_stream_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_read_write_stream_logging.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_read_write_stream_timeout.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_read_write_stream_tracing.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_retry_loop.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_retry_unary_rpc.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_rpc_details.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_read_rpc.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_read_rpc_auth.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_read_rpc_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_read_rpc_logging.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_read_rpc_timeout.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_read_rpc_tracing.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_write_rpc.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_write_rpc_auth.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_write_rpc_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_write_rpc_logging.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_write_rpc_timeout.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/async_streaming_write_rpc_tracing.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/background_threads_impl.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/background_threads_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/completion_queue_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/debug_string_protobuf.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/debug_string_protobuf.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/debug_string_status.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/debug_string_status.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/default_completion_queue_impl.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/default_completion_queue_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/extract_long_running_result.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/extract_long_running_result.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_access_token_authentication.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_access_token_authentication.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_api_key_authentication.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_api_key_authentication.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_async_access_token_cache.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_async_access_token_cache.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_channel_credentials_authentication.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_channel_credentials_authentication.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_impersonate_service_account.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_impersonate_service_account.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_metadata_view.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_opentelemetry.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_opentelemetry.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_request_metadata.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_request_metadata.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_service_account_authentication.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/grpc_service_account_authentication.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/log_wrapper.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/log_wrapper.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/minimal_iam_credentials_stub.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/minimal_iam_credentials_stub.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/populate_grpc_options.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/populate_grpc_options.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/resumable_streaming_read_rpc.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/retry_loop.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/routing_matcher.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/setup_context.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/streaming_read_rpc.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/streaming_read_rpc.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/streaming_read_rpc_logging.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/streaming_read_rpc_tracing.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/streaming_write_rpc.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/streaming_write_rpc_impl.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/streaming_write_rpc_impl.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/streaming_write_rpc_logging.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/streaming_write_rpc_tracing.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/time_utils.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/time_utils.h + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/unified_grpc_credentials.cc + ${GOOGLE_CLOUD_CPP_COMMON_DIR}/internal/unified_grpc_credentials.h) +target_link_libraries( + google_cloud_cpp_grpc_utils + PUBLIC absl::function_ref + absl::memory + absl::time + absl::variant + google-cloud-cpp::iam_credentials_v1_iamcredentials_protos + google-cloud-cpp::iam_v1_policy_protos + google-cloud-cpp::longrunning_operations_protos + google-cloud-cpp::iam_v1_iam_policy_protos + google-cloud-cpp::rpc_error_details_protos + google-cloud-cpp::rpc_status_protos + google-cloud-cpp::common + gRPC::grpc++ + gRPC::grpc) +google_cloud_cpp_add_common_options(google_cloud_cpp_grpc_utils) +target_include_directories( + google_cloud_cpp_grpc_utils PUBLIC $ + $) +target_compile_options(google_cloud_cpp_grpc_utils + PUBLIC ${GOOGLE_CLOUD_CPP_EXCEPTIONS_FLAG}) +set_target_properties( + google_cloud_cpp_grpc_utils + PROPERTIES EXPORT_NAME "google-cloud-cpp::grpc_utils" + VERSION ${PROJECT_VERSION} + SOVERSION ${PROJECT_VERSION_MAJOR}) +add_library(google-cloud-cpp::grpc_utils ALIAS google_cloud_cpp_grpc_utils) + +#create_bazel_config(google_cloud_cpp_grpc_utils YEAR 2019) + +# # Install the libraries and headers in the locations determined by +# # GNUInstallDirs +# install( +# TARGETS +# EXPORT grpc_utils-targets +# RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} +# LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} +# ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR} +# COMPONENT google_cloud_cpp_development) + +# # Export the CMake targets to make it easy to create configuration files. +# install( +# EXPORT grpc_utils-targets +# DESTINATION "${CMAKE_INSTALL_LIBDIR}/cmake/google_cloud_cpp_grpc_utils" +# COMPONENT google_cloud_cpp_development) + +# install( +# TARGETS google_cloud_cpp_grpc_utils +# EXPORT grpc_utils-targets +# RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} +# COMPONENT google_cloud_cpp_runtime +# LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} +# COMPONENT google_cloud_cpp_runtime +# NAMELINK_COMPONENT google_cloud_cpp_development +# ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR} +# COMPONENT google_cloud_cpp_development) + +# google_cloud_cpp_install_headers(google_cloud_cpp_grpc_utils +# include/google/cloud) + +# google_cloud_cpp_add_pkgconfig( +# grpc_utils +# "gRPC Utilities for the Google Cloud C++ Client Library" +# "Provides gRPC Utilities for the Google Cloud C++ Client Library." +# "google_cloud_cpp_common" +# "google_cloud_cpp_iam_credentials_v1_iamcredentials_protos" +# "google_cloud_cpp_iam_v1_policy_protos" +# "google_cloud_cpp_iam_v1_iam_policy_protos" +# "google_cloud_cpp_longrunning_operations_protos" +# "google_cloud_cpp_rpc_status_protos" +# "absl_function_ref" +# "absl_strings" +# "absl_time" +# "absl_time_zone" +# "absl_variant" +# "openssl") + +# # Create and install the CMake configuration files. +# configure_file("grpc_utils/config.cmake.in" +# "google_cloud_cpp_grpc_utils-config.cmake" @ONLY) +# write_basic_package_version_file( +# "google_cloud_cpp_grpc_utils-config-version.cmake" +# VERSION ${PROJECT_VERSION} +# COMPATIBILITY ExactVersion) + +# install( +# FILES +# "${CMAKE_CURRENT_BINARY_DIR}/google_cloud_cpp_grpc_utils-config.cmake" +# "${CMAKE_CURRENT_BINARY_DIR}/google_cloud_cpp_grpc_utils-config-version.cmake" +# DESTINATION "${CMAKE_INSTALL_LIBDIR}/cmake/google_cloud_cpp_grpc_utils" +# COMPONENT google_cloud_cpp_development) + +# function (google_cloud_cpp_grpc_utils_add_test fname labels) +# google_cloud_cpp_add_executable(target "common" "${fname}") +# target_link_libraries( +# ${target} +# PRIVATE google-cloud-cpp::grpc_utils +# google_cloud_cpp_testing_grpc +# google_cloud_cpp_testing +# google-cloud-cpp::common +# absl::variant +# GTest::gmock_main +# GTest::gmock +# GTest::gtest +# gRPC::grpc++ +# gRPC::grpc) +# google_cloud_cpp_add_common_options(${target}) +# add_test(NAME ${target} COMMAND ${target}) +# set_tests_properties(${target} PROPERTIES LABELS "${labels}") +# endfunction () + +# if (BUILD_TESTING) +# include(FindBenchmarkWithWorkarounds) + +# # List the unit tests, then setup the targets and dependencies. +# set(google_cloud_cpp_grpc_utils_unit_tests +# # cmake-format: sort +# completion_queue_test.cc +# connection_options_test.cc +# grpc_error_delegate_test.cc +# grpc_options_test.cc +# internal/async_connection_ready_test.cc +# internal/async_long_running_operation_test.cc +# internal/async_polling_loop_test.cc +# internal/async_read_write_stream_auth_test.cc +# internal/async_read_write_stream_impl_test.cc +# internal/async_read_write_stream_logging_test.cc +# internal/async_read_write_stream_timeout_test.cc +# internal/async_read_write_stream_tracing_test.cc +# internal/async_retry_loop_test.cc +# internal/async_retry_unary_rpc_test.cc +# internal/async_streaming_read_rpc_auth_test.cc +# internal/async_streaming_read_rpc_impl_test.cc +# internal/async_streaming_read_rpc_logging_test.cc +# internal/async_streaming_read_rpc_timeout_test.cc +# internal/async_streaming_read_rpc_tracing_test.cc +# internal/async_streaming_write_rpc_auth_test.cc +# internal/async_streaming_write_rpc_impl_test.cc +# internal/async_streaming_write_rpc_logging_test.cc +# internal/async_streaming_write_rpc_timeout_test.cc +# internal/async_streaming_write_rpc_tracing_test.cc +# internal/background_threads_impl_test.cc +# internal/debug_string_protobuf_test.cc +# internal/debug_string_status_test.cc +# internal/extract_long_running_result_test.cc +# internal/grpc_access_token_authentication_test.cc +# internal/grpc_async_access_token_cache_test.cc +# internal/grpc_channel_credentials_authentication_test.cc +# internal/grpc_opentelemetry_test.cc +# internal/grpc_request_metadata_test.cc +# internal/grpc_service_account_authentication_test.cc +# internal/log_wrapper_test.cc +# internal/minimal_iam_credentials_stub_test.cc +# internal/populate_grpc_options_test.cc +# internal/resumable_streaming_read_rpc_test.cc +# internal/retry_loop_test.cc +# internal/routing_matcher_test.cc +# internal/streaming_read_rpc_logging_test.cc +# internal/streaming_read_rpc_test.cc +# internal/streaming_read_rpc_tracing_test.cc +# internal/streaming_write_rpc_logging_test.cc +# internal/streaming_write_rpc_test.cc +# internal/streaming_write_rpc_tracing_test.cc +# internal/time_utils_test.cc +# internal/unified_grpc_credentials_test.cc) + +# # List the unit tests, then setup the targets and dependencies. +# set(google_cloud_cpp_grpc_utils_integration_tests +# # cmake-format: sort +# internal/grpc_impersonate_service_account_integration_test.cc) + +# # Export the list of unit and integration tests so the Bazel BUILD file can +# # pick them up. +# export_list_to_bazel("google_cloud_cpp_grpc_utils_unit_tests.bzl" +# "google_cloud_cpp_grpc_utils_unit_tests" YEAR "2019") +# export_list_to_bazel( +# "google_cloud_cpp_grpc_utils_integration_tests.bzl" +# "google_cloud_cpp_grpc_utils_integration_tests" YEAR "2021") + +# foreach (fname ${google_cloud_cpp_grpc_utils_unit_tests}) +# google_cloud_cpp_grpc_utils_add_test("${fname}" "") +# endforeach () + +# # TODO(#12485) - remove dependency on bigtable in this integration test. +# if (NOT bigtable IN_LIST GOOGLE_CLOUD_CPP_ENABLE) +# list(REMOVE_ITEM google_cloud_cpp_grpc_utils_integration_tests +# "internal/grpc_impersonate_service_account_integration_test.cc") +# endif () + +# foreach (fname ${google_cloud_cpp_grpc_utils_integration_tests}) +# google_cloud_cpp_add_executable(target "common" "${fname}") +# target_link_libraries( +# ${target} +# PRIVATE google-cloud-cpp::grpc_utils +# google_cloud_cpp_testing_grpc +# google_cloud_cpp_testing +# google-cloud-cpp::common +# google-cloud-cpp::iam_credentials_v1_iamcredentials_protos +# absl::variant +# GTest::gmock_main +# GTest::gmock +# GTest::gtest +# gRPC::grpc++ +# gRPC::grpc) +# google_cloud_cpp_add_common_options(${target}) +# add_test(NAME ${target} COMMAND ${target}) +# set_tests_properties(${target} PROPERTIES LABELS +# "integration-test-production") +# # TODO(12485) - remove dep on bigtable_protos +# if (bigtable IN_LIST GOOGLE_CLOUD_CPP_ENABLE) +# target_link_libraries(${target} +# PRIVATE google-cloud-cpp::bigtable_protos) +# endif () +# endforeach () + +# set(google_cloud_cpp_grpc_utils_benchmarks # cmake-format: sortable +# completion_queue_benchmark.cc) + +# # Export the list of benchmarks to a .bzl file so we do not need to maintain +# # the list in two places. +# export_list_to_bazel("google_cloud_cpp_grpc_utils_benchmarks.bzl" +# "google_cloud_cpp_grpc_utils_benchmarks" YEAR "2020") + +# # Generate a target for each benchmark. +# foreach (fname ${google_cloud_cpp_grpc_utils_benchmarks}) +# google_cloud_cpp_add_executable(target "common" "${fname}") +# add_test(NAME ${target} COMMAND ${target}) +# target_link_libraries( +# ${target} +# PRIVATE google-cloud-cpp::grpc_utils google-cloud-cpp::common +# benchmark::benchmark_main) +# google_cloud_cpp_add_common_options(${target}) +# endforeach () +# endif () diff --git a/contrib/google-cloud-cpp-cmake/googleapis/e60db19f11f94175ac682c5898cce0f77cc508ea.tar.gz b/contrib/google-cloud-cpp-cmake/googleapis/e60db19f11f94175ac682c5898cce0f77cc508ea.tar.gz new file mode 100644 index 00000000000..07c1a189d9b Binary files /dev/null and b/contrib/google-cloud-cpp-cmake/googleapis/e60db19f11f94175ac682c5898cce0f77cc508ea.tar.gz differ diff --git a/contrib/jwt-cpp b/contrib/jwt-cpp new file mode 160000 index 00000000000..a6927cb8140 --- /dev/null +++ b/contrib/jwt-cpp @@ -0,0 +1 @@ +Subproject commit a6927cb8140858c34e05d1a954626b9849fbcdfc diff --git a/contrib/jwt-cpp-cmake/CMakeLists.txt b/contrib/jwt-cpp-cmake/CMakeLists.txt new file mode 100644 index 00000000000..4cb8716bc68 --- /dev/null +++ b/contrib/jwt-cpp-cmake/CMakeLists.txt @@ -0,0 +1,23 @@ +set(ENABLE_JWT_CPP_DEFAULT OFF) +if(ENABLE_LIBRARIES AND CLICKHOUSE_CLOUD) + set(ENABLE_JWT_CPP_DEFAULT ON) +endif() + +option(ENABLE_JWT_CPP "Enable jwt-cpp library" ${ENABLE_JWT_CPP_DEFAULT}) + +if (NOT ENABLE_JWT_CPP) + message(STATUS "Not using jwt-cpp") + return() +endif() + +if(ENABLE_JWT_CPP) + if(NOT TARGET OpenSSL::Crypto) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use jwt-cpp without OpenSSL") + endif() +endif() + +set (JWT_CPP_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/jwt-cpp/include") + +add_library (_jwt-cpp INTERFACE) +target_include_directories(_jwt-cpp SYSTEM BEFORE INTERFACE ${JWT_CPP_INCLUDE_DIR}) +add_library(ch_contrib::jwt-cpp ALIAS _jwt-cpp) diff --git a/contrib/update-submodules.sh b/contrib/update-submodules.sh index 072d7a5dc2f..e496f905356 100755 --- a/contrib/update-submodules.sh +++ b/contrib/update-submodules.sh @@ -24,7 +24,7 @@ git config --file .gitmodules --get-regexp '.*path' | sed 's/[^ ]* //' | xargs - # We don't want to depend on any third-party CMake files. # To check it, find and delete them. grep -o -P '"contrib/[^"]+"' .gitmodules | - grep -v -P 'contrib/(llvm-project|google-protobuf|grpc|abseil-cpp|corrosion|aws-crt-cpp)' | + grep -v -P 'contrib/(llvm-project|google-protobuf|grpc|abseil-cpp|corrosion|aws-crt-cpp|google-cloud-cpp)' | xargs -I@ find @ \ -'(' -name 'CMakeLists.txt' -or -name '*.cmake' -')' -and -not -name '*.h.cmake' \ -delete diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 118b688f8e8..0d75c0bd225 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -31,8 +31,8 @@ COPY entrypoint.sh /entrypoint.sh ARG TARGETARCH RUN arch=${TARGETARCH:-amd64} \ && case $arch in \ - amd64) mkdir -p /lib64 && ln -sf /lib/ld-2.31.so /lib64/ld-linux-x86-64.so.2 ;; \ - arm64) ln -sf /lib/ld-2.31.so /lib/ld-linux-aarch64.so.1 ;; \ + amd64) mkdir -p /lib64 && ln -sf /lib/ld-2.35.so /lib64/ld-linux-x86-64.so.2 ;; \ + arm64) ln -sf /lib/ld-2.35.so /lib/ld-linux-aarch64.so.1 ;; \ esac # lts / testing / prestable / etc @@ -86,7 +86,8 @@ RUN arch=${TARGETARCH:-amd64} \ ARG DEFAULT_CONFIG_DIR="/etc/clickhouse-keeper" ARG DEFAULT_DATA_DIR="/var/lib/clickhouse-keeper" ARG DEFAULT_LOG_DIR="/var/log/clickhouse-keeper" -RUN mkdir -p "${DEFAULT_DATA_DIR}" "${DEFAULT_LOG_DIR}" "${DEFAULT_CONFIG_DIR}" \ +RUN clickhouse-keeper --version \ + && mkdir -p "${DEFAULT_DATA_DIR}" "${DEFAULT_LOG_DIR}" "${DEFAULT_CONFIG_DIR}" \ && chown clickhouse:clickhouse "${DEFAULT_DATA_DIR}" \ && chown root:clickhouse "${DEFAULT_LOG_DIR}" \ && chmod ugo+Xrw -R "${DEFAULT_DATA_DIR}" "${DEFAULT_LOG_DIR}" "${DEFAULT_CONFIG_DIR}" diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index fd27d4b6ed9..9868b2a05a8 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -258,7 +258,7 @@ CREATE TABLE table_with_asterisk (name String, value UInt32) - [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. - [s3_create_new_file_on_insert](/docs/en/operations/settings/settings.md#s3_create_new_file_on_insert) - allows to create a new file on each insert if format has suffix. Disabled by default. -- [s3_skip_empty_files](/docs/en/operations/settings/settings.md#s3_skip_empty_files) - allows to skip empty files while reading. Disabled by default. +- [s3_skip_empty_files](/docs/en/operations/settings/settings.md#s3_skip_empty_files) - allows to skip empty files while reading. Enabled by default. ## S3-related Settings {#settings} diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 8cc34a93c4a..9442aad230b 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -3286,3 +3286,17 @@ Use the legacy MongoDB integration implementation. Deprecated. Type: Bool Default value: `true`. + +## allowed_feature_tier + +Controls if the user can change settings related to the different feature tiers. +0 - Changes to any setting are allowed (experimental, beta, production). +1 - Only changes to beta and production feature settings are allowed. Changes to experimental settings are rejected. +2 - Only changes to production settings are allowed. Changes to experimental or beta settings are rejected. + +This is equivalent to setting a readonly constraint on all EXPERIMENTAL / BETA features. +``` + +Type: UInt32 + +Default value: `0` (all settings can be changed). diff --git a/docs/en/sql-reference/table-functions/deltalake.md b/docs/en/sql-reference/table-functions/deltalake.md index 885d8df6a1e..4f8515a539f 100644 --- a/docs/en/sql-reference/table-functions/deltalake.md +++ b/docs/en/sql-reference/table-functions/deltalake.md @@ -49,4 +49,4 @@ LIMIT 2 **See Also** - [DeltaLake engine](/docs/en/engines/table-engines/integrations/deltalake.md) - +- [DeltaLake cluster table function](/docs/en/sql-reference/table-functions/deltalakeCluster.md) diff --git a/docs/en/sql-reference/table-functions/deltalakeCluster.md b/docs/en/sql-reference/table-functions/deltalakeCluster.md new file mode 100644 index 00000000000..49c2264823f --- /dev/null +++ b/docs/en/sql-reference/table-functions/deltalakeCluster.md @@ -0,0 +1,30 @@ +--- +slug: /en/sql-reference/table-functions/deltalakeCluster +sidebar_position: 46 +sidebar_label: deltaLakeCluster +title: "deltaLakeCluster Table Function" +--- +This is an extension to the [deltaLake](/docs/en/sql-reference/table-functions/deltalake.md) table function. + +Allows processing files from [Delta Lake](https://github.com/delta-io/delta) tables in Amazon S3 in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. + +**Syntax** + +``` sql +deltaLakeCluster(cluster_name, url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) +``` + +**Arguments** + +- `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. + +- Description of all other arguments coincides with description of arguments in equivalent [deltaLake](/docs/en/sql-reference/table-functions/deltalake.md) table function. + +**Returned value** + +A table with the specified structure for reading data from cluster in the specified Delta Lake table in S3. + +**See Also** + +- [deltaLake engine](/docs/en/engines/table-engines/integrations/deltalake.md) +- [deltaLake table function](/docs/en/sql-reference/table-functions/deltalake.md) diff --git a/docs/en/sql-reference/table-functions/hudi.md b/docs/en/sql-reference/table-functions/hudi.md index 959a32fe26d..f4cdb0bf948 100644 --- a/docs/en/sql-reference/table-functions/hudi.md +++ b/docs/en/sql-reference/table-functions/hudi.md @@ -29,4 +29,4 @@ A table with the specified structure for reading data in the specified Hudi tabl **See Also** - [Hudi engine](/docs/en/engines/table-engines/integrations/hudi.md) - +- [Hudi cluster table function](/docs/en/sql-reference/table-functions/hudiCluster.md) diff --git a/docs/en/sql-reference/table-functions/hudiCluster.md b/docs/en/sql-reference/table-functions/hudiCluster.md new file mode 100644 index 00000000000..985b7479f66 --- /dev/null +++ b/docs/en/sql-reference/table-functions/hudiCluster.md @@ -0,0 +1,30 @@ +--- +slug: /en/sql-reference/table-functions/hudiCluster +sidebar_position: 86 +sidebar_label: hudiCluster +title: "hudiCluster Table Function" +--- +This is an extension to the [hudi](/docs/en/sql-reference/table-functions/hudi.md) table function. + +Allows processing files from Apache [Hudi](https://hudi.apache.org/) tables in Amazon S3 in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. + +**Syntax** + +``` sql +hudiCluster(cluster_name, url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) +``` + +**Arguments** + +- `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. + +- Description of all other arguments coincides with description of arguments in equivalent [hudi](/docs/en/sql-reference/table-functions/hudi.md) table function. + +**Returned value** + +A table with the specified structure for reading data from cluster in the specified Hudi table in S3. + +**See Also** + +- [Hudi engine](/docs/en/engines/table-engines/integrations/hudi.md) +- [Hudi table function](/docs/en/sql-reference/table-functions/hudi.md) diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index 4f54b2cd440..28063330008 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -72,3 +72,4 @@ Table function `iceberg` is an alias to `icebergS3` now. **See Also** - [Iceberg engine](/docs/en/engines/table-engines/integrations/iceberg.md) +- [Iceberg cluster table function](/docs/en/sql-reference/table-functions/icebergCluster.md) diff --git a/docs/en/sql-reference/table-functions/icebergCluster.md b/docs/en/sql-reference/table-functions/icebergCluster.md new file mode 100644 index 00000000000..bc444f361d5 --- /dev/null +++ b/docs/en/sql-reference/table-functions/icebergCluster.md @@ -0,0 +1,43 @@ +--- +slug: /en/sql-reference/table-functions/icebergCluster +sidebar_position: 91 +sidebar_label: icebergCluster +title: "icebergCluster Table Function" +--- +This is an extension to the [iceberg](/docs/en/sql-reference/table-functions/iceberg.md) table function. + +Allows processing files from Apache [Iceberg](https://iceberg.apache.org/) in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. + +**Syntax** + +``` sql +icebergS3Cluster(cluster_name, url [, NOSIGN | access_key_id, secret_access_key, [session_token]] [,format] [,compression_method]) +icebergS3Cluster(cluster_name, named_collection[, option=value [,..]]) + +icebergAzureCluster(cluster_name, connection_string|storage_account_url, container_name, blobpath, [,account_name], [,account_key] [,format] [,compression_method]) +icebergAzureCluster(cluster_name, named_collection[, option=value [,..]]) + +icebergHDFSCluster(cluster_name, path_to_table, [,format] [,compression_method]) +icebergHDFSCluster(cluster_name, named_collection[, option=value [,..]]) +``` + +**Arguments** + +- `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. + +- Description of all other arguments coincides with description of arguments in equivalent [iceberg](/docs/en/sql-reference/table-functions/iceberg.md) table function. + +**Returned value** + +A table with the specified structure for reading data from cluster in the specified Iceberg table. + +**Examples** + +```sql +SELECT * FROM icebergS3Cluster('cluster_simple', 'http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') +``` + +**See Also** + +- [Iceberg engine](/docs/en/engines/table-engines/integrations/iceberg.md) +- [Iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index b14eb84392f..ea7820c1aec 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -317,7 +317,7 @@ SELECT * from s3('s3://data/path/date=*/country=*/code=*/*.parquet') where _date - [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3_truncate_on_insert) - allows to truncate file before insert into it. Disabled by default. - [s3_create_new_file_on_insert](/docs/en/operations/settings/settings.md#s3_create_new_file_on_insert) - allows to create a new file on each insert if format has suffix. Disabled by default. -- [s3_skip_empty_files](/docs/en/operations/settings/settings.md#s3_skip_empty_files) - allows to skip empty files while reading. Disabled by default. +- [s3_skip_empty_files](/docs/en/operations/settings/settings.md#s3_skip_empty_files) - allows to skip empty files while reading. Enabled by default. **See Also** diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e6f8ecef097..3ecc6ecf24d 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -79,6 +79,7 @@ namespace Setting namespace ServerSetting { + extern const ServerSettingsUInt32 allowed_feature_tier; extern const ServerSettingsDouble cache_size_to_ram_max_ratio; extern const ServerSettingsUInt64 compiled_expression_cache_elements_size; extern const ServerSettingsUInt64 compiled_expression_cache_size; @@ -789,6 +790,9 @@ void LocalServer::processConfig() /// Initialize a dummy query cache. global_context->setQueryCache(0, 0, 0, 0); + /// Initialize allowed tiers + global_context->getAccessControl().setAllowTierSettings(server_settings[ServerSetting::allowed_feature_tier]); + #if USE_EMBEDDED_COMPILER size_t compiled_expression_cache_max_size_in_bytes = server_settings[ServerSetting::compiled_expression_cache_size]; size_t compiled_expression_cache_max_elements = server_settings[ServerSetting::compiled_expression_cache_elements_size]; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 68f262079ff..af383334128 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -166,6 +166,7 @@ namespace MergeTreeSetting namespace ServerSetting { + extern const ServerSettingsUInt32 allowed_feature_tier; extern const ServerSettingsUInt32 asynchronous_heavy_metrics_update_period_s; extern const ServerSettingsUInt32 asynchronous_metrics_update_period_s; extern const ServerSettingsBool asynchronous_metrics_enable_heavy_metrics; @@ -1771,6 +1772,7 @@ try global_context->setMaxDictionaryNumToWarn(new_server_settings[ServerSetting::max_dictionary_num_to_warn]); global_context->setMaxDatabaseNumToWarn(new_server_settings[ServerSetting::max_database_num_to_warn]); global_context->setMaxPartNumToWarn(new_server_settings[ServerSetting::max_part_num_to_warn]); + global_context->getAccessControl().setAllowTierSettings(new_server_settings[ServerSetting::allowed_feature_tier]); /// Only for system.server_settings global_context->setConfigReloaderInterval(new_server_settings[ServerSetting::config_reload_interval_ms]); @@ -2161,9 +2163,12 @@ try /// Check sanity of MergeTreeSettings on server startup { + /// All settings can be changed in the global config + bool allowed_experimental = true; + bool allowed_beta = true; size_t background_pool_tasks = global_context->getMergeMutateExecutor()->getMaxTasksCount(); - global_context->getMergeTreeSettings().sanityCheck(background_pool_tasks); - global_context->getReplicatedMergeTreeSettings().sanityCheck(background_pool_tasks); + global_context->getMergeTreeSettings().sanityCheck(background_pool_tasks, allowed_experimental, allowed_beta); + global_context->getReplicatedMergeTreeSettings().sanityCheck(background_pool_tasks, allowed_experimental, allowed_beta); } /// try set up encryption. There are some errors in config, error will be printed and server wouldn't start. CompressionCodecEncrypted::Configuration::instance().load(config(), "encryption_codecs"); diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 9b3b8d2a977..1583ccecf94 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -876,4 +876,28 @@ void AccessControl::allowAllSettings() custom_settings_prefixes->registerPrefixes({""}); } +void AccessControl::setAllowTierSettings(UInt32 value) +{ + allow_experimental_tier_settings = value == 0; + allow_beta_tier_settings = value <= 1; +} + +UInt32 AccessControl::getAllowTierSettings() const +{ + if (allow_experimental_tier_settings) + return 0; + if (allow_beta_tier_settings) + return 1; + return 2; +} + +bool AccessControl::getAllowExperimentalTierSettings() const +{ + return allow_experimental_tier_settings; +} + +bool AccessControl::getAllowBetaTierSettings() const +{ + return allow_beta_tier_settings; +} } diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index a342c5300bf..1fbe7a3fccd 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -243,6 +243,11 @@ public: /// Allow all setting names - this can be used in clients to pass-through unknown settings to the server. void allowAllSettings(); + void setAllowTierSettings(UInt32 value); + UInt32 getAllowTierSettings() const; + bool getAllowExperimentalTierSettings() const; + bool getAllowBetaTierSettings() const; + private: class ContextAccessCache; class CustomSettingsPrefixes; @@ -272,6 +277,8 @@ private: std::atomic_bool table_engines_require_grant = false; std::atomic_int bcrypt_workfactor = 12; std::atomic default_password_type = AuthenticationType::SHA256_PASSWORD; + std::atomic_bool allow_experimental_tier_settings = true; + std::atomic_bool allow_beta_tier_settings = true; }; } diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 3a78420f411..1246992f9b6 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -9,6 +9,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int INVALID_GRANT; +} + namespace { void formatOptions(bool grant_option, bool is_partial_revoke, String & result) @@ -211,18 +217,43 @@ AccessRightsElement::AccessRightsElement( { } -void AccessRightsElement::eraseNonGrantable() +AccessFlags AccessRightsElement::getGrantableFlags() const { if (isGlobalWithParameter() && !anyParameter()) - access_flags &= AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel(); + return access_flags & AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel(); else if (!anyColumn()) - access_flags &= AccessFlags::allFlagsGrantableOnColumnLevel(); + return access_flags & AccessFlags::allFlagsGrantableOnColumnLevel(); else if (!anyTable()) - access_flags &= AccessFlags::allFlagsGrantableOnTableLevel(); + return access_flags & AccessFlags::allFlagsGrantableOnTableLevel(); else if (!anyDatabase()) - access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel(); + return access_flags & AccessFlags::allFlagsGrantableOnDatabaseLevel(); else - access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel(); + return access_flags & AccessFlags::allFlagsGrantableOnGlobalLevel(); +} + +void AccessRightsElement::throwIfNotGrantable() const +{ + if (empty()) + return; + auto grantable_flags = getGrantableFlags(); + if (grantable_flags) + return; + + if (!anyColumn()) + throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the column level", access_flags.toString()); + if (!anyTable()) + throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the table level", access_flags.toString()); + if (!anyDatabase()) + throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the database level", access_flags.toString()); + if (!anyParameter()) + throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the global with parameter level", access_flags.toString()); + + throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted", access_flags.toString()); +} + +void AccessRightsElement::eraseNotGrantable() +{ + access_flags = getGrantableFlags(); } void AccessRightsElement::replaceEmptyDatabase(const String & current_database) @@ -251,11 +282,17 @@ bool AccessRightsElements::sameOptions() const return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameOptions(front()); }); } -void AccessRightsElements::eraseNonGrantable() +void AccessRightsElements::throwIfNotGrantable() const +{ + for (const auto & element : *this) + element.throwIfNotGrantable(); +} + +void AccessRightsElements::eraseNotGrantable() { std::erase_if(*this, [](AccessRightsElement & element) { - element.eraseNonGrantable(); + element.eraseNotGrantable(); return element.empty(); }); } @@ -269,4 +306,45 @@ void AccessRightsElements::replaceEmptyDatabase(const String & current_database) String AccessRightsElements::toString() const { return toStringImpl(*this, true); } String AccessRightsElements::toStringWithoutOptions() const { return toStringImpl(*this, false); } +void AccessRightsElements::formatElementsWithoutOptions(WriteBuffer & buffer, bool hilite) const +{ + bool no_output = true; + for (size_t i = 0; i != size(); ++i) + { + const auto & element = (*this)[i]; + auto keywords = element.access_flags.toKeywords(); + if (keywords.empty() || (!element.anyColumn() && element.columns.empty())) + continue; + + for (const auto & keyword : keywords) + { + if (!std::exchange(no_output, false)) + buffer << ", "; + + buffer << (hilite ? IAST::hilite_keyword : "") << keyword << (hilite ? IAST::hilite_none : ""); + if (!element.anyColumn()) + element.formatColumnNames(buffer); + } + + bool next_element_on_same_db_and_table = false; + if (i != size() - 1) + { + const auto & next_element = (*this)[i + 1]; + if (element.sameDatabaseAndTableAndParameter(next_element)) + { + next_element_on_same_db_and_table = true; + } + } + + if (!next_element_on_same_db_and_table) + { + buffer << " "; + element.formatONClause(buffer, hilite); + } + } + + if (no_output) + buffer << (hilite ? IAST::hilite_keyword : "") << "USAGE ON " << (hilite ? IAST::hilite_none : "") << "*.*"; +} + } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 4749db09c56..85997b48a38 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -79,8 +79,14 @@ struct AccessRightsElement return (grant_option == other.grant_option) && (is_partial_revoke == other.is_partial_revoke); } + /// Returns only those flags which can be granted. + AccessFlags getGrantableFlags() const; + + /// Throws an exception if some flags can't be granted. + void throwIfNotGrantable() const; + /// Resets flags which cannot be granted. - void eraseNonGrantable(); + void eraseNotGrantable(); bool isEmptyDatabase() const { return database.empty() and !anyDatabase(); } @@ -110,8 +116,11 @@ public: bool sameDatabaseAndTable() const; bool sameOptions() const; + /// Throws an exception if some flags can't be granted. + void throwIfNotGrantable() const; + /// Resets flags which cannot be granted. - void eraseNonGrantable(); + void eraseNotGrantable(); /// If the database is empty, replaces it with `current_database`. Otherwise does nothing. void replaceEmptyDatabase(const String & current_database); @@ -119,6 +128,7 @@ public: /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; String toStringWithoutOptions() const; + void formatElementsWithoutOptions(WriteBuffer & buffer, bool hilite) const; }; } diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index cdf3dac192e..cb1d433766a 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -397,13 +397,35 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu /** The `readonly` value is understood as follows: * 0 - no read-only restrictions. - * 1 - only read requests, as well as changing settings with `changable_in_readonly` flag. + * 1 - only read requests, as well as changing settings with `changeable_in_readonly` flag. * 2 - only read requests, as well as changing settings, except for the `readonly` setting. */ if (current_settings[Setting::readonly] > 1 && resolved_name == "readonly") return Checker(PreformattedMessage::create("Cannot modify 'readonly' setting in readonly mode"), ErrorCodes::READONLY); + if (access_control) + { + bool allowed_experimental = access_control->getAllowExperimentalTierSettings(); + bool allowed_beta = access_control->getAllowBetaTierSettings(); + if (!allowed_experimental || !allowed_beta) + { + auto setting_tier = current_settings.getTier(resolved_name); + if (setting_tier == SettingsTierType::EXPERIMENTAL && !allowed_experimental) + return Checker( + PreformattedMessage::create( + "Cannot modify setting '{}'. Changes to EXPERIMENTAL settings are disabled in the server config ('allowed_feature_tier')", + setting_name), + ErrorCodes::READONLY); + if (setting_tier == SettingsTierType::BETA && !allowed_beta) + return Checker( + PreformattedMessage::create( + "Cannot modify setting '{}'. Changes to BETA settings are disabled in the server config ('allowed_feature_tier')", + setting_name), + ErrorCodes::READONLY); + } + } + auto it = constraints.find(resolved_name); if (current_settings[Setting::readonly] == 1) { diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 5bbff86ff61..f5e4335252c 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -40,7 +40,7 @@ class AccessControl; * * * - * + * * * * @@ -50,7 +50,7 @@ class AccessControl; * If a setting cannot be change due to the read-only mode this class throws an exception. * The value of `readonly` is understood as follows: * 0 - not read-only mode, no additional checks. - * 1 - only read queries, as well as changing settings with flag. + * 1 - only read queries, as well as changing settings with flag. * 2 - only read queries and you can change the settings, except for the `readonly` setting. * */ diff --git a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp index 9608ca26f37..cc72a26af16 100644 --- a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp @@ -79,6 +79,14 @@ public: "Illegal type {} of second argument of aggregate function {} because the values of that data type are not comparable", type_val->getName(), getName()); + + if (isDynamic(this->type_val) || isVariant(this->type_val)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of aggregate function {} because the column of that type can contain values with different " + "data types. Consider using typed subcolumns or cast column to a specific data type", + this->type_val->getName(), + getName()); } void create(AggregateDataPtr __restrict place) const override /// NOLINT diff --git a/src/AggregateFunctions/AggregateFunctionsMinMax.cpp b/src/AggregateFunctions/AggregateFunctionsMinMax.cpp index 5fa9a4ff5d1..0c21be574c4 100644 --- a/src/AggregateFunctions/AggregateFunctionsMinMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsMinMax.cpp @@ -35,6 +35,14 @@ public: "Illegal type {} of argument of aggregate function {} because the values of that data type are not comparable", this->result_type->getName(), getName()); + + if (isDynamic(this->result_type) || isVariant(this->result_type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of aggregate function {} because the column of that type can contain values with different " + "data types. Consider using typed subcolumns or cast column to a specific data type", + this->result_type->getName(), + getName()); } String getName() const override diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionCombinatorsArgMinArgMax.cpp b/src/AggregateFunctions/Combinators/AggregateFunctionCombinatorsArgMinArgMax.cpp index a1716f18725..fc8b5b6d4cd 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionCombinatorsArgMinArgMax.cpp +++ b/src/AggregateFunctions/Combinators/AggregateFunctionCombinatorsArgMinArgMax.cpp @@ -63,6 +63,14 @@ public: "Illegal type {} for combinator {} because the values of that data type are not comparable", arguments[key_col]->getName(), getName()); + + if (isDynamic(arguments[key_col]) || isVariant(arguments[key_col])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of aggregate function {} because the column of that type can contain values with different " + "data types. Consider using typed subcolumns or cast column to a specific data type", + arguments[key_col]->getName(), + getName()); } String getName() const override diff --git a/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp b/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp index fc122730b37..d7ccc53041b 100644 --- a/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp +++ b/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp @@ -4,6 +4,7 @@ #include #include +#include #include diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 03ebd893c47..d118cb281ae 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -13,6 +13,7 @@ #include #include +#include #include #include #include diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3627d760d4c..55228b2d1ec 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -639,6 +639,10 @@ if (TARGET ch_rust::skim) dbms_target_link_libraries(PUBLIC ch_rust::skim) endif() +if (TARGET ch_contrib::google_cloud_cpp) + dbms_target_link_libraries(PUBLIC ch_contrib::google_cloud_cpp) +endif() + if (ENABLE_TESTS) macro (grep_gtest_sources BASE_DIR DST_VAR) # Cold match files that are not in tests/ directories diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index bb4433f8956..73366150e7d 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -8,6 +8,10 @@ #include #include +#include +#include + +#include #include #include @@ -30,6 +34,19 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +template +const char * ColumnDecimal::getFamilyName() const +{ + return TypeName.data(); +} + +template +TypeIndex ColumnDecimal::getDataType() const +{ + return TypeToTypeIndex; +} + + template #if !defined(DEBUG_OR_SANITIZER_BUILD) int ColumnDecimal::compareAt(size_t n, size_t m, const IColumn & rhs_, int) const @@ -46,6 +63,12 @@ int ColumnDecimal::doCompareAt(size_t n, size_t m, const IColumn & rhs_, int) return decimalLess(b, a, other.scale, scale) ? 1 : (decimalLess(a, b, scale, other.scale) ? -1 : 0); } +template +Float64 ColumnDecimal::getFloat64(size_t n) const +{ + return DecimalUtils::convertTo(data[n], scale); +} + template const char * ColumnDecimal::deserializeAndInsertFromArena(const char * pos) { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 6f8360a54dd..690549e4a56 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -1,14 +1,9 @@ #pragma once -#include -#include -#include -#include -#include -#include #include #include #include +#include namespace DB @@ -39,8 +34,8 @@ private: {} public: - const char * getFamilyName() const override { return TypeName.data(); } - TypeIndex getDataType() const override { return TypeToTypeIndex; } + const char * getFamilyName() const override; + TypeIndex getDataType() const override; bool isNumeric() const override { return false; } bool canBeInsideNullable() const override { return true; } @@ -98,7 +93,7 @@ public: return StringRef(reinterpret_cast(&data[n]), sizeof(data[n])); } - Float64 getFloat64(size_t n) const final { return DecimalUtils::convertTo(data[n], scale); } + Float64 getFloat64(size_t n) const final; const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; diff --git a/src/Columns/ColumnFunction.cpp b/src/Columns/ColumnFunction.cpp index cc80d04444e..5e41e95fdc5 100644 --- a/src/Columns/ColumnFunction.cpp +++ b/src/Columns/ColumnFunction.cpp @@ -347,7 +347,7 @@ ColumnWithTypeAndName ColumnFunction::reduce() const if (is_function_compiled) ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute); - res.column = function->execute(columns, res.type, elements_size); + res.column = function->execute(columns, res.type, elements_size, /* dry_run = */ false); if (res.column->getDataType() != res.type->getColumnType()) throw Exception( ErrorCodes::LOGICAL_ERROR, diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 84fc6ebc61d..3c7727f37c4 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -32,6 +32,8 @@ # include #endif +#include "config.h" + #if USE_MULTITARGET_CODE # include #endif @@ -658,7 +660,7 @@ inline void doFilterAligned(const UInt8 *& filt_pos, const UInt8 *& filt_end_ali reinterpret_cast(&res_data[current_offset]), mask & KMASK); current_offset += std::popcount(mask & KMASK); /// prepare mask for next iter, if ELEMENTS_PER_VEC = 64, no next iter - if (ELEMENTS_PER_VEC < 64) + if constexpr (ELEMENTS_PER_VEC < 64) { mask >>= ELEMENTS_PER_VEC; } @@ -992,6 +994,151 @@ ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, c return res; } +DECLARE_DEFAULT_CODE( + template void vectorIndexImpl( + const Container & data, const PaddedPODArray & indexes, size_t limit, Container & res_data) + { + for (size_t i = 0; i < limit; ++i) + res_data[i] = data[indexes[i]]; + } +); + +DECLARE_AVX512VBMI_SPECIFIC_CODE( + template + void vectorIndexImpl(const Container & data, const PaddedPODArray & indexes, size_t limit, Container & res_data) + { + static constexpr UInt64 MASK64 = 0xffffffffffffffff; + const size_t limit64 = limit & ~63; + size_t pos = 0; + size_t data_size = data.size(); + + auto data_pos = reinterpret_cast(data.data()); + auto indexes_pos = reinterpret_cast(indexes.data()); + auto res_pos = reinterpret_cast(res_data.data()); + + if (limit == 0) + return; /// nothing to do, just return + + if (data_size <= 64) + { + /// one single mask load for table size <= 64 + __mmask64 last_mask = MASK64 >> (64 - data_size); + __m512i table1 = _mm512_maskz_loadu_epi8(last_mask, data_pos); + + /// 64 bytes table lookup using one single permutexvar_epi8 + while (pos < limit64) + { + __m512i vidx = _mm512_loadu_epi8(indexes_pos + pos); + __m512i out = _mm512_permutexvar_epi8(vidx, table1); + _mm512_storeu_epi8(res_pos + pos, out); + pos += 64; + } + /// tail handling + if (limit > limit64) + { + __mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit); + __m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos); + __m512i out = _mm512_permutexvar_epi8(vidx, table1); + _mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out); + } + } + else if (data_size <= 128) + { + /// table size (64, 128] requires 2 zmm load + __mmask64 last_mask = MASK64 >> (128 - data_size); + __m512i table1 = _mm512_loadu_epi8(data_pos); + __m512i table2 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 64); + + /// 128 bytes table lookup using one single permute2xvar_epi8 + while (pos < limit64) + { + __m512i vidx = _mm512_loadu_epi8(indexes_pos + pos); + __m512i out = _mm512_permutex2var_epi8(table1, vidx, table2); + _mm512_storeu_epi8(res_pos + pos, out); + pos += 64; + } + if (limit > limit64) + { + __mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit); + __m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos); + __m512i out = _mm512_permutex2var_epi8(table1, vidx, table2); + _mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out); + } + } + else + { + if (data_size > 256) + { + /// byte index will not exceed 256 boundary. + data_size = 256; + } + + __m512i table1 = _mm512_loadu_epi8(data_pos); + __m512i table2 = _mm512_loadu_epi8(data_pos + 64); + __m512i table3, table4; + if (data_size <= 192) + { + /// only 3 tables need to load if size <= 192 + __mmask64 last_mask = MASK64 >> (192 - data_size); + table3 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 128); + table4 = _mm512_setzero_si512(); + } + else + { + __mmask64 last_mask = MASK64 >> (256 - data_size); + table3 = _mm512_loadu_epi8(data_pos + 128); + table4 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 192); + } + + /// 256 bytes table lookup can use: 2 permute2xvar_epi8 plus 1 blender with MSB + while (pos < limit64) + { + __m512i vidx = _mm512_loadu_epi8(indexes_pos + pos); + __m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2); + __m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4); + __mmask64 msb = _mm512_movepi8_mask(vidx); + __m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2); + _mm512_storeu_epi8(res_pos + pos, out); + pos += 64; + } + if (limit > limit64) + { + __mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit); + __m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos); + __m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2); + __m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4); + __mmask64 msb = _mm512_movepi8_mask(vidx); + __m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2); + _mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out); + } + } + } +); + +template +template +ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + chassert(limit <= indexes.size()); + + auto res = this->create(limit); + typename Self::Container & res_data = res->getData(); +#if USE_MULTITARGET_CODE + if constexpr (sizeof(T) == 1 && sizeof(Type) == 1) + { + /// VBMI optimization only applicable for (U)Int8 types + if (isArchSupported(TargetArch::AVX512VBMI)) + { + TargetSpecific::AVX512VBMI::vectorIndexImpl(data, indexes, limit, res_data); + return res; + } + } +#endif + TargetSpecific::Default::vectorIndexImpl(data, indexes, limit, res_data); + + return res; +} + /// Explicit template instantiations - to avoid code bloat in headers. template class ColumnVector; template class ColumnVector; @@ -1012,4 +1159,17 @@ template class ColumnVector; template class ColumnVector; template class ColumnVector; +INSTANTIATE_INDEX_TEMPLATE_IMPL(ColumnVector) +/// Used by ColumnVariant.cpp +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; + +#if defined(OS_DARWIN) +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const; +#endif } diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index e8bb6ad6798..1387cca1ece 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -13,10 +13,6 @@ #include "config.h" -#if USE_MULTITARGET_CODE -# include -#endif - namespace DB { @@ -320,151 +316,6 @@ protected: Container data; }; -DECLARE_DEFAULT_CODE( -template -inline void vectorIndexImpl(const Container & data, const PaddedPODArray & indexes, size_t limit, Container & res_data) -{ - for (size_t i = 0; i < limit; ++i) - res_data[i] = data[indexes[i]]; -} -); - -DECLARE_AVX512VBMI_SPECIFIC_CODE( -template -inline void vectorIndexImpl(const Container & data, const PaddedPODArray & indexes, size_t limit, Container & res_data) -{ - static constexpr UInt64 MASK64 = 0xffffffffffffffff; - const size_t limit64 = limit & ~63; - size_t pos = 0; - size_t data_size = data.size(); - - auto data_pos = reinterpret_cast(data.data()); - auto indexes_pos = reinterpret_cast(indexes.data()); - auto res_pos = reinterpret_cast(res_data.data()); - - if (limit == 0) - return; /// nothing to do, just return - - if (data_size <= 64) - { - /// one single mask load for table size <= 64 - __mmask64 last_mask = MASK64 >> (64 - data_size); - __m512i table1 = _mm512_maskz_loadu_epi8(last_mask, data_pos); - - /// 64 bytes table lookup using one single permutexvar_epi8 - while (pos < limit64) - { - __m512i vidx = _mm512_loadu_epi8(indexes_pos + pos); - __m512i out = _mm512_permutexvar_epi8(vidx, table1); - _mm512_storeu_epi8(res_pos + pos, out); - pos += 64; - } - /// tail handling - if (limit > limit64) - { - __mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit); - __m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos); - __m512i out = _mm512_permutexvar_epi8(vidx, table1); - _mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out); - } - } - else if (data_size <= 128) - { - /// table size (64, 128] requires 2 zmm load - __mmask64 last_mask = MASK64 >> (128 - data_size); - __m512i table1 = _mm512_loadu_epi8(data_pos); - __m512i table2 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 64); - - /// 128 bytes table lookup using one single permute2xvar_epi8 - while (pos < limit64) - { - __m512i vidx = _mm512_loadu_epi8(indexes_pos + pos); - __m512i out = _mm512_permutex2var_epi8(table1, vidx, table2); - _mm512_storeu_epi8(res_pos + pos, out); - pos += 64; - } - if (limit > limit64) - { - __mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit); - __m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos); - __m512i out = _mm512_permutex2var_epi8(table1, vidx, table2); - _mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out); - } - } - else - { - if (data_size > 256) - { - /// byte index will not exceed 256 boundary. - data_size = 256; - } - - __m512i table1 = _mm512_loadu_epi8(data_pos); - __m512i table2 = _mm512_loadu_epi8(data_pos + 64); - __m512i table3, table4; - if (data_size <= 192) - { - /// only 3 tables need to load if size <= 192 - __mmask64 last_mask = MASK64 >> (192 - data_size); - table3 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 128); - table4 = _mm512_setzero_si512(); - } - else - { - __mmask64 last_mask = MASK64 >> (256 - data_size); - table3 = _mm512_loadu_epi8(data_pos + 128); - table4 = _mm512_maskz_loadu_epi8(last_mask, data_pos + 192); - } - - /// 256 bytes table lookup can use: 2 permute2xvar_epi8 plus 1 blender with MSB - while (pos < limit64) - { - __m512i vidx = _mm512_loadu_epi8(indexes_pos + pos); - __m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2); - __m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4); - __mmask64 msb = _mm512_movepi8_mask(vidx); - __m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2); - _mm512_storeu_epi8(res_pos + pos, out); - pos += 64; - } - if (limit > limit64) - { - __mmask64 tail_mask = MASK64 >> (limit64 + 64 - limit); - __m512i vidx = _mm512_maskz_loadu_epi8(tail_mask, indexes_pos + pos); - __m512i tmp1 = _mm512_permutex2var_epi8(table1, vidx, table2); - __m512i tmp2 = _mm512_permutex2var_epi8(table3, vidx, table4); - __mmask64 msb = _mm512_movepi8_mask(vidx); - __m512i out = _mm512_mask_blend_epi8(msb, tmp1, tmp2); - _mm512_mask_storeu_epi8(res_pos + pos, tail_mask, out); - } - } -} -); - -template -template -ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const -{ - assert(limit <= indexes.size()); - - auto res = this->create(limit); - typename Self::Container & res_data = res->getData(); -#if USE_MULTITARGET_CODE - if constexpr (sizeof(T) == 1 && sizeof(Type) == 1) - { - /// VBMI optimization only applicable for (U)Int8 types - if (isArchSupported(TargetArch::AVX512VBMI)) - { - TargetSpecific::AVX512VBMI::vectorIndexImpl(data, indexes, limit, res_data); - return res; - } - } -#endif - TargetSpecific::Default::vectorIndexImpl(data, indexes, limit, res_data); - - return res; -} - template concept is_col_vector = std::is_same_v>; diff --git a/src/Columns/ColumnsCommon.h b/src/Columns/ColumnsCommon.h index 99f1d2da47e..f0d6cff2e35 100644 --- a/src/Columns/ColumnsCommon.h +++ b/src/Columns/ColumnsCommon.h @@ -142,4 +142,10 @@ ColumnPtr permuteImpl(const Column & column, const IColumn::Permutation & perm, template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; + +#define INSTANTIATE_INDEX_TEMPLATE_IMPL(ColumnTemplate) \ + template ColumnPtr ColumnTemplate::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ + template ColumnPtr ColumnTemplate::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ + template ColumnPtr ColumnTemplate::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ + template ColumnPtr ColumnTemplate::indexImpl(const PaddedPODArray & indexes, size_t limit) const; } diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 949b884636f..201b586f067 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -131,6 +131,7 @@ public: const char * getTypeName(std::string_view name) const; const char * getDescription(std::string_view name) const; + SettingsTierType getTier(std::string_view name) const; /// Checks if it's possible to assign a field to a specified value and throws an exception if not. /// This function doesn't change the fields, it performs check only. @@ -380,6 +381,18 @@ const char * BaseSettings::getDescription(std::string_view name) const BaseSettingsHelpers::throwSettingNotFound(name); } +template +SettingsTierType BaseSettings::getTier(std::string_view name) const +{ + name = TTraits::resolveName(name); + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.getTier(index); + if (tryGetCustomSetting(name)) + return SettingsTierType::PRODUCTION; + BaseSettingsHelpers::throwSettingNotFound(name); +} + template void BaseSettings::checkCanSet(std::string_view name, const Field & value) { diff --git a/src/Core/DecimalComparison.h b/src/Core/DecimalComparison.h index 77402adf164..4b6783265d0 100644 --- a/src/Core/DecimalComparison.h +++ b/src/Core/DecimalComparison.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -52,8 +53,8 @@ struct DecCompareInt using TypeB = Type; }; -template typename Operation, bool _check_overflow = true, - bool _actual = is_decimal || is_decimal> +template typename Operation> +requires is_decimal || is_decimal class DecimalComparison { public: @@ -65,20 +66,17 @@ public: using ArrayA = typename ColVecA::Container; using ArrayB = typename ColVecB::Container; - static ColumnPtr apply(const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right) + static ColumnPtr apply(const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right, bool check_overflow) { - if constexpr (_actual) - { - ColumnPtr c_res; - Shift shift = getScales(col_left.type, col_right.type); + ColumnPtr c_res; + Shift shift = getScales(col_left.type, col_right.type); - return applyWithScale(col_left.column, col_right.column, shift); - } - else - return nullptr; + if (check_overflow) + return applyWithScale(col_left.column, col_right.column, shift); + return applyWithScale(col_left.column, col_right.column, shift); } - static bool compare(A a, B b, UInt32 scale_a, UInt32 scale_b) + static bool compare(A a, B b, UInt32 scale_a, UInt32 scale_b, bool check_overflow) { static const UInt32 max_scale = DecimalUtils::max_precision; if (scale_a > max_scale || scale_b > max_scale) @@ -90,7 +88,9 @@ public: if (scale_a > scale_b) shift.b = static_cast(DecimalUtils::scaleMultiplier(scale_a - scale_b)); - return applyWithScale(a, b, shift); + if (check_overflow) + return applyWithScale(a, b, shift); + return applyWithScale(a, b, shift); } private: @@ -104,14 +104,14 @@ private: bool right() const { return b != 1; } }; - template + template static auto applyWithScale(T a, U b, const Shift & shift) { if (shift.left()) - return apply(a, b, shift.a); + return apply(a, b, shift.a); if (shift.right()) - return apply(a, b, shift.b); - return apply(a, b, 1); + return apply(a, b, shift.b); + return apply(a, b, 1); } template @@ -125,8 +125,8 @@ private: if (decimal0 && decimal1) { auto result_type = DecimalUtils::binaryOpResult(*decimal0, *decimal1); - shift.a = static_cast(result_type.scaleFactorFor(decimal0->getTrait(), false).value); - shift.b = static_cast(result_type.scaleFactorFor(decimal1->getTrait(), false).value); + shift.a = static_cast(result_type.scaleFactorFor(DecimalUtils::DataTypeDecimalTrait{decimal0->getPrecision(), decimal0->getScale()}, false).value); + shift.b = static_cast(result_type.scaleFactorFor(DecimalUtils::DataTypeDecimalTrait{decimal1->getPrecision(), decimal1->getScale()}, false).value); } else if (decimal0) shift.b = static_cast(decimal0->getScaleMultiplier().value); @@ -158,66 +158,63 @@ private: return shift; } - template + template static ColumnPtr apply(const ColumnPtr & c0, const ColumnPtr & c1, CompareInt scale) { auto c_res = ColumnUInt8::create(); - if constexpr (_actual) + bool c0_is_const = isColumnConst(*c0); + bool c1_is_const = isColumnConst(*c1); + + if (c0_is_const && c1_is_const) { - bool c0_is_const = isColumnConst(*c0); - bool c1_is_const = isColumnConst(*c1); + const ColumnConst & c0_const = checkAndGetColumnConst(*c0); + const ColumnConst & c1_const = checkAndGetColumnConst(*c1); - if (c0_is_const && c1_is_const) + A a = c0_const.template getValue(); + B b = c1_const.template getValue(); + UInt8 res = apply(a, b, scale); + return DataTypeUInt8().createColumnConst(c0->size(), toField(res)); + } + + ColumnUInt8::Container & vec_res = c_res->getData(); + vec_res.resize(c0->size()); + + if (c0_is_const) + { + const ColumnConst & c0_const = checkAndGetColumnConst(*c0); + A a = c0_const.template getValue(); + if (const ColVecB * c1_vec = checkAndGetColumn(c1.get())) + constantVector(a, c1_vec->getData(), vec_res, scale); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong column in Decimal comparison"); + } + else if (c1_is_const) + { + const ColumnConst & c1_const = checkAndGetColumnConst(*c1); + B b = c1_const.template getValue(); + if (const ColVecA * c0_vec = checkAndGetColumn(c0.get())) + vectorConstant(c0_vec->getData(), b, vec_res, scale); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong column in Decimal comparison"); + } + else + { + if (const ColVecA * c0_vec = checkAndGetColumn(c0.get())) { - const ColumnConst & c0_const = checkAndGetColumnConst(*c0); - const ColumnConst & c1_const = checkAndGetColumnConst(*c1); - - A a = c0_const.template getValue(); - B b = c1_const.template getValue(); - UInt8 res = apply(a, b, scale); - return DataTypeUInt8().createColumnConst(c0->size(), toField(res)); - } - - ColumnUInt8::Container & vec_res = c_res->getData(); - vec_res.resize(c0->size()); - - if (c0_is_const) - { - const ColumnConst & c0_const = checkAndGetColumnConst(*c0); - A a = c0_const.template getValue(); if (const ColVecB * c1_vec = checkAndGetColumn(c1.get())) - constantVector(a, c1_vec->getData(), vec_res, scale); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong column in Decimal comparison"); - } - else if (c1_is_const) - { - const ColumnConst & c1_const = checkAndGetColumnConst(*c1); - B b = c1_const.template getValue(); - if (const ColVecA * c0_vec = checkAndGetColumn(c0.get())) - vectorConstant(c0_vec->getData(), b, vec_res, scale); + vectorVector(c0_vec->getData(), c1_vec->getData(), vec_res, scale); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong column in Decimal comparison"); } else - { - if (const ColVecA * c0_vec = checkAndGetColumn(c0.get())) - { - if (const ColVecB * c1_vec = checkAndGetColumn(c1.get())) - vectorVector(c0_vec->getData(), c1_vec->getData(), vec_res, scale); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong column in Decimal comparison"); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong column in Decimal comparison"); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong column in Decimal comparison"); } return c_res; } - template + template static NO_INLINE UInt8 apply(A a, B b, CompareInt scale [[maybe_unused]]) { CompareInt x; @@ -232,7 +229,7 @@ private: else y = static_cast(b); - if constexpr (_check_overflow) + if constexpr (check_overflow) { bool overflow = false; @@ -264,9 +261,8 @@ private: return Op::apply(x, y); } - template - static void NO_INLINE vectorVector(const ArrayA & a, const ArrayB & b, PaddedPODArray & c, - CompareInt scale) + template + static void NO_INLINE vectorVector(const ArrayA & a, const ArrayB & b, PaddedPODArray & c, CompareInt scale) { size_t size = a.size(); const A * a_pos = a.data(); @@ -276,14 +272,14 @@ private: while (a_pos < a_end) { - *c_pos = apply(*a_pos, *b_pos, scale); + *c_pos = apply(*a_pos, *b_pos, scale); ++a_pos; ++b_pos; ++c_pos; } } - template + template static void NO_INLINE vectorConstant(const ArrayA & a, B b, PaddedPODArray & c, CompareInt scale) { size_t size = a.size(); @@ -293,13 +289,13 @@ private: while (a_pos < a_end) { - *c_pos = apply(*a_pos, b, scale); + *c_pos = apply(*a_pos, b, scale); ++a_pos; ++c_pos; } } - template + template static void NO_INLINE constantVector(A a, const ArrayB & b, PaddedPODArray & c, CompareInt scale) { size_t size = b.size(); @@ -309,7 +305,7 @@ private: while (b_pos < b_end) { - *c_pos = apply(a, *b_pos, scale); + *c_pos = apply(a, *b_pos, scale); ++b_pos; ++c_pos; } diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index e774a95e19f..90f30b52c0c 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -529,22 +529,25 @@ Field Field::restoreFromDump(std::string_view dump_) template bool decimalEqual(T x, T y, UInt32 x_scale, UInt32 y_scale) { + bool check_overflow = true; using Comparator = DecimalComparison; - return Comparator::compare(x, y, x_scale, y_scale); + return Comparator::compare(x, y, x_scale, y_scale, check_overflow); } template bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale) { + bool check_overflow = true; using Comparator = DecimalComparison; - return Comparator::compare(x, y, x_scale, y_scale); + return Comparator::compare(x, y, x_scale, y_scale, check_overflow); } template bool decimalLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale) { + bool check_overflow = true; using Comparator = DecimalComparison; - return Comparator::compare(x, y, x_scale, y_scale); + return Comparator::compare(x, y, x_scale, y_scale, check_overflow); } diff --git a/src/Core/Field.h b/src/Core/Field.h index c08d5c9eb42..5a6ee9cdf29 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -863,6 +863,9 @@ template <> struct Field::EnumToType { usi template <> struct Field::EnumToType { using Type = CustomType; }; template <> struct Field::EnumToType { using Type = UInt64; }; +/// Use it to prevent inclusion of magic_enum in headers, which is very expensive for the compiler +std::string_view fieldTypeToString(Field::Types::Which type); + constexpr bool isInt64OrUInt64FieldType(Field::Types::Which t) { return t == Field::Types::Int64 @@ -886,7 +889,7 @@ auto & Field::safeGet() if (target != which && !(which == Field::Types::Bool && (target == Field::Types::UInt64 || target == Field::Types::Int64)) && !(isInt64OrUInt64FieldType(which) && isInt64OrUInt64FieldType(target))) - throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), target); + throw Exception(ErrorCodes::BAD_GET, "Bad get: has {}, requested {}", getTypeName(), fieldTypeToString(target)); return get(); } @@ -1002,8 +1005,6 @@ void readQuoted(DecimalField & x, ReadBuffer & buf); void writeFieldText(const Field & x, WriteBuffer & buf); String toString(const Field & x); - -std::string_view fieldTypeToString(Field::Types::Which type); } template <> diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 2f8e7b6843a..4bea23d4e90 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -26,6 +27,8 @@ extern const Metric BackgroundMessageBrokerSchedulePoolSize; namespace DB { +// clang-format off + #define LIST_OF_SERVER_SETTINGS(DECLARE, ALIAS) \ DECLARE(Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0) \ DECLARE(Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0) \ @@ -201,7 +204,11 @@ namespace DB DECLARE(UInt64, load_marks_threadpool_pool_size, 50, "Size of background pool for marks loading", 0) \ DECLARE(UInt64, load_marks_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \ DECLARE(UInt64, threadpool_writer_pool_size, 100, "Size of background pool for write requests to object storages", 0) \ - DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) + DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) \ + DECLARE(UInt32, allowed_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \ + + +// clang-format on /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below @@ -284,40 +291,42 @@ void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParam /// with new the setting values but the settings themselves are not stored between re-loads. As a result, if one wants to know the /// current setting values, one needs to ask the components directly. std::unordered_map> changeable_settings - = {{"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}}, + = { + {"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}}, - {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}}, - {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}}, + {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}}, + {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}}, - {"max_concurrent_queries", {std::to_string(context->getProcessList().getMaxSize()), ChangeableWithoutRestart::Yes}}, - {"max_concurrent_insert_queries", + {"max_concurrent_queries", {std::to_string(context->getProcessList().getMaxSize()), ChangeableWithoutRestart::Yes}}, + {"max_concurrent_insert_queries", {std::to_string(context->getProcessList().getMaxInsertQueriesAmount()), ChangeableWithoutRestart::Yes}}, - {"max_concurrent_select_queries", + {"max_concurrent_select_queries", {std::to_string(context->getProcessList().getMaxSelectQueriesAmount()), ChangeableWithoutRestart::Yes}}, - {"max_waiting_queries", {std::to_string(context->getProcessList().getMaxWaitingQueriesAmount()), ChangeableWithoutRestart::Yes}}, + {"max_waiting_queries", {std::to_string(context->getProcessList().getMaxWaitingQueriesAmount()), ChangeableWithoutRestart::Yes}}, - {"background_buffer_flush_schedule_pool_size", - {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize)), - ChangeableWithoutRestart::IncreaseOnly}}, - {"background_schedule_pool_size", - {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, - {"background_message_broker_schedule_pool_size", - {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize)), - ChangeableWithoutRestart::IncreaseOnly}}, - {"background_distributed_schedule_pool_size", - {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize)), - ChangeableWithoutRestart::IncreaseOnly}}, + {"background_buffer_flush_schedule_pool_size", + {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + {"background_schedule_pool_size", + {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + {"background_message_broker_schedule_pool_size", + {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, + {"background_distributed_schedule_pool_size", + {std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize)), ChangeableWithoutRestart::IncreaseOnly}}, - {"mark_cache_size", {std::to_string(context->getMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, - {"uncompressed_cache_size", {std::to_string(context->getUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, - {"index_mark_cache_size", {std::to_string(context->getIndexMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, - {"index_uncompressed_cache_size", - {std::to_string(context->getIndexUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, - {"mmap_cache_size", {std::to_string(context->getMMappedFileCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"mark_cache_size", {std::to_string(context->getMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"uncompressed_cache_size", {std::to_string(context->getUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"index_mark_cache_size", {std::to_string(context->getIndexMarkCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"index_uncompressed_cache_size", + {std::to_string(context->getIndexUncompressedCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, + {"mmap_cache_size", {std::to_string(context->getMMappedFileCache()->maxSizeInBytes()), ChangeableWithoutRestart::Yes}}, - {"merge_workload", {context->getMergeWorkload(), ChangeableWithoutRestart::Yes}}, - {"mutation_workload", {context->getMutationWorkload(), ChangeableWithoutRestart::Yes}}, - {"config_reload_interval_ms", {std::to_string(context->getConfigReloaderInterval()), ChangeableWithoutRestart::Yes}}}; + {"merge_workload", {context->getMergeWorkload(), ChangeableWithoutRestart::Yes}}, + {"mutation_workload", {context->getMutationWorkload(), ChangeableWithoutRestart::Yes}}, + {"config_reload_interval_ms", {std::to_string(context->getConfigReloaderInterval()), ChangeableWithoutRestart::Yes}}, + + {"allowed_feature_tier", + {std::to_string(context->getAccessControl().getAllowTierSettings()), ChangeableWithoutRestart::Yes}}, + }; if (context->areBackgroundExecutorsInitialized()) { diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 2cd3d272490..c8582b6f646 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -433,7 +433,7 @@ Possible values: - 0 — `INSERT` query appends new data to the end of the file. - 1 — `INSERT` query creates a new file. )", 0) \ - DECLARE(Bool, s3_skip_empty_files, false, R"( + DECLARE(Bool, s3_skip_empty_files, true, R"( Enables or disables skipping empty files in [S3](../../engines/table-engines/integrations/s3.md) engine tables. Possible values: @@ -4323,7 +4323,7 @@ Disable limit on kafka_num_consumers that depends on the number of available CPU )", 0) \ DECLARE(Bool, allow_experimental_kafka_offsets_storage_in_keeper, false, R"( Allow experimental feature to store Kafka related offsets in ClickHouse Keeper. When enabled a ClickHouse Keeper path and replica name can be specified to the Kafka table engine. As a result instead of the regular Kafka engine, a new type of storage engine will be used that stores the committed offsets primarily in ClickHouse Keeper -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Bool, enable_software_prefetch_in_aggregation, true, R"( Enable use of software prefetch in aggregation )", 0) \ @@ -5659,10 +5659,10 @@ Skip index analysis on workers. Effective only with enabled parallel_replicas_lo \ DECLARE(Bool, allow_experimental_analyzer, true, R"( Allow new query analyzer. -)", IMPORTANT | BETA) ALIAS(enable_analyzer) \ +)", IMPORTANT) ALIAS(enable_analyzer) \ DECLARE(Bool, analyzer_compatibility_join_using_top_level_identifier, false, R"( Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`). -)", BETA) \ +)", 0) \ \ DECLARE(Timezone, session_timezone, "", R"( Sets the implicit time zone of the current session or query. @@ -5773,7 +5773,7 @@ Possible values: - 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. - 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_vector_similarity_index, false, R"( Allow experimental vector similarity index )", EXPERIMENTAL) \ @@ -5846,7 +5846,7 @@ If it is set to true, allow to use experimental full-text index. \ DECLARE(Bool, allow_experimental_join_condition, false, R"( Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y. -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, allow_experimental_live_view, false, R"( Allows creation of a deprecated LIVE VIEW. @@ -5855,7 +5855,7 @@ Possible values: - 0 — Working with live views is disabled. - 1 — Working with live views is enabled. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Seconds, live_view_heartbeat_interval, 15, R"( The heartbeat interval in seconds to indicate live query is alive. )", EXPERIMENTAL) \ @@ -6209,6 +6209,11 @@ bool Settings::isChanged(std::string_view name) const return impl->isChanged(name); } +SettingsTierType Settings::getTier(std::string_view name) const +{ + return impl->getTier(name); +} + bool Settings::tryGet(std::string_view name, Field & value) const { return impl->tryGet(name, value); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ac3b1fe651e..b66f4403ddf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -117,6 +118,7 @@ struct Settings /// General API as needed bool has(std::string_view name) const; bool isChanged(std::string_view name) const; + SettingsTierType getTier(std::string_view name) const; bool tryGet(std::string_view name, Field & value) const; Field get(std::string_view name) const; diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 18a9dd6ecbf..f0d3e001362 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -87,6 +87,7 @@ static std::initializer_list +static bool callOnBasicTypeSecondArg(TypeIndex number, F && f) +{ + if constexpr (_int) + { + switch (number) + { + case TypeIndex::UInt8: return f(TypePair()); + case TypeIndex::UInt16: return f(TypePair()); + case TypeIndex::UInt32: return f(TypePair()); + case TypeIndex::UInt64: return f(TypePair()); + case TypeIndex::UInt128: return f(TypePair()); + case TypeIndex::UInt256: return f(TypePair()); + + case TypeIndex::Int8: return f(TypePair()); + case TypeIndex::Int16: return f(TypePair()); + case TypeIndex::Int32: return f(TypePair()); + case TypeIndex::Int64: return f(TypePair()); + case TypeIndex::Int128: return f(TypePair()); + case TypeIndex::Int256: return f(TypePair()); + + case TypeIndex::Enum8: return f(TypePair()); + case TypeIndex::Enum16: return f(TypePair()); + + default: + break; + } + } + + if constexpr (_decimal) + { + switch (number) + { + case TypeIndex::Decimal32: return f(TypePair()); + case TypeIndex::Decimal64: return f(TypePair()); + case TypeIndex::Decimal128: return f(TypePair()); + case TypeIndex::Decimal256: return f(TypePair()); + default: + break; + } + } + + if constexpr (_float) + { + switch (number) + { + case TypeIndex::BFloat16: return f(TypePair()); + case TypeIndex::Float32: return f(TypePair()); + case TypeIndex::Float64: return f(TypePair()); + default: + break; + } + } + + if constexpr (_datetime) + { + switch (number) + { + case TypeIndex::Date: return f(TypePair()); + case TypeIndex::Date32: return f(TypePair()); + case TypeIndex::DateTime: return f(TypePair()); + case TypeIndex::DateTime64: return f(TypePair()); + default: + break; + } + } + + return false; +} + /// Unroll template using TypeIndex template static inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f) diff --git a/src/DataTypes/DataTypeDecimalBase.cpp b/src/DataTypes/DataTypeDecimalBase.cpp index 68bfba475d6..423ab2e4765 100644 --- a/src/DataTypes/DataTypeDecimalBase.cpp +++ b/src/DataTypes/DataTypeDecimalBase.cpp @@ -1,7 +1,8 @@ +#include +#include #include #include #include -#include namespace DB { @@ -14,6 +15,12 @@ namespace ErrorCodes { } +template +constexpr size_t DataTypeDecimalBase::maxPrecision() +{ + return DecimalUtils::max_precision; +} + bool decimalCheckComparisonOverflow(ContextPtr context) { return context->getSettingsRef()[Setting::decimal_check_overflow]; @@ -41,6 +48,18 @@ T DataTypeDecimalBase::getScaleMultiplier(UInt32 scale_) return DecimalUtils::scaleMultiplier(scale_); } +template +T DataTypeDecimalBase::wholePart(T x) const +{ + return DecimalUtils::getWholePart(x, scale); +} + +template +T DataTypeDecimalBase::fractionalPart(T x) const +{ + return DecimalUtils::getFractionalPart(x, scale); +} + /// Explicit template instantiations. template class DataTypeDecimalBase; diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index c1e1d27557f..beba3c42616 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -3,11 +3,10 @@ #include #include -#include -#include #include -#include +#include #include +#include #include @@ -64,7 +63,7 @@ public: static constexpr bool is_parametric = true; - static constexpr size_t maxPrecision() { return DecimalUtils::max_precision; } + static constexpr size_t maxPrecision(); DataTypeDecimalBase(UInt32 precision_, UInt32 scale_) : precision(precision_), @@ -104,15 +103,8 @@ public: UInt32 getScale() const { return scale; } T getScaleMultiplier() const { return getScaleMultiplier(scale); } - T wholePart(T x) const - { - return DecimalUtils::getWholePart(x, scale); - } - - T fractionalPart(T x) const - { - return DecimalUtils::getFractionalPart(x, scale); - } + T wholePart(T x) const; + T fractionalPart(T x) const; T maxWholeValue() const { return getScaleMultiplier(precision - scale) - T(1); } @@ -147,11 +139,6 @@ public: static T getScaleMultiplier(UInt32 scale); - DecimalUtils::DataTypeDecimalTrait getTrait() const - { - return {precision, scale}; - } - protected: const UInt32 precision; const UInt32 scale; @@ -167,50 +154,35 @@ inline const DataTypeDecimalBase * checkDecimalBase(const IDataType & data_ty return nullptr; } -template typename DecimalType> -inline auto decimalResultType(const DecimalType & tx, const DecimalType & ty) -{ - const auto result_trait = DecimalUtils::binaryOpResult(tx, ty); - return DecimalType(result_trait.precision, result_trait.scale); -} +template <> constexpr size_t DataTypeDecimalBase::maxPrecision() { return 9; }; +template <> constexpr size_t DataTypeDecimalBase::maxPrecision() { return 18; }; +template <> constexpr size_t DataTypeDecimalBase::maxPrecision() { return 18; }; +template <> constexpr size_t DataTypeDecimalBase::maxPrecision() { return 38; }; +template <> constexpr size_t DataTypeDecimalBase::maxPrecision() { return 76; }; -template typename DecimalType> -inline DecimalType decimalResultType(const DecimalType & tx, const DataTypeNumber & ty) -{ - const auto result_trait = DecimalUtils::binaryOpResult(tx, ty); - return DecimalType(result_trait.precision, result_trait.scale); -} - -template typename DecimalType> -inline DecimalType decimalResultType(const DataTypeNumber & tx, const DecimalType & ty) -{ - const auto result_trait = DecimalUtils::binaryOpResult(tx, ty); - return DecimalType(result_trait.precision, result_trait.scale); -} +extern template class DataTypeDecimalBase; +extern template class DataTypeDecimalBase; +extern template class DataTypeDecimalBase; +extern template class DataTypeDecimalBase; +extern template class DataTypeDecimalBase; template