From df5105594f980c146d2035a1214e517048481cd0 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 3 Apr 2019 22:09:21 -0400 Subject: [PATCH] cockroachdb: introduce multi-register test This change introduces a new "multi-register" jepsen test for CockroachDB. The test is similar to "register" in that it performs reads and writes on atomic registers. However, unlike "register", the new test runs transactions that access multiple registers. It then uses the new Knossos "multi-register" checker to search for non-linearizable histories across the component transactions. The goal here is to tickle single-key linearizability violations that only occur with multi-Range transactions. To that end, the test splits each register into its own table. --- cockroachdb/README.md | 33 ++++- cockroachdb/src/jepsen/cockroach/client.clj | 19 ++- .../src/jepsen/cockroach/multiregister.clj | 131 ++++++++++++++++++ cockroachdb/src/jepsen/cockroach/register.clj | 3 +- cockroachdb/src/jepsen/cockroach/runner.clj | 2 + 5 files changed, 178 insertions(+), 10 deletions(-) create mode 100644 cockroachdb/src/jepsen/cockroach/multiregister.clj 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