diff --git a/cockroachdb/README.md b/cockroachdb/README.md index 20f0503c8..f3cfb545a 100644 --- a/cockroachdb/README.md +++ b/cockroachdb/README.md @@ -27,6 +27,9 @@ The following tests are implemented: ``register`` concurrent atomic updates to a shared register; +``multi-register`` + concurrent atomic updates to multiple shared registers; + ``sets`` concurrent unique appends to a shared table; @@ -92,8 +95,8 @@ Nemeses: Jepsen will test every combination of `nemesis` and `nemesis2`, except where both nemeses would be identical, or both would introduce clock skew. -Test details: atomic updates ------------------------------ +Test details: register +---------------------- One table contains a single row. @@ -108,6 +111,23 @@ At the end, a linearizability checker validates that the trace of reads as observed from each client is compatible with a linearizable history of across all nodes. +Test details: multi-register +---------------------------- + +One table containing a fixed set of rows. + +Jepsen sends concurrently to different nodes transactions that +either read from a random subset of registers or write to a random +subset of registers. + +Each node may report ok, the operation is known to have succeeded; +fail, the operation is known to have failed; and unknown otherwise +(e.g. the connection dropped before the answer could be read). + +At the end, a linearizability checker validates that the trace of +reads as observed from each client is compatible with a linearizable +history of across all nodes. + Test details: unique appends (sets) ----------------------------------- @@ -159,7 +179,8 @@ fail, the operation is known to have failed; and unknown otherwise At the end, the checker validates that the sum of the remaining balances of all accounts is the same as the initial sum. -## Test details: sequential +Test details: sequential +------------------------ Cockroach does not offer strict serializability. However, as a consequence of its implementation of hybrid logical clocks, all transactions *on a particular @@ -179,13 +200,15 @@ occur from the same process, they must also be visible to any single process in that order. This implies that once a process observes kn, any subsequent read must see kn-1, and by induction, all smaller keys. -## Test details: G2 +Test details: G2 +---------------- Transactions select a predicate over two tables, then insert to one or the other if no rows are present. Serializability implies that at most one transaction may commit per predicate. -## Test details: comments +Test details: comments +---------------------- This test demonstrates a known strict serializability violation in Cockroach and is intended to fail. It performs a sequence of concurrent inserts to a diff --git a/cockroachdb/src/jepsen/cockroach/client.clj b/cockroachdb/src/jepsen/cockroach/client.clj index 47167e850..9190e7fbb 100644 --- a/cockroachdb/src/jepsen/cockroach/client.clj +++ b/cockroachdb/src/jepsen/cockroach/client.clj @@ -285,6 +285,11 @@ [conn table values where] (j/update! conn table values where {:timeout timeout-delay})) +(defn execute! + "Like jdbc execute!, but includes a default timeout." + [conn sql-params] + (j/execute! conn sql-params {:timeout timeout-delay})) + (defn db-time "Retrieve the current time (precise, monotonic) from the database." [c] @@ -301,11 +306,19 @@ (.toBigInteger) (str)))) +(defn val->sql-str + "Converts a scalar value to its SQL string representation" + [v] + (if (number? v) + v + (str "'" v "'"))) + (defn split! "Split the given table at the given key." [conn table k] (query conn [(str "alter table " (name table) " split at values (" - (if (number? k) - k - (str "'" k "'")) + (if (coll? k) + (str/join ", " + (map val->sql-str k)) + (val->sql-str k)) ")")])) diff --git a/cockroachdb/src/jepsen/cockroach/multiregister.clj b/cockroachdb/src/jepsen/cockroach/multiregister.clj new file mode 100644 index 000000000..81a19cfd7 --- /dev/null +++ b/cockroachdb/src/jepsen/cockroach/multiregister.clj @@ -0,0 +1,131 @@ +(ns jepsen.cockroach.multiregister + "Multiple atomic registers test + + Splits registers up into different tables to make sure they fall in + different ranges" + (:refer-clojure :exclude [test]) + (:require [jepsen [cockroach :as cockroach] + [client :as client] + [checker :as checker] + [generator :as gen] + [reconnect :as rc] + [independent :as independent] + [util :as util]] + [jepsen.checker.timeline :as timeline] + [jepsen.cockroach.client :as c] + [jepsen.cockroach.nemesis :as cln] + [clojure.java.jdbc :as j] + [clojure.tools.logging :refer :all] + [knossos.model :as model])) + +(def reg-count 5) +(def reg-range (range reg-count)) + +(def table-prefix "String prepended to all table names." "register_") +(defn id->table + "Turns an id into a table name string" + [id] + (str table-prefix id)) +(def table-names (map id->table reg-range)) + +(defn r + "Read a random subset of keys." + [_ _] + ;; Reading a random subset of keys after writing a random subset of + ;; keys may trigger the linearizability checker due to causal reverse. + ;; We may want to switch this to only reading a single key per read-only + ;; transaction. Doing so will still test against stale reads. + (->> (util/random-nonempty-subset reg-range) + (mapv (fn [id] [:read id nil])) + (array-map :type :invoke, :f :txn, :value))) + +(defn w + "Write a random subset of keys." + [_ _] + (->> (util/random-nonempty-subset reg-range) + (mapv (fn [id] [:write id (rand-int 5)])) + (array-map :type :invoke, :f :txn, :value))) + +(defrecord MultiAtomicClient [tbl-created? conn] + client/Client + + (open! [this test node] + (assoc this :conn (c/client node))) + + (setup! [this test] + ;; Everyone's gotta block until we've made the table. + (locking tbl-created? + (when (compare-and-set! tbl-created? false true) + (c/with-conn [c conn] + (info "Creating tables" (pr-str table-names)) + (Thread/sleep 1000) + (doseq [t table-names] + (j/execute! c [(str "drop table if exists " t)])) + (Thread/sleep 1000) + (doseq [t table-names] + (j/execute! c [(str "create table " t + " (ik int primary key, val int)")]) + (info "Created table" t)))))) + + (invoke! [this test op] + (c/with-idempotent #{:read} + (c/with-exception->op op + (c/with-conn [c conn] + (c/with-timeout + (try + (c/with-txn [c c] + (let [[ik txn] (:value op) + txn' (mapv + (fn [[f id val]] + (let [t (id->table id) + val' (case f + :read + (-> c + ; Look up and return current value + (c/query [(str "select val from " t " where ik = ?") ik] + {:row-fn :val :timeout c/timeout-delay}) + first) + + :write + (do + ; Perform blind write on key, return value + (c/execute! c [(str "upsert into " t " values (?, ?)") ik val]) + (cockroach/update-keyrange! test t ik) + val))] + [f id val'])) + txn)] + (assoc op :type :ok, :value (independent/tuple ik txn')))) + (catch org.postgresql.util.PSQLException e + (if (re-find #"ERROR: restart transaction" (.getMessage e)) + ; Definitely failed + (assoc op :type :fail) + (throw e))))))))) + + (teardown! [this test] + nil) + + (close! [this test] + (rc/close! conn))) + +(defn test + [opts] + (cockroach/basic-test + (merge + {:name "multi-register" + :client {:client (MultiAtomicClient. (atom false) nil) + :during (independent/concurrent-generator + (* 2 (count (:nodes opts))) + (range) + (fn [k] + (->> (gen/mix [r w]) + (gen/delay-til 1/2) + (gen/stagger 1/100) + (gen/limit 100))))} + :model (model/multi-register {}) + :checker (checker/compose + {:perf (checker/perf) + :details (independent/checker + (checker/compose + {:timeline (timeline/html) + :linearizable (checker/linearizable)}))})} + opts))) diff --git a/cockroachdb/src/jepsen/cockroach/register.clj b/cockroachdb/src/jepsen/cockroach/register.clj index a82cc5937..d4015353c 100644 --- a/cockroachdb/src/jepsen/cockroach/register.clj +++ b/cockroachdb/src/jepsen/cockroach/register.clj @@ -6,8 +6,7 @@ [checker :as checker] [generator :as gen] [reconnect :as rc] - [independent :as independent] - [util :refer [meh]]] + [independent :as independent]] [jepsen.checker.timeline :as timeline] [jepsen.cockroach.client :as c] [jepsen.cockroach.nemesis :as cln] diff --git a/cockroachdb/src/jepsen/cockroach/runner.clj b/cockroachdb/src/jepsen/cockroach/runner.clj index 31ff80f72..5a97e9c5a 100644 --- a/cockroachdb/src/jepsen/cockroach/runner.clj +++ b/cockroachdb/src/jepsen/cockroach/runner.clj @@ -17,6 +17,7 @@ [bank :as bank] [comments :as comments] [register :as register] + [multiregister :as multiregister] [monotonic :as monotonic] [nemesis :as cln] [sets :as sets] @@ -28,6 +29,7 @@ "bank-multitable" bank/multitable-test "comments" comments/test "register" register/test + "multi-register" multiregister/test "monotonic" monotonic/test "sets" sets/test "sequential" sequential/test