├── .gitignore ├── .travis.yml ├── HTTP.md ├── README.md ├── doc ├── intro.md ├── ns-dep-graph.png └── rackham.jpg ├── project.clj ├── src └── skuld │ ├── Bytes.java │ ├── aae.clj │ ├── admin.clj │ ├── bin.clj │ ├── claim.clj │ ├── client.clj │ ├── clock_sync.clj │ ├── core.clj │ ├── curator.clj │ ├── db.clj │ ├── db │ └── level.clj │ ├── flake.clj │ ├── http.clj │ ├── logging.clj │ ├── net.clj │ ├── node.clj │ ├── politics.clj │ ├── queue.clj │ ├── scanner.clj │ ├── task.clj │ ├── util.clj │ └── vnode.clj └── test └── skuld ├── claim_test.clj ├── core_test.clj ├── db └── level_test.clj ├── flake_test.clj ├── leader_test.clj ├── net_test.clj ├── node_test.clj ├── queue_test.clj ├── stress_test.clj ├── task_test.clj ├── util_test.clj └── zk_test.clj /.gitignore: -------------------------------------------------------------------------------- 1 | .*.swp 2 | /target 3 | /lib 4 | /classes 5 | /checkouts 6 | pom.xml 7 | pom.xml.asc 8 | *.jar 9 | *.class 10 | .lein-deps-sum 11 | .lein-failures 12 | .lein-plugins 13 | .lein-repl-history 14 | *.level 15 | *.un~ 16 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: clojure 2 | lein: lein2 3 | jdk: 4 | - openjdk7 5 | - oraclejdk7 6 | 7 | before_script: 8 | - export JVM_OPTS="-Xmx2g" 9 | 10 | branches: 11 | except: 12 | - gh-pages 13 | -------------------------------------------------------------------------------- /HTTP.md: -------------------------------------------------------------------------------- 1 | # HTTP Interface 2 | 3 | Skuld exposes an HTTP interface on a per-node basis. The available endpoints 4 | are outlined below. 5 | 6 | ## Queue 7 | 8 | ### Count queues 9 | 10 | Where `r` is the read concern. 11 | 12 | **Example request** 13 | ```http 14 | GET /queue/count?r=3 HTTP/1.1 15 | Host: 127.0.0.1:13100 16 | Content-Type: application/json 17 | ``` 18 | 19 | **Example response** 20 | ```http 21 | HTTP/1.1 200 OK 22 | Content-Type: application/json 23 | 24 | { 25 | "count": 13, 26 | } 27 | ``` 28 | 29 | ## Tasks 30 | 31 | ### Claim a task 32 | 33 | Where `dt` is the claim expiry delta from now in milliseconds. 34 | 35 | **Example request** 36 | ```http 37 | POST /tasks/claim HTTP/1.1 38 | Host: 127.0.0.1:13100 39 | Content-Type: application/json 40 | 41 | { 42 | "dt": 300 43 | } 44 | ``` 45 | 46 | **Example response** 47 | ```http 48 | HTTP/1.1 200 OK 49 | Content-Type: application/json 50 | 51 | { 52 | "task": { 53 | "claims": [ 54 | { 55 | "completed": null, 56 | "end": 1387126824854, 57 | "start": 1387126814854 58 | } 59 | ], 60 | "id": "AAABQvcd+CGAAAABrYuJIPwyloY=", 61 | "data": "foobar" 62 | } 63 | } 64 | ``` 65 | 66 | ### Complete a task 67 | 68 | Where `w` is the write concern and `cid` is the claim id, i.e. 1-indexed 69 | position of the claim in the `claims` array. 70 | 71 | **Example request** 72 | ```http 73 | POST /tasks/complete/AAABQvcd+CGAAAABrYuJIPwyloY=?w=3 HTTP/1.1 74 | Host: 127.0.0.1:13100 75 | Content-Type: application/json 76 | 77 | { 78 | "cid": 1 79 | } 80 | ``` 81 | 82 | **Example response** 83 | ```http 84 | HTTP/1.1 200 OK 85 | Content-Type: application/json 86 | 87 | { 88 | "w": 3 89 | } 90 | ``` 91 | 92 | ### Count tasks 93 | 94 | **Example request** 95 | ```http 96 | GET /tasks/count HTTP/1.1 97 | Host: 127.0.0.1:13100 98 | Content-Type: application/json 99 | ``` 100 | 101 | **Example response** 102 | ```http 103 | HTTP/1.1 200 OK 104 | Content-Type: application/json 105 | 106 | { 107 | "count": 13, 108 | "partitions": { 109 | "skuld_7": 2, 110 | "skuld_6": 0, 111 | "skuld_5": 1, 112 | "skuld_4": 0, 113 | "skuld_3": 0, 114 | "skuld_2": 0, 115 | "skuld_1": 6, 116 | "skuld_0": 4 117 | } 118 | } 119 | ``` 120 | 121 | ### Enqueue a task 122 | 123 | Where `w` is the write concern and `task` is the task-related data object. 124 | 125 | **Example request** 126 | ```http 127 | POST /tasks/enqueue HTTP/1.1 128 | Host: 127.0.0.1:13100 129 | Content-Type: application/json 130 | 131 | { 132 | "task": { 133 | "data": "foobar" 134 | }, 135 | "w": 3 136 | } 137 | ``` 138 | 139 | **Example response** 140 | ```http 141 | HTTP/1.1 200 OK 142 | Content-Type: application/json 143 | 144 | { 145 | "id": "AAABQvcd+CGAAAABrYuJIPwyloY=", 146 | "n": 3 147 | } 148 | ``` 149 | 150 | ### List tasks 151 | 152 | **Example request** 153 | ```http 154 | GET /tasks/list HTTP/1.1 155 | Host: 127.0.0.1:13100 156 | Content-Type: application/json 157 | ``` 158 | 159 | **Example response** 160 | ```http 161 | HTTP/1.1 200 OK 162 | Content-Type: application/json 163 | 164 | { 165 | "tasks": [ 166 | { 167 | "claims": [], 168 | "id": "AAABQvcd+CGAAAABrYuJIPwyloY=", 169 | "data": "foobar" 170 | }, 171 | ... 172 | ] 173 | } 174 | ``` 175 | 176 | ### Get a task 177 | 178 | Where `r` is read concern. 179 | 180 | **Example request** 181 | ```http 182 | GET /tasks/AAABQvcd+CGAAAABrYuJIPwyloY=?r=3 HTTP/1.1 183 | Host: 127.0.0.1:13100 184 | Content-Type: application/json 185 | ``` 186 | 187 | **Example response** 188 | ```http 189 | HTTP/1.1 200 OK 190 | Content-Type: application/json 191 | 192 | { 193 | "task": { 194 | "claims": [], 195 | "id": "AAABQvcd+CGAAAABrYuJIPwyloY=", 196 | "data": "foobar" 197 | }, 198 | "n": 3 199 | } 200 | ``` 201 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | 2 | 3 | [![Build Status](https://travis-ci.org/Factual/skuld.png)](https://travis-ci.org/Factual/skuld) 4 | 5 | # Skuld 6 | 7 | Skuld is (or aims to become) a hybrid AP/CP distributed task queue, targeting 8 | linear scaling with nodes, robustness to N/2-1 failures, extremely high 9 | availability for enqueues, guaranteed at-least-once delivery, approximate 10 | priority+FIFO ordering, and reasonable bounds on task execution mutexing. Each 11 | run of a task can log status updates to Skuld, checkpointing their progress and 12 | allowing users to check how far along their tasks have gone. 13 | 14 | Skuld combines techniques from many distributed systems: [Dynamo][dynamo]-style 15 | consistent hashing, quorums over vnodes, and anti-entropy provide the 16 | highly-available basis for Skuld's immutable dataset, including enqueues, 17 | updates, and completions. All AP operations are represented by [Convergent 18 | Replicated Data Types (CRDTs)][crdts], ensuring convergence in the absence of 19 | strong consistency. CP operations (e.g. claims) are supported by a leader 20 | election/quorum protocol similar to [Viewstamped Replication][viewstamped] or 21 | [Raft][raft], supported by additional constraints on handoff transitions between 22 | disparate cohorts. 23 | 24 | Skuld relies on semi-synchronized clocks: the network, GC pauses, and clock 25 | skew may introduce message skew of no more than eta seconds. Skuld is intended 26 | for systems where eta is on the order of minutes to hours, and task execution 27 | failures may delay retries by at least eta. Failure to maintain this bound 28 | could result in overlapping task execution or other anomalies, though outright 29 | data loss is unlikely. 30 | 31 | ## Current status 32 | 33 | Skuld is experimental at best. Many subsystems are slow, buggy, incorrect, or 34 | missing. We're open-sourcing Skuld at this juncture because many people have 35 | expressed interest in contributing code. *Do not use Skuld in production.* It 36 | *will* explode and destroy your data. 37 | 38 | Right now, Skuld can: 39 | 40 | - Create and destroy clusters. 41 | - Balance vnodes 42 | - Heal failures via [Merkle-tree][merkle]-backed active anti-entropy 43 | - Elect leaders when a majority component is available 44 | - Enqueue tasks 45 | - Claim tasks 46 | - Mark tasks as completed 47 | 48 | Major drawbacks include but are not limited to: 49 | 50 | - Losing all data on nodes when restarted 51 | - Performing an O(tasks) scan over all tasks for every claim. 52 | - Merkle trees are O(n log n) in size, including a leaf node for every task 53 | - All sorts of poorly chosen, hardcoded timeouts and parameters 54 | - Task listing holds all tasks in memory 55 | - No concept of streaming requests 56 | - Clock sync detector doesn't do anything when it finds nodes are misaligned 57 | - Incoherent logging 58 | - All cluster transitions rely on [ZooKeeper][zk] (via [Apache Helix][helix]) 59 | - Probably a whole bunch of bugs I don't know about yet 60 | 61 | And missing subsystems include: 62 | 63 | - HTTP interface 64 | - Protobuf/TCP interface 65 | - Named queues (right now all tasks are in a single FIFO queue) 66 | - Priorities 67 | - Task updates 68 | - Task dependencies 69 | - Garbage collection 70 | 71 | ## Concepts 72 | 73 | Skuld works with queues, which are named by strings. Queues comprise a set of 74 | tasks. Tasks are identified by a unique, k-ordered string id. A task has some 75 | immutable payload, which is arbitrary bytes; a priority, which is a signed 76 | 32-bit integer (higher numbers dequeued first), and a list of dependencies, 77 | which are the keys of other tasks. A task is only eligible for claim when all 78 | its dependencies are complete. Tasks can be claimed by a worker at least once, 79 | and Skuld makes every effort to guarantee claims do not temporally or causally 80 | overlap; providing monotonic claim sequence identifiers to clients to assist in 81 | this. Claims last until some time t; workers must regularly renew their claim 82 | on the task if that time approaches, or risk the claim expiring. 83 | 84 | As work occurs on a task, Skuld can add immutable updates for that task, which 85 | form a task log of the task's progress. Tasks can be marked as complete when 86 | they are finished and should not be run again. Updates and the complete message 87 | include a mandatory sequence number, which provides a total order for the 88 | history of the task and ensures only contiguous histories are visible. At any 89 | point, one can ask about the current state of a task, receiving the task 90 | payload, claim state, priority, etc; optionally including the sequence number 91 | of the most recent update seen, to ensure monotonicity. 92 | 93 | If a claim expires and the task is not flagged as complete, it is eligible for 94 | claim again, and can be retried under a new claim ID. Workers can request the 95 | log for a task's earlier claims–for example, to recover from a checkpoint. 96 | 97 | When every task connected by dependency pointers is complete, those tasks may 98 | be eligible for garbage collection, flushed to long-term storage, or retained 99 | indefinitely. 100 | 101 | ## Operations 102 | 103 | ### Topology 104 | 105 | The ID space is hashed and sharded into partitions. Partitions are claimed by 106 | cohorts of N nodes. One node is elected the leader for that partition, and all 107 | others are followers. The leader is responsible for handing out claims and 108 | recovering failed tasks. The followers receive claim messages from the leader, 109 | so that all N nodes maintain a somewhat-in-sync set of the currently claimed 110 | tasks. All other messages (enqueue, update, complete, etc) are eventually 111 | consistent, and are gossiped continuously between all N nodes via hash trees. 112 | 113 | ### Enqueue 114 | 115 | To enqueue a task, generate a 160-bit [Flake ID][flake] and assign it to the 116 | task. Compute the partition for that ID. Broadcast the task to all nodes in that 117 | partition. Wait for a quorum of nodes to respond by default; though the client 118 | could request a smaller or larger W val. Once a W nodes have acked the task, 119 | respond to the client with the number of acking nodes. By giving the generated 120 | ID back to the client, the client can retry its write idempotently. 121 | 122 | ### Claim 123 | 124 | Try to find a leader on the local node which has a task to claim. If there are 125 | none, proxy the claim request to each other node in a randomized order, and 126 | look for local leaders there. The leader selects a task which is a.) in the 127 | correct queue, b.) unclaimed, c.) uncompleted, d.) has the highest priority, 128 | e.) is the oldest. The leader examines the claim set for that task, and selects 129 | a new claim ID. Primary broadcasts a message to all nodes in cohort requesting 130 | that they mark that task as claimed until time t. Once a quorum have 131 | acknowledged that operation, the leader considers the task claimed and returns 132 | the task, and claim ID, to the coordinator. 133 | 134 | ### Renew 135 | 136 | Claims are only valid for a certain amount of time. The client must regularly 137 | renew its claim to prevent it from expiring. To renew a claim, the node 138 | broadcasts the renew message to all peers for that partition. If a quorum 139 | respond, the renew is successful. Renew is idempotent. 140 | 141 | ### Update 142 | 143 | The client must maintain an internal sequence number, beginning at zero, and 144 | incrementing by one for each update. To send an update for a task, the 145 | client sends its update, with the claim number and sequence number, to any 146 | node. That node broadcasts the update to all nodes for that task's partition. 147 | When W nodes have acknowledged the update, the coordinator returns success to 148 | the client. The client should retry updates which fail for as long as 149 | reasonably possible. Update is idempotent. 150 | 151 | ### Complete 152 | 153 | To mark the message as complete, send a complete message to any node with a 154 | sequence number, just like an update. The coordinator broadcasts that update to 155 | all nodes for the task's partition. Those nodes immediately remove the task from 156 | their claim set, write the complete message to disk, and return an 157 | acknowledgement. When W nodes have acknowledged the update, the coordinator 158 | returns success to the client. 159 | 160 | ### Clocksync 161 | 162 | All nodes regularly exchange heartbeat messages with their current clock. If 163 | any differ by more than a few seconds, the furthest ahead kills itself. 164 | 165 | ### Active anti-entropy 166 | 167 | All nodes regularly gossip Merkle trees of their immutable dataset to any peers 168 | in their partition, and replicate missing blocks. All followers regularly 169 | exchange Merkle trees of their claim set with the current leader, and copy the 170 | leader's data where different. 171 | 172 | ### Leaders 173 | 174 | In becoming a leader, we need to ensure that: 175 | 176 | 1. Leaders are logically sequential 177 | 2. Each leader's claim set is a superset of the previous leader 178 | 179 | We have: a target cohort of nodes for the new epoch, provided by Helix. 180 | Some previous cohort of nodes belonging to the old epoch, tracked by ZooKeeper. 181 | 182 | To become a leader, one must successfully: 183 | 184 | 1. Read the previous epoch+cohort from ZooKeeper 185 | 2. (optimization) Ensure that the previous epoch is strictly less than the 186 | epoch this node is going to be the leader for. 187 | 3. Broadcast a claim message to the new cohort, union the old cohort 188 | 4. Receive votes from a majority of the nodes in the old cohort 189 | 5. Receive votes from a majority of the nodes in the new cohort 190 | - At this juncture, neither the new nor the old cohort can commit claims 191 | for our target epoch or lower, making the set of claims made in older epochs 192 | immutable. If we are beat by another node using a newer epoch, it will have 193 | recovered a superset of those claims; we'll fail to CAS in step 8, and no 194 | claims will be lost. 195 | 6. Obtain all claims from a majority of the old cohort, and union them in to 196 | our local claim set. 197 | - This ensures that we are aware of all claims made prior to our epoch. 198 | 7. Broadcast our local claim set to a majority of the new cohort. 199 | - This ensures that any *future* epoch will correctly recover our claim 200 | set. 6 + 7 provide a continuous history of claims, by induction. 201 | 8. CAS our new epoch and cohort into ZooKeeper, ensuring that nobody else 202 | beat us to it. 203 | 204 | If any stage fails, delay randomly and retry. 205 | 206 | A note on zombies: 207 | 208 | Zombies are nodes which are ready to give up ownership of a vnode but cannot, 209 | because they may still be required to hand off their claim set. After step 8, 210 | we inform all zombies which are not a part of our new cohort that it is safe 211 | to drop their claim set." 212 | 213 | ### Recovery 214 | 215 | A vnode allows a task to be claimed, in the event of failure, once the claim's 216 | time has expired relative to the local clock *including* a buffer larger than 217 | the maximum clock skew, message propagation delay, GC time, etc: on the order 218 | of minutes to hours. Shorter buffers increase the probability that claims will 219 | overlap. 220 | 221 | ## Getting started 222 | 223 | You'll need a ZooKeeper cluster, and lein 2. On OS X you will also need Java 1.7. To create a cluster, run: 224 | 225 | ``` 226 | lein run cluster create skuld -z some.zk.node:2181 --partitions 8 --replicas 3 227 | ``` 228 | 229 | And add a few nodes. Yeah, this is slow. I'm sorry. 230 | 231 | ``` 232 | lein run cluster add --host "127.0.0.1" --port 13000 233 | lein run cluster add --host "127.0.0.1" --port 13001 234 | lein run cluster add --host "127.0.0.1" --port 13002 235 | ``` 236 | 237 | Open a few shells, and fire em up! 238 | 239 | ``` 240 | lein run start --host 127.0.0.1 --port 13000 241 | lein run start --host 127.0.0.1 --port 13001 242 | lein run start --host 127.0.0.1 --port 13002 243 | ``` 244 | 245 | Open a repl (`lein repl`) 246 | 247 | ```clj 248 | ; Suck in the library 249 | skuld.bin=> (use 'skuld.client) 250 | 251 | ; And set up the request ID generator 252 | skuld.bin=> (skuld.flake/init!) 253 | 254 | ; OK, let's define a client with a few hosts to talk to: 255 | skuld.bin=> (def c (client [{:host "127.0.0.1" :port 13000} {:host "127.0.0.1" :port 13001}])) 256 | 257 | ; There are no tasks in the system now: 258 | skuld.bin=> (count-tasks c) 259 | 0 260 | 261 | ; We can enqueue a task with some payload 262 | skuld.bin=> (enqueue! c {:queue "general" :data "sup?"}) 263 | # 264 | 265 | ; Now there's 1 task 266 | skuld.bin=> (count-tasks c) 267 | 1 268 | 269 | ; Which we can show like so: 270 | skuld.bin=> (pprint (list-tasks c)) 271 | ({:data "sup?", 272 | :id #, 273 | :claims []}) 274 | nil 275 | 276 | ; And now we can claim a task for 10 seconds 277 | skuld.bin=> (def t (claim! c "general" 10000)) 278 | #'skuld.bin/t 279 | skuld.bin=> (pprint t) 280 | {:data "sup?", 281 | :id #, 282 | :claims [{:start 1377913791803, :end 1377913801803, :completed nil}]} 283 | nil 284 | 285 | ; We can't claim any other tasks during this time 286 | skuld.bin=> (claim! c "general" 10000) 287 | nil 288 | 289 | ; But if we wait long enough, Skuld will decide the claim has expired. Once a 290 | ; quorum of nodes agree that the claim is outdated, we can re-claim the same 291 | ; task: 292 | skuld.bin=> (Thread/sleep 60000) (def t2 (claim! c "general" 10000)) 293 | nil 294 | #'skuld.bin/t2 295 | skuld.bin=> t2 296 | {:data "sup?", :id #, :claims [{:start 1377913791803, :end 1377913801803, :completed nil} {:start 1377913903904, :end 1377913913904, :completed nil}]} 297 | 298 | ; Now let's mark that task, in claim 1, as being complete: 299 | skuld.bin=> (complete! c (:id t2) 1) 300 | 2 301 | 302 | ; If we ask Skuld what happened to the task, it'll show both claims, one 303 | ; completed: 304 | skuld.bin=> (pprint (get-task c (:id t2))) 305 | {:data "sup?", 306 | :id #, 307 | :claims 308 | [{:start 1377913791803, :end 1377913801803, :completed nil} 309 | {:start 1377913903904, :end 1377913913904, :completed 1377913942979}]} 310 | nil 311 | ``` 312 | 313 | Below is the namespace dependency graph for Skuld. 314 | 315 | 316 | 317 | 318 | ## How can I help? 319 | 320 | Hop on #skuld on Freenode, or hit me up at 321 | [kingsbury@factual.com](mailto:kingsbury@factual.com); I'm happy to answer 322 | questions and help you get started working on improvements. You'll want a basic 323 | command of Clojure to start, but distributed systems expertise isn't required; 324 | there are lots of Skuld internals from disk persistence to data structures to 325 | HTTP servers that need your help! 326 | 327 | ## License 328 | 329 | Copyright © 2013 Factual, Inc 330 | 331 | Distributed under the Eclipse Public License, the same as Clojure. 332 | 333 | [dynamo]: http://www.allthingsdistributed.com/2007/10/amazons_dynamo.html 334 | [crdts]: http://pagesperso-systeme.lip6.fr/Marc.Shapiro/papers/RR-6956.pdf 335 | [raft]: https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf 336 | [viewstamped]: http://www.cs.princeton.edu/courses/archive/fall09/cos518/papers/viewstamped.pdf 337 | [merkle]: http://en.wikipedia.org/wiki/Merkle_tree 338 | [flake]: http://boundary.com/blog/2012/01/12/flake-a-decentralized-k-ordered-unique-id-generator-in-erlang/ 339 | [zk]: http://zookeeper.apache.org/ 340 | [helix]: http://helix.incubator.apache.org/ 341 | 342 | -------------------------------------------------------------------------------- /doc/intro.md: -------------------------------------------------------------------------------- 1 | # Introduction to skuld 2 | 3 | TODO: write [great documentation](http://jacobian.org/writing/great-documentation/what-to-write/) 4 | -------------------------------------------------------------------------------- /doc/ns-dep-graph.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Factual/skuld/79599f9b13aee35c680183d6bf9e2fcbfde1d7c7/doc/ns-dep-graph.png -------------------------------------------------------------------------------- /doc/rackham.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Factual/skuld/79599f9b13aee35c680183d6bf9e2fcbfde1d7c7/doc/rackham.jpg -------------------------------------------------------------------------------- /project.clj: -------------------------------------------------------------------------------- 1 | (defproject skuld "0.1.0-SNAPSHOT" 2 | :description "A hybrid AP/CP distributed task queue." 3 | :url "http://github.com/factual/skuld" 4 | :main skuld.bin 5 | :java-source-paths ["src/skuld/"] 6 | :license {:name "Eclipse Public License" 7 | :url "http://www.eclipse.org/legal/epl-v10.html"} 8 | :repositories [["jboss-fs-public" "http://repository.jboss.org/nexus/content/groups/fs-public/"]] 9 | :exclusions [[log4j] 10 | [org.slf4j/slf4j-log4j12]] 11 | :dependencies [[org.clojure/clojure "1.5.1"] 12 | [org.clojure/tools.cli "0.2.2"] 13 | [org.clojure/tools.logging "0.2.3"] 14 | [ch.qos.logback/logback-classic "1.0.13"] 15 | [org.slf4j/jcl-over-slf4j "1.7.5"] 16 | [org.slf4j/log4j-over-slf4j "1.7.5"] 17 | [factual/clj-helix "0.1.0"] 18 | [io.netty/netty "4.0.0.Alpha8"] 19 | [com.google.protobuf/protobuf-java "2.5.0"] 20 | [org.clojure/data.codec "0.1.0"] 21 | [potemkin "0.3.0"] 22 | [merkle "0.1.0"] 23 | [org.apache.curator/curator-recipes "2.0.1-incubating"] 24 | [factual/clj-leveldb "0.1.0"] 25 | [clout "1.1.0"] 26 | [ring/ring-jetty-adapter "1.2.1"] 27 | [cheshire "5.2.0"] 28 | [ring/ring-json "0.2.0"] 29 | [ring/ring-codec "1.0.0"] 30 | [org.clojure/data.fressian "0.2.0"]] 31 | :jvm-opts ^:replace ["-server"] 32 | :profiles {:dev {:dependencies [[criterium "0.4.1"] 33 | [com.google.guava/guava "14.0.1"] 34 | [org.apache.curator/curator-test "2.0.1-incubating"] 35 | [clj-http "0.7.7"] 36 | [hexdump "0.1.0"]]}} 37 | :test-selectors {:default (fn [x] (not (or (:integration x) 38 | (:time x) 39 | (:bench x)))) 40 | :focus :focus 41 | :bench :bench} 42 | :global-vars {*warn-on-reflection* true}) 43 | -------------------------------------------------------------------------------- /src/skuld/Bytes.java: -------------------------------------------------------------------------------- 1 | package com.aphyr.skuld; 2 | 3 | import java.util.Arrays; 4 | import com.google.common.primitives.UnsignedBytes; 5 | 6 | // Immutable (unenforced) byte arrays with sane value semantics. 7 | public class Bytes implements Comparable { 8 | public final byte[] bytes; 9 | public final int hashCode; 10 | 11 | public Bytes(final byte[] bytes) { 12 | if (bytes == null) { 13 | throw new IllegalArgumentException( 14 | "bytes should have been a byte[], was null"); 15 | } 16 | 17 | this.bytes = bytes; 18 | this.hashCode = Arrays.hashCode(bytes); 19 | } 20 | 21 | @Override 22 | public int hashCode() { 23 | return this.hashCode; 24 | } 25 | 26 | @Override 27 | public boolean equals(final Object o) { 28 | if (! (o instanceof Bytes)) { 29 | return false; 30 | } 31 | 32 | final Bytes b = (Bytes) o; 33 | return Arrays.equals(bytes, b.bytes); 34 | } 35 | 36 | @Override 37 | public int compareTo(final Object o) { 38 | if (! (o instanceof Bytes)) { 39 | throw new IllegalArgumentException("Can't compare Bytes to" + 40 | o.toString()); 41 | } 42 | 43 | return UnsignedBytes.lexicographicalComparator(). 44 | compare(bytes, ((Bytes) o).bytes); 45 | } 46 | 47 | @Override 48 | public String toString() { 49 | final StringBuilder sb = new StringBuilder(); 50 | for(byte b: bytes) 51 | sb.append(String.format("%02x", b&0xff)); 52 | return sb.toString(); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/skuld/aae.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.aae 2 | "Active-anti-entropy service. This watches over local partitions and 3 | continually synchronizes their data with their peers." 4 | (:use clojure.tools.logging) 5 | (:require [skuld.vnode :as vnode] 6 | [skuld.net :as net] 7 | [merkle.kv.linear :as merkle] 8 | [clj-helix.route :as route])) 9 | 10 | (defn merkle-tree 11 | "Computes a merkle-tree of a vnode." 12 | [vnode] 13 | (merkle/tree (vnode/tasks vnode) 14 | :id 15 | identity)) 16 | 17 | (defn merge-updates! 18 | "Merges :updates from a message into the given vnode. Returns true if 19 | messages merged, nil otherwise." 20 | [vnode message] 21 | (when message 22 | (dorun (map (partial vnode/merge-task! vnode) (:updates message))) 23 | true)) 24 | 25 | (defn vnode 26 | "Gets the local vnode for an AAE message." 27 | [vnodes msg] 28 | (let [part (:partition msg)] 29 | (or (get @vnodes part) 30 | (throw (RuntimeException. (str "no such vnode " part)))))) 31 | 32 | (defn handle-tree 33 | "Returns the local {:tree ...} for a requested vnode." 34 | [vnodes msg] 35 | (let [vnode (vnode vnodes msg)] 36 | {:tree (merkle/node->map (merkle-tree vnode))})) 37 | 38 | (defn handle-diff 39 | "Given a message with a requester's tree, computes diff and returns {:updates 40 | ...} for requester." 41 | [vnodes msg] 42 | (let [vnode (vnode vnodes msg) 43 | ; Diff against our local collection. 44 | remote-tree (merkle/map->node (:tree msg)) 45 | diffs (merkle/diff (vnode/tasks vnode) 46 | (merkle-tree vnode) 47 | remote-tree 48 | :id)] 49 | {:updates diffs})) 50 | 51 | (defn handle-updates! 52 | "Given {:updates ...} from requester, applies them to local vnode and returns 53 | {}." 54 | [vnodes msg] 55 | (or 56 | (and (merge-updates! (vnode vnodes msg) msg) 57 | {}) 58 | (throw (RuntimeException. "expected a :updates key.")))) 59 | 60 | (defn handler 61 | "Returns a handler function that checks AAE messages against an atom wrapping 62 | a map of partitions to vnodes." 63 | [vnodes] 64 | (fn [msg] 65 | (case (:type msg) 66 | :aae-tree (handle-tree vnodes msg) 67 | :aae-diff (handle-diff vnodes msg) 68 | :aae-updates (handle-updates! vnodes msg) 69 | nil))) 70 | 71 | (defn sync-from! 72 | "Fills in the local vnode with tasks from the remote peer. If the remote peer 73 | is immutable, this means the local node will have a complete copy of the 74 | remote's data. Returns true if vnode copied; false otherwise." 75 | [vnode peer] 76 | (let [tree (-> vnode 77 | merkle-tree) 78 | tree (merkle/node->map tree) 79 | [res] (net/sync-req! (:net vnode) [peer] {} 80 | {:type :aae-diff 81 | :partition (:partition vnode) 82 | :tree tree})] 83 | (merge-updates! vnode res))) 84 | 85 | (defn sync-to! 86 | "Pushes data from a local vnode to the remote peer. If the local vnode is 87 | immutable, this means the remote peer will have a complete copy of the 88 | vnode's data. Returns true if vnode copied; false otherwise." 89 | [vnode peer] 90 | ; Get remote tree 91 | (when-let [response (-> vnode 92 | :net 93 | (net/sync-req! 94 | [peer] 95 | {} 96 | {:type :aae-tree 97 | :partition (:partition vnode)}) 98 | first)] 99 | 100 | ; Compute diffs 101 | (let [remote-tree (merkle/map->node (:tree response)) 102 | updates (merkle/diff (vnode/tasks vnode) 103 | (merkle-tree vnode) 104 | remote-tree 105 | :id)] 106 | 107 | ; Send updates 108 | (when-not (:error (net/sync-req! (:net vnode) 109 | [peer] 110 | {} 111 | {:type :aae-updates 112 | :partition (:partition vnode) 113 | :updates updates})) 114 | true)))) 115 | 116 | (defn sync-vnode! 117 | "Given a vnode, hashes it and syncs with peers." 118 | [net router vnode] 119 | (let [self (vnode/net-id vnode) 120 | peers (set (vnode/peers vnode))] 121 | (dorun (map (partial sync-from! vnode) (disj peers self))))) 122 | 123 | (defn initiator 124 | "Periodically initiates sync with peers." 125 | [net router vnodes] 126 | (let [running (promise)] 127 | (future 128 | (when (deref running 10000 true) 129 | (loop [] 130 | (try 131 | (->> vnodes 132 | deref 133 | vals 134 | (map (partial sync-vnode! net router)) 135 | dorun) 136 | (catch Throwable t 137 | (warn t "aae caught"))) 138 | 139 | (when (deref running 10000 true) 140 | (recur))))) 141 | running)) 142 | 143 | (defn service 144 | "Creates a new AAE service." 145 | [net router vnodes] 146 | ; Register handler 147 | (net/add-handler! net (handler vnodes)) 148 | (initiator net router vnodes)) 149 | 150 | (defn shutdown! 151 | "Stops an AAE service." 152 | [aae] 153 | (deliver aae false)) 154 | -------------------------------------------------------------------------------- /src/skuld/admin.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.admin 2 | "Administrative cluster-management tasks." 3 | (:require [clj-helix.admin :as helix] 4 | [skuld.node :as node])) 5 | 6 | (defn admin 7 | "Create an administrator tool. Options: 8 | 9 | :cluster the name of the skuld cluster (default \"skuld\") 10 | :zookeeper a zookeeper connect string (\"localhost:2181\") 11 | :partitions number of partitions to shard into (1) 12 | :replicas number of replicas to maintain (3) 13 | :max-partitions-per-node (:partitions)" 14 | [opts] 15 | {:helix (helix/helix-admin (get opts :zookeeper "localhost:2181")) 16 | :cluster (get opts :cluster "skuld") 17 | :partitions (get opts :partitions 1) 18 | :replicas (get opts :replicas 3) 19 | :max-partitions-per-node (get opts :max-partitions-per-node 20 | (get opts :partitions 1))}) 21 | 22 | (defn shutdown! 23 | "Shuts down an admin tool." 24 | [admin] 25 | (.close (:helix admin))) 26 | 27 | (defn destroy-cluster! 28 | [admin] 29 | (helix/drop-cluster (:helix admin) 30 | (:cluster admin))) 31 | 32 | (defn create-cluster! 33 | [admin] 34 | ; Create cluster itself 35 | (helix/add-cluster (:helix admin) 36 | (:cluster admin)) 37 | 38 | ; Set FSM 39 | (helix/add-fsm-definition (:helix admin) 40 | (:cluster admin) 41 | node/fsm-def) 42 | 43 | ; Create resource 44 | (helix/add-resource (:helix admin) 45 | (:cluster admin) 46 | {:resource :skuld 47 | :partitions (:partitions admin) 48 | :max-partitions-per-node (:max-partitions-per-node admin) 49 | :replicas (:replicas admin) 50 | :state-model (:name node/fsm-def)})) 51 | 52 | (defn add-node! 53 | "Adds an node to the cluster." 54 | [admin node] 55 | (helix/add-instance (:helix admin) (:cluster admin) 56 | (select-keys node [:host :port]))) 57 | -------------------------------------------------------------------------------- /src/skuld/bin.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.bin 2 | (:use [clj-helix.logging :only [mute]] 3 | [clojure.tools.cli :only [cli]] 4 | clojure.tools.logging) 5 | (:require [skuld.admin :as admin] 6 | [skuld.node :as node] 7 | [skuld.flake :as flake] 8 | [skuld.http]) 9 | (:import (sun.misc Signal SignalHandler)) 10 | (:gen-class)) 11 | 12 | (defn parse-int 13 | "Parse an integer." 14 | [^String s] 15 | (Integer. s)) 16 | 17 | (defmacro signal 18 | "Adds a signal handler." 19 | [signal & body] 20 | `(when-not (.contains (System/getProperty "os.name") "Windows") 21 | (Signal/handle 22 | (Signal. (name ~signal)) 23 | (reify SignalHandler 24 | (handle [this# sig#] 25 | ~@body))))) 26 | 27 | (def admin-spec 28 | [["-z" "--zookeeper" "Zookeeper connection string" 29 | :default "localhost:2181"] 30 | ["-c" "--cluster" "Cluster name" 31 | :default "skuld"] 32 | ["-partitions" "--partitions" "Number of partitions" 33 | :default 1 :parse-fn parse-int] 34 | ["-r" "--replicas" "Number of replicas" 35 | :default 3 :parse-fn parse-int]]) 36 | 37 | (def node-spec 38 | [["-z" "--zookeeper" "Zookeeper connection string" 39 | :default "localhost:2181"] 40 | ["-p" "--port" "Port" :default "13000" :parse-fn parse-int] 41 | ["-h" "--host" "Hostname" :default "127.0.0.1"]]) 42 | 43 | ; Cluster configuration 44 | (defn cluster-create [& args] 45 | (let [[opts _ _] (apply cli args admin-spec)] 46 | (admin/create-cluster! (admin/admin opts)))) 47 | 48 | (defn cluster-destroy [& args] 49 | (let [[opts _ _] (apply cli args admin-spec)] 50 | (admin/destroy-cluster! (admin/admin opts)))) 51 | 52 | (defn cluster-add [& args] 53 | (let [[opts _ _] (apply cli args (concat node-spec admin-spec))] 54 | (admin/add-node! (admin/admin opts) 55 | (select-keys opts [:host :port])))) 56 | 57 | (defn cluster 58 | [cmd & args] 59 | (apply (case cmd 60 | "create" cluster-create 61 | "add" cluster-add 62 | "destroy" cluster-destroy) 63 | args)) 64 | 65 | ; Node management 66 | (defn controller [& args] 67 | (let [[opts _ _] (apply cli args node-spec) 68 | controller (node/controller opts)] 69 | 70 | (.addShutdownHook (Runtime/getRuntime) 71 | (Thread. (bound-fn [] 72 | (node/shutdown! controller)))) 73 | 74 | (info "Controller started.") 75 | (debug controller) 76 | @(promise))) 77 | 78 | (defn start [& args] 79 | (flake/init!) 80 | (let [[opts _ _] (apply cli args node-spec) 81 | node (node/node opts)] 82 | 83 | (.addShutdownHook (Runtime/getRuntime) 84 | (Thread. (bound-fn [] 85 | (node/shutdown! node)))) 86 | 87 | (info :started node) 88 | @(promise))) 89 | 90 | (defn -main 91 | [cmd & args] 92 | (try 93 | (apply (case cmd 94 | "cluster" cluster 95 | "controller" controller 96 | "start" start) 97 | args) 98 | (catch Throwable t 99 | (.printStackTrace t) 100 | (System/exit 1)))) 101 | -------------------------------------------------------------------------------- /src/skuld/claim.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.claim 2 | (:refer-clojure :exclude [merge]) 3 | (:require [skuld.flake :as flake])) 4 | 5 | (defn claim 6 | "Generates a new claim valid for dt millis." 7 | [task-id epoch dt] 8 | (let [now (flake/linear-time)] 9 | {:epoch epoch 10 | :start now 11 | :end (+ now dt)})) 12 | 13 | (def clock-skew-limit 14 | "The number of milliseconds we allow clocks, nodes, messages, and networks to 15 | drift." 16 | 60000) 17 | 18 | (defn current? 19 | "Is a claim currently held?" 20 | [claim] 21 | (-> claim 22 | :end 23 | (+ clock-skew-limit) 24 | (< (flake/linear-time)))) 25 | 26 | (defn merge 27 | [c1 c2] 28 | (if (< (:epoch c1) (:epoch c2)) 29 | c2 30 | c1)) 31 | -------------------------------------------------------------------------------- /src/skuld/client.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.client 2 | (:require [skuld.net :as net])) 3 | 4 | (defn client 5 | "Constructs a client which talks to a set of peers." 6 | [peers] 7 | {:peers peers 8 | :net (doto (net/node {:server? false}) 9 | (net/start!))}) 10 | 11 | (defn shutdown! 12 | "Shuts down a client." 13 | [client] 14 | (net/shutdown! (:net client))) 15 | 16 | (defn peer 17 | "Select a peer for a client." 18 | [node] 19 | (rand-nth (:peers node))) 20 | 21 | (defn sync-req! 22 | [client opts msg] 23 | (let [res (-> client 24 | :net 25 | (net/sync-req! (list (peer client)) opts msg) 26 | first)] 27 | (when (nil? res) 28 | (throw (RuntimeException. "request timed out."))) 29 | (when-let [error (:error res)] 30 | (throw (RuntimeException. (str "server error: " error)))) 31 | res)) 32 | 33 | (defn wipe! [client] 34 | (sync-req! client {} {:type :wipe})) 35 | 36 | (defn enqueue! 37 | "Enqueue a single task. Returns a task ID." 38 | ([client task] 39 | (enqueue! client {} task)) 40 | ([client opts task] 41 | (:id (sync-req! client {} {:type :enqueue 42 | :w (get opts :w 1) 43 | :task task})))) 44 | 45 | (defn claim! 46 | "Claim a task for dt milliseconds. Returns a task." 47 | ([client queue dt] 48 | (claim! client {} queue dt)) 49 | ([client opts queue dt] 50 | (:task (sync-req! client opts {:type :claim 51 | :queue queue 52 | :dt dt})))) 53 | 54 | (defn complete! 55 | "Complete a task with the given task ID and claim ID." 56 | ([client task-id claim-id] 57 | (complete! client {} task-id claim-id)) 58 | ([client opts task-id claim-id] 59 | (:w (sync-req! client {} {:type :complete 60 | :w (:w opts) 61 | :task-id task-id 62 | :claim-id claim-id})))) 63 | 64 | (defn get-task 65 | "Gets a task by ID." 66 | ([client task-id] 67 | (get-task client {} task-id)) 68 | ([client opts task-id] 69 | (:task (sync-req! client {} {:type :get-task 70 | :r (:r opts) 71 | :id task-id})))) 72 | 73 | (defn count-tasks 74 | "Returns a count of how many tasks are in the cluster." 75 | [client] 76 | (:count (sync-req! client {} {:type :count-tasks}))) 77 | 78 | (defn count-queue 79 | "Returns a count of how many tasks are claimable." 80 | [client queue] 81 | (:count (sync-req! client {} {:type :count-queue 82 | :queue queue}))) 83 | 84 | (defn list-tasks 85 | "Returns a list of tasks." 86 | [client] 87 | (:tasks (sync-req! client {} {:type :list-tasks}))) 88 | -------------------------------------------------------------------------------- /src/skuld/clock_sync.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.clock-sync 2 | "Exchanges heartbeats with peers to ensure clocks are synchronized." 3 | (:use clojure.tools.logging) 4 | (:require [skuld.net :as net] 5 | [skuld.flake :as flake] 6 | [clj-helix.route :as route])) 7 | 8 | (defn service 9 | "Creates a new clock-sync service." 10 | [net router vnodes] 11 | ; Register handler 12 | (net/add-handler! net 13 | (fn [msg] 14 | (when (= (:type msg) :clock-sync) 15 | (let [delta (- (flake/linear-time) (:time msg))] 16 | (when (< 30000 delta) 17 | (warn "Clock skew with" 18 | (pr-str (:node msg)) 19 | "is" 20 | delta 21 | "milliseconds!"))) 22 | {}))) 23 | 24 | (let [running (promise)] 25 | ; Periodically emit heartbeats to peers 26 | (future 27 | (when (deref running 10000 true) 28 | (loop [] 29 | (try 30 | (->> vnodes 31 | deref 32 | keys 33 | (mapcat #(route/instances router :skuld % :peer)) 34 | set 35 | (map (fn [peer] 36 | (try 37 | (net/send! net peer {:type :clock-sync 38 | :node (select-keys net [:host :port]) 39 | :time (flake/linear-time)}) 40 | (catch io.netty.channel.ChannelException ex 41 | (warnf "clock-sync to %s: %s" peer ex)) 42 | (catch Throwable t 43 | (warn t "clock-sync to " peer))))) 44 | dorun) 45 | (catch Throwable t 46 | (warn t "clock-sync caught"))) 47 | 48 | (when (deref running 10000 true) 49 | (recur))))) 50 | 51 | running)) 52 | 53 | (defn shutdown! 54 | "Stop a clock-sync service." 55 | [clock-sync] 56 | (deliver clock-sync false)) 57 | -------------------------------------------------------------------------------- /src/skuld/core.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.core) 2 | -------------------------------------------------------------------------------- /src/skuld/curator.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.curator 2 | (:require 3 | [clojure.edn :as edn] 4 | [clojure.tools.logging :as log] 5 | [clojure.set :as set]) 6 | (:import 7 | (clojure.lang IDeref 8 | IRef) 9 | (java.util.concurrent TimeUnit) 10 | (java.io ByteArrayInputStream 11 | InputStreamReader 12 | PushbackReader) 13 | (org.apache.curator.framework CuratorFrameworkFactory 14 | CuratorFramework) 15 | (org.apache.curator.framework.api CuratorWatcher 16 | PathAndBytesable 17 | Versionable) 18 | (org.apache.curator.framework.state ConnectionStateListener 19 | ConnectionState) 20 | (org.apache.curator.framework.imps CuratorFrameworkState) 21 | (org.apache.curator.retry BoundedExponentialBackoffRetry) 22 | (org.apache.curator.framework.recipes.locks InterProcessMutex) 23 | (org.apache.curator.framework.recipes.atomic DistributedAtomicValue 24 | PromotedToLock) 25 | (org.apache.curator.framework.recipes.shared SharedValue 26 | SharedValueListener) 27 | (org.apache.zookeeper KeeperException 28 | KeeperException$NoNodeException 29 | KeeperException$NodeExistsException 30 | KeeperException$BadVersionException) 31 | (org.apache.zookeeper.data Stat))) 32 | 33 | (defn encode 34 | "Serialize an object to bytes." 35 | [o] 36 | (if (nil? o) 37 | (byte-array 0) 38 | (binding [*print-dup* false] 39 | (-> o pr-str .getBytes)))) 40 | 41 | (defn decode 42 | "Deserialize bytes to an object." 43 | [^bytes bytes] 44 | (if (or (nil? bytes) (= 0 (alength bytes))) 45 | nil 46 | (with-open [s (ByteArrayInputStream. bytes) 47 | i (InputStreamReader. s) 48 | r (PushbackReader. i)] 49 | (binding [*read-eval* false] 50 | (edn/read r))))) 51 | 52 | (defn retry-policy 53 | "A bounded exponential backoff retry policy." 54 | [] 55 | (BoundedExponentialBackoffRetry. 1000 30000 5)) 56 | 57 | (defn ^CuratorFramework framework 58 | "Returns a new Curator framework with the given ZK connection string and 59 | namespace." 60 | [zk-connect-string namespace] 61 | (doto 62 | (-> (CuratorFrameworkFactory/builder) 63 | (.namespace namespace) 64 | (.connectString zk-connect-string) 65 | (.retryPolicy (BoundedExponentialBackoffRetry. 1000 30000 3)) 66 | .build) 67 | .start)) 68 | 69 | (defn shutdown! 70 | "Shuts down a curator client." 71 | [^CuratorFramework curator] 72 | (.close curator)) 73 | 74 | (defonce ^CuratorFramework curator nil) 75 | 76 | (defn create! 77 | "Create a znode." 78 | ([^CuratorFramework curator path] 79 | (.. curator create creatingParentsIfNeeded (forPath path))) 80 | ([^CuratorFramework curator path data] 81 | (.. curator create creatingParentsIfNeeded (forPath (encode data))))) 82 | 83 | (defn delete! 84 | "Ensures a znode does not exist. Idempotent--will not throw if the znode 85 | doesn't exist already." 86 | ([^CuratorFramework curator path] 87 | (try 88 | (.. curator delete (forPath path)) 89 | (catch KeeperException$NoNodeException e nil)))) 90 | 91 | (defn interrupt-when-lost 92 | "Returns a ConnectionStateListener which interrupts the current thread when 93 | the connection transitions to LOST." 94 | [] 95 | (let [thread (Thread/currentThread) 96 | interrupter (delay (.interrupt thread))] 97 | (reify ConnectionStateListener 98 | (stateChanged [this client state] 99 | (condp = state 100 | ConnectionState/LOST @interrupter 101 | ConnectionState/SUSPENDED nil ;; maybe interrupt here too? 102 | nil))))) 103 | 104 | (defmacro with-curator-listener 105 | "Registers a curator state listener for the duration of the body." 106 | [^CuratorFramework curator listener & body] 107 | `(let [l# (.getConnectionStateListenable curator) 108 | listener# ~listener] 109 | (try 110 | (.addListener l# listener#) 111 | ~@body 112 | (finally 113 | (.removeListener l# listener#))))) 114 | 115 | (defmacro with-lock 116 | "Acquires a distributed lock on a path, then evaluates body with the lock 117 | held. Always releases the lock when at the end of the body. If acquisition 118 | fails, throws. If the lock is lost during execution of the body, will 119 | interrupt the thread which invoked locking-zk." 120 | [^CuratorFramework curator path wait-ms & body] 121 | `(let [path# ~path 122 | lock# (InterProcessMutex. curator path#)] 123 | (with-curator-listener curator (interrupt-when-lost) 124 | (when-not (.acquire lock# ~wait-ms TimeUnit/MILLISECONDS) 125 | (throw (IllegalStateException. (str "Failed to lock " path#)))) 126 | (try 127 | ~@body 128 | (finally 129 | (try 130 | (.release lock#) 131 | ; If we lost the session, our znode will be deleted for us 132 | (catch IllegalStateException e# nil))))))) 133 | 134 | (defprotocol Atomic 135 | "Protocol for atomic Zookeeper operations." 136 | (swap- [this f args]) 137 | (reset!! [this value])) 138 | 139 | (defn distributed-atom 140 | "Creates a distributed atom at a given path. Takes an initial value which 141 | will be set only if the znode does not exist yet." 142 | [^CuratorFramework curator path initial-value] 143 | ; Initial value 144 | (try 145 | (.. curator 146 | create 147 | creatingParentsIfNeeded 148 | (forPath path (encode initial-value))) 149 | (catch KeeperException$NodeExistsException e)) 150 | 151 | ; Create distributed atomic value 152 | (let [dav (DistributedAtomicValue. 153 | curator 154 | path 155 | (retry-policy) 156 | (.. (PromotedToLock/builder) 157 | (lockPath path) 158 | (retryPolicy (retry-policy)) 159 | build))] 160 | 161 | ; Wrap it in our protocols 162 | (reify 163 | IDeref 164 | (deref [this] 165 | (decode (.postValue (.get dav)))) 166 | 167 | Atomic 168 | (swap- [this f args] 169 | (loop [value @this] 170 | (let [value' (apply f value args) 171 | result (.compareAndSet dav (encode value) (encode value'))] 172 | (if (.succeeded result) 173 | (decode (.postValue result)) 174 | (do 175 | ; Randomized backoff might be useful if we use optimistic 176 | ; concurrency instead of lock promotion 177 | ; (Thread/sleep (rand-int 10)) 178 | (recur (decode (.preValue result)))))))) 179 | 180 | (reset!! [this value] 181 | (let [encoded-value (encode value)] 182 | (loop [] 183 | (if (.succeeded (.trySet dav encoded-value)) 184 | value 185 | (recur)))))))) 186 | 187 | (declare update-shared-atom-) 188 | (declare refresh-shared-atom!) 189 | 190 | (deftype SharedAtom 191 | [^CuratorFramework curator 192 | path 193 | state ; Atom of {:state :value :version :watches} 194 | listener ; Atom to a connection state listener 195 | watcher] ; Atom to a ZK watcher for the object 196 | 197 | IDeref 198 | (deref [this] 199 | (:value @state)) 200 | 201 | IRef 202 | (getValidator [this] (constantly true)) 203 | (getWatches [this] (:watches @state)) 204 | (addWatch [this k f] (do (swap! state update-in [:watches] assoc k f) 205 | this)) 206 | (removeWatch [this k] (do (swap! state update-in [:watches] dissoc k) 207 | this)) 208 | 209 | Atomic 210 | (reset!! [this value] 211 | ; Possible race condition: doesn't respect shutdown correctly. 212 | (assert (= :started (:state @state))) 213 | 214 | ; Update value in ZK 215 | (.. curator setData (forPath path (encode value))) 216 | (refresh-shared-atom! this)) 217 | 218 | (swap- [this f args] 219 | ; Possible race condition: doesn't respect shutdown correctly. 220 | (assert (= :started (:state @state))) 221 | 222 | (let [res (try 223 | (let [{:keys [value version]} @state 224 | value (apply f value args) 225 | ; Write new value 226 | ^Stat stat (.. curator 227 | setData 228 | (withVersion version) 229 | (forPath path (encode value)))] 230 | ; Broadcast changes 231 | (update-shared-atom- this (.getVersion stat) value) 232 | value) 233 | 234 | ; If that update process fails, return ::bad-version instead. 235 | (catch KeeperException$BadVersionException e ::bad-version))] 236 | (if (= ::bad-version res) 237 | (do 238 | ; Back off, re-read, and retry. 239 | (Thread/sleep (rand-int 10)) 240 | (refresh-shared-atom! this) 241 | (recur f args)) 242 | res)))) 243 | 244 | (defn update-shared-atom- 245 | "Updates a shared atom with a new value. This broadcasts changes locally--it 246 | doesn't send updates back out to ZK. You probably don't want to call this 247 | unless you really know what you're doing. Returns the latest value." 248 | [^SharedAtom a version' value'] 249 | (assert version') 250 | (locking a 251 | (let [old (atom nil) 252 | new (swap! (.state a) 253 | (fn [state] 254 | (if (< (:version state) version') 255 | ; Allow monotonic updates 256 | (do (reset! old state) 257 | (merge state {:version version' 258 | :value value'})) 259 | ; Otherwise, ignore 260 | (do (reset! old nil) 261 | state)))) 262 | old @old] 263 | 264 | (when old 265 | ; We advanced; call watches 266 | (doseq [[k f] (:watches new)] 267 | (f k a (:value old) (:value new)))) 268 | 269 | (:value new)))) 270 | 271 | (defn refresh-shared-atom! 272 | "Reads the most recent value for a SharedAtom, re-establishing a watch. 273 | Returns the latest value." 274 | [^SharedAtom a] 275 | (locking a 276 | (let [stat (Stat.) 277 | bytes (.. (.curator a) 278 | getData 279 | (storingStatIn stat) 280 | (usingWatcher (deref (.watcher a))) 281 | (forPath (.path a)))] 282 | (update-shared-atom- a (.getVersion stat) (decode bytes))))) 283 | 284 | (defn shutdown-shared-atom 285 | "Call when you're done using a SharedAtom." 286 | [^SharedAtom a] 287 | ; Stop listening for connection updates. 288 | (.. a curator getConnectionStateListenable (removeListener @(.listener a))) 289 | (reset! (.state a) {:state :closed})) 290 | 291 | (defn shared-atom-listener 292 | "Watches the connection state of Curator for a SharedAtom." 293 | [a] 294 | (reify ConnectionStateListener 295 | (stateChanged [this client state] 296 | (condp = state 297 | ConnectionState/READ_ONLY nil 298 | ConnectionState/CONNECTED (refresh-shared-atom! a) 299 | ConnectionState/RECONNECTED (refresh-shared-atom! a) 300 | ConnectionState/LOST nil 301 | ConnectionState/SUSPENDED nil 302 | nil)))) 303 | 304 | (defn shared-atom-watcher 305 | "A curator watcher which refreshes the SharedAtom atom when changes occur." 306 | [^SharedAtom a] 307 | (reify CuratorWatcher 308 | (process [this event] 309 | (when (= :started (-> a .state deref :state)) 310 | (refresh-shared-atom! a))))) 311 | 312 | (defn shared-atom 313 | "Creates a shared atom at a given path. Takes an initial value which will be 314 | set only if the znode does not exist yet. Unlike distributed-atom, 315 | shared-atom does not use lock promotion; all writes are optimistic with 316 | exponential backoff. Reads are cheaper because they can use a locally cached 317 | value. Also unlike distributed-atom, shared-atom is watchable for changes. 318 | 319 | You must explicitly shut down a shared-atom using shutdown-shared." 320 | [^CuratorFramework curator path initial-value] 321 | 322 | (let [curator curator 323 | ; Initialize value if nonexistent 324 | _ (try 325 | (.. curator 326 | create 327 | creatingParentsIfNeeded 328 | (forPath path (encode initial-value))) 329 | (catch KeeperException$NodeExistsException e)) 330 | 331 | ; Create shared atom structure 332 | a (SharedAtom. curator 333 | path 334 | (atom {:state :started 335 | :version -1 336 | :value nil 337 | :watches {}}) 338 | (atom nil) 339 | (atom nil))] 340 | 341 | ; Hook up watchers and listeners 342 | (reset! (.listener a) (shared-atom-listener a)) 343 | (reset! (.watcher a) (shared-atom-watcher a)) 344 | 345 | ; Do an initial read 346 | (refresh-shared-atom! a) 347 | 348 | a)) 349 | 350 | (defn swap!! 351 | [atomic f & args] 352 | (swap- atomic f args)) 353 | -------------------------------------------------------------------------------- /src/skuld/db.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.db 2 | "Database backend interface." 3 | (:import (java.io File)) 4 | (:use [potemkin :only [defprotocol+]]) 5 | (:require [clojure.java.io :as io])) 6 | 7 | (defprotocol+ DB 8 | (ids [db]) 9 | (tasks [db]) 10 | (unclaimed [db]) 11 | (count-tasks [db]) 12 | (get-task [db ^Bytes id]) 13 | (claim-task! [db ^Bytes id dt] 14 | [db ^Bytes id i claim]) 15 | (merge-task! [db task]) 16 | (close! [db]) 17 | (wipe! [db])) 18 | 19 | (defn path 20 | "Constructs a path name for storing data in a vnode. Options: 21 | 22 | :host 23 | :port 24 | :partition 25 | :ext" 26 | [{:keys [host port partition ext]}] 27 | (-> "data" 28 | (io/file (str host ":" port) 29 | (str partition "." ext)) 30 | str)) 31 | 32 | (defn path! 33 | "Like path, but ensures the path exists." 34 | [opts] 35 | (let [p (path opts)] 36 | (.mkdirs (io/file p)) 37 | p)) 38 | 39 | (defn rm-rf! [path] 40 | (let [f (fn [f ^File p] 41 | (when (.isDirectory p) 42 | (doseq [child (.listFiles p)] 43 | (f f child))) 44 | (io/delete-file p))] 45 | (f f (io/file path)))) 46 | 47 | (defn destroy-data! 48 | "Removes all data for the given vnode." 49 | [opts] 50 | (rm-rf! (path opts))) 51 | 52 | (defn local-data? 53 | "Is there local data for the given vnode?" 54 | [opts] 55 | (-> opts path io/file .exists)) 56 | -------------------------------------------------------------------------------- /src/skuld/db/level.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.db.level 2 | (:import (java.io Closeable) 3 | (com.aphyr.skuld Bytes)) 4 | (:require [clj-leveldb :as level] 5 | [skuld.task :as task] 6 | [skuld.util :refer [fress-read fress-write]]) 7 | (:use skuld.db 8 | clojure.tools.logging)) 9 | 10 | (defrecord Level [level count-cache running] 11 | DB 12 | (ids [db] 13 | (assert @running) 14 | (map #(Bytes. (first %)) (level/iterator level))) 15 | 16 | (tasks [db] 17 | (assert @running) 18 | (map #(nth % 1) (level/iterator level))) 19 | 20 | (count-tasks [db] 21 | @count-cache) 22 | 23 | (get-task [db task-id] 24 | (assert @running) 25 | (level/get level (.bytes ^Bytes task-id))) 26 | 27 | (claim-task! [db task-id dt] 28 | (locking db 29 | (assert @running) 30 | (when-let [task (get-task db task-id)] 31 | (let [claimed (task/claim task dt)] 32 | (level/put level 33 | (.bytes ^Bytes (:id task)) 34 | claimed) 35 | claimed)))) 36 | 37 | (claim-task! [db task-id i claim] 38 | (locking db 39 | (assert @running) 40 | (when-let [task (get-task db task-id)] 41 | (let [claimed (task/request-claim task i claim)] 42 | (level/put level 43 | (.bytes ^Bytes (:id task)) 44 | claimed) 45 | claimed)))) 46 | 47 | (merge-task! [db task] 48 | (locking db 49 | (assert @running) 50 | (when-not (let [current (get-task db (:id task))] 51 | (level/put level (.bytes ^Bytes (:id task)) 52 | (task/merge current task)) 53 | current) 54 | ; We enqueued something new! 55 | (swap! count-cache inc)))) 56 | 57 | (close! [db] 58 | (locking db 59 | (swap! running (fn [was] 60 | (when was 61 | (.close ^Closeable level) 62 | false))))) 63 | 64 | 65 | (wipe! [db] 66 | (locking db 67 | (when @running 68 | (->> level 69 | level/iterator 70 | (map (comp (partial level/delete level) first)) 71 | dorun) 72 | (reset! count-cache 0))))) 73 | 74 | (defn open 75 | "Start a database service. Initializes the local DB storage and binds the 76 | database for further calls. Options: 77 | 78 | :host 79 | :port 80 | :partition 81 | :data-dir" 82 | [opts] 83 | (let [level (level/create-db (path! (assoc opts :ext "level")) 84 | {:val-decoder #(and % (fress-read %)) 85 | :val-encoder #(and % (fress-write %))}) 86 | c (count (level/iterator level))] 87 | (Level. level (atom c) (atom true)))) 88 | -------------------------------------------------------------------------------- /src/skuld/flake.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.flake 2 | "ID generation. Flake IDs are 160 bits, and comprise: 3 | 4 | [64 bits | Timestamp, in milliseconds since the epoch] 5 | [32 bits | a per-process counter, reset each millisecond] 6 | [48 bits | a host identifier] 7 | [16 bits | the process ID] 8 | 9 | Note that the timestamp is not Posix time or UTC time. Instead we use the 10 | JVM's nanoTime, which is a linear time source over intervals smaller than 11 | ~292 years, and use it to compute an offset from the POSIX time as measured 12 | *once* by System/currentTimeMillis. 13 | 14 | Regressions in time are not allowed; Flake will periodically serialize the 15 | current time to disk to prevent regressions." 16 | (:require [primitive-math :as p]) 17 | (:use [potemkin :only [deftype+]]) 18 | (:import (java.lang.management ManagementFactory) 19 | (java.net InetAddress 20 | NetworkInterface) 21 | (java.nio ByteBuffer) 22 | (java.security MessageDigest) 23 | (java.util Arrays) 24 | (java.util.concurrent.atomic AtomicInteger))) 25 | 26 | (defonce initialized (atom false)) 27 | 28 | ; Cached state 29 | (declare time-offset*) 30 | (declare ^"[B" node-fragment*) 31 | 32 | ; Mutable state 33 | (deftype+ Counter [^long time ^int count]) 34 | (defonce counter (atom (Counter. Long/MIN_VALUE Integer/MIN_VALUE))) 35 | 36 | (defn pid 37 | "Process identifier, such as it is on the JVM. :-/" 38 | [] 39 | (let [name (.. ManagementFactory getRuntimeMXBean getName)] 40 | (Integer. ^String (get (re-find #"^(\d+).*" name) 1)))) 41 | 42 | (defn ^"[B" node-id 43 | "We take all hardware addresses, sort them bytewise, concatenate, hash, and 44 | truncate to 48 bits." 45 | [] 46 | (let [addrs (->> (NetworkInterface/getNetworkInterfaces) 47 | enumeration-seq 48 | (map #(.getHardwareAddress ^NetworkInterface %)) 49 | (remove nil?)) 50 | md (MessageDigest/getInstance "SHA-1")] 51 | (assert (< 0 (count addrs))) 52 | (doseq [addr addrs] 53 | (.update md ^bytes addr)) 54 | ; 6 bytes is 48 bits 55 | (Arrays/copyOf (.digest md) 6))) 56 | 57 | (defn time-offset-estimate 58 | "Tries to figure out the time offset between epochMillis and nanoTime. You 59 | can add this offset to nanoTime to reconstruct POSIX time, only linearized 60 | from a particular instant." 61 | [] 62 | (- (* 1000000 (System/currentTimeMillis)) 63 | (System/nanoTime))) 64 | 65 | (defn mean-time-offset 66 | "Computes the offset, in nanoseconds, between the current nanoTime and the 67 | system. Takes n samples." 68 | [n] 69 | (-> (->> (repeatedly time-offset-estimate) 70 | (map double) 71 | (take n) 72 | (reduce +)) 73 | (/ n) 74 | long)) 75 | 76 | (defn linear-time 77 | "Returns a linearized time in milliseconds, roughly corresponding to time 78 | since the epoch." 79 | [] 80 | (p/div (p/+ (unchecked-long time-offset*) (System/nanoTime)) 81 | 1000000)) 82 | 83 | (defn node-fragment 84 | "Constructs an eight-byte long byte array containing the node ID and process 85 | ID." 86 | [] 87 | (let [a (byte-array 8)] 88 | (doto (ByteBuffer/wrap a) 89 | (.put (node-id)) 90 | (.putShort (unchecked-short (pid)))) 91 | a)) 92 | 93 | (defn init! 94 | "Initializes the flake generator state." 95 | [] 96 | (locking initialized 97 | (if (false? @initialized) 98 | (do 99 | (def ^long time-offset* (mean-time-offset 10)) 100 | (def ^"[B" node-fragment* (node-fragment)) 101 | 102 | (reset! initialized true))))) 103 | 104 | (defn ^long count! 105 | "Increments and gets the count for a given time." 106 | [t'] 107 | (.count ^Counter (swap! counter 108 | (fn [^Counter c] 109 | (cond 110 | (< (.time c) t') (Counter. t' Integer/MIN_VALUE) 111 | (= (.time c) t') (Counter. t' (p/inc (.count c))) 112 | :else (throw (IllegalStateException. 113 | "time can't flow backwards."))))))) 114 | 115 | (defn ^bytes id 116 | "Generate a new flake ID; returning a byte array." 117 | [] 118 | (let [id (try 119 | (let [t (linear-time) 120 | c (count! t) 121 | b (ByteBuffer/allocate 20)] 122 | (.putLong b t) 123 | (.putInt b (count! t)) 124 | (.put b node-fragment*) 125 | (.array b)) 126 | (catch IllegalStateException e 127 | ; Lost the race to count for this time; retry. 128 | ::recur))] 129 | (if (= id ::recur) 130 | (recur) 131 | id))) 132 | 133 | (defn byte-buffer 134 | "Wraps a byte[] in a ByteBuffer." 135 | [^bytes b] 136 | (ByteBuffer/wrap b)) 137 | -------------------------------------------------------------------------------- /src/skuld/http.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.http 2 | "An HTTP interface to a Skuld node." 3 | (:require [cheshire.core :as json] 4 | [cheshire.generate :refer [add-encoder encode-str]] 5 | [clout.core :refer [route-compile route-matches]] 6 | [clojure.data.codec.base64 :as b64] 7 | [clojure.tools.logging :refer :all] 8 | [clojure.walk :refer [keywordize-keys]] 9 | [ring.adapter.jetty :refer [run-jetty]] 10 | [ring.middleware.json :refer [wrap-json-body]] 11 | [ring.util.codec :refer [form-decode]] 12 | [skuld.node :as node] 13 | [skuld.flake :as flake]) 14 | (:import [com.aphyr.skuld Bytes] 15 | [com.fasterxml.jackson.core JsonGenerator JsonParseException] 16 | [org.eclipse.jetty.server Server])) 17 | 18 | (defn- encode-bytes 19 | "Encode a bytes to the json generator." 20 | [^Bytes b ^JsonGenerator jg] 21 | (.writeString jg (-> ^Bytes b .bytes b64/encode String. (.replaceAll "\\+" "-") (.replaceAll "/" "_")))) 22 | 23 | ;; Custom Cheshire encoder for the Bytes type 24 | (add-encoder Bytes encode-bytes) 25 | 26 | (defn- http-response 27 | "Given a status and body and optionally a headers map, returns a ring 28 | response." 29 | [status body & [headers]] 30 | {:status status 31 | :headers (or headers {}) 32 | :body body}) 33 | 34 | (def ^:private ok-response (partial http-response 200)) 35 | (def ^:private bad-request (partial http-response 400)) 36 | (def ^:private not-found (partial http-response 404)) 37 | 38 | (defn- serialize 39 | "Given a request map and a response body, serializes the response body first 40 | based on `Accept` header and then falling back to JSON. Currently only 41 | serializes to JSON. Returns the body `Content-Type` header in a vector." 42 | [req resp-body] 43 | (let [accept (get-in req [:headers "Accept"] "application/json") 44 | json-ct {"Content-Type" "application/json;charset=utf-8"}] 45 | (condp re-find accept 46 | #"^application/(vnd.+)?json" [(json/generate-string resp-body) json-ct] 47 | [(json/generate-string resp-body) json-ct]))) 48 | 49 | (defn- endpoint 50 | "Defines an HTTP endpoint with an allowed request method. Takes an allowed 51 | method, a request map, and the response body." 52 | [allowed-method req resp-body & [http-resp-fn]] 53 | (let [http-resp (or http-resp-fn ok-response)] 54 | (if (= (:request-method req) allowed-method) 55 | (if resp-body 56 | (apply http-resp (serialize req resp-body)) 57 | (not-found "Not Found")) 58 | (http-response 405 "Method Not Allowed")))) 59 | 60 | (def ^:private GET (partial endpoint :get)) 61 | (def ^:private POST (partial endpoint :post)) 62 | 63 | (defn- b64->id 64 | "Coerces a base64-encoded id into a Bytes type." 65 | [^String b64-id] 66 | (-> b64-id (.replaceAll "-" "+") (.replaceAll "_" "/" ) .getBytes b64/decode Bytes.)) 67 | 68 | (defn- parse-int 69 | "Safely coerces a string into an integer. If the conversion is impossible, 70 | returns a fallback value if provided or nil." 71 | [s & [fallback]] 72 | (try (Integer/parseInt s) 73 | (catch Exception e 74 | fallback))) 75 | 76 | (defn- count-queue 77 | "Like `node/count-queue`, but wrapped around an HTTP request." 78 | [node req] 79 | (let [r (-> req :query-params :r parse-int) 80 | queue (-> req :query-params :queue)] 81 | (GET req (node/count-queue node {:r r 82 | :queue queue})))) 83 | 84 | (defn- claim! 85 | "Like `node/claim!`, but wrapped around an HTTP request." 86 | [node req] 87 | (let [dt (-> req :body :dt) 88 | queue (-> req :body :queue) 89 | ret (node/claim! node {:queue queue 90 | :dt dt})] 91 | (POST req (dissoc ret :request-id)))) 92 | 93 | (defn- complete! 94 | "Like `node/complete!`, but wrapped around an HTTP request." 95 | [node req id] 96 | (let [id (b64->id id) 97 | cid (-> req :body :cid) 98 | w (-> req :query-params :w parse-int) 99 | msg {:task-id id :claim-id cid :w w} 100 | ret (node/complete! node msg)] 101 | (POST req (dissoc ret :responses)))) 102 | 103 | (defn- count-tasks 104 | "Like `node/count-tasks`, but wrapped around an HTTP request." 105 | [node req] 106 | (GET req (node/count-tasks node {}))) 107 | 108 | (defn- enqueue! 109 | "Like `node/enqueue!`, but wrapped around an HTTP request." 110 | [node req] 111 | (if-let [;; Explicitly suck out the task key to avoid passing bad params to 112 | ;; `node/enqueue!` 113 | task (-> req :body :task)] 114 | (try (let [w (-> req :body :w) 115 | msg {:task task :w w} 116 | ret (node/enqueue! node msg)] 117 | (POST req (dissoc ret :responses))) 118 | ;; Handle vnode assertion; return an error to 119 | ;; the client 120 | (catch java.lang.AssertionError e 121 | (let [err {:error (.getMessage e)}] 122 | (POST req err bad-request)))) 123 | ;; Missing parameters, i.e. POST body 124 | (let [err {:error "Missing required params"}] 125 | (POST req err bad-request)))) 126 | 127 | (defn- list-tasks 128 | "Like `node/list-tasks`, but wrapped around an HTTP request." 129 | [node req] 130 | (GET req (node/list-tasks node {}))) 131 | 132 | (defn- get-task 133 | "Like `node/get-task`, but wrapped around an HTTP request." 134 | [node req id] 135 | (let [r (-> req :query-params :r parse-int) 136 | msg {:id (b64->id id) :r r} 137 | ret (node/get-task node msg)] 138 | (if-not (-> ret :task :id) 139 | (GET req {:error "No such task"} not-found) 140 | (GET req (dissoc ret :responses))))) 141 | 142 | (defn- id [req] 143 | "Returns a flake id" 144 | (GET req {:id (Bytes. (flake/id))})) 145 | 146 | (defn- make-handler 147 | "Given a node, constructs the handler function. Returns a response map." 148 | [node] 149 | (fn [req] 150 | (condp route-matches req 151 | "/queue/count" (count-queue node req) 152 | "/tasks/claim" (claim! node req) 153 | "/tasks/complete/:id" :>> (fn [{:keys [id]}] (complete! node req id)) 154 | "/tasks/count" (count-tasks node req) 155 | "/tasks/enqueue" (enqueue! node req) 156 | "/tasks/list" (list-tasks node req) 157 | "/tasks/:id" :>> (fn [{:keys [id]}] (get-task node req id)) 158 | "/id" (id req) 159 | not-found))) 160 | 161 | ;; Lifted from `ring.middleware.params` 162 | (defn- parse-params [params encoding keywords?] 163 | (let [params (if keywords? 164 | (keywordize-keys (form-decode params encoding)) 165 | (form-decode params encoding))] 166 | (if (map? params) params {}))) 167 | 168 | (defn- assoc-query-params 169 | "Parse and assoc parameters from the query string with the request." 170 | [request encoding keywords?] 171 | (merge-with merge request 172 | (if-let [query-string (:query-string request)] 173 | (let [params (parse-params query-string encoding keywords?)] 174 | {:query-params params}) 175 | {:query-params {}}))) 176 | 177 | (defn- wrap-params 178 | "A middleware that attempts to parse incoming query strings into maps." 179 | [handler & [opts]] 180 | (fn [request] 181 | (let [encoding (or (:encoding opts) 182 | (:character-encoding request) 183 | "UTF-8") 184 | keywords? (:keywords? opts) 185 | request (if (:query-params request) 186 | request 187 | (assoc-query-params request encoding keywords?))] 188 | (handler request)))) 189 | 190 | (defn- wrap-json-body-safe 191 | "A wrapper for `wrap-json-body` which catches JSON parsing exceptions and 192 | returns a Bad Request." 193 | [handler & [opts]] 194 | (let [request-handler (wrap-json-body handler opts)] 195 | (fn [request] 196 | (try (request-handler request) 197 | (catch JsonParseException e 198 | (handler request) ;; resolve request before generating a response 199 | (bad-request "Bad Request")))))) 200 | 201 | (defn service 202 | "Given a node and port, constructs a Jetty instance." 203 | [node port] 204 | (info "Starting HTTP server on" (str (:host node) ":" port)) 205 | (let [handler (-> 206 | (make-handler node) 207 | (wrap-json-body-safe {:keywords? true}) 208 | (wrap-params {:keywords? true})) 209 | jetty (run-jetty handler {:host (:host node) 210 | :port port 211 | :join? false})] 212 | jetty)) 213 | 214 | (defn shutdown! 215 | "Stops a given Jetty instance." 216 | [^Server jetty] 217 | (.stop jetty)) 218 | -------------------------------------------------------------------------------- /src/skuld/logging.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.logging 2 | "Configures loggers" 3 | (:import (ch.qos.logback.classic 4 | Level 5 | Logger 6 | LoggerContext 7 | PatternLayout) 8 | (org.slf4j 9 | LoggerFactory) 10 | (ch.qos.logback.classic.jul 11 | LevelChangePropagator) 12 | (ch.qos.logback.core 13 | ConsoleAppender))) 14 | 15 | (defn level-for 16 | "Get the level for a given symbol" 17 | [level] 18 | (if (instance? Level level) 19 | level 20 | (Level/toLevel (name level)))) 21 | 22 | (def ^String root-logger-name 23 | org.slf4j.Logger/ROOT_LOGGER_NAME) 24 | 25 | (def ^Logger root-logger 26 | (LoggerFactory/getLogger root-logger-name)) 27 | 28 | (def ^LoggerContext root-logger-context 29 | (.getLoggerContext root-logger)) 30 | 31 | (defn logger-name 32 | "The name of a logger." 33 | [^Logger logger] 34 | (.getName logger)) 35 | 36 | (defn level 37 | "The level of a logger." 38 | [^Logger logger] 39 | (.getLevel logger)) 40 | 41 | (defn all-loggers 42 | "Returns a list of all registered Loggers." 43 | [] 44 | (.getLoggerList root-logger-context)) 45 | 46 | (defn all-logger-names 47 | "Returns the names of all loggers." 48 | [] 49 | (map logger-name (all-loggers))) 50 | 51 | (defn ^Logger get-loggers 52 | "Returns a singleton list containing a logger for a string or, if given a 53 | regular expression, a list of all loggers finding matching names." 54 | [logger-pattern] 55 | (cond 56 | (nil? logger-pattern) 57 | (list root-logger) 58 | 59 | (instance? Logger logger-pattern) 60 | (list logger-pattern) 61 | 62 | (string? logger-pattern) 63 | (list (LoggerFactory/getLogger ^String logger-pattern)) 64 | 65 | :else (filter (comp (partial re-find logger-pattern) logger-name) 66 | (all-loggers)))) 67 | 68 | (defn ^Logger get-logger 69 | [logger-pattern] 70 | (first (get-loggers logger-pattern))) 71 | 72 | (defn set-level 73 | "Set the level for the given logger, by string name. Use: 74 | 75 | (set-level (get-logger \"skuld.node\") :debug)" 76 | ([level] 77 | (set-level root-logger level)) 78 | ([logger-pattern level] 79 | (doseq [logger (get-loggers logger-pattern)] 80 | (.setLevel logger (level-for level))))) 81 | 82 | (defmacro with-level 83 | "Sets logging for the evaluation of body to the desired level." 84 | [level-name logger-patterns & body] 85 | `(let [level# (level-for ~level-name) 86 | root-logger-level# (level root-logger) 87 | loggers-and-levels# (doall 88 | (->> (list ~logger-patterns) 89 | flatten 90 | (mapcat get-loggers) 91 | (map #(list % (or (level %) 92 | root-logger-level#)))))] 93 | (try 94 | (doseq [[logger# _#] loggers-and-levels#] 95 | (set-level logger# level#)) 96 | (do ~@body) 97 | (finally 98 | (dorun (map (partial apply set-level) loggers-and-levels#)))))) 99 | 100 | (defmacro mute 101 | "Turns off logging for all loggers the evaluation of body." 102 | [& body] 103 | `(with-level :off (all-logger-names) ~@body)) 104 | 105 | (defmacro suppress 106 | "Turns off logging for the evaluation of body." 107 | [loggers & body] 108 | `(with-level :off ~loggers ~@body)) 109 | 110 | (defn init 111 | [] 112 | (doto root-logger-context 113 | .reset 114 | (.addListener 115 | (doto (LevelChangePropagator.) 116 | (.setContext root-logger-context) 117 | (.setResetJUL true)))) 118 | (.addAppender root-logger 119 | (doto (ConsoleAppender.) 120 | (.setName "console-appender") 121 | (.setContext root-logger-context) 122 | (.setLayout 123 | (doto (PatternLayout.) 124 | (.setContext root-logger-context) 125 | (.setPattern "%date{YYYY-MM-dd'T'HH:mm:ss.SSS} %-5p %c: %m%n%xEx") 126 | (.start))) 127 | (.start)) 128 | ) 129 | (set-level :info)) 130 | 131 | (init) 132 | -------------------------------------------------------------------------------- /src/skuld/net.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.net 2 | "Handles network communication between nodes. Automatically maintains TCP 3 | connections encapsulated in a single stateful component. Allows users to 4 | register callbacks to receive messages." 5 | (:require [clojure.edn :as edn] 6 | [skuld.flake :as flake] 7 | [clojure.stacktrace :as trace] 8 | [hexdump.core :as hex] 9 | [skuld.util :refer [fress-read fress-write]]) 10 | (:use clojure.tools.logging 11 | skuld.util) 12 | (:import (com.aphyr.skuld Bytes) 13 | (java.io ByteArrayInputStream 14 | DataInputStream 15 | InputStreamReader 16 | PushbackReader) 17 | (java.net InetSocketAddress ConnectException) 18 | (java.nio.charset Charset) 19 | (java.util List) 20 | (java.util.concurrent TimeUnit 21 | CountDownLatch) 22 | (io.netty.bootstrap Bootstrap 23 | ServerBootstrap) 24 | (io.netty.buffer ByteBuf 25 | ByteBufInputStream 26 | ByteBufOutputStream 27 | Unpooled) 28 | (io.netty.channel Channel 29 | ChannelFuture 30 | ChannelHandlerContext 31 | ChannelInboundMessageHandlerAdapter 32 | ChannelInitializer 33 | ChannelOption 34 | ChannelStateHandlerAdapter 35 | DefaultEventExecutorGroup) 36 | (io.netty.channel.socket SocketChannel) 37 | (io.netty.channel.socket.nio NioEventLoopGroup 38 | NioSocketChannel 39 | NioServerSocketChannel) 40 | (io.netty.handler.codec MessageToMessageCodec) 41 | (io.netty.handler.codec.protobuf ProtobufVarint32FrameDecoder 42 | ProtobufVarint32LengthFieldPrepender) 43 | (io.netty.util Attribute 44 | AttributeKey) 45 | (java.nio.channels ClosedChannelException))) 46 | 47 | (def logger (agent nil)) 48 | (defn log-print 49 | [_ & things] 50 | (apply println things)) 51 | (defn log- 52 | [& things] 53 | (apply send-off logger log-print things)) 54 | 55 | (declare started?) 56 | (declare handle-response!) 57 | 58 | (defonce event-executor 59 | (DefaultEventExecutorGroup. 32)) 60 | 61 | (defn protobuf-varint32-frame-decoder [] 62 | (ProtobufVarint32FrameDecoder.)) 63 | 64 | (def protobuf-varint32-frame-encoder 65 | (ProtobufVarint32LengthFieldPrepender.)) 66 | 67 | (defn edn-codec [] 68 | (proxy [MessageToMessageCodec] 69 | [(into-array Class [ByteBuf]) (into-array Class [Object])] 70 | 71 | (encode [^ChannelHandlerContext ctx object] 72 | (->> object pr-str .getBytes Unpooled/wrappedBuffer)) 73 | 74 | (decode [^ChannelHandlerContext ctx ^ByteBuf buffer] 75 | (with-open [is (ByteBufInputStream. buffer) 76 | i (InputStreamReader. is) 77 | r (PushbackReader. i)] 78 | (binding [*read-eval* false] 79 | ; (debug "Got" (.toString buffer (Charset/forName "UTF-8"))) 80 | (edn/read r)))))) 81 | 82 | (defn fressian-codec [] 83 | (proxy [MessageToMessageCodec] 84 | [(into-array Class [ByteBuf]) (into-array Class [Object])] 85 | 86 | (encode [^ChannelHandlerContext ctx object] 87 | (-> object fress-write Unpooled/wrappedBuffer)) 88 | 89 | (decode [^ChannelHandlerContext ctx ^ByteBuf buffer] 90 | (let [a (byte-array (.readableBytes buffer))] 91 | (.readBytes buffer a) 92 | ; (with-open [is (ByteBufInputStream. buffer) 93 | ; dis (DataInputStream. is)] 94 | ; (try 95 | (binding [*read-eval* false] 96 | (fress-read a)))))) 97 | ; (catch Exception e 98 | ; (hex/hexdump (seq a)) 99 | ; (throw e))) 100 | 101 | (defn handler 102 | "Returns a Netty handler that calls f with its messages, and writes non-nil 103 | return values back. Response will automatically have :request-id assoc'd into 104 | them." 105 | [f] 106 | (proxy [ChannelInboundMessageHandlerAdapter] [(into-array Class [Object])] 107 | (messageReceived [^ChannelHandlerContext ctx message] 108 | ; Pretty sure we're seeing deadlocks due to threadpool starvation. 109 | ; Let's try futures here. 110 | (future 111 | (when-let [response (try (f message) 112 | (catch Throwable t 113 | (warn t "net: server handler caught an exception") 114 | {:error (str (.getMessage t) 115 | (with-out-str 116 | (trace/print-cause-trace t)))}))] 117 | (.write ctx (assoc response :request-id (:request-id message)))))) 118 | (exceptionCaught [^ChannelHandlerContext ctx ^Throwable cause] 119 | (let [local (.. ctx channel localAddress) 120 | remote (.. ctx channel remoteAddress)] 121 | (condp instance? cause 122 | ClosedChannelException (warnf "net: server handler found a closed channel between %s -> %s" local remote) 123 | (warn cause "net: server handler caught exception with %s -> %s" local remote)))))) 124 | 125 | (defonce peer-attr 126 | (AttributeKey. "skuld-peer")) 127 | 128 | (defn server 129 | "Starts a netty server for this node." 130 | [node] 131 | (let [bootstrap ^ServerBootstrap (ServerBootstrap.) 132 | boss-group (NioEventLoopGroup.) 133 | worker-group (NioEventLoopGroup.)] 134 | (try 135 | (doto bootstrap 136 | (.group boss-group worker-group) 137 | (.channel NioServerSocketChannel) 138 | (.localAddress ^String (:host node) ^int (int (:port node))) 139 | (.option ChannelOption/SO_BACKLOG (int 100)) 140 | (.childOption ChannelOption/TCP_NODELAY true) 141 | (.childHandler 142 | (proxy [ChannelInitializer] [] 143 | (initChannel [^SocketChannel ch] 144 | (.. ch 145 | (pipeline) 146 | (addLast "fdecoder" (protobuf-varint32-frame-decoder)) 147 | (addLast "fencoder" protobuf-varint32-frame-encoder) 148 | (addLast "codec" (fressian-codec)) 149 | (addLast event-executor "handler" 150 | (handler @(:handler node)))))))) 151 | 152 | {:listener (.. bootstrap bind sync) 153 | :bootstrap bootstrap 154 | :boss-group boss-group 155 | :worker-group worker-group} 156 | 157 | (catch Throwable t 158 | (.shutdown boss-group) 159 | (.shutdown worker-group) 160 | (.awaitTermination boss-group 30 TimeUnit/SECONDS) 161 | (.awaitTermination worker-group 30 TimeUnit/SECONDS) 162 | (throw t))))) 163 | 164 | (defn inactive-client-handler 165 | "When client conns go inactive, unregisters them from the corresponding 166 | node." 167 | [node] 168 | (proxy [ChannelStateHandlerAdapter] [] 169 | (^void channelInactive [^ChannelHandlerContext ctx] 170 | (let [peer (.. ctx channel (attr peer-attr) get)] 171 | ; Dissoc from node's map 172 | (swap! (:conns node) 173 | (fn [conns] 174 | (let [ch (get conns peer)] 175 | (if (= ch (.channel ctx)) 176 | ; Yep, remove ourselves. 177 | (dissoc conns peer) 178 | conns)))))))) 179 | 180 | (defn client-response-handler 181 | "When messages arrive, routes them through the node's request map." 182 | [node] 183 | (let [requests (:requests node)] 184 | (proxy [ChannelInboundMessageHandlerAdapter] [(into-array Class [Object])] 185 | (messageReceived [^ChannelHandlerContext ctx message] 186 | (when-let [id (:request-id message)] 187 | (try 188 | (handle-response! requests id message) 189 | (catch Throwable t 190 | (warn t "node handler caught"))))) 191 | (exceptionCaught [^ChannelHandlerContext ctx ^Throwable cause] 192 | (let [peer (.. ctx channel (attr peer-attr) get)] 193 | (condp instance? cause 194 | ConnectException (warnf "client handle caught exception with %s: %s" peer (.getMessage cause)) 195 | (warn cause "client handle caught exception with" peer))))))) 196 | 197 | (defn client 198 | [node] 199 | (let [bootstrap ^Bootstrap (Bootstrap.) 200 | group- (NioEventLoopGroup. 16)] 201 | (try 202 | (doto bootstrap 203 | (.group group-) 204 | (.option ChannelOption/TCP_NODELAY true) 205 | (.option ChannelOption/SO_KEEPALIVE true) 206 | (.channel NioSocketChannel) 207 | (.handler (proxy [ChannelInitializer] [] 208 | (initChannel [^SocketChannel ch] 209 | (.. ch 210 | (pipeline) 211 | (addLast "fdecoder" (protobuf-varint32-frame-decoder)) 212 | (addLast "fencoder" protobuf-varint32-frame-encoder) 213 | (addLast "codec" (fressian-codec)) 214 | (addLast "inactive" (inactive-client-handler node)) 215 | (addLast event-executor "handler" 216 | (client-response-handler node))))))) 217 | {:bootstrap bootstrap 218 | :group group-} 219 | 220 | (catch Throwable t 221 | (.shutdown group-) 222 | (.awaitTermination group- 30 TimeUnit/SECONDS) 223 | (throw t))))) 224 | 225 | (defn string-id 226 | "A string identifier for a node." 227 | [node] 228 | (str (:host node) ":" (:port node))) 229 | 230 | (defn id 231 | "A map identifier for a node." 232 | [node] 233 | {:host (:host node) 234 | :port (:port node)}) 235 | 236 | (defn connect 237 | "Opens a new client connection to the given peer, identified by host and 238 | port. Returns a netty Channel." 239 | [node peer] 240 | (assert (started? node)) 241 | (assert (integer? (:port peer)) 242 | (str (pr-str peer) " peer doesn't have an integer port")) 243 | (assert (string? (:host peer)) 244 | (str (pr-str peer) " peer doesn't have a string host")) 245 | 246 | (let [^Bootstrap bootstrap (:bootstrap @(:client node))] 247 | (locking bootstrap 248 | (let [cf (.. bootstrap 249 | (remoteAddress ^String (:host peer) (int (:port peer))) 250 | (connect)) 251 | ch (.channel cf)] 252 | ; Store the peer ID in the channel's attributes for later. 253 | (.. ch 254 | (attr peer-attr) 255 | (set (string-id peer))) 256 | (.sync cf) 257 | ch)))) 258 | 259 | (defn ^Channel conn 260 | "Find or create a client to the given peer." 261 | [node peer] 262 | (let [id (string-id peer) 263 | conns (:conns node)] 264 | ; Standard double-locking strategy 265 | (or (get @conns id) 266 | ; Open a connection 267 | (let [conn (connect node peer)] 268 | ; Lock and see if we lost a race 269 | (locking conns 270 | (if-let [existing-conn (get @conns id)] 271 | (do 272 | ; Abandon new conn 273 | (.close ^Channel conn) 274 | existing-conn) 275 | (do 276 | ; Use new conn 277 | (swap! conns assoc id conn) 278 | conn))))))) 279 | 280 | (defn send! 281 | "Sends a message to a peer." 282 | [node peer msg] 283 | (assert (started? node)) 284 | (let [c (conn node peer)] 285 | ; (log- (:port node) "->" (:port peer) (pr-str msg)) 286 | (.write c msg)) 287 | node) 288 | 289 | (defn send-sync! 290 | "Sends a message to a peer, blocking for the write to complete." 291 | [node peer msg] 292 | (-> (conn node peer) 293 | (.write msg) 294 | (.sync)) 295 | node) 296 | 297 | (defn compile-handler 298 | "Given a set of n functions, returns a function which invokes each function 299 | with a received message, returning the first non-nil return value. The first 300 | function which returns non-nil terminates execution." 301 | [functions] 302 | (if (empty? functions) 303 | (constantly nil) 304 | (fn compiled-handler [msg] 305 | (loop [[f & fns] functions] 306 | (when f 307 | (if-let [value (f msg)] 308 | value 309 | (recur fns))))))) 310 | 311 | (defn add-handler! 312 | "Registers a handler for incoming messages." 313 | [node f] 314 | (locking (:handler node) 315 | (if (deref (:handler node)) 316 | (throw (IllegalStateException. 317 | "node already running; can't add handlers now")) 318 | (swap! (:handlers node) conj f))) 319 | node) 320 | 321 | ; High-level messaging primitives 322 | 323 | ; A Request represents a particular request made to N nodes. It includes a 324 | ; unique identifier, the time (in nanoTime) the request is valid until, some 325 | ; debugging information, the number of responses necessary to satisfy the 326 | ; request, and a mutable list of responses received. Finally, takes a function 327 | ; to invoke with responses, when r have been accrued. 328 | (defrecord Request [debug ^bytes id ^long max-time ^int r responses f]) 329 | 330 | (defn ^Request new-request 331 | "Constructs a new Request. Automatically generates an ID, and computes the 332 | max-time by adding the timeout to the current time. 333 | 334 | Options: 335 | 336 | :debug Debugging info 337 | :timeout Milliseconds to wait 338 | :r Number of responses to await 339 | :f A callback to receive response" 340 | [opts] 341 | (Request. (:debug opts) 342 | (flake/id) 343 | (+ (flake/linear-time) (get opts :timeout 5000)) 344 | (or (:r opts) 1) 345 | (atom (list)) 346 | (:f opts))) 347 | 348 | (defn request! 349 | "Given a node, sends a message to several nodes, and invokes f when r have 350 | responded within the timeout. The message must be a map, and a :request-id 351 | field will be automatically assoc'ed on to it. Returns the Request. Options: 352 | 353 | :debug 354 | :timeout 355 | :r 356 | :f" 357 | [node peers opts msg] 358 | (assert (started? node)) 359 | (let [r (majority (count peers)) 360 | req (new-request (assoc opts :r (or (:r opts) 361 | (majority (count peers))))) 362 | id (.id req) 363 | msg (assoc msg :request-id id)] 364 | 365 | ; Save request 366 | (swap! (:requests node) assoc (Bytes. id) req) 367 | 368 | ; Send messages 369 | (doseq [peer peers] 370 | (try 371 | (send! node peer msg) 372 | (catch io.netty.channel.ChannelException e 373 | (handle-response! (:requests node) id nil)))))) 374 | 375 | (defmacro req! 376 | "Like request!, but with the body captured into the callback function. 377 | Captures file and line number and merges into the debug map. 378 | 379 | (req! node some-peers {:timeout 3 :r 2} 380 | {:type :anti-entropy :keys [a b c]} 381 | 382 | [responses] 383 | (assert (= 3 responses)) 384 | (doseq [r responses] 385 | (prn :got r)))" 386 | [node peers opts msg binding-form & body] 387 | (let [debug (meta &form)] 388 | `(let [opts# (-> ~opts 389 | (update-in [:debug] merge ~debug) 390 | (assoc :f (fn ~binding-form ~@body)))] 391 | (request! ~node ~peers opts# ~msg)))) 392 | 393 | (defmacro sync-req! 394 | "Like request!, but *returns* a list of responses, synchronously." 395 | [node peers opts msg] 396 | (let [debug (meta &form)] 397 | `(let [p# (promise) 398 | opts# (-> ~opts 399 | (update-in [:debug] merge ~debug) 400 | (assoc :f (fn [rs#] (deliver p# rs#))))] 401 | (request! ~node ~peers opts# ~msg) 402 | @p#))) 403 | 404 | (defn expired-request? 405 | "Is the given request past its timeout?" 406 | [request] 407 | (< (:max-time request) (flake/linear-time))) 408 | 409 | (defn gc-requests! 410 | "Given an atom mapping ids to Requests, expires timed-out requests." 411 | [reqs] 412 | (let [expired (->> reqs 413 | deref 414 | (filter (comp expired-request? val)))] 415 | (doseq [[id ^Request req] expired] 416 | (swap! reqs dissoc id) 417 | ((.f req) @(.responses req))))) 418 | 419 | (defn shutdown-requests! 420 | "Given a mapping of ids to Requests, expire all of the requests." 421 | [requests] 422 | (doseq [[id ^Request req] requests] 423 | ((.f req) @(.responses req)))) 424 | 425 | (defn periodically-gc-requests! 426 | "Starts a thread to GC requests. Returns a promise which, when set to false, 427 | shuts down." 428 | [reqs] 429 | (let [p (promise)] 430 | (future 431 | (loop [] 432 | (when (deref p 1000 true) 433 | (try 434 | (gc-requests! reqs) 435 | (catch Throwable t 436 | (warn t "Caught while GC-ing request map"))) 437 | (recur)))) 438 | p)) 439 | 440 | (defn handle-response! 441 | "Given an atom mapping ids to Requests, slots this response into the 442 | appropriate request. When the response is fulfilled, removes it from the 443 | requests atom." 444 | [reqs id response] 445 | (let [id (Bytes. id)] 446 | (when-let [req ^Request (get @reqs id)] 447 | (let [responses (swap! (.responses req) conj response)] 448 | (when (= (:r req) (count responses)) 449 | (swap! reqs dissoc id) 450 | ((.f req) responses)))))) 451 | 452 | ; Node constructor 453 | (defn node 454 | "Creates a new network node. Options: 455 | 456 | :host 457 | :port 458 | :server? Should we be a server as well? (default true) 459 | 460 | Handlers are invoked with each message, and may return an response to be sent 461 | back to the client." 462 | [opts] 463 | (let [host (get opts :host "127.0.0.1") 464 | port (get opts :port 13000)] 465 | {:host host 466 | :port port 467 | :handlers (atom []) 468 | :handler (atom nil) 469 | :server? (get opts :server? true) 470 | :server (atom nil) 471 | :client (atom nil) 472 | :conns (atom {}) 473 | :gc (atom nil) 474 | :requests (atom {})})) 475 | 476 | ;; Lifecycle management--start, stop, etc. 477 | (defn started? 478 | "Is node started?" 479 | [node] 480 | (when-let [h (:handler node)] 481 | (not (nil? @h)))) 482 | 483 | (defn start! 484 | "Starts the node, when all handlers have been registered." 485 | [node] 486 | (locking (:handler node) 487 | (when-not (started? node) 488 | (let [handler (compile-handler (deref (:handlers node)))] 489 | (reset! (:handler node) handler) 490 | (when (:server? node) 491 | (reset! (:server node) (server node))) 492 | (reset! (:client node) (client node)) 493 | (reset! (:gc node) (periodically-gc-requests! (:requests node))) 494 | node)))) 495 | 496 | (defn close-conns! 497 | "Closes all open connections on a node." 498 | [node] 499 | (doseq [[peer ^Channel channel] @(:conns node)] 500 | (.close channel))) 501 | 502 | (defn shutdown-client! 503 | "Shuts down a netty client for a node." 504 | [client] 505 | (when client 506 | (let [^NioEventLoopGroup g (:group client)] 507 | (.shutdown g) 508 | (.awaitTermination g 30 TimeUnit/SECONDS)))) 509 | 510 | (defn shutdown-server! 511 | "Shuts down a netty server for a node." 512 | [server] 513 | (when server 514 | (let [^NioEventLoopGroup g1 (:boss-group server) 515 | ^NioEventLoopGroup g2 (:worker-group server)] 516 | (.shutdown g1) 517 | (.shutdown g2) 518 | (.awaitTermination g1 30 TimeUnit/SECONDS) 519 | (.awaitTermination g2 30 TimeUnit/SECONDS)))) 520 | 521 | (defn shutdown! 522 | "Shuts down a node." 523 | [node] 524 | (locking (:handler node) 525 | (close-conns! node) 526 | (shutdown-client! @(:client node)) 527 | (shutdown-server! @(:server node)) 528 | (shutdown-requests! @(:requests node)) 529 | (deliver @(:gc node) false) 530 | (reset! (:requests node) {}) 531 | (reset! (:conns node) {}) 532 | (reset! (:handler node) nil) 533 | node)) 534 | -------------------------------------------------------------------------------- /src/skuld/node.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.node 2 | "A single node in the Skuld cluster. Manages any number of vnodes." 3 | (:use skuld.util 4 | clojure.tools.logging) 5 | (:require [clj-helix.manager :as helix] 6 | clj-helix.admin 7 | clj-helix.fsm 8 | [clj-helix.route :as route] 9 | [skuld.aae :as aae] 10 | [skuld.clock-sync :as clock-sync] 11 | [skuld.curator :as curator] 12 | [skuld.db :as db] 13 | [skuld.flake :as flake] 14 | [skuld.net :as net] 15 | [skuld.politics :as politics] 16 | [skuld.task :as task] 17 | [skuld.vnode :as vnode]) 18 | (:import (java.util Arrays) 19 | com.aphyr.skuld.Bytes)) 20 | 21 | ; DEAL WITH IT 22 | ; (circular deps) 23 | (in-ns 'skuld.http) 24 | (clojure.core/declare service) 25 | (clojure.core/declare shutdown!) 26 | (in-ns 'skuld.node) 27 | 28 | ;; Logging 29 | (defn trace-log-prefix 30 | "Prefix for trace log messages" 31 | [node] 32 | (format "%s:%d:" (:host node) (:port node))) 33 | 34 | (defmacro trace-log 35 | "Log a message with context" 36 | [node & args] 37 | `(let [node-prefix# (trace-log-prefix ~node)] 38 | (info node-prefix# ~@args))) 39 | 40 | ;; 41 | 42 | (defn vnodes 43 | "Returns a map of partitions to vnodes for a node." 44 | [node] 45 | @(:vnodes node)) 46 | 47 | (defn vnode 48 | "Returns a particular vnode for a node." 49 | [node partition-id] 50 | (get (vnodes node) partition-id)) 51 | 52 | (defn tasks 53 | "Given a node, emits all tasks for all vnodes." 54 | [node] 55 | (->> node vnodes vals (mapcat vnode/tasks))) 56 | 57 | (defn helix-admin 58 | "Returns a HelixAdmin for a given node, via its participant." 59 | [node] 60 | (-> node :participant helix/admin)) 61 | 62 | (defn cluster-name 63 | "The name of a cluster a node is participating in." 64 | [node] 65 | (-> node :participant helix/cluster-name)) 66 | 67 | (def num-partitions 68 | "The number of partitions in the cluster." 69 | (memoize (fn [node] 70 | (-> node 71 | helix-admin 72 | (clj-helix.admin/resource-ideal-state 73 | (cluster-name node) 74 | :skuld) 75 | .getNumPartitions)))) 76 | 77 | (def num-replicas 78 | "How many replicas are there for the Skuld resource?" 79 | (memoize (fn [node] 80 | (-> node 81 | helix-admin 82 | (clj-helix.admin/resource-ideal-state 83 | (cluster-name node) 84 | :skuld) 85 | .getReplicas 86 | Integer.)))) 87 | 88 | (defn partition-name 89 | "Calculates which partition is responsible for a given ID." 90 | [node ^Bytes id] 91 | (str "skuld_" (-> id 92 | .bytes 93 | Arrays/hashCode 94 | (mod (num-partitions node))))) 95 | 96 | (defn all-partitions 97 | "A list of all partitions in the system." 98 | [node] 99 | (->> node 100 | num-partitions 101 | range 102 | (map (partial str "skuld_")))) 103 | 104 | (defn peers 105 | "All peers which own a partition, or all peers in the cluster." 106 | ([node] 107 | (map #(select-keys % [:host :port]) 108 | (route/instances (:router node) :skuld :peer))) 109 | ([node part] 110 | (map #(select-keys % [:host :port]) 111 | (route/instances (:router node) :skuld part :peer)))) 112 | 113 | (defn preflist 114 | "Returns a set of nodes responsible for a Bytes id." 115 | [node ^Bytes id] 116 | (assert (not (nil? id))) 117 | (peers node (partition-name node id))) 118 | 119 | (defn enqueue! 120 | "Proxies to enqueue-local on all nodes in the preflist for this task." 121 | [node msg] 122 | (let [task (task/task (:task msg)) 123 | id (:id task)] 124 | (let [r (or (:w msg) 1) 125 | preflist (preflist node id) 126 | _ (assert (<= r (count preflist)) 127 | (str "need " r " vnodes but only " 128 | (pr-str preflist) " known in local preflist")) 129 | responses (net/sync-req! (:net node) 130 | preflist 131 | {:r r} 132 | {:type :enqueue-local 133 | :task task}) 134 | acks (remove :error responses)] 135 | (if (<= r (count acks)) 136 | {:n (count acks) 137 | :id id} 138 | {:n (count acks) 139 | :id id 140 | :error (str "not enough acks from " (prn-str preflist)) 141 | :responses responses})))) 142 | 143 | (defn enqueue-local! 144 | "Enqueues a message on the local vnode for this task." 145 | [node msg] 146 | (let [task (:task msg) 147 | part (partition-name node (:id task))] 148 | (if-let [vnode (vnode node part)] 149 | (do (vnode/merge-task! vnode task) 150 | (trace-log node "enqueue-local: enqueued id" (:id task) "on vnode" (vnode/full-id vnode) "for task:" task) 151 | {:task-id (:id task)}) 152 | {:error (str "I don't have partition" part "for task" (:id task))}))) 153 | 154 | (defn get-task 155 | "Gets the current state of a task." 156 | [node msg] 157 | (let [id (:id msg) 158 | r (or (:r msg) 2) 159 | responses (net/sync-req! (:net node) 160 | (preflist node id) 161 | {:r r} 162 | {:type :get-task-local 163 | :id id}) 164 | acks (remove :error responses) 165 | _ (trace-log node "get-task:" responses) 166 | task (->> responses (map :task) (reduce task/merge))] 167 | (if (<= r (count acks)) 168 | {:n (count acks) 169 | :task task} 170 | {:n (count acks) 171 | :task task 172 | :error "not enough acks" 173 | :responses responses}))) 174 | 175 | (defn get-task-local 176 | "Gets the current state of a task from a local vnode" 177 | [node msg] 178 | (let [id (:id msg) 179 | part (partition-name node id)] 180 | (if-let [vnode (vnode node part)] 181 | {:task (vnode/get-task vnode id)} 182 | {:error (str "I don't have partition" part "for task" id)}))) 183 | 184 | (defn count-tasks 185 | "Estimates the total number of tasks in the system." 186 | [node msg] 187 | (let [parts (set (all-partitions node)) 188 | counts (atom {}) 189 | done (promise)] 190 | 191 | ; Issue requests to all nodes for their local couns 192 | (doseq [peer (route/instances (:router node) :skuld :peer)] 193 | (net/req! (:net node) [peer] {:r 1} {:type :count-tasks-local} 194 | [[response]] 195 | (let [remote-counts (:partitions response) 196 | counts (swap! counts 197 | (partial merge-with max remote-counts))] 198 | (when (= parts (set (keys counts))) 199 | ; We've accrued a response for each partition. 200 | (deliver done 201 | {:partitions counts 202 | :count (reduce + (vals counts))}))))) 203 | 204 | (deref done 5000 {:error "timed out" :partitions @counts}))) 205 | 206 | (defn count-tasks-local 207 | "Estimates the total number of tasks on the local node." 208 | [node msg] 209 | (let [partition-counts (reduce (fn [counts [k vnode]] 210 | (if (vnode/active? vnode) 211 | (assoc counts k (vnode/count-tasks vnode)) 212 | counts)) 213 | {} 214 | (vnodes node))] 215 | (trace-log node "count-tasks-local:" partition-counts) 216 | {:partitions partition-counts})) 217 | 218 | (defn cover 219 | "Returns a map of nodes to lists of partitions on that node, such that each 220 | partition appears exactly once. Useful when you want to know something about 221 | every partition." 222 | [node] 223 | (->> node 224 | all-partitions 225 | (reduce (fn [m part] 226 | (if-let [peer (first (peers node part))] 227 | (update-in m [peer] conj part) 228 | (throw (RuntimeException. 229 | (str "no known nodes for partition " part))))) 230 | {}))) 231 | 232 | (defn coverage 233 | "Issues a query to a cover of nodes. The message sent to each peer will 234 | include a new key :partitions with a value like [\"skuld_0\" \"skuld_13\" 235 | ...]. The responses for this message should look like: 236 | 237 | {:partitions {\"skuld_0\" some-value \"skuld_13\" something-else ...} 238 | 239 | Coverage will return a map of partition names to one value for each 240 | partition." 241 | [node msg] 242 | (let [responses (atom {}) 243 | done (promise) 244 | cover (cover node) 245 | all-parts (set (all-partitions node))] 246 | (doseq [[peer parts] cover] 247 | (assert peer) 248 | (net/req! (:net node) [peer] {} (assoc msg :partitions parts) 249 | [[response]] 250 | (let [responses (swap! responses merge (:partitions response))] 251 | (when (= all-parts (set (keys responses))) 252 | (deliver done responses))))) 253 | (or (deref done 5000 false) 254 | (throw (RuntimeException. 255 | "did not receive a complete set of responses for coverage query"))))) 256 | 257 | (defn list-tasks 258 | "Lists all tasks in the system." 259 | [node msg] 260 | {:tasks (->> {:type :list-tasks-local} 261 | (coverage node) 262 | vals 263 | (apply sorted-interleave-by :id))}) 264 | 265 | (defn list-tasks-local 266 | [node msg] 267 | {:partitions (reduce (fn [m part] 268 | (if-let [vnode (vnode node part)] 269 | (if (vnode/active? vnode) 270 | (assoc m part (vnode/tasks vnode)) 271 | m) 272 | m)) 273 | {} 274 | (:partitions msg))}) 275 | 276 | (defn count-queue 277 | "Estimates the number of enqueued tasks." 278 | [node msg] 279 | ; Issue requests to all nodes for their local counts 280 | (let [peers (peers node) 281 | queue (:queue msg)] 282 | {:count (->> (net/sync-req! (:net node) peers {:r (count peers)} 283 | {:type :count-queue-local 284 | :queue queue}) 285 | (map :count) 286 | (reduce +))})) 287 | 288 | (defn count-queue-local 289 | "Estimates the number of enqueued tasks on this node." 290 | [node msg] 291 | (let [queue (:queue msg)] 292 | {:count (->> node 293 | vnodes 294 | vals 295 | (map #(vnode/count-queue % queue)) 296 | (reduce +))})) 297 | 298 | (defn claim-local! 299 | "Tries to claim a task from a local vnode." 300 | [node msg] 301 | ; Find the next task 302 | (loop [[vnode & vnodes] (->> node 303 | vnodes 304 | vals 305 | (filter vnode/leader?) 306 | shuffle)] 307 | (let [queue (:queue msg) 308 | dt (or (:dt msg) 10000) 309 | task (when vnode 310 | (try 311 | (if-let [ta (vnode/claim! vnode queue dt)] 312 | (do 313 | (trace-log node (format "claim-local: claim from %s on %s returned task: %s" (vnode/full-id vnode) queue ta)) 314 | ta) 315 | :retry) 316 | (catch IllegalStateException ex 317 | (trace-log node (format "claim-local: failed to claim from %s on %s: %s" (vnode/full-id vnode) queue (.getMessage ex))) 318 | :retry) 319 | (catch Throwable t 320 | (warn t (trace-log-prefix node) "caught while claiming from vnode" (vnode/full-id vnode) "on" queue) 321 | :retry)))] 322 | (if (not= :retry task) 323 | {:task task} 324 | (recur vnodes))))) 325 | 326 | (defn claim! 327 | "Tries to claim a task." 328 | [node msg] 329 | ; Try a local claim first 330 | (or (let [t (claim-local! node msg)] 331 | (and (:task t) t)) 332 | ; Ask each peer in turn for a task 333 | (loop [[peer & peers] (shuffle (disj (set (peers node)) 334 | (net/id (:net node))))] 335 | (if-not peer 336 | ; Done 337 | {} 338 | (do 339 | (let [[response] (net/sync-req! (:net node) [peer] {} 340 | (assoc msg :type :claim-local))] 341 | (if (:task response) 342 | response 343 | (recur peers)))))))) 344 | 345 | (defn request-claim! 346 | "Accepts a request from a leader to claim a given task." 347 | [node msg] 348 | (vnode/request-claim! 349 | (->> msg 350 | :id 351 | (partition-name node) 352 | (vnode node)) 353 | msg)) 354 | 355 | (defn complete-local! 356 | "Completes a given task on a local vnode." 357 | [node msg] 358 | (let [part (->> msg :task-id (partition-name node))] 359 | (if-let [vnode (vnode node part)] 360 | (do (vnode/complete! vnode msg) 361 | {:w 1}) 362 | {:error (str "I don't have partition" part "for task" (:task-id msg))}))) 363 | 364 | (defn complete! 365 | "Completes a given task in a given run. Proxies to all nodes owning that 366 | task." 367 | [node msg] 368 | (let [w (or (:w msg) 2) 369 | responses (net/sync-req! (:net node) 370 | (preflist node (:task-id msg)) 371 | {:r w} 372 | (merge msg {:type :complete-local 373 | :time (flake/linear-time)})) 374 | acks (remove :error responses) 375 | w' (reduce + (map :w acks))] 376 | (if (<= w w') 377 | {:w w'} 378 | {:w w' 379 | :error "not enough nodes acknowledged request for complete" 380 | :responses responses}))) 381 | 382 | (defn wipe! 383 | "Wipes all data clean." 384 | [node msg] 385 | (net/sync-req! (:net node) (peers node) {} {:type :wipe-local}) 386 | {}) 387 | 388 | (defn wipe-local! 389 | "Wipe all data on the local node." 390 | [node msg] 391 | (->> node vnodes vals 392 | (pmap (fn [v] 393 | (try (vnode/wipe! v) 394 | (catch RuntimeException e nil)))) 395 | dorun) 396 | {}) 397 | 398 | (defn request-vote! 399 | "Handles a request for a vote from another node." 400 | [node msg] 401 | (if-let [vnode (vnode node (:partition msg))] 402 | (vnode/request-vote! vnode msg) 403 | (do 404 | {:error (str (net/id (:net node)) 405 | " has no vnode for " (:partition msg))}))) 406 | 407 | (defn heartbeat! 408 | "Handles a request for a heartbeat from a leader." 409 | [node msg] 410 | (if-let [vnode (vnode node (:partition msg))] 411 | (vnode/heartbeat! vnode msg) 412 | (do 413 | {:error (str (net/id (:net node)) 414 | " has no vnode for " (:partition msg))}))) 415 | 416 | (defn handler 417 | "Returns a fn which handles messages for a node." 418 | [node] 419 | (fn handler [msg] 420 | ((case (:type msg) 421 | :enqueue enqueue! 422 | :enqueue-local enqueue-local! 423 | :get-task get-task 424 | :get-task-local get-task-local 425 | :count-tasks count-tasks 426 | :count-tasks-local count-tasks-local 427 | :count-queue count-queue 428 | :count-queue-local count-queue-local 429 | :list-tasks list-tasks 430 | :list-tasks-local list-tasks-local 431 | :claim claim! 432 | :claim-local claim-local! 433 | :request-claim request-claim! 434 | :complete complete! 435 | :complete-local complete-local! 436 | :wipe wipe! 437 | :wipe-local wipe-local! 438 | :request-vote request-vote! 439 | :heartbeat heartbeat! 440 | (constantly {:error (str "unknown message type" (:type msg))})) 441 | node msg))) 442 | 443 | (def fsm-def (clj-helix.fsm/fsm-definition 444 | {:name :skuld 445 | :states {:DROPPED {:transitions :offline} 446 | :offline {:initial? true 447 | :transitions [:peer :DROPPED]} 448 | :peer {:priority 1 449 | :upper-bound :R 450 | :transitions :offline}}})) 451 | 452 | (defn new-vnode 453 | "Creates a new vnode for the given partition." 454 | [node part] 455 | (vnode/vnode {:partition part 456 | :curator (:curator node) 457 | :router (:router node) 458 | :net (:net node)})) 459 | 460 | (defn fsm 461 | "Compiles a new FSM to manage a vnodes map. Takes an atom of partitions to 462 | vnodes, a net node, and a promise of a router." 463 | [vnodes curator net routerp] 464 | (clj-helix.fsm/fsm 465 | fsm-def 466 | (:offline :peer [part m c] 467 | (try 468 | (locking vnodes 469 | (trace-log net part "coming online") 470 | (if-let [existing (get @vnodes part)] 471 | (vnode/revive! existing) 472 | (swap! vnodes assoc part 473 | (vnode/vnode {:partition part 474 | :curator curator 475 | :router @routerp 476 | :net net})))) 477 | (catch Throwable t 478 | (fatal t (:port net) "bringing" part "online")))) 479 | 480 | (:offline :DROPPED [part m c] 481 | (try 482 | (locking vnodes 483 | (trace-log net part "dropped") 484 | (when-let [vnode (get @vnodes part)] 485 | (vnode/shutdown! vnode) 486 | (swap! vnodes dissoc part))) 487 | (catch Throwable t 488 | (fatal t (trace-log-prefix net) "dropping" part)))) 489 | 490 | (:peer :offline [part m c] 491 | (try 492 | (locking vnodes 493 | (trace-log net part "going offline") 494 | (when-let [v (get @vnodes part)] 495 | (vnode/zombie! v))) 496 | (catch Throwable t 497 | (fatal t (trace-log-prefix net) "taking" part "offline")))))) 498 | 499 | (defn start-local-vnodes! 500 | "Spins up a local zombie vnode for any local data." 501 | [node] 502 | (let [vnodes (:vnodes node)] 503 | (->> node 504 | all-partitions 505 | (map (fn [part] 506 | (locking vnodes 507 | (when (and (not (get @vnodes part)) 508 | (db/local-data? {:partition part 509 | :host (:host node) 510 | :port (:port node) 511 | :ext "level"})) 512 | (trace-log node "spooling up zombie vnode" part) 513 | (let [v (new-vnode node part)] 514 | (vnode/zombie! v) 515 | (swap! vnodes assoc part v)))))) 516 | dorun))) 517 | 518 | (defn node 519 | "Creates a new node with the given options. 520 | 521 | :zookeeper \"localhost:2181\" 522 | :cluster :skuld 523 | :host \"127.0.0.1\" 524 | :port 13000" 525 | [opts] 526 | (let [zk (get opts :zookeeper "localhost:2181") 527 | curator (curator/framework zk "skuld") 528 | host (get opts :host "127.0.0.1") 529 | port (get opts :port 13000) 530 | cluster (get opts :cluster :skuld) 531 | vnodes (atom {}) 532 | net (net/node {:host host 533 | :port port}) 534 | routerp (promise) 535 | fsm (fsm vnodes curator net routerp) 536 | 537 | ; Initialize services 538 | controller (helix/controller {:zookeeper zk 539 | :cluster cluster 540 | :instance {:host host :port port}}) 541 | participant (helix/participant {:zookeeper zk 542 | :cluster cluster 543 | :instance {:host host :port port} 544 | :fsm fsm}) 545 | router (clj-helix.route/router! participant) 546 | _ (deliver routerp router) 547 | clock-sync (clock-sync/service net router vnodes) 548 | aae (aae/service net router vnodes) 549 | politics (politics/service vnodes) 550 | 551 | ; Construct node 552 | node {:host host 553 | :port port 554 | :net net 555 | :curator curator 556 | :router router 557 | :clock-sync clock-sync 558 | :aae aae 559 | :politics politics 560 | :participant participant 561 | :controller controller 562 | :vnodes vnodes 563 | :running (atom true)} 564 | 565 | ; Initialize HTTP service 566 | http (skuld.http/service node (+ port 100)) 567 | node (assoc node :http http)] 568 | 569 | ; Final startup sequence 570 | (start-local-vnodes! node) 571 | (net/add-handler! net (handler node)) 572 | (net/start! net) 573 | 574 | node)) 575 | 576 | (defn wait-for-peers 577 | "Blocks until all partitions are known to exist on a peer, then returns node." 578 | [node] 579 | (while (empty? (all-partitions node)) 580 | (trace-log node "waiting-for-partition-list") 581 | (Thread/sleep 10)) 582 | 583 | (while (->> node 584 | all-partitions 585 | (map (partial peers node)) 586 | (some (comp (partial > 2) count))) 587 | (debug (:port node) " preflists are " (->> node 588 | all-partitions 589 | (map #(vector % (peers node %))) 590 | (into (sorted-map)))) 591 | (Thread/sleep 1000)) 592 | 593 | (debug (:port node) " preflists are " (->> node 594 | all-partitions 595 | (map #(vector % (peers node %))) 596 | (into (sorted-map)))) 597 | 598 | 599 | node) 600 | 601 | (defn controller 602 | "Creates a new controller, with the given options. 603 | 604 | :zookeeper \"localhost:2181\" 605 | :cluster :skuld 606 | :host \"127.0.0.1\" 607 | :port 13000" 608 | [opts] 609 | (let [zk (get opts :zookeeper "localhost:2181") 610 | host (get opts :host "127.0.0.1") 611 | port (get opts :port 13000) 612 | cluster (get opts :cluster :skuld)] 613 | {:host host 614 | :port port 615 | :zookeeper zk 616 | :controller (helix/controller {:zookeeper zk 617 | :cluster cluster 618 | :instance {:host host :port port}})})) 619 | 620 | (defn shutdown? 621 | [node] 622 | (-> node :running deref not)) 623 | 624 | (defn shutdown! 625 | "Shuts down a node." 626 | [node] 627 | (locking node 628 | (when-not (shutdown? node) 629 | (when-let [c (:clock-sync node)] (clock-sync/shutdown! c)) 630 | (when-let [aae (:aae node)] (aae/shutdown! aae)) 631 | (when-let [p (:politics node)] (politics/shutdown! p)) 632 | (when-let [net (:net node)] (net/shutdown! net)) 633 | (when-let [c (:curator node)] (curator/shutdown! c)) 634 | (when-let [h (:http node)] (skuld.http/shutdown! h)) 635 | 636 | (->> (select-keys node [:participant :controller]) 637 | vals 638 | (remove nil?) 639 | (map helix/disconnect!) 640 | dorun) 641 | 642 | (locking (:vnodes node) 643 | (->> node 644 | vnodes 645 | vals 646 | (pmap vnode/shutdown!) 647 | dorun)) 648 | 649 | (reset! (:running node) false)))) 650 | -------------------------------------------------------------------------------- /src/skuld/politics.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.politics 2 | "Periodically initiates the election cycle." 3 | (:require [skuld.vnode :as vnode]) 4 | (:use clojure.tools.logging)) 5 | 6 | (defn service 7 | "Creates a new politics service." 8 | [vnodes] 9 | (let [running (promise)] 10 | (future 11 | (loop [] 12 | (try 13 | (when-let [vnodes (-> vnodes deref vals)] 14 | (->> vnodes 15 | shuffle 16 | (pmap 17 | (fn [vnode] 18 | (try 19 | (Thread/sleep (rand-int 300)) 20 | (if (vnode/leader? vnode) 21 | (vnode/broadcast-heartbeat! vnode) 22 | (vnode/elect! vnode)) 23 | (catch Throwable t 24 | (warn t "exception while electing" (:partition vnode)))))) 25 | dorun)) 26 | (catch Throwable t 27 | (warn t "exception in election cycle"))) 28 | (when (deref running 1000 true) 29 | (recur)))) 30 | running)) 31 | 32 | (defn shutdown! 33 | "Shuts down a politics service." 34 | [politics] 35 | (deliver politics false)) 36 | -------------------------------------------------------------------------------- /src/skuld/queue.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.queue 2 | "A vnode service which actually provides queuing semantics. You know, 3 | ordering, priorities, etc. 4 | 5 | Our general strategy with this queue is to be approximately consistent. If 6 | claim fails, come back and get another task. If a task is missing from this 7 | queue, AAE will recover it when it does a scan over the DB. 8 | 9 | There's one set of queues per vnode. Each named queue is a rough approximation 10 | of all tasks that could be claimed on that vnode." 11 | (:import (java.util.concurrent TimeUnit 12 | ConcurrentSkipListSet)) 13 | (:use [skuld.util :exclude [update!]] 14 | clojure.tools.logging) 15 | (:require [skuld.task :as task] 16 | [skuld.flake :as flake])) 17 | 18 | (defrecord Task [id priority] 19 | Comparable 20 | (compareTo [a b] 21 | (compare+ a b :priority :id))) 22 | 23 | (defn queues 24 | "Creates a new set of queues." 25 | [] 26 | (atom {})) 27 | 28 | (defn named-queue 29 | [] 30 | {:queue (ConcurrentSkipListSet.) 31 | :last-modified (atom (flake/linear-time))}) 32 | 33 | (defn touch-named-queue 34 | [named-queue] 35 | (swap! (:last-modified named-queue) max (flake/linear-time))) 36 | 37 | 38 | (defn update! 39 | "Update a task in the queue" 40 | [queues task] 41 | (let [queue-name (:queue task)] 42 | (if-not queue-name 43 | (throw (IllegalArgumentException. (str "Task did not specify a queue: " task)))) 44 | 45 | (let [named-queue (if-let [queue (get @queues queue-name)] 46 | queue 47 | (get 48 | (swap! queues 49 | (fn [queues] 50 | (if-not (get queues queue-name) 51 | (assoc queues queue-name (named-queue)) 52 | queues))) 53 | queue-name)) 54 | q ^ConcurrentSkipListSet (:queue named-queue)] 55 | 56 | ; Mark the last time the queue was used 57 | (swap! (:last-modified named-queue) max (flake/linear-time)) 58 | 59 | (if (or (nil? task) 60 | (task/claimed? task) 61 | (task/completed? task)) 62 | (.remove q (Task. (:id task) 63 | (:priority task))) 64 | (.add q (Task. (:id task) 65 | (:priority task))))))) 66 | 67 | (defn poll! 68 | [queues queue-name] 69 | (if-let [named-queue (get @queues queue-name)] 70 | (.pollFirst ^ConcurrentSkipListSet (:queue named-queue)))) 71 | 72 | (defn count-queue 73 | [queues queue-name] 74 | (if-let [named-queue (get @queues queue-name)] 75 | (count (:queue named-queue)) 76 | 0)) -------------------------------------------------------------------------------- /src/skuld/scanner.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.scanner 2 | "Periodically scans over vnodes to rebuild internal data structures." 3 | (:use clojure.tools.logging) 4 | (:require [skuld.queue :as queue])) 5 | 6 | ; DEAL WITH IT 7 | (in-ns 'skuld.vnode) 8 | (clojure.core/declare tasks) 9 | (in-ns 'skuld.scanner) 10 | 11 | 12 | (def interval 13 | "How long to sleep between scanning runs, in ms" 14 | 1000) 15 | 16 | (defn scan! 17 | "Scans over all tasks in a vnode." 18 | [queue vnode] 19 | (doseq [task (skuld.vnode/tasks vnode)] 20 | (queue/update! queue task))) 21 | 22 | (defn service 23 | "Starts a new scanning service. Takes an atom wrapping a map of partitions to 24 | vnodes, and a queue." 25 | [vnode queue] 26 | (let [running (promise)] 27 | (future 28 | (when (deref running interval true) 29 | (loop [] 30 | (try 31 | (scan! queue vnode) 32 | (catch Throwable t 33 | (warn t "queue refiller caught"))) 34 | 35 | (when (deref running interval true) 36 | (recur))))) 37 | 38 | running)) 39 | 40 | (defn shutdown! 41 | "Stops a scanning service." 42 | [scanner] 43 | (deliver scanner false)) 44 | -------------------------------------------------------------------------------- /src/skuld/task.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.task 2 | "Operations on individual tasks. Tasks have this structure: 3 | 4 | {:id (Bytes) a 20-byte unique identifier for the task 5 | :data (bytes) an arbitrary payload 6 | :claims [...] a vector of claims} 7 | 8 | A claim is a map of: 9 | 10 | {:start (long) milliseconds in linear time 11 | :end (long) milliseconds in linear time 12 | :completed (long) milliseconds in linear time}" 13 | (:refer-clojure :exclude [merge]) 14 | (:use skuld.util) 15 | (:require [skuld.flake :as flake] 16 | [skuld.util :refer [fress-read fress-write]]) 17 | (:import com.aphyr.skuld.Bytes)) 18 | 19 | (def clock-skew-buffer 20 | "We allow nodes and clocks to drift by this much." 21 | (* 1000 60)) 22 | 23 | (defn task 24 | "Creates a new task around the given map." 25 | [task] 26 | (clojure.core/merge task 27 | {:id (or (:id task) (Bytes. (flake/id))) 28 | :claims []})) 29 | 30 | (defn new-claim 31 | "Creates a new claim, valid for dt milliseconds." 32 | [dt] 33 | (let [now (flake/linear-time)] 34 | {:start now 35 | :end (+ now dt) 36 | :completed nil})) 37 | 38 | (defn valid-claim? 39 | "Is a claim currently valid?" 40 | [claim] 41 | (when-let [end (:end claim)] 42 | (< (flake/linear-time) 43 | (+ end clock-skew-buffer)))) 44 | 45 | (defn claimed? 46 | "Is this task currently claimed?" 47 | [task] 48 | (try 49 | (some valid-claim? (:claims task)) 50 | (catch Exception e 51 | (throw e)))) 52 | 53 | (declare completed?) 54 | 55 | (defn request-claim 56 | "Tries to apply the given claim to the given task. Throws if the given claim 57 | would be inconsistent." 58 | [task idx claim] 59 | (when-not task 60 | (throw (IllegalStateException. "task is nil"))) 61 | 62 | (when (completed? task) 63 | (throw (IllegalStateException. "task is completed"))) 64 | 65 | (when (nth (:claims task) idx nil) 66 | (throw (IllegalStateException. (str "already have a claim for " idx)))) 67 | 68 | (let [start (:start claim)] 69 | (if (some #(<= start (+ clock-skew-buffer (:end %))) (:claims task)) 70 | (throw (IllegalStateException. "task already claimed")) 71 | (assoc task :claims 72 | (assocv (:claims task) idx claim))))) 73 | 74 | (defn claim 75 | "Returns a copy of a task claimed for dt milliseconds. (last (:claims task)) 76 | will be the claim applied. Throws if the task is presently claimed." 77 | [task dt] 78 | (request-claim task (count (:claims task)) (new-claim dt))) 79 | 80 | (defn completed? 81 | "Is this task completed?" 82 | [task] 83 | (some :completed (:claims task))) 84 | 85 | (defn complete 86 | "Returns a copy of the task, but completed. Takes a claim index, and a time 87 | to mark the task as completed at." 88 | [task claim-idx t] 89 | (assoc-in task [:claims claim-idx :completed] t)) 90 | 91 | 92 | (defn merge-by 93 | "Merges times by merge-fn" 94 | [merge-fn & times] 95 | (let [valid-times (filter (comp not nil?) times)] 96 | (if (empty? valid-times) 97 | nil 98 | (apply merge-fn valid-times)))) 99 | 100 | (defn merge-claims 101 | "Merges a collection of vectors of claims together." 102 | [claims] 103 | (if (empty? claims) 104 | claims 105 | ; Determine how many claims there are 106 | (->> claims 107 | (map count) 108 | (apply max) 109 | range 110 | ; Combine the ith claim from each vector 111 | (mapv (fn [i] 112 | (reduce (fn combine [merged claims] 113 | (if-let [claim (nth claims i nil)] 114 | (if merged 115 | {:start (merge-by min 116 | (:start merged) 117 | (:start claim)) 118 | :end (merge-by max 119 | (:end merged) 120 | (:end claim)) 121 | :completed (merge-by min 122 | (:completed merged) 123 | (:completed claim))} 124 | claim) 125 | merged)) 126 | nil 127 | claims)))))) 128 | 129 | (defn merge 130 | "Merges n tasks together. Associative, commutative, idempotent." 131 | [& tasks] 132 | (-> (apply clojure.core/merge tasks) 133 | (assoc :claims (merge-claims (map :claims tasks))))) 134 | -------------------------------------------------------------------------------- /src/skuld/util.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.util 2 | "An Kitsch Sink" 3 | (:require 4 | [clojure.data.fressian :as fress] 5 | [primitive-math :as p]) 6 | (:import 7 | [com.aphyr.skuld Bytes] 8 | [org.fressian.handlers ReadHandler WriteHandler] 9 | [java.util 10 | Collection 11 | PriorityQueue] 12 | [java.nio ByteBuffer])) 13 | 14 | (deftype SeqContainer [key-fn ^long idx s] 15 | Comparable 16 | (equals [_ x] 17 | (and 18 | (instance? SeqContainer x) 19 | (p/== (.idx ^SeqContainer x) idx) 20 | (identical? (.s ^SeqContainer x) s))) 21 | (compareTo [_ x] 22 | (let [^SeqContainer x x 23 | cmp (compare (key-fn (first s)) (key-fn (first (.s x))))] 24 | (if (p/zero? cmp) 25 | (compare idx (.idx x)) 26 | cmp)))) 27 | 28 | (defn- sorted-interleave- [key-fn ^PriorityQueue heap] 29 | (lazy-seq 30 | (loop [chunk-idx 0, buf (chunk-buffer 32)] 31 | (if (.isEmpty heap) 32 | (chunk-cons (chunk buf) nil) 33 | (let [^SeqContainer container (.poll heap)] 34 | (chunk-append buf (first (.s container))) 35 | (when-let [s' (seq (rest (.s container)))] 36 | (.offer heap (SeqContainer. key-fn (.idx container) s'))) 37 | (let [chunk-idx' (unchecked-inc chunk-idx)] 38 | (if (< chunk-idx' 32) 39 | (recur chunk-idx' buf) 40 | (chunk-cons 41 | (chunk buf) 42 | (sorted-interleave- key-fn heap))))))))) 43 | 44 | (defn sorted-interleave-by 45 | "Like sorted-interleave, but takes a specific keyfn, like sort-by." 46 | [key-fn & seqs] 47 | (sorted-interleave- 48 | key-fn 49 | (PriorityQueue. 50 | ^Collection 51 | (map #(SeqContainer. key-fn %1 %2) (range) (remove empty? seqs))))) 52 | 53 | (defn sorted-interleave 54 | "Given n sorted sequences, yields a lazy sequence which yields all elements 55 | in all n collections, in order." 56 | [& seqs] 57 | (apply sorted-interleave-by identity seqs)) 58 | 59 | (defn majority 60 | "For N replicas, what would consititute a majority?" 61 | [n] 62 | (if (zero? n) 63 | 0 64 | (int (Math/floor (inc (/ n 2)))))) 65 | 66 | (defn majority-value 67 | "What element of a collection appears greater than 50% of the time?" 68 | [coll] 69 | (let [m (majority (count coll))] 70 | (when-let [pair (->> coll 71 | (group-by identity) 72 | (filter (comp (partial <= m) count val)) 73 | first)] 74 | (key pair)))) 75 | 76 | (defn assocv 77 | "Like assoc for vectors, but unlike assoc, allows you to assoc in indices 78 | which are greater than (count v)." 79 | [v idx value] 80 | (let [c (count v)] 81 | (if (<= idx c) 82 | (assoc v idx value) 83 | (-> v 84 | (concat (repeat (- idx c) nil) 85 | (list value)) 86 | vec)))) 87 | 88 | (defn update 89 | "Like update-in, but takes a single key. Given a map, a key, a function, and 90 | args, updates the value of (get map key) to be (f current-value arg1 arg2 91 | ...)" 92 | [m k f & args] 93 | (assoc m k (apply f (get m k) args))) 94 | 95 | (defn update! 96 | "Transient version of update" 97 | [m k f & args] 98 | (assoc! m k (apply f (get m k) args))) 99 | 100 | (defmacro compare+ 101 | "Expands into a comparison between a and b on the basis of function f1, then 102 | f2, etc." 103 | [a b f & fs] 104 | (if (empty? fs) 105 | `(compare (~f ~a) (~f ~b)) 106 | `(let [x# (compare (~f ~a) (~f ~b))] 107 | (if-not (zero? x#) 108 | x# 109 | (compare+ ~a ~b ~@fs))))) 110 | 111 | (type []) 112 | 113 | ;; Fressian 114 | (def ^:private bytes-write-handler 115 | {Bytes 116 | {"skuld-bytes" 117 | (reify WriteHandler 118 | (fress/write [_ w bs] 119 | (.writeTag w "skuld-bytes" 1) 120 | (.writeBytes w (.bytes ^Bytes bs))))} 121 | clojure.lang.PersistentVector 122 | {"vector" 123 | (reify WriteHandler 124 | (fress/write [_ w s] 125 | (.writeTag w "vector" 1) 126 | (.writeList w s)))}}) 127 | 128 | (def ^:private bytes-read-handler 129 | {"skuld-bytes" 130 | (reify ReadHandler (fress/read [_ rdr tag component-count] 131 | (Bytes. (.readObject rdr)))) 132 | "vector" 133 | (reify ReadHandler (fress/read [_ rdr tag component-count] 134 | (vec (.readObject rdr)))) 135 | "set" 136 | (reify ReadHandler (fress/read [_ rdr tag component-count] 137 | (set (.readObject rdr))))}) 138 | 139 | (def ^:private skuld-write-handlers 140 | (-> (conj bytes-write-handler fress/clojure-write-handlers) 141 | fress/associative-lookup 142 | fress/inheritance-lookup)) 143 | 144 | (def ^:private skuld-read-handlers 145 | (fress/associative-lookup (conj bytes-read-handler 146 | fress/clojure-read-handlers))) 147 | 148 | (defn ^ByteBuffer fress-write [x] (fress/write x :handlers skuld-write-handlers)) 149 | 150 | (defn fress-read [x] (fress/read x :handlers skuld-read-handlers)) 151 | -------------------------------------------------------------------------------- /src/skuld/vnode.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.vnode 2 | "A state machine which manages an instance of a partition on this node." 3 | (:use skuld.util 4 | clojure.tools.logging) 5 | (:require [skuld.task :as task] 6 | [skuld.db :as db] 7 | [skuld.db.level :as level] 8 | [skuld.net :as net] 9 | [skuld.flake :as flake] 10 | [skuld.curator :as curator] 11 | [skuld.scanner :as scanner] 12 | [skuld.queue :as queue] 13 | [clj-helix.route :as route] 14 | [clojure.set :as set]) 15 | (:import com.aphyr.skuld.Bytes)) 16 | 17 | ; DEAL WITH IT 18 | (in-ns 'skuld.aae) 19 | (clojure.core/declare sync-from!) 20 | (clojure.core/declare sync-to!) 21 | (in-ns 'skuld.vnode) 22 | 23 | (declare wipe!) 24 | 25 | (defn vnode 26 | "Create a new vnode. Options: 27 | 28 | :partition 29 | :state" 30 | [opts] 31 | (info (format "%s/%s:" (net/string-id (net/id (:net opts))) (:partition opts)) "starting vnode") 32 | (let [queue (queue/queues) 33 | vnode {:partition (:partition opts) 34 | :net (:net opts) 35 | :router (:router opts) 36 | :queue queue 37 | :db (level/open {:partition (:partition opts) 38 | :host (:host (:net opts)) 39 | :port (:port (:net opts))}) 40 | :last-leader-msg-time (atom Long/MIN_VALUE) 41 | :zk-leader (delay 42 | (curator/distributed-atom (:curator opts) 43 | (str "/" (:partition opts) 44 | "/leader") 45 | {:epoch 0 46 | :cohort #{}})) 47 | :state (atom {:type :follower 48 | :leader nil 49 | :epoch 0})} 50 | 51 | scanner (scanner/service vnode queue) 52 | vnode (assoc vnode :scanner scanner)] 53 | vnode)) 54 | 55 | ;; Leaders 56 | 57 | (defn peers 58 | "Peers for this vnode." 59 | [vnode] 60 | (doall 61 | (map #(select-keys % [:host :port]) 62 | (route/instances (:router vnode) :skuld (:partition vnode) :peer)))) 63 | 64 | (defn zk-leader 65 | "A curator distributed atom backed by Zookeeper, containing the current epoch 66 | and cohort." 67 | [vnode] 68 | @(:zk-leader vnode)) 69 | 70 | (defn net-id 71 | "The net id of the node hosting this vnode." 72 | [vnode] 73 | (net/id (:net vnode))) 74 | 75 | (defn state 76 | "The current state of a vnode." 77 | [vnode] 78 | @(:state vnode)) 79 | 80 | (defn leader? 81 | "Is this vnode a leader?" 82 | [vnode] 83 | (= :leader (:type (state vnode)))) 84 | 85 | (defn follower? 86 | "Is this vnode a follower?" 87 | [vnode] 88 | (= :follower (:type (state vnode)))) 89 | 90 | (defn candidate? 91 | "Is this vnode a candidate?" 92 | [vnode] 93 | (= :candidate (:type (state vnode)))) 94 | 95 | (defn active? 96 | "Is this vnode a leader or peer?" 97 | [vnode] 98 | (-> vnode 99 | state 100 | :type 101 | #{:leader :follower :candidate})) 102 | 103 | (defn zombie? 104 | "Is this vnode a zombie?" 105 | [vnode] 106 | (= :zombie (:type (state vnode)))) 107 | 108 | (defn dead? 109 | "Is this vnode dead?" 110 | [vnode] 111 | (= :dead (:type (state vnode)))) 112 | 113 | (defn epoch 114 | "The current epoch for this vnode." 115 | [vnode] 116 | (:epoch (state vnode))) 117 | 118 | (defn leader 119 | "What's the current leader for this vnode?" 120 | [vnode] 121 | (:leader (state vnode))) 122 | 123 | 124 | ;; Logging 125 | 126 | (defn full-id 127 | "Full ID for vnode" 128 | [vnode] 129 | (format "%s/%s" (net/string-id (net-id vnode)) (:partition vnode))) 130 | 131 | (defmacro trace-log 132 | "Log a message with context" 133 | [vnode & args] 134 | `(let [vnode-id# (format "%s:" (full-id ~vnode))] 135 | (info vnode-id# ~@args))) 136 | 137 | (def election-timeout 138 | "How long do we have to wait before initiating an election, in ms?" 139 | 10000) 140 | 141 | (defn leader-alive? 142 | "Is the current leader still alive?" 143 | [vnode] 144 | (> (+ @(:last-leader-msg-time vnode) election-timeout) 145 | (flake/linear-time))) 146 | 147 | (defn update-last-leader-msg-time! 148 | "Update the last leader message time" 149 | [vnode] 150 | (swap! (:last-leader-msg-time vnode) max (flake/linear-time))) 151 | 152 | (defn suppress-election! 153 | [vnode msg] 154 | (when (= (:epoch msg) (epoch vnode)) 155 | (update-last-leader-msg-time! vnode))) 156 | 157 | (defn accept-newer-epoch! 158 | "Any node with newer epoch information than us can update us to that epoch 159 | and convert us to a follower. Returns state if epoch updated, nil otherwise." 160 | [vnode msg] 161 | (when-let [leader-epoch (:epoch msg)] 162 | (when (< (epoch vnode) leader-epoch) 163 | (let [state (-> vnode 164 | :state 165 | (swap! (fn [state] 166 | (if (< (:epoch state) leader-epoch) 167 | {:type (if (= :zombie (:type state)) 168 | :zombie 169 | :follower) 170 | :epoch leader-epoch 171 | :cohort (:cohort msg) 172 | :leader (:leader msg) 173 | :updated true} 174 | (dissoc state :updated)))))] 175 | (when (:updated state) 176 | (suppress-election! vnode msg) 177 | (trace-log vnode "accepted newer epoch of" leader-epoch) 178 | state))))) 179 | 180 | (defn request-vote! 181 | "Accepts a new-leader message from a peer, and returns a vote for the 182 | node--or an error if our epoch is current or newer. Returns a response 183 | message." 184 | [vnode msg] 185 | (if-let [state (accept-newer-epoch! vnode msg)] 186 | (assoc state :vote (net-id vnode)) 187 | (state vnode))) 188 | 189 | (defn demote! 190 | "Forces a leader to step down." 191 | [vnode] 192 | (locking vnode 193 | (trace-log vnode "demoted") 194 | (swap! (:state vnode) (fn [state] 195 | (if (= :leader (:type state)) 196 | (assoc state :type :follower) 197 | state))))) 198 | 199 | (defn zombie! 200 | "Places the vnode into an immutable zombie mode, where it waits to hand off 201 | its data to a leader." 202 | [vnode] 203 | (locking vnode 204 | (trace-log vnode "now zombie") 205 | (swap! (:state vnode) assoc :type :zombie))) 206 | 207 | (defn revive! 208 | "Converts dead or zombie vnodes into followers." 209 | [vnode] 210 | (locking vnode 211 | (trace-log vnode "revived!") 212 | (swap! (:state vnode) (fn [state] 213 | (if (#{:zombie :dead} (:type state)) 214 | (assoc state :type :follower) 215 | state))))) 216 | 217 | (defn shutdown! 218 | "Converts a zombie to state :dead, and closes resources." 219 | [vnode] 220 | (locking vnode 221 | (when-not (= :dead @(:state vnode)) 222 | (reset! (:state vnode) :dead) 223 | (scanner/shutdown! (:scanner vnode)) 224 | (db/close! (:db vnode))))) 225 | 226 | (defn majority-excluding-self 227 | "Given a vnode, and a set of nodes, how many responses from *other* nodes 228 | (i.e. assuming we vote yes for ourself) are required to comprise a majority 229 | of the set?" 230 | [vnode cohort] 231 | ((if (cohort (net-id vnode)) dec identity) 232 | (majority (count cohort)))) 233 | 234 | (defn sufficient-votes? 235 | "Given a vnode, old cohort, and new cohort, does the given collection of 236 | request-vote responses allow us to become a leader?" 237 | [vnode old-cohort new-cohort votes] 238 | (let [votes (set (keep :vote votes))] 239 | (and (<= (majority-excluding-self vnode old-cohort) 240 | (count (set/intersection old-cohort votes))) 241 | (<= (majority-excluding-self vnode new-cohort) 242 | (count (set/intersection new-cohort votes)))))) 243 | 244 | (defn sync-with-majority! 245 | "Tries to copy tasks to or from a majority of the given cohort, using 246 | sync-fn. Returns true if successful." 247 | [vnode cohort sync-fn] 248 | (not (pos? (loop [remaining (majority-excluding-self vnode cohort) 249 | [node & more] (seq (disj cohort (net-id vnode)))] 250 | (if-not (and node (< 0 remaining)) 251 | ; Done 252 | remaining 253 | ; Copy data from another node and repeat 254 | (recur (if (try (sync-fn vnode node) 255 | (catch RuntimeException e 256 | (error e "while synchronizing" (:partition vnode) "with" node) 257 | false)) 258 | (dec remaining) 259 | remaining) 260 | more)))))) 261 | 262 | (defn heartbeat! 263 | "Handles a request for a heartbeat from a leader." 264 | [vnode msg] 265 | (suppress-election! vnode msg) 266 | (accept-newer-epoch! vnode msg)) 267 | 268 | (defn broadcast-heartbeat! 269 | "Send a heartbeat to all followers to retain leadership." 270 | [vnode] 271 | (let [state (state vnode) 272 | peers (disj (set (peers vnode)) (net-id vnode)) 273 | epoch (:epoch state)] 274 | (if (= :leader (:type state)) 275 | (net/sync-req! (:net vnode) peers {:r (count peers)} 276 | {:type :heartbeat 277 | :partition (:partition vnode) 278 | :leader (net-id vnode) 279 | :epoch epoch})))) 280 | 281 | 282 | (defn elect! 283 | "Attempt to become a primary. We need to ensure that: 284 | 285 | 1. Leaders are logically sequential 286 | 2. Each leader's claim set is a superset of the previous leader 287 | 288 | We have: a target cohort of nodes for the new epoch, provided by helix. 289 | Some previous cohort of nodes belonging to the old epoch, tracked by ZK. 290 | 291 | To become a leader, one must successfully: 292 | 293 | 1. Read the previous epoch+cohort from ZK 294 | 295 | 2. (optimization) Ensure that the previous epoch is strictly less than the 296 | epoch this node is going to be the leader for. 297 | 298 | 3. Broadcast a claim message to the new cohort, union the old cohort 299 | 300 | 4. Receive votes from a majority of the nodes in the old cohort 301 | 302 | 5. Receive votes from a majority of the nodes in the new cohort 303 | 304 | - At this juncture, neither the new nor the old cohort can commit claims 305 | for our target epoch or lower, making the set of claims made in older epochs 306 | immutable. If we are beat by another node using a newer epoch, it will have 307 | recovered a superset of those claims; we'll fail to CAS in step 8, and no 308 | claims will be lost. 309 | 310 | 6. Obtain all claims from a majority of the old cohort, and union them in to 311 | our local claim set. 312 | 313 | - This ensures that we are aware of all claims made prior to our epoch. 314 | 315 | 7. Broadcast our local claim set to a majority of the new cohort. 316 | 317 | - This ensures that any *future* epoch will correctly recover our claim 318 | set. 6 + 7 provide a continuous history of claims, by induction. 319 | 320 | 8. CAS our new epoch and cohort into zookeeper, ensuring that nobody else 321 | beat us to it. 322 | 323 | If any stage fails, delay randomly and retry. 324 | 325 | A note on zombies: 326 | 327 | Zombies are nodes which are ready to give up ownership of a vnode but cannot, 328 | because they may still be required to hand off their claim set. After step 8, 329 | we inform all zombies which are not a part of our new cohort that it is safe 330 | to drop their claim set." 331 | [vnode] 332 | (locking vnode 333 | (when (and 334 | (active? vnode) 335 | (not (leader? vnode)) 336 | (not (leader-alive? vnode))) 337 | (trace-log vnode "elect: initiating election. current leader is not alive for epoch" (:epoch (state vnode))) 338 | 339 | ; First, compute the set of peers that will comprise the next epoch. 340 | (let [self (net-id vnode) 341 | new-cohort (set (peers vnode)) 342 | 343 | ; Increment the epoch and update the node set. 344 | epoch (-> vnode 345 | :state 346 | (swap! (fn [state] 347 | (merge state {:epoch (inc (:epoch state)) 348 | :leader self 349 | :type :candidate 350 | :cohort new-cohort}))) 351 | :epoch) 352 | 353 | ; Check ZK's last leader information 354 | old (deref (zk-leader vnode))] 355 | (if (<= epoch (:epoch old)) 356 | ; We're outdated; fast-forward to the new epoch. 357 | (do 358 | (trace-log vnode "elect: Outdated epoch relative to ZK; aborting election. Fast-forwarding from" epoch "to" (:epoch old)) 359 | (swap! (:state vnode) (fn [state] 360 | (if (<= (:epoch state) (:epoch old)) 361 | (merge state {:epoch (:epoch old) 362 | :leader false 363 | :type :follower 364 | :cohort (:cohort old)}) 365 | state)))) 366 | 367 | ; Issue requests to all nodes in old and new cohorts 368 | (let [old-cohort (set (:cohort old)) 369 | responses (atom (list)) 370 | accepted? (promise) 371 | peers (disj (set/union new-cohort old-cohort) self)] 372 | (trace-log vnode "elect: Issuing requests for election for" epoch "transitioning from" old-cohort "to" new-cohort) 373 | (doseq [node peers] 374 | (net/req! (:net vnode) (list node) {:r 1} 375 | {:type :request-vote 376 | :partition (:partition vnode) 377 | :leader self 378 | :cohort new-cohort 379 | :epoch epoch} 380 | [[r]] 381 | (let [rs (swap! responses conj r)] 382 | (if (accept-newer-epoch! vnode r) 383 | ; Cancel request; we saw a newer epoch from a peer. 384 | (do 385 | (trace-log vnode "elect: aborting candidacy due to newer epoch of" (:epoch r)) 386 | (deliver accepted? false)) 387 | ; Have we enough votes? 388 | (if (sufficient-votes? vnode old-cohort new-cohort rs) 389 | (do 390 | (trace-log vnode "elect: Received enough votes:" rs) 391 | (deliver accepted? true)) 392 | (when (<= (count peers) (count rs)) 393 | (trace-log vnode "elect: all votes in; election was lost:" rs) 394 | (deliver accepted? false))))))) 395 | 396 | ; Await responses 397 | (if-not (deref accepted? 5000 false) 398 | (trace-log vnode "elect: election failed; not enough votes") 399 | 400 | ; Sync from old cohort. 401 | (if-not (sync-with-majority! vnode 402 | old-cohort 403 | skuld.aae/sync-from!) 404 | (trace-log vnode "elect: Wasn't able to replicate from enough of old cohort; cannot become leader.") 405 | 406 | ; Sync to new cohort. 407 | (if-not (sync-with-majority! vnode 408 | new-cohort 409 | skuld.aae/sync-to!) 410 | (errorf "%s: elect: Wasn't able to replicate to enough of new cohort; cannot become leader." (full-id vnode)) 411 | 412 | ; Update ZK with new cohort and epoch--but only if nobody else 413 | ; got there first. 414 | (let [new-leader {:epoch epoch 415 | :cohort new-cohort} 416 | set-leader (curator/swap!! (zk-leader vnode) 417 | (fn [current] 418 | (if (= old current) 419 | new-leader 420 | current)))] 421 | (if (not= new-leader set-leader) 422 | (trace-log vnode "elect: election failed: another leader updated zk") 423 | 424 | ; Success! 425 | (let [state (swap! (:state vnode) 426 | (fn [state] 427 | (if (= epoch (:epoch state)) 428 | ; Still waiting for responses 429 | (assoc state :type :leader) 430 | ; We voted for someone else in the 431 | ; meantime 432 | state)))] 433 | (trace-log vnode "elect: election successful: epoch is" epoch "for cohort" new-cohort) 434 | (broadcast-heartbeat! vnode))))))))))))) 435 | 436 | ;; Tasks 437 | 438 | (defn merge-task! 439 | "Takes a task and merges it into this vnode." 440 | [vnode task] 441 | (db/merge-task! (:db vnode) task) 442 | (queue/update! (:queue vnode) task)) 443 | 444 | (defn get-task 445 | "Returns a specific task by ID." 446 | [vnode id] 447 | (db/get-task (:db vnode) id)) 448 | 449 | (defn ids 450 | "All task IDs in this vnode." 451 | [vnode] 452 | (db/ids (:db vnode))) 453 | 454 | (defn count-queue 455 | "Estimates the number of enqueued tasks." 456 | [vnode queue-name] 457 | (if (leader? vnode) 458 | (queue/count-queue (:queue vnode) queue-name) 459 | 0)) 460 | 461 | (defn count-tasks 462 | "How many tasks are in this vnode?" 463 | [vnode] 464 | (db/count-tasks (:db vnode))) 465 | 466 | (defn tasks 467 | "All tasks in this vnode." 468 | [vnode] 469 | (db/tasks (:db vnode))) 470 | 471 | (defn claimed 472 | "A subset of tasks which are claimed." 473 | [vnode] 474 | (->> vnode 475 | tasks 476 | (filter task/claimed?))) 477 | 478 | (defn unclaimed 479 | "A subset of tasks which are eligible for claim." 480 | [vnode] 481 | (->> vnode 482 | tasks 483 | (remove task/completed?) 484 | (remove task/claimed?))) 485 | 486 | (defn request-claim! 487 | "Applies a claim to a given task. Takes a message from a leader like 488 | 489 | {:epoch The leader's epoch 490 | :id The task ID 491 | :i The index of the claim 492 | :claim A claim map} 493 | 494 | ... and applies the given claim to our copy of that task. Returns an empty 495 | map if the claim is successful, or {:error ...} if the claim failed." 496 | [vnode {:keys [id i claim] :as msg}] 497 | (suppress-election! vnode msg) 498 | (accept-newer-epoch! vnode msg) 499 | (try 500 | (locking vnode 501 | (assert (not (zombie? vnode))) 502 | (if (= (:epoch msg) (epoch vnode)) 503 | (if-let [task (db/claim-task! (:db vnode) id i claim)] 504 | (do 505 | (queue/update! (:queue vnode) task) 506 | {}) 507 | {:error (str "task " id " was not found on vnode " (full-id vnode))}) 508 | 509 | {:error (str "leader epoch " epoch 510 | " does not match local epoch " (epoch vnode))})) 511 | (catch IllegalStateException e 512 | {:error (.getMessage e)}))) 513 | 514 | (defn claim! 515 | "Claims a particular task ID from this vnode, for dt milliseconds. Returns the 516 | claimed task." 517 | [vnode queue-name dt] 518 | (let [state (state vnode) 519 | cur-epoch (:epoch state) 520 | cohort (:cohort state) 521 | ; How many followers need to ack us? 522 | maj (-> cohort 523 | set 524 | (disj (net-id vnode)) 525 | count 526 | majority 527 | dec)] 528 | 529 | (when-not (= :leader (:type state)) 530 | (throw (IllegalStateException. (format "can't initiate claim: not a leader. current vnode type: %s" (:type state))))) 531 | 532 | ; Look for the next available task 533 | (when-let [task-id (:id (queue/poll! (:queue vnode) queue-name))] 534 | ; Attempt to claim a task locally. 535 | (if-let [task (db/claim-task! (:db vnode) task-id dt)] 536 | (do 537 | (trace-log vnode "claim: claiming task:" task) 538 | (let [; Get claim details 539 | i (dec (count (:claims task))) 540 | claim (nth (:claims task) i) 541 | 542 | ; Try to replicate claim remotely 543 | responses (net/sync-req! (:net vnode) 544 | (disj cohort (net-id vnode)) 545 | {:r maj} 546 | {:type :request-claim 547 | :epoch cur-epoch 548 | :id (:id task) 549 | :i i 550 | :claim claim}) 551 | successes (count (remove :error responses))] 552 | 553 | ; Check that we're still in the same epoch; a leader could 554 | ; have subsumed us. 555 | (when (not= cur-epoch (epoch vnode)) 556 | (throw (IllegalStateException. (str "epoch changed from " 557 | cur-epoch 558 | " to " 559 | (epoch vnode) 560 | ", claim coordinator aborting")))) 561 | 562 | (if (<= maj successes) 563 | (do 564 | ; We succeeded at this claim, so our cohort has updated their last-leader-msg-time 565 | (update-last-leader-msg-time! vnode) 566 | task) 567 | (throw (IllegalStateException. (str "needed " maj 568 | " acks from followers, only received " 569 | successes)))))) 570 | 571 | ; The ID in the queue did not exist in the database 572 | (recur vnode queue-name dt))))) 573 | 574 | 575 | (defn complete! 576 | "Completes the given task in the specified claim. Msg should contain: 577 | 578 | :task-id The task identifier 579 | :claim-id The claim index 580 | :time The time the task was completed at, in linear time." 581 | [vnode msg] 582 | (merge-task! vnode 583 | (-> vnode 584 | (get-task (:task-id msg)) 585 | (task/complete (:claim-id msg) (:time msg))))) 586 | 587 | (defn wipe! 588 | "Wipe a vnode's data clean." 589 | [vnode] 590 | (db/wipe! (:db vnode)) 591 | (.clear (:queue vnode)) 592 | vnode) 593 | -------------------------------------------------------------------------------- /test/skuld/claim_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.claim-test 2 | (:use clojure.tools.logging 3 | clojure.test 4 | skuld.util 5 | skuld.node 6 | skuld.node-test) 7 | 8 | (:require [skuld.client :as client] 9 | [skuld.admin :as admin] 10 | [skuld.vnode :as vnode] 11 | [skuld.flake :as flake] 12 | [skuld.curator :as curator] 13 | [skuld.net :as net] 14 | [skuld.task :as task] 15 | [skuld.aae :as aae] 16 | [clojure.set :as set] 17 | [skuld.logging :as logging] 18 | clj-helix.admin) 19 | (:import com.aphyr.skuld.Bytes)) 20 | 21 | (use-fixtures :once once) 22 | (use-fixtures :each each) 23 | 24 | 25 | (deftest claim-test 26 | (elect! *nodes*) 27 | (let [id (client/enqueue! *client* {:w 3} {:queue "queue1" :data "hi"})] 28 | (is id) 29 | (let [task (client/claim! *client* "queue1" 1000)] 30 | (is (= id (:id task))) 31 | (is (task/claimed? task))))) 32 | 33 | (deftest reclaim-test 34 | (elect! *nodes*) 35 | (with-redefs [task/clock-skew-buffer 500] 36 | (let [id (client/enqueue! *client* {:w 3} {:queue "queue2" :data "maus"})] 37 | (is (= id (:id (client/claim! *client* "queue2" 1000)))) 38 | 39 | ; Can't reclaim, because it's already claimed 40 | (is (nil? (client/claim! *client* "queue2" 1000))) 41 | 42 | ; Can't reclaim after 1000ms because clock skew buffer still holds 43 | (Thread/sleep 1001) 44 | (is (nil? (client/claim! *client* "queue2" 1000))) 45 | 46 | ; But after the buffer has elapsed, good to go. 47 | (Thread/sleep 1500) 48 | 49 | (let [t (client/claim! *client* "queue2" 1000)] 50 | (is (= id (:id t))) 51 | (is (= 2 (count (:claims t)))) 52 | (is (= "maus" (:data t))))))) 53 | 54 | (deftest complete-test 55 | (elect! *nodes*) 56 | (with-redefs [task/clock-skew-buffer 0] 57 | (let [id (client/enqueue! *client* {:queue "queue3" :data "sup"})] 58 | (is (client/claim! *client* "queue3" 1)) 59 | 60 | ; Isn't completed 61 | (is (not (task/completed? (client/get-task *client* id)))) 62 | 63 | (client/complete! *client* id 0) 64 | 65 | ; Is completed 66 | (is (task/completed? (client/get-task *client* id))) 67 | 68 | ; Can't re-claim. 69 | (Thread/sleep 2) 70 | (is (nil? (client/claim! *client* "queue3" 100)))))) 71 | -------------------------------------------------------------------------------- /test/skuld/core_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.core-test 2 | (:require [clojure.test :refer :all] 3 | [skuld.core :refer :all])) 4 | 5 | -------------------------------------------------------------------------------- /test/skuld/db/level_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.db.level-test 2 | (:use skuld.db.level 3 | skuld.db 4 | clojure.test) 5 | (:require [skuld.flake-test :as flake] 6 | [skuld.task :as task])) 7 | 8 | (def ^:dynamic *db*) 9 | 10 | (use-fixtures :each (fn [f] 11 | (binding [*db* (open {:host "localhost" 12 | :port 0 13 | :partition "skuld-0"})] 14 | (try 15 | (wipe! *db*) 16 | (f) 17 | (finally 18 | (close! *db*)))))) 19 | 20 | (deftest exists-test 21 | (let [vnode {:host "localhost" :port 0 :partition "skuld-1" :ext "test"}] 22 | (is (not (local-data? vnode))) 23 | (path! vnode) 24 | (is (local-data? vnode)) 25 | (destroy-data! vnode) 26 | (is (not (local-data? vnode))))) 27 | 28 | (deftest merge-test 29 | (let [t (task/task {:data "meow"})] 30 | (merge-task! *db* t) 31 | (is (= t (get-task *db* (:id t)))) 32 | 33 | (let [t (task/task {:data "mause"}) 34 | t' (task/claim t 123)] 35 | (merge-task! *db* t') 36 | (is (= t' (get-task *db* (:id t)))) 37 | 38 | (merge-task! *db* t) 39 | (is (= t' (get-task *db* (:id t))))))) 40 | 41 | (deftest count-test 42 | (is (= 0 (count-tasks *db*))) 43 | (dotimes [i 1000] 44 | (merge-task! *db* (task/task {}))) 45 | (is (= 1000 (count-tasks *db*))) 46 | (wipe! *db*) 47 | (is (= 0 (count-tasks *db*)))) 48 | 49 | (deftest tasks-test 50 | (let [ts (->> (fn [] {:data (rand)}) 51 | repeatedly 52 | (map task/task) 53 | (take 100) 54 | doall 55 | shuffle)] 56 | (doseq [t ts] 57 | (merge-task! *db* t)) 58 | (is (= (sort-by :id ts) 59 | (tasks *db*))))) 60 | -------------------------------------------------------------------------------- /test/skuld/flake_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.flake-test 2 | (:use skuld.flake 3 | clojure.test 4 | [criterium.core :only [quick-bench]]) 5 | (:require [clojure.data.codec.base64 :as base64]) 6 | (:import (java.util Arrays) 7 | (com.google.common.primitives UnsignedBytes))) 8 | 9 | (in-ns 'skuld.flake) 10 | (def node-id (constantly (byte-array 6))) 11 | (in-ns 'skuld.flake-test) 12 | 13 | (init!) 14 | 15 | (deftest linear-time-test 16 | (dotimes [i 10] 17 | (Thread/sleep 1) 18 | (is (>= 1 (Math/abs ^long (- (System/currentTimeMillis) 19 | (linear-time))))))) 20 | 21 | (deftest id-test 22 | (let [ids (->> (repeatedly id) 23 | (take 10000))] 24 | (is (= ids (sort (UnsignedBytes/lexicographicalComparator) ids))) 25 | (is (= ids (distinct ids))))) 26 | 27 | (deftest node-test 28 | (is (Arrays/equals ^bytes (node-fragment) ^bytes (node-fragment)))) 29 | 30 | (deftest ^:bench perf-test 31 | (quick-bench (id))) 32 | -------------------------------------------------------------------------------- /test/skuld/leader_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.leader-test 2 | (:use clojure.tools.logging 3 | clojure.test 4 | skuld.util 5 | skuld.node 6 | skuld.node-test) 7 | 8 | (:require [skuld.client :as client] 9 | [skuld.admin :as admin] 10 | [skuld.vnode :as vnode] 11 | [skuld.flake :as flake] 12 | [skuld.curator :as curator] 13 | [skuld.net :as net] 14 | [skuld.task :as task] 15 | [skuld.aae :as aae] 16 | [clojure.set :as set] 17 | clj-helix.admin) 18 | (:import com.aphyr.skuld.Bytes)) 19 | 20 | (use-fixtures :once once) 21 | (use-fixtures :each each) 22 | 23 | (defn test-election-consistent 24 | "Asserts that the current state of the given vnodes is consistent, from a 25 | leader-election perspective." 26 | [vnodes] 27 | ; Take a snapshot of the states (so we're operating on locally consistent 28 | ; information 29 | (let [states (->> vnodes 30 | (map vnode/state) 31 | (map (fn [vnode state] 32 | (assoc state :id (net/id (:net vnode)))) 33 | vnodes) 34 | doall) 35 | leaders (filter #(= :leader (:type %)) states) 36 | true-leader (promise)] 37 | 38 | ; Exactly one leader for each epoch 39 | (doseq [[epoch leaders] (group-by :epoch leaders)] 40 | (is (= 1 (count leaders)))) 41 | 42 | ; For all leaders 43 | (doseq [leader leaders] 44 | ; Find all nodes which this leader could write to 45 | (let [cohort (->> states 46 | (filter #(and (= :follower (:type %)) 47 | (= (:epoch leader) (:epoch %)) 48 | (= (:cohort leader) (:cohort %)))))] 49 | 50 | ; There should be exactly one leader which could satisfy a quorum 51 | (when (<= (majority (count (:cohort leader))) 52 | (count cohort)) 53 | (is (deliver true-leader leader))))))) 54 | 55 | (deftest election-test 56 | (let [part "skuld_0" 57 | nodes (filter (fn [node] 58 | (when-let [v (vnode node part)] 59 | (vnode/active? v))) 60 | *nodes*) 61 | vnodes (map #(vnode % part) nodes)] 62 | 63 | (testing "Initially" 64 | (test-election-consistent vnodes)) 65 | 66 | (testing "Stress" 67 | (let [running (promise)] 68 | ; Measure consistency continuously 69 | (future 70 | (while (deref running 1 true) 71 | (test-election-consistent vnodes))) 72 | 73 | ; Initiate randomized elections 74 | (with-redefs [vnode/election-timeout 0] 75 | (->> vnodes 76 | (map #(future 77 | (dotimes [i (rand-int 10)] 78 | (vnode/elect! %)) 79 | (Thread/sleep (rand-int 10)))) 80 | (map deref) 81 | doall)) 82 | 83 | (deliver running false) 84 | (test-election-consistent vnodes))))) 85 | -------------------------------------------------------------------------------- /test/skuld/net_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.net-test 2 | (:use skuld.net 3 | clojure.test) 4 | (:import java.util.concurrent.CountDownLatch) 5 | (:require skuld.flake-test 6 | [clojure.set :as set])) 7 | 8 | (deftest solipsist 9 | (let [node (node {:host "127.0.0.1" :port 13000}) 10 | log (atom []) 11 | msgs [{:x 1} 12 | {:x "foo"} 13 | {:x {:hi 2}} 14 | {:set #{:a "foo"} :vec [1/2 'yo]} 15 | {:done? true}] 16 | done (promise)] 17 | (try 18 | (add-handler! node (fn [msg] 19 | (swap! log conj msg) 20 | (when (:done? msg) 21 | (deliver done true)) 22 | nil)) 23 | (start! node) 24 | (doseq [m msgs] (send-sync! node node m)) 25 | @done 26 | (is (= (set msgs) (set @log))) 27 | 28 | (finally 29 | (shutdown! node))))) 30 | 31 | (deftest ring 32 | "Pass an integer around a ring." 33 | (let [n 2 34 | limit 1000 35 | nodes (->> (range n) 36 | (map (fn [i] (node {:port (+ 13000 i)})))) 37 | x (atom 0) 38 | done (promise)] 39 | (try 40 | (->> nodes 41 | (map-indexed 42 | (fn [i node] 43 | (add-handler! node 44 | (let [next-node (nth nodes (mod (inc i) n))] 45 | (fn [msg] 46 | (assert (= @x msg)) 47 | (if (<= limit @x) 48 | ; Done 49 | (deliver done true) 50 | ; Forward along 51 | (send! node next-node (swap! x inc))) 52 | nil))))) 53 | dorun) 54 | (dorun (pmap start! nodes)) 55 | 56 | ; Start things rolling 57 | (send! (first nodes) (second nodes) @x) 58 | 59 | ; Wait 60 | @done 61 | 62 | (is (= limit @x)) 63 | 64 | (finally (dorun (pmap shutdown! nodes)))))) 65 | 66 | (deftest echo-test 67 | "Sends a request and waits for an echo server to respond." 68 | (let [a (node {:port 13001}) 69 | b (node {:port 13002}) 70 | done (promise)] 71 | (try 72 | ; Make B an echo server. 73 | (add-handler! b identity) 74 | (start! a) 75 | (start! b) 76 | 77 | (req! a [b] {} {:echo? :echo!} 78 | [responses] 79 | (is (= 1 (count responses))) 80 | (deliver done true)) 81 | 82 | (is (deref done 5000 false)) 83 | 84 | (finally 85 | (shutdown! a) 86 | (shutdown! b))))) 87 | 88 | (deftest scatter-test 89 | "Broadcasts requests to several nodes." 90 | (let [n 5 91 | nodes (->> (range n) 92 | (map (fn [i] (node {:port (+ 13000 i)})))) 93 | responses (promise) 94 | done (CountDownLatch. n)] 95 | (try 96 | (doseq [node nodes] 97 | (add-handler! node (fn [msg] 98 | (.countDown done) 99 | {:i-am (:port node)}))) 100 | (dorun (pmap start! nodes)) 101 | 102 | (req! (first nodes) nodes {:r 3} {} 103 | [rs] 104 | (deliver responses rs)) 105 | 106 | (or (deref responses 5000 false) (throw (RuntimeException. "stuck"))) 107 | (is (<= 3 (count @responses))) 108 | (is (= (distinct @responses) @responses)) 109 | (is (set/subset? (set (map :i-am @responses)) (set (map :port nodes)))) 110 | 111 | (.await done) 112 | 113 | (finally 114 | (dorun (pmap shutdown! nodes)))))) 115 | 116 | (deftest timeout-test 117 | "Verifies that timeouts work correctly." 118 | (let [a (node {:port 13000}) 119 | b (node {:port 13001}) 120 | rs (promise) 121 | done (promise)] 122 | (try 123 | ; Here's a slow handler for B 124 | (add-handler! b (fn [x] (Thread/sleep 3000) (deliver done true) x)) 125 | (dorun (pmap start! [a b])) 126 | 127 | (req! a [b] {:r 1 :timeout 1000} {:hi :there} 128 | [responses] 129 | (deliver rs responses)) 130 | 131 | (is (= [] @rs)) 132 | @done 133 | (Thread/sleep 100) 134 | 135 | (finally 136 | (dorun (pmap shutdown! [a b])))))) 137 | -------------------------------------------------------------------------------- /test/skuld/node_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.node-test 2 | (:use clojure.tools.logging 3 | clojure.test 4 | skuld.zk-test 5 | skuld.util 6 | skuld.node) 7 | (:require [skuld.client :as client] 8 | [skuld.admin :as admin] 9 | [skuld.vnode :as vnode] 10 | [skuld.curator :as curator] 11 | [skuld.net :as net] 12 | [skuld.task :as task] 13 | [skuld.flake :as flake] 14 | [skuld.aae :as aae] 15 | [skuld.politics :as politics] 16 | [skuld.logging :as logging] 17 | [clojure.set :as set] 18 | [clj-http.client :as http] 19 | [cheshire.core :as json] 20 | skuld.http 21 | skuld.flake-test 22 | clj-helix.admin) 23 | (:import com.aphyr.skuld.Bytes)) 24 | 25 | (def b64->id #'skuld.http/b64->id) 26 | 27 | (defn admin 28 | [zk] 29 | (logging/with-level :warn ["org.apache.zookeeper" "org.apache.helix" "org.I0Itec.zkclient"] 30 | (admin/admin {:partitions 2 31 | :replicas 3 32 | :zookeeper zk}))) 33 | 34 | (def ^:dynamic *client* nil) 35 | (def ^:dynamic *nodes* nil) 36 | (def ^:dynamic *zk* nil) 37 | 38 | (defn ensure-cluster! 39 | "Ensures a test cluster exists." 40 | [admin] 41 | (when-not (some #{"skuld"} (clj-helix.admin/clusters (:helix admin))) 42 | (admin/destroy-cluster! admin) 43 | (admin/create-cluster! admin) 44 | (dotimes [i 7] 45 | (admin/add-node! admin {:host "127.0.0.1" :port (+ 13000 i)})))) 46 | 47 | (defn start-nodes! 48 | "Returns a vector of a bunch of started nodes." 49 | [zk] 50 | (let [nodes (->> (range 5) 51 | (pmap #(wait-for-peers (node {:port (+ 13000 %) 52 | :zookeeper zk}))) 53 | doall) 54 | vnodes (->> nodes 55 | (map vnodes) 56 | (mapcat vals))] 57 | (->> vnodes 58 | (pmap #(curator/reset!! (vnode/zk-leader %) {:epoch 0 59 | :cohort #{}})) 60 | dorun) 61 | nodes)) 62 | 63 | (defn wipe-and-shutdown-nodes! 64 | "Wipe and shutdown a seq of nodes." 65 | [nodes] 66 | (->> nodes 67 | (pmap (fn wipe-and-shutdown [node] 68 | (wipe-local! node nil) 69 | (shutdown! node))) 70 | dorun)) 71 | 72 | (defn wipe-nodes! 73 | "Wipe a seq of nodes." 74 | [nodes] 75 | (->> nodes 76 | (pmap (fn wipe [node] 77 | (wipe-local! node nil))) 78 | dorun)) 79 | 80 | 81 | (defn partition-available? 82 | "Given a set of vnodes for a partition, do they comprise an available 83 | cohort?" 84 | [vnodes] 85 | (when-let [majority-epoch (majority-value (map vnode/epoch vnodes))] 86 | (some #(and (= majority-epoch (vnode/epoch %)) 87 | (vnode/leader? %)) 88 | vnodes))) 89 | 90 | (defn elect! 91 | "Force election of a leader in all vnodes." 92 | [nodes] 93 | (loop [unelected (->> nodes 94 | (map vnodes) 95 | (mapcat vals) 96 | (filter vnode/active?) 97 | (group-by :partition) 98 | vals 99 | (remove partition-available?))] 100 | (when-not (empty? unelected) 101 | (info (count unelected) "unelected partitions") 102 | ; (debug (map (partial map (juxt (comp :port vnode/net-id) 103 | ; :partition 104 | ; vnode/state)) 105 | ; unelected))) 106 | (doseq [vnodes unelected] 107 | (vnode/elect! (rand-nth vnodes))) 108 | (Thread/sleep 100) 109 | (recur (remove partition-available? unelected))))) 110 | 111 | (defn once 112 | [f] 113 | (with-zk [zk] 114 | ; Set up cluster 115 | (let [admin (admin zk)] 116 | (try 117 | (ensure-cluster! admin) 118 | (admin/shutdown! admin))) 119 | 120 | ; Set up nodes 121 | (binding [*zk* zk 122 | *nodes* (start-nodes! zk)] 123 | (try 124 | (f) 125 | (finally 126 | (info "wiping and shutting down nodes") 127 | (wipe-and-shutdown-nodes! *nodes*)))))) 128 | 129 | (defn each 130 | [f] 131 | ; If any nodes were killed by the test, re-initialize the cluster before 132 | ; proceeding. 133 | (binding [*client* (client/client *nodes*)] 134 | (try 135 | (if (not-any? shutdown? *nodes*) 136 | (do 137 | (info "wiping nodes:" (map net/string-id *nodes*)) 138 | (wipe-nodes! *nodes*) 139 | (f)) 140 | (do 141 | (info "repairing cluster by shutting down, and restarting nodes") 142 | (wipe-and-shutdown-nodes! *nodes*) 143 | (set! *nodes* (start-nodes! *zk*)) 144 | (info "wiping nodes:" (map net/string-id *nodes*)) 145 | (wipe-nodes! *nodes*) 146 | (f))) 147 | (finally 148 | (client/shutdown! *client*))))) 149 | 150 | 151 | (use-fixtures :once once) 152 | (use-fixtures :each each) 153 | 154 | ; (def byte-array-class ^:const (type (byte-array 0))) 155 | 156 | (defn log-cohorts 157 | [] 158 | (println "cohorts are\n" (->> *nodes* 159 | (map vnodes) 160 | (mapcat vals) 161 | (filter vnode/leader?) 162 | (map (juxt (comp :port vnode/net-id) 163 | :partition 164 | vnode/epoch 165 | (comp (partial map :port) 166 | :cohort vnode/state))) 167 | (map pr-str) 168 | (interpose "\n") 169 | (apply str)))) 170 | 171 | (defn log-counts 172 | [] 173 | (->> *nodes* 174 | (mapcat (fn [node] 175 | (->> node 176 | vnodes 177 | (map (fn [[part vnode]] 178 | [(:port (net/id (:net node))) 179 | part 180 | (vnode/leader? vnode) 181 | (vnode/count-tasks vnode)]))))) 182 | (clojure.pprint/pprint))) 183 | 184 | (deftest enqueue-test 185 | ; Enqueue a task 186 | (let [id (client/enqueue! *client* {:queue "queue4" :data "hi there"})] 187 | (is id) 188 | (is (instance? Bytes id)) 189 | 190 | ; Read it back 191 | (is (= (client/get-task *client* {:r 3} id) 192 | {:id id 193 | :queue "queue4" 194 | :claims [] 195 | :data "hi there"})))) 196 | 197 | (deftest count-test 198 | ; We need leaders for queue counts to work. 199 | (elect! *nodes*) 200 | 201 | ; Enqueue a few tasks 202 | (let [n 10] 203 | (dotimes [i n] 204 | (client/enqueue! *client* {:w 3} {:queue "queue5" :data "sup"})) 205 | 206 | (is (= n (client/count-tasks *client*))) 207 | 208 | (is (= n (client/count-queue *client* "queue5"))) 209 | )) 210 | 211 | (deftest count-http-test 212 | (elect! *nodes*) 213 | 214 | (let [n 10] 215 | (dotimes [i n] 216 | (http/post "http://127.0.0.1:13100/tasks/enqueue" 217 | {:form-params {:task {:queue "queue6" :data "sup"} :w 3} 218 | :content-type :json 219 | :as :json})) 220 | 221 | (let [resp (http/get "http://127.0.0.1:13100/tasks/count" {:as :json}) 222 | content-type (get-in resp [:headers "content-type"])] 223 | (is (= 200 (:status resp))) 224 | (is (= "application/json;charset=utf-8" content-type)) 225 | (is (= n (-> resp :body :count)))) 226 | (let [resp (http/post "http://127.0.0.1:13100/tasks/count" 227 | {:throw-exceptions false})] 228 | (is (= 405 (:status resp)))) 229 | 230 | (let [resp (http/get "http://127.0.0.1:13100/queue/count" 231 | {:query-params {:queue "queue6"} 232 | :as :json}) 233 | content-type (get-in resp [:headers "content-type"])] 234 | (is (= 200 (:status resp))) 235 | (is (= "application/json;charset=utf-8" content-type)) 236 | (is (= n (-> resp :body :count)))) 237 | (let [resp (http/post "http://127.0.0.1:13100/queue/count" 238 | {:throw-exceptions false})] 239 | (is (= 405 (:status resp)))))) 240 | 241 | (deftest list-tasks-test 242 | ; Enqueue 243 | (let [n 10] 244 | (dotimes [i n] 245 | (client/enqueue! *client* {:w 3} {:queue "queue7" :data "sup"})) 246 | 247 | ; List 248 | (let [tasks (client/list-tasks *client*)] 249 | (is (= n (count tasks))) 250 | (is (= (sort (map :id tasks)) (map :id tasks))) 251 | (is (every? :data tasks))))) 252 | 253 | (deftest list-tasks-http-test 254 | (let [n 10] 255 | (dotimes [i n] 256 | (client/enqueue! *client* {:w 3} {:queue "queue8" :data "sup"})) 257 | 258 | (let [resp (http/get "http://127.0.0.1:13100/tasks/list" {:as :json}) 259 | content-type (get-in resp [:headers "content-type"]) 260 | tasks (-> resp :body :tasks) 261 | ids (map (comp b64->id :id) tasks)] 262 | (is (= 200 (:status resp))) 263 | (is (= "application/json;charset=utf-8" content-type)) 264 | (is (= n (count tasks))) 265 | (is (= (sort ids) ids)) 266 | (is (every? :data tasks))) 267 | (let [resp (http/post "http://127.0.0.1:13100/tasks/list" 268 | {:throw-exceptions false})] 269 | (is (= 405 (:status resp)))))) 270 | 271 | (deftest get-task-http-test 272 | (let [id (http/post "http://127.0.0.1:13100/tasks/enqueue" 273 | {:form-params {:task {:queue "queue9" :data "sup"} :w 3} 274 | :content-type :json 275 | :as :json}) 276 | id (-> id :body :id)] 277 | (let [resp (http/get (str "http://127.0.0.1:13100/tasks/" id) {:as :json}) 278 | content-type (get-in resp [:headers "content-type"]) 279 | data (-> resp :body :task :data)] 280 | (is (= 200 (:status resp))) 281 | (is (= "application/json;charset=utf-8" content-type)) 282 | (is (= data "sup"))) 283 | 284 | (let [resp (http/post (str "http://127.0.0.1:13100/tasks/" id) 285 | {:throw-exceptions false})] 286 | (is (= 405 (:status resp)))))) 287 | 288 | (deftest get-id-http-test 289 | (let [resp (http/get "http://127.0.0.1:13100/id" 290 | {:content-type :json 291 | :as :json}) 292 | id (-> resp :body :id)] 293 | (is (= 200 (:status resp))) 294 | (is (string? id)) 295 | (is (= 28 (count id))))) 296 | 297 | (deftest enqueue-http-test 298 | (let [resp (http/post "http://127.0.0.1:13100/tasks/enqueue" 299 | {:form-params {:task {:queue "queue10" :data "sup"} :w 3} 300 | :content-type :json 301 | :as :json}) 302 | content-type (get-in resp [:headers "content-type"]) 303 | id (-> resp :body :id)] 304 | (is (= 200 (:status resp))) 305 | (is (= "application/json;charset=utf-8" content-type)) 306 | (is (not (nil? id))) 307 | 308 | ;; Ensure we can retrieve the task 309 | (let [resp* (http/get (str "http://127.0.0.1:13100/tasks/" id) {:as :json}) 310 | data (-> resp* :body :task :data)] 311 | (is (= data "sup"))) 312 | 313 | (let [resp (http/get "http://127.0.0.1:13100/tasks/enqueue" 314 | {:throw-exceptions false})] 315 | (is (= 405 (:status resp)))))) 316 | 317 | (deftest complete-http-test 318 | (elect! *nodes*) 319 | 320 | ;; First enqueue a task 321 | (let [resp (http/post "http://127.0.0.1:13100/tasks/enqueue" 322 | {:form-params {:task {:queue "queue11" :data "sup"} :w 3} 323 | :content-type :json 324 | :as :json}) 325 | id (-> resp :body :id) 326 | resp* (http/get (str "http://127.0.0.1:13100/tasks/" id "?r=3") 327 | {:as :json}) 328 | claims (-> resp* :body :task :claims)] 329 | (is (= claims [])) 330 | 331 | ;; Now let's claim a task, i.e. the task we just enqueued 332 | (let [deadline (+ (flake/linear-time) 20000)] 333 | (loop [] 334 | (let [resp (http/post "http://127.0.0.1:13100/tasks/claim" 335 | {:form-params {:queue "queue11" 336 | :dt 300000} 337 | :content-type :json 338 | :as :json}) 339 | content-type (get-in resp [:headers "content-type"]) 340 | id* (-> resp :body :task :id)] 341 | (if (and (not id*) (< (flake/linear-time) deadline)) 342 | (do 343 | (Thread/sleep 500) 344 | (recur)) 345 | (let [resp* (http/get (str "http://127.0.0.1:13100/tasks/" id "?r=3") 346 | {:as :json}) 347 | claims (-> resp* :body :task :claims)] 348 | (is (= 200 (:status resp))) 349 | (is (= "application/json;charset=utf-8" content-type)) 350 | (is (= id id*)) 351 | (is (not= claims [])))))) 352 | 353 | 354 | ;; Finally let's complete it 355 | (let [uri (str "http://127.0.0.1:13100/tasks/complete/" id) 356 | cid 0 357 | resp (http/post uri {:form-params {:cid cid} 358 | :content-type :json 359 | :as :json}) 360 | content-type (get-in resp [:headers "content-type"]) 361 | resp* (http/get (str "http://127.0.0.1:13100/tasks/" id "?r=3") 362 | {:as :json}) 363 | completed (-> resp* :body :task :claims (nth cid) :completed)] 364 | (is (= 200 (:status resp))) 365 | (is (= "application/json;charset=utf-8" content-type)) 366 | (is (not (nil? completed))))))) 367 | 368 | (deftest bad-json-http-test 369 | (let [resp (http/post "http://127.0.0.1:13100/tasks/enqueue" 370 | {:body "{" ;; Bogus JSON 371 | :content-type :json 372 | :throw-exceptions false})] 373 | (is (= 400 (:status resp))))) 374 | 375 | (deftest bad-request-enqueue-http-test 376 | (let [resp (http/post "http://127.0.0.1:13100/tasks/enqueue" 377 | {:throw-exceptions false}) 378 | data (-> resp :body (json/parse-string true))] 379 | (is (= 400 (:status resp))) 380 | (is (= {:error "Missing required params"} data)))) 381 | 382 | (deftest missing-task-http-test 383 | (let [resp (http/get "http://127.0.0.1:13100/tasks/foo" 384 | {:throw-exceptions false}) 385 | content-type (get-in resp [:headers "content-type"]) 386 | data (-> resp :body (json/parse-string true))] 387 | (is (= 404 (:status resp))) 388 | (is (= "application/json;charset=utf-8" content-type)) 389 | (is (= {:error "No such task"} data)))) 390 | -------------------------------------------------------------------------------- /test/skuld/queue_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.queue-test 2 | (:use skuld.queue 3 | clojure.test) 4 | (:require [skuld.task :as task])) 5 | 6 | (skuld.flake/init!) 7 | 8 | (deftest enqueue-test 9 | (let [q (queues)] 10 | (is (zero? (count-queue q "one"))) 11 | 12 | (update! q {:queue "one" :id 0 :foo :bar}) 13 | (is (= 1 (count-queue q "one"))) 14 | 15 | (update! q {:queue "one" :id 0 :bar :baz}) 16 | (is (= 1 (count-queue q "one"))) 17 | 18 | (is (= (->Task 0 nil) (poll! q "one"))) 19 | (is (= nil (poll! q "one"))))) 20 | 21 | (deftest order-test 22 | (let [q (queues)] 23 | (update! q {:queue "two" :id 1 :priority 1}) 24 | (update! q {:queue "one" :id 1 :priority 1}) 25 | (update! q {:queue "one" :id 3 :priority 0}) 26 | (update! q {:queue "one" :id 2 :priority 2}) 27 | (update! q {:queue "one" :id 0 :priority 2}) 28 | 29 | (is (= [3 1 0 2 nil] 30 | (->> (partial poll! q "one") 31 | repeatedly 32 | (take 5) 33 | (map :id)))))) 34 | -------------------------------------------------------------------------------- /test/skuld/stress_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.stress-test 2 | (:use clojure.tools.logging 3 | clojure.test 4 | skuld.util 5 | skuld.node 6 | skuld.node-test) 7 | 8 | (:require [skuld.client :as client] 9 | [skuld.admin :as admin] 10 | [skuld.vnode :as vnode] 11 | [skuld.flake :as flake] 12 | [skuld.curator :as curator] 13 | [skuld.politics :as politics] 14 | [skuld.net :as net] 15 | [skuld.task :as task] 16 | [skuld.aae :as aae] 17 | [clojure.set :as set] 18 | clj-helix.admin) 19 | (:import com.aphyr.skuld.Bytes)) 20 | 21 | (use-fixtures :once once) 22 | (use-fixtures :each each) 23 | 24 | (deftest claim-stress-test 25 | (elect! *nodes*) 26 | 27 | (let [n 100 28 | ids (->> (repeatedly (fn [] 29 | (client/enqueue! *client* {:w 3} {:queue "queue12" :data "sup"}))) 30 | (take n) 31 | doall)] 32 | 33 | (is (not-any? nil? ids)) 34 | (is (= n (client/count-tasks *client*))) 35 | ; Claim all extant IDs 36 | (let [deadline (+ (flake/linear-time) 20000) 37 | claims (loop [claims {}] 38 | (if-let [t (client/claim! *client* "queue12" 100000)] 39 | (do 40 | ; Make sure we never double-claim 41 | (assert (not (get claims (:id t)))) 42 | (let [claims (assoc claims (:id t) t)] 43 | (if (= (count ids) (count claims)) 44 | claims 45 | (do 46 | (if-not t 47 | (Thread/sleep 500)) 48 | (recur claims))))) 49 | 50 | ; Out of claims? 51 | (if (> (flake/linear-time) deadline) 52 | claims 53 | (do 54 | (Thread/sleep 100) 55 | (recur claims)))))] 56 | (is (= (count ids) (count claims))) 57 | (is (= (set (keys claims)) (set ids)))))) 58 | 59 | (deftest election-handoff-test 60 | ; Shut down normal AAE initiators; we don't want them recovering data behind 61 | ; our backs. ;-) 62 | (dorun (pmap (comp aae/shutdown! :aae) *nodes*)) 63 | (dorun (pmap (comp politics/shutdown! :politics) *nodes*)) 64 | 65 | (elect! *nodes*) 66 | 67 | ; Enqueue something and claim it. 68 | (let [id (client/enqueue! *client* {:w 3} {:queue "queue13" :data "meow"}) 69 | deadline (+ (flake/linear-time) 20000) 70 | claim (loop [] 71 | (if-let [claim (client/claim! *client* "queue13" 100000)] 72 | claim 73 | (when (< (flake/linear-time) deadline) 74 | (Thread/sleep 500) 75 | (recur))))] 76 | (is (= id (:id claim))) 77 | 78 | ; Now kill 2 of the nodes which own that id, leaving one copy 79 | (let [originals (filter (fn [node] 80 | (->> node 81 | vnodes 82 | vals 83 | (mapcat (fn [v] 84 | (try 85 | (vnode/tasks v) 86 | (catch RuntimeException e [])))) 87 | (map :id) 88 | (some #{id}))) 89 | *nodes*) 90 | fallbacks (remove (set originals) *nodes*) 91 | [dead alive] (split-at 1 originals) 92 | _ (is (= 2 (count alive))) 93 | replacements (concat alive fallbacks)] 94 | 95 | ; Shut down 1/3 nodes 96 | (dorun (pmap shutdown! dead)) 97 | 98 | ; At this point, 1-2 nodes should have the claim 99 | ;(->> replacements 100 | ; (map vnodes) 101 | ; (map vals) 102 | ; (map (partial mapcat (fn [vnode] 103 | ; (try (doall (vnode/tasks vnode)) 104 | ; (catch RuntimeException e 105 | ; (.printStackTrace e) 106 | ; []))))) 107 | ; clojure.pprint/pprint) 108 | 109 | ; Wait for the preflist to converge on the replacement cohort 110 | (let [deadline (+ (flake/linear-time) 20000)] 111 | (while (not (and (= 3 (count (preflist (first alive) id))) 112 | (set/subset? 113 | (set (preflist (first alive) id)) 114 | (set (map (comp net/id :net) replacements))))) 115 | (if (> (flake/linear-time) deadline) 116 | (throw (RuntimeException. "Could not converge before deadline")) 117 | (Thread/sleep 500)))) 118 | 119 | 120 | ; Elect a new cohort 121 | (elect! replacements) 122 | 123 | ; Verify that we cannot re-claim the element. 124 | (is (<= 2 (->> replacements 125 | (map vnodes) 126 | (map vals) 127 | (map (partial mapcat vnode/tasks)) 128 | (map (partial some #(= id (:id %)))) 129 | (filter true?) 130 | count))) 131 | 132 | (let [c (client/client [(select-keys (first alive) [:host :port])])] 133 | (try 134 | (is (not (client/claim! c "queue13" 1000))) 135 | (finally 136 | (client/shutdown! c))))))) 137 | -------------------------------------------------------------------------------- /test/skuld/task_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.task-test 2 | (:refer-clojure :exclude [merge]) 3 | (:use skuld.task 4 | clojure.test) 5 | (:require skuld.flake-test 6 | [skuld.flake :as flake])) 7 | 8 | 9 | (deftest merge-claim-test 10 | (testing "empty" 11 | (is (= (merge-claims []) 12 | []))) 13 | 14 | (testing "one claim set" 15 | (is (= (merge-claims [[{:start 1 :end 2} nil {:start 3 :end 5}]]) 16 | [{:start 1 :end 2} nil {:start 3 :end 5}]))) 17 | 18 | (testing "several claim sets" 19 | (is (= (->> (merge-claims 20 | [[{:start 0 :end 2} nil {:start 6 :end 9}] 21 | [nil nil ] 22 | [nil {:start 3 :end 5} ] 23 | [{:start 1 :end 3} {:start 4 :end 5} nil ]]) 24 | (map #(select-keys % [:start :end]))) 25 | [{:start 0 :end 3} {:start 3 :end 5} {:start 6 :end 9}])))) 26 | 27 | (deftest merge-test 28 | (testing "empty" 29 | (is (= (merge) 30 | {:claims []}))) 31 | 32 | (testing "one" 33 | (let [t (task {:data :hi})] 34 | (is (= (merge t) t)))) 35 | 36 | (testing "more" 37 | (let [t (task {:data :hi})] 38 | (is (= (merge (assoc t :claims [{:start 0 :end 1}]) 39 | (assoc t :claims [nil nil {:start 2 :end 4}])) 40 | (assoc t :claims [{:start 0 :end 1} 41 | nil 42 | {:start 2 :end 4}]))))) 43 | 44 | (testing "completed" 45 | (let [t (task {:data :hi})] 46 | (is (= (merge (assoc t :claims [{:start 0 :end 1 :completed 100}]) 47 | (assoc t :claims [{:start 2 :end 4 :completed 50}])) 48 | (assoc t :claims [{:start 0 :end 4 :completed 50}]))))) 49 | 50 | (testing "completed without start" 51 | (let [t (task {:data :hi})] 52 | (is (= (merge (assoc t :claims [{:start 0 :end 1 :completed 100}]) 53 | (assoc t :claims [{:start 2 :end 4}]) 54 | (assoc t :claims [{:completed 50}])) 55 | (assoc t :claims [{:start 0 :end 4 :completed 50}])))))) 56 | 57 | 58 | (deftest claim-test 59 | (let [t (claim (task {:data :hi}) 10)] 60 | (is (claimed? t))) 61 | 62 | (with-redefs [clock-skew-buffer 0] 63 | (let [t (claim (task {:data :hi}) 0)] 64 | (Thread/sleep 1) 65 | (is (not (claimed? t)))))) 66 | 67 | (deftest request-claim-test 68 | (testing "nil" 69 | (is (thrown? IllegalStateException 70 | (request-claim nil 0 {:start 0 :end 1})))) 71 | 72 | (testing "completed" 73 | (is (thrown? IllegalStateException 74 | (-> (task {:data :cat}) 75 | (claim 10) 76 | (complete 0 0) 77 | (request-claim 0 {:start 0 :end 1}))))) 78 | 79 | (testing "conflicting index" 80 | (is (thrown? IllegalStateException 81 | (with-redefs [clock-skew-buffer 0] 82 | (-> (task {:data :kitten}) 83 | (request-claim 0 {:start 0 :end 10}) 84 | (request-claim 0 {:start 10000000 :end 20000000})))))) 85 | 86 | (testing "conflicting claim" 87 | (is (thrown? IllegalStateException 88 | (with-redefs [clock-skew-buffer 0] 89 | (-> (task {:data :kit}) 90 | (request-claim 0 {:start 0 :end 10}) 91 | (request-claim 1 {:start 9 :end 11})))))) 92 | 93 | (is (thrown? IllegalStateException 94 | (let [t (flake/linear-time)] 95 | (with-redefs [clock-skew-buffer 10] 96 | (-> (task {:data :kit}) 97 | (request-claim 0 {:start 0 :end 10}) 98 | (request-claim 1 {:start 19 :end 29})))))) 99 | 100 | (testing "valid claim" 101 | (is (with-redefs [clock-skew-buffer 0] 102 | (-> (task {:data :treat}) 103 | (request-claim 0 {:start 0 :end 10}) 104 | (request-claim 1 {:start 11 :end 21}) 105 | :claims 106 | (nth 1) 107 | (= {:start 11 :end 21})))) 108 | 109 | (is (with-redefs [clock-skew-buffer 10] 110 | (-> (task {:data :treat}) 111 | (request-claim 0 {:start 0 :end 10}) 112 | (request-claim 1 {:start 21 :end 31}) 113 | :claims 114 | (nth 1) 115 | (= {:start 21 :end 31}))))) 116 | 117 | (testing "sparse claim" 118 | (is (-> (task {:data :treat}) 119 | (request-claim 10 {:start 0 :end 10}) 120 | :claims 121 | (nth 10) 122 | (= {:start 0 :end 10}))))) 123 | 124 | (deftest complete-test 125 | (let [t (claim (task {:data :meow}) 10)] 126 | (is (not (completed? t))) 127 | (let [t' (complete t 0 50)] 128 | (testing "is completed" 129 | (is (completed? t')) 130 | (is (= 50 (:completed (first (:claims t')))))) 131 | 132 | (testing "is not claimable" 133 | (is (thrown? IllegalStateException 134 | (claim t' 10))))))) 135 | -------------------------------------------------------------------------------- /test/skuld/util_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.util-test 2 | (:use skuld.util 3 | clojure.test 4 | clojure.pprint)) 5 | 6 | (deftest sorted-interleave-test 7 | (is (= [1 2 3 4 5 6 7] 8 | (sorted-interleave [1 5] 9 | [] 10 | [2 4 6 7] 11 | nil 12 | [3]))) 13 | (testing "infinite" 14 | (is (= [1 1 2 2 3 3 4 5 6 7] 15 | (->> (iterate inc 1) 16 | (sorted-interleave [1 2 3]) 17 | (take 10))))) 18 | 19 | (testing "bench" 20 | (time (dorun (sorted-interleave (take 1000000 (iterate inc 0)) 21 | (take 1000000 (iterate inc 1)) 22 | (range 500 100000)))))) 23 | 24 | (deftest compare+-test 25 | (let [a {:a 0 :b 1 :c 2} 26 | b {:a 2 :b 1 :c 0}] 27 | (testing "unary" 28 | (is (= -1 (compare+ a b :a))) 29 | (is (= 0 (compare+ a b :b))) 30 | (is (= 1 (compare+ a b :c)))) 31 | 32 | (testing "binary" 33 | (is (= -1 (compare+ a b :a :c))) 34 | (is (= 1 (compare+ a b :c :a))) 35 | (is (= -1 (compare+ a b :b :a))) 36 | (is (= 1 (compare+ a b :b :c))) 37 | (is (= -1 (compare+ a b :a :b))) 38 | (is (= 1 (compare+ a b :c :b)))) 39 | 40 | (testing "ternary" 41 | (is (= -1 (compare+ a b :b :b :a))) 42 | (is (= -1 (compare+ a b :b :a :c)))))) 43 | -------------------------------------------------------------------------------- /test/skuld/zk_test.clj: -------------------------------------------------------------------------------- 1 | (ns skuld.zk-test 2 | "Supports zookeeper testing." 3 | (:require [skuld.logging :as logging]) 4 | (:import (org.apache.curator.test TestingServer))) 5 | 6 | (defmacro with-zk 7 | "Evaluates body with a zookeeper server running, and the connect string bound 8 | to the given variable. Ensures the ZK server is shut down at the end of the 9 | body. Example: 10 | 11 | (with-zk [zk-string] 12 | (connect-to zk-string) 13 | ...)" 14 | [[connect-string] & body] 15 | `(logging/suppress 16 | ["org.apache.zookeeper" "org.apache.helix" "org.apache.curator" "org.I0Itec.zkclient" "org.apache.zookeeper.server.SessionTrackerImpl"] 17 | (let [zk# (TestingServer.) 18 | ~connect-string (.getConnectString zk#)] 19 | (try 20 | ~@body 21 | (finally 22 | (.stop zk#) 23 | (.close zk#)))))) 24 | --------------------------------------------------------------------------------