mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge pull request #42619 from ClickHouse/jepen-multiple-tests
Extend Jepsen tests with support for server
This commit is contained in:
commit
e512d7f036
33
.github/workflows/jepsen.yml
vendored
33
.github/workflows/jepsen.yml
vendored
@ -32,10 +32,41 @@ jobs:
|
|||||||
mkdir -p "$TEMP_PATH"
|
mkdir -p "$TEMP_PATH"
|
||||||
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
|
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
|
||||||
cd "$REPO_COPY/tests/ci"
|
cd "$REPO_COPY/tests/ci"
|
||||||
python3 keeper_jepsen_check.py
|
python3 jepsen_check.py keeper
|
||||||
- name: Cleanup
|
- name: Cleanup
|
||||||
if: always()
|
if: always()
|
||||||
run: |
|
run: |
|
||||||
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
|
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
|
||||||
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
|
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
|
||||||
sudo rm -fr "$TEMP_PATH"
|
sudo rm -fr "$TEMP_PATH"
|
||||||
|
# ServerJepsenRelease:
|
||||||
|
# runs-on: [self-hosted, style-checker]
|
||||||
|
# if: ${{ always() }}
|
||||||
|
# needs: [KeeperJepsenRelease]
|
||||||
|
# steps:
|
||||||
|
# - name: Set envs
|
||||||
|
# run: |
|
||||||
|
# cat >> "$GITHUB_ENV" << 'EOF'
|
||||||
|
# TEMP_PATH=${{runner.temp}}/server_jepsen
|
||||||
|
# REPO_COPY=${{runner.temp}}/server_jepsen/ClickHouse
|
||||||
|
# EOF
|
||||||
|
# - name: Clear repository
|
||||||
|
# run: |
|
||||||
|
# sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
|
||||||
|
# - name: Check out repository code
|
||||||
|
# uses: actions/checkout@v2
|
||||||
|
# with:
|
||||||
|
# fetch-depth: 0
|
||||||
|
# - name: Jepsen Test
|
||||||
|
# run: |
|
||||||
|
# sudo rm -fr "$TEMP_PATH"
|
||||||
|
# mkdir -p "$TEMP_PATH"
|
||||||
|
# cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
|
||||||
|
# cd "$REPO_COPY/tests/ci"
|
||||||
|
# python3 jepsen_check.py server
|
||||||
|
# - name: Cleanup
|
||||||
|
# if: always()
|
||||||
|
# run: |
|
||||||
|
# docker ps --quiet | xargs --no-run-if-empty docker kill ||:
|
||||||
|
# docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
|
||||||
|
# sudo rm -fr "$TEMP_PATH"
|
||||||
|
@ -15,8 +15,8 @@ if [ -z "$CLICKHOUSE_REPO_PATH" ]; then
|
|||||||
ls -lath ||:
|
ls -lath ||:
|
||||||
fi
|
fi
|
||||||
|
|
||||||
cd "$CLICKHOUSE_REPO_PATH/tests/jepsen.clickhouse-keeper"
|
cd "$CLICKHOUSE_REPO_PATH/tests/jepsen.clickhouse"
|
||||||
|
|
||||||
(lein run test-all --nodes-file "$NODES_FILE_PATH" --username "$NODES_USERNAME" --logging-json --password "$NODES_PASSWORD" --time-limit "$TIME_LIMIT" --concurrency 50 -r 50 --snapshot-distance 100 --stale-log-gap 100 --reserved-log-items 10 --lightweight-run --clickhouse-source "$CLICKHOUSE_PACKAGE" -q --test-count "$TESTS_TO_RUN" || true) | tee "$TEST_OUTPUT/jepsen_run_all_tests.log"
|
(lein run keeper test-all --nodes-file "$NODES_FILE_PATH" --username "$NODES_USERNAME" --logging-json --password "$NODES_PASSWORD" --time-limit "$TIME_LIMIT" --concurrency 50 -r 50 --snapshot-distance 100 --stale-log-gap 100 --reserved-log-items 10 --lightweight-run --clickhouse-source "$CLICKHOUSE_PACKAGE" -q --test-count "$TESTS_TO_RUN" || true) | tee "$TEST_OUTPUT/jepsen_run_all_tests.log"
|
||||||
|
|
||||||
mv store "$TEST_OUTPUT/"
|
mv store "$TEST_OUTPUT/"
|
||||||
|
43
docker/test/server-jepsen/Dockerfile
Normal file
43
docker/test/server-jepsen/Dockerfile
Normal file
@ -0,0 +1,43 @@
|
|||||||
|
# rebuild in #33610
|
||||||
|
# docker build -t clickhouse/server-jepsen-test .
|
||||||
|
ARG FROM_TAG=latest
|
||||||
|
FROM clickhouse/test-base:$FROM_TAG
|
||||||
|
|
||||||
|
ENV DEBIAN_FRONTEND=noninteractive
|
||||||
|
ENV CLOJURE_VERSION=1.10.3.814
|
||||||
|
|
||||||
|
# arguments
|
||||||
|
ENV PR_TO_TEST=""
|
||||||
|
ENV SHA_TO_TEST=""
|
||||||
|
|
||||||
|
ENV NODES_USERNAME="root"
|
||||||
|
ENV NODES_PASSWORD=""
|
||||||
|
ENV TESTS_TO_RUN="8"
|
||||||
|
ENV TIME_LIMIT="30"
|
||||||
|
|
||||||
|
ENV KEEPER_NODE=""
|
||||||
|
|
||||||
|
|
||||||
|
# volumes
|
||||||
|
ENV NODES_FILE_PATH="/nodes.txt"
|
||||||
|
ENV TEST_OUTPUT="/test_output"
|
||||||
|
|
||||||
|
RUN mkdir "/root/.ssh"
|
||||||
|
RUN touch "/root/.ssh/known_hosts"
|
||||||
|
|
||||||
|
# install java
|
||||||
|
RUN apt-get update && apt-get install default-jre default-jdk libjna-java libjna-jni ssh gnuplot graphviz --yes --no-install-recommends
|
||||||
|
|
||||||
|
# install clojure
|
||||||
|
RUN curl -O "https://download.clojure.org/install/linux-install-${CLOJURE_VERSION}.sh" && \
|
||||||
|
chmod +x "linux-install-${CLOJURE_VERSION}.sh" && \
|
||||||
|
bash "./linux-install-${CLOJURE_VERSION}.sh"
|
||||||
|
|
||||||
|
# install leiningen
|
||||||
|
RUN curl -O "https://raw.githubusercontent.com/technomancy/leiningen/stable/bin/lein" && \
|
||||||
|
chmod +x ./lein && \
|
||||||
|
mv ./lein /usr/bin
|
||||||
|
|
||||||
|
COPY run.sh /
|
||||||
|
|
||||||
|
CMD ["/bin/bash", "/run.sh"]
|
22
docker/test/server-jepsen/run.sh
Normal file
22
docker/test/server-jepsen/run.sh
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
set -euo pipefail
|
||||||
|
|
||||||
|
|
||||||
|
CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-15_relwithdebuginfo_none_unsplitted_disable_False_binary/clickhouse"}
|
||||||
|
CLICKHOUSE_REPO_PATH=${CLICKHOUSE_REPO_PATH:=""}
|
||||||
|
|
||||||
|
|
||||||
|
if [ -z "$CLICKHOUSE_REPO_PATH" ]; then
|
||||||
|
CLICKHOUSE_REPO_PATH=ch
|
||||||
|
rm -rf ch ||:
|
||||||
|
mkdir ch ||:
|
||||||
|
wget -nv -nd -c "https://clickhouse-test-reports.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/repo/clickhouse_no_subs.tar.gz"
|
||||||
|
tar -C ch --strip-components=1 -xf clickhouse_no_subs.tar.gz
|
||||||
|
ls -lath ||:
|
||||||
|
fi
|
||||||
|
|
||||||
|
cd "$CLICKHOUSE_REPO_PATH/tests/jepsen.clickhouse"
|
||||||
|
|
||||||
|
(lein run server test-all --keeper "$KEEPER_NODE" --nodes-file "$NODES_FILE_PATH" --username "$NODES_USERNAME" --logging-json --password "$NODES_PASSWORD" --time-limit "$TIME_LIMIT" --concurrency 50 -r 50 --clickhouse-source "$CLICKHOUSE_PACKAGE" --test-count "$TESTS_TO_RUN" || true) | tee "$TEST_OUTPUT/jepsen_run_all_tests.log"
|
||||||
|
|
||||||
|
mv store "$TEST_OUTPUT/"
|
@ -347,6 +347,9 @@ CI_CONFIG = {
|
|||||||
"ClickHouse Keeper Jepsen": {
|
"ClickHouse Keeper Jepsen": {
|
||||||
"required_build": "binary_release",
|
"required_build": "binary_release",
|
||||||
},
|
},
|
||||||
|
"ClickHouse Server Jepsen": {
|
||||||
|
"required_build": "binary_release",
|
||||||
|
},
|
||||||
"Performance Comparison": {
|
"Performance Comparison": {
|
||||||
"required_build": "package_release",
|
"required_build": "package_release",
|
||||||
"test_grep_exclude_filter": "",
|
"test_grep_exclude_filter": "",
|
||||||
|
@ -5,6 +5,8 @@ import logging
|
|||||||
import os
|
import os
|
||||||
import sys
|
import sys
|
||||||
|
|
||||||
|
import argparse
|
||||||
|
|
||||||
import boto3
|
import boto3
|
||||||
from github import Github
|
from github import Github
|
||||||
import requests
|
import requests
|
||||||
@ -25,9 +27,15 @@ from build_download_helper import get_build_name_for_check
|
|||||||
from rerun_helper import RerunHelper
|
from rerun_helper import RerunHelper
|
||||||
|
|
||||||
JEPSEN_GROUP_NAME = "jepsen_group"
|
JEPSEN_GROUP_NAME = "jepsen_group"
|
||||||
DESIRED_INSTANCE_COUNT = 3
|
|
||||||
IMAGE_NAME = "clickhouse/keeper-jepsen-test"
|
KEEPER_DESIRED_INSTANCE_COUNT = 3
|
||||||
CHECK_NAME = "ClickHouse Keeper Jepsen"
|
SERVER_DESIRED_INSTANCE_COUNT = 4
|
||||||
|
|
||||||
|
KEEPER_IMAGE_NAME = "clickhouse/keeper-jepsen-test"
|
||||||
|
KEEPER_CHECK_NAME = "ClickHouse Keeper Jepsen"
|
||||||
|
|
||||||
|
SERVER_IMAGE_NAME = "clickhouse/server-jepsen-test"
|
||||||
|
SERVER_CHECK_NAME = "ClickHouse Server Jepsen"
|
||||||
|
|
||||||
|
|
||||||
SUCCESSFUL_TESTS_ANCHOR = "# Successful tests"
|
SUCCESSFUL_TESTS_ANCHOR = "# Successful tests"
|
||||||
@ -49,8 +57,7 @@ def _parse_jepsen_output(path):
|
|||||||
current_type = "FAIL"
|
current_type = "FAIL"
|
||||||
|
|
||||||
if (
|
if (
|
||||||
line.startswith("store/clickhouse-keeper")
|
line.startswith("store/clickhouse") or line.startswith("clickhouse")
|
||||||
or line.startswith("clickhouse-keeper")
|
|
||||||
) and current_type:
|
) and current_type:
|
||||||
test_results.append((line.strip(), current_type))
|
test_results.append((line.strip(), current_type))
|
||||||
|
|
||||||
@ -82,15 +89,15 @@ def get_instances_addresses(ec2_client, instance_ids):
|
|||||||
return instance_ips
|
return instance_ips
|
||||||
|
|
||||||
|
|
||||||
def prepare_autoscaling_group_and_get_hostnames():
|
def prepare_autoscaling_group_and_get_hostnames(count):
|
||||||
asg_client = boto3.client("autoscaling", region_name="us-east-1")
|
asg_client = boto3.client("autoscaling", region_name="us-east-1")
|
||||||
asg_client.set_desired_capacity(
|
asg_client.set_desired_capacity(
|
||||||
AutoScalingGroupName=JEPSEN_GROUP_NAME, DesiredCapacity=DESIRED_INSTANCE_COUNT
|
AutoScalingGroupName=JEPSEN_GROUP_NAME, DesiredCapacity=count
|
||||||
)
|
)
|
||||||
|
|
||||||
instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME)
|
instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME)
|
||||||
counter = 0
|
counter = 0
|
||||||
while len(instances) < DESIRED_INSTANCE_COUNT:
|
while len(instances) < count:
|
||||||
time.sleep(5)
|
time.sleep(5)
|
||||||
instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME)
|
instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME)
|
||||||
counter += 1
|
counter += 1
|
||||||
@ -132,17 +139,30 @@ def get_run_command(
|
|||||||
repo_path,
|
repo_path,
|
||||||
build_url,
|
build_url,
|
||||||
result_path,
|
result_path,
|
||||||
|
extra_args,
|
||||||
docker_image,
|
docker_image,
|
||||||
):
|
):
|
||||||
return (
|
return (
|
||||||
f"docker run --network=host -v '{ssh_sock_dir}:{ssh_sock_dir}' -e SSH_AUTH_SOCK={ssh_auth_sock} "
|
f"docker run --network=host -v '{ssh_sock_dir}:{ssh_sock_dir}' -e SSH_AUTH_SOCK={ssh_auth_sock} "
|
||||||
f"-e PR_TO_TEST={pr_info.number} -e SHA_TO_TEST={pr_info.sha} -v '{nodes_path}:/nodes.txt' -v {result_path}:/test_output "
|
f"-e PR_TO_TEST={pr_info.number} -e SHA_TO_TEST={pr_info.sha} -v '{nodes_path}:/nodes.txt' -v {result_path}:/test_output "
|
||||||
f"-e 'CLICKHOUSE_PACKAGE={build_url}' -v '{repo_path}:/ch' -e 'CLICKHOUSE_REPO_PATH=/ch' -e NODES_USERNAME=ubuntu {docker_image}"
|
f"-e 'CLICKHOUSE_PACKAGE={build_url}' -v '{repo_path}:/ch' -e 'CLICKHOUSE_REPO_PATH=/ch' -e NODES_USERNAME=ubuntu {extra_args} {docker_image}"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
if __name__ == "__main__":
|
if __name__ == "__main__":
|
||||||
logging.basicConfig(level=logging.INFO)
|
logging.basicConfig(level=logging.INFO)
|
||||||
|
parser = argparse.ArgumentParser(
|
||||||
|
prog="Jepsen Check",
|
||||||
|
description="Check that uses Jepsen. Both Keeper and Server can be tested.",
|
||||||
|
)
|
||||||
|
parser.add_argument(
|
||||||
|
"program", help='What should be tested. Valid values "keeper", "server"'
|
||||||
|
)
|
||||||
|
args = parser.parse_args()
|
||||||
|
|
||||||
|
if args.program != "server" and args.program != "keeper":
|
||||||
|
logging.warning("Invalid argument '%s'", args.program)
|
||||||
|
sys.exit(0)
|
||||||
|
|
||||||
stopwatch = Stopwatch()
|
stopwatch = Stopwatch()
|
||||||
|
|
||||||
@ -161,7 +181,9 @@ if __name__ == "__main__":
|
|||||||
|
|
||||||
gh = Github(get_best_robot_token(), per_page=100)
|
gh = Github(get_best_robot_token(), per_page=100)
|
||||||
|
|
||||||
rerun_helper = RerunHelper(gh, pr_info, CHECK_NAME)
|
check_name = KEEPER_CHECK_NAME if args.program == "keeper" else SERVER_CHECK_NAME
|
||||||
|
|
||||||
|
rerun_helper = RerunHelper(gh, pr_info, check_name)
|
||||||
if rerun_helper.is_already_finished_by_status():
|
if rerun_helper.is_already_finished_by_status():
|
||||||
logging.info("Check is already finished according to github status, exiting")
|
logging.info("Check is already finished according to github status, exiting")
|
||||||
sys.exit(0)
|
sys.exit(0)
|
||||||
@ -173,13 +195,19 @@ if __name__ == "__main__":
|
|||||||
if not os.path.exists(result_path):
|
if not os.path.exists(result_path):
|
||||||
os.makedirs(result_path)
|
os.makedirs(result_path)
|
||||||
|
|
||||||
instances = prepare_autoscaling_group_and_get_hostnames()
|
instances = prepare_autoscaling_group_and_get_hostnames(
|
||||||
nodes_path = save_nodes_to_file(instances, TEMP_PATH)
|
KEEPER_DESIRED_INSTANCE_COUNT
|
||||||
|
if args.program == "keeper"
|
||||||
|
else SERVER_DESIRED_INSTANCE_COUNT
|
||||||
|
)
|
||||||
|
nodes_path = save_nodes_to_file(
|
||||||
|
instances[:KEEPER_DESIRED_INSTANCE_COUNT], TEMP_PATH
|
||||||
|
)
|
||||||
|
|
||||||
# always use latest
|
# always use latest
|
||||||
docker_image = IMAGE_NAME
|
docker_image = KEEPER_IMAGE_NAME if args.program == "keeper" else SERVER_IMAGE_NAME
|
||||||
|
|
||||||
build_name = get_build_name_for_check(CHECK_NAME)
|
build_name = get_build_name_for_check(check_name)
|
||||||
|
|
||||||
if pr_info.number == 0:
|
if pr_info.number == 0:
|
||||||
version = get_version_from_repo()
|
version = get_version_from_repo()
|
||||||
@ -203,6 +231,10 @@ if __name__ == "__main__":
|
|||||||
logging.warning("Cannot fetch build in 30 minutes, exiting")
|
logging.warning("Cannot fetch build in 30 minutes, exiting")
|
||||||
sys.exit(0)
|
sys.exit(0)
|
||||||
|
|
||||||
|
extra_args = ""
|
||||||
|
if args.program == "server":
|
||||||
|
extra_args = f"-e KEEPER_NODE={instances[-1]}"
|
||||||
|
|
||||||
with SSHKey(key_value=get_parameter_from_ssm("jepsen_ssh_key") + "\n"):
|
with SSHKey(key_value=get_parameter_from_ssm("jepsen_ssh_key") + "\n"):
|
||||||
ssh_auth_sock = os.environ["SSH_AUTH_SOCK"]
|
ssh_auth_sock = os.environ["SSH_AUTH_SOCK"]
|
||||||
auth_sock_dir = os.path.dirname(ssh_auth_sock)
|
auth_sock_dir = os.path.dirname(ssh_auth_sock)
|
||||||
@ -214,6 +246,7 @@ if __name__ == "__main__":
|
|||||||
REPO_COPY,
|
REPO_COPY,
|
||||||
build_url,
|
build_url,
|
||||||
result_path,
|
result_path,
|
||||||
|
extra_args,
|
||||||
docker_image,
|
docker_image,
|
||||||
)
|
)
|
||||||
logging.info("Going to run jepsen: %s", cmd)
|
logging.info("Going to run jepsen: %s", cmd)
|
||||||
@ -255,11 +288,11 @@ if __name__ == "__main__":
|
|||||||
pr_info.sha,
|
pr_info.sha,
|
||||||
test_result,
|
test_result,
|
||||||
[run_log_path] + additional_data,
|
[run_log_path] + additional_data,
|
||||||
CHECK_NAME,
|
check_name,
|
||||||
)
|
)
|
||||||
|
|
||||||
print(f"::notice ::Report url: {report_url}")
|
print(f"::notice ::Report url: {report_url}")
|
||||||
post_commit_status(gh, pr_info.sha, CHECK_NAME, description, status, report_url)
|
post_commit_status(gh, pr_info.sha, check_name, description, status, report_url)
|
||||||
|
|
||||||
ch_helper = ClickHouseHelper()
|
ch_helper = ClickHouseHelper()
|
||||||
prepared_events = prepare_tests_results_for_clickhouse(
|
prepared_events = prepare_tests_results_for_clickhouse(
|
||||||
@ -269,7 +302,7 @@ if __name__ == "__main__":
|
|||||||
stopwatch.duration_seconds,
|
stopwatch.duration_seconds,
|
||||||
stopwatch.start_time_str,
|
stopwatch.start_time_str,
|
||||||
report_url,
|
report_url,
|
||||||
CHECK_NAME,
|
check_name,
|
||||||
)
|
)
|
||||||
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
|
||||||
clear_autoscaling_group()
|
clear_autoscaling_group()
|
@ -1,20 +0,0 @@
|
|||||||
(ns jepsen.clickhouse-keeper.constants)
|
|
||||||
|
|
||||||
(def common-prefix "/home/robot-clickhouse")
|
|
||||||
|
|
||||||
(def binary-name "clickhouse")
|
|
||||||
|
|
||||||
(def binary-path (str common-prefix "/" binary-name))
|
|
||||||
(def pid-file-path (str common-prefix "/clickhouse.pid"))
|
|
||||||
|
|
||||||
(def data-dir (str common-prefix "/db"))
|
|
||||||
(def logs-dir (str common-prefix "/logs"))
|
|
||||||
(def configs-dir (str common-prefix "/config"))
|
|
||||||
(def sub-configs-dir (str configs-dir "/config.d"))
|
|
||||||
(def coordination-data-dir (str data-dir "/coordination"))
|
|
||||||
(def coordination-snapshots-dir (str coordination-data-dir "/snapshots"))
|
|
||||||
(def coordination-logs-dir (str coordination-data-dir "/logs"))
|
|
||||||
|
|
||||||
(def stderr-file (str logs-dir "/stderr.log"))
|
|
||||||
|
|
||||||
(def binaries-cache-dir (str common-prefix "/binaries"))
|
|
@ -1,170 +0,0 @@
|
|||||||
(ns jepsen.clickhouse-keeper.db
|
|
||||||
(:require [clojure.tools.logging :refer :all]
|
|
||||||
[jepsen
|
|
||||||
[control :as c]
|
|
||||||
[db :as db]
|
|
||||||
[util :as util :refer [meh]]]
|
|
||||||
[jepsen.clickhouse-keeper.constants :refer :all]
|
|
||||||
[jepsen.clickhouse-keeper.utils :refer :all]
|
|
||||||
[clojure.java.io :as io]
|
|
||||||
[jepsen.control.util :as cu]
|
|
||||||
[jepsen.os.ubuntu :as ubuntu]))
|
|
||||||
|
|
||||||
|
|
||||||
(ns jepsen.control.scp)
|
|
||||||
|
|
||||||
;; We need to overwrite Jepsen's implementation of scp! because it
|
|
||||||
;; doesn't use strict-host-key-checking
|
|
||||||
|
|
||||||
(defn scp!
|
|
||||||
"Runs an SCP command by shelling out. Takes a conn-spec (used for port, key,
|
|
||||||
etc), a seq of sources, and a single destination, all as strings."
|
|
||||||
[conn-spec sources dest]
|
|
||||||
(apply util/sh "scp" "-rpC"
|
|
||||||
"-P" (str (:port conn-spec))
|
|
||||||
(concat (when-let [k (:private-key-path conn-spec)]
|
|
||||||
["-i" k])
|
|
||||||
(if-not (:strict-host-key-checking conn-spec)
|
|
||||||
["-o StrictHostKeyChecking=no"])
|
|
||||||
sources
|
|
||||||
[dest]))
|
|
||||||
nil)
|
|
||||||
|
|
||||||
(ns jepsen.clickhouse-keeper.db)
|
|
||||||
|
|
||||||
(defn get-clickhouse-url
|
|
||||||
[url]
|
|
||||||
(non-precise-cached-wget! url))
|
|
||||||
|
|
||||||
(defn get-clickhouse-scp
|
|
||||||
[path]
|
|
||||||
(c/upload path (str common-prefix "/clickhouse")))
|
|
||||||
|
|
||||||
(defn download-clickhouse
|
|
||||||
[source]
|
|
||||||
(info "Downloading clickhouse from" source)
|
|
||||||
(cond
|
|
||||||
(clojure.string/starts-with? source "http") (get-clickhouse-url source)
|
|
||||||
(.exists (io/file source)) (get-clickhouse-scp source)
|
|
||||||
:else (throw (Exception. (str "Don't know how to download clickhouse from" source)))))
|
|
||||||
|
|
||||||
(defn unpack-deb
|
|
||||||
[path]
|
|
||||||
(do
|
|
||||||
(c/exec :dpkg :-x path common-prefix)
|
|
||||||
(c/exec :rm :-f path)
|
|
||||||
(c/exec :mv (str common-prefix "/usr/bin/clickhouse") common-prefix)
|
|
||||||
(c/exec :rm :-rf (str common-prefix "/usr") (str common-prefix "/etc"))))
|
|
||||||
|
|
||||||
(defn unpack-tgz
|
|
||||||
[path]
|
|
||||||
(do
|
|
||||||
(c/exec :mkdir :-p (str common-prefix "/unpacked"))
|
|
||||||
(c/exec :tar :-zxvf path :-C (str common-prefix "/unpacked"))
|
|
||||||
(c/exec :rm :-f path)
|
|
||||||
(let [subdir (c/exec :ls (str common-prefix "/unpacked"))]
|
|
||||||
(c/exec :mv (str common-prefix "/unpacked/" subdir "/usr/bin/clickhouse") common-prefix)
|
|
||||||
(c/exec :rm :-fr (str common-prefix "/unpacked")))))
|
|
||||||
|
|
||||||
(defn chmod-binary
|
|
||||||
[path]
|
|
||||||
(info "Binary path chmod" path)
|
|
||||||
(c/exec :chmod :+x path))
|
|
||||||
|
|
||||||
(defn install-downloaded-clickhouse
|
|
||||||
[path]
|
|
||||||
(cond
|
|
||||||
(clojure.string/ends-with? path ".deb") (unpack-deb path)
|
|
||||||
(clojure.string/ends-with? path ".tgz") (unpack-tgz path)
|
|
||||||
(clojure.string/ends-with? path "clickhouse") (chmod-binary path)
|
|
||||||
:else (throw (Exception. (str "Don't know how to install clickhouse from path" path)))))
|
|
||||||
|
|
||||||
(defn prepare-dirs
|
|
||||||
[]
|
|
||||||
(do
|
|
||||||
(c/exec :mkdir :-p common-prefix)
|
|
||||||
(c/exec :mkdir :-p data-dir)
|
|
||||||
(c/exec :mkdir :-p coordination-data-dir)
|
|
||||||
(c/exec :mkdir :-p logs-dir)
|
|
||||||
(c/exec :mkdir :-p configs-dir)
|
|
||||||
(c/exec :mkdir :-p sub-configs-dir)
|
|
||||||
(c/exec :touch stderr-file)
|
|
||||||
(c/exec :chown :-R :root common-prefix)))
|
|
||||||
|
|
||||||
(defn cluster-config
|
|
||||||
[test node config-template]
|
|
||||||
(let [nodes (:nodes test)
|
|
||||||
replacement-map {#"\{srv1\}" (get nodes 0)
|
|
||||||
#"\{srv2\}" (get nodes 1)
|
|
||||||
#"\{srv3\}" (get nodes 2)
|
|
||||||
#"\{id\}" (str (inc (.indexOf nodes node)))
|
|
||||||
#"\{quorum_reads\}" (str (boolean (:quorum test)))
|
|
||||||
#"\{snapshot_distance\}" (str (:snapshot-distance test))
|
|
||||||
#"\{stale_log_gap\}" (str (:stale-log-gap test))
|
|
||||||
#"\{reserved_log_items\}" (str (:reserved-log-items test))}]
|
|
||||||
(reduce #(clojure.string/replace %1 (get %2 0) (get %2 1)) config-template replacement-map)))
|
|
||||||
|
|
||||||
(defn install-configs
|
|
||||||
[test node]
|
|
||||||
(c/exec :echo (cluster-config test node (slurp (io/resource "keeper_config.xml"))) :> (str configs-dir "/keeper_config.xml")))
|
|
||||||
|
|
||||||
(defn collect-traces
|
|
||||||
[test node]
|
|
||||||
(let [pid (c/exec :pidof "clickhouse")]
|
|
||||||
(c/exec :timeout :-s "KILL" "60" :gdb :-ex "set pagination off" :-ex (str "set logging file " logs-dir "/gdb.log") :-ex
|
|
||||||
"set logging on" :-ex "backtrace" :-ex "thread apply all backtrace"
|
|
||||||
:-ex "backtrace" :-ex "detach" :-ex "quit" :--pid pid :|| :true)))
|
|
||||||
|
|
||||||
(defn db
|
|
||||||
[version reuse-binary]
|
|
||||||
(reify db/DB
|
|
||||||
(setup! [_ test node]
|
|
||||||
(c/su
|
|
||||||
(do
|
|
||||||
(info "Preparing directories")
|
|
||||||
(prepare-dirs)
|
|
||||||
(if (or (not (cu/exists? binary-path)) (not reuse-binary))
|
|
||||||
(do (info "Downloading clickhouse")
|
|
||||||
(install-downloaded-clickhouse (download-clickhouse version)))
|
|
||||||
(info "Binary already exsist on path" binary-path "skipping download"))
|
|
||||||
(info "Installing configs")
|
|
||||||
(install-configs test node)
|
|
||||||
(info "Starting server")
|
|
||||||
(start-clickhouse! node test)
|
|
||||||
(info "ClickHouse started"))))
|
|
||||||
|
|
||||||
(teardown! [_ test node]
|
|
||||||
(info node "Tearing down clickhouse")
|
|
||||||
(c/su
|
|
||||||
(kill-clickhouse! node test)
|
|
||||||
(if (not reuse-binary)
|
|
||||||
(c/exec :rm :-rf binary-path))
|
|
||||||
(c/exec :rm :-rf pid-file-path)
|
|
||||||
(c/exec :rm :-rf data-dir)
|
|
||||||
(c/exec :rm :-rf logs-dir)
|
|
||||||
(c/exec :rm :-rf configs-dir)))
|
|
||||||
|
|
||||||
db/LogFiles
|
|
||||||
(log-files [_ test node]
|
|
||||||
(c/su
|
|
||||||
;(if (cu/exists? pid-file-path)
|
|
||||||
;(do
|
|
||||||
; (info node "Collecting traces")
|
|
||||||
; (collect-traces test node))
|
|
||||||
;(info node "Pid files doesn't exists"))
|
|
||||||
(kill-clickhouse! node test)
|
|
||||||
(if (cu/exists? coordination-data-dir)
|
|
||||||
(do
|
|
||||||
(info node "Coordination files exists, going to compress")
|
|
||||||
(c/cd data-dir
|
|
||||||
(c/exec :tar :czf "coordination.tar.gz" "coordination"))))
|
|
||||||
(if (cu/exists? (str logs-dir))
|
|
||||||
(do
|
|
||||||
(info node "Logs exist, going to compress")
|
|
||||||
(c/cd common-prefix
|
|
||||||
(c/exec :tar :czf "logs.tar.gz" "logs"))) (info node "Logs are missing")))
|
|
||||||
(let [common-logs [(str common-prefix "/logs.tar.gz") (str data-dir "/coordination.tar.gz")]
|
|
||||||
gdb-log (str logs-dir "/gdb.log")]
|
|
||||||
(if (cu/exists? (str logs-dir "/gdb.log"))
|
|
||||||
(conj common-logs gdb-log)
|
|
||||||
common-logs)))))
|
|
@ -1,14 +1,16 @@
|
|||||||
(defproject jepsen.keeper "0.1.0-SNAPSHOT"
|
(defproject jepsen.clickhouse "0.1.0-SNAPSHOT"
|
||||||
:injections [(.. System (setProperty "zookeeper.request.timeout" "10000"))]
|
:injections [(.. System (setProperty "zookeeper.request.timeout" "10000"))]
|
||||||
:description "A jepsen tests for ClickHouse Keeper"
|
:description "A jepsen tests for ClickHouse"
|
||||||
:url "https://clickhouse.com/"
|
:url "https://clickhouse.com/"
|
||||||
:license {:name "EPL-2.0"
|
:license {:name "EPL-2.0"
|
||||||
:url "https://www.eclipse.org/legal/epl-2.0/"}
|
:url "https://www.eclipse.org/legal/epl-2.0/"}
|
||||||
:main jepsen.clickhouse-keeper.main
|
:main jepsen.clickhouse.main
|
||||||
:plugins [[lein-cljfmt "0.7.0"]]
|
:plugins [[lein-cljfmt "0.7.0"]]
|
||||||
:dependencies [[org.clojure/clojure "1.10.1"]
|
:dependencies [[org.clojure/clojure "1.10.1"]
|
||||||
[jepsen "0.2.7"]
|
[jepsen "0.2.7"]
|
||||||
[zookeeper-clj "0.9.4"]
|
[zookeeper-clj "0.9.4"]
|
||||||
|
[org.clojure/java.jdbc "0.7.12"]
|
||||||
[com.hierynomus/sshj "0.34.0"]
|
[com.hierynomus/sshj "0.34.0"]
|
||||||
|
[com.clickhouse/clickhouse-jdbc "0.3.2-patch11"]
|
||||||
[org.apache.zookeeper/zookeeper "3.6.1" :exclusions [org.slf4j/slf4j-log4j12]]]
|
[org.apache.zookeeper/zookeeper "3.6.1" :exclusions [org.slf4j/slf4j-log4j12]]]
|
||||||
:repl-options {:init-ns jepsen.clickhouse-keeper.main})
|
:repl-options {:init-ns jepsen.clickhouse-keeper.main})
|
16
tests/jepsen.clickhouse/resources/config.xml
Normal file
16
tests/jepsen.clickhouse/resources/config.xml
Normal file
@ -0,0 +1,16 @@
|
|||||||
|
<clickhouse>
|
||||||
|
<listen_host>::</listen_host>
|
||||||
|
<tcp_port>9000</tcp_port>
|
||||||
|
<http_port>8123</http_port>
|
||||||
|
|
||||||
|
<logger>
|
||||||
|
<level>trace</level>
|
||||||
|
<size>never</size>
|
||||||
|
</logger>
|
||||||
|
<user_directories>
|
||||||
|
<users_xml>
|
||||||
|
<!-- Path to configuration file with predefined users. -->
|
||||||
|
<path>users.xml</path>
|
||||||
|
</users_xml>
|
||||||
|
</user_directories>
|
||||||
|
</clickhouse>
|
34
tests/jepsen.clickhouse/resources/keeper_config_solo.xml
Normal file
34
tests/jepsen.clickhouse/resources/keeper_config_solo.xml
Normal file
@ -0,0 +1,34 @@
|
|||||||
|
<clickhouse>
|
||||||
|
<listen_host>::</listen_host>
|
||||||
|
|
||||||
|
<logger>
|
||||||
|
<level>trace</level>
|
||||||
|
<log>/var/log/clickhouse-keeper/clickhouse-keeper.log</log>
|
||||||
|
<errorlog>/var/log/clickhouse-keeper/clickhouse-keeper.err.log</errorlog>
|
||||||
|
<size>never</size>
|
||||||
|
</logger>
|
||||||
|
|
||||||
|
<keeper_server>
|
||||||
|
<tcp_port>9181</tcp_port>
|
||||||
|
<server_id>1</server_id>
|
||||||
|
|
||||||
|
<coordination_settings>
|
||||||
|
<operation_timeout_ms>10000</operation_timeout_ms>
|
||||||
|
<session_timeout_ms>30000</session_timeout_ms>
|
||||||
|
<force_sync>false</force_sync>
|
||||||
|
<startup_timeout>120000</startup_timeout>
|
||||||
|
<raft_logs_level>trace</raft_logs_level>
|
||||||
|
<heart_beat_interval_ms>1000</heart_beat_interval_ms>
|
||||||
|
<election_timeout_lower_bound_ms>2000</election_timeout_lower_bound_ms>
|
||||||
|
<election_timeout_upper_bound_ms>4000</election_timeout_upper_bound_ms>
|
||||||
|
</coordination_settings>
|
||||||
|
|
||||||
|
<raft_configuration>
|
||||||
|
<server>
|
||||||
|
<id>1</id>
|
||||||
|
<hostname>{srv1}</hostname>
|
||||||
|
<port>9444</port>
|
||||||
|
</server>
|
||||||
|
</raft_configuration>
|
||||||
|
</keeper_server>
|
||||||
|
</clickhouse>
|
52
tests/jepsen.clickhouse/resources/replicated_merge_tree.xml
Normal file
52
tests/jepsen.clickhouse/resources/replicated_merge_tree.xml
Normal file
@ -0,0 +1,52 @@
|
|||||||
|
<clickhouse>
|
||||||
|
<default_replica_path>/clickhouse/tables/{database}/{table}</default_replica_path>
|
||||||
|
<default_replica_name>{replica}</default_replica_name>
|
||||||
|
|
||||||
|
<macros>
|
||||||
|
<replica>{replica_name}</replica>
|
||||||
|
</macros>
|
||||||
|
|
||||||
|
<logger>
|
||||||
|
<level>information</level>
|
||||||
|
<log>/var/log/clickhouse-keeper/clickhouse-keeper.log</log>
|
||||||
|
<errorlog>/var/log/clickhouse-keeper/clickhouse-keeper.err.log</errorlog>
|
||||||
|
<size>never</size>
|
||||||
|
</logger>
|
||||||
|
|
||||||
|
<zookeeper>
|
||||||
|
<node>
|
||||||
|
<host>{keeper}</host>
|
||||||
|
<port>9181</port>
|
||||||
|
</node>
|
||||||
|
</zookeeper>
|
||||||
|
|
||||||
|
<remote_servers>
|
||||||
|
<test_cluster>
|
||||||
|
<shard>
|
||||||
|
<replica>
|
||||||
|
<host>{server1}</host>
|
||||||
|
<port>9000</port>
|
||||||
|
</replica>
|
||||||
|
<replica>
|
||||||
|
<host>{server2}</host>
|
||||||
|
<port>9000</port>
|
||||||
|
</replica>
|
||||||
|
<replica>
|
||||||
|
<host>{server3}</host>
|
||||||
|
<port>9000</port>
|
||||||
|
</replica>
|
||||||
|
</shard>
|
||||||
|
</test_cluster>
|
||||||
|
</remote_servers>
|
||||||
|
|
||||||
|
<interserver_http_port>9009</interserver_http_port>
|
||||||
|
|
||||||
|
<distributed_ddl>
|
||||||
|
<path>/clickhouse/task_queue/ddl</path>
|
||||||
|
<profile>default</profile>
|
||||||
|
<pool_size>1</pool_size>
|
||||||
|
<task_max_lifetime>604800</task_max_lifetime>
|
||||||
|
<cleanup_delay_period>60</cleanup_delay_period>
|
||||||
|
<max_tasks_in_queue>1000</max_tasks_in_queue>
|
||||||
|
</distributed_ddl>
|
||||||
|
</clickhouse>
|
16
tests/jepsen.clickhouse/resources/users.xml
Normal file
16
tests/jepsen.clickhouse/resources/users.xml
Normal file
@ -0,0 +1,16 @@
|
|||||||
|
<clickhouse>
|
||||||
|
<profiles>
|
||||||
|
<default>
|
||||||
|
</default>
|
||||||
|
</profiles>
|
||||||
|
|
||||||
|
<users>
|
||||||
|
<default>
|
||||||
|
<password></password>
|
||||||
|
<networks>
|
||||||
|
<ip>::/0</ip>
|
||||||
|
</networks>
|
||||||
|
<profile>default</profile>
|
||||||
|
</default>
|
||||||
|
</users>
|
||||||
|
</clickhouse>
|
21
tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj
Normal file
21
tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj
Normal file
@ -0,0 +1,21 @@
|
|||||||
|
(ns jepsen.clickhouse.constants)
|
||||||
|
|
||||||
|
(def root-folder "/home/robot-clickhouse")
|
||||||
|
|
||||||
|
(def binary-name "clickhouse")
|
||||||
|
|
||||||
|
(def binary-path (str root-folder "/" binary-name))
|
||||||
|
(def pid-file-path (str root-folder "/clickhouse.pid"))
|
||||||
|
|
||||||
|
(def data-dir (str root-folder "/db"))
|
||||||
|
(def logs-dir (str root-folder "/logs"))
|
||||||
|
(def configs-dir (str root-folder "/config"))
|
||||||
|
(def sub-configs-dir (str configs-dir "/config.d"))
|
||||||
|
|
||||||
|
(def coordination-data-dir (str data-dir "/coordination"))
|
||||||
|
(def coordination-snapshots-dir (str coordination-data-dir "/snapshots"))
|
||||||
|
(def coordination-logs-dir (str coordination-data-dir "/logs"))
|
||||||
|
|
||||||
|
(def stderr-file (str logs-dir "/stderr.log"))
|
||||||
|
|
||||||
|
(def binaries-cache-dir (str root-folder "/binaries"))
|
@ -1,4 +1,4 @@
|
|||||||
(ns jepsen.clickhouse-keeper.bench
|
(ns jepsen.clickhouse.keeper.bench
|
||||||
(:require [clojure.tools.logging :refer :all]
|
(:require [clojure.tools.logging :refer :all]
|
||||||
[jepsen
|
[jepsen
|
||||||
[client :as client]])
|
[client :as client]])
|
@ -1,11 +1,12 @@
|
|||||||
(ns jepsen.clickhouse-keeper.counter
|
(ns jepsen.clickhouse.keeper.counter
|
||||||
(:require
|
(:require
|
||||||
[clojure.tools.logging :refer :all]
|
[clojure.tools.logging :refer :all]
|
||||||
[jepsen
|
[jepsen
|
||||||
[checker :as checker]
|
[checker :as checker]
|
||||||
[client :as client]
|
[client :as client]
|
||||||
[generator :as gen]]
|
[generator :as gen]]
|
||||||
[jepsen.clickhouse-keeper.utils :refer :all]
|
[jepsen.clickhouse.keeper.utils :refer :all]
|
||||||
|
[jepsen.clickhouse.utils :as chu]
|
||||||
[zookeeper :as zk])
|
[zookeeper :as zk])
|
||||||
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
||||||
|
|
||||||
@ -22,13 +23,17 @@
|
|||||||
:nodename node))
|
:nodename node))
|
||||||
|
|
||||||
(setup! [this test]
|
(setup! [this test]
|
||||||
(exec-with-retries 30 (fn []
|
(chu/exec-with-retries 30 (fn []
|
||||||
(zk-create-if-not-exists conn root-path ""))))
|
(zk-create-if-not-exists conn root-path ""))))
|
||||||
|
|
||||||
(invoke! [this test op]
|
(invoke! [this test op]
|
||||||
(case (:f op)
|
(case (:f op)
|
||||||
:read (exec-with-retries 30 (fn []
|
:read (try
|
||||||
(assoc op
|
(assoc op
|
||||||
|
:type :ok
|
||||||
|
:value (count (zk-list conn root-path)))
|
||||||
|
(catch Exception _ (assoc op :type :info, :error :connect-error)))
|
||||||
|
:final-read (chu/exec-with-retries 30 (fn [] (assoc op
|
||||||
:type :ok
|
:type :ok
|
||||||
:value (count (zk-list conn root-path)))))
|
:value (count (zk-list conn root-path)))))
|
||||||
:add (try
|
:add (try
|
||||||
@ -49,7 +54,5 @@
|
|||||||
:checker (checker/compose
|
:checker (checker/compose
|
||||||
{:counter (checker/counter)
|
{:counter (checker/counter)
|
||||||
:perf (checker/perf)})
|
:perf (checker/perf)})
|
||||||
:generator (->> (range)
|
:generator (gen/mix [r add])
|
||||||
(map (fn [x]
|
:final-generator (gen/once {:type :invoke, :f :final-read, :value nil})})
|
||||||
(->> (gen/mix [r add])))))
|
|
||||||
:final-generator (gen/once {:type :invoke, :f :read, :value nil})})
|
|
60
tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/db.clj
Normal file
60
tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/db.clj
Normal file
@ -0,0 +1,60 @@
|
|||||||
|
(ns jepsen.clickhouse.keeper.db
|
||||||
|
(:require [clojure.tools.logging :refer :all]
|
||||||
|
[jepsen
|
||||||
|
[control :as c]
|
||||||
|
[util :as util :refer [meh]]]
|
||||||
|
[jepsen.clickhouse.constants :refer :all]
|
||||||
|
[jepsen.clickhouse.keeper.utils :refer :all]
|
||||||
|
[jepsen.clickhouse.utils :as chu]
|
||||||
|
[clojure.java.io :as io]
|
||||||
|
[jepsen.control.util :as cu]
|
||||||
|
[jepsen.os.ubuntu :as ubuntu]))
|
||||||
|
|
||||||
|
|
||||||
|
(ns jepsen.control.scp)
|
||||||
|
|
||||||
|
;; We need to overwrite Jepsen's implementation of scp! because it
|
||||||
|
;; doesn't use strict-host-key-checking
|
||||||
|
|
||||||
|
(defn scp!
|
||||||
|
"Runs an SCP command by shelling out. Takes a conn-spec (used for port, key,
|
||||||
|
etc), a seq of sources, and a single destination, all as strings."
|
||||||
|
[conn-spec sources dest]
|
||||||
|
(apply util/sh "scp" "-rpC"
|
||||||
|
"-P" (str (:port conn-spec))
|
||||||
|
(concat (when-let [k (:private-key-path conn-spec)]
|
||||||
|
["-i" k])
|
||||||
|
(if-not (:strict-host-key-checking conn-spec)
|
||||||
|
["-o StrictHostKeyChecking=no"])
|
||||||
|
sources
|
||||||
|
[dest]))
|
||||||
|
nil)
|
||||||
|
|
||||||
|
(ns jepsen.clickhouse.keeper.db)
|
||||||
|
|
||||||
|
(defn cluster-config
|
||||||
|
[test node config-template]
|
||||||
|
(let [nodes (:nodes test)
|
||||||
|
replacement-map {#"\{srv1\}" (get nodes 0)
|
||||||
|
#"\{srv2\}" (get nodes 1)
|
||||||
|
#"\{srv3\}" (get nodes 2)
|
||||||
|
#"\{id\}" (str (inc (.indexOf nodes node)))
|
||||||
|
#"\{quorum_reads\}" (str (boolean (:quorum test)))
|
||||||
|
#"\{snapshot_distance\}" (str (:snapshot-distance test))
|
||||||
|
#"\{stale_log_gap\}" (str (:stale-log-gap test))
|
||||||
|
#"\{reserved_log_items\}" (str (:reserved-log-items test))}]
|
||||||
|
(reduce #(clojure.string/replace %1 (get %2 0) (get %2 1)) config-template replacement-map)))
|
||||||
|
|
||||||
|
(defn install-configs
|
||||||
|
[test node]
|
||||||
|
(c/exec :echo (cluster-config test node (slurp (io/resource "keeper_config.xml"))) :> (str configs-dir "/keeper_config.xml")))
|
||||||
|
|
||||||
|
(defn extra-setup
|
||||||
|
[test node]
|
||||||
|
(do
|
||||||
|
(info "Installing configs")
|
||||||
|
(install-configs test node)))
|
||||||
|
|
||||||
|
(defn db
|
||||||
|
[version reuse-binary]
|
||||||
|
(chu/db version reuse-binary start-clickhouse! extra-setup))
|
@ -1,112 +1,18 @@
|
|||||||
(ns jepsen.control.sshj
|
(ns jepsen.clickhouse.keeper.main
|
||||||
(:require [jepsen.control [core :as core]
|
|
||||||
[sshj :as sshj]]
|
|
||||||
[slingshot.slingshot :refer [try+ throw+]])
|
|
||||||
(:import (net.schmizz.sshj SSHClient
|
|
||||||
DefaultConfig)
|
|
||||||
(net.schmizz.sshj.transport.verification PromiscuousVerifier)
|
|
||||||
(java.util.concurrent Semaphore)))
|
|
||||||
|
|
||||||
(defrecord SSHJRemote [concurrency-limit
|
|
||||||
conn-spec
|
|
||||||
^SSHClient client
|
|
||||||
^Semaphore semaphore]
|
|
||||||
core/Remote
|
|
||||||
(connect [this conn-spec]
|
|
||||||
(if (:dummy conn-spec)
|
|
||||||
(assoc this :conn-spec conn-spec)
|
|
||||||
(try+ (let [c (as-> (SSHClient.) client
|
|
||||||
(do
|
|
||||||
(if (:strict-host-key-checking conn-spec)
|
|
||||||
(.loadKnownHosts client)
|
|
||||||
(.addHostKeyVerifier client (PromiscuousVerifier.)))
|
|
||||||
(.connect client (:host conn-spec) (:port conn-spec))
|
|
||||||
(auth! client conn-spec)
|
|
||||||
client))]
|
|
||||||
(assoc this
|
|
||||||
:conn-spec conn-spec
|
|
||||||
:client c
|
|
||||||
:semaphore (Semaphore. concurrency-limit true)))
|
|
||||||
(catch Exception e
|
|
||||||
; SSHJ wraps InterruptedException in its own exceptions, so we
|
|
||||||
; have to see through that and rethrow properly.
|
|
||||||
(let [cause (util/ex-root-cause e)]
|
|
||||||
(when (instance? InterruptedException cause)
|
|
||||||
(throw cause)))
|
|
||||||
(throw+ (assoc conn-spec
|
|
||||||
:type :jepsen.control/session-error
|
|
||||||
:message "Error opening SSH session. Verify username, password, and node hostnames are correct."))))))
|
|
||||||
|
|
||||||
(disconnect! [this]
|
|
||||||
(when-let [c client]
|
|
||||||
(.close c)))
|
|
||||||
|
|
||||||
(execute! [this ctx action]
|
|
||||||
; (info :permits (.availablePermits semaphore))
|
|
||||||
(when (:dummy conn-spec)
|
|
||||||
(throw+ {:type :jepsen.control/dummy}))
|
|
||||||
(.acquire semaphore)
|
|
||||||
(sshj/with-errors conn-spec ctx
|
|
||||||
(try
|
|
||||||
(with-open [session (.startSession client)]
|
|
||||||
(let [cmd (.exec session (:cmd action))
|
|
||||||
; Feed it input
|
|
||||||
_ (when-let [input (:in action)]
|
|
||||||
(let [stream (.getOutputStream cmd)]
|
|
||||||
(bs/transfer input stream)
|
|
||||||
(send-eof! client session)
|
|
||||||
(.close stream)))
|
|
||||||
; Read output
|
|
||||||
out (.toString (IOUtils/readFully (.getInputStream cmd)))
|
|
||||||
err (.toString (IOUtils/readFully (.getErrorStream cmd)))
|
|
||||||
; Wait on command
|
|
||||||
_ (.join cmd)]
|
|
||||||
; Return completion
|
|
||||||
(assoc action
|
|
||||||
:out out
|
|
||||||
:err err
|
|
||||||
; There's also a .getExitErrorMessage that might be
|
|
||||||
; interesting here?
|
|
||||||
:exit (.getExitStatus cmd))))
|
|
||||||
(finally
|
|
||||||
(.release semaphore)))))
|
|
||||||
|
|
||||||
(upload! [this ctx local-paths remote-path _opts]
|
|
||||||
(when (:dummy conn-spec)
|
|
||||||
(throw+ {:type :jepsen.control/dummy}))
|
|
||||||
(with-errors conn-spec ctx
|
|
||||||
(with-open [sftp (.newSFTPClient client)]
|
|
||||||
(.put sftp (FileSystemFile. local-paths) remote-path))))
|
|
||||||
|
|
||||||
(download! [this ctx remote-paths local-path _opts]
|
|
||||||
(when (:dummy conn-spec)
|
|
||||||
(throw+ {:type :jepsen.control/dummy}))
|
|
||||||
(with-errors conn-spec ctx
|
|
||||||
(with-open [sftp (.newSFTPClient client)]
|
|
||||||
(.get sftp remote-paths (FileSystemFile. local-path))))))
|
|
||||||
|
|
||||||
(defn remote
|
|
||||||
"Constructs an SSHJ remote."
|
|
||||||
[]
|
|
||||||
(-> (SSHJRemote. concurrency-limit nil nil nil)
|
|
||||||
; We *can* use our own SCP, but shelling out is faster.
|
|
||||||
scp/remote
|
|
||||||
retry/remote))
|
|
||||||
|
|
||||||
(ns jepsen.clickhouse-keeper.main
|
|
||||||
(:require [clojure.tools.logging :refer :all]
|
(:require [clojure.tools.logging :refer :all]
|
||||||
[jepsen.clickhouse-keeper.utils :refer :all]
|
|
||||||
[clojure.pprint :refer [pprint]]
|
[clojure.pprint :refer [pprint]]
|
||||||
[jepsen.clickhouse-keeper.set :as set]
|
[jepsen.clickhouse.keeper.utils :refer :all]
|
||||||
[jepsen.clickhouse-keeper.db :refer :all]
|
[jepsen.clickhouse.keeper.set :as set]
|
||||||
[jepsen.clickhouse-keeper.zookeeperdb :refer :all]
|
[jepsen.clickhouse.keeper.db :refer :all]
|
||||||
[jepsen.clickhouse-keeper.nemesis :as custom-nemesis]
|
[jepsen.clickhouse.keeper.zookeeperdb :refer :all]
|
||||||
[jepsen.clickhouse-keeper.register :as register]
|
[jepsen.clickhouse.keeper.nemesis :as custom-nemesis]
|
||||||
[jepsen.clickhouse-keeper.unique :as unique]
|
[jepsen.clickhouse.keeper.register :as register]
|
||||||
[jepsen.clickhouse-keeper.queue :as queue]
|
[jepsen.clickhouse.keeper.unique :as unique]
|
||||||
[jepsen.clickhouse-keeper.counter :as counter]
|
[jepsen.clickhouse.keeper.queue :as queue]
|
||||||
[jepsen.clickhouse-keeper.bench :as bench]
|
[jepsen.clickhouse.keeper.counter :as counter]
|
||||||
[jepsen.clickhouse-keeper.constants :refer :all]
|
[jepsen.clickhouse.keeper.bench :as bench]
|
||||||
|
[jepsen.clickhouse.constants :refer :all]
|
||||||
|
[jepsen.clickhouse.utils :as chu]
|
||||||
[clojure.string :as str]
|
[clojure.string :as str]
|
||||||
[jepsen
|
[jepsen
|
||||||
[checker :as checker]
|
[checker :as checker]
|
||||||
@ -164,7 +70,7 @@
|
|||||||
:validate [#(and (number? %) (pos? %)) "Must be a positive number"]]
|
:validate [#(and (number? %) (pos? %)) "Must be a positive number"]]
|
||||||
[nil "--ops-per-key NUM" "Maximum number of operations on any given key."
|
[nil "--ops-per-key NUM" "Maximum number of operations on any given key."
|
||||||
:default 100
|
:default 100
|
||||||
:parse-fn parse-long
|
:parse-fn chu/parse-long
|
||||||
:validate [pos? "Must be a positive integer."]]
|
:validate [pos? "Must be a positive integer."]]
|
||||||
[nil, "--lightweight-run" "Subset of workloads/nemesises which is simple to validate"]
|
[nil, "--lightweight-run" "Subset of workloads/nemesises which is simple to validate"]
|
||||||
[nil, "--reuse-binary" "Use already downloaded binary if it exists, don't remove it on shutdown"]
|
[nil, "--reuse-binary" "Use already downloaded binary if it exists, don't remove it on shutdown"]
|
||||||
@ -257,19 +163,12 @@
|
|||||||
"blind-node-partitioner"
|
"blind-node-partitioner"
|
||||||
"blind-others-partitioner"])
|
"blind-others-partitioner"])
|
||||||
|
|
||||||
(defn cart [colls]
|
|
||||||
(if (empty? colls)
|
|
||||||
'(())
|
|
||||||
(for [more (cart (rest colls))
|
|
||||||
x (first colls)]
|
|
||||||
(cons x more))))
|
|
||||||
|
|
||||||
(defn all-test-options
|
(defn all-test-options
|
||||||
"Takes base cli options, a collection of nemeses, workloads, and a test count,
|
"Takes base cli options, a collection of nemeses, workloads, and a test count,
|
||||||
and constructs a sequence of test options."
|
and constructs a sequence of test options."
|
||||||
[cli worload-nemeseis-collection]
|
[cli workload-nemesis-collection]
|
||||||
(take (:test-count cli)
|
(take (:test-count cli)
|
||||||
(shuffle (for [[workload nemesis] worload-nemeseis-collection]
|
(shuffle (for [[workload nemesis] workload-nemesis-collection]
|
||||||
(assoc cli
|
(assoc cli
|
||||||
:nemesis nemesis
|
:nemesis nemesis
|
||||||
:workload workload
|
:workload workload
|
||||||
@ -278,10 +177,10 @@
|
|||||||
"Turns CLI options into a sequence of tests."
|
"Turns CLI options into a sequence of tests."
|
||||||
[test-fn cli]
|
[test-fn cli]
|
||||||
(if (boolean (:lightweight-run cli))
|
(if (boolean (:lightweight-run cli))
|
||||||
(map test-fn (all-test-options cli (cart [lightweight-workloads useful-nemesises])))
|
(map test-fn (all-test-options cli (chu/cart [lightweight-workloads useful-nemesises])))
|
||||||
(map test-fn (all-test-options cli (cart [all-workloads all-nemesises])))))
|
(map test-fn (all-test-options cli (chu/cart [all-workloads all-nemesises])))))
|
||||||
|
|
||||||
(defn -main
|
(defn main
|
||||||
"Handles command line arguments. Can either run a test, or a web server for
|
"Handles command line arguments. Can either run a test, or a web server for
|
||||||
browsing results."
|
browsing results."
|
||||||
[& args]
|
[& args]
|
@ -1,34 +1,14 @@
|
|||||||
(ns jepsen.clickhouse-keeper.nemesis
|
(ns jepsen.clickhouse.keeper.nemesis
|
||||||
(:require
|
(:require
|
||||||
[clojure.tools.logging :refer :all]
|
[clojure.tools.logging :refer :all]
|
||||||
[jepsen
|
[jepsen
|
||||||
[nemesis :as nemesis]
|
[nemesis :as nemesis]
|
||||||
[control :as c]
|
[control :as c]
|
||||||
[generator :as gen]]
|
[generator :as gen]]
|
||||||
[jepsen.clickhouse-keeper.constants :refer :all]
|
[jepsen.clickhouse.nemesis :as chnem]
|
||||||
[jepsen.clickhouse-keeper.utils :refer :all]))
|
[jepsen.clickhouse.constants :refer :all]
|
||||||
|
[jepsen.clickhouse.utils :as chu]
|
||||||
(defn random-node-killer-nemesis
|
[jepsen.clickhouse.keeper.utils :refer :all]))
|
||||||
[]
|
|
||||||
(nemesis/node-start-stopper
|
|
||||||
rand-nth
|
|
||||||
(fn start [test node] (kill-clickhouse! node test))
|
|
||||||
(fn stop [test node] (start-clickhouse! node test))))
|
|
||||||
|
|
||||||
(defn all-nodes-killer-nemesis
|
|
||||||
[]
|
|
||||||
(nemesis/node-start-stopper
|
|
||||||
identity
|
|
||||||
(fn start [test node] (kill-clickhouse! node test))
|
|
||||||
(fn stop [test node] (start-clickhouse! node test))))
|
|
||||||
|
|
||||||
(defn random-node-hammer-time-nemesis
|
|
||||||
[]
|
|
||||||
(nemesis/hammer-time "clickhouse"))
|
|
||||||
|
|
||||||
(defn all-nodes-hammer-time-nemesis
|
|
||||||
[]
|
|
||||||
(nemesis/hammer-time identity "clickhouse"))
|
|
||||||
|
|
||||||
(defn select-last-file
|
(defn select-last-file
|
||||||
[path]
|
[path]
|
||||||
@ -62,7 +42,7 @@
|
|||||||
(c/on-nodes test nodes
|
(c/on-nodes test nodes
|
||||||
(fn [test node]
|
(fn [test node]
|
||||||
(c/su
|
(c/su
|
||||||
(kill-clickhouse! node test)
|
(chu/kill-clickhouse! node test)
|
||||||
(corruption-op path)
|
(corruption-op path)
|
||||||
(start-clickhouse! node test))))
|
(start-clickhouse! node test))))
|
||||||
(assoc op :type :info, :value :corrupted))
|
(assoc op :type :info, :value :corrupted))
|
||||||
@ -93,40 +73,10 @@
|
|||||||
(corruptor-nemesis coordination-data-dir (fn [path]
|
(corruptor-nemesis coordination-data-dir (fn [path]
|
||||||
(c/exec :rm :-fr path))))
|
(c/exec :rm :-fr path))))
|
||||||
|
|
||||||
(defn partition-bridge-nemesis
|
|
||||||
[]
|
|
||||||
(nemesis/partitioner nemesis/bridge))
|
|
||||||
|
|
||||||
(defn blind-node
|
|
||||||
[nodes]
|
|
||||||
(let [[[victim] others] (nemesis/split-one nodes)]
|
|
||||||
{victim (into #{} others)}))
|
|
||||||
|
|
||||||
(defn blind-node-partition-nemesis
|
|
||||||
[]
|
|
||||||
(nemesis/partitioner blind-node))
|
|
||||||
|
|
||||||
(defn blind-others
|
|
||||||
[nodes]
|
|
||||||
(let [[[victim] others] (nemesis/split-one nodes)]
|
|
||||||
(into {} (map (fn [node] [node #{victim}])) others)))
|
|
||||||
|
|
||||||
(defn blind-others-partition-nemesis
|
|
||||||
[]
|
|
||||||
(nemesis/partitioner blind-others))
|
|
||||||
|
|
||||||
(defn network-non-symmetric-nemesis
|
(defn network-non-symmetric-nemesis
|
||||||
[]
|
[]
|
||||||
(nemesis/partitioner nemesis/bridge))
|
(nemesis/partitioner nemesis/bridge))
|
||||||
|
|
||||||
(defn start-stop-generator
|
|
||||||
[time-corrupt time-ok]
|
|
||||||
(->>
|
|
||||||
(cycle [(gen/sleep time-ok)
|
|
||||||
{:type :info, :f :start}
|
|
||||||
(gen/sleep time-corrupt)
|
|
||||||
{:type :info, :f :stop}])))
|
|
||||||
|
|
||||||
(defn corruption-generator
|
(defn corruption-generator
|
||||||
[]
|
[]
|
||||||
(->>
|
(->>
|
||||||
@ -134,16 +84,16 @@
|
|||||||
{:type :info, :f :corrupt}])))
|
{:type :info, :f :corrupt}])))
|
||||||
|
|
||||||
(def custom-nemesises
|
(def custom-nemesises
|
||||||
{"random-node-killer" {:nemesis (random-node-killer-nemesis)
|
{"random-node-killer" {:nemesis (chnem/random-node-killer-nemesis start-clickhouse!)
|
||||||
:generator (start-stop-generator 5 5)}
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
"all-nodes-killer" {:nemesis (all-nodes-killer-nemesis)
|
"all-nodes-killer" {:nemesis (chnem/all-nodes-killer-nemesis start-clickhouse!)
|
||||||
:generator (start-stop-generator 1 10)}
|
:generator (chnem/start-stop-generator 1 10)}
|
||||||
"simple-partitioner" {:nemesis (nemesis/partition-random-halves)
|
"simple-partitioner" {:nemesis (nemesis/partition-random-halves)
|
||||||
:generator (start-stop-generator 5 5)}
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
"random-node-hammer-time" {:nemesis (random-node-hammer-time-nemesis)
|
"random-node-hammer-time" {:nemesis (chnem/random-node-hammer-time-nemesis)
|
||||||
:generator (start-stop-generator 5 5)}
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
"all-nodes-hammer-time" {:nemesis (all-nodes-hammer-time-nemesis)
|
"all-nodes-hammer-time" {:nemesis (chnem/all-nodes-hammer-time-nemesis)
|
||||||
:generator (start-stop-generator 1 10)}
|
:generator (chnem/start-stop-generator 1 10)}
|
||||||
"logs-corruptor" {:nemesis (logs-corruption-nemesis)
|
"logs-corruptor" {:nemesis (logs-corruption-nemesis)
|
||||||
:generator (corruption-generator)}
|
:generator (corruption-generator)}
|
||||||
"snapshots-corruptor" {:nemesis (snapshots-corruption-nemesis)
|
"snapshots-corruptor" {:nemesis (snapshots-corruption-nemesis)
|
||||||
@ -152,9 +102,9 @@
|
|||||||
:generator (corruption-generator)}
|
:generator (corruption-generator)}
|
||||||
"drop-data-corruptor" {:nemesis (drop-all-corruption-nemesis)
|
"drop-data-corruptor" {:nemesis (drop-all-corruption-nemesis)
|
||||||
:generator (corruption-generator)}
|
:generator (corruption-generator)}
|
||||||
"bridge-partitioner" {:nemesis (partition-bridge-nemesis)
|
"bridge-partitioner" {:nemesis (chnem/partition-bridge-nemesis)
|
||||||
:generator (start-stop-generator 5 5)}
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
"blind-node-partitioner" {:nemesis (blind-node-partition-nemesis)
|
"blind-node-partitioner" {:nemesis (chnem/blind-node-partition-nemesis)
|
||||||
:generator (start-stop-generator 5 5)}
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
"blind-others-partitioner" {:nemesis (blind-others-partition-nemesis)
|
"blind-others-partitioner" {:nemesis (chnem/blind-others-partition-nemesis)
|
||||||
:generator (start-stop-generator 5 5)}})
|
:generator (chnem/start-stop-generator 5 5)}})
|
@ -1,4 +1,4 @@
|
|||||||
(ns jepsen.clickhouse-keeper.queue
|
(ns jepsen.clickhouse.keeper.queue
|
||||||
(:require
|
(:require
|
||||||
[clojure.tools.logging :refer :all]
|
[clojure.tools.logging :refer :all]
|
||||||
[jepsen
|
[jepsen
|
||||||
@ -7,7 +7,8 @@
|
|||||||
[generator :as gen]]
|
[generator :as gen]]
|
||||||
[knossos.model :as model]
|
[knossos.model :as model]
|
||||||
[jepsen.checker.timeline :as timeline]
|
[jepsen.checker.timeline :as timeline]
|
||||||
[jepsen.clickhouse-keeper.utils :refer :all]
|
[jepsen.clickhouse.keeper.utils :refer :all]
|
||||||
|
[jepsen.clickhouse.utils :as chu]
|
||||||
[zookeeper :as zk])
|
[zookeeper :as zk])
|
||||||
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
||||||
|
|
||||||
@ -24,7 +25,7 @@
|
|||||||
:nodename node))
|
:nodename node))
|
||||||
|
|
||||||
(setup! [this test]
|
(setup! [this test]
|
||||||
(exec-with-retries 30 (fn []
|
(chu/exec-with-retries 30 (fn []
|
||||||
(zk-create-if-not-exists conn root-path ""))))
|
(zk-create-if-not-exists conn root-path ""))))
|
||||||
|
|
||||||
(invoke! [this test op]
|
(invoke! [this test op]
|
||||||
@ -43,7 +44,7 @@
|
|||||||
(catch Exception _ (assoc op :type :info, :error :connect-error)))
|
(catch Exception _ (assoc op :type :info, :error :connect-error)))
|
||||||
:drain
|
:drain
|
||||||
; drain via delete is to long, just list all nodes
|
; drain via delete is to long, just list all nodes
|
||||||
(exec-with-retries 30 (fn []
|
(chu/exec-with-retries 30 (fn []
|
||||||
(zk-sync conn)
|
(zk-sync conn)
|
||||||
(assoc op :type :ok :value (into #{} (map #(str %1) (zk-list conn root-path))))))))
|
(assoc op :type :ok :value (into #{} (map #(str %1) (zk-list conn root-path))))))))
|
||||||
|
|
@ -1,12 +1,13 @@
|
|||||||
(ns jepsen.clickhouse-keeper.register
|
(ns jepsen.clickhouse.keeper.register
|
||||||
(:require [jepsen
|
(:require [jepsen
|
||||||
[checker :as checker]
|
[checker :as checker]
|
||||||
[client :as client]
|
[client :as client]
|
||||||
[independent :as independent]
|
[independent :as independent]
|
||||||
[generator :as gen]]
|
[generator :as gen]]
|
||||||
[jepsen.checker.timeline :as timeline]
|
[jepsen.checker.timeline :as timeline]
|
||||||
|
[jepsen.clickhouse.utils :as chu]
|
||||||
[knossos.model :as model]
|
[knossos.model :as model]
|
||||||
[jepsen.clickhouse-keeper.utils :refer :all]
|
[jepsen.clickhouse.keeper.utils :refer :all]
|
||||||
[zookeeper :as zk])
|
[zookeeper :as zk])
|
||||||
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
||||||
|
|
||||||
@ -20,14 +21,15 @@
|
|||||||
(assoc this :conn (zk-connect node 9181 30000)))
|
(assoc this :conn (zk-connect node 9181 30000)))
|
||||||
|
|
||||||
(setup! [this test]
|
(setup! [this test]
|
||||||
(zk-create-range conn 300)) ; 300 nodes to be sure
|
(chu/exec-with-retries 30 (fn []
|
||||||
|
(zk-create-range conn 300))))
|
||||||
|
|
||||||
(invoke! [_ test op]
|
(invoke! [_ test op]
|
||||||
(let [[k v] (:value op)
|
(let [[k v] (:value op)
|
||||||
zk-k (zk-path k)]
|
zk-k (zk-path k)]
|
||||||
(case (:f op)
|
(case (:f op)
|
||||||
:read (try
|
:read (try
|
||||||
(assoc op :type :ok, :value (independent/tuple k (parse-long (:data (zk-get-str conn zk-k)))))
|
(assoc op :type :ok, :value (independent/tuple k (chu/parse-long (:data (zk-get-str conn zk-k)))))
|
||||||
(catch Exception _ (assoc op :type :fail, :error :connect-error)))
|
(catch Exception _ (assoc op :type :fail, :error :connect-error)))
|
||||||
:write (try
|
:write (try
|
||||||
(do (zk-set conn zk-k v)
|
(do (zk-set conn zk-k v)
|
@ -1,11 +1,12 @@
|
|||||||
(ns jepsen.clickhouse-keeper.set
|
(ns jepsen.clickhouse.keeper.set
|
||||||
(:require
|
(:require
|
||||||
[clojure.tools.logging :refer :all]
|
[clojure.tools.logging :refer :all]
|
||||||
[jepsen
|
[jepsen
|
||||||
[checker :as checker]
|
[checker :as checker]
|
||||||
[client :as client]
|
[client :as client]
|
||||||
[generator :as gen]]
|
[generator :as gen]]
|
||||||
[jepsen.clickhouse-keeper.utils :refer :all]
|
[jepsen.clickhouse.keeper.utils :refer :all]
|
||||||
|
[jepsen.clickhouse.utils :as chu]
|
||||||
[zookeeper :as zk])
|
[zookeeper :as zk])
|
||||||
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
||||||
|
|
||||||
@ -18,12 +19,12 @@
|
|||||||
:nodename node))
|
:nodename node))
|
||||||
|
|
||||||
(setup! [this test]
|
(setup! [this test]
|
||||||
(exec-with-retries 30 (fn []
|
(chu/exec-with-retries 30 (fn []
|
||||||
(zk-create-if-not-exists conn k "#{}"))))
|
(zk-create-if-not-exists conn k "#{}"))))
|
||||||
|
|
||||||
(invoke! [this test op]
|
(invoke! [this test op]
|
||||||
(case (:f op)
|
(case (:f op)
|
||||||
:read (exec-with-retries 30 (fn []
|
:read (chu/exec-with-retries 30 (fn []
|
||||||
(zk-sync conn)
|
(zk-sync conn)
|
||||||
(assoc op
|
(assoc op
|
||||||
:type :ok
|
:type :ok
|
@ -1,11 +1,11 @@
|
|||||||
(ns jepsen.clickhouse-keeper.unique
|
(ns jepsen.clickhouse.keeper.unique
|
||||||
(:require
|
(:require
|
||||||
[clojure.tools.logging :refer :all]
|
[clojure.tools.logging :refer :all]
|
||||||
[jepsen
|
[jepsen
|
||||||
[checker :as checker]
|
[checker :as checker]
|
||||||
[client :as client]
|
[client :as client]
|
||||||
[generator :as gen]]
|
[generator :as gen]]
|
||||||
[jepsen.clickhouse-keeper.utils :refer :all]
|
[jepsen.clickhouse.keeper.utils :refer :all]
|
||||||
[zookeeper :as zk])
|
[zookeeper :as zk])
|
||||||
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
||||||
|
|
@ -1,35 +1,15 @@
|
|||||||
(ns jepsen.clickhouse-keeper.utils
|
(ns jepsen.clickhouse.keeper.utils
|
||||||
(:require [clojure.string :as str]
|
(:require [clojure.string :as str]
|
||||||
[zookeeper.data :as data]
|
[zookeeper.data :as data]
|
||||||
[zookeeper :as zk]
|
[zookeeper :as zk]
|
||||||
[zookeeper.internal :as zi]
|
[zookeeper.internal :as zi]
|
||||||
[jepsen.control.util :as cu]
|
[jepsen.clickhouse.constants :refer :all]
|
||||||
[jepsen.clickhouse-keeper.constants :refer :all]
|
[jepsen.clickhouse.utils :as chu]
|
||||||
[jepsen.control :as c]
|
[clojure.tools.logging :refer :all])
|
||||||
[clojure.tools.logging :refer :all]
|
|
||||||
[clojure.java.io :as io])
|
|
||||||
(:import (org.apache.zookeeper.data Stat)
|
(:import (org.apache.zookeeper.data Stat)
|
||||||
(org.apache.zookeeper CreateMode
|
(org.apache.zookeeper CreateMode
|
||||||
ZooKeeper)
|
ZooKeeper)
|
||||||
(org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)
|
(org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
||||||
(java.security MessageDigest)))
|
|
||||||
|
|
||||||
(defn exec-with-retries
|
|
||||||
[retries f & args]
|
|
||||||
(let [res (try {:value (apply f args)}
|
|
||||||
(catch Exception e
|
|
||||||
(if (zero? retries)
|
|
||||||
(throw e)
|
|
||||||
{:exception e})))]
|
|
||||||
(if (:exception res)
|
|
||||||
(do (Thread/sleep 1000) (recur (dec retries) f args))
|
|
||||||
(:value res))))
|
|
||||||
|
|
||||||
(defn parse-long
|
|
||||||
"Parses a string to a Long. Passes through `nil` and empty strings."
|
|
||||||
[s]
|
|
||||||
(if (and s (> (count s) 0))
|
|
||||||
(Long/parseLong s)))
|
|
||||||
|
|
||||||
(defn parse-and-get-counter
|
(defn parse-and-get-counter
|
||||||
[path]
|
[path]
|
||||||
@ -45,7 +25,7 @@
|
|||||||
|
|
||||||
(defn zk-connect
|
(defn zk-connect
|
||||||
[host port timeout]
|
[host port timeout]
|
||||||
(exec-with-retries 30 (fn [] (zk/connect (str host ":" port) :timeout-msec timeout))))
|
(zk/connect (str host ":" port) :timeout-msec timeout))
|
||||||
|
|
||||||
(defn zk-create-range
|
(defn zk-create-range
|
||||||
[conn n]
|
[conn n]
|
||||||
@ -77,7 +57,7 @@
|
|||||||
(defn zk-cas
|
(defn zk-cas
|
||||||
[conn path old-value new-value]
|
[conn path old-value new-value]
|
||||||
(let [current-value (zk-get-str conn path)]
|
(let [current-value (zk-get-str conn path)]
|
||||||
(if (= (parse-long (:data current-value)) old-value)
|
(if (= (chu/parse-long (:data current-value)) old-value)
|
||||||
(do (zk-set conn path new-value (:version (:stat current-value)))
|
(do (zk-set conn path new-value (:version (:stat current-value)))
|
||||||
true))))
|
true))))
|
||||||
|
|
||||||
@ -149,34 +129,18 @@
|
|||||||
|
|
||||||
(defn clickhouse-alive?
|
(defn clickhouse-alive?
|
||||||
[node test]
|
[node test]
|
||||||
(info "Checking server alive on" node)
|
(info "Checking Keeper alive on" node)
|
||||||
(try
|
(try
|
||||||
(zk-connect (name node) 9181 30000)
|
(zk-connect (name node) 9181 30000)
|
||||||
(catch Exception _ false)))
|
(catch Exception _ false)))
|
||||||
|
|
||||||
(defn wait-clickhouse-alive!
|
|
||||||
[node test & {:keys [maxtries] :or {maxtries 30}}]
|
|
||||||
(loop [i 0]
|
|
||||||
(cond (> i maxtries) false
|
|
||||||
(clickhouse-alive? node test) true
|
|
||||||
:else (do (Thread/sleep 1000) (recur (inc i))))))
|
|
||||||
|
|
||||||
(defn kill-clickhouse!
|
|
||||||
[node test]
|
|
||||||
(info "Killing server on node" node)
|
|
||||||
(c/su
|
|
||||||
(cu/stop-daemon! binary-path pid-file-path)
|
|
||||||
(c/exec :rm :-fr (str data-dir "/status"))))
|
|
||||||
|
|
||||||
(defn start-clickhouse!
|
(defn start-clickhouse!
|
||||||
[node test]
|
[node test]
|
||||||
(info "Starting server on node" node)
|
(info "Starting server on node" node)
|
||||||
(c/su
|
(chu/start-clickhouse!
|
||||||
(cu/start-daemon!
|
node
|
||||||
{:pidfile pid-file-path
|
test
|
||||||
:logfile stderr-file
|
clickhouse-alive?
|
||||||
:chdir data-dir}
|
|
||||||
binary-path
|
|
||||||
:keeper
|
:keeper
|
||||||
:--config (str configs-dir "/keeper_config.xml")
|
:--config (str configs-dir "/keeper_config.xml")
|
||||||
:--
|
:--
|
||||||
@ -184,27 +148,4 @@
|
|||||||
:--logger.errorlog (str logs-dir "/clickhouse-keeper.err.log")
|
:--logger.errorlog (str logs-dir "/clickhouse-keeper.err.log")
|
||||||
:--keeper_server.snapshot_storage_path coordination-snapshots-dir
|
:--keeper_server.snapshot_storage_path coordination-snapshots-dir
|
||||||
:--keeper_server.log_storage_path coordination-logs-dir
|
:--keeper_server.log_storage_path coordination-logs-dir
|
||||||
:--path coordination-data-dir)
|
:--path coordination-data-dir))
|
||||||
(wait-clickhouse-alive! node test)))
|
|
||||||
|
|
||||||
(defn md5 [^String s]
|
|
||||||
(let [algorithm (MessageDigest/getInstance "MD5")
|
|
||||||
raw (.digest algorithm (.getBytes s))]
|
|
||||||
(format "%032x" (BigInteger. 1 raw))))
|
|
||||||
|
|
||||||
(defn non-precise-cached-wget!
|
|
||||||
[url]
|
|
||||||
(let [encoded-url (md5 url)
|
|
||||||
expected-file-name (.getName (io/file url))
|
|
||||||
dest-folder (str binaries-cache-dir "/" encoded-url)
|
|
||||||
dest-file (str dest-folder "/clickhouse")
|
|
||||||
dest-symlink (str common-prefix "/" expected-file-name)
|
|
||||||
wget-opts (concat cu/std-wget-opts [:-O dest-file])]
|
|
||||||
(when-not (cu/exists? dest-file)
|
|
||||||
(info "Downloading" url)
|
|
||||||
(do (c/exec :mkdir :-p dest-folder)
|
|
||||||
(c/cd dest-folder
|
|
||||||
(cu/wget-helper! wget-opts url))))
|
|
||||||
(c/exec :rm :-rf dest-symlink)
|
|
||||||
(c/exec :ln :-s dest-file dest-symlink)
|
|
||||||
dest-symlink))
|
|
@ -1,6 +1,6 @@
|
|||||||
(ns jepsen.clickhouse-keeper.zookeeperdb
|
(ns jepsen.clickhouse.keeper.zookeeperdb
|
||||||
(:require [clojure.tools.logging :refer :all]
|
(:require [clojure.tools.logging :refer :all]
|
||||||
[jepsen.clickhouse-keeper.utils :refer :all]
|
[jepsen.clickhouse.keeper.utils :refer :all]
|
||||||
[clojure.java.io :as io]
|
[clojure.java.io :as io]
|
||||||
[jepsen
|
[jepsen
|
||||||
[control :as c]
|
[control :as c]
|
106
tests/jepsen.clickhouse/src/jepsen/clickhouse/main.clj
Normal file
106
tests/jepsen.clickhouse/src/jepsen/clickhouse/main.clj
Normal file
@ -0,0 +1,106 @@
|
|||||||
|
(ns jepsen.control.sshj
|
||||||
|
(:require [jepsen.control [core :as core]
|
||||||
|
[sshj :as sshj]]
|
||||||
|
[slingshot.slingshot :refer [try+ throw+]])
|
||||||
|
(:import (net.schmizz.sshj SSHClient
|
||||||
|
DefaultConfig)
|
||||||
|
(net.schmizz.sshj.transport.verification PromiscuousVerifier)
|
||||||
|
(java.util.concurrent Semaphore)))
|
||||||
|
|
||||||
|
(defrecord SSHJRemote [concurrency-limit
|
||||||
|
conn-spec
|
||||||
|
^SSHClient client
|
||||||
|
^Semaphore semaphore]
|
||||||
|
core/Remote
|
||||||
|
(connect [this conn-spec]
|
||||||
|
(if (:dummy conn-spec)
|
||||||
|
(assoc this :conn-spec conn-spec)
|
||||||
|
(try+ (let [c (as-> (SSHClient.) client
|
||||||
|
(do
|
||||||
|
(if (:strict-host-key-checking conn-spec)
|
||||||
|
(.loadKnownHosts client)
|
||||||
|
(.addHostKeyVerifier client (PromiscuousVerifier.)))
|
||||||
|
(.connect client (:host conn-spec) (:port conn-spec))
|
||||||
|
(auth! client conn-spec)
|
||||||
|
client))]
|
||||||
|
(assoc this
|
||||||
|
:conn-spec conn-spec
|
||||||
|
:client c
|
||||||
|
:semaphore (Semaphore. concurrency-limit true)))
|
||||||
|
(catch Exception e
|
||||||
|
; SSHJ wraps InterruptedException in its own exceptions, so we
|
||||||
|
; have to see through that and rethrow properly.
|
||||||
|
(let [cause (util/ex-root-cause e)]
|
||||||
|
(when (instance? InterruptedException cause)
|
||||||
|
(throw cause)))
|
||||||
|
(throw+ (assoc conn-spec
|
||||||
|
:type :jepsen.control/session-error
|
||||||
|
:message "Error opening SSH session. Verify username, password, and node hostnames are correct."))))))
|
||||||
|
|
||||||
|
(disconnect! [this]
|
||||||
|
(when-let [c client]
|
||||||
|
(.close c)))
|
||||||
|
|
||||||
|
(execute! [this ctx action]
|
||||||
|
; (info :permits (.availablePermits semaphore))
|
||||||
|
(when (:dummy conn-spec)
|
||||||
|
(throw+ {:type :jepsen.control/dummy}))
|
||||||
|
(.acquire semaphore)
|
||||||
|
(sshj/with-errors conn-spec ctx
|
||||||
|
(try
|
||||||
|
(with-open [session (.startSession client)]
|
||||||
|
(let [cmd (.exec session (:cmd action))
|
||||||
|
; Feed it input
|
||||||
|
_ (when-let [input (:in action)]
|
||||||
|
(let [stream (.getOutputStream cmd)]
|
||||||
|
(bs/transfer input stream)
|
||||||
|
(send-eof! client session)
|
||||||
|
(.close stream)))
|
||||||
|
; Read output
|
||||||
|
out (.toString (IOUtils/readFully (.getInputStream cmd)))
|
||||||
|
err (.toString (IOUtils/readFully (.getErrorStream cmd)))
|
||||||
|
; Wait on command
|
||||||
|
_ (.join cmd)]
|
||||||
|
; Return completion
|
||||||
|
(assoc action
|
||||||
|
:out out
|
||||||
|
:err err
|
||||||
|
; There's also a .getExitErrorMessage that might be
|
||||||
|
; interesting here?
|
||||||
|
:exit (.getExitStatus cmd))))
|
||||||
|
(finally
|
||||||
|
(.release semaphore)))))
|
||||||
|
|
||||||
|
(upload! [this ctx local-paths remote-path _opts]
|
||||||
|
(when (:dummy conn-spec)
|
||||||
|
(throw+ {:type :jepsen.control/dummy}))
|
||||||
|
(with-errors conn-spec ctx
|
||||||
|
(with-open [sftp (.newSFTPClient client)]
|
||||||
|
(.put sftp (FileSystemFile. local-paths) remote-path))))
|
||||||
|
|
||||||
|
(download! [this ctx remote-paths local-path _opts]
|
||||||
|
(when (:dummy conn-spec)
|
||||||
|
(throw+ {:type :jepsen.control/dummy}))
|
||||||
|
(with-errors conn-spec ctx
|
||||||
|
(with-open [sftp (.newSFTPClient client)]
|
||||||
|
(.get sftp remote-paths (FileSystemFile. local-path))))))
|
||||||
|
|
||||||
|
(defn remote
|
||||||
|
"Constructs an SSHJ remote."
|
||||||
|
[]
|
||||||
|
(-> (SSHJRemote. concurrency-limit nil nil nil)
|
||||||
|
; We *can* use our own SCP, but shelling out is faster.
|
||||||
|
scp/remote
|
||||||
|
retry/remote))
|
||||||
|
|
||||||
|
(ns jepsen.clickhouse.main
|
||||||
|
(:require [jepsen.clickhouse.keeper.main]
|
||||||
|
[jepsen.clickhouse.server.main]))
|
||||||
|
|
||||||
|
(defn -main
|
||||||
|
[f & args]
|
||||||
|
(cond
|
||||||
|
(= f "keeper") (apply jepsen.clickhouse.keeper.main/main args)
|
||||||
|
(= f "server") (apply jepsen.clickhouse.server.main/main args)
|
||||||
|
(some #(= f %) ["test" "test-all"]) (apply jepsen.clickhouse.keeper.main/main f args) ;; backwards compatibility
|
||||||
|
:unknown (throw (Exception. (str "Unknown option specified: " f)))))
|
59
tests/jepsen.clickhouse/src/jepsen/clickhouse/nemesis.clj
Normal file
59
tests/jepsen.clickhouse/src/jepsen/clickhouse/nemesis.clj
Normal file
@ -0,0 +1,59 @@
|
|||||||
|
(ns jepsen.clickhouse.nemesis
|
||||||
|
(:require
|
||||||
|
[clojure.tools.logging :refer :all]
|
||||||
|
[jepsen.clickhouse.utils :as chu]
|
||||||
|
[jepsen
|
||||||
|
[nemesis :as nemesis]
|
||||||
|
[generator :as gen]]))
|
||||||
|
|
||||||
|
(defn random-node-hammer-time-nemesis
|
||||||
|
[]
|
||||||
|
(nemesis/hammer-time "clickhouse"))
|
||||||
|
|
||||||
|
(defn all-nodes-hammer-time-nemesis
|
||||||
|
[]
|
||||||
|
(nemesis/hammer-time identity "clickhouse"))
|
||||||
|
|
||||||
|
(defn start-stop-generator
|
||||||
|
[time-corrupt time-ok]
|
||||||
|
(->>
|
||||||
|
(cycle [(gen/sleep time-ok)
|
||||||
|
{:type :info, :f :start}
|
||||||
|
(gen/sleep time-corrupt)
|
||||||
|
{:type :info, :f :stop}])))
|
||||||
|
|
||||||
|
(defn random-node-killer-nemesis
|
||||||
|
[start-clickhouse!]
|
||||||
|
(nemesis/node-start-stopper
|
||||||
|
rand-nth
|
||||||
|
(fn start [test node] (chu/kill-clickhouse! node test))
|
||||||
|
(fn stop [test node] (start-clickhouse! node test))))
|
||||||
|
|
||||||
|
(defn all-nodes-killer-nemesis
|
||||||
|
[start-clickhouse!]
|
||||||
|
(nemesis/node-start-stopper
|
||||||
|
identity
|
||||||
|
(fn start [test node] (chu/kill-clickhouse! node test))
|
||||||
|
(fn stop [test node] (start-clickhouse! node test))))
|
||||||
|
|
||||||
|
(defn partition-bridge-nemesis
|
||||||
|
[]
|
||||||
|
(nemesis/partitioner nemesis/bridge))
|
||||||
|
|
||||||
|
(defn blind-node
|
||||||
|
[nodes]
|
||||||
|
(let [[[victim] others] (nemesis/split-one nodes)]
|
||||||
|
{victim (into #{} others)}))
|
||||||
|
|
||||||
|
(defn blind-node-partition-nemesis
|
||||||
|
[]
|
||||||
|
(nemesis/partitioner blind-node))
|
||||||
|
|
||||||
|
(defn blind-others
|
||||||
|
[nodes]
|
||||||
|
(let [[[victim] others] (nemesis/split-one nodes)]
|
||||||
|
(into {} (map (fn [node] [node #{victim}])) others)))
|
||||||
|
|
||||||
|
(defn blind-others-partition-nemesis
|
||||||
|
[]
|
||||||
|
(nemesis/partitioner blind-others))
|
@ -0,0 +1,72 @@
|
|||||||
|
(ns jepsen.clickhouse.server.client
|
||||||
|
(:require [clojure.java.jdbc :as j]
|
||||||
|
[clojure.tools.logging :refer :all]
|
||||||
|
[jepsen.util :as util]
|
||||||
|
[jepsen.reconnect :as rc]))
|
||||||
|
|
||||||
|
(def operation-timeout "Default operation timeout in ms" 10000)
|
||||||
|
|
||||||
|
(defn db-spec
|
||||||
|
[node]
|
||||||
|
{:dbtype "clickhouse"
|
||||||
|
:dbname "default"
|
||||||
|
:classname "com.clickhouse.ClickhouseDriver"
|
||||||
|
:host (name node)
|
||||||
|
:port 8123
|
||||||
|
:connectTimeout 30
|
||||||
|
:socketTimeout 30
|
||||||
|
:jdbcCompliant false})
|
||||||
|
|
||||||
|
(defn open-connection
|
||||||
|
[node]
|
||||||
|
(util/timeout 30000
|
||||||
|
(throw (RuntimeException.
|
||||||
|
(str "Connection to " node " timed out")))
|
||||||
|
(util/retry 0.1
|
||||||
|
(let [spec (db-spec node)
|
||||||
|
connection (j/get-connection spec)
|
||||||
|
added-connection (j/add-connection spec connection)]
|
||||||
|
(assert added-connection)
|
||||||
|
added-connection))))
|
||||||
|
|
||||||
|
(defn close-connection
|
||||||
|
"Close connection"
|
||||||
|
[connection]
|
||||||
|
(when-let [c (j/db-find-connection connection)]
|
||||||
|
(.close c))
|
||||||
|
(dissoc connection :connection))
|
||||||
|
|
||||||
|
(defn client
|
||||||
|
"Client JDBC"
|
||||||
|
[node]
|
||||||
|
(rc/open!
|
||||||
|
(rc/wrapper
|
||||||
|
{:name (name node)
|
||||||
|
:open (partial open-connection node)
|
||||||
|
:close close-connection
|
||||||
|
:log? true})))
|
||||||
|
|
||||||
|
(defmacro with-connection
|
||||||
|
"Like jepsen.reconnect/with-conn, but also asserts that the connection has
|
||||||
|
not been closed. If it has, throws an ex-info with :type :conn-not-ready.
|
||||||
|
Delays by 1 second to allow time for the DB to recover."
|
||||||
|
[[c client] final & body]
|
||||||
|
`(do
|
||||||
|
(when ~final
|
||||||
|
(rc/reopen! ~client))
|
||||||
|
(rc/with-conn [~c ~client]
|
||||||
|
(when (.isClosed (j/db-find-connection ~c))
|
||||||
|
(Thread/sleep 1000)
|
||||||
|
(throw (ex-info "Connection not yet ready."
|
||||||
|
{:type :conn-not-ready})))
|
||||||
|
~@body)))
|
||||||
|
|
||||||
|
(defmacro with-exception
|
||||||
|
"Takes an operation and a body. Evaluates body, catches exceptions, and maps
|
||||||
|
them to ops with :type :info and a descriptive :error."
|
||||||
|
[op & body]
|
||||||
|
`(try ~@body
|
||||||
|
(catch Exception e#
|
||||||
|
(if-let [message# (.getMessage e#)]
|
||||||
|
(assoc ~op :type :fail, :error message#)
|
||||||
|
(throw e#)))))
|
185
tests/jepsen.clickhouse/src/jepsen/clickhouse/server/db.clj
Normal file
185
tests/jepsen.clickhouse/src/jepsen/clickhouse/server/db.clj
Normal file
@ -0,0 +1,185 @@
|
|||||||
|
(ns jepsen.control.scp)
|
||||||
|
|
||||||
|
;; We need to overwrite Jepsen's implementation of scp! because it
|
||||||
|
;; doesn't use strict-host-key-checking
|
||||||
|
|
||||||
|
(defn scp!
|
||||||
|
"Runs an SCP command by shelling out. Takes a conn-spec (used for port, key,
|
||||||
|
etc), a seq of sources, and a single destination, all as strings."
|
||||||
|
[conn-spec sources dest]
|
||||||
|
(apply util/sh "scp" "-rpC"
|
||||||
|
"-P" (str (:port conn-spec))
|
||||||
|
(concat (when-let [k (:private-key-path conn-spec)]
|
||||||
|
["-i" k])
|
||||||
|
(if-not (:strict-host-key-checking conn-spec)
|
||||||
|
["-o StrictHostKeyChecking=no"])
|
||||||
|
sources
|
||||||
|
[dest]))
|
||||||
|
nil)
|
||||||
|
|
||||||
|
(ns jepsen.clickhouse.keeper.db)
|
||||||
|
|
||||||
|
(ns jepsen.clickhouse.server.db
|
||||||
|
(:require [clojure.tools.logging :refer :all]
|
||||||
|
[clojure.java.io :as io]
|
||||||
|
[clojure.string :as str]
|
||||||
|
[zookeeper :as zk]
|
||||||
|
[jepsen
|
||||||
|
[control :as c]
|
||||||
|
[store :as store]
|
||||||
|
[core :as core]
|
||||||
|
[os :as os]
|
||||||
|
[db :as db]]
|
||||||
|
[jepsen.control.util :as cu]
|
||||||
|
[jepsen.clickhouse.constants :refer :all]
|
||||||
|
[jepsen.clickhouse.server.utils :refer :all]
|
||||||
|
[jepsen.clickhouse.keeper.utils :as keeperutils]
|
||||||
|
[jepsen.clickhouse.utils :as chu]))
|
||||||
|
|
||||||
|
(defn replicated-merge-tree-config
|
||||||
|
[test node config-template]
|
||||||
|
(let [nodes (:nodes test)
|
||||||
|
replacement-map {#"\{server1\}" (get nodes 0)
|
||||||
|
#"\{server2\}" (get nodes 1)
|
||||||
|
#"\{server3\}" (get nodes 2)
|
||||||
|
#"\{keeper\}" (:keeper test)
|
||||||
|
#"\{replica_name\}" node}]
|
||||||
|
(reduce #(clojure.string/replace %1 (get %2 0) (get %2 1)) config-template replacement-map)))
|
||||||
|
|
||||||
|
(defn install-configs
|
||||||
|
[test node]
|
||||||
|
(c/exec :echo (slurp (io/resource "config.xml")) :> (str configs-dir "/config.xml"))
|
||||||
|
(c/exec :echo (slurp (io/resource "users.xml")) :> (str configs-dir "/users.xml"))
|
||||||
|
(c/exec :echo (replicated-merge-tree-config test node (slurp (io/resource "replicated_merge_tree.xml"))) :> (str sub-configs-dir "/replicated_merge_tree.xml")))
|
||||||
|
|
||||||
|
(defn extra-setup
|
||||||
|
[test node]
|
||||||
|
(do
|
||||||
|
(info "Installing configs")
|
||||||
|
(install-configs test node)))
|
||||||
|
|
||||||
|
(defn keeper-config
|
||||||
|
[test node config-template]
|
||||||
|
(let [replacement-map {#"\{srv1\}" node}]
|
||||||
|
(reduce #(clojure.string/replace %1 (get %2 0) (get %2 1)) config-template replacement-map)))
|
||||||
|
|
||||||
|
(defn install-keeper-configs
|
||||||
|
[test node]
|
||||||
|
(c/exec :echo (keeper-config test node (slurp (io/resource "keeper_config_solo.xml"))) :> (str configs-dir "/keeper_config.xml")))
|
||||||
|
|
||||||
|
(defn keeper
|
||||||
|
[version reuse-binary]
|
||||||
|
(chu/db version reuse-binary keeperutils/start-clickhouse! install-keeper-configs))
|
||||||
|
|
||||||
|
(defn snarf-keeper-logs!
|
||||||
|
"Downloads Keeper logs"
|
||||||
|
[test]
|
||||||
|
; Download logs
|
||||||
|
(let [keeper-node (:keeper test)]
|
||||||
|
(info "Snarfing Keeper log files")
|
||||||
|
(c/on keeper-node
|
||||||
|
(doseq [[remote local] (db/log-files-map (:db test) test keeper-node)]
|
||||||
|
(when (cu/exists? remote)
|
||||||
|
(info "downloading" remote "to" local)
|
||||||
|
(try
|
||||||
|
(c/download
|
||||||
|
remote
|
||||||
|
(.getCanonicalPath
|
||||||
|
(store/path! test (name keeper-node)
|
||||||
|
; strip leading /
|
||||||
|
(str/replace local #"^/" ""))))
|
||||||
|
(catch java.io.IOException e
|
||||||
|
(if (= "Pipe closed" (.getMessage e))
|
||||||
|
(info remote "pipe closed")
|
||||||
|
(throw e)))
|
||||||
|
(catch java.lang.IllegalArgumentException e
|
||||||
|
; This is a jsch bug where the file is just being
|
||||||
|
; created
|
||||||
|
(info remote "doesn't exist"))))))))
|
||||||
|
|
||||||
|
(defn is-primary
|
||||||
|
"Is node primary"
|
||||||
|
[test node]
|
||||||
|
(= 0 (.indexOf (:nodes test) node)))
|
||||||
|
|
||||||
|
(defn zk-connect
|
||||||
|
[host port timeout]
|
||||||
|
(let [conn (zk/connect (str host ":" port) :timeout-msec timeout)
|
||||||
|
sessionId (.getSessionId conn)]
|
||||||
|
(when (= -1 sessionId)
|
||||||
|
(throw (RuntimeException.
|
||||||
|
(str "Connection to " host " failed"))))
|
||||||
|
conn))
|
||||||
|
|
||||||
|
(defn keeper-alive?
|
||||||
|
[node test]
|
||||||
|
(info "Checking Keeper alive on" node)
|
||||||
|
(try
|
||||||
|
(zk-connect (name node) 9181 30000)
|
||||||
|
(catch Exception _ false)))
|
||||||
|
|
||||||
|
(defn db
|
||||||
|
[version reuse-binary]
|
||||||
|
(reify db/DB
|
||||||
|
(setup! [this test node]
|
||||||
|
(let [keeper-node (:keeper test)]
|
||||||
|
(when (is-primary test node)
|
||||||
|
(info (str "Starting Keeper on " keeper-node))
|
||||||
|
(c/on keeper-node
|
||||||
|
(os/setup! (:os test) test keeper-node)
|
||||||
|
(db/setup! (keeper version reuse-binary) test keeper-node)))
|
||||||
|
(c/su
|
||||||
|
(do
|
||||||
|
(info "Preparing directories")
|
||||||
|
(chu/prepare-dirs)
|
||||||
|
(if (or (not (cu/exists? binary-path)) (not reuse-binary))
|
||||||
|
(do (info "Downloading clickhouse")
|
||||||
|
(let [clickhouse-path (chu/download-clickhouse version)]
|
||||||
|
(chu/install-downloaded-clickhouse clickhouse-path)))
|
||||||
|
(info "Binary already exsist on path" binary-path "skipping download"))
|
||||||
|
(extra-setup test node)
|
||||||
|
(info "Waiting for Keeper")
|
||||||
|
(chu/wait-clickhouse-alive! keeper-node test keeper-alive?)
|
||||||
|
(info "Starting server")
|
||||||
|
(start-clickhouse! node test)
|
||||||
|
(info "ClickHouse started")))))
|
||||||
|
|
||||||
|
(teardown! [_ test node]
|
||||||
|
(let [keeper-node (:keeper test)]
|
||||||
|
(when (is-primary test node)
|
||||||
|
(info (str "Tearing down Keeper on " keeper-node))
|
||||||
|
(c/on keeper-node
|
||||||
|
(db/teardown! (keeper version reuse-binary) test keeper-node))
|
||||||
|
(os/teardown! (:os test) test keeper-node)))
|
||||||
|
(info node "Tearing down clickhouse")
|
||||||
|
(c/su
|
||||||
|
(chu/kill-clickhouse! node test)
|
||||||
|
(if (not reuse-binary)
|
||||||
|
(c/exec :rm :-rf binary-path))
|
||||||
|
(c/exec :rm :-rf pid-file-path)
|
||||||
|
(c/exec :rm :-rf data-dir)
|
||||||
|
(c/exec :rm :-rf logs-dir)
|
||||||
|
(c/exec :rm :-rf configs-dir)))
|
||||||
|
|
||||||
|
db/LogFiles
|
||||||
|
(log-files [_ test node]
|
||||||
|
(when (is-primary test node)
|
||||||
|
(info "Downloading Keeper logs")
|
||||||
|
(snarf-keeper-logs! test))
|
||||||
|
(c/su
|
||||||
|
(chu/kill-clickhouse! node test)
|
||||||
|
(if (cu/exists? data-dir)
|
||||||
|
(do
|
||||||
|
(info node "Data folder exists, going to compress")
|
||||||
|
(c/cd root-folder
|
||||||
|
(c/exec :tar :czf "data.tar.gz" "db"))))
|
||||||
|
(if (cu/exists? (str logs-dir))
|
||||||
|
(do
|
||||||
|
(info node "Logs exist, going to compress")
|
||||||
|
(c/cd root-folder
|
||||||
|
(c/exec :tar :czf "logs.tar.gz" "logs"))) (info node "Logs are missing")))
|
||||||
|
(let [common-logs [(str root-folder "/logs.tar.gz") (str root-folder "/data.tar.gz")]
|
||||||
|
gdb-log (str logs-dir "/gdb.log")]
|
||||||
|
(if (cu/exists? (str logs-dir "/gdb.log"))
|
||||||
|
(conj common-logs gdb-log)
|
||||||
|
common-logs)))))
|
114
tests/jepsen.clickhouse/src/jepsen/clickhouse/server/main.clj
Normal file
114
tests/jepsen.clickhouse/src/jepsen/clickhouse/server/main.clj
Normal file
@ -0,0 +1,114 @@
|
|||||||
|
(ns jepsen.clickhouse.server.main
|
||||||
|
(:require [clojure.tools.logging :refer :all]
|
||||||
|
[clojure.pprint :refer [pprint]]
|
||||||
|
[clojure.string :as str]
|
||||||
|
[jepsen
|
||||||
|
[checker :as checker]
|
||||||
|
[cli :as cli]
|
||||||
|
[generator :as gen]
|
||||||
|
[tests :as tests]
|
||||||
|
[util :as util :refer [meh]]]
|
||||||
|
[jepsen.clickhouse.server
|
||||||
|
[db :refer :all]
|
||||||
|
[nemesis :as ch-nemesis]]
|
||||||
|
[jepsen.clickhouse.server
|
||||||
|
[set :as set]]
|
||||||
|
[jepsen.clickhouse.utils :as chu]
|
||||||
|
[jepsen.control.util :as cu]
|
||||||
|
[jepsen.os.ubuntu :as ubuntu]
|
||||||
|
[jepsen.checker.timeline :as timeline]
|
||||||
|
[clojure.java.io :as io])
|
||||||
|
(:import (ch.qos.logback.classic Level)
|
||||||
|
(org.slf4j Logger LoggerFactory)))
|
||||||
|
|
||||||
|
(def workloads
|
||||||
|
"A map of workload names to functions that construct workloads, given opts."
|
||||||
|
{"set" set/workload})
|
||||||
|
|
||||||
|
(def cli-opts
|
||||||
|
"Additional command line options."
|
||||||
|
[["-w" "--workload NAME" "What workload should we run?"
|
||||||
|
:default "set"
|
||||||
|
:validate [workloads (cli/one-of workloads)]]
|
||||||
|
[nil "--keeper ADDRESS", "Address of a Keeper instance"
|
||||||
|
:default ""
|
||||||
|
:validate [#(not-empty %) "Address for Keeper cannot be empty"]]
|
||||||
|
[nil "--nemesis NAME" "Which nemesis will poison our lives?"
|
||||||
|
:default "random-node-killer"
|
||||||
|
:validate [ch-nemesis/custom-nemeses (cli/one-of ch-nemesis/custom-nemeses)]]
|
||||||
|
["-r" "--rate HZ" "Approximate number of requests per second, per thread."
|
||||||
|
:default 10
|
||||||
|
:parse-fn read-string
|
||||||
|
:validate [#(and (number? %) (pos? %)) "Must be a positive number"]]
|
||||||
|
[nil, "--reuse-binary" "Use already downloaded binary if it exists, don't remove it on shutdown"]
|
||||||
|
["-c" "--clickhouse-source URL" "URL for clickhouse deb or tgz package"]])
|
||||||
|
|
||||||
|
(defn get-db
|
||||||
|
[opts]
|
||||||
|
(db (:clickhouse-source opts) (boolean (:reuse-binary opts))))
|
||||||
|
|
||||||
|
(defn clickhouse-func-tests
|
||||||
|
[opts]
|
||||||
|
(info "Test opts\n" (with-out-str (pprint opts)))
|
||||||
|
(let [quorum (boolean (:quorum opts))
|
||||||
|
workload ((get workloads (:workload opts)) opts)
|
||||||
|
current-nemesis (get ch-nemesis/custom-nemeses (:nemesis opts))]
|
||||||
|
(merge tests/noop-test
|
||||||
|
opts
|
||||||
|
{:name (str "clickhouse-server-" (name (:workload opts)) "-" (name (:nemesis opts)))
|
||||||
|
:os ubuntu/os
|
||||||
|
:db (get-db opts)
|
||||||
|
:pure-generators true
|
||||||
|
:nemesis (:nemesis current-nemesis)
|
||||||
|
:client (:client workload)
|
||||||
|
:checker (checker/compose
|
||||||
|
{:perf (checker/perf)
|
||||||
|
:workload (:checker workload)})
|
||||||
|
:generator (gen/phases
|
||||||
|
(->> (:generator workload)
|
||||||
|
(gen/stagger (/ (:rate opts)))
|
||||||
|
(gen/nemesis (:generator current-nemesis))
|
||||||
|
(gen/time-limit (:time-limit opts)))
|
||||||
|
(gen/log "Healing cluster")
|
||||||
|
(gen/nemesis (gen/once {:type :info, :f :stop}))
|
||||||
|
(gen/log "Waiting for recovery")
|
||||||
|
(gen/sleep 10)
|
||||||
|
(gen/clients (:final-generator workload)))})))
|
||||||
|
|
||||||
|
(defn clickhouse-server-test
|
||||||
|
"Given an options map from the command line runner (e.g. :nodes, :ssh,
|
||||||
|
:concurrency, ...), constructs a test map."
|
||||||
|
[opts]
|
||||||
|
(clickhouse-func-tests opts))
|
||||||
|
|
||||||
|
(def all-workloads (keys workloads))
|
||||||
|
|
||||||
|
(def all-nemeses (keys ch-nemesis/custom-nemeses))
|
||||||
|
|
||||||
|
(defn all-test-options
|
||||||
|
"Takes base cli options, a collection of nemeses, workloads, and a test count,
|
||||||
|
and constructs a sequence of test options."
|
||||||
|
[cli workload-nemesis-collection]
|
||||||
|
(take (:test-count cli)
|
||||||
|
(shuffle (for [[workload nemesis] workload-nemesis-collection]
|
||||||
|
(assoc cli
|
||||||
|
:nemesis nemesis
|
||||||
|
:workload workload
|
||||||
|
:test-count 1)))))
|
||||||
|
(defn all-tests
|
||||||
|
"Turns CLI options into a sequence of tests."
|
||||||
|
[test-fn cli]
|
||||||
|
(map test-fn (all-test-options cli (chu/cart [all-workloads all-nemeses]))))
|
||||||
|
|
||||||
|
(defn main
|
||||||
|
"Handles command line arguments. Can either run a test, or a web server for
|
||||||
|
browsing results."
|
||||||
|
[& args]
|
||||||
|
(.setLevel
|
||||||
|
(LoggerFactory/getLogger "org.apache.zookeeper") Level/OFF)
|
||||||
|
(cli/run! (merge (cli/single-test-cmd {:test-fn clickhouse-server-test
|
||||||
|
:opt-spec cli-opts})
|
||||||
|
(cli/test-all-cmd {:tests-fn (partial all-tests clickhouse-server-test)
|
||||||
|
:opt-spec cli-opts})
|
||||||
|
(cli/serve-cmd))
|
||||||
|
args))
|
@ -0,0 +1,22 @@
|
|||||||
|
(ns jepsen.clickhouse.server.nemesis
|
||||||
|
(:require [jepsen.clickhouse.nemesis :as chnem]
|
||||||
|
[jepsen.clickhouse.server.utils :refer :all]
|
||||||
|
[jepsen.nemesis :as nemesis]))
|
||||||
|
|
||||||
|
(def custom-nemeses
|
||||||
|
{"random-node-killer" {:nemesis (chnem/random-node-killer-nemesis start-clickhouse!)
|
||||||
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
|
"all-nodes-killer" {:nemesis (chnem/all-nodes-killer-nemesis start-clickhouse!)
|
||||||
|
:generator (chnem/start-stop-generator 1 10)}
|
||||||
|
"simple-partitioner" {:nemesis (nemesis/partition-random-halves)
|
||||||
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
|
"random-node-hammer-time" {:nemesis (chnem/random-node-hammer-time-nemesis)
|
||||||
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
|
"all-nodes-hammer-time" {:nemesis (chnem/all-nodes-hammer-time-nemesis)
|
||||||
|
:generator (chnem/start-stop-generator 1 10)}
|
||||||
|
"bridge-partitioner" {:nemesis (chnem/partition-bridge-nemesis)
|
||||||
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
|
"blind-node-partitioner" {:nemesis (chnem/blind-node-partition-nemesis)
|
||||||
|
:generator (chnem/start-stop-generator 5 5)}
|
||||||
|
"blind-others-partitioner" {:nemesis (chnem/blind-others-partition-nemesis)
|
||||||
|
:generator (chnem/start-stop-generator 5 5)}})
|
51
tests/jepsen.clickhouse/src/jepsen/clickhouse/server/set.clj
Normal file
51
tests/jepsen.clickhouse/src/jepsen/clickhouse/server/set.clj
Normal file
@ -0,0 +1,51 @@
|
|||||||
|
(ns jepsen.clickhouse.server.set
|
||||||
|
(:require
|
||||||
|
[clojure.tools.logging :refer :all]
|
||||||
|
[clojure.java.jdbc :as j]
|
||||||
|
[jepsen
|
||||||
|
[util :as util]
|
||||||
|
[reconnect :as rc]
|
||||||
|
[checker :as checker]
|
||||||
|
[client :as client]
|
||||||
|
[generator :as gen]]
|
||||||
|
[jepsen.clickhouse.server.client :as chc]
|
||||||
|
[jepsen.clickhouse.utils :as chu]))
|
||||||
|
|
||||||
|
(defrecord SetClient [table-created? conn]
|
||||||
|
client/Client
|
||||||
|
(open! [this test node]
|
||||||
|
(assoc this :conn (chc/client node)))
|
||||||
|
|
||||||
|
(setup! [this test]
|
||||||
|
(locking table-created?
|
||||||
|
(when (compare-and-set! table-created? false true)
|
||||||
|
(chc/with-connection [c conn] false
|
||||||
|
(j/query c "DROP TABLE IF EXISTS set ON CLUSTER test_cluster")
|
||||||
|
(j/query c "CREATE TABLE set ON CLUSTER test_cluster (value Int64) Engine=ReplicatedMergeTree ORDER BY value")))))
|
||||||
|
|
||||||
|
(invoke! [this test op]
|
||||||
|
(chc/with-exception op
|
||||||
|
(chc/with-connection [c conn] (= :read (:f op))
|
||||||
|
(case (:f op)
|
||||||
|
:add (do
|
||||||
|
(j/query c (str "INSERT INTO set VALUES (" (:value op) ")"))
|
||||||
|
(assoc op :type :ok))
|
||||||
|
:read (->> (j/query c "SELECT value FROM set")
|
||||||
|
(mapv :value)
|
||||||
|
(assoc op :type :ok, :value))))))
|
||||||
|
|
||||||
|
(teardown! [_ test])
|
||||||
|
|
||||||
|
(close! [_ test]
|
||||||
|
(rc/close! conn)))
|
||||||
|
|
||||||
|
(defn workload
|
||||||
|
"A generator, client, and checker for a set test."
|
||||||
|
[opts]
|
||||||
|
{:client (SetClient. (atom false) nil)
|
||||||
|
:checker (checker/compose
|
||||||
|
{:set (checker/set)
|
||||||
|
:perf (checker/perf)})
|
||||||
|
:generator (->> (range)
|
||||||
|
(map (fn [x] {:type :invoke, :f :add, :value x})))
|
||||||
|
:final-generator (gen/once {:type :invoke, :f :read, :value nil})})
|
@ -0,0 +1,27 @@
|
|||||||
|
(ns jepsen.clickhouse.server.utils
|
||||||
|
(:require [jepsen.clickhouse.utils :as chu]
|
||||||
|
[jepsen.clickhouse.constants :refer :all]
|
||||||
|
[jepsen.clickhouse.server.client :as chc]
|
||||||
|
[clojure.tools.logging :refer :all]
|
||||||
|
[clojure.java.jdbc :as jdbc]))
|
||||||
|
|
||||||
|
(defn clickhouse-alive?
|
||||||
|
[node test]
|
||||||
|
(try
|
||||||
|
(let [c (chc/open-connection node)]
|
||||||
|
(jdbc/query c "SELECT 1")
|
||||||
|
(chc/close-connection c))
|
||||||
|
(catch Exception e false)))
|
||||||
|
|
||||||
|
(defn start-clickhouse!
|
||||||
|
[node test]
|
||||||
|
(chu/start-clickhouse!
|
||||||
|
node
|
||||||
|
test
|
||||||
|
clickhouse-alive?
|
||||||
|
:server
|
||||||
|
:--config (str configs-dir "/config.xml")
|
||||||
|
:--
|
||||||
|
:--logger.log (str logs-dir "/clickhouse.log")
|
||||||
|
:--logger.errorlog (str logs-dir "/clickhouse.err.log")
|
||||||
|
:--path data-dir))
|
202
tests/jepsen.clickhouse/src/jepsen/clickhouse/utils.clj
Normal file
202
tests/jepsen.clickhouse/src/jepsen/clickhouse/utils.clj
Normal file
@ -0,0 +1,202 @@
|
|||||||
|
(ns jepsen.clickhouse.utils
|
||||||
|
(:require [jepsen.control.util :as cu]
|
||||||
|
[jepsen
|
||||||
|
[control :as c]
|
||||||
|
[db :as db]]
|
||||||
|
[jepsen.clickhouse.constants :refer :all]
|
||||||
|
[clojure.tools.logging :refer :all]
|
||||||
|
[clojure.java.io :as io])
|
||||||
|
(:import (java.security MessageDigest)))
|
||||||
|
|
||||||
|
(defn exec-with-retries
|
||||||
|
[retries f & args]
|
||||||
|
(let [res (try {:value (apply f args)}
|
||||||
|
(catch Exception e
|
||||||
|
(if (zero? retries)
|
||||||
|
(throw e)
|
||||||
|
{:exception e})))]
|
||||||
|
(if (:exception res)
|
||||||
|
(do (Thread/sleep 1000) (recur (dec retries) f args))
|
||||||
|
(:value res))))
|
||||||
|
|
||||||
|
(defn parse-long
|
||||||
|
"Parses a string to a Long. Passes through `nil` and empty strings."
|
||||||
|
[s]
|
||||||
|
(if (and s (> (count s) 0))
|
||||||
|
(Long/parseLong s)))
|
||||||
|
|
||||||
|
(defn cart [colls]
|
||||||
|
(if (empty? colls)
|
||||||
|
'(())
|
||||||
|
(for [more (cart (rest colls))
|
||||||
|
x (first colls)]
|
||||||
|
(cons x more))))
|
||||||
|
|
||||||
|
(defn md5 [^String s]
|
||||||
|
(let [algorithm (MessageDigest/getInstance "MD5")
|
||||||
|
raw (.digest algorithm (.getBytes s))]
|
||||||
|
(format "%032x" (BigInteger. 1 raw))))
|
||||||
|
|
||||||
|
(defn non-precise-cached-wget!
|
||||||
|
[url]
|
||||||
|
(let [encoded-url (md5 url)
|
||||||
|
expected-file-name (.getName (io/file url))
|
||||||
|
dest-folder (str binaries-cache-dir "/" encoded-url)
|
||||||
|
dest-file (str dest-folder "/clickhouse")
|
||||||
|
dest-symlink (str root-folder "/" expected-file-name)
|
||||||
|
wget-opts (concat cu/std-wget-opts [:-O dest-file])]
|
||||||
|
(when-not (cu/exists? dest-file)
|
||||||
|
(info "Downloading" url)
|
||||||
|
(do (c/exec :mkdir :-p dest-folder)
|
||||||
|
(c/cd dest-folder
|
||||||
|
(cu/wget-helper! wget-opts url))))
|
||||||
|
(c/exec :rm :-rf dest-symlink)
|
||||||
|
(c/exec :ln :-s dest-file dest-symlink)
|
||||||
|
dest-symlink))
|
||||||
|
|
||||||
|
(defn get-clickhouse-url
|
||||||
|
[url]
|
||||||
|
(non-precise-cached-wget! url))
|
||||||
|
|
||||||
|
(defn get-clickhouse-scp
|
||||||
|
[path]
|
||||||
|
(c/upload path (str root-folder "/clickhouse")))
|
||||||
|
|
||||||
|
(defn download-clickhouse
|
||||||
|
[source]
|
||||||
|
(info "Downloading clickhouse from" source)
|
||||||
|
(cond
|
||||||
|
(clojure.string/starts-with? source "http") (get-clickhouse-url source)
|
||||||
|
(.exists (io/file source)) (get-clickhouse-scp source root-folder)
|
||||||
|
:else (throw (Exception. (str "Don't know how to download clickhouse from" source)))))
|
||||||
|
|
||||||
|
(defn unpack-deb
|
||||||
|
[path]
|
||||||
|
(do
|
||||||
|
(c/exec :dpkg :-x path root-folder)
|
||||||
|
(c/exec :rm :-f path)
|
||||||
|
(c/exec :mv (str root-folder "/usr/bin/clickhouse") root-folder)
|
||||||
|
(c/exec :rm :-rf (str root-folder "/usr") (str root-folder "/etc"))))
|
||||||
|
|
||||||
|
(defn unpack-tgz
|
||||||
|
[path]
|
||||||
|
(do
|
||||||
|
(c/exec :mkdir :-p (str root-folder "/unpacked"))
|
||||||
|
(c/exec :tar :-zxvf path :-C (str root-folder "/unpacked"))
|
||||||
|
(c/exec :rm :-f path)
|
||||||
|
(let [subdir (c/exec :ls (str root-folder "/unpacked"))]
|
||||||
|
(c/exec :mv (str root-folder "/unpacked/" subdir "/usr/bin/clickhouse") root-folder)
|
||||||
|
(c/exec :rm :-fr (str root-folder "/unpacked")))))
|
||||||
|
|
||||||
|
(defn chmod-binary
|
||||||
|
[path]
|
||||||
|
(info "Binary path chmod" path)
|
||||||
|
(c/exec :chmod :+x path))
|
||||||
|
|
||||||
|
(defn install-downloaded-clickhouse
|
||||||
|
[path]
|
||||||
|
(cond
|
||||||
|
(clojure.string/ends-with? path ".deb") (unpack-deb path root-folder)
|
||||||
|
(clojure.string/ends-with? path ".tgz") (unpack-tgz path root-folder)
|
||||||
|
(clojure.string/ends-with? path "clickhouse") (chmod-binary path)
|
||||||
|
:else (throw (Exception. (str "Don't know how to install clickhouse from path" path)))))
|
||||||
|
|
||||||
|
(defn collect-traces
|
||||||
|
[test node]
|
||||||
|
(let [pid (c/exec :pidof "clickhouse")]
|
||||||
|
(c/exec :timeout :-s "KILL" "60" :gdb :-ex "set pagination off" :-ex (str "set logging file " logs-dir "/gdb.log") :-ex
|
||||||
|
"set logging on" :-ex "backtrace" :-ex "thread apply all backtrace"
|
||||||
|
:-ex "backtrace" :-ex "detach" :-ex "quit" :--pid pid :|| :true)))
|
||||||
|
|
||||||
|
(defn wait-clickhouse-alive!
|
||||||
|
[node test clickhouse-alive? & {:keys [maxtries] :or {maxtries 30}}]
|
||||||
|
(loop [i 0]
|
||||||
|
(cond (> i maxtries) false
|
||||||
|
(clickhouse-alive? node test) true
|
||||||
|
:else (do (Thread/sleep 1000) (recur (inc i))))))
|
||||||
|
|
||||||
|
(defn kill-clickhouse!
|
||||||
|
[node test]
|
||||||
|
(info "Killing server on node" node)
|
||||||
|
(c/su
|
||||||
|
(cu/stop-daemon! binary-path pid-file-path)
|
||||||
|
(c/exec :rm :-fr (str data-dir "/status"))))
|
||||||
|
|
||||||
|
(defn start-clickhouse!
|
||||||
|
[node test clickhouse-alive? & binary-args]
|
||||||
|
(info "Starting server on node" node)
|
||||||
|
(c/su
|
||||||
|
(cu/start-daemon!
|
||||||
|
{:pidfile pid-file-path
|
||||||
|
:logfile stderr-file
|
||||||
|
:chdir data-dir}
|
||||||
|
binary-path
|
||||||
|
binary-args)
|
||||||
|
(info "Waiting for server")
|
||||||
|
(wait-clickhouse-alive! node test clickhouse-alive?)))
|
||||||
|
|
||||||
|
(defn prepare-dirs
|
||||||
|
[]
|
||||||
|
(do
|
||||||
|
(c/exec :mkdir :-p root-folder)
|
||||||
|
(c/exec :mkdir :-p data-dir)
|
||||||
|
(c/exec :mkdir :-p coordination-data-dir)
|
||||||
|
(c/exec :mkdir :-p logs-dir)
|
||||||
|
(c/exec :mkdir :-p configs-dir)
|
||||||
|
(c/exec :mkdir :-p sub-configs-dir)
|
||||||
|
(c/exec :touch stderr-file)
|
||||||
|
(c/exec :chown :-R :root root-folder)))
|
||||||
|
|
||||||
|
(defn db
|
||||||
|
[version reuse-binary start-clickhouse! extra-setup]
|
||||||
|
(reify db/DB
|
||||||
|
(setup! [_ test node]
|
||||||
|
(c/su
|
||||||
|
(do
|
||||||
|
(info "Preparing directories")
|
||||||
|
(prepare-dirs)
|
||||||
|
(if (or (not (cu/exists? binary-path)) (not reuse-binary))
|
||||||
|
(do (info "Downloading clickhouse")
|
||||||
|
(let [clickhouse-path (download-clickhouse version)]
|
||||||
|
(install-downloaded-clickhouse clickhouse-path)))
|
||||||
|
(info "Binary already exsist on path" binary-path "skipping download"))
|
||||||
|
(extra-setup test node)
|
||||||
|
(info "Starting server")
|
||||||
|
(start-clickhouse! node test)
|
||||||
|
(info "ClickHouse started"))))
|
||||||
|
|
||||||
|
(teardown! [_ test node]
|
||||||
|
(info node "Tearing down clickhouse")
|
||||||
|
(c/su
|
||||||
|
(kill-clickhouse! node test)
|
||||||
|
(if (not reuse-binary)
|
||||||
|
(c/exec :rm :-rf binary-path))
|
||||||
|
(c/exec :rm :-rf pid-file-path)
|
||||||
|
(c/exec :rm :-rf data-dir)
|
||||||
|
(c/exec :rm :-rf logs-dir)
|
||||||
|
(c/exec :rm :-rf configs-dir)))
|
||||||
|
|
||||||
|
db/LogFiles
|
||||||
|
(log-files [_ test node]
|
||||||
|
(c/su
|
||||||
|
;(if (cu/exists? pid-file-path)
|
||||||
|
;(do
|
||||||
|
; (info node "Collecting traces")
|
||||||
|
; (collect-traces test node logs-dir))
|
||||||
|
;(info node "Pid files doesn't exists"))
|
||||||
|
(kill-clickhouse! node test)
|
||||||
|
(if (cu/exists? coordination-data-dir)
|
||||||
|
(do
|
||||||
|
(info node "Coordination files exists, going to compress")
|
||||||
|
(c/cd data-dir
|
||||||
|
(c/exec :tar :czf "coordination.tar.gz" "coordination"))))
|
||||||
|
(if (cu/exists? (str logs-dir))
|
||||||
|
(do
|
||||||
|
(info node "Logs exist, going to compress")
|
||||||
|
(c/cd root-folder
|
||||||
|
(c/exec :tar :czf "logs.tar.gz" "logs"))) (info node "Logs are missing")))
|
||||||
|
(let [common-logs [(str root-folder "/logs.tar.gz") (str data-dir "/coordination.tar.gz")]
|
||||||
|
gdb-log (str logs-dir "/gdb.log")]
|
||||||
|
(if (cu/exists? (str logs-dir "/gdb.log"))
|
||||||
|
(conj common-logs gdb-log)
|
||||||
|
common-logs)))))
|
Loading…
Reference in New Issue
Block a user