ClickHouse/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/set.clj

51 lines
1.6 KiB
Clojure
Raw Normal View History

2021-03-29 08:24:56 +00:00
(ns jepsen.clickhouse-keeper.set
2021-03-16 08:03:47 +00:00
(:require
2021-03-16 12:37:46 +00:00
[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-16 12:37:46 +00:00
[zookeeper :as zk])
(:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException)))
2021-03-16 11:44:43 +00:00
(defrecord SetClient [k conn nodename]
client/Client
(open! [this test node]
2021-03-16 11:44:43 +00:00
(assoc
(assoc this
:conn (zk-connect node 9181 30000))
:nodename node))
(setup! [this test]
2021-04-03 15:05:10 +00:00
(exec-with-retries 30 (fn []
(zk-create-if-not-exists conn k "#{}"))))
2021-03-16 11:44:43 +00:00
(invoke! [this test op]
(case (:f op)
2021-03-26 16:56:08 +00:00
:read (exec-with-retries 30 (fn []
2021-03-26 16:57:23 +00:00
(zk-sync conn)
(assoc op
:type :ok
:value (read-string (:data (zk-get-str conn k))))))
:add (try
(do
(zk-add-to-set conn k (:value op))
(assoc op :type :ok))
(catch KeeperException$BadVersionException _ (assoc op :type :fail, :error :bad-version))
(catch Exception _ (assoc op :type :info, :error :connect-error)))))
(teardown! [_ test])
2021-03-18 20:55:11 +00:00
(close! [_ test]
(zk/close conn)))
(defn workload
"A generator, client, and checker for a set test."
[opts]
2021-03-16 11:44:43 +00:00
{:client (SetClient. "/a-set" nil nil)
:checker (checker/set)
:generator (->> (range)
(map (fn [x] {:type :invoke, :f :add, :value x})))
:final-generator (gen/once {:type :invoke, :f :read, :value nil})})