Add support for testing different products

This commit is contained in:
Antonio Andelic 2022-10-24 08:12:24 +00:00
parent 729692f862
commit ec8d5b0ddd
16 changed files with 152 additions and 135 deletions

View File

@ -1,10 +1,10 @@
(defproject jepsen.keeper "0.1.0-SNAPSHOT"
(defproject jepsen.clickhouse "0.1.0-SNAPSHOT"
:injections [(.. System (setProperty "zookeeper.request.timeout" "10000"))]
:description "A jepsen tests for ClickHouse Keeper"
:description "A jepsen tests for ClickHouse"
:url "https://clickhouse.com/"
:license {:name "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"]]
:dependencies [[org.clojure/clojure "1.10.1"]
[jepsen "0.2.7"]

View File

@ -1,5 +1,5 @@
<clickhouse>
<listen_host>::</listen_host>
<listen_host>0.0.0.0</listen_host>
<logger>
<level>trace</level>

View File

@ -1,4 +1,4 @@
(ns jepsen.clickhouse-keeper.bench
(ns jepsen.clickhouse.keeper.bench
(:require [clojure.tools.logging :refer :all]
[jepsen
[client :as client]])

View File

@ -1,4 +1,4 @@
(ns jepsen.clickhouse-keeper.constants)
(ns jepsen.clickhouse.keeper.constants)
(def common-prefix "/home/robot-clickhouse")

View File

@ -1,11 +1,11 @@
(ns jepsen.clickhouse-keeper.counter
(ns jepsen.clickhouse.keeper.counter
(:require
[clojure.tools.logging :refer :all]
[jepsen
[checker :as checker]
[client :as client]
[generator :as gen]]
[jepsen.clickhouse-keeper.utils :refer :all]
[jepsen.clickhouse.keeper.utils :refer :all]
[zookeeper :as zk])
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))

View File

@ -1,11 +1,11 @@
(ns jepsen.clickhouse-keeper.db
(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]
[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]))
@ -30,7 +30,7 @@
[dest]))
nil)
(ns jepsen.clickhouse-keeper.db)
(ns jepsen.clickhouse.keeper.db)
(defn get-clickhouse-url
[url]

View File

@ -1,112 +1,17 @@
(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-keeper.main
(ns jepsen.clickhouse.keeper.main
(:require [clojure.tools.logging :refer :all]
[jepsen.clickhouse-keeper.utils :refer :all]
[clojure.pprint :refer [pprint]]
[jepsen.clickhouse-keeper.set :as set]
[jepsen.clickhouse-keeper.db :refer :all]
[jepsen.clickhouse-keeper.zookeeperdb :refer :all]
[jepsen.clickhouse-keeper.nemesis :as custom-nemesis]
[jepsen.clickhouse-keeper.register :as register]
[jepsen.clickhouse-keeper.unique :as unique]
[jepsen.clickhouse-keeper.queue :as queue]
[jepsen.clickhouse-keeper.counter :as counter]
[jepsen.clickhouse-keeper.bench :as bench]
[jepsen.clickhouse-keeper.constants :refer :all]
[jepsen.clickhouse.keeper.utils :refer :all]
[jepsen.clickhouse.keeper.set :as set]
[jepsen.clickhouse.keeper.db :refer :all]
[jepsen.clickhouse.keeper.zookeeperdb :refer :all]
[jepsen.clickhouse.keeper.nemesis :as custom-nemesis]
[jepsen.clickhouse.keeper.register :as register]
[jepsen.clickhouse.keeper.unique :as unique]
[jepsen.clickhouse.keeper.queue :as queue]
[jepsen.clickhouse.keeper.counter :as counter]
[jepsen.clickhouse.keeper.bench :as bench]
[jepsen.clickhouse.keeper.constants :refer :all]
[clojure.string :as str]
[jepsen
[checker :as checker]
@ -281,7 +186,7 @@
(map test-fn (all-test-options cli (cart [lightweight-workloads useful-nemesises])))
(map test-fn (all-test-options cli (cart [all-workloads all-nemesises])))))
(defn -main
(defn main
"Handles command line arguments. Can either run a test, or a web server for
browsing results."
[& args]

View File

@ -1,12 +1,12 @@
(ns jepsen.clickhouse-keeper.nemesis
(ns jepsen.clickhouse.keeper.nemesis
(:require
[clojure.tools.logging :refer :all]
[jepsen
[nemesis :as nemesis]
[control :as c]
[generator :as gen]]
[jepsen.clickhouse-keeper.constants :refer :all]
[jepsen.clickhouse-keeper.utils :refer :all]))
[jepsen.clickhouse.keeper.constants :refer :all]
[jepsen.clickhouse.keeper.utils :refer :all]))
(defn random-node-killer-nemesis
[]

View File

@ -1,4 +1,4 @@
(ns jepsen.clickhouse-keeper.queue
(ns jepsen.clickhouse.keeper.queue
(:require
[clojure.tools.logging :refer :all]
[jepsen
@ -7,7 +7,7 @@
[generator :as gen]]
[knossos.model :as model]
[jepsen.checker.timeline :as timeline]
[jepsen.clickhouse-keeper.utils :refer :all]
[jepsen.clickhouse.keeper.utils :refer :all]
[zookeeper :as zk])
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))

View File

@ -1,4 +1,4 @@
(ns jepsen.clickhouse-keeper.register
(ns jepsen.clickhouse.keeper.register
(:require [jepsen
[checker :as checker]
[client :as client]
@ -6,7 +6,7 @@
[generator :as gen]]
[jepsen.checker.timeline :as timeline]
[knossos.model :as model]
[jepsen.clickhouse-keeper.utils :refer :all]
[jepsen.clickhouse.keeper.utils :refer :all]
[zookeeper :as zk])
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))

View File

@ -1,11 +1,11 @@
(ns jepsen.clickhouse-keeper.set
(ns jepsen.clickhouse.keeper.set
(:require
[clojure.tools.logging :refer :all]
[jepsen
[checker :as checker]
[client :as client]
[generator :as gen]]
[jepsen.clickhouse-keeper.utils :refer :all]
[jepsen.clickhouse.keeper.utils :refer :all]
[zookeeper :as zk])
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))

View File

@ -1,11 +1,11 @@
(ns jepsen.clickhouse-keeper.unique
(ns jepsen.clickhouse.keeper.unique
(:require
[clojure.tools.logging :refer :all]
[jepsen
[checker :as checker]
[client :as client]
[generator :as gen]]
[jepsen.clickhouse-keeper.utils :refer :all]
[jepsen.clickhouse.keeper.utils :refer :all]
[zookeeper :as zk])
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))

View File

@ -1,10 +1,10 @@
(ns jepsen.clickhouse-keeper.utils
(ns jepsen.clickhouse.keeper.utils
(:require [clojure.string :as str]
[zookeeper.data :as data]
[zookeeper :as zk]
[zookeeper.internal :as zi]
[jepsen.control.util :as cu]
[jepsen.clickhouse-keeper.constants :refer :all]
[jepsen.clickhouse.keeper.constants :refer :all]
[jepsen.control :as c]
[clojure.tools.logging :refer :all]
[clojure.java.io :as io])

View File

@ -1,6 +1,6 @@
(ns jepsen.clickhouse-keeper.zookeeperdb
(ns jepsen.clickhouse.keeper.zookeeperdb
(:require [clojure.tools.logging :refer :all]
[jepsen.clickhouse-keeper.utils :refer :all]
[jepsen.clickhouse.keeper.utils :refer :all]
[clojure.java.io :as io]
[jepsen
[control :as c]

View File

@ -0,0 +1,105 @@
(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
[product & args]
(case product
"keeper" (apply jepsen.clickhouse.keeper.main/main args)
"server" (apply jepsen.clickhouse.server.main/main args)
(throw (Exception. (str "Unknown option specified: " product)))))

View File

@ -0,0 +1,7 @@
(ns jepsen.clickhouse.server.main)
(defn main
"Handles command line arguments. Can either run a test, or a web server for
browsing results."
[& args]
(print "Not yet implemented" args))