2021-03-29 08:24:56 +00:00
|
|
|
(ns jepsen.clickhouse-keeper.unique
|
2021-03-17 12:58:01 +00:00
|
|
|
(:require
|
|
|
|
[clojure.tools.logging :refer :all]
|
|
|
|
[jepsen
|
|
|
|
[checker :as checker]
|
|
|
|
[client :as client]
|
|
|
|
[generator :as gen]]
|
2021-03-29 08:24:56 +00:00
|
|
|
[jepsen.clickhouse-keeper.utils :refer :all]
|
2021-03-17 12:58:01 +00:00
|
|
|
[zookeeper :as zk])
|
|
|
|
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
|
|
|
|
|
|
|
|
(defrecord UniqueClient [conn nodename]
|
|
|
|
client/Client
|
|
|
|
(open! [this test node]
|
|
|
|
(assoc
|
|
|
|
(assoc this
|
|
|
|
:conn (zk-connect node 9181 30000))
|
|
|
|
:nodename node))
|
|
|
|
|
|
|
|
(setup! [this test])
|
|
|
|
|
|
|
|
(invoke! [this test op]
|
|
|
|
(case
|
2021-03-17 13:00:08 +00:00
|
|
|
:generate
|
|
|
|
(try
|
|
|
|
(let [result-path (zk-create-sequential conn "/seq-" "")]
|
|
|
|
(assoc op :type :ok :value (parse-and-get-counter result-path)))
|
|
|
|
(catch Exception _ (assoc op :type :info, :error :connect-error)))))
|
2021-03-17 12:58:01 +00:00
|
|
|
|
|
|
|
(teardown! [_ test])
|
|
|
|
|
2021-03-18 20:55:11 +00:00
|
|
|
(close! [_ test]
|
|
|
|
(zk/close conn)))
|
2021-03-17 12:58:01 +00:00
|
|
|
|
|
|
|
(defn workload
|
|
|
|
"A generator, client, and checker for a set test."
|
|
|
|
[opts]
|
|
|
|
{:client (UniqueClient. nil nil)
|
|
|
|
:checker (checker/unique-ids)
|
|
|
|
:generator (->>
|
|
|
|
(range)
|
|
|
|
(map (fn [_] {:type :invoke, :f :generate})))})
|