diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 83b4975..368670c 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -8,10 +8,10 @@ on: branches: [main, master] jobs: - jepsen-test: + legacy-test: runs-on: ubuntu-latest - name: The simplest jepsen test for HStreamDB - timeout-minutes: 60 + name: Jepsen test for hstream legacy mode + timeout-minutes: 20 steps: - uses: actions/checkout@v2 with: @@ -24,30 +24,70 @@ jobs: - name: Build base image run: | - docker build -t jepsen-hstream:base ./docker/base/ + ./scripts/build_base.sh - - name: Build test images + - name: Build test images (legacy) run: | - docker-compose --file ./docker/docker-compose.yml \ - --compatibility \ - -p jepsen \ - build - - name: Clean dangling images + ./scripts/build_legacy.sh + + - name: Check disk space + run: | + df -h + + - name: Start test cluster and run jepsen test (legacy) run: | - docker image prune -f + ./scripts/up_legacy.sh + + - name: Cleanup resources (legacy) + run: | + ./scripts/clean_legacy.sh + + - name: Upload Artifact + uses: actions/upload-artifact@v2 + if: ${{ success() }} || ${{ failure() }} + with: + name: legacy-result + path: | + store/legacy-husky + store/legacy-list-append + /tmp/*.log + !store/latest + !store/current + retention-days: 7 + + kafka-test: + runs-on: ubuntu-latest + name: Jepsen test for hstream kafka mode + timeout-minutes: 20 + steps: + - uses: actions/checkout@v2 + with: + submodules: "recursive" + + - name: Prepare environment for test + run: | + cd ./docker + ./init-secret.sh + + - name: Build base image + run: | + ./scripts/build_base.sh + + - name: Build test images (kafka) + run: | + ./scripts/build_kafka.sh - name: Check disk space run: | df -h - - name: Start test cluster and run jepsen test + - name: Start test cluster and run jepsen test (kafka) + run: | + ./scripts/up_kafka.sh + + - name: Cleanup resources (kafka) run: | - docker-compose --file ./docker/docker-compose.yml \ - --compatibility \ - -p jepsen \ - up \ - --renew-anon-volumes \ - --exit-code-from control + ./scripts/clean_kafka.sh - name: Upload Artifact uses: actions/upload-artifact@v2 @@ -55,7 +95,8 @@ jobs: with: name: jepsen-test-result path: | - store/HStream + store/kafka /tmp/*.log - !store/HStream/latest + !store/latest + !store/current retention-days: 7 diff --git a/.github/workflows/ci_repo_dispatch.yml b/.github/workflows/ci_repo_dispatch.yml index e13ed20..6f1a1ff 100644 --- a/.github/workflows/ci_repo_dispatch.yml +++ b/.github/workflows/ci_repo_dispatch.yml @@ -5,10 +5,10 @@ on: types: image_updated jobs: - jepsen-test: + legacy-test: runs-on: ubuntu-latest - name: The simplest jepsen test for HStreamDB - timeout-minutes: 60 + name: Jepsen test for hstream legacy mode + timeout-minutes: 20 steps: - uses: actions/checkout@v2 with: @@ -21,41 +21,97 @@ jobs: - name: Build base image run: | - docker build -t jepsen-hstream:base ./docker/base/ + ./scripts/build_base.sh - - name: Build test images + - name: Build test images (legacy) run: | - docker-compose --file ./docker/docker-compose.yml \ - --compatibility \ - -p jepsen \ - build + ./scripts/build_legacy.sh - - name: Clean dangling images + - name: Check disk space + run: | + df -h + + - name: Start test cluster and run jepsen test (legacy) + run: | + ./scripts/up_legacy.sh + + - name: Cleanup resources (legacy) + run: | + ./scripts/clean_legacy.sh + + - name: Upload Artifact + uses: actions/upload-artifact@v2 + if: ${{ success() }} || ${{ failure() }} + with: + name: legacy-result + path: | + store/legacy-husky + store/legacy-list-append + /tmp/*.log + !store/latest + !store/current + retention-days: 7 + + - name: Post to Slack channel + if: ${{ success() }} || ${{ failure() }} + run: | + if [ "${{ job.status }}" = 'success' ] + then JOB_STATUS_EMOJI=':white_check_mark:' + else JOB_STATUS_EMOJI=':x:' + fi + curl \ + -X POST \ + -H 'Content-Type: application/json' \ + ${{ secrets.SLACK_WEBHOOK_URL }} \ + --data '{"GitHub Action build result": "${{ job.status }} '"$JOB_STATUS_EMOJI"'", + "Commit info": "https://github.com/hstreamdb/hstream/commit/${{ github.event.client_payload.commit_sha }}", + "Action summary": "https://github.com/'"$GITHUB_REPOSITORY"'/actions/runs/'"$GITHUB_RUN_ID"'"}' + + kafka-test: + runs-on: ubuntu-latest + name: Jepsen test for hstream kafka mode + timeout-minutes: 20 + steps: + - uses: actions/checkout@v2 + with: + submodules: "recursive" + + - name: Prepare environment for test + run: | + cd ./docker + ./init-secret.sh + + - name: Build base image run: | - docker image prune -f + ./scripts/build_base.sh + + - name: Build test images (kafka) + run: | + ./scripts/build_kafka.sh - name: Check disk space run: | df -h - - name: Start test cluster and run jepsen test + - name: Start test cluster and run jepsen test (kafka) run: | - docker-compose --file ./docker/docker-compose.yml \ - --compatibility \ - -p jepsen \ - up \ - --renew-anon-volumes \ - --exit-code-from control - - - name: 'Upload Artifact' + ./scripts/up_kafka.sh + + - name: Cleanup resources (kafka) + run: | + ./scripts/clean_kafka.sh + + - name: Upload Artifact uses: actions/upload-artifact@v2 if: ${{ success() }} || ${{ failure() }} with: name: jepsen-test-result path: | - store/HStream + store/kafka /tmp/*.log - !store/HStream/latest + !store/latest + !store/current + retention-days: 7 - name: Post to Slack channel if: ${{ success() }} || ${{ failure() }} diff --git a/README.md b/README.md index 8600e70..08edf33 100644 --- a/README.md +++ b/README.md @@ -2,6 +2,13 @@ Jepsen test instances for [HStreamDB](https://github.com/hstreamdb/hstream). +**Note:** + +The following files are forked & modified from [https://github.com/jepsen-io/redpanda](Jepsen's official repository), with the same license. + +- `src/jepsen/hstream/kafka_test.clj` +- `src/jepsen/hstream/kafka/*` + ## Overview The project is at its early stage and is under active development. @@ -13,27 +20,32 @@ It currently contains the following tests: ## Usage ``` -docker-compose --file ./docker/docker-compose.yml --compatibility -p jepsen build - -docker-compose --file ./docker/docker-compose.yml --compatibility -p jepsen up --renew-anon-volumes --exit-code-from control +./scripts/build_base.sh +./scripts/build_legacy.sh +./scripts/up_legacy.sh +./scripts/clean_legacy.sh ``` +For kafka version test, run `*_kafka.sh`. + ## Check Test Results The test results will be stored at `./store` directory. Check it manually or by a simple server: - If you have [`leiningen`](https://leiningen.org/) installed: ``` -lein with-profile husky run serve +lein with-profile legacy-husky run serve ``` - If you do not have `leiningen`: ``` -docker run -t --rm --network host -v $(pwd):/working clojure:temurin-21-lein /bin/bash -c "cd /working && lein with-profile husky run serve" +docker run -t --rm --network host -v $(pwd):/working clojure:temurin-21-lein /bin/bash -c "cd /working && lein with-profile legacy-husky run serve" ``` Then browse the results at `localhost:8080`. ## Customization -- Add `HSTREAM_IMAGE` arg on building step to use your own hstream image. -- Adjust test parameters in `docker/control/Dockerfile`. +- Add `BASE_IMAGE` and `HSTREAM_IMAGE` arg on building step to use your own hstream image. +- Add `USE_CHINA_MIRROR` arg to speed up downloading. +- Add `env_http_proxy` and `env_https_proxy` arg to use proxy from your host (it should allow LAN requests). +- Adjust test parameters in `docker/control/Dockerfile` or `docker/control-kafka/Dockerfile`, then **rebuild images** (no need to rebuild base image). diff --git a/docker/admin-kafka/Dockerfile b/docker/admin-kafka/Dockerfile new file mode 100644 index 0000000..fcbde5a --- /dev/null +++ b/docker/admin-kafka/Dockerfile @@ -0,0 +1,37 @@ +ARG BASE_IMAGE="jepsen-hstream:base" +FROM ${BASE_IMAGE} + +# Waiting for Zookeeper +COPY ./wait-zk.sh /usr/local/bin/wait-zk +RUN chmod +x /usr/local/bin/wait-zk + +# Waiting for logdeviced +COPY ./wait-hstore.sh /usr/local/bin/wait-hstore +RUN chmod +x /usr/local/bin/wait-hstore + +# Waiting for hservers +COPY ./wait-hservers.sh /usr/local/bin/wait-hservers +RUN chmod +x /usr/local/bin/wait-hservers + +# Config for LogDevice +COPY ./logdevice.json /etc/logdevice.json + +EXPOSE 22 6440 + +CMD /usr/local/bin/init-ssh && \ + ZK_IP=$(dig +short zookeeper) && \ + sed -i "s/172.16.0.10:2181/$ZK_IP:2181/g" /etc/logdevice.json && \ + /usr/local/bin/wait-zk && \ + /usr/share/zookeeper/bin/zkCli.sh -server zookeeper:2181 create /logdevice.conf "`cat /etc/logdevice.json`" && \ + ld-admin-server \ + --config-path zk:zookeeper:2181/logdevice.conf \ + --enable-maintenance-manager \ + --maintenance-log-snapshotting \ + --enable-safety-check-periodic-metadata-update \ + >> /tmp/$HOSTNAME.log 2>&1 & \ + /usr/local/bin/wait-hstore && \ + hadmin store nodes-config bootstrap --metadata-replicate-across node:3 && \ + /usr/local/bin/wait-hservers && \ + hstream-kafka --host hserver-1 node init >> /tmp/$HOSTNAME.log 2>&1 && \ + echo "Bootstraped" > /var/jepsen/shared/hserver-cluster-started && \ + tail -f /dev/null diff --git a/docker/admin-kafka/logdevice.json b/docker/admin-kafka/logdevice.json new file mode 100644 index 0000000..3e22ec5 --- /dev/null +++ b/docker/admin-kafka/logdevice.json @@ -0,0 +1,52 @@ +{ + "cluster": "logdevice-dev", + "server_settings": { + "enable-node-self-registration": "true", + "enable-nodes-configuration-manager": "true", + "use-nodes-configuration-manager-nodes-configuration": "true", + "enable-cluster-maintenance-state-machine": "true", + "rocksdb-memtable-size-per-node": "1024M", + "free-disk-space-threshold": 0.1 + }, + "client_settings": { + "enable-nodes-configuration-manager": "true", + "use-nodes-configuration-manager-nodes-configuration": "true", + "admin-client-capabilities": "true" + }, + "internal_logs": { + "config_log_deltas": { + "replicate_across": { + "node": 3 + } + }, + "config_log_snapshots": { + "replicate_across": { + "node": 3 + } + }, + "event_log_deltas": { + "replicate_across": { + "node": 3 + } + }, + "event_log_snapshots": { + "replicate_across": { + "node": 3 + } + }, + "maintenance_log_deltas": { + "replicate_across": { + "node": 3 + } + }, + "maintenance_log_snapshots": { + "replicate_across": { + "node": 3 + } + } + }, + "zookeeper": { + "zookeeper_uri": "ip://172.16.0.10:2181", + "timeout": "30s" + } +} diff --git a/docker/admin-kafka/wait-hservers.sh b/docker/admin-kafka/wait-hservers.sh new file mode 100644 index 0000000..0b8c3e1 --- /dev/null +++ b/docker/admin-kafka/wait-hservers.sh @@ -0,0 +1,11 @@ +#!/usr/bin/bash + +until ( \ + /usr/local/bin/hstream-kafka --host hserver-1 --port 9092 node status && \ + /usr/local/bin/hstream-kafka --host hserver-2 --port 9092 node status && \ + /usr/local/bin/hstream-kafka --host hserver-3 --port 9092 node status && \ + /usr/local/bin/hstream-kafka --host hserver-4 --port 9092 node status && \ + /usr/local/bin/hstream-kafka --host hserver-5 --port 9092 node status \ +) >/dev/null 2>&1; do + sleep 1 +done; diff --git a/docker/admin-kafka/wait-hstore.sh b/docker/admin-kafka/wait-hstore.sh new file mode 100644 index 0000000..e208f04 --- /dev/null +++ b/docker/admin-kafka/wait-hstore.sh @@ -0,0 +1,22 @@ +#!/usr/bin/bash + +Test_Open () { + /dev/null +ssh-add /root/.ssh/id_rsa &> /dev/null diff --git a/docker/control-kafka/init-ssh-control.sh b/docker/control-kafka/init-ssh-control.sh new file mode 100644 index 0000000..da13a44 --- /dev/null +++ b/docker/control-kafka/init-ssh-control.sh @@ -0,0 +1,21 @@ +#!/bin/sh + +sleep 1 && \ + +: "${SSH_PRIVATE_KEY?SSH_PRIVATE_KEY is empty, please use up.sh}" +: "${SSH_PUBLIC_KEY?SSH_PUBLIC_KEY is empty, please use up.sh}" + +if [ ! -f ~/.ssh/known_hosts ]; then + mkdir -m 700 ~/.ssh + echo $SSH_PRIVATE_KEY | perl -p -e 's/↩/\n/g' > ~/.ssh/id_rsa + chmod 600 ~/.ssh/id_rsa + echo $SSH_PUBLIC_KEY > ~/.ssh/id_rsa.pub + echo > ~/.ssh/known_hosts + # Get nodes list + sort -V /var/jepsen/shared/nodes > ~/nodes + # Scan SSH keys + while read node; do + ssh-keyscan -t rsa $node >> ~/.ssh/known_hosts + ssh-keyscan -t ed25519 $node >> ~/.ssh/known_hosts + done <~/nodes +fi diff --git a/docker/control/Dockerfile b/docker/control/Dockerfile index 0363546..3a7b8e1 100644 --- a/docker/control/Dockerfile +++ b/docker/control/Dockerfile @@ -52,8 +52,8 @@ CMD /init-ssh.sh && \ lein deps && \ unset http_proxy && \ unset https_proxy && \ - ### For jepsen.write-then-read - # lein with-profile write-then-read run test \ + ### For jepsen.hstream.list-append-test + # lein with-profile legacy-list-append run test \ # --nodes "ld1,ld2,ld3,n1,n2,n3,n4,n5,zk" \ # --ssh-private-key "/root/.ssh/id_rsa" \ # --dummy false \ @@ -68,8 +68,8 @@ CMD /init-ssh.sh && \ # --nemesis-interval 15 \ # --nemesis-on true \ # --max-partitions 10 && \ - ### For jepsen.husky-test - lein with-profile husky run test \ + ### For jepsen.hstream.legacy-husky + lein with-profile legacy-husky run test \ --nodes "ld1,ld2,ld3,n1,n2,n3,n4,n5,zk" \ --ssh-private-key "/root/.ssh/id_rsa" \ --dummy false \ diff --git a/docker/docker-compose-kafka.yml b/docker/docker-compose-kafka.yml new file mode 100644 index 0000000..3aaa948 --- /dev/null +++ b/docker/docker-compose-kafka.yml @@ -0,0 +1,200 @@ +version: '3.7' +x-node: + &hserver-node + build: ./node-kafka + env_file: + - ./secret/node.env + - ./secret/control.env + secrets: + - authorized_keys + tty: true + tmpfs: + - /run:size=100M + - /run/lock:size=100M + volumes: + - "jepsen-kafka-shared:/var/jepsen/shared" + - "/sys/fs/cgroup:/sys/fs/cgroup:ro" + - "/tmp:/tmp:rw" + networks: + - jepsen-kafka + cap_add: + - ALL + ports: + - ${JEPSEN_PORT:-22} + +x-ld: + &logdevice-node + build: ./ld + env_file: + - ./secret/node.env + - ./secret/control.env + secrets: + - authorized_keys + tty: true + tmpfs: + - /run:size=100M + - /run/lock:size=100M + volumes: + - "jepsen-kafka-shared:/var/jepsen/shared" + - "/sys/fs/cgroup:/sys/fs/cgroup:ro" + - "/tmp:/tmp:rw" + networks: + - jepsen-kafka + cap_add: + - ALL + ports: + - ${JEPSEN_PORT:-22} + expose: + - "4440" + - "4441" + - "4443" + - "6440" + +volumes: + jepsen-kafka-shared: + +secrets: + authorized_keys: + file: ./secret/authorized_keys + +networks: + jepsen-kafka: + driver: bridge + +services: + ld-admin: + container_name: jepsen-kafka-ld-admin + hostname: admin + depends_on: + - zookeeper + build: ./admin-kafka + env_file: ./secret/node.env + secrets: + - authorized_keys + tmpfs: + - /run:size=100M + - /run/lock:size=100M + expose: + - "4440" + - "4441" + - "4443" + - "6440" + networks: + - jepsen-kafka + cap_add: + - ALL + volumes: + - "jepsen-kafka-shared:/var/jepsen/shared" + - "/sys/fs/cgroup:/sys/fs/cgroup:ro" + - "/tmp:/tmp:rw" + + zookeeper: + container_name: jepsen-kafka-zookeeper + hostname: zk + build: ./zk + env_file: ./secret/node.env + secrets: + - authorized_keys + tmpfs: + - /run:size=100M + - /run/lock:size=100M + expose: + - "2181" + - "2888" + - "3888" + networks: + - jepsen-kafka + cap_add: + - ALL + volumes: + - "jepsen-kafka-shared:/var/jepsen/shared" + - "/sys/fs/cgroup:/sys/fs/cgroup:ro" + - "/tmp:/tmp:rw" + + control: + container_name: jepsen-kafka-control + hostname: control-kafka + depends_on: + - zookeeper + - hserver-1 + - hserver-2 + - hserver-3 + - hserver-4 + - hserver-5 + - ld-admin + build: ./control-kafka + env_file: ./secret/control.env + privileged: true + expose: + - "22" + - "8080" + networks: + - jepsen-kafka + volumes: + - "../:/home/Work" + - "jepsen-kafka-shared:/var/jepsen/shared" + + hstore-1: + << : *logdevice-node + container_name: jepsen-kafka-ld1 + hostname: ld1 + depends_on: + - zookeeper + hstore-2: + << : *logdevice-node + container_name: jepsen-kafka-ld2 + hostname: ld2 + depends_on: + - zookeeper + hstore-3: + << : *logdevice-node + container_name: jepsen-kafka-ld3 + hostname: ld3 + depends_on: + - zookeeper + + hserver-1: + << : *hserver-node + container_name: jepsen-kafka-n1 + hostname: n1 + depends_on: + - zookeeper + - hstore-1 + - hstore-2 + - hstore-3 + hserver-2: + << : *hserver-node + container_name: jepsen-kafka-n2 + hostname: n2 + depends_on: + - zookeeper + - hstore-1 + - hstore-2 + - hstore-3 + hserver-3: + << : *hserver-node + container_name: jepsen-kafka-n3 + hostname: n3 + depends_on: + - zookeeper + - hstore-1 + - hstore-2 + - hstore-3 + hserver-4: + << : *hserver-node + container_name: jepsen-kafka-n4 + hostname: n4 + depends_on: + - zookeeper + - hstore-1 + - hstore-2 + - hstore-3 + hserver-5: + << : *hserver-node + container_name: jepsen-kafka-n5 + hostname: n5 + depends_on: + - zookeeper + - hstore-1 + - hstore-2 + - hstore-3 diff --git a/docker/node-kafka/Dockerfile b/docker/node-kafka/Dockerfile new file mode 100644 index 0000000..87eb8d9 --- /dev/null +++ b/docker/node-kafka/Dockerfile @@ -0,0 +1,11 @@ +ARG BASE_IMAGE="jepsen-hstream:base" +FROM ${BASE_IMAGE} + +# Script for restarting hstream-server +COPY start-server.sh /bin/start-server +RUN chmod +x /bin/start-server + +EXPOSE 22 9092 + +CMD /bin/start-server && \ + bash # Keep container alive diff --git a/docker/node-kafka/start-server.sh b/docker/node-kafka/start-server.sh new file mode 100644 index 0000000..978cc8c --- /dev/null +++ b/docker/node-kafka/start-server.sh @@ -0,0 +1,22 @@ +#!/bin/bash + +/usr/local/bin/init-ssh + +# Waiting for logdevice to start +sleep 2 + +SERVER_ID=$(shuf -i 1-4294967296 -n 1) +MY_IP=$(hostname -I | head -n1 | awk '{print $1;}') +hstream-server kafka \ + --config-path /etc/hstream/config.yaml \ + --bind-address 0.0.0.0 \ + --port 9092 \ + --gossip-port 6571 \ + --advertised-address $MY_IP \ + --store-config zk:zookeeper:2181/logdevice.conf \ + --metastore-uri "zk://zookeeper:2181" \ + --server-id $SERVER_ID \ + --log-level debug \ + --log-with-color \ + --seed-nodes hserver-1,hserver-2,hserver-3,hserver-4,hserver-5 \ + >>/tmp/$HOSTNAME.log 2>&1 & diff --git a/docker/node/Dockerfile b/docker/node/Dockerfile index ad190cc..ca7920e 100644 --- a/docker/node/Dockerfile +++ b/docker/node/Dockerfile @@ -7,5 +7,8 @@ RUN chmod +x /bin/start-server EXPOSE 22 6570 -CMD /bin/start-server && \ +CMD /usr/local/bin/init-ssh && \ + # Waiting for logdevice to start + sleep 2 && \ + /bin/start-server && \ bash # Keep container alive diff --git a/docker/node/start-server.sh b/docker/node/start-server.sh index 73b04bd..5774484 100644 --- a/docker/node/start-server.sh +++ b/docker/node/start-server.sh @@ -1,10 +1,5 @@ #!/bin/bash -/usr/local/bin/init-ssh - -# Waiting for logdevice to start -sleep 2 - SERVER_ID=$(shuf -i 1-4294967296 -n 1) MY_IP=$(hostname -I | head -n1 | awk '{print $1;}') hstream-server \ diff --git a/project.clj b/project.clj index 741cd46..7b1e127 100644 --- a/project.clj +++ b/project.clj @@ -2,12 +2,33 @@ :description "Jepsen test instances for HStreamDB" :license {:name "EPL-2.0 OR GPL-2.0-or-later WITH Classpath-exception-2.0", :url "https://www.eclipse.org/legal/epl-2.0/"} - :profiles {:write-then-read {:main jepsen.write-then-read}, - :husky {:main jepsen.husky-test}} + :profiles {:legacy-list-append {:main jepsen.hstream.list-append-test}, + :legacy-husky {:main jepsen.hstream.husky-test} + :kafka {:main jepsen.hstream.kafka-test}} :repositories [["sonatype-snapshot" "https://s01.oss.sonatype.org/content/repositories/snapshots"]] :dependencies [[org.clojure/clojure "1.11.2"] - [jepsen "0.3.5"] + [jepsen "0.2.7"] + [cheshire "5.12.0"] + [clj-http "3.12.3"] [random-string "0.1.0"] - [io.hstream/hstreamdb-java "0.17.0"]]) + [org.apache.kafka/kafka-clients "3.7.0"] + [io.hstream/hstreamdb-java "0.17.0" + :exclusions [org.apache.logging.log4j/log4j-slf4j-impl]] + ] + :jvm-opts ["-server" + ;"-XX:-OmitStackTraceInFastThrow" + "-Djava.awt.headless=true" + ; GC tuning--see + ; https://wiki.openjdk.java.net/display/shenandoah/Main + ; https://wiki.openjdk.java.net/display/zgc/Main + ;"-XX+UseZGC" + ;"-XX+UseShenandoahGC" + "-Xmx24g" + ;"-XX:+UseLargePages" ; requires users do so some OS-level config + "-XX:+AlwaysPreTouch" + ; Instrumentation + ;"-agentpath:/home/aphyr/yourkit/bin/linux-x86-64/libyjpagent.so=disablestacktelemetry,exceptions=disable,delay=10000" + ] + ) diff --git a/scripts/build_base.sh b/scripts/build_base.sh new file mode 100755 index 0000000..c2a65e7 --- /dev/null +++ b/scripts/build_base.sh @@ -0,0 +1,4 @@ +#!/usr/bin/bash +docker build -t jepsen-hstream:base \ + --build-arg "USE_CHINA_MIRROR=false" \ + ./docker/base/ diff --git a/scripts/build_kafka.sh b/scripts/build_kafka.sh new file mode 100755 index 0000000..2951a84 --- /dev/null +++ b/scripts/build_kafka.sh @@ -0,0 +1,10 @@ +#!/usr/bin/bash +docker-compose --file ./docker/docker-compose-kafka.yml \ + --compatibility \ + -p jepsen-kafka \ + build \ + --build-arg USE_CHINA_MIRROR=false \ + --build-arg arg_http_proxy="" \ + --build-arg arg_https_proxy="" \ + --build-arg BASE_IMAGE=jepsen-hstream:base \ + --build-arg HSTREAM_IMAGE=hstreamdb/hstream:latest diff --git a/scripts/build_legacy.sh b/scripts/build_legacy.sh new file mode 100755 index 0000000..dba5388 --- /dev/null +++ b/scripts/build_legacy.sh @@ -0,0 +1,10 @@ +#!/usr/bin/bash +docker-compose --file ./docker/docker-compose.yml \ + --compatibility \ + -p jepsen \ + build \ + --build-arg USE_CHINA_MIRROR=false \ + --build-arg arg_http_proxy="" \ + --build-arg arg_https_proxy="" \ + --build-arg BASE_IMAGE=jepsen-hstream:base \ + --build-arg HSTREAM_IMAGE=hstreamdb/hstream:latest diff --git a/scripts/clean_kafka.sh b/scripts/clean_kafka.sh new file mode 100755 index 0000000..15e090f --- /dev/null +++ b/scripts/clean_kafka.sh @@ -0,0 +1,12 @@ +docker rm -f jepsen-kafka-n1 && \ + docker rm -f jepsen-kafka-n2 && \ + docker rm -f jepsen-kafka-n3 && \ + docker rm -f jepsen-kafka-n4 && \ + docker rm -f jepsen-kafka-n5 && \ + docker rm -f jepsen-kafka-ld1 && \ + docker rm -f jepsen-kafka-ld2 && \ + docker rm -f jepsen-kafka-ld3 && \ + docker rm -f jepsen-kafka-zookeeper && \ + docker rm -f jepsen-kafka-ld-admin && \ + docker rm -f jepsen-kafka-control && \ + docker volume rm jepsen-kafka_jepsen-kafka-shared diff --git a/scripts/clean_legacy.sh b/scripts/clean_legacy.sh new file mode 100755 index 0000000..eefa75c --- /dev/null +++ b/scripts/clean_legacy.sh @@ -0,0 +1,12 @@ +docker rm -f jepsen-n1 && \ + docker rm -f jepsen-n2 && \ + docker rm -f jepsen-n3 && \ + docker rm -f jepsen-n4 && \ + docker rm -f jepsen-n5 && \ + docker rm -f jepsen-ld1 && \ + docker rm -f jepsen-ld2 && \ + docker rm -f jepsen-ld3 && \ + docker rm -f jepsen-zookeeper && \ + docker rm -f jepsen-ld-admin && \ + docker rm -f jepsen-control && \ + docker volume rm jepsen_jepsen-shared diff --git a/scripts/up_kafka.sh b/scripts/up_kafka.sh new file mode 100755 index 0000000..35aa5ef --- /dev/null +++ b/scripts/up_kafka.sh @@ -0,0 +1,9 @@ +#!/usr/bin/bash +docker-compose --file ./docker/docker-compose-kafka.yml \ + --compatibility \ + -p jepsen-kafka \ + up \ + --renew-anon-volumes \ + --exit-code-from control + +# lein run -- test --db hstream --no-txn --no-server-idempotence --nemesis none --workload queue --time-limit 60 --sub-via subscribe diff --git a/scripts/up_legacy.sh b/scripts/up_legacy.sh new file mode 100755 index 0000000..2e9c93f --- /dev/null +++ b/scripts/up_legacy.sh @@ -0,0 +1,7 @@ +#!/usr/bin/bash +docker-compose --file ./docker/docker-compose.yml \ + --compatibility \ + -p jepsen \ + up \ + --renew-anon-volumes \ + --exit-code-from control diff --git a/src/jepsen/hstream/mcons.clj b/src/jepsen/hstream/common/mcons.clj similarity index 93% rename from src/jepsen/hstream/mcons.clj rename to src/jepsen/hstream/common/mcons.clj index 9975642..42f529b 100644 --- a/src/jepsen/hstream/mcons.clj +++ b/src/jepsen/hstream/common/mcons.clj @@ -1,7 +1,7 @@ ;; https://gist.github.com/fredyr/6341286 ;; http://docs.racket-lang.org/reference/mpairs.html -(ns jepsen.hstream.mcons (:gen-class)) +(ns jepsen.hstream.common.mcons) (defprotocol IMCons (mcar [p]) diff --git a/src/jepsen/hstream/mvar.clj b/src/jepsen/hstream/common/mvar.clj similarity index 89% rename from src/jepsen/hstream/mvar.clj rename to src/jepsen/hstream/common/mvar.clj index 9e45a9c..32a8511 100644 --- a/src/jepsen/hstream/mvar.clj +++ b/src/jepsen/hstream/common/mvar.clj @@ -1,8 +1,7 @@ ;; https://gist.github.com/fredyr/6345191 -(ns jepsen.hstream.mvar - (:gen-class) - (:require [jepsen.hstream.mcons :refer :all])) +(ns jepsen.hstream.common.mvar + (:require [jepsen.hstream.common.mcons :refer :all])) (defn empty-mvar [] diff --git a/src/jepsen/hstream/net.clj b/src/jepsen/hstream/common/net.clj similarity index 99% rename from src/jepsen/hstream/net.clj rename to src/jepsen/hstream/common/net.clj index 96e59fa..de3682d 100644 --- a/src/jepsen/hstream/net.clj +++ b/src/jepsen/hstream/common/net.clj @@ -1,4 +1,4 @@ -(ns jepsen.hstream.net +(ns jepsen.hstream.common.net (:require [clojure.string :as str] [jepsen.control :refer :all] [jepsen.control.net :as control.net] @@ -13,7 +13,7 @@ (def iptables+ "Default iptables (assumes we control everything)." (reify - p/Net + Net (drop! [net test src dest] (on dest (su (exec :iptables diff --git a/src/jepsen/hstream/utils.clj b/src/jepsen/hstream/common/utils.clj similarity index 98% rename from src/jepsen/hstream/utils.clj rename to src/jepsen/hstream/common/utils.clj index 6dc4294..b95ba47 100644 --- a/src/jepsen/hstream/utils.clj +++ b/src/jepsen/hstream/common/utils.clj @@ -1,5 +1,4 @@ -(ns jepsen.hstream.utils - (:gen-class) +(ns jepsen.hstream.common.utils (:require [random-string.core :as rs])) ;;;; I. Clojure common helper functions diff --git a/src/jepsen/husky_test.clj b/src/jepsen/hstream/husky_test.clj similarity index 86% rename from src/jepsen/husky_test.clj rename to src/jepsen/hstream/husky_test.clj index 0ae5a0c..f629e14 100644 --- a/src/jepsen/husky_test.clj +++ b/src/jepsen/hstream/husky_test.clj @@ -1,21 +1,19 @@ -(ns jepsen.husky-test - (:gen-class) +(ns jepsen.hstream.husky-test (:require [clojure.tools.logging :refer :all] [jepsen [db :as db] [cli :as cli] [checker :as checker] [client :as client] [control :as c] [generator :as gen] [independent :as independent] [nemesis :as nemesis] [tests :as tests]] [jepsen.checker.timeline :as timeline] - [jepsen.hstream.checker :as local-checker] - [jepsen.hstream.client :refer :all] - [jepsen.hstream.common :as common] - [jepsen.hstream.husky :as husky] - [jepsen.hstream.mvar :refer :all] - [jepsen.hstream.utils :refer :all] - [jepsen.hstream.nemesis :as local-nemesis] - [jepsen.hstream.net :as net+]) - (:import [jepsen.hstream.common Default-Client])) - + [jepsen.hstream.legacy.checker :as local-checker] + [jepsen.hstream.legacy.client :refer :all] + [jepsen.hstream.legacy.common :as common] + [jepsen.hstream.legacy.husky :as husky] + [jepsen.hstream.common.mvar :refer :all] + [jepsen.hstream.common.utils :refer :all] + [jepsen.hstream.legacy.nemesis :as local-nemesis] + [jepsen.hstream.common.net :as net+]) + (:import [jepsen.hstream.legacy.common Default-Client])) (defn hstream-test "Given an options map from the command-line runner (e.g. :nodes, :ssh, @@ -28,10 +26,10 @@ tests/noop-test opts {:pure-generators true, - :name "HStream", + :name "legacy-husky", :net net+/iptables+, :plot local-nemesis/plot-spec, - :db (common/db-empty "0.16.0"), + :db (common/db-empty "0.19.0"), :client (common/Default-Client. opts subscription-results subscription-ack-timeout), diff --git a/src/jepsen/hstream/kafka/client.clj b/src/jepsen/hstream/kafka/client.clj new file mode 100644 index 0000000..3621d27 --- /dev/null +++ b/src/jepsen/hstream/kafka/client.clj @@ -0,0 +1,378 @@ +(ns jepsen.hstream.kafka.client + "Wrapper for the Java Kafka client." + (:require [clojure.tools.logging :refer [info warn]] + [dom-top.core :as dt] + [jepsen.util :as util :refer [await-fn + map-vals + pprint-str]] + [slingshot.slingshot :refer [try+ throw+]]) + (:import (java.time Duration) + (java.util Properties) + (java.util.concurrent ExecutionException) + (org.apache.kafka.clients.admin Admin + AdminClientConfig + NewTopic) + (org.apache.kafka.clients.consumer ConsumerConfig + ConsumerRebalanceListener + ConsumerRecord + ConsumerRecords + KafkaConsumer + OffsetAndMetadata) + (org.apache.kafka.clients.producer KafkaProducer + ProducerConfig + ProducerRecord) + (org.apache.kafka.common KafkaException + TopicPartition) + (org.apache.kafka.common.errors InvalidTopicException + TopicExistsException))) + +(def port + "What port do we connect to?" + 9092) + +(def next-transactional-id + "We automatically assign each producer a unique transactional ID" + (atom -1)) + +(defn new-transactional-id + "Returns a unique transactional ID (mutating the global counter)" + [] + (str "jt" (swap! next-transactional-id inc))) + +(defn ^Properties ->properties + "Turns a map into a Properties object." + [m] + (doto (Properties.) + (.putAll (map-vals str m)))) + +(def consumer-config-logged? + "Used to ensure that we only log consumer configs once." + (atom false)) + +(def producer-config-logged? + "Used to ensure that we only log producer configs once." + (atom false)) + +(def consumer-group + "Right now all consumers are a single consumer group." + "jepsen-group") + +(defn consumer-config + "Constructs a properties map for talking to a given Kafka node." + [node opts] + ; See https://javadoc.io/doc/org.apache.kafka/kafka-clients/latest/org/apache/kafka/clients/consumer/ConsumerConfig.html + ; And https://docs.confluent.io/platform/current/installation/configuration/consumer-configs.html + (cond-> + {ConsumerConfig/KEY_DESERIALIZER_CLASS_CONFIG + "org.apache.kafka.common.serialization.LongDeserializer" + + ConsumerConfig/VALUE_DESERIALIZER_CLASS_CONFIG + "org.apache.kafka.common.serialization.LongDeserializer" + + ConsumerConfig/BOOTSTRAP_SERVERS_CONFIG + (str node ":" port) + + ConsumerConfig/SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG + 1000 + + ConsumerConfig/SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG + 500 + + ConsumerConfig/METADATA_MAX_AGE_CONFIG + 60000 + + ConsumerConfig/REQUEST_TIMEOUT_MS_CONFIG + 10000 + + ConsumerConfig/DEFAULT_API_TIMEOUT_MS_CONFIG + 10000 + + ConsumerConfig/HEARTBEAT_INTERVAL_MS_CONFIG + 300 + + ConsumerConfig/SESSION_TIMEOUT_MS_CONFIG + 6000 ; Bounded by server + + ConsumerConfig/CONNECTIONS_MAX_IDLE_MS_CONFIG + 60000 + + ; ConsumerConfig/DEFAULT_ISOLATION_LEVEL + ; ??? + } + (:subscribe (:sub-via opts)) + (assoc ConsumerConfig/GROUP_ID_CONFIG consumer-group) + + (not= nil (:isolation-level opts)) + (assoc ConsumerConfig/ISOLATION_LEVEL_CONFIG (:isolation-level opts)) + + (not= nil (:auto-offset-reset opts)) + (assoc ConsumerConfig/AUTO_OFFSET_RESET_CONFIG (:auto-offset-reset opts)) + + (not= nil (:enable-auto-commit opts)) + (assoc ConsumerConfig/ENABLE_AUTO_COMMIT_CONFIG (:enable-auto-commit opts)))) + +(defn producer-config + "Constructs a config map for talking to a given node." + [node opts] + ; See https://javadoc.io/doc/org.apache.kafka/kafka-clients/latest/org/apache/kafka/clients/producer/ProducerConfig.html + ; See https://docs.confluent.io/platform/current/installation/configuration/producer-configs.html + (cond-> {ProducerConfig/BOOTSTRAP_SERVERS_CONFIG + (str node ":" port) + + ProducerConfig/KEY_SERIALIZER_CLASS_CONFIG + ;"org.apache.kafka.common.serialization.StringSerializer" + "org.apache.kafka.common.serialization.LongSerializer" + + ProducerConfig/VALUE_SERIALIZER_CLASS_CONFIG + ;"org.apache.kafka.common.serialization.StringSerializer" + "org.apache.kafka.common.serialization.LongSerializer" + + ProducerConfig/DELIVERY_TIMEOUT_MS_CONFIG 10000 + ; We choose this lower than DELIVERY_TIMEOUT_MS so that we have a + ; chance to retry + ProducerConfig/REQUEST_TIMEOUT_MS_CONFIG 3000 + ProducerConfig/MAX_BLOCK_MS_CONFIG 10000 + ; Client sometimes complains `The configuration + ; 'transaction.timeout.ms' was supplied but isn't a known config`; + ; not sure what's up with that + ProducerConfig/TRANSACTION_TIMEOUT_CONFIG + 1000 + ; We want rapid reconnects so we can observe broken-ness + ProducerConfig/RECONNECT_BACKOFF_MAX_MS_CONFIG 1000 + ProducerConfig/SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG 500 + ProducerConfig/SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG 1000 + + } + (not= nil (:acks opts)) + (assoc ProducerConfig/ACKS_CONFIG (:acks opts)) + + (not= nil (:idempotence opts)) + (assoc ProducerConfig/ENABLE_IDEMPOTENCE_CONFIG (:idempotence opts)) + + (not= nil (:retries opts)) + (assoc ProducerConfig/RETRIES_CONFIG (:retries opts)) + + (not= nil (:transactional-id opts)) + (assoc ProducerConfig/TRANSACTIONAL_ID_CONFIG (:transactional-id opts)))) + +(defn admin-config + "Constructs a config map for an admin client connected to the given node." + [node] + ; See https://javadoc.io/doc/org.apache.kafka/kafka-clients/latest/org/apache/kafka/clients/admin/AdminClientConfig.html + {AdminClientConfig/BOOTSTRAP_SERVERS_CONFIG (str node ":" port) + AdminClientConfig/DEFAULT_API_TIMEOUT_MS_CONFIG 3000 + AdminClientConfig/RECONNECT_BACKOFF_MAX_MS_CONFIG 1000 + AdminClientConfig/REQUEST_TIMEOUT_MS_CONFIG 3000 + AdminClientConfig/SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG 500 + AdminClientConfig/SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG 1000 + ; Never retry + AdminClientConfig/RETRIES_CONFIG 0}) + +(defn ^Duration ms->duration + "Constructs a Duration from millis." + [ms] + (Duration/ofMillis ms)) + +(defn close! + "Closes any AutoCloseable." + [^java.lang.AutoCloseable c] + (.close c)) + +(defn close-consumer! + "Closes a consumer *immediately*. The default close claims it only blocks + 30s, but I've seen it deadlock for an hour." + [^KafkaConsumer c] + (.close c (ms->duration 0))) + +(defn close-producer! + "Closes a producer *immediately*, without waiting for incomplete requests." + [^KafkaProducer p] + (.close p (ms->duration 0))) + +(defn consumer + "Opens a new consumer for the given node." + [opts node] + (let [config (consumer-config node opts)] + (when (compare-and-set! consumer-config-logged? false true) + (info "Consumer config:\n" (pprint-str config))) + (KafkaConsumer. (->properties config)))) + +(defn producer* + "Opens a new producer for a node. Doesn't initialize transactions." + [opts node] + (let [config (producer-config node opts)] + (when (compare-and-set! producer-config-logged? false true) + (info "Producer config:\n" (pprint-str config))) + (KafkaProducer. (->properties config)))) + +(defn producer + "Opens a new producer for a node. Automatically initializes transactions, if + :transactional-id opts is set." + [opts node] + (if-not (:transactional-id opts) + (producer* opts node) + ; initTransactions loves to explode for nondetermistic (possibly + ; server-specific?) reasons, and when it does the entire producer winds up + ; locked in an irrecoverable state, so we have to do this akward + ; open-init-close dance + (await-fn (fn init-txns [] + (let [p (producer* opts node)] + (try (.initTransactions p) + p + (catch Throwable t + (close-producer! p) + (throw t))))) + {:log-interval 5000 + :log-message "Waiting for initTransactions()"}))) + +(defn admin + "Opens an admin client for a node." + [test node] + (Admin/create (->properties (admin-config node)))) + +(defn create-topic! + "Creates a new topic using an admin client. Synchronous. If the topic already + exists, returns :already-exists instead of throwing." + [^Admin admin name partitions replication-factor] + (try + (let [topic (NewTopic. ^String name, + ^int partitions, + ^short replication-factor) + res (.createTopics admin [topic])] + (.. res values (get name) get)) + (catch java.util.concurrent.ExecutionException e + (condp instance? (util/ex-root-cause e) + TopicExistsException :already-exists + (throw e))))) + +(defn ^TopicPartition topic-partition + "A tuple of a topic and a partition number together." + [topic partition] + (TopicPartition. topic partition)) + +(defn ^ProducerRecord producer-record + "Constructs a ProducerRecord from a topic, partition, key, and value." + [topic partition key value] + (ProducerRecord. topic (int partition) key value)) + +(defn ^OffsetAndMetadata offset+metadata + "Constructs an OffsetAndMetadata." + [^long offset] + (OffsetAndMetadata. offset)) + +(defn subscribe! + "Subscribes to the given set of topics." + ([^KafkaConsumer consumer, topics] + (.subscribe consumer topics)) + ([^KafkaConsumer consumer, topics, rebalance-listener] + (.subscribe consumer topics rebalance-listener))) + +(defn poll-up-to + "Takes a consumer, and polls it (with duration 0) for records up to and + including (dec offset), and (quite possibly) higher. Returns a lazy sequence + of ConsumerRecords. Helpful when you want to read everything up to at least + the given offset. You can pass an offset from consumer.endOffsets(...) to + this function directly to read everything in the topic up to that point. + + Assumes the consumer is subscribed to precisely one topic-partition. + + If offset is 0, returns nil. If offset is 1 (and consumer is seeked to 0), + returns a single element (the one at offset 0), and possibly more elements + after. If offset is 2, returns at least messages with offsets 0 and 1, and so + on. + + TODO: for reasons I don't really understand, the first call here (with + duration 1 ms) ALWAYS seems to return an empty list even when there's a bunch + of records pending. Subsequent requests (with duration 100 ms) return the + full set. Not sure what to do about this." + ([consumer offset] + (poll-up-to consumer offset (ms->duration 10))) + ([^KafkaConsumer consumer offset duration] + ; If the offset is zero, the partition is empty and we can return + ; immediately. + (when (pos? offset) + (let [records (.poll consumer duration) + records (vec records) + last-record ^ConsumerRecord (peek records)] + ;(info :poll-through-records offset records) + (cond ; Empty window; we should poll with a longer duration next time. + (nil? last-record) + (poll-up-to consumer offset (ms->duration 100)) + + ; We read far enough; we're done + (<= (dec offset) (.offset last-record)) + records + + ; Possibly more to come + true + (concat records + (lazy-seq (poll-up-to consumer offset duration)))))))) + +(defn ^KafkaConsumer reset-to-last-committed-positions! + "Takes a Consumer, and seeks back to the last offsets that were committed. + Returns consumer. Adapted from + https://github.com/apache/kafka/blob/7d9b9847f184ec72c4c80c046edc408789dcc066/examples/src/main/java/kafka/examples/ExactlyOnceMessageProcessor.java#L177-L184." + [^KafkaConsumer consumer] + (let [assignment (.assignment consumer) + committed (.committed consumer assignment)] + (doseq [^TopicPartition topic-partition assignment] + (if-let [^OffsetAndMetadata offset+metadata + (.get committed topic-partition)] + (.seek consumer topic-partition (.offset offset+metadata)) + (.seekToBeginning [topic-partition])))) + consumer) + +(defn abort-txn! + "Aborts a transaction." + [^KafkaProducer producer] + (.abortTransaction producer)) + +(defmacro unwrap-errors + "Depending on whether you're doing a future get or a sync call, Kafka might + throw its exceptions wrapped in a j.u.c.ExecutionException. This macro + transparently unwraps those." + [& body] + `(try ~@body + (catch ExecutionException e# + ; For debugging cases where the root exception gives us + ; zero stacktrace info + ; (throw e#) + (let [cause# (util/ex-root-cause e#)] + (if (instance? KafkaException cause#) + (throw cause#) + (throw e#)))))) + +(defn panicky-rebalance-listener + "A ConsumerRebalanceListener which throws at the drop of a hat. We use this + to make sure transactions aren't quietly having their topics/indices + reassigned during execution." + [] + (reify ConsumerRebalanceListener + (onPartitionsRevoked [_ topic-partitions] + (throw+ {:type :partitions-revoked + :partitions topic-partitions})) + + (onPartitionsAssigned [_ topic-partitions] + (throw+ {:type :partitions-assigned + :partitions topic-partitions})) + + (onPartitionsLost [_ topic-partitions] + (throw+ {:type :partitions-lost + :partitions topic-partitions})))) + +(defn logging-rebalance-listener + "A rebalance listener which journals each event to an atom containing a + vector." + [log-atom] + (reify ConsumerRebalanceListener + (onPartitionsRevoked [_ topic-partitions] + (swap! log-atom conj {:type :revoked + :partitions topic-partitions})) + + (onPartitionsAssigned [_ topic-partitions] + (swap! log-atom conj {:type :assigned + :partitions topic-partitions})) + + (onPartitionsLost [_ topic-partitions] + (swap! log-atom conj {:type :lost + :partitions topic-partitions})))) diff --git a/src/jepsen/hstream/kafka/db.clj b/src/jepsen/hstream/kafka/db.clj new file mode 100644 index 0000000..cc5d7eb --- /dev/null +++ b/src/jepsen/hstream/kafka/db.clj @@ -0,0 +1,9 @@ +(ns jepsen.hstream.kafka.db + "Common functions across both Kafka and Redpanda.") + +(defprotocol DB + (node-id [db test node] + "What's the ID of this node?") + + (topic-partition-state [db node topic-partition] + "Fetches information about the topic-partition state from this node.")) diff --git a/src/jepsen/hstream/kafka/db/hstream.clj b/src/jepsen/hstream/kafka/db/hstream.clj new file mode 100644 index 0000000..5e78deb --- /dev/null +++ b/src/jepsen/hstream/kafka/db/hstream.clj @@ -0,0 +1,38 @@ +(ns jepsen.hstream.kafka.db.hstream + (:require [clojure.tools.logging :refer [info]] + [jepsen.db :as db] + [jepsen.hstream.kafka.db :as redpanda.db])) + +(defn db + "HStream DB for a particular version. No action is executed after the DB is ready." + [version] + (reify + db/DB + (setup! [this test node] + (info ">>> Setting up DB: HStream" version "on node" node + "But in fact we did nothing here.")) + (teardown! [this test node] + (info ">>> Tearing down DB: HStream" version "on node" node + "But in fact we did nothing here.")) + + db/Process + (start! [this test node] + ) + (kill! [this test node] + ) + + db/Pause + (pause! [this test node] + ) + (resume! [this test node] + ) + + db/LogFiles + (log-files [this test node] + {}) + + redpanda.db/DB + (node-id [this test node] + 0) + (topic-partition-state [this node topic-partition] + :not-implemented))) diff --git a/src/jepsen/hstream/kafka/db/kafka.clj b/src/jepsen/hstream/kafka/db/kafka.clj new file mode 100644 index 0000000..a3a2234 --- /dev/null +++ b/src/jepsen/hstream/kafka/db/kafka.clj @@ -0,0 +1,221 @@ +(ns jepsen.hstream.kafka.db.kafka + "Database automation for the Kafka database: setup, teardown, etc." + (:require [cheshire.core :as json] + [clj-http.client :as http] + [clojure [string :as str]] + [clojure.tools.logging :refer [info warn]] + [dom-top.core :refer [assert+]] + [jepsen [control :as c :refer [|]] + [core :as jepsen] + [db :as db] + [util :as util :refer [pprint-str meh]]] + [jepsen.control [net :as cn] + [util :as cu]] + [jepsen.os.debian :as debian] + [jepsen.hstream.kafka.db :as redpanda.db] + [jepsen.hstream.kafka.db.redpanda :as rdb] + [slingshot.slingshot :refer [try+ throw+]]) + (:import (org.apache.kafka.common TopicPartition))) + +(def dir "Where do we install Kafka?" + "/opt/kafka") + +(def zk + "Program that launches zookeeper." + (str dir "/bin/zookeeper-server-start.sh")) + +(def zk-config + "Properties file for Zookeeper" + (str dir "/config/zookeeper.properties")) + +(def kafka-config + "Properties file for Kafka" + (str dir "/config/server.properties")) + +(def kafka + "Program that launches kafka." + (str dir "/bin/kafka-server-start.sh")) + +(def kafka-log-file + (str dir "/kafka.log")) + +(def zk-log-file + (str dir "/zk.log")) + +(def kafka-pid-file + (str dir "/kafka.pid")) + +(def zk-pid-file + (str dir "/zk.pid")) + +(def kafka-data + (str dir "/data/kafka")) + +(def zk-data + (str dir "/data/zk")) + +(defn install! + "Installs Kafka." + [test] + ; We ignore version and deb here; this is just a quick comparison against a + ; hardcoded version. + (let [url (str "https://dlcdn.apache.org/kafka/3.0.0/kafka_2.13-3.0.0.tgz")] + (c/su (cu/install-archive! url dir) + ; Make data dirs + (c/exec :mkdir :-p zk-data) + (c/exec :mkdir :-p kafka-data)))) + +(defn configure! + "Writes config files" + [test node] + (c/su + ; Write node IDs + (let [id (rdb/node-id test node)] + (c/exec :echo id :> (str zk-data "/myid")) + ; Replace ZK data dir + (c/exec :sed :-i :-e (str "s/dataDir=.*/dataDir=" + (str/escape zk-data {\/ "\\/"}) + "/") zk-config) + ; Add cluster nodes, init limit, etc to ZK config + (let [nodes (->> (:nodes test) + (map (fn [node] + (let [id (rdb/node-id test node)] + (str "server." id "=" node ":2888:3888"))))) + conf (->> (concat nodes + ["tickTime=1000" + "initLimit=10" + "syncLimit=5"]) + (str/join "\n"))] + (c/exec :echo conf :>> zk-config)) + + ; Replace Kafka node ID + (c/exec :sed :-i :-e (str "s/broker\\.id=.*/broker.id=" id "/") kafka-config) + ; Set internal replication factors + (c/exec :sed :-i :-e "s/offsets\\.topic\\.replication\\.factor=.*/offsets.topic.replication.factor=3/" kafka-config) + (c/exec :sed :-i :-e "s/transaction\\.state\\.log\\.replication\\.factor=.*/transaction.state.log.replication.factor=3/" kafka-config) + (c/exec :sed :-i :-e "s/transaction\\.state\\.log\\.min\\.isr=.*/transaction.state.log.min.isr=3/" kafka-config) + ; And rebalance delay, to speed up startup + (c/exec :sed :-i :-e "s/group\\.initial\\.rebalance\\.delay\\.ms=.*/group.initial.rebalance.delay.ms=3000/" kafka-config) + + ; And data dir + (c/exec :sed :-i :-e (str "s/log\\.dirs=.*/log.dirs=" + (str/escape kafka-data {\/ "\\/"}) + "/") kafka-config) + ; We'll write our own in a second + (c/exec :sed :-i :-e "s/zookeeper\\.connect=.*//" kafka-config) + + ; Add advertised listeners etc to Kafka settings + (let [lines [(str "advertised.listeners=PLAINTEXT://" node ":9092") + (let [r (:default-topic-replications test)] + (when-not (nil? r) + (str "default.replication.factor=" r))) + ; Default ISR is too weak + "min.insync.replicas=2" + ; Shorten ZK timeouts, or else Kafka will take forever to + ; recover from faults + "zookeeper.session.tmeout.ms=1000" + ; ZK nodes + (->> (:nodes test) + (map (fn [node] (str node ":2181"))) + (str/join ",") + (str "zookeeper.connect="))] + lines (str/join "\n" lines)] + (c/exec :echo lines :>> kafka-config))))) + +(defn start-zk! + "Starts zookeeper." + [] + (c/su + (cu/start-daemon! {:chdir dir + :logfile zk-log-file + :pidfile zk-pid-file} + zk + zk-config))) + +(defn kill-zk! + "Kills zookeeper." + [] + (c/su + (cu/stop-daemon! zk zk-pid-file) + (try+ + (c/exec (c/env {:SIGNAL "KILL"}) + (str dir "/bin/zookeeper-server-stop.sh")) + (catch [:exit 1] e + (if (re-find #"No zookeeper server to stop" (:out e)) + nil + (throw+ e))) + (catch [:exit 127] e + (if (re-find #"No such file or directory" (:err e)) + ; First run + nil + (throw+ e)))))) + +(defrecord DB [node-ids] + db/DB + (setup! [this test node] + ; We need a full set of node IDs for this + (rdb/gen-node-id! test node) + (jepsen/synchronize test) + + (install! test) + (configure! test node) + (start-zk!) + (jepsen/synchronize test) + + (db/start! this test node)) + + (teardown! [this test node] + (db/kill! this test node) + (kill-zk!) + (c/su + (c/exec :rm :-rf dir))) + + db/Process + (start! [this test node] + (c/su + (cu/start-daemon! {:chdir dir + :logfile kafka-log-file + :pidfile kafka-pid-file} + kafka + kafka-config))) + + (kill! [this test node] + (c/su + (cu/stop-daemon! kafka kafka-pid-file) + (try+ + (c/exec (c/env {:SIGNAL "KILL"}) + (str dir "/bin/kafka-server-stop.sh")) + (catch [:exit 1] e + (if (re-find #"No kafka server to stop" (:out e)) + nil + (throw+ e))) + (catch [:exit 127] e + (if (re-find #"No such file or directory" (:err e)) + ; First run + nil + (throw+ e)))))) + + db/Pause + (pause! [this test node] + ) + + (resume! [this test node] + ) + + db/LogFiles + (log-files [this test node] + {zk-log-file "zk.log" + kafka-log-file "kafka.log"}) + + redpanda.db/DB + (node-id [this test node] + (rdb/node-id test node)) + + (topic-partition-state [this node topic-partition] + :not-implemented)) + +(defn db + "Constructs a Jepsen database object which knows how to set up and teardown a + Kafka cluster." + [] + (map->DB {:node-ids (atom {})})) diff --git a/src/jepsen/hstream/kafka/db/redpanda.clj b/src/jepsen/hstream/kafka/db/redpanda.clj new file mode 100644 index 0000000..346236f --- /dev/null +++ b/src/jepsen/hstream/kafka/db/redpanda.clj @@ -0,0 +1,468 @@ +(ns jepsen.hstream.kafka.db.redpanda + "Database automation: setup, teardown, some node-related nemesis operations + for the Redpanda database." + (:require [cheshire.core :as json] + [clj-http.client :as http] + [clojure [string :as str]] + [clojure.tools.logging :refer [info warn]] + [dom-top.core :refer [assert+]] + [jepsen [control :as c :refer [|]] + [core :as jepsen] + [db :as db] + [util :as util :refer [pprint-str meh]]] + [jepsen.control [net :as cn] + [util :as cu]] + [jepsen.os.debian :as debian] + [jepsen.hstream.kafka.db :as redpanda.db] + [slingshot.slingshot :refer [try+ throw+]]) + (:import (org.apache.kafka.common TopicPartition))) + +(def user + "What user do we run RPK commands as?" + "redpanda") + +(def data-dir + "Where does Redpanda data live?" + "/var/lib/redpanda/data") + +(def pid-file + "Where do we store the redpanda pid?" + "/var/lib/redpanda/data/pid.lock") + +(def log-file + "Where do we send stdout/stderr logs?" + "/var/log/redpanda.log") + +(def config-file + "Where does the redpanda config file live?" + "/etc/redpanda/redpanda.yaml") + +(def enabled-file + "A file we use to tell whether redpanda is able to start or not." + "/etc/redpanda/jepsen-enabled") + +(def nofile + "The ulimit number of files we apply to the redpanda process." + (long (Math/pow 2 20))) ; ~ 1 million + +(defn rpk! + "Runs an RPK command, just like c/exec." + [& args] + (c/sudo user + (c/exec :rpk args))) + +(defn install! + "Installs Redpanda on the local node." + [test] + (c/su + (c/exec :curl :-1sLf "https://packages.vectorized.io/nzc4ZYQK3WRGd9sy/redpanda/cfg/setup/bash.deb.sh" | :sudo :-E :bash) + ; Wipe out config file so we don't re-use previous run settings + (c/exec :rm :-f "/etc/redpanda/redpanda.yaml") + (if-let [deb (:deb test)] + ; Install custom debian package + (let [dir "/tmp/jepsen/redpanda" + remote-deb (str dir "/deb")] + (info "Installing" deb) + (c/exec :mkdir :-p dir) + (c/upload deb remote-deb) + (c/exec :dpkg :--force-confmiss :-i remote-deb)) + ; Install from apt repo + (c/exec :apt-get :-y + :--allow-downgrades + :--allow-change-held-packages + :-o "DPkg::options::=--force-confmiss" + :--reinstall :install (str "redpanda=" (:version test)))) + ; We're going to manage daemons ourselves + (c/exec :systemctl :stop :redpanda) + (c/exec :systemctl :stop :wasm_engine) + (c/exec :systemctl :disable :redpanda) + (c/exec :systemctl :disable :wasm_engine) + )) + +(defn gen-node-id! + "Generates a new node ID for a node, mutating the node-ids atom in the test's + DB. Returns that node ID. Node IDs are allocated so that all IDs for a given + node have the same (mod id node-count) value." + [test node] + (-> test :db :node-ids + (swap! (fn [node-ids] + (let [id (or (when-let [id (get node-ids node)] + (+ id (count (:nodes test)))) + (.indexOf ^java.util.List (:nodes test) node))] + (when (neg? id) + (throw+ {:type ::node-not-in-cluster + :node node + :nodes (:nodes test)})) + (assoc node-ids node id)))) + (get node))) + +(defn node-id->node + "Takes a test and a node ID and returns the node name that ID must have been + for." + [test node-id] + (let [nodes (:nodes test)] + (nth nodes (mod node-id (count nodes))))) + +(defn node-id + "Takes a test and a node name, and returns the current numeric ID (e.g. 0, 1, + ...) for that node. The first node in the test is initially node 0, the root + node." + [test node] + (-> test :db :node-ids deref (get node))) + +(defn configure! + "Sets up the local node's config. Pass initial? true when first bootstrapping + the cluster." + ([test node] + (configure! test node false)) + ([test node initial?] + (c/su + (let [id (node-id test node)] + ; If Redpanda creates a topic automatically, it might be under-replicated + (let [e (:enable-server-auto-create-topics test)] + (when-not (nil? e) + ;(info "Setting redpanda.auto_create_topics_enabled" e) + (rpk! :config :set "redpanda.auto_create_topics_enabled" false))) + + ; Redpanda's internal topic kafka_internal/group/0 only has replication + ; factor of 1; you need to set it to 3 if you want actual fault + ; tolerance for consumer groups. + (let [r (:default-topic-replications test)] + (when-not (nil? r) + ;(info "Setting default-topic_replications" r) + (rpk! :config :set "redpanda.default_topic_replications" r))) + + ; Set up idempotence + (when (:server-idempotence test) + (rpk! :config :set :redpanda.id_allocator_replication 3) + (rpk! :config :set :redpanda.enable_idempotence true)) + + ; And transactions + (when (:txn test) + (rpk! :config :set :redpanda.transaction_coordinator_replication 3) + (rpk! :config :set :redpanda.enable_transactions true)) + + (rpk! :config :bootstrap + :--id id + :--self (cn/local-ip) + ; On the initial run, the 0th node has an empty ips list and all + ; other nodes join it. On subsequent runs, we join to every ip. + (when-not (and initial? (zero? id)) + [:--ips (->> (:nodes test) + (remove #{node}) + (map cn/ip) + (str/join ","))])))))) + +(defn check-topic-creation + "Checks that you can create a topic." + [] + (c/su + (let [topic "jepsen-test" + res (rpk! :topic :create topic)] + (when-not (re-find #"\s+(OK|TOPIC_ALREADY_EXISTS)" res) + (throw+ {:type ::topic-create-failed + :res res})) + ; Clean up if we can + (meh (rpk! :topic :delete topic)) + topic))) + +(defn await-topic-creation + "Waits until you can create a topic." + [] + (util/await-fn check-topic-creation + {:log-interval 10000 + :log-message "Waiting for topic creation"})) + +(defn disable! + "Disables starting redpanda on the current node." + [] + (c/su (c/exec :rm :-f enabled-file))) + +(defn enable! + "Enables starting redpanda on the current node." + [] + (c/su (c/exec :touch enabled-file))) + +(defn enabled? + "Can we start redpanda on this node?" + [] + (cu/exists? enabled-file)) + +(defn nuke! + "Kills the process, wipes data files, and makes it impossible to start this + node. Leaves the log intact." + [test node] + (info "Nuking" node) + (disable!) + (db/kill! (:db test) test node) + (c/su + (c/exec :rm :-rf + pid-file + (c/lit (str data-dir "/*"))))) + +(declare topic-partition-state) + +(defrecord DB [node-ids tcpdump] + db/DB + (setup! [this test node] + ; Generate an initial node ID + (gen-node-id! test node) + (install! test) + (enable!) + (configure! test node true) + + (when (:tcpdump test) + (db/setup! tcpdump test node)) + + (c/su + ; Make sure log file is ready + (c/exec :touch log-file) + (c/exec :chown (str user ":" user) log-file)) + + ; Start primary + (when (zero? (node-id test node)) + (db/start! this test node)) + (jepsen/synchronize test) + + ; Then secondaries + (when-not (zero? (node-id test node)) + (db/start! this test node)) + + ; Wait for cluster to be ready. + (await-topic-creation)) + + (teardown! [this test node] + (nuke! test node) + (c/su + (c/exec :rm :-f log-file)) + (when (:tcpdump test) + (db/teardown! tcpdump test node))) + + db/LogFiles + (log-files [this test node] + ; Tar up the data dir + (let [tarball "/tmp/jepsen/data.tar.bz2"] + (c/su (c/exec :mkdir :-p "/tmp/jepsen") + (c/exec :tar :cjf tarball data-dir)) + (merge (when (:tcpdump test) + (db/log-files tcpdump test node)) + {log-file "redpanda.log" + tarball "data.tar.bz2"}))) + + db/Process + (start! [this test node] + (if-not (enabled?) + :disabled + (do (c/sudo user + (cu/start-daemon! + {:chdir "/" + :logfile log-file + :pidfile pid-file + :make-pidfile? false} + "/usr/bin/redpanda" + :--redpanda-cfg config-file)) + ; Bump up filehandle limit + (c/su (let [pid (util/await-fn + (fn get-pid [] + (let [pid (c/exec :cat pid-file)] + (when-not (re-find #"\d+\n?" pid) + (throw+ {:type :no-pid-in-file})) + pid)) + {:log-message "waiting for startup to apply ulimit" + :log-interval 10000})] + (try+ + (c/exec :prlimit (str "--nofile=" nofile) :--pid pid) + (catch [:type :jepsen.control/nonzero-exit] e + (if (re-find #"No such process" (:err e)) + (throw+ {:type :ulimit-failed}) + (throw+ e))))))))) + + + (kill! [this test node] + (c/su + (cu/stop-daemon! :redpanda pid-file))) + + db/Pause + (pause! [this test node] + (c/su + (cu/grepkill! :stop :redpanda))) + + (resume! [this test node] + (c/su + (cu/grepkill! :cont :redpanda))) + + redpanda.db/DB + (node-id [this test node] + (node-id test node)) + + (topic-partition-state [this node topic-partition] + (topic-partition-state node topic-partition))) + +(defn db + "Constructs a Jepsen database object which knows how to set up and tear down + a Redpanda cluster." + [] + (map->DB {:node-ids (atom {}) + :tcpdump (db/tcpdump {:ports [8082 9092 9644 33145]})})) + +;; Cluster ops + +(defn lowercase + "Converts a string to lowercase." + [^String s] + (.toLowerCase s)) + +(defn part-str + "Takes a string s and a seq of column widths. Returns a seq of substrings + of s, each corresponding to one of those columns, plus one final column of + unbounded width." + ([s widths] + (part-str s 0 widths [])) + ([s offset widths columns] + (if-not (seq widths) + ; Last column + (conj columns (subs s offset)) + ; Intermediate column + (let [[width & widths'] widths + offset' (+ offset width)] + (recur s offset' widths' (conj columns (subs s offset offset'))))))) + +(defn parse-header + "Takes a string of whitespace-separated names and returns a seq of {:name + :width} maps describing those columns. Names are lowercased keywords." + ([s] + (->> (re-seq #"[^\s]+\s*" s) + (map (fn [^String column] + {:name (-> column str/trimr lowercase keyword) + :width (.length column)}))))) + +(defn parse-tables + "Parses a table structure like + + BROKERS + ======= + ID HOST PORT + 0* 192.168.122.101 9092 + 1 192.168.122.102 9092 + 2 192.168.122.103 9092 + + TOPICS + ====== + NAME PARTITIONS REPLICAS + t4 2 3 + + Into a map like {:brokers [{:id \"0*\" :host \"192...\" ...} ...] ...}" + [s] + (loop [lines (str/split s #"\n") ; Unconsumed lines + section nil ; What section are we in? + expect :section ; What do we expect next? + column-names [] ; A seq of names for each column + column-widths [] ; A seq of how many characters wide each col is + data {}] ; A map of sections to vectors of rows, + ; each row a map of column names to strings + (if-not (seq lines) + data + (let [[line & lines'] lines] + (case expect + :divider + (do (assert+ (re-find #"^=+$" line) + {:type ::parse-error + :expected "A divider like ===" + :actual line}) + (recur lines' section :header column-names column-widths data)) + + :header + (let [header (parse-header line)] + (recur lines' section :row-or-blank + (mapv :name header) + (butlast (map :width header)) + data)) + + :row-or-blank + (condp re-find line + ; Blank + #"^$" (recur lines' nil :section [] [] data) + ; Row + (let [row (->> (part-str line column-widths) + (map str/trimr) + (zipmap column-names)) + rows (get data section []) + rows' (conj rows row) + data' (assoc data section rows')] + (recur lines' section expect column-names column-widths data'))) + + :section + (recur lines' (-> line lowercase keyword) :divider [] [] data)))))) + +(defn parse-cluster-info + "Parses a cluster info string." + [s] + (-> (parse-tables s) + (update :brokers + (partial map + (fn [broker] + (let [[_ id star] (re-find #"(\d+)(\*)?" (:id broker))] + (-> broker + (assoc :id (parse-long id)) + (cond-> star (assoc :star? true)) + (update :port parse-long)))))) + (update :topics + (partial map + (fn [topic] + (-> topic + (update :partitions parse-long) + (update :replicas parse-long))))))) + +(defn cluster-info + "Returns cluster info from the current node as a clojure structure, by + shelling out to rpk. Returns nil if node is unavailable." + [] + (try+ + (parse-cluster-info (rpk! :cluster :info)) + (catch [:type :jepsen.control/nonzero-exit, :exit 1] e + nil))) + +(def common-http-opts + "Common options for HTTP requests." + {:socket-timeout 10000 + :connection-timeout 1000 + :as :json}) + +(defn broker-state + "Fetches the broker state from the given node via HTTP." + [node] + (-> (http/get (str "http://" node ":9644/v1/brokers") + common-http-opts) + :body)) + +(defn topic-partition-state + "Takes a node and a TopicPartition. Fetches the topic-partition data from + this node's /partitions/kafka// API. Topics must be + URL-safe; we're not encoding them here." + [node ^TopicPartition topic-partition] + (:body (http/get (str "http://" node ":9644/v1/partitions/kafka/" + (.topic topic-partition) "/" + (.partition topic-partition)) + common-http-opts))) + +(defn cluster-view + "Returns a versioned view of the cluster from the new API. This was + introduced in development builds after 21.11.2 because neither `rpk cluster + info` nor `/brokers` could be trusted. ;-)" + [node] + (:body (http/get (str "http://" node ":9644/v1/cluster_view") + common-http-opts))) + +(defn decommission! + "Asks a node `via` to decommission a node id `target`" + [via target] + (try+ + (-> (http/put (str "http://" via ":9644/v1/brokers/" target "/decommission") + common-http-opts) + :body) + (catch [:status 400] e + ; Try parsing message as data structure + (throw+ (update e :body json/parse-string true))) + (catch [:status 500] e + ; Try parsing message as data structure + (throw+ (update e :body json/parse-string true))))) diff --git a/src/jepsen/hstream/kafka/nemesis.clj b/src/jepsen/hstream/kafka/nemesis.clj new file mode 100644 index 0000000..ffd655a --- /dev/null +++ b/src/jepsen/hstream/kafka/nemesis.clj @@ -0,0 +1,364 @@ +(ns jepsen.hstream.kafka.nemesis + "Fault injection for Redpanda clusters" + (:require [clojure [set :as set]] + [clojure.tools.logging :refer [info warn]] + [dom-top.core :as dt :refer [assert+]] + [jepsen [control :as c] + [db :as db] + [nemesis :as n] + [util :as util :refer [pprint-str rand-nth-empty max-by]]] + [jepsen.control.net :as cn] + [jepsen.nemesis [combined :as nc] + [membership :as membership]] + [jepsen.hstream.kafka.db.redpanda :as rdb] + [slingshot.slingshot :refer [try+ throw+]])) + +(def min-cluster-size + "How small do we allow clusters to become? I think 1 breaks Raft maybe?" + 3) + +(def ip->node-cache + "A map of IP addresses to node names. Lazily built on first use." + (promise)) + +(defn ip->node + "Turns an IP address into a node." + [test ip] + (when-not (realized? ip->node-cache) + (deliver ip->node-cache + (->> (c/on-nodes test (fn [test node] (cn/local-ip))) + (map (juxt second first)) + (into {})))) + (let [node (@ip->node-cache ip)] + (assert+ node + {:type ::no-node-with-ip + :ip ip + :cache @ip->node-cache}) + node)) + +(defn node->ip + "Turns a node into an IP address" + [test node] + (first (vals (c/on-nodes test [node] + (fn [test node] + (cn/ip node)))))) + +(defn find-by + "Takes a function f, a value v, and a collection of xs. Returns the first x + such that (= v (f x)). Helpful in extracting, say, a node with the given id + from a list of node maps: (find-by :id 0 brokers)." + [f v xs] + (first (filter (fn [x] (= v (f x))) xs))) + +(defn nodes-in-states + "Takes a node state map from the membership and returns a vector of all nodes + which are in the given set of states." + [states nodes] + (->> nodes + (filter (comp states val)) + (mapv key))) + +(defn known-to-active-majority? + "Is a given node ID known to a majority of nodes in the :active state? Argh, + this is such a race-condition-prone hack, but... hopefully it'll work + sometimes?" + [{:keys [node-views nodes]} id] + (let [active (nodes-in-states #{:active} nodes) + known (->> active + (keep (fn [node] + (->> (get node-views node) + :brokers + (find-by :id id)))))] + (if (< (count active) 3) + (do (warn "Fewer than 3 active nodes in cluster:" (pr-str active) "\n" + (pprint-str nodes) + "\n\n" + (pprint-str node-views)) + false) + (< 1/2 (/ (count known) (count active)))))) + +(defn remove-node-op + "We can remove a node from a cluster if it's in the view, and removing it + wouldn't bring us below the minimum cluster size." + [test {:keys [view nodes]}] + (let [; What nodes, if we were to issue a remove command for them, would + ; leave the cluster? + active-set (set (nodes-in-states #{:active :adding} nodes))] + ; What nodes *could* we try to remove? Note that it's important that we try + ; to re-remove nodes which are already removing, because the request to + ; remove that node might have crashed in an indeterminate way, resulting in + ; a remove which would never actually complete unless we retry. + (->> (nodes-in-states #{:adding :active :removing} nodes) + (keep (fn [node] + (let [;ip (node->ip test node) + id (->> view :brokers (find-by :node node) :id) + ; What would the new active set be? + active' (disj active-set node) + ; Who should we ask to do the remove? + via (->> nodes + (nodes-in-states #{:active}) + rand-nth-empty)] + (when (and id via (<= min-cluster-size (count active'))) + {:type :info + :f :remove-node + :value {:node node, :id id, :via via}})))) + vec + rand-nth-empty))) + +(defn free-node-op + "Nodes which have been removed can be freed, killing the process, deleting + their data files, and marking them for possible re-entry to the cluster." + [test {:keys [nodes]}] + (->> (nodes-in-states #{:removed} nodes) + (map (fn [node] + {:type :info + :f :free-node + :value {:node node}})) + rand-nth-empty)) + +(defn add-node-op + "We can add any free node to the cluster." + [test {:keys [nodes]}] + (->> (nodes-in-states #{:free} nodes) + (map (fn [node] + {:type :info + :f :add-node + :value {:node node}})) + rand-nth-empty)) + +(defn resolve-op + "Takes a membership state and an [op op'] pair of nemesis operations within + its pending set. Attempts to resolve that operation, returning a new + membership state." + [this [{:keys [f value] :as op} op' :as pair]] + (cond ; It's possible to add and remove a node in rapid succession, + ; such that the add op remains pending because we never got a chance to + ; see it. Once a node is free, we clear all pending ops associated with + ; that node, regardless of instance. + (= :free (get (:nodes this) (:node (:value op)))) + (update this :pending disj pair) + + true + (case f + ; Once an adding node is known to be active in the current view, we + ; consider it active, and mark its add op as resolved. + :add-node + (if (->> this :view :brokers + (find-by :id (:id (:value op'))) + :status (= :active)) + ; Accepted! + (let [node (:node value)] + (-> this + (update :nodes assoc node :active) + (update :pending disj pair))) + this) ; Still waiting! + + ; Free always completes synchronously + :free-node + (update this :pending disj pair) + + :remove-node + (cond ; Definitely didn't happen; we're done here. + (= 400 (:code (:error op'))) + (update this :pending disj pair) + + ; Likewise, can't have happened. + (= :connect-exception (:type (:error op'))) + (update this :pending disj pair) + + ; OK, this op might have or definitely did take place. But if a + ; majority of active nodes still think it's in the cluster, + ; we'll wait. + :else + (let [state (->> this :view :brokers + (find-by :id (:id (:value op))))] + (info "Removing node" (:value op) "has state" (pr-str state)) + (if state + ; Node still in cluster; not done! + this + ; Done! + (-> this + (update :nodes assoc (:node (:value op)) :removed) + (update :pending disj pair))))) + + ; Dunno how to resolve this :f + nil + this))) + +(defrecord Membership + [node-views ; A map of nodes to that node's local view of the cluster + view ; Our merged view of cluster state. Looks like: + ; {:version 123 + ; :brokers [{:node "n1", :id 3, :alive? true, :status :active}]} + pending ; Pending [op op'] pairs of membership operations we're waiting + ; to resolve + nodes ; A map of nodes to states. Node states are: + ; :init Freshly joined. We need to wait for these to be + ; reflected on all test nodes before we + ; consider them :active. + ; :active In the cluster + ; :adding Joining the cluster but not yet complete + ; :removing Being removed from the cluster but not yet complete + ; :removed The cluster believes this node is gone, but we + ; haven't killed it and wiped its data files yet + ; :free Not in the cluster at all; blank slate + ] + + membership/State + (setup! [this test] + (assoc this + :nodes (zipmap (:nodes test) (repeat :init)))) + + (node-view [this test node] + (try+ (let [enabled? (-> test + (c/on-nodes [node] + (fn enabled? [_ _] (rdb/enabled?))) + (get node))] + (assert (or (= true enabled?) (= false enabled?)) + (str "Expected bool, got " (pr-str enabled?))) + (if-not enabled? + {:version ##-Inf} ; Not in cluster + (let [{:keys [version brokers]} (rdb/cluster-view node)] + {:version version + :brokers + (map (fn broker [{:keys [node_id membership_status is_alive]}] + {:id node_id + :node (rdb/node-id->node test node_id) + :status (keyword membership_status) + :alive? is_alive}) + (sort-by :node_id brokers))}))) + (catch java.net.ConnectException e + nil) + (catch [:status 503] e + (warn "Node" node + "returned 503 for membership view:" (pr-str e)) + nil) + (catch [:status 404] _ + (warn "Node" node + "returned 404 for membership view--does it support this API?") + nil))) + + (merge-views [this test] + ; Pick the highest version + (->> node-views vals (max-by :version))) + + (fs [this] + #{:add-node + :free-node + :remove-node}) + + (op [this test] + (if-let [op (->> [(free-node-op test this) + (add-node-op test this) + (remove-node-op test this)] + (remove nil?) + rand-nth-empty)] + (assoc-in op [:value :view-version] (:version view)) + :pending)) + + (invoke! [this test {:keys [f value] :as op}] + (case f + :add-node + (try+ + (let [this' (update this :nodes assoc (:node value) :adding) + node (:node value) + id (rdb/gen-node-id! test node)] + [(-> (c/on-nodes test [node] + (fn [test node] + (rdb/configure! test node) + (rdb/enable!) + (assoc op :value + (assoc value + :id id + :result (db/start! (:db test) + test node))))) + first + val) + this'])) + + :free-node + (let [node (:node value)] + (c/on-nodes test [node] rdb/nuke!) + [(assoc op :done? true) + (update this :nodes assoc node :free)]) + + :remove-node + ; First, flag this node as being in state removing + (let [this' (update this :nodes assoc (:node value) :removing)] + ; Then try to actually remove it + (try+ + (rdb/decommission! (:via value) (:id value)) + [(update op :value assoc :ok? true) this'] + (catch [:status 400] e + ; When our request is rejected, we know the node isn't being + ; removed and leave it in the original state. + [(assoc op :error (:body e)) this]) + (catch [:status 503] e + ; This could go either way, so we flag the node as :removing just + ; in case. + [(assoc op :error (:body e)) this']) + (catch [:status 500] e + ; This could go either way, so we flag the node as :removing just + ; in case. + [(assoc op :error (:body e)) this']) + (catch java.net.SocketTimeoutException e + ; Not sure if this happened or not + [(assoc op :error {:type :socket-timeout}) this']) + (catch java.net.ConnectException e + ; Can't have happened + [(assoc op :error {:type :connect-exception}) this]))))) + + (resolve [this test] + ; When we start off, we've got nodes in state :init--we try to transition + ; those to :active as soon as they're present in the view. This is our + ; bootstrapping process--we can't trust the active set on startup because + ; cluster join might be incomplete, leaving some "active" nodes not + ; actually knowing who's in the cluster. + (let [this (reduce (fn check-init-join [this node] + (if (->> this :view :brokers (find-by :node node)) + ; This node is now in the cluster view. + (update this :nodes assoc node :active) + ; Still waiting + this)) + this + (nodes-in-states #{:init} (:nodes this)))] + ; Now handle pending ops + (reduce resolve-op this pending))) + + ; Unused; we implement resolve manually. + (resolve-op [this test pair]) + + (teardown! [this test])) + +(defn membership-package + "A nemesis package which can perform membership changes." + [opts] + (-> opts + (assoc :membership {:state (map->Membership + {}) + :log-node-views? true + :log-resolve? true + :log-view? true}) + membership/package + (assoc :perf #{{:name "member" + :fs #{:add-node :free-node :remove-node} + :color "#A66AD8"}}))) + +(defn package + "Given CLI options, constructs a package of {:generator, :final-generator, + :nemesis, ...}" + [opts] + (let [faults (set (:faults opts [:partition :kill :pause :clock])) + opts (assoc opts "faults" faults)] + (nc/compose-packages [(nc/partition-package opts) + (nc/db-package opts) + (membership-package opts)]))) +;; Note: Docker does not support clock-related nemesis. +;; So we just expanded `nc/nemesis-packages` and +;; omitted `nc/clock-package`. +;; TODO: Jepsen 0.3.x added some new nemesis such as +;; packet and file-corruption. Add them if needed. +;; (-> opts +;; nc/nemesis-packages +;; (conj (membership-package opts)) +;; nc/compose-packages)) diff --git a/src/jepsen/hstream/kafka/workload/list_append.clj b/src/jepsen/hstream/kafka/workload/list_append.clj new file mode 100644 index 0000000..e20b0d1 --- /dev/null +++ b/src/jepsen/hstream/kafka/workload/list_append.clj @@ -0,0 +1,160 @@ +(ns jepsen.hstream.kafka.workload.list-append + "A workload which treats a Kafka topic as an ordered list of numbers, and + performs transactional (or non-transactional) appends and reads of the entire + list." + (:require [clojure.tools.logging :refer [info warn]] + [dom-top.core :refer [assert+]] + [jepsen [client :as client] + [generator :as gen] + [util :as util :refer [pprint-str]]] + [jepsen.tests.cycle.append :as append] + [jepsen.hstream.kafka [client :as rc]]) + (:import (java.util.concurrent ExecutionException) + (org.apache.kafka.clients.consumer ConsumerRecord) + (org.apache.kafka.common.errors InvalidTopicException + NotLeaderOrFollowerException + TimeoutException + UnknownTopicOrPartitionException + ))) + +(def partition-count + "How many partitions per topic?" + 2) + +(defn k->topic + "Turns a logical key into a topic." + [k] + (str "t" (quot k partition-count))) + +(defn k->partition + "Turns a logical key into a partition within a topic." + [k] + (mod k partition-count)) + +(defn k->topic-partition + "Turns a logical key into a TopicPartition." + [k] + (rc/topic-partition (k->topic k) (k->partition k))) + +(def replication-factor + "What replication factor should we use for each topic?" + 3) + +(defn mop! + "Applies a micro-operation from a transaction: either a :r read or a :append + operation." + [{:keys [extant-topics admin producer consumer] :as client} [f k v :as mop]] + (let [topic (k->topic k) + topic-partition (k->topic-partition k)] + ; Create topic if it doesn't exist. + (when-not (contains? @extant-topics topic) + (rc/create-topic! admin topic partition-count replication-factor) + (swap! extant-topics conj topic)) + + (case f + :r + (try + ; Start by assigning our consumer to this particular topic, seeking + ; the consumer to the beginning, then reading the entire topic. + (doto consumer + (.assign [topic-partition]) + (.seekToBeginning [topic-partition])) + + ; How far do we have to read? + (let [end-offset (-> consumer + (.endOffsets [topic-partition]) + (get topic-partition)) + ; Read at least that far + records (rc/poll-up-to consumer end-offset) + ; Map records back into a list of integer elements + elements (mapv (fn record->element [^ConsumerRecord r] + (.value r)) + records)] + [f k elements]) + (catch InvalidTopicException _ + ; This can happen when a topic is created on one side of a partition + ; but another node doesn't know about it yet. + [f k nil])) + + :append + (let [record (rc/producer-record topic (k->partition k) nil v) + res @(.send producer record)] + mop)))) + +(defrecord Client [; Our three Kafka clients + admin producer consumer + ; An atom with a set of topics we've created. We have to + ; create topics before they can be used. + extant-topics] + client/Client + (open! [this test node] + (assoc this + :admin (rc/admin test node) + :producer (rc/producer test node) + :consumer (rc/consumer test node))) + + (setup! [this test]) + + (invoke! [this test op] + (try + (let [txn (:value op) + txn' (mapv (partial mop! this) txn)] + (assoc op :type :ok, :value txn')) + (catch ExecutionException e + (condp instance? (util/ex-root-cause e) + InvalidTopicException + (assoc op :type :fail, :error :invalid-topic) + + NotLeaderOrFollowerException + (assoc op :type :fail, :error :not-leader-or-follower) + + ; Love that we have to catch this in two different ways + TimeoutException + (assoc op :type :info, :error :timeout) + + UnknownTopicOrPartitionException + (assoc op :type :fail, :error :unknown-topic-or-partition) + + (throw e))) + (catch TimeoutException e + (assoc op :type :info, :error :timeout)))) + + (teardown! [this test]) + + (close! [this test] + (rc/close! admin) + (rc/close-producer! producer) + (rc/close-consumer! consumer))) + +(defn client + "Constructs a fresh client for this workload." + [] + (map->Client {:extant-topics (atom #{})})) + +(defn workload + "Constructs a workload (a map with a generator, client, checker, etc) given + an options map. Options are: + + (none) + + ... plus those taken by jepsen.tests.cycle.append/test, e.g. :key-count, + :min-txn-length, ..." + [opts] + (let [workload (append/test + (assoc opts + ; TODO: don't hardcode these + :max-txn-length 1 + :consistency-models [:strict-serializable]))] + (-> workload + (assoc :client (client)) + ; Rewrite generator ops to use :f :read or :f :write if they're read or + ; write-only. Elle doesn't care, but this helps us visualize read vs + ; write perf better. + (update :generator + (fn wrap-gen [gen] + (gen/map (fn tag-rw [op] + (case (->> op :value (map first) set) + #{:r} (assoc op :f :read) + #{:append} (assoc op :f :write) + op)) + gen)))))) diff --git a/src/jepsen/hstream/kafka/workload/queue.clj b/src/jepsen/hstream/kafka/workload/queue.clj new file mode 100644 index 0000000..168316c --- /dev/null +++ b/src/jepsen/hstream/kafka/workload/queue.clj @@ -0,0 +1,2548 @@ +(ns jepsen.hstream.kafka.workload.queue + "A workload which treats Redpanda more as a queue. Each client maintains a + producer and consumer. To subscribe to a new set of topics, we issue an + operation like: + + {:f :subscribe, :value [k1, k2, ...]} + + or + + {:f :assign, :value [k1, k2, ...]} + + ... where k1, k2, etc denote specific topics and partitions. For subscribe, + we just use that key's topic, and allow Redpanda to control which partitions + we get. Just like the Kafka client API, both subscribe and assign replace the + current topics for the consumer. + + Reads and writes (and mixes thereof) are encoded as a vector of + micro-operations: + + {:f :poll, :value [op1, op2, ...]} + {:f :send, :value [op1, op2, ...]} + {:f :txn, :value [op1, op2, ...]} + + Where :poll and :send denote transactions comprising only reads or writes, + respectively, and :txn indicates a general-purpose transaction. Operations + are of two forms: + + [:send key value] + + ... instructs a client to append `value` to the integer `key`--which maps + uniquely to a single topic and partition. These operations are returned as: + + [:send key [offset value]] + + where offset is the returned offset of the write, if available, or `nil` if + it is unknown (e.g. if the write times out). + + Reads are invoked as: + + [:poll] + + ... which directs the client to perform a single `poll` operation on its + consumer. The results of that poll are expanded to: + + [:poll {key1 [[offset1 value1] [offset2 value2] ...], + key2 [...]}] + + Where key1, key2, etc are integer keys obtained from the topic-partitions + returned by the call to poll, and the value for that key is a vector of + [offset value] pairs, corresponding to the offset of that message in that + particular topic-partition, and the value of the message---presumably, + whatever was written by `[:send key value]` earlier. + + Before a transaction completes, we commit its offsets. + + From this history we can perform a number of analyses: + + 1. For any observed value of a key, we check to make sure that its writer was + either :ok or :info; if the writer :failed, we know this constitutes an + aborted read. + + 2. We verify that all sends and polls agree on the value for a given key and + offset. We do not require contiguity in offsets, because transactions add + invisible messages which take up an offset slot but are not visible to the + API. If we find divergence, we know that Kakfa disagreed about the value at + some offset. + + Having verified that each [key offset] pair uniquely identifies a single + value, we eliminate the offsets altogether and perform the remainder of the + analysis purely in terms of keys and values. We construct a graph where + vertices are values, and an edge v1 -> v2 means that v1 immediately precedes + v2 in the offset order (ignoring gaps in the offsets, which we assume are due + to transaction metadata messages). + + 3. For each key, we take the highest observed offset, and then check that + every :ok :send operation with an equal or lower offset was *also* read by at + least one consumer. If we find one, we know a write was lost! + + 4. We build a dependency graph between pairs of transactions T1 and T2, where + T1 != T2, like so: + + ww. T1 sent value v1 to key k, and T2 sent v2 to k, and o1 < o2 + in the version order for k. + + wr. T1 sent v1 to k, and T2's highest read of k was v1. + + rw. T1's highest read of key k was offset o1, and T2 sent offset o2 to k, + and o1 < o2 in the version order for k. + + Our use of \"highest offset\" is intended to capture the fact that each poll + operation observes a *range* of offsets, but in general those offsets could + have been generated by *many* transactions. If we drew wr edges for every + offset polled, we'd generate superfluous edges--all writers are already + related via ww dependencies, so the final wr edge, plus those ww edges, + captures those earlier read values. + + We draw rw edges only for the final versions of each key observed by a + transaction. If we drew rw edges for an earlier version, we would incorrectly + be asserting that later transactions were *not* observed! + + We perform cycle detection and categorization of anomalies from this graph + using Elle. + + 5. Internal Read Contiguity: Within a transaction, each pair of reads on the + same key should be directly related in the version order. If we observe a gap + (e.g. v1 < ... < v2) that indicates this transaction skipped over some + values. If we observe an inversion (e.g. v2 < v1, or v2 < ... < v1) then we + know that the transaction observed an order which disagreed with the \"true\" + order of the log. + + 6. Internal Write Contiguity: Gaps between sequential pairs of writes to the + same key are detected via Elle as write cycles. Inversions are not, so we + check for them explicitly: a transaction sends v1, then v2, but v2 < v1 or v2 + < ... v1 in the version order. + + 6. Intermediate reads? I assume these happen constantly, but are they + supposed to? It's not totally clear what this MEANS, but I think it might + look like a transaction T1 which writes [v1 v2 v3] to k, and another T2 which + polls k and observes any of v1, v2, or v3, but not *all* of them. This + miiight be captured as a wr-rw cycle in some cases, but perhaps not all, + since we're only generating rw edges for final reads." + (:require [analemma [xml :as xml] + [svg :as svg]] + [clojure [pprint :refer [pprint]] + [set :as set]] + [clojure.java.io :as io] + [clojure.tools.logging :refer [info warn]] + [dom-top.core :refer [assert+ real-pmap loopr]] + [elle [core :as elle] + [graph :as g] + [list-append :refer [rand-bg-color]] + [txn :as txn] + [util :refer [index-of]]] + [gnuplot.core :as gnuplot] + [hiccup.core :as h] + [jepsen [checker :as checker] + [client :as client] + [generator :as gen] + [store :as store] + [util :as util :refer [map-vals + meh + nanos->secs + pprint-str]]] + [jepsen.checker.perf :as perf] + [jepsen.tests.cycle.append :as append] + [jepsen.hstream.kafka [client :as rc] + [db :as db]] + [knossos [history :as history]] + [slingshot.slingshot :refer [try+ throw+]]) + (:import (java.util.concurrent ExecutionException) + (org.apache.kafka.clients.admin Admin) + (org.apache.kafka.clients.consumer ConsumerRecords + ConsumerRecord + KafkaConsumer) + (org.apache.kafka.clients.producer KafkaProducer + RecordMetadata) + (org.apache.kafka.common KafkaException + TopicPartition) + (org.apache.kafka.common.errors AuthorizationException + DisconnectException + InterruptException + InvalidProducerEpochException + InvalidReplicationFactorException + InvalidTopicException + InvalidTxnStateException + NetworkException + NotControllerException + NotLeaderOrFollowerException + OutOfOrderSequenceException + ProducerFencedException + TimeoutException + UnknownTopicOrPartitionException + UnknownServerException + ))) + +(def partition-count + "How many partitions per topic?" + 2) + +(def replication-factor + "What replication factor should we use for each topic?" + 3) + +(def poll-ms + "How long should we poll for, in ms?" + 100) + +(defn k->topic + "Turns a logical key into a topic." + [k] + (str "t" (quot k partition-count))) + +(defn k->partition + "Turns a logical key into a partition within a topic." + [k] + (mod k partition-count)) + +(defn k->topic-partition + "Turns a logical key into a TopicPartition." + [k] + (rc/topic-partition (k->topic k) (k->partition k))) + +(defn topic-partition->k + "Turns a TopicPartition into a key." + ([^TopicPartition tp] + (topic-partition->k (.topic tp) (.partition tp))) + ([topic partition] + (+ (* partition-count (parse-long (nth (re-find #"t(\d+)" topic) 1))) + partition))) + +(defn mop! + "Applies a micro-operation from a transaction: either a :r read or a :append + operation." + [{:keys [extant-topics + ^Admin admin + ^KafkaProducer producer + ^KafkaConsumer consumer] :as client} + poll-ms + mop] + (case (first mop) + :poll (try + (rc/unwrap-errors + (let [records (.poll consumer (rc/ms->duration poll-ms))] + (->> (.partitions records) + (map (fn per-topic-partition [topic-partition] + ; Return a key, messages pair + [(topic-partition->k topic-partition) + ; Turn each message into an offset, record] pair + (mapv (fn xform-messages [^ConsumerRecord record] + [(.offset record) + (.value record)]) + (.records records topic-partition))])) + (into (sorted-map)) + (vector :poll)))) + (catch InvalidTopicException _ + [:poll {}]) + (catch IllegalStateException e + (if (re-find #"not subscribed to any" (.getMessage e)) + [:poll {}] + (throw e)))) + + :send (let [[f k v] mop + topic (k->topic k) + ; Create topic if it doesn't exist yet + _ (when-not (contains? @extant-topics topic) + (rc/create-topic! admin topic partition-count + replication-factor) + (swap! extant-topics conj topic)) + ; Send message to Redpanda + partition (k->partition k) + record (rc/producer-record topic (k->partition k) nil v) + res ^RecordMetadata (-> producer + (.send record) + (deref 10000 nil) + (or (throw+ {:type :timeout}))) + k' (topic-partition->k (.topic res) + (.partition res)) + offset (when (.hasOffset res) + (.offset res))] + (assert+ (= k k') + {:type ::key-mismatch + :k k + :k' k'}) + ; As it turns out offsets go missing fairly often + (when-not offset + (info "Missing offset for send() acknowledgement of key" k "value" v)) + ;(assert+ offset + ; {:type ::missing-offset + ; :res res}) + ; And record offset and value. + [f k' [offset v]]))) + +(defn send-offsets! + "Takes a client and a completed txn operation. Finds the highest polled + offsets from that op's :value atom, and calls .sendOffsetsToTransaction on + the producer." + [client op] + (loopr [offsets {}] ; Map of key to offset. + [[_ poll] (->> op :value deref (filter (comp #{:poll} first))) + [k pairs] poll + [offset value] pairs] + (recur (update offsets k (fnil max ##-Inf) offset)) + ; Convert that to topic-partitions -> OffsetAndMetadata + (let [kafka-offsets (->> (for [[k offset] offsets] + [(k->topic-partition k) + ; Note that we need to send the *next* + ; offset, not the one we observed. + (rc/offset+metadata (inc offset))]) + (into {})) + producer ^KafkaProducer (:producer client) + consumer ^KafkaConsumer (:consumer client)] + (when-not (empty? offsets) + (info :send-offsets offsets) + (.sendOffsetsToTransaction + producer + kafka-offsets + ; Redpanda doesn't support this version yet; it'll throw: + ; + ; org.apache.kafka.common.errors.UnsupportedVersionException: + ; Broker doesn't support group metadata commit API on version 2, + ; minimum supported request version is 3 which requires brokers + ; to be on version 2.5 or above. + ;(.groupMetadata consumer) + ; Instead we send the consumer group--this is the old API + rc/consumer-group))))) + +(defn safe-abort! + "Transactional aborts in the Kafka client can themselves fail, which requires + that we do a complex error-handling dance to retain the original exception as + well as the one thrown during the abort process. This function takes a + client, whether we're aborting before or after calling commit, and an + exception thrown by the transaction body (the reason why we're aborting in + the first place). Tries to abort the transaction on the current producer. + Then throws a map of: + + {:type :abort + :abort-ok? True if we successfully aborted, false if the abort threw. + :tried-commit? Whether we attempted to commit the transaction. + :body-error The exception which caused us to try the abort + :abort-error If the abort call itself crashed, the exception thrown. + :definite? If true, the transaction *should* not have taken place. If + false, the transaction may have taken place.}" + [client tried-commit?, body-error] + (try (rc/abort-txn! (:producer client)) + ; The example we're following for transactional workloads resets the + ; committed offsets for the consumer on abort. It might *seem* like we + ; should do this, but it might also create new weird behavior where + ; the poller jumps over records. Part of the reason they do this in the + ; EOS demo is because they're using some external consumer group stuff + ; to ensure there's no concurrent consumers, and we're *not* doing that + ; here. I'm not sure whether we should include this or not--it doesn't + ; seem to have a huge effect on safety, so I'm leaving it here for y'all + ; to consider later. + ; (rc/reset-to-last-committed-positions! (:consumer client)) + (catch RuntimeException abort-error + ; But of course the abort can crash! We throw a wrapper exception + ; which captures both the abort error and the error thrown from the + ; body that made us *try* the abort. + ; + ; When this happens, we need to close the producer. + (throw+ {:type :abort + :abort-ok? false + :tried-commit? tried-commit? + :definite? + (or ; If we didn't try to commit, we know + ; this definitely didn't happen. + (not tried-commit?) + ; Likewise, if we see an invalid txn state during commit, + ; that should signal the txn didn't happen. + (instance? InvalidTxnStateException body-error)) + :body-error body-error + :abort-error abort-error}))) + ; Here we know that the abort completed; it's + ; safe to re-use this producer. + (throw+ {:type :abort + :abort-ok? true + :tried-commit? tried-commit? + :definite? true + :body-error body-error})) + +(defmacro with-txn + "Takes a test, a client, an operation, and a body. If (:txn test) is false, + evaluates body. If (:txn) test is true, evaluates body in a transaction: + beginning the transaction before the body begins, committing the transaction + once the body completes, and handling errors appropriately. + + Expects the body to produce a completed version of op. Parses op to identify + the highest observed offset for each poll, and sends those offsets to the + transaction coordinator. + + If the body throws a fencing/auth/order error, returns a :type :info version + of op, which forces the client to tear down the producer and create a new + one. + + TODO: this isn't entirely what we want, because these might be definite + errors and now we're recording them as indefinite, but it feels safer to + force a full process crash because a lot of our analysis assumes processes + are 1:1 with consumers/producers. Maybe later we should close/reopen the + producer and wrap it in an atom? + + For all other errors, aborts the transaction and throws." + [test client op & body] + ; These three specific errors are definite but *cannot* be aborted, so we + ; have to throw them separately; they'll be handled by with-errors. We also + ; have to repeat them in two separate try/catch clauses, so we write them + ; down here. + (let [definite-non-abortable-catches + `[(catch ProducerFencedException e# (throw e#)) + (catch OutOfOrderSequenceException e# (throw e#)) + (catch AuthorizationException e# (throw e#))]] + `(if-not (:txn ~test) + ; Not a transaction; evaluate normally. + (do ~@body) + ; Great, we're a transaction. Let the producer know. + (let [producer# ^KafkaProducer (:producer ~client)] + (.beginTransaction producer#) + ; Actually evaluate body, and possibly send offsets to the txn + (let [op'# (try (let [op'# (do ~@body)] + (send-offsets! ~client op'#) + op'#) + ; Some errors aren't allowed to abort + ~@definite-non-abortable-catches + ; For all other errors... + (catch RuntimeException body-err# + ; If we crash *prior* to commit, we're allowed to + ; abort + (safe-abort! ~client false body-err#)))] + ; Now we can commit the txn + (try (.commitTransaction producer#) + ; Again, some errors aren't allowed to abort + ~@definite-non-abortable-catches + ; Timeouts and interrupts are indefinite and also + ; non-abortable; we throw these and let the enclosing error + ; handler grab them. + (catch TimeoutException e# + (throw e#)) + (catch InterruptException e# + (throw e#)) + ; But for *other* exceptions, we need to abort, and of + ; course that can fail again as well, so we call safe-abort! + (catch RuntimeException e# + (safe-abort! ~client true e#))) + op'#))))) + +(defmacro with-errors + "Takes an operation and a body. Evaluates body, catching common exceptions + and returning appropriate fail/info operations when they occur." + [op & body] + `(try+ ~@body + (catch AuthorizationException _# + (assoc ~op + :type :fail + :error :authorization + :end-process? true)) + + (catch DisconnectException e# + (assoc ~op :type :info, :error [:disconnect (.getMessage e#)])) + + (catch InvalidProducerEpochException e# + (assoc ~op + :type :fail + :error [:invalid-producer-epoch (.getMessage e#)])) + + (catch InvalidTopicException _# + (assoc ~op :type :fail, :error :invalid-topic)) + + (catch InvalidReplicationFactorException _# + (assoc ~op :type :fail :error :invalid-replication-factor)) + + (catch NetworkException e# + (assoc ~op :type :info, :error [:network (.getMessage e#)])) + + (catch NotControllerException e# + (assoc ~op :type :fail, :error :not-controller)) + + (catch NotLeaderOrFollowerException _# + ; This is, surprisingly enough, not a definite failure! See + ; https://issues.apache.org/jira/browse/KAFKA-13574 + (assoc ~op :type :info, :error :not-leader-or-follower)) + + (catch OutOfOrderSequenceException _# + (assoc ~op + :type :fail + :error :out-of-order-sequence + :end-process? true)) + + (catch ProducerFencedException _# + (assoc ~op + :type :fail, + :error :producer-fenced + :end-process? true)) + + (catch UnknownTopicOrPartitionException _# + (assoc ~op :type :fail, :error :unknown-topic-or-partition)) + + (catch UnknownServerException e# + (assoc ~op :type :info, :error [:unknown-server-exception + (.getMessage e#)])) + + (catch TimeoutException _# + (assoc ~op :type :info, :error :kafka-timeout)) + + (catch KafkaException e# + (condp re-find (.getMessage e#) + #"broker is not available" + (assoc ~op :type :fail, :error :broker-not-available) + + ; This signifies that the producer is borked somehow and we need + ; to tear it down and start a new one. + #"Cannot execute transactional method because we are in an error state" + (assoc ~op + :type :fail + :error [:txn-in-error-state (.getMessage e#)] + :end-process? true) + + #"Unexpected error in AddOffsetsToTxnResponse" + (assoc ~op :type :fail, :error [:add-offsets (.getMessage e#)]) + + #"Unexpected error in TxnOffsetCommitResponse" + (assoc ~op + :type :fail + :error [:txn-offset-commit (.getMessage e#)]) + + #"Unhandled error in EndTxnResponse" + (assoc ~op :type :info, :error [:end-txn (.getMessage e#)]) + + (throw e#))) + + (catch [:type :abort] e# + (assoc ~op + :type (if (:definite? e#) :fail :info) + ; If we were able to abort successfully, it's OK to re-use + ; this process; otherwise we need to tear it down. + :end-process? (not (:abort-ok? e#)) + :error (cond-> e# + ; We'd like string representations rather + ; than full objects here, so we can serialize + ; the history. + true (update :body-error str) + (:abort-error e#) (update :abort-error str)))) + + (catch [:type :partitions-assigned] e# + (assoc ~op :type :fail, :error e#)) + + (catch [:type :partitions-lost] e# + (assoc ~op :type :fail, :error e#)) + + (catch [:type :partitions-revoked] e# + (assoc ~op :type :fail, :error e#)) + + (catch [:type :timeout] e# + (assoc ~op :type :info, :error :timeout)))) + +(defmacro with-rebalance-log + "Ugh, everything needs state tracking. I'm so sorry, this is an enormous + spaghetti pile. + + Basically, we want to know whether rebalances happened during a given + transaction, because we suspect that they might be implicated in certain + Weird Polling Events. We take a client and a body. Before the body, we clear + the client's rebalance log atom, and when the body is complete (returning + op') we snarf the client's rebalance log and store it in the completion op, + so that it's available for inspection later." + [client & body] + `(let [log# (:rebalance-log ~client)] + (reset! log# []) + (let [op'# ~@body + log# (map (fn [entry#] + (-> entry# + (dissoc :partitions) + (assoc :keys (map topic-partition->k + (:partitions entry#))))) + @log#)] + (if (seq log#) + (assoc op'# :rebalance-log log#) + op'#)))) + +(defmacro with-mutable-value + "Takes a symbol referring to an invocation operation, and evaluates body + where `op` has a :value which is an atom wrapping the original op's :value. + Expects body to return an op, and replaces that op's :value with the current + value of the atom. + + We do this Rather Weird Thing because Redpanda might actually give us + informatrion like offsets for `send` during a transaction which will later + crash, and we want to preserve those offsets in the completed info op." + [op & body] + (assert (symbol? op)) + `(let [value# (atom (:value ~op)) + ~op (assoc ~op :value value#) + op'# (do ~@body)] + (assoc op'# :value @value#))) + +(defrecord Client [; What node are we bound to? + node + ; Our three Kafka clients + ^Admin admin + ^KafkaProducer producer + ^KafkaConsumer consumer + ; An atom with a set of topics we've created. We have to + ; create topics before they can be used. + extant-topics + ; An atom of a vector of consumer rebalance events + ; love too track state everywhere + rebalance-log] + client/Client + (open! [this test node] + (let [;tx-id "jepsen-txn" + tx-id (rc/new-transactional-id) + producer (rc/producer + (if (:txn test) + (assoc test :transactional-id tx-id) + test) + node)] + (assoc this + :node node + :admin (rc/admin test node) + :consumer (rc/consumer test node) + :producer producer + :rebalance-log (atom [])))) + + (setup! [this test]) + + (invoke! [this test op] + (case (:f op) + ; Assign this consumer new topic-partitions + :assign (let [tps (map k->topic-partition (:value op))] + (.assign consumer tps) + (when (:seek-to-beginning? op) + (info "Seeking to beginning") + (.seekToBeginning consumer tps)) + (assoc op :type :ok)) + + ; Crash this client, forcing us to open a new client (and consumer etc) + :crash (assoc op :type :info) + + ; Get some debugging information about the partition distribution on this + ; node + :debug-topic-partitions + (try+ + (try + (let [tps (->> (:value op) + (real-pmap (fn [k] + [k (db/topic-partition-state (:db test) + node (k->topic-partition k))])) + (into (sorted-map)))] + (assoc op :type :ok, :value {:node node + :node-id (db/node-id (:db test) + test node) + :partitions tps})) + (catch java.util.concurrent.ExecutionException e + (throw (util/ex-root-cause e)))) + (catch [:type :clj-http.client/unexceptional-status] e + (assoc op :type :fail, :error (:body e))) + (catch java.net.SocketTimeoutException _ + (assoc op :type :fail, :error :timeout)) + (catch java.net.ConnectException _ + (assoc op :type :fail, :error :connection-refused))) + + ; Subscribe to the topics containing these keys + :subscribe + (let [topics (->> (:value op) + (map k->topic) + distinct)] + (if (:txn test) + (rc/subscribe! consumer + topics + (rc/logging-rebalance-listener rebalance-log)) + (rc/subscribe! consumer topics)) + (assoc op :type :ok)) + + ; Apply poll/send transactions + (:poll, :send, :txn) + (with-mutable-value op + (with-rebalance-log this + (with-errors op + (with-txn test this op + (rc/unwrap-errors + (do ; Evaluate micro-ops for side effects, incrementally + ; transforming the transaction's micro-ops + (reduce (fn [i mop] + (let [mop' (mop! this (:poll-ms op poll-ms) mop)] + (swap! (:value op) assoc i mop')) + (inc i)) + 0 + @(:value op)) + ; If we read, AND we're using :subscribe instead of assign, + ; commit offsets. My understanding is that with assign + ; you're not supposed to use the offset system? + ; + ; Also note that per + ; https://stackoverflow.com/questions/45195010/meaning-of-sendoffsetstotransaction-in-kafka-0-11, + ; commitSync is more intended for non-transactional + ; workflows. + (when (and (#{:poll :txn} (:f op)) + (not (:txn test)) + (:subscribe (:sub-via test))) + (try (.commitSync consumer) + ; If we crash during commitSync *outside* a + ; transaction, it might be that we poll()ed some + ; values in this txn which Kafka will think we + ; consumed. We won't have any record of them if we + ; fail the txn. Instead, we return an :ok txn *with* + ; the reads, but note the lack of commit. + (catch RuntimeException e + (assoc op :type :ok + :error [:consumer-commit (.getMessage e)])))) + (assoc op :type :ok))))))))) + + (teardown! [this test]) + + (close! [this test] + (rc/close! admin) + (rc/close-producer! producer) + (rc/close-consumer! consumer)) + + client/Reusable + (reusable? [this test] + ; When we crash, we want to tear down our connections + false)) + +(defn client + "Constructs a fresh client for this workload." + [] + (map->Client {:extant-topics (atom #{})})) + +;; Generator + +(defn txn-generator + "Takes a list-append generator and rewrites its transactions to be [:poll] or + [:send k v] micro-ops. Also adds a :keys field onto each operation, with a + set of keys that txn would have interacted with; we use this to generate + :subscribe ops later." + [la-gen] + (gen/map (fn rewrite-op [op] + (-> op + (assoc :keys (set (map second (:value op)))) + (update :value + (partial mapv (fn rewrite-mop [[f k v]] + (case f + :append [:send k v] + :r [:poll])))))) + la-gen)) + +(def subscribe-ratio + "How many subscribe ops should we issue per txn op?" + 1/8) + +(defrecord InterleaveSubscribes [gen] + gen/Generator + (op [this test context] + ; When we're asked for an operation, ask the underlying generator for + ; one... + (when-let [[op gen'] (gen/op gen test context)] + (if (= :pending op) + [:pending this] + (let [this' (InterleaveSubscribes. gen')] + (if (< (rand) subscribe-ratio) + ; At random, emit a subscribe/assign op instead. + (let [f (rand-nth (vec (:sub-via test))) + op {:f f, :value (vec (:keys op))}] + [(gen/fill-in-op op context) this]) + ; Or pass through the op directly + [(dissoc op :keys) (InterleaveSubscribes. gen')]))))) + + ; Pass through updates + (update [this test context event] + (InterleaveSubscribes. (gen/update gen test context event)))) + +(defn interleave-subscribes + "Takes a txn generator and keeps track of the keys flowing through it, + interspersing occasional :subscribe or :assign operations for recently seen + keys." + [txn-gen] + (InterleaveSubscribes. txn-gen)) + +(defn tag-rw + "Takes a generator and tags operations as :f :poll or :send if they're + entirely comprised of send/polls." + [gen] + (gen/map (fn tag-rw [op] + (case (->> op :value (map first) set) + #{:poll} (assoc op :f :poll) + #{:send} (assoc op :f :send) + op)) + gen)) + +(defn op->max-offsets + "Takes an operation (presumably, an OK one) and returns a map of keys to the + highest offsets interacted with in that op." + [{:keys [type f value]}] + (case type + (:info, :ok) + (case f + (:poll, :send, :txn) + (->> value + (map (fn [[f :as mop]] + (case f + :poll (->> (second mop) + (map-vals (fn [pairs] + (->> pairs + (map first) + (remove nil?) + (reduce max -1))))) + :send (let [[_ k v] mop] + (when (and (vector? v) (first v)) + {k (first v)}))))) + (reduce (partial merge-with max))) + + nil) + nil)) + +(defrecord TrackKeyOffsets [gen offsets] + gen/Generator + (op [this test context] + (when-let [[op gen'] (gen/op gen test context)] + (if (= :pending op) + [:pending this] + [op (TrackKeyOffsets. gen' offsets)]))) + + (update [this test context event] + (when (= :ok (:type event)) + (let [op-offsets (op->max-offsets event)] + (when-not (empty? op-offsets) + (swap! offsets #(merge-with max % op-offsets))))) + (TrackKeyOffsets. + (gen/update gen test context event) offsets))) + +(defn track-key-offsets + "Wraps a generator. Keeps track of every key that generator touches in the + given atom, which is a map of keys to highest offsets seen." + [keys-atom gen] + (TrackKeyOffsets. gen keys-atom)) + +(defrecord FinalPolls [target-offsets gen] + gen/Generator + (op [this test context] + ;(info "waiting for" target-offsets) + (when-not (empty? target-offsets) + (when-let [[op gen'] (gen/op gen test context)] + [op (assoc this :gen gen')]))) + + (update [this test context {:keys [type f value] :as event}] + ; (info :update event) + (if (and (= type :ok) + (= f :poll)) + (let [offsets' (reduce (fn [target-offsets' [k seen-offset]] + (if (<= (get target-offsets' k -1) + seen-offset) + ; We've read past our target offset for + ; this key + (dissoc target-offsets' k) + target-offsets')) + target-offsets + (op->max-offsets event))] + (when-not (identical? target-offsets offsets') + (info "Process" (:process event) "now waiting for" offsets')) + (FinalPolls. offsets' gen)) + ; Not relevant + this))) + +(defn final-polls + "Takes an atom containing a map of keys to offsets. Constructs a generator + which: + + 1. Checks the topic-partition state from the admin API + + 2. Crashes the client, to force a fresh one to be opened, just in case + there's broken state inside the client. + + 3. Assigns the new client to poll every key, and seeks to the beginning + + 4. Polls repeatedly + + This process repeats every 10 seconds until polls have caught up to the + offsets in the offsets atom." + [offsets] + (delay + (let [offsets @offsets] + (info "Polling up to offsets" offsets) + (->> [{:f :crash} + {:f :debug-topic-partitions, :value (keys offsets)} + {:f :assign, :value (keys offsets), :seek-to-beginning? true} + (->> {:f :poll, :value [[:poll]], :poll-ms 1000} + repeat + (gen/stagger 1/5))] + (gen/time-limit 10000) + repeat + (FinalPolls. offsets))))) + +(defn crash-client-gen + "A generator which, if the test has :crash-clients true, periodically emits + an operation to crash a random client." + [opts] + (when (:crash-clients opts) + (->> (repeat {:f :crash}) + (gen/stagger (/ (:crash-client-interval opts) + (:concurrency opts)))))) + +;; Checker ;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;; + +(defn assocv + "An assoc on vectors which allows you to assoc at arbitrary indexes, growing + the vector as needed. When v is nil, constructs a fresh vector rather than a + map." + [v i value] + (if v + (if (<= i (count v)) + (assoc v i value) + (let [nils (repeat (- i (count v)) nil)] + (assoc (into v nils) i value))) + ; Nil is treated as an empty vector. + (recur [] i value))) + +(defn nth+ + "Nth for vectors, but returns nil instead of out-of-bounds." + [v i] + (when (< i (count v)) + (nth v i))) + +(defn op-writes-helper + "Takes an operation and a function which takes an offset-value pair. Returns + a map of keys written by this operation to the sequence of (f [offset value]) + sends for that key. Note that offset may be nil." + [op f] + (when (#{:txn :send} (:f op)) + (reduce (fn [writes mop] + (if (= :send (first mop)) + (let [[_ k v] mop + vs (get writes k []) + ; Values can be either a literal value or a [offset value] + ; pair. + value (if (vector? v) + (f v) + (f [nil v]))] + (assoc writes k (conj vs value))) + ; Something other than a send + writes)) + {} + (:value op)))) + +(defn op-writes + "Returns a map of keys to the sequence of all values written to that key in + an op." + [op] + (op-writes-helper op second)) + +(defn op-write-offsets + "Returns a map of keys to the sequence of all offsets written to that key in + an op." + [op] + (op-writes-helper op first)) + +(defn op-write-pairs + "Returns a map of keys to the sequence of all [offset value] pairs written to + that key in an op." + [op] + (op-writes-helper op identity)) + +(defn op-reads-helper + "Takes an operation and a function which takes an offset-value pair. Returns + a map of keys read by this operation to the sequence of (f [offset value]) + read for that key." + [op f] + (when (#{:txn :poll} (:f op)) + (reduce (fn mop [res mop] + (if (= :poll (first mop)) + (reduce (fn per-key [res [k pairs]] + (let [vs (get res k []) + vs' (into vs (map f pairs))] + (assoc res k vs'))) + res + (second mop)) + res)) + {} + (:value op)))) + +(defn op-read-pairs + "Returns a map of keys to the sequence of all [offset value] pairs read for + that key." + [op] + (op-reads-helper op identity)) + +(defn op-read-offsets + "Returns a map of keys to the sequence of all offsets read for that key." + [op] + (op-reads-helper op first)) + +(defn op-reads + "Returns a map of keys to the sequence of all values read for that key." + [op] + (op-reads-helper op second)) + +(defn op-pairs + "Returns a map of keys to the sequence of all [offset value] pairs either + written or read for that key; writes first." + [op] + (merge-with concat (op-write-pairs op) (op-read-pairs op))) + +(defn reads-of-key + "Returns a seq of all operations which read the given key, and, optionally, + read the given value." + ([k history] + (->> history + (filter (comp #{:txn :send :poll} :f)) + (filter (fn [op] + (contains? (op-reads op) k))))) + ([k v history] + (->> history + (reads-of-key k) + (filter (fn [op] + (some #{v} (get (op-reads op) k))))))) + +(defn writes-of-key + "Returns a seq of all operations which wrote the given key, and, optionally, + sent the given value." + ([k history] + (->> history + (filter (comp #{:txn :send :poll} :f)) + (filter (fn [op] + (contains? (op-writes op) k))))) + ([k v history] + (->> history + (writes-of-key k) + (filter (fn [op] + (some #{v} (get (op-writes op) k))))))) + +(defn reads-of-key-offset + "Returns a seq of all operations which read the given key and offset." + [k offset history] + (->> history + (reads-of-key k) + (filter (fn [op] + (some #{offset} (get (op-read-offsets op) k)))))) + +(defn writes-of-key-offset + "Returns a seq of all operations which wrote the given key and offset." + [k offset history] + (->> history + (writes-of-key k) + (filter (fn [op] + (some #{offset} (get (op-write-offsets op) k)))))) + +(defn reads-of-key-value + "Returns a seq of all operations which read the given key and value." + [k value history] + (->> history + (reads-of-key k) + (filter (fn [op] (some #{value} (get (op-reads op) k)))))) + +(defn writes-of-key-value + "Returns a seq of all operations which wrote the given key and value." + [k value history] + (->> history + (writes-of-key k) + (filter (fn [op] (some #{value} (get (op-writes op) k)))))) + +(defn op-around-key-offset + "Takes an operation and returns that operation with its value trimmed so that + any send/poll operations are constrained to just the given key, and values + within n of the given offset. Returns nil if operation is not relevant." + ([k offset op] + (op-around-key-offset k offset 3 op)) + ([k offset n op] + (when (and (not= :invoke (:type op)) + (#{:send :poll :txn} (:f op))) + (let [value' + (keep (fn [[f v :as mop]] + (case f + :poll (when-let [pairs (get v k)] + ; Trim pairs to region around offset + (let [trimmed + (filter (fn [[o v]] + (<= (- offset n) o (+ offset n))) + pairs)] + (when (seq trimmed) + [:poll {k trimmed}]))) + :send (let [[_ k2 v-or-pair] mop] + (when (vector? v-or-pair) + (let [[o v] v-or-pair] + (when (and (= k k2) + (<= (- offset n) o (+ offset n))) + mop)))))) + (:value op))] + (when-not (empty? value') + (assoc op :value value')))))) + +(defn around-key-offset + "Filters a history to just those operations around a given key and offset; + trimming their mops to just those regions as well." + ([k offset history] + (around-key-offset k offset 3 history)) + ([k offset n history] + (keep (partial op-around-key-offset k offset n) history))) + +(defn around-some + "Clips a sequence to just those elements near a predicate. Takes a predicate, + a range n, and a sequence xs. Returns the series of all x in xs such x is + within n elements of some x' matching predicate." + [pred n coll] + (let [indices (first + (reduce (fn [[indices i] x] + (if (pred x) + [(into indices (range (- i n) (+ i n 1))) (inc i)] + [indices (inc i)])) + [#{} 0] + coll))] + (first (reduce (fn [[out i] x] + (if (indices i) + [(conj out x) (inc i)] + [out (inc i)])) + [[] 0] + coll)))) + +(defn op-around-key-value + "Takes an operation and returns that operation with its value trimmed so that + any send/poll operations are constrained to just the given key, and values + within n of the given value. Returns nil if operation is not relevant." + ([k value op] + (op-around-key-value k value 3 op)) + ([k value n op] + (when (and (= :ok (:type op)) + (#{:send :poll :txn} (:f op))) + (let [value' + (keep (fn [[f v :as mop]] + (case f + :poll (when-let [pairs (get v k)] + ; Trim pairs to region around offset + (let [trimmed (around-some (comp #{value} second) + n pairs)] + (when (seq trimmed) + {k trimmed}))) + :send (let [[_ k2 [o v]] mop] + (when (and (= k k2) (= value v)) + mop)))) + (:value op))] + (when-not (empty? value') + (assoc op :value value')))))) + +(defn around-key-value + "Filters a history to just those operations around a given key and value; + trimming their mops to just those regions as well." + ([k value history] + (around-key-value k value 3 history)) + ([k value n history] + (keep (partial op-around-key-value k value n) history))) + +(defn writes-by-type + "Takes a history and constructs a map of types (:ok, :info, :fail) to maps of + keys to the set of all values which were written for that key. We use this to + identify, for instance, what all the known-failed writes were for a given + key." + [history] + (->> history + (remove (comp #{:invoke} :type)) + (filter (comp #{:txn :send} :f)) + (group-by :type) + (map-vals (fn [ops] + (->> ops + ; Construct a seq of {key [v1 v2 ...]} maps + (map op-writes) + ; And turn [v1 v2 ...] into #{v1 v2 ...} + (map (partial map-vals set)) + ; Then merge them all together + (reduce (partial merge-with set/union) {})))))) + +(defn reads-by-type + "Takes a history and constructs a map of types (:ok, :info, :fail) to maps of + keys to the set of all values which were read for that key. We use this to + identify, for instance, the known-successful reads for some key as a part of + finding lost updates." + [history] + (->> history + (remove (comp #{:invoke} :type)) + (filter (comp #{:txn :poll} :f)) + (group-by :type) + (map-vals (fn [ops] + (->> ops + (map op-reads) + (map (partial map-vals set)) + (reduce (partial merge-with set/union) {})))))) + + +(defn must-have-committed? + "Takes a reads-by-type map and a (presumably :info) transaction which sent + something. Returns true iff the transaction was :ok, or if it was :info and + we can prove that some send from this transaction was successfully read." + [reads-by-type op] + (or (= :ok (:type op)) + (and (= :info (:type op)) + (let [ok (:ok reads-by-type)] + (some (fn [[k vs]] + (let [ok-vs (get ok k #{})] + (some ok-vs vs))) + (op-writes op)))))) + +(defn version-orders-update-log + "Updates a version orders log with the given offset and value." + [log offset value] + (if-let [values (nth+ log offset)] + (assoc log offset (conj values value)) ; Already have values + (assocv log offset #{value}))) ; First time we've seen this offset + +(defn version-orders-reduce-mop + "Takes a logs object from version-orders and a micro-op, and integrates that + micro-op's information about offsets into the logs." + [logs mop] + (case (first mop) + :send (let [[_ k v] mop] + (if (vector? v) + (let [[offset value] v] + (if offset + ; We completed the send and know an offset + (update logs k version-orders-update-log offset value) + ; Not sure what the offset was + logs)) + ; Not even offset structure: maybe an :info txn + logs)) + + :poll (reduce (fn poll-key [logs [k pairs]] + (reduce (fn pair [logs [offset value]] + (if offset + (update logs k version-orders-update-log + offset value) + logs)) + logs + pairs)) + logs + (second mop)))) + +(defn index-seq + "Takes a seq of distinct values, and returns a map of: + + {:by-index A vector of the sequence + :by-value A map of values to their indices in the vector.}" + [xs] + {:by-index (vec xs) + :by-value (into {} (map-indexed (fn [i x] [x i]) xs))}) + +(defn log->value->first-index + "Takes a log: a vector of sets of read values for each offset in a partition, + possibly including `nil`s. Returns a map which takes a value to the index + where it first appeared." + [log] + (->> (remove nil? log) + (reduce (fn [[earliest i] values] + [(reduce (fn [earliest value] + (if (contains? earliest value) + earliest + (assoc earliest value i))) + earliest + values) + (inc i)]) + [{} 0]) + first)) + +(defn log->last-index->values + "Takes a log: a vector of sets of read values for each offset in a partition, + possibly including `nil`s. Returns a vector which takes indices (dense + offsets) to sets of values whose *last* appearance was at that position." + [log] + (->> (remove nil? log) + ; Build up a map of values to their latest indexes + (reduce (fn latest [[latest i] values] + [(reduce (fn [latest value] + (assoc latest value i)) + latest + values) + (inc i)]) + [{} 0]) + first + ; Then invert that map into a vector of indexes to sets + (reduce (fn [log [value index]] + (let [values (get log index #{})] + (assocv log index (conj values value)))) + []))) + +(defn version-orders + "Takes a history and a reads-by-type structure. Constructs a map of: + + {:orders A map of keys to orders for that key. Each order is a map of: + {:by-index A vector which maps indices to single values, + in log order. + :by-value A map of values to indices in the log. + :log A vector which maps offsets to sets of values + in log order.} + + :errors A series of error maps describing any incompatible orders, where + a single offset for a key maps to multiple values.} + + Offsets are directly from Kafka. Indices are *dense* offsets, removing gaps + in the log." + ([history reads-by-type] + (version-orders history reads-by-type {})) + ([history reads-by-type logs] + ; Logs is a map of keys to vectors, where index i in one of those vectors is + ; the vector of all observed values for that index. + (if (seq history) + (let [op (first history) + history' (next history)] + (if (must-have-committed? reads-by-type op) + (case (:f op) + (:poll, :send, :txn) + (recur history' reads-by-type + (reduce version-orders-reduce-mop logs (:value op))) + ; Some non-transactional op, like an assign/subscribe + (recur history' reads-by-type logs)) + ; We can't assume this committed; don't consider it. + (recur history' reads-by-type logs))) + ; All done; transform our logs to orders. + {:errors (->> logs + (mapcat (fn errors [[k log]] + (->> log + (reduce (fn [[offset index errs] values] + (condp <= (count values) + ; Divergence + 2 [(inc offset) (inc index) + (conj errs {:key k + :offset offset + :index index + :values values})] + ; No divergence + 1 [(inc offset) (inc index) errs] + ; Hole in log + 0 [(inc offset) index errs])) + [0 0 []]) + last))) + seq) + :orders + (map-vals + (fn key-order [log] + (assoc (->> log (remove nil?) (map first) index-seq) + :log log)) + logs)}))) + +(defn g1a-cases + "Takes a partial analysis and looks for aborted reads, where a known-failed + write is nonetheless visible to a committed read. Returns a seq of error + maps, or nil if none are found." + [{:keys [history writes-by-type writer-of]}] + (let [failed (:fail writes-by-type) + ops (->> history + (filter (comp #{:ok} :type)) + (filter (comp #{:txn :poll} :f)))] + (->> (for [op ops + [k vs] (op-reads op) + v vs + :when (contains? (get failed k) v)] + {:key k + :value v + :writer (get-in writer-of [k v]) + :reader op}) + seq))) + +(defn lost-write-cases + "Takes a partial analysis and looks for cases of lost write: where a write + that we *should* have observed is somehow not observed. Of course we cannot + expect to observe everything: for example, if we send a message to Redpanda + at the end of a test, and don't poll for it, there's no chance of us seeing + it at all! Or a poller could fall behind. + + What we do instead is identify the highest read value for each key v_max, and + then take the set of all values *prior* to it in the version order: surely, + if we read v_max = 3, and the version order is [1 2 3 4], we should also have + read 1 and 2. + + It's not *quite* this simple. If a message appears at multiple offsets, the + version order will simply pick one for us, which leads to nondeterminism. If + an offset has multiple messages, a successfully inserted message could appear + *nowhere* in the version order. + + To deal with this, we examine the raw logs for each key, and build two index + structures. The first maps values to their earliest (index) appearance in the + log: we use this to determine the highest index that must have been read. The + second is a vector which maps indexes to sets of values whose *last* + appearance in the log was at that index. We use this vector to identify which + values ought to have been read. + + Once we've derived the set of values we ought to have read for some key k, we + run through each poll of k and cross off the values read. If there are any + values left, they must be lost updates." + [{:keys [history version-orders reads-by-type writer-of readers-of]}] + ; Start with all the values we know were read + (->> (:ok reads-by-type) + (mapcat + (fn [[k vs]] + ; Great, now for this key, find the highest index observed + (let [vo (get version-orders k) + ; For each value, what's the earliest index we observed + ; that value at? + value->first-index (log->value->first-index (:log vo)) + ; And for each index, which values appeared at that index + ; *last*? + last-index->values (log->last-index->values (:log vo)) + + ; From each value, we take the latest of the earliest + ; indices that value appeared at. + bound (->> vs + ; We might observe a value but *not* know + ; its offset from either write or read. + ; When this happens, we can't say anything + ; about how much of the partition should + ; have been observed, so we skip it. + (keep value->first-index) + (reduce max -1)) + ; Now take a prefix of last-index->values up to that + ; index; these are the values we should have observed. + must-read (->> (inc bound) + (subvec last-index->values 0) + ; This is a vector of sets. We glue them + ; together this way to preserve index order. + (mapcat identity) + distinct) + ; To demonstrate these errors to users, we want to prove that + ; some reader observed the maximum offset (and therefore + ; someone else should have observed lower offsets). + max-read (->> (nth last-index->values bound) + first ; If there were conflicts, any value ok + (vector k) + (get-in readers-of) + first) + ; Now we go *back* to the read values vs, and strip them + ; out from the must-read set; anything left is something we + ; failed to read. + lost (remove vs must-read) + ; Because we performed this computation based on the + ; version order, we may have flagged info/fail writes as + ; lost. We need to go through and check that the writers + ; are either a.) OK, or b.) info AND one of their writes + ; was read. + lost + (filter + (fn double-check [v] + (let [w (or (get-in writer-of [k v]) + (throw+ {:type :no-writer-of + :key k + :value v}))] + (must-have-committed? reads-by-type w))) + lost)] + (->> lost + (map (fn [v] + {:key k + :value v + :index (get value->first-index v) + :max-read-index bound + :writer (get-in writer-of [k v]) + :max-read max-read})))))) + seq)) + +(defn strip-types + "Takes a collection of maps, and removes their :type fields. Returns nil if + none remain." + [ms] + (seq (map (fn [m] (dissoc m :type)) ms))) + +(defn int-poll-skip+nonmonotonic-cases + "Takes a partial analysis and looks for cases where a single transaction + contains: + + {:skip A pair of poll values which read the same key and skip + over some part of the log which we know should exist. + :nonmonotonic A pair of poll values which *contradict* the log order, + or repeat the same value.} + + When a transaction's rebalance log includes a key which would otherwise be + involved in one of these violations, we don't report it as an error: we + assume that rebalances invalidate any assumption of monotonically advancing + offsets." + [{:keys [history version-orders]}] + (->> history + (pmap + (fn per-op [op] + (let [rebalanced-keys (->> op :rebalance-log + (mapcat :keys) + set)] + ; Consider each pair of reads of some key in this op... + (->> (for [[k vs] (op-reads op) + [v1 v2] (partition 2 1 vs)] + (let [{:keys [by-index by-value]} (get version-orders k) + ; What are their indices in the log? + i1 (get by-value v1) + i2 (get by-value v2) + delta (if (and i1 i2) + (- i2 i1) + 1)] + (cond ; If a key was rebalanced during the transaction, + ; all bets are off. + (rebalanced-keys k) + nil + + (< 1 delta) + {:type :skip + :key k + :values [v1 v2] + :delta delta + :skipped (map by-index (range (inc i1) i2)) + :op op} + + (< delta 1) + {:type :nonmonotonic + :key k + :values [v1 v2] + :delta delta + :op op}))) + (remove nil?))))) + (mapcat identity) + (group-by :type) + (map-vals strip-types))) + +(defn int-send-skip+nonmonotonic-cases + "Takes a partial analysis and looks for cases where a single transaction + contains a pair of sends to the same key which: + + {:skip Skips over some indexes of the log + :nonmonotonic Go backwards (or stay in the same place) in the log}" + [{:keys [history version-orders]}] + (->> history + (remove (comp #{:invoke} :type)) + (mapcat (fn per-op [op] + ; Consider each pair of sends to a given key in this op... + (->> (for [[k vs] (op-writes op) + [v1 v2] (partition 2 1 vs)] + (let [{:keys [by-index by-value]} (get version-orders k) + i1 (get by-value v1) + i2 (get by-value v2) + delta (if (and i1 i2) + (- i2 i1) + 1)] + (cond (< 1 delta) + {:type :skip + :key k + :values [v1 v2] + :delta delta + :skipped (map by-index (range (inc i1) i2)) + :op op} + + (< delta 1) + {:type :nonmonotonic + :key k + :values [v1 v2] + :delta delta + :op op}))) + (remove nil?)))) + (group-by :type) + (map-vals strip-types))) + +(defn poll-skip+nonmonotonic-cases + "Takes a partial analysis and checks each process's operations sequentially, + looking for cases where a single process either jumped backwards or skipped + over some region of a topic-partition. Returns a map: + + {:nonmonotonic Cases where a process started polling at or before a + previous operation last left off + :skip Cases where two successive operations by a single process + skipped over one or more values for some key.}" + [{:keys [history version-orders]}] + ; First, group ops by process + (->> (group-by :process history) + ; Then reduce each process, keeping track of the most recent read + ; index for each key. + (map-vals + (fn per-process [ops] + ; Iterate over this process's operations, building up a vector of + ; errors. + (loop [ops ops + errors [] + last-reads {}] + (if-not (seq ops) + ; Done + errors + ; Process this op + (let [op (first ops)] + (case (:f op) + ; When we assign or subscribe a new topic, preserve *only* + ; those last-reads which we're still subscribing to. + (:assign, :subscribe) + (recur (next ops) + errors + (if (#{:invoke :fail} (:type op)) + last-reads + (select-keys last-reads (:value op)))) + + (:txn, :poll) + (let [reads (op-reads op) + ; Look at each key and values read by this op. + errs (for [[k reads] reads] + ; What was the last op that read this key? + (if-let [last-op (get last-reads k)] + ; Compare the index of the last thing read by + ; the last op read to the index of our first + ; read + (let [vo (get-in version-orders + [k :by-value]) + v (-> last-op op-reads (get k) last) + v' (first reads) + i (get vo v) + i' (get vo v') + delta (if (and i i') + (- i' i) + 1)] + [(when (< 1 delta) + ; We can show that this op skipped an + ; index! + (let [voi (-> version-orders (get k) + :by-index) + skipped (map voi + (range (inc i) i'))] + {:type :skip + :key k + :delta delta + :skipped skipped + :ops [last-op op]})) + (when (< delta 1) + ; Aha, this wasn't monotonic! + {:type :nonmonotonic + :key k + :values [v v'] + :delta delta + :ops [last-op op]})]) + ; First read of this key + nil)) + errs (->> errs + (mapcat identity) + (remove nil?)) + ; Update our last-reads index for this op's read keys + last-reads' (->> (keys reads) + (reduce (fn update-last-read [lr k] + (assoc lr k op)) + last-reads))] + (recur (next ops) (into errors errs) last-reads')) + + ; Some other :f + (recur (next ops) errors last-reads))))))) + ; Join together errors from all processes + (mapcat val) + (group-by :type) + (map-vals strip-types))) + +(defn nonmonotonic-send-cases + "Takes a partial analysis and checks each process's operations sequentially, + looking for cases where a single process's sends to a given key go backwards + relative to the version order." + [{:keys [history version-orders]}] + ; First, consider each process' completions... + (->> (filter (comp #{:ok :info} :type) history) + (group-by :process) + ; Then reduce each process, keeping track of the most recent sent + ; index for each key. + (map-vals + (fn per-process [ops] + ; Iterate over this process's operations, building up a vector of + ; errors. + (loop [ops ops + errors [] + last-sends {}] ; Key -> Most recent op to send to that key + (if-not (seq ops) + ; Done + errors + ; Process this op + (let [op (first ops)] + (case (:f op) + ; When we assign or subscribe a new topic, preserve *only* + ; those last-reads which we're still subscribing to. + (:assign, :subscribe) + (recur (next ops) + errors + (if (#{:invoke :fail} (:type op)) + last-sends + (select-keys last-sends (:value op)))) + + (:send, :txn) + (let [; Look at each key and values sent by this op. + sends (op-writes op) + errs (for [[k sends] sends] + ; What was the last op that sent to this key? + (if-let [last-op (get last-sends k)] + ; Compare the index of the last thing sent by + ; the last op to the index of our first send + (let [vo (get-in version-orders + [k :by-value]) + v (-> last-op op-writes (get k) last) + v' (first sends) + i (get vo v) + i' (get vo v') + delta (if (and i i') + (- i' i) + 1)] + (when (< delta 1) + ; Aha, this wasn't monotonic! + {:key k + :values [v v'] + :delta (- i' i) + :ops [last-op op]})) + ; First send to this key + nil)) + errs (->> errs + (remove nil?)) + ; Update our last-reads index for this op's sent keys + last-sends' (->> (keys sends) + (reduce (fn update-last-send [lr k] + (assoc lr k op)) + last-sends))] + (recur (next ops) (into errors errs) last-sends')) + + ; Some other :f + (recur (next ops) errors last-sends))))))) + ; Join together errors from all processes + (mapcat val) + seq)) + +(defn duplicate-cases + "Takes a partial analysis and identifies cases where a single value appears + at more than one offset in a key." + [{:keys [version-orders]}] + (->> version-orders + (mapcat (fn per-key [[k version-order]] + (->> (:by-index version-order) + frequencies + (filter (fn dup? [[value number]] + (< 1 number))) + (map (fn [[value number]] + {:key k + :value value + :count number}))))) + seq)) + +(defn unseen + "Takes a history and yields a series of maps like + + {:time The time in nanoseconds + :unseen A map of keys to the number of messages in that key which have + been successfully acknowledged, but not polled by any client.} + + The final map in the series includes a :messages key: a map of keys to sets + of messages that were unseen." + [history] + (let [[out unseen] + (reduce + ; Out is a vector of output observations. sent is a map of keys to + ; successfully sent values; polled is a map of keys to successfully + ; read values. + (fn red [[out sent polled :as acc] {:keys [type time f] :as op}] + (if-not (and (= type :ok) + (#{:poll :send :txn} f)) + acc + (let [sent' (->> op op-writes (map-vals set) + (merge-with set/union sent)) + polled' (->> op op-reads (map-vals set) + (merge-with set/union polled)) + unseen (merge-with set/difference sent' polled')] + ; We don't have to keep looking for things we've seen, so we can + ; recur with unseen rather than sent'. + [(conj! out {:time time, :unseen (map-vals count unseen)}) + unseen + polled']))) + [(transient []) {} {}] + history) + out (persistent! out)] + (when (seq out) + (update out (dec (count out)) + assoc :messages unseen)))) + +(defn plot-unseen! + "Takes a test, a collection of unseen measurements, and options (e.g. those + to checker/check). Plots a graph file (unseen.png) in the store directory." + [test unseen {:keys [subdirectory]}] + (let [ks (-> unseen peek :unseen keys sort reverse vec) + ; Turn these into stacked graphs + datasets (reduce + (fn [datasets {:keys [time unseen]}] + ; We go through keys in reverse order so that the last + ; key in the legend is on the bottom of the chart + (->> ks reverse + (reduce (fn [[datasets sum] k] + (if-let [count (get unseen k 0)] + (let [t (nanos->secs time) + sum' (+ sum count) + dataset (get datasets k []) + dataset' (conj dataset [t sum'])] + [(assoc datasets k dataset') sum']) + ; Key doesn't exist yet + [datasets sum])) + [datasets 0]) + first)) + {} + unseen) + ; Grab just the final poll section of history + final-polls (->> test :history rseq + (take-while (comp #{:poll :crash :assign + :debug-topic-partitions} :f))) + ; Draw a line for the start and end of final polls. + final-poll-lines (->> [(first final-polls) (last final-polls)] + (map (comp nanos->secs :time)) + (map (fn [t] + [:set :arrow + :from (gnuplot/list t [:graph 0]) + :to (gnuplot/list t [:graph 1]) + :lc :rgb "#F3974A" + :lw "1" + :nohead]))) + output (.getCanonicalPath + (store/path! test subdirectory "unseen.png")) + preamble (concat (perf/preamble output) + [[:set :title (str (:name test) " unseen")] + '[set ylabel "Unseen messages"]] + final-poll-lines) + series (for [k ks] + {:title (str "key " k) + :with '[filledcurves x1] + :data (datasets k)})] + (-> {:preamble preamble, :series series} + perf/with-range + perf/plot! + (try+ (catch [:type :jepsen.checker.perf/no-points] _ :no-points))))) + +(defn realtime-lag + "Takes a history and yields a series of maps of the form + + {:process The process performing a poll + :key The key being polled + :time The time the read began, in nanos + :lag The realtime lag of this key, in nanos. + + The lag of a key k in a poll is the conservative estimate of how long it has + been since the highest value in that poll was the final message in log k. + + For instance, given: + + {:time 1, :type :ok, :value [:send :x [0 :a]]} + {:time 2, :type :ok, :value [:poll {:x [0 :a]}]} + + The lag of this poll is zero, since we observed the most recent completed + write to x. However, if we: + + {:time 3, :type :ok, :value [:send :x [1 :b]]} + {:time 4, :type :invoke, :value [:poll]} + {:time 5, :type :ok, :value [:poll {:x []}]} + + The lag of this read is 4 - 3 = 1. By time 3, offset 1 must have existed for + key x. However, the most recent offset we observed was 0, which could only + have been the most recent offset up until the write of offset 1 at time 3. + Since our read could have occurred as early as time 4, the lag is at least 1. + + Might want to make this into actual [lower upper] ranges, rather than just + the lower bound on lag, but conservative feels OK for starters." + [history] + ; First, build up a map of keys to vectors, where element i of the vector for + ; key k is the time at which we knew offset i+1 was no longer the most recent + ; thing in the log. Thus the first entry in each vector is the time at which + ; the log was no longer *empty*. The second entry is the time at which offset + ; 0 was no longer current, and so on. + (let [expired + (reduce + (fn expired [expired {:keys [type f time value] :as op}] + ; What are the most recent offsets we know exist? + (let [known-offsets (op->max-offsets op)] + (reduce + (fn [expired [k known-offset]] + ; For this particulary key, we know this offset exists. + ; Therefore we *also* know every lower offset should + ; exist. We zip backwards, filling in offsets with the + ; current time, until we hit an offset where a lower + ; time exists. + (loop [expired-k (get expired k []) + i known-offset] + (if (or (neg? i) (get expired-k i)) + ; We've gone back before the start of the vector, or + ; we hit an index that's already known to be expired + ; earlier. + (assoc expired k expired-k) + ; Update this index + (recur (assocv expired-k i time) (dec i))))) + expired + known-offsets))) + {} + history) + pairs (history/pair-index+ history)] + ; Now work through the history, turning each poll operation into a lag map. + (loop [history history + ; A map of processes to keys to the highest offset that process has + ; seen. Keys in this map correspond to the keys this process + ; currently has :assign'ed. If using `subscribe`, keys are filled in + ; only when they appear in poll operations. The default offset is + ; -1. + process-offsets {} + ; A vector of output lag maps. + lags []] + (if-not (seq history) + lags + (let [[op & history'] history + {:keys [type process f value]} op] + (if-not (= :ok type) + ; Only OK ops matter + (recur history' process-offsets lags) + + (case f + ; When we assign something, we replace the process's offsets map. + :assign + (recur history' + (let [offsets (get process-offsets process {})] + ; Preserve keys that we're still subscribing to; + ; anything else gets reset to offset -1. This is gonna + ; break if we use something other than auto_offset_reset + ; = earliest, but... deal with that later. I've got + ; limited time here and just need to get SOMETHING + ; working. + (assoc process-offsets process + (merge (zipmap value (repeat -1)) + (select-keys offsets value)))) + lags) + + ; When we subscribe, we're not necessarily supposed to get + ; updates for the key we subscribed to--we subscribe to *topics*, + ; not individual partitions. We might get *no* updates, if other + ; subscribers have already claimed all the partitions for that + ; topic. We reset the offsets map to empty, to be conservative. + :subscribe + (recur history' (assoc process-offsets process {}) lags) + + (:poll, :txn) + (let [invoke-time (:time (get pairs op)) + ; For poll ops, we merge all poll mop results into this + ; process's offsets, then figure out how far behind each + ; key based on the time that key offset expired. + offsets' (merge-with max + (get process-offsets process {}) + (op->max-offsets op)) + lags' (->> offsets' + (map (fn [[k offset]] + ; If we've read *nothing*, then we're at + ; offset -1. Element 0 in the expired + ; vector for k tells us the time when the + ; first element was definitely present. + ; If we read offset 0, we want to consult + ; element 1 of the vector, which tells us + ; when offset 0 was no longer the tail, + ; and so on. + (let [expired-k (get expired k []) + i (inc offset) + expired-at (get-in expired + [k (inc offset)] + ##Inf) + lag (-> invoke-time + (- expired-at) + (max 0))] + {:time invoke-time + :process process + :key k + :lag lag}))) + (into lags))] + (recur history' + (assoc process-offsets process offsets') + lags')) + + ; Anything else, pass through + (recur history' process-offsets lags)))))))) + +(defn op->thread + "Returns the thread which executed a given operation." + [test op] + (-> op :process (mod (:concurrency test)))) + +(defn plot-realtime-lag! + "Takes a test, a collection of realtime lag measurements, and options (e.g. + those to checker/check). Plots a graph file (realtime-lag.png) in the store + directory" + [test lags {:keys [nemeses subdirectory filename + group-fn group-name]}] + (let [nemeses (or nemeses (:nemeses (:plot test))) + datasets (->> lags + (group-by group-fn) + (map-vals (fn [lags] + ; At any point in time, we want the maximum + ; lag for this thread across any key. + (->> lags + (partition-by :time) + (map (partial util/max-by :lag)) + (map (juxt (comp nanos->secs :time) + (comp nanos->secs :lag))))))) + output (.getCanonicalPath + (store/path! test subdirectory + (or filename "realtime-lag.png"))) + preamble (concat (perf/preamble output) + [[:set :title (str (:name test) + " realtime lag by " + group-name)] + '[set ylabel "Lag (s)"]]) + series (for [g (util/polysort (keys datasets))] + {:title (str group-name " " g) + :with 'linespoints + :data (datasets g)})] + (-> {:preamble preamble + :series series} + perf/with-range + perf/plot! + (try+ (catch [:type :jepsen.checker.perf/no-points] _ :no-points))))) + +(defn plot-realtime-lags! + "Constructs realtime lag plots for all processes together, and then another + broken out by process, and also by key." + [test lags opts] + (->> [{:group-name "thread", :group-fn (partial op->thread test)} + {:group-name "key", :group-fn :key} + {:group-name "thread-key", :group-fn (juxt (partial op->thread test) + :key)}] + (mapv (fn [{:keys [group-name group-fn] :as opts2}] + (let [opts (merge opts opts2)] + (plot-realtime-lag! + test lags (assoc opts :filename (str "realtime-lag-" + group-name ".png"))) + (->> lags + (group-by group-fn) + (pmap (fn [[thread lags]] + (plot-realtime-lag! + test lags + (assoc opts + :subdirectory (str "realtime-lag-" + group-name) + :filename (str thread ".png"))))) + dorun)))))) + +(defn worst-realtime-lag + "Takes a seq of realtime lag measurements, and finds the point with the + highest lag." + [lags] + (or (util/max-by :lag lags) 0)) + +(defn key-order-viz + "Takes a key, a log for that key (a vector of offsets to sets of elements + which were observed at that offset) and a history of ops relevant to that + key. Constructs an XML structure visualizing all sends/polls of that log's + offsets." + [k log history] + (let [; Turn an index into a y-coordinate + i->y (fn [i] (* i 14)) + ; Turn an offset into an x-coordinate + offset->x (fn [offset] (* offset 24)) + ; Turn a log, index, and op, and pairs in that op into an SVG element + row (fn [i {:keys [type time f process value] :as op} pairs] + (let [y (i->y i)] + (into [:g [:title (str (name type) " " (name f) + " by process " process "\n" + (pr-str value))]] + (for [[offset value] pairs :when offset] + (let [compat? (-> log (nth offset) count (< 2)) + style (str (when-not compat? + (str "background: " + (rand-bg-color value) ";")))] + [:text (cond-> {:x (offset->x offset) + :y y + :style style}) + (str value)]))))) + ; Compute rows and bounds + [_ max-x max-y rows] + (loopr [i 0 + max-x 0 + max-y 0 + rows []] + [op history] + (if-let [pairs (-> op op-pairs (get k))] + (let [row (row i op pairs) + max-y (->> row next next first second :y (max max-y)) + max-x (->> row next next + (map (comp :x second)) + (reduce max max-x))] + (recur (inc i) + max-x + max-y + (conj rows row))) + ; Nothing relevant here, skip it + (recur i max-x max-y rows))) + svg (svg/svg {"version" "2.0" + "width" (+ max-x 20) + "height" (+ max-y 20)} + [:style "svg { + font-family: Helvetica, Arial, sans-serif; + font-size: 10px; + }"] + (cons :g rows))] + svg)) + +(defn render-order-viz! + "Takes a test, an analysis, and for each key with certain errors + renders an HTML timeline of how each operation perceived that key's log." + [test {:keys [version-orders errors history] :as analysis}] + (let [history (filter (comp #{:ok} :type) history)] + (->> (select-keys errors [:inconsistent-offsets :duplicate :lost-write]) + (mapcat val) + (map :key) + (concat (->> errors :unseen :unseen keys)) + distinct + (pmap (fn [k] + (let [svg (key-order-viz k + (get-in version-orders [k :log]) + history) + path (store/path! test "orders" (format "%03d.svg" k))] + (spit path (xml/emit svg))))) + dorun))) + +(defn consume-counts + "Kafka transactions are supposed to offer 'exactly once' processing: a + transaction using the subscribe workflow should be able to consume an offset + and send something to an output queue, and if this transaction is successful, + it should happen at most once. It's not exactly clear to me *how* these + semantics are supposed to work--it's clearly not once per consumer group, + because we routinely see dups with only one consumer group. As a fallback, we + look for single consumer per process, which should DEFINITELY hold, but... + appears not to. + + We verify this property by looking at all committed transactions which + performed a poll while subscribed (not assigned!) and keeping track of the + number of times each key and value is polled. Yields a map of keys to values + to consumed counts, wherever that count is more than one." + [history] + (loopr [counts {} ; process->k->v->count + subscribed #{}] ; set of processes which are subscribed + [{:keys [type f process value] :as op} history] + (if (not= type :ok) + (recur counts subscribed) + (case f + :subscribe (recur counts (conj subscribed process)) + (:txn, :poll) + (if (subscribed process) + ; Increment the count for each value read by this txn + (recur (loopr [counts counts] + [[k vs] (op-reads op) + v vs] + (recur (update-in + counts [process k v] (fnil inc 0)))) + subscribed) + ; Don't care; this might be an assign poll, and assigns are free + ; to double-consume + (recur counts subscribed)) + + ; Default + (recur counts subscribed))) + ; Finally, compute a distribution, and filter out anything which was + ; only read once. + (loopr [dist (sorted-map) + dups (sorted-map)] + [[p k->v->count] counts + [k v->count] k->v->count + [v count] v->count] + (recur (update dist count (fnil inc 0)) + (if (< 1 count) + (let [k-dups' (-> (get dups k (sorted-map)) + (assoc v count))] + (assoc dups k k-dups')) + dups)) + {:distribution dist + :dup-counts dups}))) + +(defn writer-of + "Takes a history and builds a map of keys to values to the completion + operation which attempted to write that value." + [history] + (loopr [writer-of (transient {})] + [op (remove (comp #{:invoke} :type) history) + [k vs] (op-writes op) + v vs] + (let [k-writer-of (get writer-of k (transient {})) + k-writer-of' (assoc! k-writer-of v op)] + (recur (assoc! writer-of k k-writer-of'))) + (map-vals persistent! (persistent! writer-of)))) + +(defn readers-of + "Takes a history and builds a map of keys to values to vectors of completion + operations which observed those that value." + [history] + (loopr [readers (transient {})] + [op (remove (comp #{:invoke} :type) history) + [k vs] (op-reads op) + v vs] + (let [k-readers (get readers k (transient {})) + kv-readers (get k-readers v []) + kv-readers' (conj kv-readers op) + k-readers' (assoc! k-readers v kv-readers')] + (recur (assoc! readers k k-readers'))) + (map-vals persistent! (persistent! readers)))) + +(defn previous-value + "Takes a version order for a key and a value. Returns the previous value in + the version order, or nil if either we don't know v2's index or v2 was the + first value in the version order." + [version-order v2] + (when-let [i2 (-> version-order :by-value (get v2))] + (when (< 0 i2) + (-> version-order :by-index (nth (dec i2)))))) + +(defn mop-index + "Takes an operation, a function f (:poll or :send), a key k, and a value v. + Returns the index (0, 1, ...) within that operation's value which performed + that poll or send, or nil if none could be found." + [op f k v] + (loopr [i 0 + mop-index nil] + [[fun a b] (:value op)] + (if mop-index + (recur i mop-index) + (if (and (= f fun) + (case f + :send (and (= k a) + (if (vector? b) + (= v (second b)) + (= v b))) + :poll (when-let [pairs (get a k)] + (some (comp #{v} second) pairs)))) + (recur (inc i) i) + (recur (inc i) mop-index))) + mop-index)) + +(defrecord WWExplainer [writer-of version-orders] + elle/DataExplainer + (explain-pair-data [_ a b] + (->> (for [[k vs] (op-writes b) + v2 vs] + (when-let [v1 (previous-value (version-orders k) v2)] + (if-let [writer (-> writer-of (get k) (get v1))] + (when (= a writer) + {:type :ww + :key k + :value v1 + :value' v2 + :a-mop-index (mop-index a :send k v1) + :b-mop-index (mop-index b :send k v2)}) + (throw+ {:type :no-writer-of-value, :key k, :value v1})))) + (remove nil?) + first)) + + (render-explanation [_ {:keys [key value value'] :as m} a-name b-name] + (str a-name " sent " (pr-str value) " to " (pr-str key) + " before " b-name " sent " (pr-str value')))) + +; A trivial explainer which refuses to acknowledge any connection between +; things. +(defrecord NeverExplainer [] + elle/DataExplainer + (explain-pair-data [_ a b] nil) + (render-explanation [_ _ _ _] nil)) + +(defn ww-graph + "Analyzes a history to extract write-write dependencies. T1 < T2 iff T1 sends + some v1 to k and T2 sends some v2 to k and v1 < v2 in the version order." + [{:keys [writer-of version-orders ww-deps]} history] + {:graph (if-not ww-deps + ; We might ask not to infer ww dependencies, in which case this + ; graph is empty. + (g/digraph) + (loopr [g (g/linear (g/digraph))] + [[k v->writer] writer-of ; For every key + [v2 op2] v->writer] ; And very value written in that key + (let [version-order (get version-orders k)] + (if-let [v1 (previous-value version-order v2)] + (if-let [op1 (v->writer v1)] + (if (= op1 op2) + (recur g) ; No self-edges + (recur (g/link g op1 op2 :ww))) + (throw+ {:type :no-writer-of-value + :key k + :value v1})) + ; This is the first value in the version order. + (recur g))) + (g/forked g))) + :explainer (if-not ww-deps + (NeverExplainer.) + (WWExplainer. writer-of version-orders))}) + +(defrecord WRExplainer [writer-of] + elle/DataExplainer + (explain-pair-data [_ a b] + (->> (for [[k vs] (op-reads b) + v vs] + (if-let [writer (-> writer-of (get k) (get v))] + (when (= a writer) + {:type :wr + :key k + :value v + :a-mop-index (mop-index a :send k v) + :b-mop-index (mop-index b :poll k v)}) + (throw+ {:type :no-writer-of-value, :key k, :value v}))) + (remove nil?) + first)) + + (render-explanation [_ {:keys [key value] :as m} a-name b-name] + (str a-name " sent " (pr-str value) " to " (pr-str key) + " which was polled by " b-name))) + +(defn wr-graph + "Analyzes a history to extract write-read dependencies. T1 < T2 iff T1 writes + some v to k and T2 reads k." + [{:keys [writer-of readers-of]} history] + {:graph (loopr [g (g/linear (g/digraph))] + [[k v->readers] readers-of + [v readers] v->readers] + (if-let [writer (-> writer-of (get k) (get v))] + (let [readers (remove #{writer} readers)] + (recur (g/link-to-all g writer readers :wr))) + (throw+ {:type :no-writer-of-value, :key k, :value v})) + (g/forked g)) + :explainer (WRExplainer. writer-of)}) + +(defn graph + "A combined Elle dependency graph between completion operations." + [analysis history] + ((elle/combine (partial ww-graph analysis) + (partial wr-graph analysis) + ;(partial rw-graph analysis)) + ) + history)) + +(defn cycles! + "Finds a map of cycle names to cyclic anomalies in a partial analysis." + [{:keys [history directory] :as analysis}] + ; Bit of a hack--our tests leave off :type fairly often, so we don't bother + ; running this analysis for those tests. + (when (:type (first history)) + (let [opts (cond-> {:consistency-models [:strict-serializable]} + (contains? analysis :directory) + (assoc :directory (str directory "/elle"))) + ; For our purposes, we only want to infer cycles over txn/poll/send + ; ops + history (->> history + (filter (comp #{:txn :poll :send} :f)) + vec) + analyzer (->> opts + txn/additional-graphs + (into [(partial graph analysis)]) + (apply elle/combine))] + (:anomalies (txn/cycles! opts analyzer history))))) + +(defn analysis + "Builds up intermediate data structures used to understand a history. Options + include: + + :directory - Used for generating output files + :ww-deps - Whether to perform write-write inference on the basis of log + offsets." + ([history] + (analysis history {})) + ([history opts] + (let [history (history/index history) + history (remove (comp #{:nemesis} :process) history) + writes-by-type (future (writes-by-type history)) + reads-by-type (future (reads-by-type history)) + version-orders (future (version-orders history @reads-by-type)) + writer-of (future (writer-of history)) + readers-of (future (readers-of history)) + ; Sort of a hack; we only bother computing this for "real" histories + ; because our test suite often leaves off processes and times + realtime-lag (future + (let [op (first history)] + (when (every? op [:process :time]) + (realtime-lag history)))) + worst-realtime-lag (future (worst-realtime-lag @realtime-lag)) + unseen (future (unseen history)) + version-order-errors (:errors @version-orders) + version-orders (:orders @version-orders) + analysis (assoc opts + :history history + :writer-of @writer-of + :readers-of @readers-of + :writes-by-type @writes-by-type + :reads-by-type @reads-by-type + :version-orders version-orders) + g1a-cases (future (g1a-cases analysis)) + lost-write-cases (future (lost-write-cases analysis)) + poll-skip+nm-cases (future (poll-skip+nonmonotonic-cases analysis)) + nonmonotonic-send-cases (future (nonmonotonic-send-cases analysis)) + int-poll-skip+nm-cases (future (int-poll-skip+nonmonotonic-cases analysis)) + int-send-skip+nm-cases (future (int-send-skip+nonmonotonic-cases analysis)) + duplicate-cases (future (duplicate-cases analysis)) + cycles (future (cycles! analysis)) + poll-skip-cases (:skip @poll-skip+nm-cases) + nonmonotonic-poll-cases (:nonmonotonic @poll-skip+nm-cases) + int-poll-skip-cases (:skip @int-poll-skip+nm-cases) + int-nm-poll-cases (:nonmonotonic @int-poll-skip+nm-cases) + int-send-skip-cases (:skip @int-send-skip+nm-cases) + int-nm-send-cases (:nonmonotonic @int-send-skip+nm-cases) + last-unseen (-> (peek @unseen) + (update :unseen + (fn [unseen] + (->> unseen + (filter (comp pos? val)) + (into (sorted-map))))) + (update :messages + (fn [messages] + (->> messages + (filter (comp seq val)) + (into (sorted-map)))))) + ] + {:errors (cond-> {} + @duplicate-cases + (assoc :duplicate @duplicate-cases) + + int-poll-skip-cases + (assoc :int-poll-skip int-poll-skip-cases) + + int-nm-poll-cases + (assoc :int-nonmonotonic-poll int-nm-poll-cases) + + int-nm-send-cases + (assoc :int-nonmonotonic-send int-nm-send-cases) + + int-send-skip-cases + (assoc :int-send-skip int-send-skip-cases) + + version-order-errors + (assoc :inconsistent-offsets version-order-errors) + + @g1a-cases + (assoc :G1a @g1a-cases) + + @lost-write-cases + (assoc :lost-write @lost-write-cases) + + nonmonotonic-poll-cases + (assoc :nonmonotonic-poll nonmonotonic-poll-cases) + + @nonmonotonic-send-cases + (assoc :nonmonotonic-send @nonmonotonic-send-cases) + + poll-skip-cases + (assoc :poll-skip poll-skip-cases) + + (seq (:unseen last-unseen)) + (assoc :unseen last-unseen) + + true + (merge @cycles)) + :history history + :realtime-lag @realtime-lag + :worst-realtime-lag @worst-realtime-lag + :unseen @unseen + :version-orders version-orders}))) + +(defn condense-error + "Takes a test and a pair of an error type (e.g. :lost-write) and a seq of + errors. Returns a pair of [type, {:count n, :errors [...]}], which tries to + show the most interesting or severe errors without making the pretty-printer + dump out two gigabytes of EDN." + [test [type errs]] + [type + (case type + :unseen (if (:all-errors test) + errs + (assoc errs :messages + (map-vals (comp (partial take 32) sort) + (:messages errs)))) + {:count (count errs) + :errs + (if (:all-errors test) + errs + (case type + :duplicate (take 32 (sort-by :count errs)) + (:G0, :G0-process, :G0-realtime, + :G1c, :G1c-process, :G1c-realtime) + (take 8 (sort-by (comp count :steps) errs)) + :inconsistent-offsets (take 32 (sort-by (comp count :values) errs)) + :int-nonmonotonic-poll (take 8 (sort-by :delta errs)) + :int-nonmonotonic-send (take 8 (sort-by :delta errs)) + :int-poll-skip (take-last 8 (sort-by :delta errs)) + :int-send-skip (take-last 8 (sort-by :delta errs)) + :nonmonotonic-poll (take 8 (sort-by :delta errs)) + :nonmonotonic-send (take 8 (sort-by :delta errs)) + :poll-skip (take-last 8 (sort-by :delta errs)) + errs))})]) + +(defn allowed-error-types + "Redpanda does a lot of *things* that are interesting to know about, but not + necessarily bad or against-spec. For instance, g0 cycles are normal in the + Kafka transactional model, and g1c is normal with wr-only edges at + read-uncommitted but *not* with read-committed. This is a *very* ad-hoc + attempt to encode that so that Jepsen's valid/invalid results are somewhat + meaningful.] + + Takes a test, and returns a set of keyword error types (e.g. :poll-skip) + which this test considers allowable." + [test] + (cond-> #{; int-send-skip is normal behavior: transaction writes interleave + ; constantly in the Kafka transaction model. We don't even bother + ; looking at external send skips. + :int-send-skip + ; Likewise, G0 is always allowed, since writes are never isolated + :G0 :G0-process :G0-realtime + } + + ; With subscribe, we expect external poll skips and nonmonotonic + ; polls, because our consumer might be rebalanced between + ; transactions. Without subscribe, we expect consumers to proceed + ; in order. + (:subscribe (:sub-via test)) (conj :poll-skip :nonmonotonic-poll) + + ; When we include ww edges, G1c is normal--the lack of write + ; isolation means we should expect cycles like t0 > history + (filter (comp #{:info} :type)) + (filter (comp #{:txn :send :poll} :f)) + (map :error) + distinct) + ; Which errors are bad enough to invalidate the test? + bad-error-types (->> (keys errors) + (remove (allowed-error-types test)) + sort)] + ; Render plots + (render-order-viz! test analysis) + (plot-unseen! test unseen opts) + (plot-realtime-lags! test realtime-lag opts) + ; Write out a file with consume counts + (store/with-out-file test "consume-counts.edn" + (pprint (consume-counts history))) + ; Construct results + (->> errors + (map (partial condense-error test)) + (into (sorted-map)) + (merge {:valid? (empty? bad-error-types) + :worst-realtime-lag (-> worst-realtime-lag + (update :time nanos->secs) + (update :lag nanos->secs)) + :bad-error-types bad-error-types + :error-types (sort (keys errors)) + :info-txn-causes info-txn-causes})))))) + +(defn workload + "Constructs a workload (a map with a generator, client, checker, etc) given + an options map. Options are: + + (none) + + ... plus those taken by jepsen.tests.cycle.append/test, e.g. :key-count, + :min-txn-length, ..." + [opts] + (let [workload (append/test + (assoc opts + ; TODO: don't hardcode these + :max-txn-length (if (:txn opts) 4 1))) + max-offsets (atom (sorted-map))] + (assoc workload + :client (client) + :checker (checker) + :final-generator (gen/each-thread + (final-polls max-offsets) + ;(->> (gen/phases (final-polls max-offsets) + ; (repeat (gen/sleep 1))) + ; (gen/time-limit 10) + ; repeat)) + ) + :generator (gen/any + (crash-client-gen opts) + (->> (:generator workload) + txn-generator + tag-rw + (track-key-offsets max-offsets) + interleave-subscribes))))) diff --git a/src/jepsen/hstream/kafka_test.clj b/src/jepsen/hstream/kafka_test.clj new file mode 100644 index 0000000..4872add --- /dev/null +++ b/src/jepsen/hstream/kafka_test.clj @@ -0,0 +1,425 @@ +(ns jepsen.hstream.kafka-test + "Entry point for command line runner. Constructs tests and runs them." + (:require [clojure [string :as str] + [walk :as walk]] + [clojure.tools.logging :refer [info warn]] + [jepsen [checker :as checker] + [cli :as cli] + [generator :as gen] + [tests :as tests]] + [jepsen.os :as os] + [jepsen.hstream.kafka [nemesis :as nemesis]] + [jepsen.hstream.kafka.db [kafka :as db.kafka] + [hstream :as db.hstream] + [redpanda :as db.redpanda]] + [jepsen.hstream.kafka.workload [list-append :as list-append] + [queue :as queue]] + [slingshot.slingshot :refer [try+ throw+]] + [jepsen.hstream.kafka.db.hstream :as db.hstream]) + (:import (org.apache.http.impl.client InternalHttpClient))) + +(def workloads + "A map of workload names to workload constructor functions." + {:list-append list-append/workload + :queue queue/workload}) + +(def nemeses + "The types of faults our nemesis can produce" + #{:pause :kill :partition :clock :membership}) + +(def special-nemeses + "A map of special nemesis names to collections of faults" + {:none [] + :standard [:pause :kill :partition :clock] + :all [:pause :kill :partition :clock]}) + +(def db-targets + "Valid targets for DB nemesis operations." + #{:one :primaries :minority-third :majority :all}) + +(def partition-targets + "Valid targets for partition nemesis operations." + #{:one :primaries :minority-third :majority-ring}) + +(def standard-nemeses + "A collection of partial options maps for various nemeses we want to run as a + part of test-all." + [{:nemesis nil} + {:nemesis #{:partition}} + {:nemesis #{:kill}} + {:nemesis #{:pause}} + {:nemesis #{:membership}} + {:nemesis #{:clock}} + {:nemesis #{:kill :partition :clock :membership}}]) + +(defn parse-comma-kws + "Takes a comma-separated string and returns a collection of keywords." + [spec] + (->> (str/split spec #",") + (remove #{""}) + (map keyword))) + +(defn parse-nemesis-spec + "Takes a comma-separated nemesis string and returns a collection of keyword + faults." + [spec] + (->> (parse-comma-kws spec) + (mapcat #(get special-nemeses % [%])) + set)) + +(def logging-overrides + "New logging levels for various Kafka packages--otherwise this test is going + to be NOISY" + {"org.apache.kafka.clients.FetchSessionHandler" :warn + ; This complains about invalid topics during partitions, too + "org.apache.kafka.clients.Metadata" :off + ; This is going to give us all kinds of NOT_CONTROLLER or + ; UNKNOWN_SERVER_ERROR messages during partitions + "org.apache.kafka.clients.NetworkClient" :error + "org.apache.kafka.clients.admin.AdminClientConfig" :warn + "org.apache.kafka.clients.admin.KafkaAdminClient" :warn + "org.apache.kafka.clients.admin.internals.AdminMetadataManager" :warn + "org.apache.kafka.clients.consumer.ConsumerConfig" :warn + "org.apache.kafka.clients.consumer.internals.ConsumerCoordinator" :warn + ; This is also going to kvetch about unknown topic/partitions when listing + ; offsets + "org.apache.kafka.clients.consumer.internals.Fetcher" :error + "org.apache.kafka.clients.consumer.internals.SubscriptionState" :warn + "org.apache.kafka.clients.consumer.KafkaConsumer" :warn + "org.apache.kafka.clients.producer.KafkaProducer" :warn + ; Comment this to see the config opts for producers + "org.apache.kafka.clients.producer.ProducerConfig" :warn + ; We're gonna get messages constantly about NOT_LEADER_OR_FOLLOWER whenever + ; we create a topic, and it's also going to complain when trying to send to + ; paused brokers that they're not available + "org.apache.kafka.clients.producer.internals.Sender" :off + "org.apache.kafka.clients.producer.internals.TransactionManager" :warn + "org.apache.kafka.common.metrics.Metrics" :warn + "org.apache.kafka.common.utils.AppInfoParser" :warn + }) + +(defn contains-http-client? + "Seriously, where the hell is this HTTP client coming from?" + [x] + (let [found? (atom false)] + (walk/postwalk (fn [x] + ; Can't refer to a private class directly... + (when (and x (re-find #"HttpClient" (.getName (class x)))) + (reset! found? true)) + x) + x) + @found?)) + +(defn http-client-warner + "Takes a generator and listens for updates, warning about any HTTP clients + that leak out. You can wrap a generator in this if you wind up hitting + serialization errors due to an HTTP client leaking into the history, and + can't figure out just from the logs where it came from--this will cause the + whole test to explode when it happens." + [gen] + (reify gen/Generator + (op [this test context] + (when-let [[op gen'] (gen/op gen test context)] + [op (http-client-warner gen')])) + + (update [this test context event] + (let [gen' (gen/update gen test context event)] + (when (contains-http-client? event) + (throw+ {:type :http-client-in-history + :event event})) + (http-client-warner gen'))))) + +(defn short-version + "Takes CLI options and returns a short version string like \"21.11.2\" or + \"foo.deb\"." + [{:keys [version deb]}] + (if deb + (nth (re-find #"/?([^\/]+)$" deb) 1) + (if-let [[_ short] (re-find #"^([\d\.]+)-" version)] + short + version))) + +(defn stats-checker + "A modified version of the stats checker which doesn't care if :crash or + :debug-topic-partitions ops always crash." + [] + (let [c (checker/stats)] + (reify checker/Checker + (check [this test history opts] + (let [res (checker/check c test history opts)] + (if (every? :valid? (vals (dissoc (:by-f res) + :debug-topic-partitions + :crash))) + (assoc res :valid? true) + res)))))) + +(defn perf-checker + "A modified perf checker which doesn't render debug-topic-partitions, assign, + or crash operations." + [perf-opts] + (let [c (checker/perf perf-opts)] + (reify checker/Checker + (check [this test history opts] + (checker/check c test + (->> history + (remove (comp #{:assign + :crash + :debug-topic-partitions} + :f))) + opts))))) + +(defn test-name + "Takes CLI options and constructs a test name as a string." + [opts] + (str (case (:db opts) + :kafka "kafka" + :redpanda (str "redpanda " (short-version opts)) + :hstream "kafka") + " " (name (:workload opts)) + (when (:txn opts) " txn") + " " + (->> opts :sub-via (map name) sort (str/join ",")) + (when-let [acks (:acks opts)] (str " acks=" acks)) + (when-let [r (:retries opts)] (str " retries=" r)) + (when-let [aor (:auto-offset-reset opts)] + (str " aor=" aor)) + (when-let [r (:default-topic-replications opts)] + (str " default-r=" r)) + (when (contains? + opts :enable-server-auto-create-topics) + (str " auto-topics=" (:enable-server-auto-create-topics opts))) + (when (contains? opts :idempotence) + (str " idem=" (:idempotence opts))) + (when-let [n (:nemesis opts)] + (str " " (->> n (map name) sort (str/join ",")))))) + +(defn redpanda-test + "Constructs a test for RedPanda from parsed CLI options." + [opts] + (let [workload-name (:workload opts) + workload ((workloads workload-name) opts) + db (case (:db opts) + :redpanda (db.redpanda/db) + :kafka (db.kafka/db) + :hstream (db.hstream/db "0.19.0")) + nemesis (nemesis/package + {:db db + :nodes (:nodes opts) + :faults (:nemesis opts) + :partition {:targets (:partition-targets opts)} + :clock {:targets (:db-targets opts)} + :pause {:targets (:db-targets opts)} + :kill {:targets (:db-targets opts)} + :interval (:nemesis-interval opts)}) + generator (let [fg (:final-generator workload)] + (-> (gen/phases + (->> (:generator workload) + (gen/stagger (/ (:rate opts))) + (gen/nemesis (:generator nemesis)) + (gen/time-limit (:time-limit opts))) + (gen/nemesis (:final-generator nemesis)) + (when fg + (gen/phases + (gen/log "Waiting for recovery") + (gen/sleep 10) + ; Redpanda might not give consumers elements + ; they want to see, so we eventually give up + ; here + (gen/time-limit (:final-time-limit opts) + (gen/clients + (:final-generator workload)))))) + ; Uncomment this if you're having trouble debugging + ; serialization errors with HTTPClients + ;http-client-warner + ))] + (merge tests/noop-test + opts + {:name (test-name opts) + :db db + :os os/noop + :client (:client workload) + :nemesis (:nemesis nemesis) + :generator generator + :checker (checker/compose + {:stats (stats-checker) + :clock (checker/clock-plot) + :perf (perf-checker + {:nemeses (:perf nemesis)}) + :ex (checker/unhandled-exceptions) + :assert (checker/log-file-pattern + #"\] assert -" "redpanda.log") + :workload (:checker workload)}) + :perf-opts {:nemeses (:perf nemesis)} + :logging {:overrides logging-overrides}}))) + +(def validate-non-neg + [#(and (number? %) (not (neg? %))) "Must be non-negative"]) + +(def cli-opts + "Command line options." + [[nil "--acks ACKS" "What level of acknowledgement should our producers use? Default is unset (uses client default); try 1 or 'all'." + :default nil] + + [nil "--auto-offset-reset BEHAVIOR" "How should consumers handle it when there's no initial offset in Kafka?" + :default nil] + + [nil "--crash-clients" "If set, periodically crashes clients and forces them to set up fresh consumers/producers/etc." + :default false] + + [nil "--crash-client-interval" "Roughly how long in seconds does a single client get to run for before crashing?" + :default 30 + :parse-fn read-string + :validate [#(and (number? %) (pos? %)) "must be a positive number"]] + + [nil "--db TYPE" "Which DB do we test? Either `hstream` (default), `redpanda` or `kafka`" + :default :hstream + :parse-fn keyword + :validate [#(some #{%} '(:hstream :kafka :redpanda)) "Must be one of hstream, kafka or redpanda"]] + + [nil "--db-targets TARGETS" "A comma-separated list of nodes to pause/kill/etc; e.g. one,all" + ;:default [:primaries :all] + :default [:one :all] + :parse-fn parse-comma-kws + :validate [(partial every? db-targets) (cli/one-of db-targets)]] + + [nil "--deb FILE" "Install this specific .deb file instead of downloading --version"] + + [nil "--default-topic-replications INT" "If set, sets Redpanda's default topic replications to this factor. If unset, leaves it as the default value." + :default nil + :parse-fn parse-long + :validate [pos? "must be positive"]] + + [nil "--enable-auto-commit" "If set, disables automatic commits via Kafka consumers. If not provided, uses the client default." + :default nil + :assoc-fn (fn [m _ _] (assoc m :enable-auto-commit true))] + + [nil "--disable-auto-commit" "If set, enables automatic commits via Kafka consumers. If not provided, uses the client default." + :assoc-fn (fn [m _ _] (assoc m :enable-auto-commit false))] + + [nil "--enable-server-auto-create-topics" "If set, enables automatic topic creation on the server. If not provided, uses the server default." + :assoc-fn (fn [m _ _] (assoc m :enable-server-auto-create-topics true))] + + [nil "--disable-server-auto-create-topics" "If set, disables automatic topic creation on the server. If not provided, uses the server default." + :assoc-fn (fn [m _ _] (assoc m :enable-server-auto-create-topics false))] + + [nil "--final-time-limit SECONDS" "How long should we run the final generator for, at most? In seconds." + :default 200 + :parse-fn read-string + :validate [#(and (number? %) (pos? %)) "must be a positive number"]] + + [nil "--[no-]idempotence" "If true, asks producers to enable idempotence. If omitted, uses client defaults."] + + [nil "--isolation-level NAME" "What isolation level should we request for consumers? e.g. read_committed"] + + [nil "--max-writes-per-key LIMIT" "How many writes do we perform per key at most?" + :default 1024 + :parse-fn parse-long + :validate [pos? "Must be positive"]] + + [nil "--nemesis FAULTS" "A comma-separated list of nemesis faults to enable" + :parse-fn parse-nemesis-spec + :validate [(partial every? (into nemeses (keys special-nemeses))) + (str "Faults must be one of " nemeses " or " + (cli/one-of special-nemeses))]] + + [nil "--nemesis-interval SECONDS" "How long to wait between nemesis faults." + :default 15 + :parse-fn read-string + :validate [#(and (number? %) (pos? %)) "must be a positive number"]] + + [nil "--partition-targets TARGETS" "A comma-separated list of nodes to target for network partitions; e.g. one,all" + ;:default [:primaries :majorities-ring] + :default [:one :majority :majorities-ring] + :parse-fn parse-comma-kws + :validate [(partial every? partition-targets) (cli/one-of partition-targets)]] + + [nil "--rate HZ" "Target number of ops/sec" + :default 100 + :parse-fn read-string + :validate validate-non-neg] + + [nil "--retries COUNT" "Producer retries. If omitted, uses client default." + :parse-fn parse-long] + + ["-s" "--safe" "Runs with the safest settings: --disable-auto-commit, --disable-server-auto-create-topics, --acks all, --default-topic-replications 3, --disable-server --retries 1000, --idempotence, --isolation-level read_committed --auto-offset-reset earliest, --sub-via assign. You can override individual settings by following -s with additional arguments, like so: -s --acks 0" + :assoc-fn (fn [m _ _] + (assoc m + :default-topic-replications 3 + :enable-auto-commit false + :acks "all" + :retries 1000 + :idempotence true + :isolation-level "read_committed" + :auto-offset-reset "earliest" + :enable-server-auto-create-topics false + :sub-via #{:assign}))] + + [nil "--[no-]server-idempotence" "If set, enables server idempotence support." + :default true] + + [nil "--sub-via STRATEGIES" "A comma-separated list like `assign,subscribe`, which denotes how we ask clients to assign topics to themselves." + :default #{:subscribe} + :parse-fn (comp set parse-comma-kws) + :validate [#(every? #{:assign :subscribe} %) + "Can only be assign and/or subscribe"]] + + [nil "--tcpdump" "If set, grabs tcpdump traces of client->server traffic on each node."] + + [nil "--[no-]txn" "Enables transactions for the queue workload."] + + ["-v" "--version STRING" "What version of Redpanda should we install? See apt list --all-versions redpanda for a full list of available versions." + :default "21.10.1-1-e7b6714a"] + + ["-w" "--workload NAME" "Which workload should we run?" + :parse-fn keyword + :default :queue + :validate [workloads (cli/one-of workloads)]] + + [nil "--[no-]ww-deps" "Enables or disables support for write-write dependency inference based on offsets in the queue workload. Redpanda allows write cycles everywhere. Disabling ww edges is helpful in hunting for pure wr cycles." + :default true] +]) + +(def test-all-cli-opts + "Additional options for test-all" + [[nil "--versions VERSIONS" "Comma separated versions to test." + :parse-fn #(str/split % #",\s*") + :default nil + :validate [seq "Must not be empty"]]]) + +(defn all-tests + "Takes parsed CLI options and constructs a sequence of tests." + [opts] + (let [workloads (if-let [w (:workload opts)] + [w] + (keys workloads)) + versions (or (:versions opts) + [(:version opts)]) + nemeses (if (nil? (:nemesis opts)) + standard-nemeses + [{}])] + (for [i (range (:test-count opts)) + version versions + workload workloads + nemesis nemeses + txn (case workload + ; No txn support + :list-append [false] + ; Prefer CLI opts, or both true and false. + :queue (if (nil? (:txn opts)) + [true false] + [(:txn opts)]))] + (-> opts + (assoc :workload workload, :version version, :txn txn) + (merge nemesis) + redpanda-test)))) + +(defn -main + "CLI entry point." + [& args] + (cli/run! (merge (cli/single-test-cmd {:test-fn redpanda-test + :opt-spec cli-opts}) + (cli/test-all-cmd {:tests-fn all-tests + :opt-spec (concat cli-opts + test-all-cli-opts)}) + (cli/serve-cmd)) + args)) diff --git a/src/jepsen/hstream/checker.clj b/src/jepsen/hstream/legacy/checker.clj similarity index 99% rename from src/jepsen/hstream/checker.clj rename to src/jepsen/hstream/legacy/checker.clj index 028529e..c00a319 100644 --- a/src/jepsen/hstream/checker.clj +++ b/src/jepsen/hstream/legacy/checker.clj @@ -1,4 +1,4 @@ -(ns jepsen.hstream.checker +(ns jepsen.hstream.legacy.checker "Validates that a history is correct with respect to some model." (:refer-clojure :exclude [set]) (:require @@ -9,7 +9,7 @@ [knossos [op :as op]] [slingshot.slingshot :refer [try+ throw+]] [jepsen.checker :refer :all] - [jepsen.hstream.utils :refer [queue-property is-sorted? map-value]])) + [jepsen.hstream.common.utils :refer [queue-property is-sorted? map-value]])) (defn set+ "Given a set of :add operations followed by **A SERIES OF** final :read, verifies that diff --git a/src/jepsen/hstream/client.clj b/src/jepsen/hstream/legacy/client.clj similarity index 98% rename from src/jepsen/hstream/client.clj rename to src/jepsen/hstream/legacy/client.clj index 285d4c5..13d52f1 100644 --- a/src/jepsen/hstream/client.clj +++ b/src/jepsen/hstream/legacy/client.clj @@ -1,9 +1,8 @@ -(ns jepsen.hstream.client - (:gen-class) +(ns jepsen.hstream.legacy.client (:require [clojure.core.reducers :as reducers] [clojure.tools.logging :refer :all] [slingshot.slingshot :refer [try+]] - [jepsen.hstream.utils :refer :all]) + [jepsen.hstream.common.utils :refer :all]) (:import [io.hstream HStreamClient HStreamClientBuilder ProducerBuilder HRecord HRecordBuilder Subscription HRecordReceiver Stream Record])) diff --git a/src/jepsen/hstream/common.clj b/src/jepsen/hstream/legacy/common.clj similarity index 97% rename from src/jepsen/hstream/common.clj rename to src/jepsen/hstream/legacy/common.clj index 1785ac4..bbcdb5c 100644 --- a/src/jepsen/hstream/common.clj +++ b/src/jepsen/hstream/legacy/common.clj @@ -1,15 +1,14 @@ -(ns jepsen.hstream.common - (:gen-class) +(ns jepsen.hstream.legacy.common (:require [clojure.pprint :refer [pprint]] [clojure.tools.logging :refer :all] [jepsen [db :as db] [cli :as cli] [checker :as checker] [client :as client] [generator :as gen] [nemesis :as nemesis] [tests :as tests]] - [jepsen.hstream.client :refer :all] - [jepsen.hstream.mvar :refer :all] - [jepsen.hstream.nemesis :as local-nemesis] + [jepsen.hstream.legacy.client :refer :all] + [jepsen.hstream.common.mvar :refer :all] + [jepsen.hstream.legacy.nemesis :as local-nemesis] [slingshot.slingshot :refer [throw+ try+]] - [jepsen.hstream.utils :refer :all])) + [jepsen.hstream.common.utils :refer :all])) (defn db-with-streams-initialized "HStream DB for a particular version. Here we use the FIRST diff --git a/src/jepsen/hstream/husky.clj b/src/jepsen/hstream/legacy/husky.clj similarity index 97% rename from src/jepsen/hstream/husky.clj rename to src/jepsen/hstream/legacy/husky.clj index c3b2782..90ae2ff 100644 --- a/src/jepsen/hstream/husky.clj +++ b/src/jepsen/hstream/legacy/husky.clj @@ -1,7 +1,6 @@ -(ns jepsen.hstream.husky - (:gen-class) - (:require [jepsen.hstream.husky.utils :refer :all] - [jepsen.hstream.utils :refer +(ns jepsen.hstream.legacy.husky + (:require [jepsen.hstream.legacy.husky.utils :refer :all] + [jepsen.hstream.common.utils :refer [insert first-index random-stream-name]] [jepsen.generator :as gen])) diff --git a/src/jepsen/hstream/husky/utils.clj b/src/jepsen/hstream/legacy/husky/utils.clj similarity index 95% rename from src/jepsen/hstream/husky/utils.clj rename to src/jepsen/hstream/legacy/husky/utils.clj index d9296bc..280d5cd 100644 --- a/src/jepsen/hstream/husky/utils.clj +++ b/src/jepsen/hstream/legacy/husky/utils.clj @@ -1,7 +1,6 @@ -(ns jepsen.hstream.husky.utils - (:gen-class) +(ns jepsen.hstream.legacy.husky.utils (:require [clojure.core.reducers :as reducers] - [jepsen.hstream.utils :refer [first-index]] + [jepsen.hstream.common.utils :refer [first-index]] [jepsen.generator :as gen])) (defn rand-int-exclude-zero diff --git a/src/jepsen/hstream/nemesis.clj b/src/jepsen/hstream/legacy/nemesis.clj similarity index 96% rename from src/jepsen/hstream/nemesis.clj rename to src/jepsen/hstream/legacy/nemesis.clj index d3980b7..0017ab4 100644 --- a/src/jepsen/hstream/nemesis.clj +++ b/src/jepsen/hstream/legacy/nemesis.clj @@ -1,14 +1,13 @@ -(ns jepsen.hstream.nemesis - (:gen-class) +(ns jepsen.hstream.legacy.nemesis (:require [clojure.tools.logging :refer :all] [slingshot.slingshot :refer [try+]] [jepsen [db :as db] [cli :as cli] [checker :as checker] [client :as client] [control :as c] [generator :as gen] [independent :as independent] [nemesis :as nemesis] [tests :as tests]] - [jepsen.hstream.client :refer :all] - [jepsen.hstream.mvar :refer :all] - [jepsen.hstream.utils :refer :all] + [jepsen.hstream.legacy.client :refer :all] + [jepsen.hstream.common.mvar :refer :all] + [jepsen.hstream.common.utils :refer :all] [jepsen.net :as net])) (defn kill-node diff --git a/src/jepsen/write_then_read.clj b/src/jepsen/hstream/list_append_test.clj similarity index 89% rename from src/jepsen/write_then_read.clj rename to src/jepsen/hstream/list_append_test.clj index 81bab1a..a6ff327 100644 --- a/src/jepsen/write_then_read.clj +++ b/src/jepsen/hstream/list_append_test.clj @@ -1,20 +1,19 @@ -(ns jepsen.write-then-read - (:gen-class) +(ns jepsen.hstream.list-append-test (:require [clojure.tools.logging :refer :all] [jepsen [db :as db] [cli :as cli] [checker :as checker] [client :as client] [control :as c] [generator :as gen] [independent :as independent] [nemesis :as nemesis] [tests :as tests]] [jepsen.checker.timeline :as timeline] - [jepsen.hstream.checker :as local-checker] - [jepsen.hstream.client :refer :all] - [jepsen.hstream.common :as common] - [jepsen.hstream.husky :refer :all] - [jepsen.hstream.mvar :refer :all] - [jepsen.hstream.utils :refer :all] - [jepsen.hstream.nemesis :as local-nemesis] - [jepsen.hstream.net :as net+]) - (:import [jepsen.hstream.common Default-Client])) + [jepsen.hstream.legacy.checker :as local-checker] + [jepsen.hstream.legacy.client :refer :all] + [jepsen.hstream.legacy.common :as common] + [jepsen.hstream.legacy.husky :refer :all] + [jepsen.hstream.common.mvar :refer :all] + [jepsen.hstream.common.utils :refer :all] + [jepsen.hstream.legacy.nemesis :as local-nemesis] + [jepsen.hstream.common.net :as net+]) + (:import [jepsen.hstream.legacy.common Default-Client])) ;;;;;;;;;; Global Variables ;;;;;;;;;; @@ -56,10 +55,10 @@ tests/noop-test opts {:pure-generators true, - :name "HStream", + :name "legacy-list-append", :net net+/iptables+, :plot local-nemesis/plot-spec, - :db (common/db-with-streams-initialized "0.16.0" opts test-streams), + :db (common/db-with-streams-initialized "0.19.0" opts test-streams), :client (common/Default-Client. opts subscription-results subscription-ack-timeout),