├── .gitignore ├── .jshintrc ├── .travis.yml ├── README.md ├── index.js ├── lib ├── broadcast.js ├── candidate.js ├── cluster.js ├── default_cluster_options.js ├── default_node_options.js ├── follower.js ├── idle.js ├── leader.js ├── log.js ├── log_applier.js ├── node.js ├── peer.js ├── replicator.js ├── standby.js ├── state.js └── stopped.js ├── package.json └── tests ├── _cluster2.js ├── _connection.js ├── _connection2.js ├── _debug.js ├── _node.js ├── _node2.js ├── _persistence.js ├── _transport.js ├── _transport2.js ├── candidate.js ├── cluster.js ├── follower.js ├── integration.js ├── leader.js ├── log.js ├── node.js └── peer.js /.gitignore: -------------------------------------------------------------------------------- 1 | .DS_Store 2 | node_modules 3 | npm-debug.log 4 | coverage 5 | -------------------------------------------------------------------------------- /.jshintrc: -------------------------------------------------------------------------------- 1 | { 2 | "boss": true, 3 | "node": true, 4 | "strict": true, 5 | "white": true, 6 | "smarttabs": true, 7 | "maxlen": 80, 8 | "newcap": false, 9 | "undef": true, 10 | "unused": true, 11 | "onecase": true, 12 | "indent": 2, 13 | "sub": true 14 | } -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: node_js 2 | node_js: 3 | - "0.10" 4 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Skiff 2 | 3 | [![Build Status](https://travis-ci.org/pgte/skiff-algorithm.svg)](https://travis-ci.org/pgte/skiff-algorithm) 4 | [![Dependencies] (https://david-dm.org/pgte/skiff-algorithm.png)](https://david-dm.org/pgte/skiff-algorithm) 5 | [![Gitter](https://badges.gitter.im/Join Chat.svg)](https://gitter.im/pgte/skiff?utm_source=badge&utm_medium=badge&utm_campaign=pr-badge&utm_content=badge) 6 | 7 | Abstract Node.js implementation of the [Raft Consensus Algorithm](http://raftconsensus.github.io/). 8 | 9 | > If you're looking for a directly usable module, take a look at [skiff](https://github.com/pgte/skiff) (on top of LevelDB + Msgpack). 10 | 11 | Contents: 12 | 13 | * [Install](#install) 14 | * [Require](#require) 15 | * [Create a node](#create-a-node) 16 | * [Node API](#node-api) 17 | * [Plugins](#plugins) 18 | * [Cluster setup](#cluster-setup) 19 | * [License](#license) 20 | 21 | ## Install 22 | 23 | ```bash 24 | $ node install skiff --save 25 | ``` 26 | 27 | ## Require 28 | 29 | ```javascript 30 | var Node = require('skiff'); 31 | ``` 32 | 33 | ## Create a node 34 | 35 | ```javascript 36 | var node = Node(); 37 | ``` 38 | 39 | or, with options: 40 | 41 | ```javascript 42 | var options = { 43 | // ... 44 | }; 45 | var node = Node(options); 46 | ``` 47 | 48 | ### Node create options 49 | 50 | * `id`: id of the node. if not defined, it's self assigned. accessible on `node.id` 51 | * `standby`: if true, will start at the `standby` state instead of the `follower` state. In the `standby` state the node only waits for a leader to send commands. Defaults to `false`. 52 | * `cluster`: the id of the cluster this node will be a part of 53 | * `transport`: the transport to communicate with peers. See the [transport API](#transport-provider-api) 54 | * `persistence`: the node persistence layer. See the [persistence API](#persistence-provider-api) 55 | * `uuid`: function that generates a UUID. Defaults to using the [`cuid`](https://github.com/ericelliott/cuid) package. 56 | * `heartbeatInterval`: the interval between heartbeats sent from leader. defaults to 50 ms. 57 | * `minElectionTimeout`: the minimum election timeout. defaults to 150 ms. 58 | * `maxElectionTimeout`: the maximum election timeout. defaults to 300 ms. 59 | * `commandTimeout`: the maximum amount of time you're willing to wait for a command to propagate. Defaults to 3 seconds. You can override this in each command call. 60 | * `retainedLogEntries`: the maximum number of log entries that are committed to the state machine that should remain in memory. Defaults to 50. 61 | * `metadata`: to be used by plugins if necessary 62 | 63 | 64 | ## Node API 65 | 66 | ### .listen(options, listener) 67 | 68 | Makes the peer listen for peer communications. Takes the following arguments: 69 | 70 | * `options` - connection options, depends on the transport provider being used. 71 | * `listener` - a function with the following signature: `function (peerId, connection)`. The arguments for the listener function are: 72 | * `peerId` - the identification of the peer 73 | * `connection` - a connection with the peer, an object implementing the Connection API (see below). 74 | 75 | ### .join(peer, [peerMetadata], cb) 76 | 77 | Joins a peer into the cluster. 78 | 79 | ```javascript 80 | node.join(peer, cb); 81 | ``` 82 | 83 | The peer is a string describing the peer. The description depends on the transport you're using. 84 | 85 | ### .leave(peer, cb) 86 | 87 | Removes a peer from the cluster, 88 | 89 | ```javascript 90 | node.leave(peer, cb); 91 | ``` 92 | 93 | The peer is a string describing the peer. The description depends on the transport you're using. 94 | 95 | 96 | ### .command(command[, options], callback) 97 | 98 | Appends a command to the leader log. If node is not the leader, callback gets invoked with an error. Example: 99 | 100 | ```javascript 101 | node.command('some command', function(err) { 102 | if (err) { 103 | if (err.code == 'ENOTLEADER') { 104 | // redirect client to err.leader 105 | } 106 | } else { 107 | console.log('cluster agreed on this command'); 108 | } 109 | }); 110 | ``` 111 | 112 | This command times out after the `options.commandTimeout` passes by, but you can override this by passing in some options: 113 | 114 | ```javascript 115 | node.command('some command', {timeout: 5000}, function(err) { 116 | if (err) { 117 | if (err.code == 'ENOTLEADER') { 118 | // redirect client to err.leader 119 | } 120 | } else { 121 | console.log('cluster agreed on this command'); 122 | } 123 | }); 124 | ``` 125 | 126 | Command options are: 127 | 128 | * `timeout`: maximum time waiting to replicate to a majority. Defaults to node `options.commandTimeout`, which defaults to to 3000 (3 seconds). 129 | * `waitForNode`: node id to wait to commit to. This may be useful to enforce read-your-writes on proxying clients. Defaults to `undefined`. 130 | 131 | ### .peerMeta(url) 132 | 133 | Returns the peer metadata if the peer is known. 134 | 135 | 136 | ### Events 137 | 138 | A node emits the following events that may or not be interesting to you: 139 | 140 | * `error(error)` - when an unexpected error occurs. 141 | * `state(stateName)` - when a new state transition occurs. Possible values for `stateName` are: `idle`, `follower`, `candidate`, `leader`. 142 | * `loaded()` - when a node has loaded configuration from persistence provider. 143 | * `election timeout()` - when an election timeout occurs. 144 | * `applied log(logIndex)` - when a node applies a log entry to the state machine 145 | 146 | 147 | ## Plugins 148 | 149 | Skiff if failry high-level and doesn't implement the network transport or the persistence layers. Instead, you have to provide an implementation for these. 150 | 151 | ### Transport provider API 152 | 153 | The node `transport` option accepts a provider object that implements the following interface: 154 | 155 | * `connect(localNodeId, options)` — for connecting to the peer. returns a connection object. The `localNodeId` argument contains the local node id. 156 | * `listen(localNodeId, options, fn)` — for listening to incoming connection requests. The `fn` argument is a function with the signaure `function (peerId, connection)` that gets invoked when there is a connection request, passing in a connection object that implements the Connection API (see below). The `localNodeId` argument contains the local node id. 157 | 158 | #### Connection API 159 | 160 | The connection API implements the following interface: 161 | 162 | * `send(type, arguments, callback)` — for making a remote call into the peer. The `callback` argument is a function with the signature `function (err, result)`. 163 | * `receive(fn)` — listen for messages from the remote peer. The `fn` argument is a function with the signature `function (type, args, cb)`. `cb` is a function that accepts the reply arguments. 164 | * `close(callback)` — for closing the connection. The `callback` argument is a function with the signature `function (err)`. 165 | 166 | The connection object is an EventEmitter, emitting the following events: 167 | 168 | * `close` - once the connection closes 169 | 170 | ### Persistence provider API 171 | 172 | The node `persistence` option accepts a provider object that implements the following interface: 173 | 174 | * `saveMeta(nodeId, state, callback)` — saves the raft engine metadata. `nodeId` is a string that represents the current node. `state` is an arbitrary object (hash map) and `callback` is a function with the signature `function callback(err)`; 175 | * `loadMeta(nodeId, callback)` — loads the engine metadata state. `callback` is a function with the signature `function callback(err, state)`; 176 | * `applyCommand(nodeId, commitIndex, command, callback)` - applies a command to the node state machine. 177 | * Persistence layer should save the commitIndex if it wants to make sure that log entries are not repeated. 178 | * Saving this should be atomic: the `commitIndex` and the log application to the state machine should be successful or fail entirely. 179 | * If the commitIndex has already been applied in the past, just callback with success. 180 | `callback` is a function with the following signature: `function callback(err)`. 181 | * `lastAppliedCommitIndex(nodeId, callback)` - returns the last `commitIndex` that was successfully applied to the node state machine. 182 | * is asynchronous: `callback` is a function invoked once the result is ready 183 | * `callback` is a function with the following signature: `function(err, commitIndex)` - if operation resulted in error, `err` contains an error object. Otherwise, `commitIndex` may contain an integer with the index of the latest applied `commitIndex` if there was one. 184 | * `saveCommitIndex(nodeId, commitIndex, callback)` - saves only the commit index 185 | * `createReadStream(nodeId)` - returns a read stream that streams all the state machine data. 186 | * `createWriteStream(nodeId)` - resets the state machine and returns a write stream to overwrite all the state machine data. 187 | * `removeAllState(nodeId, callback)` - remove all state for the given node 188 | 189 | ## Cluster Setup 190 | 191 | Setting up a Skiff cluster can be kind of tricky. To avoid partitions you will need to start with a node that will become leader and then add the followers in the standby mode. Mind you that you can only send `join` commands to a leader node (to avoid partitions — it's all explained in detail in the Raft paper). Once this is done and persisted you should never need to do this again since the nodes will know each other and elect a leader at random if leader goes down. 192 | 193 | 194 | So typically the bootstrap code for the leader would be something like: 195 | 196 | ```javascript 197 | var Node = require('skiff'); 198 | var leader = Node({ 199 | transport: transport, 200 | persistence: persistence 201 | }); 202 | 203 | leader.listen(address); 204 | 205 | /// wait for the leader node to actually become a leader of it's one node 206 | leader.once('leader', function() { 207 | leader.join('node1'); 208 | leader.join('node2'); 209 | }); 210 | 211 | leader.on('joined', function(peer) { 212 | console.log('leader joined %s', peer.id); 213 | }); 214 | ``` 215 | 216 | The follower bootstrapping code would look something like this: 217 | 218 | ```javascript 219 | var Node = require('skiff'); 220 | var node = Node({ 221 | transport: transport, 222 | persistence: persistence, 223 | standby: true // important 224 | }); 225 | 226 | node.listen(address); 227 | ``` 228 | 229 | This makes the follower start in the standby mode. 230 | 231 | As mentioned, once the cluster enters stationary mode you just need to bootstrap all the nodes in the same way: 232 | 233 | ```javascript 234 | var Node = require('skiff'); 235 | var node = Node({ 236 | transport: transport, 237 | persistence: persistence, 238 | }); 239 | 240 | node.listen(address); 241 | ``` 242 | 243 | 244 | ## License 245 | 246 | ISC 247 | 248 | © Pedro Teixeira 249 | -------------------------------------------------------------------------------- /index.js: -------------------------------------------------------------------------------- 1 | module.exports = require('./lib/node'); 2 | -------------------------------------------------------------------------------- /lib/broadcast.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var inherits = require('util').inherits; 4 | var EventEmitter = require('events').EventEmitter; 5 | 6 | module.exports = Broadcast; 7 | 8 | function Broadcast(node, peers, type, args) { 9 | var self = this; 10 | 11 | EventEmitter.apply(this); 12 | 13 | node.onceLoaded(function() { 14 | peers.forEach(broadcast); 15 | }); 16 | 17 | function broadcast(peer) { 18 | peer.send(type, args, replied); 19 | } 20 | 21 | function replied() { 22 | var args = Array.prototype.slice.call(arguments); 23 | args.unshift('response'); 24 | self.emit.apply(self, args); 25 | } 26 | } 27 | 28 | inherits(Broadcast, EventEmitter); 29 | 30 | var B = Broadcast.prototype; 31 | 32 | B.cancel = function cancel() { 33 | this.removeAllListeners(); 34 | }; 35 | -------------------------------------------------------------------------------- /lib/candidate.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var async = require('async'); 4 | var State = require('./state'); 5 | var inherits = require('util').inherits; 6 | 7 | module.exports = Candidate; 8 | 9 | function Candidate(node, options) { 10 | State.call(this, node); 11 | this.options = options; 12 | this.node.commonState.persisted.votedFor = this.node.id; 13 | 14 | this._startVoting(); 15 | } 16 | 17 | inherits(Candidate, State); 18 | 19 | var C = Candidate.prototype; 20 | 21 | C.name = 'candidate'; 22 | 23 | C._startVoting = function _startVoting() { 24 | var self = this; 25 | 26 | var votedForMe = 1; 27 | 28 | async.series([ 29 | startTimeout, 30 | incrementTerm, 31 | requestVotes 32 | ]); 33 | 34 | function startTimeout(cb) { 35 | self.node.startElectionTimeout(); 36 | self.once('election timeout', onElectionTimeout); 37 | cb(); 38 | } 39 | 40 | function onElectionTimeout() { 41 | self.node.toState('candidate'); 42 | } 43 | 44 | function incrementTerm(cb) { 45 | self.node.commonState.persisted.currentTerm += 1; 46 | cb(); 47 | } 48 | 49 | function requestVotes(cb) { 50 | var lastLog; 51 | var broadcast; 52 | 53 | verifyMajority(); 54 | if (!self.stopped) { 55 | if (self.node.commonState.persisted.log.length()) { 56 | lastLog = self.node.commonState. 57 | persisted.log.entryAt(self.node.commonState.persisted.log.length()); 58 | } 59 | 60 | var args = { 61 | term: self.node.commonState.persisted.currentTerm, 62 | candidateId: self.node.id, 63 | lastLogIndex: self.node.commonState.persisted.log.length(), 64 | lastLogTerm: lastLog && lastLog.term 65 | }; 66 | 67 | broadcast = self.node.broadcast('RequestVote', args); 68 | broadcast.on('response', self.unlessStopped(onBroadcastResponse)); 69 | } 70 | 71 | function onBroadcastResponse(err, args) { 72 | // TODO: what about the term update? 73 | if (args && args.voteGranted) { 74 | votedForMe ++; 75 | verifyMajority(); 76 | } 77 | } 78 | 79 | function verifyMajority() { 80 | if (self.node.isMajority(votedForMe)) { 81 | if (broadcast) { 82 | broadcast.cancel(); 83 | } 84 | setImmediate(function() { 85 | self.node.toState('leader'); 86 | cb(); 87 | }); 88 | } 89 | } 90 | } 91 | }; 92 | -------------------------------------------------------------------------------- /lib/cluster.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var extend = require('xtend'); 4 | var defaultOptions = require('./default_cluster_options'); 5 | 6 | module.exports = Cluster; 7 | 8 | function Cluster(options) { 9 | this.options = options = extend({}, defaultOptions, options); 10 | this.id = options.cluster || options.uuid(); 11 | } 12 | -------------------------------------------------------------------------------- /lib/default_cluster_options.js: -------------------------------------------------------------------------------- 1 | module.exports = { 2 | uuid: require('cuid') 3 | }; 4 | -------------------------------------------------------------------------------- /lib/default_node_options.js: -------------------------------------------------------------------------------- 1 | module.exports = { 2 | standby: false, 3 | minElectionTimeout: 150, 4 | maxElectionTimeout: 300, 5 | heartbeatInterval: 50, 6 | uuid: require('cuid'), 7 | commandTimeout: 3e3, 8 | replicationStreamHighWaterMark: 10, 9 | retainedLogEntries: 50, 10 | metadata: {} 11 | }; 12 | -------------------------------------------------------------------------------- /lib/follower.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var State = require('./state'); 4 | var inherits = require('util').inherits; 5 | var PassThrough = require('stream').PassThrough; 6 | 7 | module.exports = Follower; 8 | 9 | function Follower(node, options) { 10 | State.call(this, node); 11 | 12 | var self = this; 13 | 14 | this.options = options; 15 | this.lastHeardFromLeader = undefined; 16 | this.lastLeader = undefined; 17 | 18 | this.node.startElectionTimeout(); 19 | 20 | this.node.commonState.persisted.votedFor = null; 21 | 22 | this.on('election timeout', onElectionTimeout); 23 | 24 | function onElectionTimeout() { 25 | if (!self.stopped) { 26 | self.node.toState('candidate'); 27 | } 28 | } 29 | } 30 | 31 | inherits(Follower, State); 32 | 33 | var F = Follower.prototype; 34 | 35 | F.name = 'follower'; 36 | 37 | F.onAppendEntries = function onAppendEntries(args, cb) { 38 | var self = this; 39 | var log = this.node.commonState.persisted.log; 40 | var logEntry; 41 | 42 | this.node.startElectionTimeout(); 43 | 44 | if (args.leaderId) { 45 | this.node.commonState.volatile.leaderId = args.leaderId; 46 | } 47 | 48 | if (args.prevLogIndex) { 49 | logEntry = log.entryAt(args.prevLogIndex); 50 | 51 | if (!logEntry && 52 | args.prevLogIndex == log.lastIncludedIndex && 53 | args.term == log.lastIncludedTerm) 54 | { 55 | logEntry = { 56 | term: log.lastIncludedTerm 57 | }; 58 | } 59 | } 60 | 61 | if (args.term < this.node.currentTerm()) { 62 | callback(false, 'term is < than current term'); 63 | } 64 | else if (args.prevLogIndex && !logEntry) { 65 | callback(false, 'local node too far behind'); 66 | } 67 | else if (args.prevLogTerm && 68 | (!logEntry || logEntry.term != args.prevLogTerm)) { 69 | callback(false, 'node too far behind'); 70 | } 71 | else { 72 | self.lastLeader = args.leaderId; 73 | self.lastHeardFromLeader = Date.now(); 74 | 75 | self.node.commonState.persisted.currentTerm = args.term; 76 | 77 | self.node.commonState.volatile.commitIndex = Math.min( 78 | args.leaderCommit, self.node.commonState.persisted.log.length()); 79 | 80 | self.node.commonState.persisted.log.pushEntries( 81 | args.prevLogIndex, args.entries); 82 | 83 | process.nextTick(function() { 84 | self.node.logApplier.persist(persisted); 85 | }); 86 | } 87 | 88 | function persisted(err) { 89 | if (err) { 90 | self.emit('error', err); 91 | cb(err); 92 | } 93 | else { 94 | callback(true); 95 | } 96 | } 97 | 98 | function callback(success, reason) { 99 | var m = { 100 | term: self.node.currentTerm(), 101 | success: success, 102 | lastApplied: self.node.commonState.volatile.lastApplied 103 | }; 104 | if (reason) { 105 | m.reason = reason; 106 | } 107 | cb(null, m); 108 | } 109 | 110 | }; 111 | 112 | F.onRequestVote = function onRequestVote(args, cb) { 113 | // do not let false candidates disrupt the cluster 114 | // detect false candidates by denying vote requests 115 | // that come in before the minimum timeout occurs 116 | // after receiving a message 117 | var minimumTimeout = this.lastHeardFromLeader + 118 | this.options.minElectionTimeout; 119 | 120 | if (this.lastHeardFromLeader && minimumTimeout > Date.now()) { 121 | cb(null, { 122 | term: this.node.currentTerm(), 123 | voteGranted: false, 124 | reason: 'too soon' 125 | }); 126 | } 127 | else { 128 | // call super 129 | State.prototype.onRequestVote.call(this, args, cb); 130 | } 131 | }; 132 | 133 | F.onInstallSnapshot = function onInstallSnapshot(args, cb) { 134 | var self = this; 135 | var calledback = false; 136 | var lastIncludedIndex; 137 | var lastIncludedTerm; 138 | 139 | this.node.startElectionTimeout(); 140 | 141 | if (args.term >= this.node.currentTerm()) { 142 | if (!this.installingSnapshot && !args.first) { 143 | cb(new Error('expected first snapshot chunk:' + JSON.stringify(args))); 144 | } 145 | else { 146 | if (args.lastIncludedIndex) { 147 | lastIncludedIndex = args.lastIncludedIndex; 148 | } 149 | if (args.lastIncludedTerm) { 150 | lastIncludedTerm = args.lastIncludedTerm; 151 | } 152 | if (args.first) { 153 | this.node.options.persistence.removeAllState( 154 | this.node.id, 155 | function(err) { 156 | if (err) { 157 | cb(err); 158 | } 159 | else { 160 | self.installingSnapshot = new PassThrough({objectMode: true}); 161 | self.installingSnapshot. 162 | pipe( 163 | self.node.options.persistence.createWriteStream( 164 | self.node.id)). 165 | once('finish', onceWriteStreamFinishes); 166 | 167 | if (args.data) { 168 | self.installingSnapshot.write(args.data, callback); 169 | } 170 | if (args.done) { 171 | self.installingSnapshot.end(); 172 | } 173 | } 174 | } 175 | ); 176 | 177 | } 178 | else if (args.data && this.installingSnapshot) { 179 | this.installingSnapshot.write(args.data, callback); 180 | } 181 | else { 182 | callback(); 183 | } 184 | 185 | if (args.done && !args.first) { 186 | this.installingSnapshot.end(args.data); 187 | } 188 | } 189 | } else { 190 | callback(new Error( 191 | 'current term is ' + this.node.currentTerm() + ', not ' + args.term)); 192 | } 193 | 194 | function onceWriteStreamFinishes() { 195 | self.installingSnapshot = false; 196 | var state = self.node.commonState; 197 | state.volatile.lastApplied = lastIncludedIndex; 198 | state.volatile.commmitIndex = lastIncludedIndex; 199 | state.persisted.log.lastIncludedIndex = lastIncludedIndex; 200 | state.persisted.log.lastIncludedTerm = lastIncludedTerm; 201 | state.persisted.log.entries = []; 202 | } 203 | 204 | function callback(err) { 205 | if (!calledback) { 206 | calledback = true; 207 | if (err) { 208 | cb(err); 209 | } else { 210 | cb(null, {term: self.node.currentTerm()}); 211 | } 212 | } 213 | } 214 | }; 215 | -------------------------------------------------------------------------------- /lib/idle.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var State = require('./state'); 4 | var inherits = require('util').inherits; 5 | 6 | module.exports = Follower; 7 | 8 | function Follower(node) { 9 | State.call(this, node); 10 | } 11 | 12 | inherits(Follower, State); 13 | 14 | var F = Follower.prototype; 15 | 16 | F.name = 'idle'; 17 | -------------------------------------------------------------------------------- /lib/leader.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var State = require('./state'); 4 | var inherits = require('util').inherits; 5 | var Replicator = require('./replicator'); 6 | 7 | module.exports = Leader; 8 | 9 | function Leader(node, options) { 10 | var self = this; 11 | 12 | State.call(this, node); 13 | 14 | this.options = options; 15 | 16 | this.interval = undefined; 17 | this.peers = {}; 18 | 19 | node.commonState.persisted.peers.forEach(addPeer); 20 | this.on('joined', addPeer); 21 | this.on('reconnected', addPeer); 22 | this.on('left', removePeer); 23 | 24 | function addPeer(peer) { 25 | self.peers[peer.id] = { 26 | meta: peer, 27 | nextIndex: node.commonState.persisted.log.length() + 1 28 | }; 29 | } 30 | function removePeer(peer) { 31 | delete self.peers[peer.id]; 32 | } 33 | 34 | this.node.commonState.volatile.leaderId = node.id; 35 | 36 | this.replicator = new Replicator(node, this.peers, options); 37 | this.replicator.on('error', function(err) { 38 | self.emit('error', err); 39 | }); 40 | this.once('stopped', function stopped() { 41 | self.removeListener('joined', addPeer); 42 | self.removeListener('reconnected', addPeer); 43 | self.removeListener('left', removePeer); 44 | self.replicator.removeListener('response', onReplicateResponse); 45 | self.replicator.stop(); 46 | }); 47 | this.replicator.on('response', onReplicateResponse); 48 | 49 | function onReplicateResponse(peerId, logIndex, entryCount, err, args) { 50 | var peer = self.peers[peerId]; 51 | if (peer) { 52 | if (err) { 53 | self.emit('warning', err); 54 | } 55 | else if (args && args.term > self.node.currentTerm()) { 56 | self.node.currentTerm(args.term); 57 | self.node.toState('follower'); 58 | } 59 | else if (args && args.success) { 60 | peer.nextIndex = logIndex + entryCount; 61 | setImmediate(function() { 62 | self.emit('replication success', peerId, logIndex, args.lastApplied); 63 | }); 64 | } 65 | else { 66 | if (typeof args.lastApplied == 'number') { 67 | peer.nextIndex = Math.max(args.lastApplied + 1); 68 | } 69 | else { 70 | peer.nextIndex = Math.max(peer.nextIndex - 1, 0); 71 | } 72 | 73 | self.replicator.retry(peerId); 74 | } 75 | } 76 | } 77 | } 78 | 79 | inherits(Leader, State); 80 | 81 | var L = Leader.prototype; 82 | 83 | L.name = 'leader'; 84 | 85 | L.replicate = function replicate(logIndex, options, cb) { 86 | var self = this; 87 | 88 | if (typeof options == 'function') { 89 | cb = options; 90 | options = {}; 91 | } 92 | 93 | if (!options) { 94 | options = {}; 95 | } 96 | 97 | var yep = 1; // count self 98 | var done = {}; 99 | var lastApplieds = {}; 100 | var timeout; 101 | var replied = false; 102 | 103 | if (!maybeStop()) { 104 | if (options.timeout > 0) { 105 | timeout = setTimeout(timedout, options.timeout); 106 | timeout.unref(); 107 | } 108 | this.on('replication success', onReplicationSuccess); 109 | this.replicator.replicate(); 110 | } 111 | 112 | function onReplicationSuccess(peerId, peerLogIndex, lastApplied) { 113 | if (!done[peerId] && peerLogIndex >= logIndex) { 114 | done[peerId] = true; 115 | yep ++; 116 | } 117 | lastApplieds[peerId] = lastApplied; 118 | maybeStop(); 119 | } 120 | 121 | function maybeStop() { 122 | var stop = shouldStop(); 123 | if (stop) { 124 | reply(); 125 | } 126 | return stop; 127 | } 128 | 129 | function shouldStop() { 130 | var stop = false; 131 | 132 | if (self.node.isMajority(yep)) { 133 | if (!options.waitForNode || options.waitForNode == self.node.id) { 134 | stop = true; 135 | } else if (done[options.waitForNode]) { 136 | if (options.waitForNodeLastApplied) { 137 | if (lastApplieds[options.waitForNode] >= 138 | options.waitForNodeLastApplied) { 139 | stop = true; 140 | } 141 | } else { 142 | stop = true; 143 | } 144 | } 145 | } 146 | 147 | return stop; 148 | } 149 | 150 | function timedout() { 151 | timeout = undefined; 152 | reply(new Error( 153 | 'timedout after ' + options.timeout + 154 | ' ms trying to replicate log index ' + logIndex)); 155 | } 156 | 157 | function reply(err) { 158 | if (!replied) { 159 | replied = true; 160 | self.removeListener('replication success', onReplicationSuccess); 161 | if (timeout) { 162 | clearTimeout(timeout); 163 | timeout = undefined; 164 | } 165 | if (cb) { 166 | cb(err); 167 | } 168 | } 169 | } 170 | 171 | }; 172 | -------------------------------------------------------------------------------- /lib/log.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var uuid = require('cuid'); 4 | 5 | module.exports = Log; 6 | 7 | function Log(node, options, doc) { 8 | if (!doc) { 9 | doc = { 10 | meta: { 11 | lastIncludedIndex: 0, 12 | lastIncludedTerm: 0 13 | }, 14 | entries: [] 15 | }; 16 | } 17 | 18 | this.node = node; 19 | this.options = options; 20 | this.entries = doc.entries; 21 | this.lastIncludedIndex = doc.meta.lastIncludedIndex; 22 | this.lastIncludedTerm = doc.meta.lastIncludedTerm; 23 | } 24 | 25 | var L = Log.prototype; 26 | 27 | L.push = function push() { 28 | var self = this; 29 | var entries = Array.prototype.slice.call(arguments); 30 | entries.forEach(processEntry); 31 | 32 | function processEntry(entry) { 33 | entry.uuid = uuid(); 34 | entry.index = self.length() + 1; 35 | self.entries.push(entry); 36 | if (entry.topologyChange) { 37 | self.node.applyTopologyChange(entry.command); 38 | } 39 | } 40 | }; 41 | 42 | L.pushEntries = function pushEntries(startIndex, entries) { 43 | if (entries && entries.length) { 44 | var self = this; 45 | this.entries.splice(startIndex - this.lastIncludedIndex); 46 | entries.forEach(function(entry) { 47 | self.push(entry); 48 | }); 49 | } 50 | }; 51 | 52 | L.applied = function applied(appliedIndex) { 53 | var toCut = appliedIndex - this.lastIncludedIndex - 54 | this.options.retainedLogEntries; 55 | if (toCut > 0) { 56 | var cutAt = this.entries[toCut - 1]; 57 | this.entries.splice(0, toCut); 58 | this.lastIncludedIndex += toCut; 59 | this.lastIncludedTerm = cutAt.term; 60 | } 61 | }; 62 | 63 | L.entryAt = function entryAt(index) { 64 | return this.entries[index - this.lastIncludedIndex - 1]; 65 | }; 66 | 67 | L.last = function last() { 68 | return this.entries.length ? 69 | this.entries[this.entries.length - 1] : undefined; 70 | }; 71 | 72 | L.length = function length() { 73 | return this.lastIncludedIndex + this.entries.length; 74 | }; 75 | 76 | L.streamline = function streamline() { 77 | return { 78 | meta: { 79 | lastIncludedIndex: this.lastIncludedIndex, 80 | lastIncludedTerm: this.lastIncludedTerm 81 | }, 82 | entries: this.entries 83 | }; 84 | }; 85 | -------------------------------------------------------------------------------- /lib/log_applier.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | module.exports = LogApplier; 4 | 5 | var inherits = require('util').inherits; 6 | var EventEmitter = require('events').EventEmitter; 7 | 8 | function LogApplier(log, node, persistence) { 9 | EventEmitter.call(this); 10 | this.setMaxListeners(Infinity); 11 | this.log = log; 12 | this.node = node; 13 | this.persistence = persistence; 14 | this.persisting = false; 15 | 16 | this.persist(); 17 | } 18 | 19 | inherits(LogApplier, EventEmitter); 20 | 21 | var LA = LogApplier.prototype; 22 | 23 | LA.persist = function persist(cb) { 24 | var self = this; 25 | var state = self.node.commonState.volatile; 26 | var toApply = state.lastApplied + 1; 27 | 28 | if (!this.persisting) { 29 | this.persisting = true; 30 | if (state.commitIndex > state.lastApplied) { 31 | var entry = self.node.commonState.persisted.log.entryAt(toApply); 32 | if (!entry) { 33 | done(); 34 | } 35 | else if (entry.topologyChange) { 36 | // this is an internal command, a topology command 37 | // that was already processed (topology commands are processed 38 | // once they are inserted into the log): 39 | // we do not need send it to the persistence layer. 40 | self.persistence.saveCommitIndex(self.node.id, toApply, function(err) { 41 | if (err) { 42 | done(err); 43 | } 44 | else { 45 | self.node.save(persisted); 46 | } 47 | }); 48 | } 49 | else { 50 | self.persistence.applyCommand( 51 | self.node.id, toApply, entry.command, persisted); 52 | } 53 | } else { 54 | this.persisting = false; 55 | self.emit('done persisting', state.lastApplied); 56 | done(); 57 | } 58 | } 59 | else if (cb) { 60 | self.once('done persisting', cb); 61 | } 62 | 63 | function done(err) { 64 | self.persisting = false; 65 | if (err && !cb) { 66 | self.emit('error', err); 67 | } 68 | if (cb) { 69 | cb(err); 70 | } 71 | } 72 | 73 | function persisted(err) { 74 | if (err) { 75 | done(err); 76 | } 77 | else { 78 | self.persisting = false; 79 | state.lastApplied = toApply; 80 | self.emit('applied log', toApply); 81 | self.persist(cb); 82 | } 83 | } 84 | }; 85 | -------------------------------------------------------------------------------- /lib/node.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Log = require('./log'); 4 | var Peer = require('./peer'); 5 | var async = require('async'); 6 | var extend = require('xtend'); 7 | var assert = require('assert'); 8 | var Cluster = require('./cluster'); 9 | var inherits = require('util').inherits; 10 | var Broadcast = require('./broadcast'); 11 | var LogApplier = require('./log_applier'); 12 | var EventEmitter = require('events').EventEmitter; 13 | var defaultOptions = require('./default_node_options'); 14 | 15 | var states = { 16 | 'standby': require('./standby'), 17 | 'idle': require('./idle'), 18 | 'stopped': require('./stopped'), 19 | 'follower': require('./follower'), 20 | 'candidate': require('./candidate'), 21 | 'leader': require('./leader') 22 | }; 23 | 24 | module.exports = Node; 25 | 26 | function Node(options) { 27 | if (!(this instanceof Node)) { 28 | return new Node(options); 29 | } 30 | 31 | this.options = extend({}, defaultOptions, options); 32 | assert(this.options.transport, 'need options.transport'); 33 | 34 | EventEmitter.apply(this); 35 | this.setMaxListeners(Infinity); 36 | 37 | var self = this; 38 | 39 | this.id = this.options.id; 40 | if (!this.id) { 41 | this.id = this.options.uuid(); 42 | } 43 | this.metadata = this.options.metadata; 44 | 45 | this.cluster = new Cluster(options); 46 | this.loaded = false; 47 | this.stopped = false; 48 | 49 | this.commonState = { 50 | volatile: { 51 | leaderId: null, 52 | commitIndex: 0, 53 | lastApplied: 0 54 | }, 55 | persisted: { 56 | currentTerm: 0, 57 | votedFor: null, 58 | log: new Log(self, this.options), 59 | peers: [] 60 | } 61 | }; 62 | 63 | this.logApplier = new LogApplier( 64 | this.commonState.persisted.log, this, this.options.persistence); 65 | this.logApplier.on('error', function(err) { 66 | self.emit('error', err); 67 | }); 68 | this.logApplier.on('applied log', function(logIndex) { 69 | self.emit('applied log', logIndex); 70 | }); 71 | this.logApplier.on('done persisting', function(lastApplied) { 72 | self.commonState.persisted.log.applied( 73 | self.commonState.volatile.lastApplied); 74 | self.emit('done persisting', lastApplied); 75 | }); 76 | 77 | this.on('election timeout', function() { 78 | this.state.emit('election timeout'); 79 | }); 80 | 81 | this.on('joined', function(peer) { 82 | this.state.emit('joined', peer); 83 | }); 84 | 85 | this.toState('idle'); 86 | 87 | this.load(loaded); 88 | 89 | function loaded(err) { 90 | if (err) { 91 | process.nextTick(function() { 92 | self.emit('error', err); 93 | }); 94 | } else { 95 | self.loaded = true; 96 | self.emit('loaded'); 97 | self.toState(self.options.standby ? 'standby' : 'follower'); 98 | } 99 | } 100 | } 101 | 102 | inherits(Node, EventEmitter); 103 | 104 | var N = Node.prototype; 105 | 106 | /// persistent node state 107 | 108 | N.load = function load(cb) { 109 | var self = this; 110 | 111 | async.parallel([ 112 | loadMeta, 113 | loadCommitIndex 114 | ], done); 115 | 116 | function loadMeta(cb) { 117 | self.options.persistence.loadMeta(self.id, cb); 118 | } 119 | 120 | function loadCommitIndex(cb) { 121 | self.options.persistence.lastAppliedCommitIndex(self.id, cb); 122 | } 123 | 124 | function done(err, results) { 125 | var state = self.commonState; 126 | if (!err && results) { 127 | if (results[0]) { 128 | state.persisted = results[0]; 129 | state.persisted.peers = state.persisted.peers.map(newPeer); 130 | state.persisted.log = new Log(self, self.options, state.persisted.log); 131 | } 132 | 133 | if (results[1]) { 134 | self.commonState.volatile.lastApplied = results[1]; 135 | } 136 | } 137 | cb(err); 138 | } 139 | 140 | function newPeer(peerDesc) { 141 | var peer = new Peer( 142 | peerDesc.id, peerDesc.metadata, self.options, null, self); 143 | self._join(peer); 144 | 145 | return peer; 146 | } 147 | }; 148 | 149 | N.save = function save(cb) { 150 | var state = extend({}, this.commonState.persisted, { 151 | log: this.commonState.persisted.log.streamline(), 152 | peers: this.commonState.persisted.peers.map(function(peer) { 153 | return { 154 | id: peer.id, 155 | metadata: peer.metadata 156 | }; 157 | }) 158 | }); 159 | this.options.persistence.saveMeta(this.id, state, cb); 160 | }; 161 | 162 | N.onceLoaded = function onceLoaded(cb) { 163 | if (this.loaded) { 164 | cb(); 165 | } 166 | else { 167 | this.once('loaded', cb); 168 | } 169 | }; 170 | 171 | /// Listen 172 | 173 | N.listen = function listen(options, cb) { 174 | var self = this; 175 | 176 | if (this.server) { 177 | this.server.close(); 178 | } 179 | this.server = this.options.transport.listen(this.id, options, listener, cb); 180 | 181 | function listener(peerId, metadata, connection) { 182 | self._join(peerId, metadata, connection); 183 | } 184 | }; 185 | 186 | /// Peers 187 | 188 | N.join = function join(peerDesc, metadata, cb) { 189 | var self = this; 190 | 191 | if (peerDesc == this.id) { 192 | done(new Error('can\'t join self')); 193 | } 194 | else { 195 | if (typeof metadata == 'function') { 196 | cb = metadata; 197 | metadata = null; 198 | } 199 | async.series([ensureLeader, pushAddPeerCommand], done); 200 | } 201 | 202 | function ensureLeader(cb) { 203 | self.ensureLeader(cb); 204 | } 205 | 206 | function pushAddPeerCommand(cb) { 207 | var cmd = ['add peer', peerDesc, metadata]; 208 | self.command(cmd, cb, true); 209 | } 210 | 211 | function done(err) { 212 | if (err && !cb) { 213 | self.emit('error', err); 214 | } 215 | else if (cb) { 216 | cb(err); 217 | } 218 | } 219 | }; 220 | 221 | N.leave = function leave(peerDesc, cb) { 222 | var self = this; 223 | 224 | async.series( 225 | [ 226 | ensureLeader, 227 | pushRemovePeerCommand, 228 | maybeStopSelf 229 | ], cb); 230 | 231 | function ensureLeader(cb) { 232 | self.ensureLeader(cb); 233 | } 234 | 235 | function pushRemovePeerCommand(cb) { 236 | var cmd = ['remove peer', peerDesc]; 237 | self.command(cmd, cb, true); 238 | } 239 | 240 | function maybeStopSelf(cb) { 241 | if (peerDesc == self.id) { 242 | self.toState('stopped'); 243 | setTimeout(function() { 244 | self.stop(); 245 | }, 1e3); 246 | } 247 | cb(); 248 | } 249 | }; 250 | 251 | N.peerMeta = function peerMeta(url) { 252 | var peers = this.commonState.persisted.peers; 253 | var found = false; 254 | var peer; 255 | for (var i = 0 ; i < peers.length ; i ++) { 256 | peer = peers[i]; 257 | if (peer.id == url) { 258 | found = true; 259 | break; 260 | } 261 | } 262 | if (found) { 263 | return peer.metadata; 264 | } 265 | }; 266 | 267 | N._join = function _join(peerDesc, metadata, connection) { 268 | var self = this; 269 | 270 | if (peerDesc != this.id) { 271 | var peer; 272 | var found; 273 | 274 | for (var i = 0 ; i < this.commonState.persisted.peers.length ; i ++) { 275 | peer = this.commonState.persisted.peers[i]; 276 | if (peer.id == peerDesc) { 277 | found = peer; 278 | break; 279 | } 280 | } 281 | if (found) { 282 | peer.disconnect(); 283 | peer.removeAllListeners(); 284 | } 285 | // peer is not on our list, let's add it 286 | peer = peerDesc; 287 | if (!(peerDesc instanceof Peer)) { 288 | peer = new Peer(peerDesc, metadata, this.options, connection, self); 289 | } 290 | if (found) { 291 | this.commonState.persisted.peers[i] = peer; 292 | } 293 | else { 294 | this.commonState.persisted.peers.push(peer); 295 | } 296 | 297 | if (!connection) { 298 | peer.connect(); 299 | } 300 | 301 | peer.on('call', onPeerCall); 302 | peer.once('connection closed', function() { 303 | peer.removeListener('call', onPeerCall); 304 | }); 305 | peer.on('outgoing call', onPeerOutgoingCall); 306 | peer.on('response', onPeerResponse); 307 | peer.once('connected', onPeerConnected); 308 | peer.once('close', onPeerClose); 309 | peer.on('connected', onPeerConnected); 310 | peer.on('disconnected', onPeerDisconnected); 311 | peer.on('connecting', onPeerConnecting); 312 | 313 | if (found) { 314 | self.emit('reconnected', peer); 315 | } 316 | else { 317 | self.emit('joined', peer); 318 | } 319 | } 320 | 321 | function onPeerCall(type, args, cb) { 322 | self.handlePeerCall(peer, type, args, cb); 323 | } 324 | 325 | function onPeerOutgoingCall(type, args) { 326 | self.emit('outgoing call', peer, type, args); 327 | } 328 | 329 | function onPeerResponse(err, args) { 330 | self.emit('response', peer, err, args); 331 | } 332 | 333 | function onPeerConnected() { 334 | self.emit('connected', peer); 335 | } 336 | 337 | function onPeerClose() { 338 | self.emit('close', peer); 339 | } 340 | 341 | function onPeerDisconnected() { 342 | self.emit('disconnected', peer); 343 | } 344 | 345 | function onPeerConnecting() { 346 | self.emit('connecting', peer); 347 | } 348 | }; 349 | 350 | N._leave = function _leave(peerDesc) { 351 | var peers = this.commonState.persisted.peers; 352 | var peer; 353 | var peerIndex; 354 | 355 | for (peerIndex = 0 ; peerIndex < peers.length ; peerIndex ++) { 356 | peer = peers[peerIndex]; 357 | if (peer.id == peerDesc) { 358 | break; 359 | } 360 | } 361 | if (peer) { 362 | this.commonState.persisted.peers.splice(peerIndex, 1); 363 | peer.disconnect(); 364 | peer.removeAllListeners(); 365 | this.state.emit('left', peer); 366 | this.emit('left', peer); 367 | } 368 | }; 369 | 370 | /// state transition 371 | 372 | N.toState = function toState(state) { 373 | var self = this; 374 | var previousState = self.state; 375 | 376 | this.cancelElectionTimeout(); 377 | 378 | var Ctor = states[state]; 379 | if (!Ctor) { 380 | throw new Error('Unknown state: ' + state); 381 | } 382 | 383 | if (previousState) { 384 | previousState.stop(); 385 | } 386 | self.state = new Ctor(self, self.options); 387 | self.state.on('error', function(err) { 388 | self.emit('error', err); 389 | }); 390 | self.state.on('warning', function(warn) { 391 | self.emit('warning', warn); 392 | }); 393 | self.emit('state', state, self); 394 | self.emit(state, self); 395 | }; 396 | 397 | /// broadcast 398 | 399 | N.broadcast = function broadcast(type, args) { 400 | return new Broadcast(this, this.commonState.persisted.peers, type, args); 401 | }; 402 | 403 | /// majority 404 | 405 | N.isMajority = function isMajority(quorum) { 406 | var majority = Math.ceil((this.commonState.persisted.peers.length + 1) / 2); 407 | return quorum >= majority; 408 | }; 409 | 410 | /// term 411 | 412 | N.currentTerm = function currentTerm(term) { 413 | if (!term) { 414 | term = this.commonState.persisted.currentTerm; 415 | } 416 | else { 417 | this.commonState.persisted.currentTerm = term; 418 | } 419 | 420 | return term; 421 | }; 422 | 423 | N.startElectionTimeout = function startElectionTimeout() { 424 | this.emit('reset election timeout'); 425 | var self = this; 426 | 427 | if (this.electionTimeout) { 428 | clearTimeout(this.electionTimeout); 429 | } 430 | 431 | this.electionTimeout = setTimeout(function() { 432 | self.emit('election timeout'); 433 | }, this.randomElectionTimeout()); 434 | }; 435 | 436 | N.cancelElectionTimeout = function cancelElectionTimeout() { 437 | if (this.electionTimeout) { 438 | clearTimeout(this.electionTimeout); 439 | this.electionTimeout = null; 440 | } 441 | }; 442 | 443 | N.randomElectionTimeout = function randomElectionTimeout() { 444 | var minElectionTimeout = this.options.minElectionTimeout; 445 | var maxElectionTimeout = this.options.maxElectionTimeout; 446 | 447 | if (maxElectionTimeout < minElectionTimeout) { 448 | this.emit('error', 449 | new Error('maxElectionTimeout is greater than minElectionTimeout')); 450 | } 451 | 452 | var diff = maxElectionTimeout - minElectionTimeout; 453 | var d = Math.floor(Math.random() * diff); 454 | 455 | var timeout = minElectionTimeout + d; 456 | 457 | return timeout; 458 | }; 459 | 460 | /// Client API 461 | 462 | N.ensureLeader = function ensureLeader(cb) { 463 | var err; 464 | 465 | if (this.state.name != 'leader') { 466 | err = new Error('not the leader'); 467 | err.code = 'ENOTLEADER'; 468 | err.leader = this.commonState.volatile.leaderId; 469 | } 470 | cb(err); 471 | }; 472 | 473 | N.currentLeader = function() { 474 | return this.commonState.volatile.leaderId; 475 | }; 476 | 477 | N.command = function command(cmd, options, cb, isTopologyChange) { 478 | var self = this; 479 | var commitIndex; 480 | 481 | if (typeof options == 'function') { 482 | isTopologyChange = cb; 483 | cb = options; 484 | options = undefined; 485 | } 486 | 487 | options = extend({}, {timeout: this.options.commandTimeout}, options); 488 | 489 | async.series([ 490 | ensureLoaded, 491 | ensureLeader, 492 | pushLogEntry, 493 | replicate, 494 | applyLogs, 495 | persist, 496 | replicateAgain 497 | ], cb); 498 | 499 | function ensureLoaded(cb) { 500 | self.onceLoaded(cb); 501 | } 502 | 503 | function ensureLeader(cb) { 504 | self.ensureLeader(cb); 505 | } 506 | 507 | function pushLogEntry(cb) { 508 | var entry = { 509 | term: self.currentTerm(), 510 | command: cmd 511 | }; 512 | if (isTopologyChange) { 513 | entry.topologyChange = true; 514 | } 515 | self.commonState.persisted.log.push(entry); 516 | cb(); 517 | } 518 | 519 | function replicate(cb) { 520 | commitIndex = self.commonState.persisted.log.length(); 521 | self.state.replicate(commitIndex, options, cb); 522 | } 523 | 524 | function applyLogs(cb) { 525 | self.commonState.volatile.commitIndex = commitIndex; 526 | self.logApplier.persist(cb); 527 | } 528 | 529 | function persist(cb) { 530 | self.save(cb); 531 | } 532 | 533 | function replicateAgain(cb) { 534 | options.waitForNodeLastApplied = commitIndex; 535 | self.state.replicate(commitIndex, options, cb); 536 | } 537 | }; 538 | 539 | /// handle peer calls 540 | 541 | N.handlePeerCall = function handlePeerCall(peer, type, args, cb) { 542 | var self = this; 543 | var requestArgs = args; 544 | 545 | if (!this.stopped) { 546 | this.onceLoaded(function() { 547 | var handler = peerCallHandlers[type]; 548 | if (!handler) { 549 | self.emit('error', new Error('unknown peer call type: ' + type)); 550 | } 551 | else { 552 | handler.call(self, args, handlerReplied); 553 | } 554 | }); 555 | } 556 | 557 | function handlerReplied() { 558 | var args = arguments; 559 | self.emit('reply', args); 560 | 561 | if (requestArgs.term && requestArgs.term > self.currentTerm()) { 562 | self.currentTerm(args.term); 563 | } 564 | 565 | self.save(function(err) { 566 | if (err) { 567 | self.emit('error', err); 568 | } 569 | else { 570 | cb.apply(null, args); 571 | } 572 | }); 573 | } 574 | }; 575 | 576 | /// peer call handlers 577 | 578 | var peerCallHandlers = {}; 579 | ['AppendEntries', 'RequestVote', 'InstallSnapshot'].forEach(installRpcHandler); 580 | 581 | function installRpcHandler(type) { 582 | var methodName = 'on' + type; 583 | N[methodName] = peerCallHandlers[type] = handler; 584 | 585 | function handler(args, cb) { 586 | /* jshint validthis: true */ 587 | this.emit(type, args); 588 | this.state[methodName](args, cb); 589 | } 590 | } 591 | 592 | /// Topology changes 593 | 594 | var topologyChangeCommands = { 595 | 'add peer': N._join, 596 | 'remove peer': N._leave 597 | }; 598 | 599 | N.applyTopologyChange = function applyTopologyChange(entry) { 600 | var fn = topologyChangeCommands[entry[0]]; 601 | if (fn) { 602 | fn.call(this, entry[1]); 603 | } 604 | }; 605 | 606 | /// stop 607 | 608 | N.stop = function stop(cb) { 609 | if (!this.stopped) { 610 | this.stopped = true; 611 | this.toState('stopped'); 612 | 613 | if (this.server) { 614 | this.server.close(cb); 615 | } else { 616 | setImmediate(cb); 617 | } 618 | 619 | this.commonState.persisted.peers.forEach(disconnectPeer); 620 | 621 | } else if (cb) { 622 | setImmediate(cb); 623 | } 624 | 625 | function disconnectPeer(peer) { 626 | peer.disconnect(); 627 | } 628 | }; 629 | 630 | N.close = N.stop; 631 | -------------------------------------------------------------------------------- /lib/peer.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var async = require('async'); 4 | var assert = require('assert'); 5 | var inherits = require('util').inherits; 6 | var Writable = require('stream').Writable; 7 | var propagate = require('propagate'); 8 | var EventEmitter = require('events').EventEmitter; 9 | 10 | module.exports = Peer; 11 | 12 | var removeOlderEntries = ['AppendEntries', 'RequestVote']; 13 | 14 | function Peer(id, metadata, options, connection, node) { 15 | var self = this; 16 | 17 | if (!(this instanceof Peer)) { 18 | return new Peer(id, metadata, options, connection, node); 19 | } 20 | 21 | EventEmitter.call(this); 22 | 23 | assert.ok(typeof id === 'string', 'peer id must be string'); 24 | assert.ok(options.transport, 'No transport defined'); 25 | 26 | this.id = id; 27 | this.metadata = metadata; 28 | this.options = options; 29 | this.transport = options.transport; 30 | this.node = node; 31 | this.disconnected = false; 32 | 33 | this.queues = { 34 | out: async.queue(invoke, 1), 35 | in: async.queue(receive, 1) 36 | }; 37 | 38 | if (connection) { 39 | this.setupConnection(connection); 40 | } 41 | 42 | function invoke(work, done) { 43 | var calledback = false; 44 | self._invoke.call(self, work.type, work.args, callback); 45 | 46 | function callback() { 47 | if (!calledback) { 48 | calledback = true; 49 | work.cb.apply(null, arguments); 50 | done(); 51 | } 52 | } 53 | } 54 | 55 | function receive(message, cb) { 56 | self.emit('call', message.type, message.args, callback); 57 | 58 | function callback() { 59 | message.cb.apply(null, arguments); 60 | cb(); 61 | } 62 | } 63 | } 64 | 65 | inherits(Peer, EventEmitter); 66 | 67 | var P = Peer.prototype; 68 | 69 | P.connect = function connect() { 70 | var connection = this.transport.connect( 71 | this.node.id, this.node.metadata, this.id, this.metadata); 72 | this.setupConnection(connection); 73 | return connection; 74 | }; 75 | 76 | P.setupConnection = function setupConnection(connection) { 77 | var self = this; 78 | 79 | this.connection = connection; 80 | connection.receive(onMessage); 81 | connection.once('close', onConnectionClose); 82 | connection.on('error', onConnectionError); 83 | 84 | propagate(['disconnected', 'connected', 'connecting'], connection, this); 85 | 86 | this.emit('connected'); 87 | 88 | function onMessage(type, args, cb) { 89 | self.queues.in.push({ 90 | type: type, 91 | args: args, 92 | cb: cb 93 | }); 94 | } 95 | 96 | function onConnectionClose() { 97 | self.emit('close'); 98 | } 99 | 100 | function onConnectionError(err) { 101 | if (!self.disconnected) { 102 | self.emit('error', err); 103 | } 104 | } 105 | }; 106 | 107 | P.disconnect = function disconnect() { 108 | var self = this; 109 | 110 | this.disconnected = true; 111 | 112 | this.connection.close(closed); 113 | 114 | function closed(err) { 115 | if (err) { 116 | self.emit('error', err); 117 | } 118 | else { 119 | self.emit('connection closed'); 120 | } 121 | } 122 | }; 123 | 124 | P._invoke = function _invoke(type, args, cb) { 125 | var self = this; 126 | var calledback = false; 127 | 128 | var timeout; 129 | 130 | if (this.node.options) { 131 | setTimeout(onTimeout, this.node.options.commandTimeout); 132 | } 133 | 134 | this.emit('outgoing call', type, args); 135 | if (!this.connection) { 136 | cb(new Error('not connected')); 137 | } 138 | else { 139 | this.connection.send(type, args, callback); 140 | } 141 | 142 | function callback(err, args) { 143 | if (!calledback) { 144 | calledback = true; 145 | if (timeout) { 146 | clearTimeout(timeout); 147 | } 148 | if (!err || !err.timeout) { 149 | self.emit('response', err, args); 150 | } 151 | 152 | cb.apply(null, arguments); 153 | } 154 | } 155 | 156 | function onTimeout() { 157 | timeout = undefined; 158 | var err = new Error( 159 | 'invoke timeout after ' + self.node.options.commandTimeout + ' ms'); 160 | err.timeout = true; 161 | callback(err); 162 | } 163 | }; 164 | 165 | P.send = function invoke(type, args, cb) { 166 | if (removeOlderEntries.indexOf(type) > -1) { 167 | var tasks = this.queues.out.tasks; 168 | var task; 169 | for (var i = tasks.length - 1 ; i >= 0; i --) { 170 | task = tasks[i]; 171 | if (task.data.type == type) { 172 | tasks.splice(i, 1); 173 | } 174 | } 175 | } 176 | 177 | this.queues.out.push({ 178 | type: type, 179 | args: args, 180 | cb: cb 181 | }); 182 | }; 183 | 184 | /// toJSON 185 | 186 | P.toJSON = function toJSON() { 187 | return this.id; 188 | }; 189 | 190 | /// createWriteStream 191 | 192 | P.createWriteStream = function createWriteStream() { 193 | var self = this; 194 | var first = true; 195 | var error; 196 | var lastIncludedIndex = self.node.commonState.volatile.lastApplied; 197 | var lastIncludedTerm = self.node.currentTerm(); 198 | 199 | var stream = new Writable({ 200 | objectMode: true, 201 | highWaterMark: this.options.replicationStreamHighWaterMark 202 | }); 203 | 204 | stream.on('error', function(err) { 205 | error = err; 206 | self.emit('error', err); 207 | }); 208 | 209 | stream._write = function(chunk, _, callback) { 210 | self.send('InstallSnapshot', { 211 | term: self.node.currentTerm(), 212 | leaderId: self.node.id, 213 | data: chunk, 214 | first: first, 215 | done: false, 216 | lastIncludedIndex: lastIncludedIndex, 217 | lastIncludedTerm: lastIncludedTerm 218 | }, callback); 219 | 220 | first = false; 221 | }; 222 | 223 | stream.once('finish', function() { 224 | if (!error) { 225 | self.send('InstallSnapshot', { 226 | term: self.node.currentTerm(), 227 | leaderId: self.node.id, 228 | data: null, 229 | first: first, 230 | done: true, 231 | lastIncludedIndex: lastIncludedIndex, 232 | lastIncludedTerm: lastIncludedTerm 233 | }, onReply); 234 | 235 | first = false; 236 | } 237 | }); 238 | 239 | return stream; 240 | 241 | function onReply(err) { 242 | if (err) { 243 | stream.emit('error', err); 244 | } 245 | } 246 | }; 247 | -------------------------------------------------------------------------------- /lib/replicator.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var inherits = require('util').inherits; 4 | var EventEmitter = require('events').EventEmitter; 5 | 6 | module.exports = Replicate; 7 | 8 | function Replicate(node, peers, options) { 9 | EventEmitter.apply(this); 10 | this.setMaxListeners(Infinity); 11 | 12 | this.node = node; 13 | this.peers = peers; 14 | this.options = options; 15 | 16 | this.replicate(); 17 | } 18 | 19 | inherits(Replicate, EventEmitter); 20 | 21 | var R = Replicate.prototype; 22 | 23 | R.scheduleHeartbeat = function scheduleHeartbeat() { 24 | var self = this; 25 | 26 | if (this.interval) { 27 | clearInterval(this.interval); 28 | } 29 | 30 | this.interval = setInterval(heartbeat, this.options.heartbeatInterval); 31 | 32 | function heartbeat() { 33 | self.node.emit('heartbeat'); 34 | self.replicate(); 35 | } 36 | }; 37 | 38 | R.stop = function cancel() { 39 | this.removeAllListeners(); 40 | clearInterval(this.interval); 41 | }; 42 | 43 | R.replicate = function replicate() { 44 | var self = this; 45 | 46 | this.scheduleHeartbeat(); 47 | 48 | var peerIds = Object.keys(this.peers); 49 | 50 | peerIds.forEach(function(peerId) { 51 | self.replicateToPeer(peerId); 52 | }); 53 | }; 54 | 55 | R.replicateToPeer = function replicateToPeer(peerId) { 56 | var self = this; 57 | var peer = this.peers[peerId]; 58 | var index = peer.nextIndex; 59 | var log = this.node.commonState.persisted.log; 60 | var entries; 61 | 62 | if (peer.nextIndex <= log.lastIncludedIndex && 63 | this.node.commonState.volatile.commitIndex > 0) 64 | { 65 | this.streamSnapshotToPeer(peerId); 66 | } 67 | else { 68 | if (log.length() >= index) { 69 | entries = [log.entryAt(index)]; 70 | } else { 71 | entries = []; 72 | } 73 | 74 | var args = { 75 | term: this.node.currentTerm(), 76 | leaderId: this.node.id, 77 | prevLogIndex: index - 1, 78 | entries: entries, 79 | leaderCommit: this.node.commonState.volatile.commitIndex 80 | }; 81 | 82 | peer.meta.send('AppendEntries', args, replied); 83 | } 84 | 85 | function replied() { 86 | var args = Array.prototype.slice.call(arguments); 87 | args = ['response', peer.meta.id, index, entries.length].concat(args); 88 | self.emit.apply(self, args); 89 | } 90 | }; 91 | 92 | R.retry = function retry(peerId) { 93 | this.replicateToPeer(peerId); 94 | }; 95 | 96 | R.streamSnapshotToPeer = function streamSnapshotToPeer(peerId) { 97 | var self = this; 98 | var peer = this.peers[peerId]; 99 | var nextIndex = this.node.commonState.volatile.commitIndex + 1; 100 | if (peer && !peer.streaming) { 101 | peer.streaming = true; 102 | 103 | var rs = this.node.options.persistence.createReadStream(this.node.id); 104 | var ws = peer.meta.createWriteStream(); 105 | rs.pipe(ws).once('finish', onStreamFinish); 106 | } 107 | 108 | function onStreamFinish() { 109 | peer.streaming = false; 110 | peer.nextIndex = nextIndex; 111 | self.replicate(peerId); 112 | } 113 | }; 114 | -------------------------------------------------------------------------------- /lib/standby.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var State = require('./state'); 4 | var inherits = require('util').inherits; 5 | 6 | module.exports = Standby; 7 | 8 | function Standby(node) { 9 | State.call(this, node); 10 | } 11 | 12 | inherits(Standby, State); 13 | 14 | var S = Standby.prototype; 15 | 16 | S.name = 'standby'; 17 | -------------------------------------------------------------------------------- /lib/state.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var inherits = require('util').inherits; 4 | var EventEmitter = require('events').EventEmitter; 5 | 6 | module.exports = State; 7 | 8 | function State(node) { 9 | this.node = node; 10 | EventEmitter.call(this); 11 | this.setMaxListeners(Infinity); 12 | this.stopped = false; 13 | this.installingSnapshot = false; 14 | 15 | this.once('stopped', function() { 16 | this.on('error', function() { 17 | // empty on purpose, we don't care about 18 | // state errors after stopped 19 | }); 20 | }); 21 | } 22 | 23 | inherits(State, EventEmitter); 24 | 25 | var S = State.prototype; 26 | 27 | S.stop = function stop() { 28 | var self = this; 29 | 30 | this.stopped = true; 31 | setImmediate(function() { 32 | self.emit('stopped'); 33 | }); 34 | }; 35 | 36 | S.unlessStopped = function unlessStopped(fn) { 37 | var self = this; 38 | 39 | return function() { 40 | if (!self.stopped) { 41 | fn.apply(this, arguments); 42 | } 43 | }; 44 | }; 45 | 46 | S.onAppendEntries = function onAppendEntries(args, cb) { 47 | if (args.term >= this.node.currentTerm()) { 48 | this.node.currentTerm(args.term); 49 | this.node.commonState.volatile.leaderId = args.leaderId; 50 | this.node.toState('follower'); 51 | this.node.onAppendEntries(args, cb); 52 | } else { 53 | cb(null, { 54 | term: this.node.currentTerm(), 55 | success: false, 56 | reason: 'term is behind current term' 57 | }); 58 | } 59 | }; 60 | 61 | S.onRequestVote = function onRequestVote(args, cb) { 62 | var self = this; 63 | var currentTerm = this.node.currentTerm(); 64 | var state = this.node.commonState.persisted; 65 | if (args.term < currentTerm) { 66 | callback(false); 67 | } 68 | else if (!state.votedFor || state.votedFor == args.candidateId) { 69 | var lastLog = state.log.last(); 70 | if (lastLog && lastLog.term < args.lastLogTerm) { 71 | callback(true); 72 | } 73 | else if (args.lastLogIndex >= state.log.length()) { 74 | callback(true); 75 | } 76 | else { 77 | callback(false); 78 | } 79 | } 80 | else { 81 | callback(false); 82 | } 83 | 84 | function callback(grant) { 85 | if (grant) { 86 | state.votedFor = args.candidateId; 87 | self.node.emit('vote granted', state.votedFor); 88 | } 89 | cb(null, {term: currentTerm, voteGranted: grant}); 90 | } 91 | }; 92 | 93 | S.onInstallSnapshot = function onInstallSnapshot(args, cb) { 94 | if (args.term >= this.node.currentTerm()) { 95 | this.node.currentTerm(args.term); 96 | this.node.commonState.volatile.leaderId = args.leaderId; 97 | this.node.toState('follower'); 98 | this.node.onInstallSnapshot(args, cb); 99 | } else { 100 | cb(null, { 101 | term: this.node.currentTerm(), 102 | success: false, 103 | reason: 'term is behind current term' 104 | }); 105 | } 106 | }; 107 | -------------------------------------------------------------------------------- /lib/stopped.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var State = require('./state'); 4 | var inherits = require('util').inherits; 5 | 6 | module.exports = Stopped; 7 | 8 | function Stopped(node) { 9 | State.call(this, node); 10 | } 11 | 12 | inherits(Stopped, State); 13 | 14 | var S = Stopped.prototype; 15 | 16 | S.name = 'stopped'; 17 | 18 | S.onAppendEntries = function onAppendEntries(args, cb) { 19 | cb(null, { 20 | term: this.node.currentTerm(), 21 | success: false, 22 | lastApplied: this.node.commonState.volatile.lastApplied, 23 | reason: 'stopped' 24 | }); 25 | }; 26 | 27 | S.onRequestVote = function onRequestVote(args, cb) { 28 | cb(null, { 29 | term: this.node.currentTerm(), 30 | voteGranted: false, 31 | reason: 'stopped' 32 | }); 33 | }; 34 | 35 | S.onInstallSnapshot = function onInstallSnapshot(args, cb) { 36 | cb(new Error('stopped')); 37 | }; 38 | -------------------------------------------------------------------------------- /package.json: -------------------------------------------------------------------------------- 1 | { 2 | "name": "skiff-algorithm", 3 | "version": "0.8.2", 4 | "description": "Implementation of the Raft consensus algorithm in Node.js", 5 | "scripts": { 6 | "test": "istanbul cover lab -- tests/*.js -l && istanbul check-coverage --statements 93 --functions 90 --lines 90 --branches 80", 7 | "jshint": "jshint -c .jshintrc --exclude-path .gitignore .", 8 | "codestyle": "jscs -p google lib/ index.js tests/", 9 | "coverage": "open coverage/lcov-report/index.html" 10 | }, 11 | "repository": { 12 | "type": "git", 13 | "url": "https://github.com/pgte/skiff.git" 14 | }, 15 | "keywords": [ 16 | "raft", 17 | "distributed", 18 | "consensus", 19 | "election", 20 | "vote" 21 | ], 22 | "author": "pgte", 23 | "license": "ISC", 24 | "bugs": { 25 | "url": "https://github.com/pgte/skiff/issues" 26 | }, 27 | "homepage": "https://github.com/pgte/skiff", 28 | "devDependencies": { 29 | "jshint": "^2.5.6", 30 | "lab": "^4.5.1", 31 | "sinon": "^1.10.3", 32 | "jscs": "^1.6.2", 33 | "pre-commit": "0.0.9", 34 | "istanbul": "^0.3.2" 35 | }, 36 | "dependencies": { 37 | "async": "^0.9.0", 38 | "cuid": "^1.2.4", 39 | "propagate": "^0.3.0", 40 | "xtend": "^4.0.0" 41 | }, 42 | "pre-commit": [ 43 | "codestyle", 44 | "jshint", 45 | "test" 46 | ] 47 | } 48 | -------------------------------------------------------------------------------- /tests/_cluster2.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var NodeC = require('./_node2'); 4 | 5 | module.exports = Cluster; 6 | 7 | function Cluster(count, cb) { 8 | var leader = NodeC(); 9 | leader.listen(leader.id); 10 | count --; 11 | leader.once('leader', function() { 12 | var nodes = []; 13 | var node; 14 | for (var i = 0 ; i < count ; i ++) { 15 | node = NodeC({standby: true}); 16 | node.listen(node.id); 17 | leader.join(node.id); 18 | nodes.push(node); 19 | } 20 | if (cb) { 21 | cb(leader, nodes); 22 | } 23 | }); 24 | 25 | return leader; 26 | } 27 | -------------------------------------------------------------------------------- /tests/_connection.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var inherits = require('util').inherits; 4 | var EventEmitter = require('events').EventEmitter; 5 | 6 | module.exports = Connection; 7 | 8 | function Connection(id, hub) { 9 | 10 | EventEmitter.call(this); 11 | 12 | this.id = id; 13 | this.hub = hub; 14 | } 15 | 16 | inherits(Connection, EventEmitter); 17 | 18 | var C = Connection.prototype; 19 | 20 | C.send = function send(type, args, cb) { 21 | var self = this; 22 | 23 | setTimeout(function() { 24 | var fn = self.hub.out[self.id]; 25 | if (fn) { 26 | fn.call(null, type, args, cb); 27 | } 28 | else { 29 | cb.call(null, new Error('cannot connect to ' + self.id)); 30 | } 31 | }, 5); 32 | }; 33 | 34 | C.receive = function receive(cb) { 35 | this.hub.in[this.id] = cb; 36 | }; 37 | 38 | C.close = function(cb) { 39 | var self = this; 40 | 41 | if (this.hub.out[this.id]) { 42 | delete this.hub.out[this.id]; 43 | } 44 | if (this.hub.in[this.id]) { 45 | delete this.hub.in[this.id]; 46 | } 47 | 48 | setTimeout(function() { 49 | self.emit('close'); 50 | if (cb) { 51 | cb(); 52 | } 53 | }, 5); 54 | }; 55 | -------------------------------------------------------------------------------- /tests/_connection2.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var inherits = require('util').inherits; 4 | var EventEmitter = require('events').EventEmitter; 5 | 6 | module.exports = Connection2; 7 | 8 | function Connection2(local, localMetadata, remote, metadata, hub, secondary) { 9 | var self = this; 10 | 11 | EventEmitter.call(this); 12 | 13 | this.local = local; 14 | this.localMetadata = localMetadata; 15 | this.metadata = metadata; 16 | this.remote = remote; 17 | this.hub = hub; 18 | 19 | hub.connected(local, remote); 20 | hub.connected(remote, local); 21 | 22 | if (!secondary) { 23 | setTimeout(function() { 24 | var listener = self.hub.listens[remote]; 25 | 26 | if (listener) { 27 | var otherConnection = new Connection2( 28 | self.remote, metadata, self.local, localMetadata, hub, true); 29 | 30 | listener.call(null, self.local, self.localMetadata, otherConnection); 31 | } 32 | }, 5); 33 | } 34 | } 35 | 36 | inherits(Connection2, EventEmitter); 37 | 38 | var C = Connection2.prototype; 39 | 40 | C.send = function send(type, args, cb) { 41 | var self = this; 42 | 43 | setTimeout(function() { 44 | var local = self.hub.connections[self.local]; 45 | var fn = local && local[self.remote]; 46 | if (fn) { 47 | fn.call(null, type, args, cb); 48 | } 49 | else { 50 | cb.call(null, new Error('cannot connect remote ' + self.remote)); 51 | } 52 | }, 5); 53 | }; 54 | 55 | C.receive = function receive(cb) { 56 | if (!this.hub.connections[this.remote]) { 57 | this.hub.connections[this.remote] = {}; 58 | } 59 | this.hub.connections[this.remote][this.local] = cb; 60 | }; 61 | 62 | C.close = function close(cb) { 63 | var self = this; 64 | 65 | if (this.hub.disconnected(this.local, this.remote)) { 66 | var local = this.hub.connections[this.local]; 67 | if (local) { 68 | delete local[this.remote]; 69 | } 70 | } 71 | 72 | if (this.hub.disconnected(this.remote, this.local)) { 73 | var remote = this.hub.connections[this.remote]; 74 | if (remote) { 75 | delete remote[this.local]; 76 | } 77 | } 78 | 79 | setTimeout(function() { 80 | self.emit('close'); 81 | if (cb) { 82 | cb(); 83 | } 84 | }, 5); 85 | }; 86 | -------------------------------------------------------------------------------- /tests/_debug.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | function Logger(node) { 4 | return function log() { 5 | var s = arguments[0] || ''; 6 | s = '[' + Date.now() + '] [' + node.id + '] ' + s; 7 | arguments[0] = s; 8 | console.log.apply(console, arguments); 9 | }; 10 | } 11 | 12 | module.exports = debug; 13 | 14 | function debug(node) { 15 | var log = Logger(node); 16 | node.once('loaded', function() { 17 | log('loaded'); 18 | }); 19 | node.on('state', function(state) { 20 | log('state:', state); 21 | }); 22 | node.on('AppendEntries', function(args) { 23 | log('-> AppendEntries: %j', args); 24 | }); 25 | node.on('RequestVote', function(args) { 26 | log('-> RequestVote: %j', args); 27 | }); 28 | node.on('vote granted', function(node) { 29 | log('vote granted to', node); 30 | }); 31 | node.on('outgoing call', function(peer, type, message) { 32 | log('<- outgoing call to %s of type "%s": %j', peer.id, type, message); 33 | }); 34 | node.on('response', function(peer, err, args) { 35 | log('<- response: %j', peer.id, err, args); 36 | }); 37 | node.on('election timeout', function() { 38 | log('election timeout'); 39 | }); 40 | node.on('reply', function() { 41 | log('-> reply %j', arguments); 42 | }); 43 | node.on('heartbeat', function() { 44 | log('heartbeat'); 45 | }); 46 | node.on('reset election timeout', function() { 47 | log('reset election timeout'); 48 | }); 49 | node.on('joined', function(peer) { 50 | log('joined %s', peer.id); 51 | }); 52 | } 53 | 54 | debug.debug2 = function(node) { 55 | var log = Logger(node); 56 | node.on('state', function(state) { 57 | log(state, node.currentTerm()); 58 | }); 59 | node.on('vote granted', function(node) { 60 | log('voted for', node); 61 | }); 62 | node.on('AppendEntries', function(args) { 63 | log('AppendEntries from', args[0].leaderId); 64 | }); 65 | node.on('election timeout', function() { 66 | log('timed out'); 67 | }); 68 | }; 69 | -------------------------------------------------------------------------------- /tests/_node.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var uuid = require('cuid'); 4 | var extend = require('xtend'); 5 | 6 | var NNode = require('../'); 7 | var transport = require('./_transport'); 8 | var persistence = require('./_persistence'); 9 | 10 | module.exports = Node; 11 | 12 | function Node(options) { 13 | var id = uuid(); 14 | options = extend({}, { 15 | id: id, 16 | transport: transport, 17 | persistence: persistence 18 | }, options); 19 | 20 | var node = NNode(options); 21 | 22 | node.once('stopped', onceStopped); 23 | function onceStopped() { 24 | node.on('error', function(err) { 25 | console.error(err); 26 | }); 27 | } 28 | 29 | return node; 30 | } 31 | -------------------------------------------------------------------------------- /tests/_node2.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var uuid = require('cuid'); 4 | var extend = require('xtend'); 5 | 6 | var NNode = require('../'); 7 | var Transport = require('./_transport2'); 8 | var persistence = require('./_persistence'); 9 | 10 | module.exports = Node; 11 | 12 | function Node(options) { 13 | var id = uuid(); 14 | options = extend({}, { 15 | id: id, 16 | transport: new Transport(id), 17 | persistence: persistence 18 | }, options); 19 | 20 | return NNode(options); 21 | } 22 | -------------------------------------------------------------------------------- /tests/_persistence.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Readable = require('stream').Readable; 4 | var Writable = require('stream').Writable; 5 | 6 | var store = { 7 | meta: {}, 8 | state: {}, 9 | commands: {} 10 | }; 11 | 12 | exports.store = store; 13 | 14 | exports.saveMeta = saveMeta; 15 | 16 | function saveMeta(nodeId, state, callback) { 17 | setTimeout(function() { 18 | store.meta[nodeId] = JSON.stringify(state); 19 | callback(); 20 | }, randomTimeout()); 21 | } 22 | 23 | exports.loadMeta = loadMeta; 24 | 25 | function loadMeta(nodeId, callback) { 26 | var data = store.meta[nodeId]; 27 | if (data) { 28 | data = JSON.parse(data); 29 | } 30 | setImmediate(callback, null, data); 31 | } 32 | 33 | exports.applyCommand = applyCommand; 34 | 35 | function applyCommand(nodeId, commitIndex, command, callback) { 36 | setTimeout(function() { 37 | if (!store.commands[nodeId]) { 38 | store.commands[nodeId] = []; 39 | } 40 | store.commands[nodeId].push(command); 41 | store.state[nodeId] = commitIndex; 42 | callback(); 43 | }, randomTimeout()); 44 | } 45 | 46 | exports.lastAppliedCommitIndex = lastAppliedCommitIndex; 47 | 48 | function lastAppliedCommitIndex(nodeId, callback) { 49 | var commitIndex = store.meta[nodeId]; 50 | setTimeout(callback, 5, null, commitIndex); 51 | } 52 | 53 | exports.saveCommitIndex = saveCommitIndex; 54 | 55 | function saveCommitIndex(nodeId, commitIndex, cb) { 56 | setTimeout(function() { 57 | store.state[nodeId] = commitIndex; 58 | cb(); 59 | }, randomTimeout()); 60 | } 61 | 62 | function randomTimeout() { 63 | return Math.floor(Math.random() * 5); 64 | } 65 | 66 | exports.createReadStream = createReadStream; 67 | 68 | function createReadStream(nodeId) { 69 | var stream = new Readable({objectMode: true}); 70 | var commandIndex = -1; 71 | var finished = false; 72 | var commands = store.commands[nodeId] || []; 73 | var length = commands.length; 74 | 75 | stream._read = function _read() { 76 | var command; 77 | commandIndex += 1; 78 | if (commandIndex < length) { 79 | command = commands && commands[commandIndex]; 80 | } 81 | if (!command && !finished) { 82 | finished = true; 83 | stream.push(null); 84 | } else if (command) { 85 | stream.push(command); 86 | } 87 | }; 88 | 89 | return stream; 90 | } 91 | 92 | exports.createWriteStream = createWriteStream; 93 | 94 | function createWriteStream(nodeId) { 95 | var stream = new Writable({objectMode: true}); 96 | stream._write = function _write(chunk, encoding, callback) { 97 | store.commands[nodeId].push(chunk); 98 | setImmediate(callback); 99 | }; 100 | 101 | return stream; 102 | } 103 | 104 | exports.removeAllState = removeAllState; 105 | 106 | function removeAllState(nodeId, callback) { 107 | setTimeout(function() { 108 | store.commands[nodeId] = []; 109 | callback(); 110 | }, randomTimeout()); 111 | } 112 | -------------------------------------------------------------------------------- /tests/_transport.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Connection = require('./_connection'); 4 | 5 | var hub = { 6 | in: {}, 7 | out: {} 8 | }; 9 | 10 | exports.connect = connect; 11 | 12 | function connect(local, localMetadata, options) { 13 | return new Connection(options, hub); 14 | } 15 | 16 | exports.listen = listen; 17 | 18 | function listen(localid, id, fn) { 19 | hub.out[id] = fn; 20 | } 21 | 22 | exports.invoke = invoke; 23 | 24 | function invoke(id) { 25 | var args = Array.prototype.slice.call(arguments); 26 | process.nextTick(function() { 27 | var fn = hub.in[id]; 28 | if (fn) { 29 | args.shift(); 30 | fn.apply(null, args); 31 | } 32 | }); 33 | } 34 | -------------------------------------------------------------------------------- /tests/_transport2.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Connection2 = require('./_connection2'); 4 | 5 | var hub = { 6 | listens: {}, 7 | connections: {}, 8 | connectionCounts: {}, 9 | connected: function(from, to) { 10 | var key = from + ':' + to; 11 | if (!this.connectionCounts[key]) { 12 | this.connectionCounts[key] = 0; 13 | } 14 | this.connectionCounts[key] ++; 15 | }, 16 | disconnected: function(from, to) { 17 | var key = from + ':' + to; 18 | if (this.connectionCounts[key]) { 19 | this.connectionCounts[key] --; 20 | } 21 | return this.connectionCounts[key] === 0; 22 | } 23 | }; 24 | 25 | module.exports = exports = Transport; 26 | 27 | function Transport(local) { 28 | this.local = local; 29 | } 30 | 31 | var T = Transport.prototype; 32 | 33 | T.connect = function connect(local, localMetadata, remote, metadata) { 34 | return new Connection2(local, localMetadata, remote, metadata, hub); 35 | }; 36 | 37 | T.listen = function listen(local, options, listener) { 38 | hub.listens[local] = listener; 39 | }; 40 | -------------------------------------------------------------------------------- /tests/candidate.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Lab = require('lab'); 4 | var lab = exports.lab = Lab.script(); 5 | var describe = lab.describe; 6 | var it = lab.it; 7 | var assert = Lab.assert; 8 | 9 | var uuid = require('cuid'); 10 | var NodeC = require('./_node'); 11 | var transport = require('./_transport'); 12 | 13 | describe('candidate', function() { 14 | 15 | it('reaches leader state if alone', function(done) { 16 | var node = NodeC(); 17 | 18 | var states = ['follower', 'candidate', 'leader']; 19 | 20 | node.on('state', function(state) { 21 | if (states.length) { 22 | assert.equal(state, states.shift()); 23 | } 24 | if (!states.length) { 25 | assert.equal(node.currentTerm(), 1); 26 | done(); 27 | } 28 | }); 29 | }); 30 | 31 | it('doesnt reach leader if cannot get majority', function(done) { 32 | var node = NodeC(); 33 | var remotes = [uuid(), uuid()]; 34 | 35 | remotes.forEach(function(id) { 36 | node._join(id); 37 | }); 38 | 39 | var states = ['follower', 'candidate', 'candidate']; 40 | 41 | node.on('state', function(state) { 42 | if (states.length) { 43 | assert.equal(state, states.shift()); 44 | 45 | if (!states.length) { 46 | assert.equal(node.currentTerm(), 2); 47 | done(); 48 | } 49 | } 50 | }); 51 | }); 52 | 53 | it('reaches leader if can get all the votes', function(done) { 54 | var node = NodeC(); 55 | var remotes = [uuid(), uuid()]; 56 | remotes.forEach(function(id) { 57 | transport.listen(node.id, id, listen); 58 | node._join(id); 59 | }); 60 | 61 | var states = ['follower', 'candidate', 'leader']; 62 | 63 | node.on('state', function(state) { 64 | if (states.length) { 65 | assert.equal(state, states.shift()); 66 | 67 | if (!states.length) { 68 | assert.equal(node.currentTerm(), 1); 69 | done(); 70 | } 71 | } 72 | }); 73 | 74 | var voteRequests = 0; 75 | 76 | function listen(type, args, cb) { 77 | if (type == 'RequestVote') { 78 | assert.equal(args.term, 1); 79 | assert.equal(args.lastLogIndex, 0); 80 | assert((++voteRequests) <= remotes.length); 81 | cb(null, {voteGranted: true}); 82 | } 83 | } 84 | }); 85 | 86 | it('reaches leader if can get majority of votes', function(done) { 87 | var node = NodeC(); 88 | var remotes = [uuid(), uuid()]; 89 | 90 | remotes.forEach(function(id, index) { 91 | transport.listen(node.id, remotes[index], listen(index)); 92 | node._join(id); 93 | }); 94 | 95 | var states = ['follower', 'candidate', 'leader']; 96 | 97 | node.on('state', function(state) { 98 | if (states.length) { 99 | assert.equal(state, states.shift()); 100 | 101 | if (!states.length) { 102 | assert.equal(node.currentTerm(), 1); 103 | done(); 104 | } 105 | } 106 | }); 107 | 108 | function listen(index) { 109 | return function(type, args, cb) { 110 | if (type == 'RequestVote') { 111 | var reply = { 112 | voteGranted: (index === 0) 113 | }; 114 | cb(null, reply); 115 | } 116 | }; 117 | } 118 | }); 119 | 120 | it('doesnt reach leader if candidate sends higher term', function(done) { 121 | var node = NodeC(); 122 | var remotes = [uuid(), uuid()]; 123 | 124 | remotes.forEach(function(id, index) { 125 | transport.listen(node.id, remotes[index], listen(id, index)); 126 | node._join(id); 127 | }); 128 | 129 | var states = ['follower', 'candidate', 'follower']; 130 | 131 | node.on('state', function(state) { 132 | if (states.length) { 133 | assert.equal(state, states.shift()); 134 | 135 | if (!states.length) { 136 | assert.equal(node.currentTerm(), 2); 137 | done(); 138 | } 139 | } 140 | }); 141 | 142 | function listen(id, index) { 143 | return function(type, args, cb) { 144 | if (index >= 1) { 145 | setTimeout(function() { 146 | cb(null, {voteGranted: true}); 147 | }, 100); 148 | } else { 149 | transport.invoke(id, 'AppendEntries', {term: node.currentTerm() + 1}, 150 | function() {}); 151 | } 152 | }; 153 | } 154 | }); 155 | 156 | it('converts to follower if other sends same term', function(done) { 157 | var node = NodeC(); 158 | var remotes = [uuid(), uuid()]; 159 | 160 | remotes.forEach(function(id, index) { 161 | transport.listen(node.id, remotes[index], listen(id, index)); 162 | node._join(id); 163 | }); 164 | 165 | var states = ['follower', 'candidate', 'follower']; 166 | 167 | node.on('state', function(state) { 168 | if (states.length) { 169 | assert.equal(state, states.shift()); 170 | 171 | if (!states.length) { 172 | assert.equal(node.currentTerm(), 1); 173 | done(); 174 | } 175 | } 176 | }); 177 | 178 | function listen(id, index) { 179 | return function(type, args, cb) { 180 | if (index >= 1) { 181 | setTimeout(function() { 182 | cb(null, {voteGranted: true}); 183 | }, 100); 184 | } else { 185 | transport.invoke(id, 'AppendEntries', {term: node.currentTerm()}, 186 | function() {}); 187 | } 188 | }; 189 | } 190 | }); 191 | }); 192 | -------------------------------------------------------------------------------- /tests/cluster.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Lab = require('lab'); 4 | var lab = exports.lab = Lab.script(); 5 | var describe = lab.describe; 6 | var it = lab.it; 7 | var assert = Lab.assert; 8 | var NodeC = require('./_node'); 9 | 10 | describe('cluster', function() { 11 | 12 | it('cluster has id if you provide it with one', function(done) { 13 | var node = NodeC({cluster: 'SOMECLUSTERID'}); 14 | var cluster = node.cluster; 15 | assert.equal(cluster.id, 'SOMECLUSTERID'); 16 | done(); 17 | }); 18 | 19 | it('cluster has id if you don\'t provide it with one', function(done) { 20 | var node = NodeC(); 21 | var cluster = node.cluster; 22 | assert.typeOf(cluster.id, 'string'); 23 | done(); 24 | }); 25 | }); 26 | -------------------------------------------------------------------------------- /tests/follower.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Lab = require('lab'); 4 | var lab = exports.lab = Lab.script(); 5 | var describe = lab.describe; 6 | var it = lab.it; 7 | var assert = Lab.assert; 8 | 9 | var uuid = require('cuid'); 10 | var NodeC = require('./_node'); 11 | var transport = require('./_transport'); 12 | 13 | describe('follower', function() { 14 | 15 | it('is the default state', function(done) { 16 | var node = NodeC(); 17 | node.once('state', function(state) { 18 | assert.equal(state, 'follower'); 19 | done(); 20 | }); 21 | }); 22 | 23 | it('replies false to request vote for lower terms', function(done) { 24 | var node = NodeC(); 25 | var peer = uuid(); 26 | node._join(peer); 27 | node.currentTerm(3); 28 | 29 | transport.invoke(peer, 'RequestVote', { 30 | term: 2, 31 | candidateId: peer, 32 | lastLogIndex: 10, 33 | lastLogTerm: 10 34 | }, onReply); 35 | 36 | function onReply(err, args) { 37 | if (err) { 38 | throw err; 39 | } 40 | assert.typeOf(args.voteGranted, 'boolean'); 41 | assert.notOk(args.voteGranted); 42 | done(); 43 | } 44 | }); 45 | 46 | it('replies false to second vote request on the same term', function(done) { 47 | var node = NodeC(); 48 | var peer1 = uuid(); 49 | var peer2 = uuid(); 50 | node._join(peer1); 51 | node._join(peer2); 52 | 53 | transport.invoke(peer1, 'RequestVote', { 54 | term: 2, 55 | candidateId: peer1, 56 | lastLogIndex: 0, 57 | lastLogTerm: 1 58 | }, onReply1); 59 | 60 | transport.invoke(peer2, 'RequestVote', { 61 | term: 2, 62 | candidateId: peer2, 63 | lastLogIndex: 0, 64 | lastLogTerm: 1 65 | }, onReply2); 66 | 67 | function onReply1(err, args) { 68 | if (err) { 69 | throw err; 70 | } 71 | assert(args.voteGranted); 72 | } 73 | 74 | function onReply2(err, args) { 75 | assert.typeOf(args.voteGranted, 'boolean'); 76 | assert.equal(args.voteGranted, false); 77 | done(); 78 | } 79 | 80 | }); 81 | 82 | it('does not grant vote if candidate lagging behind', function(done) { 83 | var node = NodeC(); 84 | var peer = uuid(); 85 | node._join(peer); 86 | node.currentTerm(3); 87 | node.commonState.persisted.log.entries.push({ 88 | command: 'a', 89 | term: 3 90 | }); 91 | node.commonState.persisted.log.entries.push({ 92 | command: 'b', 93 | term: 3 94 | }); 95 | 96 | transport.invoke(peer, 'RequestVote', { 97 | term: 4, 98 | lastLogTerm: 3, 99 | lastLogIndex: 1 100 | }, onReply); 101 | 102 | function onReply(err, args) { 103 | if (err) { 104 | throw err; 105 | } 106 | assert.typeOf(args.voteGranted, 'boolean'); 107 | assert.notOk(args.voteGranted); 108 | done(); 109 | } 110 | }); 111 | 112 | it('does grants vote if last log term < args.lastLogTerm', function(done) { 113 | var node = NodeC(); 114 | var peer = uuid(); 115 | node._join(peer); 116 | node.currentTerm(3); 117 | node.commonState.persisted.log.entries.push({ 118 | command: 'a', 119 | term: 3 120 | }); 121 | node.commonState.persisted.log.entries.push({ 122 | command: 'b', 123 | term: 3 124 | }); 125 | 126 | transport.invoke(peer, 'RequestVote', { 127 | term: 5, 128 | lastLogTerm: 4, 129 | lastLogIndex: 1 130 | }, onReply); 131 | 132 | function onReply(err, args) { 133 | if (err) { 134 | throw err; 135 | } 136 | assert(args.voteGranted); 137 | done(); 138 | } 139 | }); 140 | 141 | it('transforms into candidate when election timeout', function(done) { 142 | var node = NodeC(); 143 | assert.typeOf(node.options.maxElectionTimeout, 'number'); 144 | node.once('election timeout', function() { 145 | assert.equal(node.state.name, 'candidate'); 146 | done(); 147 | }); 148 | }); 149 | 150 | it('replies false to append entries if term < current term', function(done) { 151 | var node = NodeC(); 152 | 153 | node.commonState.persisted.currentTerm = 2; 154 | 155 | var peer = uuid(); 156 | node._join(peer); 157 | transport.invoke(peer, 'AppendEntries', {term: 1, WTF:'WTF'}, replied); 158 | 159 | function replied(err, args) { 160 | if (err) { 161 | throw err; 162 | } 163 | assert.notOk(args.success); 164 | assert.equal(args.term, 2); 165 | done(); 166 | } 167 | }); 168 | 169 | it('replies true to append entries if term = current term', function(done) { 170 | var node = NodeC(); 171 | 172 | node.commonState.persisted.currentTerm = 1; 173 | 174 | var peer = uuid(); 175 | node._join(peer); 176 | 177 | transport.invoke(peer, 'AppendEntries', {term: 1}, replied); 178 | 179 | var isDone = false; 180 | 181 | function replied(err, args) { 182 | if (!isDone) { 183 | isDone = true; 184 | if (err) { 185 | throw err; 186 | } 187 | assert.ok(args.success); 188 | done(); 189 | } 190 | } 191 | }); 192 | 193 | it('applies append entries if term > current term', function(done) { 194 | var node = NodeC(); 195 | 196 | node.commonState.persisted.currentTerm = 1; 197 | 198 | var peer = uuid(); 199 | node._join(peer); 200 | 201 | transport.invoke(peer, 'AppendEntries', {term: 2}, replied); 202 | 203 | var isDone = false; 204 | 205 | function replied(err, args) { 206 | if (!isDone) { 207 | isDone = true; 208 | if (err) { 209 | throw err; 210 | } 211 | assert.ok(args.success); 212 | assert.equal(node.currentTerm(), 2); 213 | done(); 214 | } 215 | } 216 | }); 217 | 218 | it('replies to heartbeat', function(done) { 219 | var node = NodeC(); 220 | 221 | node.commonState.persisted.currentTerm = 1; 222 | 223 | var peer = uuid(); 224 | node._join(peer); 225 | 226 | var args = { 227 | term: 1, 228 | prevLogIndex: null, 229 | prevLogTerm: null, 230 | entries: [] 231 | }; 232 | transport.invoke(peer, 'AppendEntries', args, replied); 233 | 234 | var isDone = false; 235 | 236 | function replied(err, args) { 237 | if (!isDone) { 238 | isDone = true; 239 | assert.ok(args.success); 240 | assert.equal(args.term, node.currentTerm()); 241 | done(); 242 | } 243 | } 244 | }); 245 | 246 | it('applies entries if log is empty', function(done) { 247 | var node = NodeC(); 248 | 249 | var peer = uuid(); 250 | node._join(peer); 251 | 252 | var entries = [ 253 | {term: 2}, 254 | {term: 2} 255 | ]; 256 | 257 | var args = { 258 | term: 2, 259 | prevLogIndex: 0, 260 | prevLogTerm: null, 261 | entries: entries 262 | }; 263 | transport.invoke(peer, 'AppendEntries', args, replied); 264 | 265 | var isDone = false; 266 | 267 | function replied(err, args) { 268 | if (!isDone) { 269 | isDone = true; 270 | assert.ok(args.success); 271 | assert.equal(node.currentTerm(), 2); 272 | entries.forEach(function(entry, index) { 273 | assert.deepEqual( 274 | node.commonState.persisted.log.entries[index], entry); 275 | }); 276 | done(); 277 | } 278 | } 279 | }); 280 | 281 | it('applies entries if there is no conflict', function(done) { 282 | var node = NodeC(); 283 | 284 | node.commonState.persisted.log.push({term: 1}); 285 | 286 | var peer = uuid(); 287 | node._join(peer); 288 | 289 | var entries = [ 290 | {term: 2}, 291 | {term: 2} 292 | ]; 293 | 294 | var args = { 295 | term: 2, 296 | prevLogIndex: 1, 297 | prevLogTerm: 1, 298 | entries: entries 299 | }; 300 | transport.invoke(peer, 'AppendEntries', args, replied); 301 | 302 | var isDone = false; 303 | 304 | function replied(err, args) { 305 | if (!isDone) { 306 | isDone = true; 307 | assert.ok(args.success); 308 | assert.equal(node.currentTerm(), 2); 309 | assert.equal(node.commonState.persisted.log.length(), 3); 310 | assert.equal(node.commonState.persisted.log.entries[0].term, 1); 311 | entries.forEach(function(entry, index) { 312 | assert.deepEqual( 313 | node.commonState.persisted.log.entries[index + 1], entry); 314 | }); 315 | done(); 316 | } 317 | } 318 | }); 319 | 320 | it('applies entries if there is conflict', function(done) { 321 | var node = NodeC(); 322 | 323 | node.commonState.persisted.log.push({term: 1}, {term: 2}); 324 | 325 | var peer = uuid(); 326 | node._join(peer); 327 | 328 | var entries = [ 329 | {term: 2}, 330 | {term: 2} 331 | ]; 332 | 333 | var args = { 334 | term: 2, 335 | prevLogIndex: 1, 336 | prevLogTerm: 1, 337 | leaderCommit: 0, 338 | entries: entries 339 | }; 340 | transport.invoke(peer, 'AppendEntries', args, replied); 341 | 342 | var isDone = false; 343 | 344 | function replied(err, args) { 345 | if (!isDone) { 346 | isDone = true; 347 | assert.ok(args.success); 348 | assert.equal(node.currentTerm(), 2); 349 | assert.equal(node.commonState.persisted.log.length(), 3); 350 | assert.deepEqual(node.commonState.persisted.log.entryAt(1).term, 1); 351 | entries.forEach(function(entry, index) { 352 | assert.deepEqual( 353 | node.commonState.persisted.log.entries[index + 1], entry); 354 | }); 355 | done(); 356 | } 357 | } 358 | }); 359 | 360 | it('eventually persists new log entries', function(done) { 361 | var node = NodeC(); 362 | 363 | node.commonState.persisted.log.push({term: 1}, {term: 2}); 364 | 365 | assert.equal(node.commonState.volatile.commitIndex, 0); 366 | assert.equal(node.commonState.volatile.lastApplied, 0); 367 | 368 | var peer = uuid(); 369 | node._join(peer); 370 | 371 | var entries = [ 372 | {term: 2, command: 'COMMAND 1'}, 373 | {term: 2, command: 'COMMAND 2'} 374 | ]; 375 | 376 | var args = { 377 | term: 2, 378 | prevLogIndex: 1, 379 | prevLogTerm: 1, 380 | leaderCommit: 2, 381 | entries: entries 382 | }; 383 | transport.invoke(peer, 'AppendEntries', args, replied); 384 | 385 | var isDone = false; 386 | var applied = 0; 387 | 388 | node.on('applied log', function(logIndex) { 389 | assert.equal(logIndex, ++applied); 390 | if (applied == entries.length) { 391 | assert.equal(node.commonState.volatile.lastApplied, 2); 392 | isDone = true; 393 | done(); 394 | } 395 | }); 396 | 397 | function replied(err) { 398 | if (err) { 399 | throw err; 400 | } 401 | assert.equal(node.commonState.volatile.commitIndex, 2); 402 | } 403 | }); 404 | }); 405 | -------------------------------------------------------------------------------- /tests/integration.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Lab = require('lab'); 4 | var lab = exports.lab = Lab.script(); 5 | var it = lab.it; 6 | var assert = Lab.assert; 7 | var describe = lab.describe; 8 | 9 | var uuid = require('cuid'); 10 | var async = require('async'); 11 | var NodeC = require('./_node2'); 12 | var NodeCC = require('../'); 13 | var Cluster = require('./_cluster2'); 14 | var persistence = require('./_persistence'); 15 | var Transport = require('./_transport2'); 16 | 17 | var domain = require('domain'); 18 | 19 | describe('cluster', function() { 20 | 21 | it('elects one leader', function(done) { 22 | Cluster(5, onLeader); 23 | 24 | function onLeader(leader, nodes) { 25 | setTimeout(function() { 26 | assert.equal(nodes.length, 4); 27 | assert.equal(leader.state.name, 'leader'); 28 | assert.equal(leader.commonState.volatile.leaderId, leader.id); 29 | nodes.forEach(function(node) { 30 | assert.equal(node.state.name, 'follower'); 31 | assert(node.currentTerm() >= 1); 32 | assert.equal(node.commonState.volatile.leaderId, leader.id); 33 | }); 34 | done(); 35 | }, 1e3); 36 | } 37 | }); 38 | 39 | it('commands work and get persisted', {timeout: 10e3}, function(done) { 40 | var MAX_COMMANDS = 50; 41 | 42 | Cluster(3, onLeader); 43 | 44 | var commands = []; 45 | var index = 0; 46 | 47 | function onLeader(leader, nodes) { 48 | pushCommand(); 49 | 50 | function pushCommand() { 51 | var cmd = ++index; 52 | commands.push(cmd); 53 | leader.command(cmd, commanded); 54 | } 55 | 56 | function commanded(err) { 57 | if (err) { 58 | throw err; 59 | } 60 | if (index < MAX_COMMANDS) { 61 | pushCommand(); 62 | } 63 | else { 64 | setTimeout(function() { 65 | nodes.forEach(function(node) { 66 | assert.deepEqual(persistence.store.commands[node.id], commands); 67 | }); 68 | done(); 69 | }, 1e3); 70 | } 71 | } 72 | } 73 | }); 74 | 75 | it('allows adding a node in-flight (topology change)', {timeout: 5e3}, 76 | function(done) { 77 | Cluster(5, onLeader); 78 | 79 | function onLeader(leader) { 80 | // debug(leader); 81 | var node = NodeC({standby: true}); 82 | node.listen(node.id); 83 | 84 | leader.join(node.id, joined); 85 | 86 | function joined(err) { 87 | if (err) { 88 | throw err; 89 | } 90 | 91 | var commands = []; 92 | 93 | for (var i = 0 ; i < 10 ; i ++) { 94 | commands.push(i); 95 | } 96 | 97 | async.each(commands, function(cmd, cb) { 98 | leader.command(cmd, cb); 99 | }, commanded); 100 | 101 | function commanded(err) { 102 | if (err) { 103 | throw err; 104 | } 105 | 106 | setTimeout(function() { 107 | assert.deepEqual(persistence.store.commands[node.id], commands); 108 | done(); 109 | }, 2e3); 110 | } 111 | } 112 | } 113 | } 114 | ); 115 | 116 | it('allows removing a node in-flight that is not the leader', function(done) { 117 | Cluster(5, onLeader); 118 | 119 | function onLeader(leader, nodes) { 120 | var node = nodes[0]; 121 | leader.leave(node.id, left); 122 | 123 | function left(err) { 124 | if (err) { 125 | throw err; 126 | } 127 | done(); 128 | } 129 | } 130 | }); 131 | 132 | it('allows removing a node in-flight that is the leader', {timeout: 6e3}, 133 | function(done) { 134 | var oneNewLeader = false; 135 | var d = domain.create(); 136 | 137 | d.on('error', function(err) { 138 | console.error(err.stack); 139 | throw err; 140 | }); 141 | 142 | d.run(function() { 143 | Cluster(5, onLeader); 144 | }); 145 | 146 | function onLeader(leader, nodes) { 147 | nodes.forEach(function(node) { 148 | node.once('leader', onNewLeader); 149 | }); 150 | 151 | setTimeout(function() { 152 | leader.leave(leader.id); 153 | }, 1e3); 154 | } 155 | 156 | function onNewLeader() { 157 | if (!oneNewLeader) { 158 | oneNewLeader = true; 159 | done(); 160 | } 161 | } 162 | } 163 | ); 164 | 165 | it('allows 2 nodes to start talking to each other', function(done) { 166 | var leader = NodeC(); 167 | var follower = NodeC({standby: true}); 168 | follower.listen(follower.id); 169 | 170 | leader.once('leader', function() { 171 | leader.join(follower.id, joined); 172 | }); 173 | 174 | function joined(err) { 175 | if (err) { 176 | throw err; 177 | } 178 | 179 | leader.command('COMMAND', commanded); 180 | } 181 | 182 | function commanded(err) { 183 | if (err) { 184 | throw err; 185 | } 186 | 187 | setTimeout(function() { 188 | assert.deepEqual(persistence.store.commands[follower.id], ['COMMAND']); 189 | done(); 190 | }, 1e3); 191 | } 192 | }); 193 | 194 | it('removing all nodes but 1 makes sole node leader', {timeout: 5e3}, 195 | function(done) { 196 | 197 | var d = domain.create(); 198 | d.on('error', function(err) { 199 | console.error(err.stack); 200 | throw err; 201 | }); 202 | d.run(function() { 203 | Cluster(3, onLeader); 204 | }); 205 | 206 | function onLeader(leader, nodes) { 207 | var gotNewLeader = false; 208 | var follower; 209 | 210 | setTimeout(function() { 211 | leader.leave(leader.id); 212 | 213 | nodes.forEach(function(node) { 214 | node.once('leader', onNewLeader); 215 | }); 216 | }, 5e2); 217 | 218 | function onNewLeader(newLeader) { 219 | if (!gotNewLeader) { 220 | gotNewLeader = true; 221 | setTimeout(function() { 222 | follower = nodes[(nodes.indexOf(newLeader) + 1) % 2]; 223 | follower.once('leader', onNewNewLeader); 224 | newLeader.leave(newLeader.id); 225 | }, 5e2); 226 | } 227 | } 228 | 229 | function onNewNewLeader() { 230 | done(); 231 | } 232 | } 233 | } 234 | ); 235 | 236 | it('fails to emit a command if the majority reachable', {timeout: 6e3}, 237 | function(done) { 238 | var id = uuid(); 239 | var options = { 240 | standby: false, 241 | id: id, 242 | transport: new Transport(id), 243 | persistence: persistence 244 | }; 245 | var leader = new NodeCC(options); 246 | var nodes = []; 247 | 248 | for (var i = 0 ; i < 2 ; i ++) { 249 | nodes.push(uuid()); 250 | } 251 | 252 | leader.once('leader', onceLeader); 253 | 254 | function onceLeader(leader) { 255 | nodes.forEach(function(node) { 256 | leader.join(node, dontCare); 257 | }); 258 | 259 | leader.command('COMMAND', {timeout: 2e3}, onCommand); 260 | } 261 | 262 | function onCommand(err) { 263 | assert(err instanceof Error); 264 | assert(/timedout/.test(err.message)); 265 | done(); 266 | } 267 | 268 | function dontCare() {} 269 | } 270 | ); 271 | 272 | it('streams snapshot to new peer', {timeout: 10e3}, function(done) { 273 | var MAX_COMMANDS = 100; 274 | var node; 275 | var follower; 276 | 277 | node = NodeC({ 278 | retainedLogEntries: 10 279 | }); 280 | node.listen(node.id); 281 | 282 | // debug(node); 283 | 284 | node.once('leader', function() { 285 | for (var i = 1 ; i <= MAX_COMMANDS ; i ++) { 286 | node.command('command ' + i); 287 | } 288 | 289 | node.on('applied log', function(index) { 290 | if (index == MAX_COMMANDS) { 291 | follower = NodeC({standby: true}); 292 | 293 | follower.listen(follower.id); 294 | node.join(follower.id); 295 | 296 | follower.once('done persisting', followerAppliedLog); 297 | } 298 | }); 299 | }); 300 | 301 | function followerAppliedLog(logIndex) { 302 | assert.equal(logIndex, MAX_COMMANDS + 1); 303 | done(); 304 | } 305 | }); 306 | 307 | it('can be told to wait for a node', function(done) { 308 | Cluster(5, onLeader); 309 | 310 | function onLeader(leader, nodes) { 311 | var node = nodes[nodes.length - 1]; 312 | var options = {waitForNode: node.id}; 313 | leader.command('VERY IMPORTANT COMMAND', options, commanded); 314 | 315 | function commanded(err) { 316 | if (err) { 317 | throw err; 318 | } 319 | 320 | // check if node has command 321 | var commands = persistence.store.commands[node.id]; 322 | var lastCommand = commands[commands.length - 1]; 323 | assert.equal(lastCommand, 'VERY IMPORTANT COMMAND'); 324 | done(); 325 | } 326 | } 327 | 328 | }); 329 | 330 | }); 331 | -------------------------------------------------------------------------------- /tests/leader.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Lab = require('lab'); 4 | var lab = exports.lab = Lab.script(); 5 | var describe = lab.describe; 6 | var it = lab.it; 7 | var assert = Lab.assert; 8 | 9 | var uuid = require('cuid'); 10 | var NodeC = require('./_node'); 11 | var transport = require('./_transport'); 12 | 13 | describe('leader', function() { 14 | 15 | it('sends heartbeat immediately after becoming leader', function(done) { 16 | var node = NodeC(); 17 | 18 | var peers = [uuid(), uuid()]; 19 | peers.forEach(function(peer) { 20 | node._join(peer); 21 | transport.listen(node.id, peer, peerListen); 22 | }); 23 | 24 | var heartbeats = 0; 25 | 26 | function peerListen(type, args, cb) { 27 | switch (type) { 28 | case 'RequestVote': 29 | handleRequestVote(args, cb); 30 | break; 31 | case 'AppendEntries': 32 | handleAppendEntries(args, cb); 33 | break; 34 | } 35 | } 36 | 37 | function handleRequestVote(args, cb) { 38 | cb(null, {voteGranted: true}); 39 | } 40 | 41 | function handleAppendEntries(args, cb) { 42 | assert.equal(args.term, 1); 43 | assert.equal(args.leaderId, node.id); 44 | assert.deepEqual(args.entries, []); 45 | 46 | cb(null, {success: true}); 47 | 48 | heartbeats ++; 49 | if (heartbeats == peers.length * 10) { 50 | done(); 51 | } 52 | } 53 | }); 54 | 55 | it('handles peer append entries failures by backing off', function(done) { 56 | var node = NodeC(); 57 | 58 | node.commonState.persisted.log.push({term: 1, command: 'COMMAND 1'}); 59 | node.commonState.persisted.log.push({term: 1, command: 'COMMAND 2'}); 60 | 61 | var peers = [uuid(), uuid()]; 62 | peers.forEach(function(peer, index) { 63 | transport.listen(node.id, peer, peerListen(peer, index)); 64 | node._join(peer); 65 | }); 66 | 67 | var expectedIndexes = [2, 1, 0, 1, 2, 3, 3]; 68 | var expectedEntries = [ 69 | [{term: 1, command: 'COMMAND 3', index: 3}], 70 | [{term: 1, command: 'COMMAND 2', index: 2}], 71 | [{term: 1, command: 'COMMAND 1', index: 1}], 72 | [{term: 1, command: 'COMMAND 2', index: 2}], 73 | [{term: 1, command: 'COMMAND 3', index: 3}], 74 | [], 75 | [] 76 | ]; 77 | var nodeAppendEntriesCount = {}; 78 | peers.forEach(function(peer) { 79 | nodeAppendEntriesCount[peer] = 0; 80 | }); 81 | 82 | function peerListen(id) { 83 | var lastIndex = 0; 84 | return function(type, args, cb) { 85 | switch (type) { 86 | case 'RequestVote': 87 | handleRequestVote(args, cb); 88 | break; 89 | case 'AppendEntries': 90 | handleAppendEntries(args, cb); 91 | break; 92 | } 93 | }; 94 | 95 | function handleAppendEntries(args, cb) { 96 | if (nodeAppendEntriesCount[id] < expectedIndexes.length) { 97 | nodeAppendEntriesCount[id] ++; 98 | assert.equal( 99 | args.prevLogIndex, expectedIndexes[nodeAppendEntriesCount[id] - 1]); 100 | assert.deepEqual( 101 | args.entries.map( 102 | filterLogEntry), expectedEntries[nodeAppendEntriesCount[id] - 1]); 103 | } 104 | 105 | if (args.prevLogIndex <= lastIndex) { 106 | lastIndex = args.prevLogIndex + args.entries.length; 107 | cb(null, {success: true}); 108 | } 109 | else { 110 | cb(null, {success: false}); 111 | } 112 | } 113 | } 114 | 115 | function handleRequestVote(args, cb) { 116 | cb(null, {voteGranted: true}); 117 | } 118 | 119 | node.once('leader', function() { 120 | node.command('COMMAND 3', function(err) { 121 | if (err) { 122 | throw err; 123 | } 124 | setTimeout(function() { 125 | peers.forEach(function(peer) { 126 | assert.equal(nodeAppendEntriesCount[peer], expectedIndexes.length); 127 | }); 128 | node.stop(done); 129 | }, 1e3); 130 | 131 | }); 132 | }); 133 | 134 | function filterLogEntry(entry) { 135 | return { 136 | term: entry.term, 137 | index: entry.index, 138 | command: entry.command 139 | }; 140 | } 141 | }); 142 | }); 143 | -------------------------------------------------------------------------------- /tests/log.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Lab = require('lab'); 4 | var lab = exports.lab = Lab.script(); 5 | var it = lab.it; 6 | var assert = Lab.assert; 7 | var describe = lab.describe; 8 | 9 | var Log = require('../lib/log'); 10 | 11 | describe('log', function() { 12 | 13 | it('can be created', function(done) { 14 | var node = 'NODE'; 15 | var options = 'OPTIONS'; 16 | var log = new Log(node, options); 17 | 18 | assert.equal(log.node, node); 19 | assert.equal(log.options, options); 20 | done(); 21 | }); 22 | 23 | it('can be created from a doc', function(done) { 24 | var doc = { 25 | meta: { 26 | lastIncludedIndex: 1001, 27 | lastIncludedTerm: 101 28 | }, 29 | entries: ['some', 'nice', 'entries'] 30 | }; 31 | var log = new Log(undefined, undefined, doc); 32 | 33 | assert.equal(log.lastIncludedIndex, 1001); 34 | assert.equal(log.lastIncludedTerm, 101); 35 | assert.deepEqual(log.entries, ['some', 'nice', 'entries']); 36 | done(); 37 | }); 38 | 39 | it('can push and retrieve entries', function(done) { 40 | var log = new Log(); 41 | log.push('entry 1'); 42 | log.push('entry 2'); 43 | 44 | assert.equal(log.entryAt(1), 'entry 1'); 45 | assert.equal(log.entryAt(2), 'entry 2'); 46 | done(); 47 | }); 48 | 49 | it('can push entries in bulk', function(done) { 50 | var log = new Log(); 51 | log.pushEntries(0, ['entry 1', 'entry 2']); 52 | log.pushEntries(2, ['entry 3', 'entry 4']); 53 | assert.equal(log.length(), 4); 54 | assert.equal(log.entryAt(1), 'entry 1'); 55 | assert.equal(log.entryAt(2), 'entry 2'); 56 | assert.equal(log.entryAt(3), 'entry 3'); 57 | assert.equal(log.entryAt(4), 'entry 4'); 58 | done(); 59 | }); 60 | 61 | it('can push entries overriding existing ones', function(done) { 62 | var log = new Log(); 63 | log.pushEntries(0, ['entry 1', 'entry 2', 'entry 3']); 64 | log.pushEntries(2, ['entry 3', 'entry 4', 'entry 5']); 65 | assert.equal(log.length(), 5); 66 | assert.equal(log.entryAt(1), 'entry 1'); 67 | assert.equal(log.entryAt(2), 'entry 2'); 68 | assert.equal(log.entryAt(3), 'entry 3'); 69 | assert.equal(log.entryAt(4), 'entry 4'); 70 | assert.equal(log.entryAt(5), 'entry 5'); 71 | done(); 72 | }); 73 | 74 | it('doesnt compact the logs before retained count', function(done) { 75 | var i; 76 | var options = { 77 | retainedLogEntries: 10 78 | }; 79 | var log = new Log('node', options); 80 | 81 | for (i = 1 ; i <= 10; i ++) { 82 | log.push('entry ' + i); 83 | } 84 | 85 | for (i = 1 ; i <= 10; i ++) { 86 | log.applied(10); 87 | } 88 | 89 | assert.equal(log.length(), 10); 90 | assert.equal(log.lastIncludedIndex, 0); 91 | assert.equal(log.lastIncludedTerm, 0); 92 | assert.equal(log.entries.length, 10); 93 | done(); 94 | }); 95 | 96 | it('compacts the logs after retained count is reached', function(done) { 97 | var options = { 98 | retainedLogEntries: 10 99 | }; 100 | var log = new Log('node', options); 101 | 102 | for (var i = 1 ; i <= 30; i ++) { 103 | log.push({ 104 | term: i, 105 | command: 'entry ' + i 106 | }); 107 | } 108 | 109 | log.applied(20); 110 | 111 | assert.equal(log.length(), 30); 112 | assert.equal(log.lastIncludedIndex, 10); 113 | assert.equal(log.lastIncludedTerm, 10); 114 | assert.equal(log.entries.length, 20); 115 | done(); 116 | }); 117 | 118 | }); 119 | -------------------------------------------------------------------------------- /tests/node.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Lab = require('lab'); 4 | var lab = exports.lab = Lab.script(); 5 | var it = lab.it; 6 | var assert = Lab.assert; 7 | var describe = lab.describe; 8 | 9 | var uuid = require('cuid'); 10 | var Node = require('../'); 11 | var NodeC = require('./_node'); 12 | var transport = require('./_transport'); 13 | var persistence = require('./_persistence'); 14 | 15 | describe('node', function() { 16 | 17 | it('creates an id for you', function(done) { 18 | var node = Node({ 19 | transport: transport, 20 | persistence: persistence 21 | }); 22 | assert.typeOf(node.id, 'string'); 23 | done(); 24 | }); 25 | 26 | it('errors if maxElectionTimeout < minElectionTimeout', function(done) { 27 | var node = NodeC({ 28 | minElectionTimeout: 2, 29 | maxElectionTimeout: 1 30 | }); 31 | 32 | var isDone = false; 33 | node.on('error', function(err) { 34 | if (!isDone) { 35 | isDone = true; 36 | assert.equal(err.message, 37 | 'maxElectionTimeout is greater than minElectionTimeout'); 38 | done(); 39 | } 40 | }); 41 | }); 42 | 43 | it('cannot travel to unknown state', function(done) { 44 | var node = NodeC(); 45 | assert.throws(function() { 46 | node.toState('someonemistypedthestate'); 47 | }, 'Unknown state: someonemistypedthestate'); 48 | done(); 49 | }); 50 | 51 | it('starts with an empty list of peers', function(done) { 52 | var node = NodeC(); 53 | assert.isArray(node.commonState.persisted.peers); 54 | assert.equal(node.commonState.persisted.peers.length, 0); 55 | done(); 56 | }); 57 | 58 | it('cannot join a peer without transport', function(done) { 59 | assert.throws(function() { 60 | NodeC({transport: null}); 61 | }, 'need options.transport'); 62 | done(); 63 | }); 64 | 65 | it('emits error if the loading metadata fails', function(done) { 66 | var loadMeta = persistence.loadMeta; 67 | 68 | persistence.loadMeta = function(nodeId, cb) { 69 | persistence.loadMeta = loadMeta; 70 | process.nextTick(function() { 71 | cb(new Error('oops')); 72 | }); 73 | }; 74 | var node = NodeC(); 75 | node.once('error', function(err) { 76 | assert(err instanceof Error); 77 | done(); 78 | }); 79 | }); 80 | 81 | it('saves peer data as ids', function(done) { 82 | var node = NodeC(); 83 | var peer = uuid(); 84 | node._join(peer, 'peer metadata'); 85 | 86 | node.save(saved); 87 | 88 | function saved(err) { 89 | if (err) { 90 | throw err; 91 | } 92 | var stored = persistence.store.meta[node.id]; 93 | if (stored) { 94 | stored = JSON.parse(stored); 95 | } 96 | assert.deepEqual( 97 | stored && stored.peers, [{id: peer, metadata: 'peer metadata'}]); 98 | done(); 99 | } 100 | }); 101 | 102 | it('loads peer data from persistence', function(done) { 103 | var id = uuid(); 104 | var peer = uuid(); 105 | persistence.store.meta[id] = JSON.stringify( 106 | {peers: [{id: peer, metadata: 'some peer metadata'}]} 107 | ); 108 | 109 | var node = NodeC({id: id}); 110 | node.once('loaded', function() { 111 | var ppeer = node.commonState.persisted.peers[0]; 112 | assert.equal(ppeer.id, peer); 113 | assert.equal(ppeer.metadata, 'some peer metadata'); 114 | done(); 115 | }); 116 | }); 117 | 118 | it('paopagates error in log applier to self', function(done) { 119 | var node = NodeC(); 120 | 121 | node.once('error', function(err) { 122 | assert(err instanceof Error); 123 | assert.equal(err.message, 'ayay'); 124 | done(); 125 | }); 126 | 127 | node.logApplier.emit('error', new Error('ayay')); 128 | }); 129 | 130 | it('propagates state error to self', function(done) { 131 | var node = NodeC(); 132 | 133 | node.once('error', function(err) { 134 | assert(err instanceof Error); 135 | assert.equal(err.message, 'ayay'); 136 | done(); 137 | }); 138 | 139 | node.state.emit('error', new Error('ayay')); 140 | }); 141 | 142 | it('can\'t join self', function(done) { 143 | var node = NodeC(); 144 | node.join(node.id, 'some metadata', function(err) { 145 | assert.equal(err && err.message, 'can\'t join self'); 146 | done(); 147 | }); 148 | }); 149 | 150 | it('ignores leaving non-existing peer', function(done) { 151 | var node = NodeC(); 152 | node._leave('abc'); 153 | setTimeout(done, 1e3); 154 | }); 155 | 156 | it('doesn\'t let you issue a command unless is a leader', function(done) { 157 | var node = NodeC(); 158 | node.command('abc', function(err) { 159 | assert.equal(err && err.message, 'not the leader'); 160 | assert.equal(err.code, 'ENOTLEADER'); 161 | done(); 162 | }); 163 | }); 164 | 165 | it('doesn\'t handle peer call after stopped', function(done) { 166 | var node = NodeC(); 167 | node.stop(function(err) { 168 | if (err) { 169 | throw err; 170 | } 171 | node.handlePeerCall('peer', 'RequestVote', {what:'ever'}, replied); 172 | }); 173 | 174 | function replied() { 175 | throw new Error('should not have replied'); 176 | } 177 | 178 | setTimeout(done, 1e3); 179 | }); 180 | 181 | it('if alone converts to leader and stays that way', function(done) { 182 | var node = NodeC(); 183 | 184 | node.once('leader', function() { 185 | setTimeout(function() { 186 | assert.equal(node.state.name, 'leader'); 187 | done(); 188 | }, 1e3); 189 | }); 190 | }); 191 | 192 | }); 193 | -------------------------------------------------------------------------------- /tests/peer.js: -------------------------------------------------------------------------------- 1 | 'use strict'; 2 | 3 | var Lab = require('lab'); 4 | var lab = exports.lab = Lab.script(); 5 | var describe = lab.describe; 6 | var it = lab.it; 7 | var assert = Lab.assert; 8 | 9 | var Peer = require('../lib/peer'); 10 | var uuid = require('cuid'); 11 | var sinon = require('sinon'); 12 | var transport = require('./_transport'); 13 | var Connection = require('./_connection'); 14 | 15 | describe('peer', function() { 16 | 17 | it('can create a peer and retains options', function(done) { 18 | var node = {id: uuid()}; 19 | var id = uuid(); 20 | var peer = Peer(id, 'metadata', {transport: transport}, undefined, node); 21 | assert.equal(peer.id, id); 22 | assert.equal(peer.metadata, 'metadata'); 23 | done(); 24 | }); 25 | 26 | it('connects', function(done) { 27 | var node = {id: uuid(), metadata: 'local metadata'}; 28 | var id = uuid(); 29 | var peer = Peer(id, 'metadata', {transport: transport}, undefined, node); 30 | var spy = sinon.spy(transport, 'connect'); 31 | peer.connect(); 32 | assert.ok( 33 | spy.withArgs(node.id, 'local metadata', id, 'metadata').calledOnce); 34 | done(); 35 | }); 36 | 37 | it('cannot send messages before connected', function(done) { 38 | var node = {id: uuid()}; 39 | var id = uuid(); 40 | var peer = Peer(id, 'metadata', {transport: transport}, undefined, node); 41 | peer.send('type', 'args', function(err) { 42 | assert.instanceOf(err, Error); 43 | done(); 44 | }); 45 | }); 46 | 47 | it('can make remote calls', function(done) { 48 | var options = uuid(); 49 | var peer = Peer( 50 | options, 'metadata', {transport: transport}, undefined, {id: uuid()}); 51 | var conn = peer.connect(); 52 | var spy = sinon.spy(conn, 'send'); 53 | peer.send('type', 'args', invoked); 54 | 55 | function invoked() { 56 | assert.ok(spy.called); 57 | assert.equal(spy.args[0][0], 'type'); 58 | assert.equal(spy.args[0][1], 'args'); 59 | assert.typeOf(spy.args[0][2], 'function'); 60 | done(); 61 | } 62 | }); 63 | 64 | it('serializes remote calls', function(done) { 65 | var id = uuid(); 66 | var peer = Peer( 67 | id, 'metadata', {transport: transport}, undefined, {id: uuid()}); 68 | transport.listen(uuid(), id, listen); 69 | peer.connect(); 70 | 71 | var active = true; 72 | var timeouts = [100, 0]; 73 | 74 | peer.send('type', 'args', function(err) { 75 | if (err) { 76 | throw err; 77 | } 78 | active = false; 79 | }); 80 | 81 | peer.send('type', 'args', function(err) { 82 | if (err) { 83 | throw err; 84 | } 85 | assert(!active); 86 | done(); 87 | }); 88 | 89 | function listen(type, args, cb) { 90 | assert.equal(type, 'type'); 91 | assert.equal(args, 'args'); 92 | setTimeout(cb, timeouts.shift()); 93 | } 94 | }); 95 | 96 | it('replies to remote calls', function(done) { 97 | var id = uuid(); 98 | var peer = Peer( 99 | id, 'metadata', {transport: transport}, undefined, {id: uuid()}); 100 | transport.listen(uuid(), id, listen); 101 | peer.connect(); 102 | 103 | var replies = [ 104 | [new Error('some error')], 105 | [null, 1], 106 | [null, 1, 2, 3] 107 | ]; 108 | 109 | var c = 0; 110 | 111 | for (var i = 0 ; i < replies.length ; i ++) { 112 | checkReply(i); 113 | } 114 | 115 | function checkReply(i) { 116 | peer.send('type', 'args', function() { 117 | assert.deepEqual(Array.prototype.slice.call(arguments), replies[i]); 118 | if (i == replies.length - 1) { 119 | done(); 120 | } 121 | }); 122 | } 123 | 124 | function listen(type, args, cb) { 125 | cb.apply(null, replies[c ++]); 126 | } 127 | }); 128 | 129 | it('does not call callbacks twice', function(done) { 130 | var id = uuid(); 131 | var peer = Peer( 132 | id, 'metadata', {transport: transport}, undefined, {id: uuid()}); 133 | transport.listen(uuid(), id, listen); 134 | peer.connect(); 135 | 136 | peer.send('type', 'args', done); 137 | 138 | function listen(type, args, cb) { 139 | cb(); 140 | cb(); 141 | } 142 | }); 143 | 144 | it('is able to disconnect', function(done) { 145 | var id = uuid(); 146 | var peer = Peer( 147 | id, 'metadata', {transport: transport}, undefined, {id: uuid()}); 148 | peer.connect(); 149 | peer.once('connection closed', done); 150 | peer.disconnect(); 151 | }); 152 | 153 | it('emits an error if disconnect errors', function(done) { 154 | var id = uuid(); 155 | var peer = Peer( 156 | id, 'metadata', {transport: transport}, undefined, {id: uuid()}); 157 | 158 | var close = Connection.prototype.close; 159 | Connection.prototype.close = function(cb) { 160 | cb(new Error('oops')); 161 | }; 162 | peer.connect(); 163 | peer.once('error', onError); 164 | peer.disconnect(); 165 | 166 | Connection.prototype.close = close; 167 | 168 | function onError(err) { 169 | assert(err instanceof Error); 170 | done(); 171 | } 172 | }); 173 | 174 | }); 175 | --------------------------------------------------------------------------------