Compare commits

...

10 Commits

9 changed files with 1494 additions and 265 deletions

223
anticli.js Normal file
View File

@ -0,0 +1,223 @@
#!/usr/bin/node
const fsp = require('fs').promises;
const http = require('http');
const https = require('https');
class AntiEtcdCli
{
static parse(args)
{
const cmd = [];
const options = {};
for (let i = 2; i < args.length; i++)
{
const arg = args[i].toLowerCase().replace(/^--(.+)$/, (m, m1) => '--'+m1.replace(/-/g, '_'));
if (arg === '-h' || arg === '--help')
{
console.error(
'USAGE:\n'+
' anticli.js [OPTIONS] put <key> [<value>]\n'+
' anticli.js [OPTIONS] get <key> [-p|--prefix] [-v|--print-value-only] [-k|--keys-only]\n'+
' anticli.js [OPTIONS] del <key> [-p|--prefix]\n'+
'OPTIONS:\n'+
' [--endpoints|-e http://node1:2379,http://node2:2379,http://node3:2379]\n'+
' [--timeout 1000]'
);
process.exit();
}
else if (arg == '-e' || arg == '--endpoints')
{
options['endpoints'] = args[++i].split(/\s*[,\s]+\s*/);
}
else if (arg == '-p' || arg == '--prefix')
{
options['prefix'] = true;
}
else if (arg == '-v' || arg == '--print_value_only')
{
options['print_value_only'] = true;
}
else if (arg == '-k' || arg == '--keys_only')
{
options['keys_only'] = true;
}
else if (arg[0] == '-' && arg[1] !== '-')
{
console.error('Unknown option '+arg);
process.exit(1);
}
else if (arg.substr(0, 2) == '--')
{
options[arg.substr(2)] = args[++i];
}
else
{
cmd.push(arg);
}
}
if (!cmd.length || cmd[0] != 'get' && cmd[0] != 'put' && cmd[0] != 'del')
{
console.log('Supported commands: get, put, del. Use --help to see details');
}
return [ cmd, options ];
}
async run(cmd, options)
{
this.options = options;
if (!this.options.endpoints)
{
this.options.endpoints = [ 'http://localhost:2379' ];
}
if (cmd[0] == 'get')
{
await this.get(cmd.slice(1));
}
else if (cmd[0] == 'put')
{
await this.put(cmd[1], cmd.length > 2 ? cmd[2] : undefined);
}
else if (cmd[0] == 'del')
{
await this.del(cmd.slice(1));
}
}
async get(keys)
{
if (this.options.prefix)
{
keys = keys.map(k => k.replace(/\/+$/, ''));
}
const txn = { success: keys.map(key => ({ request_range: this.options.prefix ? { key: b64(key+'/'), range_end: b64(key+'0') } : { key: b64(key) } })) };
const res = await this.request('/v3/kv/txn', txn);
for (const r of (res||{}).responses||[])
{
if (r.response_range)
{
for (const kv of r.response_range.kvs)
{
if (!this.options.print_value_only)
{
console.log(de64(kv.key));
}
if (!this.options.keys_only)
{
console.log(de64(kv.value));
}
}
}
}
}
async put(key, value)
{
if (value === undefined)
{
value = await fsp.readFile(0, { encoding: 'utf-8' });
}
const res = await this.request('/v3/kv/put', { key: b64(key), value: b64(value) });
if (res && res.header)
{
console.log('OK');
}
}
async del(keys)
{
if (this.options.prefix)
{
keys = keys.map(k => k.replace(/\/+$/, ''));
}
const txn = { success: keys.map(k => ({ request_delete_range: this.options.prefix ? { key: b64(key+'/'), range_end: b64(key+'0') } : { key: b64(key) } })) };
const res = await this.request('/v3/kv/txn', txn);
for (const r of (res||{}).responses||[])
{
if (r.response_delete_range)
{
console.log(r.response_delete_range.deleted);
}
}
}
async request(path, body)
{
for (const url of this.options.endpoints)
{
const cur_url = url.replace(/\/+$/, '')+path;
try
{
return (await POST(cur_url, body, this.options.timeout||1000)).json;
}
catch (e)
{
console.error(cur_url+': '+e.message);
}
}
return null;
}
}
function POST(url, body, timeout)
{
return new Promise(ok =>
{
const body_text = Buffer.from(JSON.stringify(body));
let timer_id = timeout > 0 ? setTimeout(() =>
{
if (req)
req.abort();
req = null;
ok({ error: 'timeout' });
}, timeout) : null;
let req = (url.substr(0, 6).toLowerCase() == 'https://' ? https : http).request(url, { method: 'POST', headers: {
'Content-Type': 'application/json',
'Content-Length': body_text.length,
} }, (res) =>
{
if (!req)
{
return;
}
clearTimeout(timer_id);
let res_body = '';
res.setEncoding('utf8');
res.on('error', (error) => ok({ error }));
res.on('data', chunk => { res_body += chunk; });
res.on('end', () =>
{
if (res.statusCode != 200)
{
ok({ error: res_body, code: res.statusCode });
return;
}
try
{
res_body = JSON.parse(res_body);
ok({ response: res, json: res_body });
}
catch (e)
{
ok({ error: e, response: res, body: res_body });
}
});
});
req.on('error', (error) => ok({ error }));
req.on('close', () => ok({ error: new Error('Connection closed prematurely') }));
req.write(body_text);
req.end();
});
}
function b64(str)
{
return Buffer.from(str).toString('base64');
}
function de64(str)
{
return Buffer.from(str, 'base64').toString();
}
new AntiEtcdCli().run(...AntiEtcdCli.parse(process.argv)).catch(console.error);

459
anticluster.js Normal file
View File

@ -0,0 +1,459 @@
const ws = require('ws');
const TinyRaft = require('./tinyraft.js');
const { runCallbacks, RequestError } = require('./common.js');
const LEADER_MISMATCH = 'raft leader/term mismatch';
class AntiCluster
{
constructor(antietcd)
{
this.antietcd = antietcd;
this.cfg = antietcd.cfg;
this.cluster_connections = {};
this.last_request_id = 1;
this.subrequests = {};
this.synced = false;
this.wait_sync = [];
if (!this.cfg.node_id || !this.cfg.cluster_key)
{
throw new Error('node_id and cluster_key are required in configuration if cluster is set');
}
if (!(this.cfg.cluster instanceof Object))
{
this.cfg.cluster = (''+this.cfg.cluster).trim().split(/[\s,]*,[\s,]*/)
.reduce((a, c) => { c = c.split(/\s*=\s*/); a[c[0]] = c[1]; return a; }, {});
}
this.raft = new TinyRaft({
nodes: Object.keys(this.cfg.cluster),
nodeId: this.cfg.node_id,
heartbeatTimeout: this.cfg.heartbeat_timeout,
electionTimeout: this.cfg.election_timeout,
initialTerm: this.antietcd.stored_term,
send: (to, msg) => this._sendRaftMessage(to, msg),
});
this.raft.on('change', (event) => this._handleRaftChange(event));
this.raft.start();
// Connect to all nodes and reconnect forever
for (const node_id in this.cfg.cluster)
{
this.connectToNode(node_id);
}
}
connectToNode(node_id)
{
if (node_id != this.cfg.node_id && this.cfg.cluster[node_id] &&
(!this.cluster_connections[node_id] || !this.antietcd.clients[this.cluster_connections[node_id]]))
{
const socket = new ws.WebSocket(this.cfg.cluster[node_id].replace(/^http/, 'ws'));
const client_id = this.antietcd.startWebsocket(socket, () => setTimeout(() => this.connectToNode(node_id), this.cfg.reconnect_interval||1000));
this.cluster_connections[node_id] = client_id;
socket.on('open', () =>
{
if (this.antietcd.clients[client_id])
{
this.antietcd.clients[client_id].ready = true;
this.antietcd.clients[client_id].raft_node_id = node_id;
this.antietcd.clients[client_id].addr = socket._socket.remoteAddress+':'+socket._socket.remotePort;
socket.send(JSON.stringify({ identify: { key: this.cfg.cluster_key, node_id: this.cfg.node_id } }));
}
});
}
}
_peerRequest(client, request, timeout)
{
const request_id = this.last_request_id++;
request.request_id = request_id;
client.socket.send(JSON.stringify(request));
const req = this.subrequests[request_id] = { client_id: client.id };
const promise = new Promise(ok => req.cb = ok);
req.timer_id = setTimeout(() => this._completeRequest(null, request_id, { error: 'timeout' }), timeout);
return promise;
}
async replicateChange(msg)
{
if (this.raft.state !== TinyRaft.LEADER)
{
return;
}
for (const follower of this.raft.followers)
{
if (follower != this.cfg.node_id)
{
const client = this._getPeer(follower);
if (!client)
{
// One of peers is unavailable - immediate failure, request should be retried
console.log('Lost peer connection during replication - restarting election');
this.raft.start();
throw new Error('Replication failed');
}
}
}
const promises = [];
for (const follower of this.raft.followers)
{
if (follower != this.cfg.node_id)
{
const client = this._getPeer(follower);
const promise = this._peerRequest(client, { replicate: msg, term: this.raft.term }, this.cfg.replication_timeout||1000);
promises.push(promise);
}
}
const results = await Promise.all(promises);
for (const result of results)
{
if (!result)
{
// One of peers is unavailable - immediate failure, request should be retried
console.log('Lost peer connection during replication - restarting election');
this.raft.start();
throw new Error('Replication failed');
}
}
}
_completeRequest(client_id, request_id, result)
{
const req = this.subrequests[request_id];
if (!req || client_id && req.client_id != client_id)
{
return;
}
delete this.subrequests[request_id];
if (req.timer_id && result !== null)
{
clearTimeout(req.timer_id);
}
req.cb(result);
}
_handleRaftChange(event)
{
console.log(
'Raft '+this.cfg.node_id+': '+(event.state == TinyRaft.FOLLOWER ? 'following '+event.leader : event.state)+
', term '+event.term+(event.state == TinyRaft.LEADER ? ', followers: '+event.followers.join(', ') : '')
);
if (event.state == TinyRaft.LEADER)
{
// (Re)sync with the new set of followers
this._resync(event.followers);
this.antietcd.etctree.resume_leases();
}
else
{
this.synced = false;
this.antietcd.etctree.pause_leases();
}
}
_resync(followers)
{
this.synced = false;
if (!this.resync_state)
{
this.resync_state = {
dumps: {},
loads: {},
};
this.resync_state.dumps[this.cfg.node_id] = { ...this.antietcd.etctree.dump(), term: this.antietcd.stored_term };
}
const seen = {};
for (const f of followers)
{
seen[f] = true;
if (f != this.cfg.node_id && !(f in this.resync_state.dumps))
{
const client = this._getPeer(f);
if (client)
{
this.resync_state.dumps[f] = null;
this._peerRequest(client, { request: {}, handler: 'dump' }, this.cfg.dump_timeout||5000).then(res =>
{
if (this.resync_state && client.raft_node_id &&
(client.raft_node_id in this.resync_state.dumps))
{
if (res.error)
{
console.error(client.raft_node_id+' dump failed with error: '+res.error);
}
else
{
console.log('Got dump from '+client.raft_node_id+' with stored term '+res.term);
}
this.resync_state.dumps[client.raft_node_id] = res.error ? null : res;
this._continueResync();
}
});
}
}
}
for (const f in this.resync_state.dumps)
{
if (!seen[f])
{
delete this.resync_state.dumps[f];
}
}
this._continueResync();
}
_continueResync()
{
let max_term = -1, with_max = [];
for (const follower in this.resync_state.dumps)
{
const dump = this.resync_state.dumps[follower];
if (!dump)
{
// Some dump(s) are still pending
return;
}
if (dump.term > max_term)
{
max_term = dump.term;
with_max = [ follower ];
}
else if (dump.term == max_term)
{
with_max.push(follower);
}
}
if (max_term < 0 || with_max.length == 0)
{
throw new Error('BUG: no max term during resync');
}
with_max = with_max.filter(w => w != this.cfg.node_id);
// Merge databases of all nodes with maximum term
// Force other nodes to replicate the merged DB, throwing away their own states
for (let i = 0; i < with_max.length; i++)
{
const update_only = !(i == 0 && this.antietcd.stored_term != max_term);
console.log(update_only ? 'Updating database from node '+with_max[i]+' state' : 'Copying node '+with_max[i]+' state');
this.antietcd.etctree.load(this.resync_state.dumps[with_max[i]], update_only);
}
let wait = 0;
const load_request = { term: this.raft.term, load: this.antietcd.etctree.dump() };
for (const follower in this.resync_state.dumps)
{
if (follower != this.cfg.node_id)
{
const dump = this.resync_state.dumps[follower];
if (dump.term <= max_term)
{
const client = this._getPeer(follower);
if (!client)
{
console.log('Lost peer connection during resync - restarting election');
this.raft.start();
return;
}
console.log('Copying state to '+follower);
const loadstate = this.resync_state.loads[follower] = {};
wait++;
this._peerRequest(client, load_request, this.cfg.load_timeout||5000).then(res =>
{
loadstate.result = res;
this._finishResync();
});
}
}
}
if (!wait)
{
this._finishResync();
}
}
_finishResync()
{
if (Object.values(this.resync_state.dumps).filter(d => !d).length > 0 ||
Object.values(this.resync_state.loads).filter(d => !d.result).length > 0)
{
return;
}
// All current peers have copied the database, we can proceed
this.antietcd.stored_term = this.raft.term;
this.synced = true;
runCallbacks(this, 'wait_sync', []);
console.log('Synchronized with followers, new term is '+this.raft.term);
}
_isWrite(path, data)
{
if (path == 'kv_txn')
{
return ((!data.compare || !data.compare.length) &&
(!data.success || !data.success.filter(f => f.request_put || f.requestPut || f.request_delete_range || f.requestDeleteRange).length) &&
(!data.failure || !data.failure.filter(f => f.request_put || f.requestPut || f.request_delete_range || f.requestDeleteRange).length));
}
return path != 'kv_range';
}
async checkRaftState(path, requestUrl, data)
{
if (!this.raft)
{
return null;
}
if (requestUrl.searchParams.get('leaderonly') &&
this.raft.state != TinyRaft.LEADER)
{
throw new RequestError(503, 'Not leader');
}
if (this.raft.state == TinyRaft.CANDIDATE)
{
throw new RequestError(503, 'Quorum not available');
}
else if (this.raft.state == TinyRaft.FOLLOWER &&
(!this.cfg.stale_read || this._isWrite(path, data)))
{
// Forward to leader
return await this._forwardToLeader(path, data);
}
else if (!this.synced)
{
// Wait for initial sync for read-only requests
await new Promise(ok => this.wait_sync.push(ok));
}
return null;
}
async _forwardToLeader(handler, data)
{
const client = this._getPeer(this.raft.leader);
if (!client)
{
throw new RequestError(503, 'Leader is unavailable');
}
return await this._peerRequest(client, { handler, request: data }, this.cfg.forward_timeout||1000);
}
handleWsMsg(client, msg)
{
if (msg.raft)
{
if (client.raft_node_id)
{
this.raft.onReceive(client.raft_node_id, msg.raft);
}
}
else if (msg.identify)
{
if (msg.identify.key === this.cfg.cluster_key &&
msg.identify.node_id != this.cfg.node_id)
{
client.raft_node_id = msg.identify.node_id;
console.log('Got a connection from '+client.raft_node_id);
}
}
else if (msg.load)
{
this._handleLoadMsg(client, msg);
}
else if (msg.replicate)
{
this._handleReplicateMsg(client, msg).catch(console.error);
}
else if (msg.request)
{
this._handleRequestMsg(client, msg).catch(console.error);
}
else if (msg.reply)
{
this._completeRequest(client.id, msg.request_id, msg.reply);
}
}
async _handleRequestMsg(client, msg)
{
const cb = this.antietcd['handle_'+msg.handler];
if (cb)
{
try
{
let res = cb.call(this.antietcd, msg.request);
if (res instanceof Promise)
{
res = await res;
}
client.socket.send(JSON.stringify({ request_id: msg.request_id, reply: res }));
}
catch (e)
{
console.error(e);
client.socket.send(JSON.stringify({ request_id: msg.request_id, reply: { error: e.message } }));
}
}
else
{
client.socket.send(JSON.stringify({ request_id: msg.request_id, reply: { error: 'unknown handler' } }));
}
}
_handleLoadMsg(client, msg)
{
if (client.raft_node_id && this.raft.state == TinyRaft.FOLLOWER &&
this.raft.leader === client.raft_node_id && this.raft.term == msg.term)
{
this.antietcd.etctree.load(msg.load);
this.antietcd.stored_term = msg.term;
this.synced = true;
runCallbacks(this, 'wait_sync', []);
console.log('Synchronized with leader, new term is '+msg.term);
client.socket.send(JSON.stringify({ request_id: msg.request_id, reply: {} }));
}
else
{
client.socket.send(JSON.stringify({ request_id: msg.request_id, reply: { error: LEADER_MISMATCH } }));
}
}
async _handleReplicateMsg(client, msg)
{
if (client.raft_node_id && this.raft.state == TinyRaft.FOLLOWER &&
this.raft.leader === client.raft_node_id && this.raft.term == msg.term)
{
await this.antietcd.etctree.apply_replication(msg.replicate);
client.socket.send(JSON.stringify({ request_id: msg.request_id, reply: {} }));
}
else
{
client.socket.send(JSON.stringify({ request_id: msg.request_id, reply: { error: LEADER_MISMATCH } }));
}
}
_getPeer(to)
{
if (to == this.cfg.node_id)
{
throw new Error('BUG: attempt to get connection to self');
}
const client_id = this.cluster_connections[to];
if (!client_id)
{
return null;
}
const client = this.antietcd.clients[client_id];
if (!client || !client.ready)
{
return null;
}
return client;
}
_sendRaftMessage(to, msg)
{
const client = this._getPeer(to);
if (client)
{
client.socket.send(JSON.stringify({ raft: msg }));
}
}
}
module.exports = AntiCluster;

View File

@ -1,4 +1,6 @@
const fs = require('fs');
#!/usr/bin/node
const fsp = require('fs').promises;
const { URL } = require('url');
const http = require('http');
const https = require('https');
@ -6,53 +8,101 @@ const https = require('https');
const ws = require('ws');
const EtcTree = require('./etctree.js');
class RequestError
{
constructor(code, text)
{
this.code = code;
this.message = text;
}
}
const AntiPersistence = require('./antipersistence.js');
const AntiCluster = require('./anticluster.js');
const { runCallbacks, RequestError } = require('./common.js');
class AntiEtcd
{
constructor(cfg)
{
this.cfg = cfg;
}
run()
{
this.clients = {};
this.client_id = 1;
this.etctree = new EtcTree(true);
this.persistence = null;
this.cluster = null;
this.stored_term = 0;
this.cfg = cfg;
this.loading = false;
this.stopped = false;
this.inflight = 0;
this.wait_inflight = [];
}
async run()
{
if (this.cfg.filename || this.cfg.cluster)
{
this.etctree.set_replicate_watcher(msg => this.persistAndReplicate(msg));
}
if (this.cfg.filename)
{
this.persistence = new AntiPersistence(this);
// Load data from disk
await this.persistence.load();
// Set exit hook
const on_stop_cb = () => this.onStop();
process.on('SIGINT', on_stop_cb);
process.on('SIGTERM', on_stop_cb);
process.on('SIGQUIT', on_stop_cb);
}
if (this.cfg.cluster)
{
this.cluster = new AntiCluster(this);
}
if (this.cfg.cert)
{
this.tls = { key: fs.readFileSync(this.cfg.key), cert: fs.readFileSync(this.cfg.cert) };
this.tls = { key: await fsp.readFile(this.cfg.key), cert: await fsp.readFile(this.cfg.cert) };
this.server = https.createServer(this.tls, (req, res) => this.handleRequest(req, res));
}
else
{
this.server = http.createServer((req, res) => this.handleRequest(req, res));
}
this.wss = new ws.WebSocketServer({ server: this.server });
this.wss.on('connection', (conn, req) => this.startWebsocket(conn, req));
// eslint-disable-next-line no-unused-vars
this.wss.on('connection', (conn, req) => this.startWebsocket(conn, null));
this.server.listen(this.cfg.port || 2379);
}
fail(res, code, text)
async onStop()
{
res.writeHead(code);
res.write(text);
res.end();
if (this.stopped)
{
return;
}
this.stopped = true;
// Wait until all requests complete
while (this.inflight > 0)
{
await new Promise(ok => this.wait_inflight.push(ok));
}
if (this.persistence)
{
await this.persistence.persist();
}
process.exit(0);
}
async persistAndReplicate(msg)
{
if (this.persistence)
{
await this.persistence.persistChange(msg);
}
if (this.cluster)
{
await this.cluster.replicateChange(msg);
}
}
handleRequest(req, res)
{
let data = [];
req.on('data', (chunk) => data.push(chunk));
req.on('end', () =>
req.on('end', async () =>
{
this.inflight++;
data = Buffer.concat(data);
let body = '';
let code = 200;
@ -76,7 +126,7 @@ class AntiEtcd
{
throw new RequestError(400, 'body should be JSON object');
}
reply = this.runHandler(req, data, res);
reply = await this.runHandler(req, data, res);
reply = JSON.stringify(reply);
}
catch (e)
@ -93,34 +143,47 @@ class AntiEtcd
reply = 'Internal error: '+e.message;
}
}
// Access log
console.log(
new Date().toISOString()+
' '+(req.headers['x-forwarded-for'] || req.socket.remoteAddress)+
' '+req.method+' '+req.url+' '+code+'\n '+body.replace(/\n/g, '\\n')+
'\n '+reply.replace(/\n/g, '\\n')
);
// FIXME: Access log :req[X-Forwarded-For] [:date[clf]] pid=:pid ":method :url HTTP/:http-version" :status :res[content-length] ":referrer" ":user-agent" - :response-time ms
reply = Buffer.from(reply);
res.writeHead(200, {
'Content-Type': 'application/json',
'Content-Length': reply.length,
});
res.write(reply);
res.end();
try
{
// Access log
console.log(
new Date().toISOString()+
' '+(req.headers['x-forwarded-for'] || (req.socket.remoteAddress + ':' + req.socket.remotePort))+
' '+req.method+' '+req.url+' '+code+'\n '+body.replace(/\n/g, '\\n')+
'\n '+reply.replace(/\n/g, '\\n')
);
reply = Buffer.from(reply);
res.writeHead(200, {
'Content-Type': 'application/json',
'Content-Length': reply.length,
});
res.write(reply);
res.end();
}
catch (e)
{
console.error(e);
}
this.inflight--;
if (!this.inflight)
{
runCallbacks(this, 'wait_inflight', []);
}
});
}
startWebsocket(socket, req)
startWebsocket(socket, reconnect)
{
const client_id = this.client_id++;
this.clients[client_id] = {
id: client_id,
addr: socket._socket ? socket._socket.remoteAddress+':'+socket._socket.remotePort : '',
socket,
alive: true,
watches: {},
};
socket.on('pong', () => this.clients[client_id].alive = true);
socket.on('error', console.error);
socket.on('error', e => console.error(e.syscall === 'connect' ? e.message : e));
const pinger = setInterval(() =>
{
if (!this.clients[client_id])
@ -160,10 +223,16 @@ class AntiEtcd
clearInterval(pinger);
delete this.clients[client_id];
socket.terminate();
if (reconnect)
{
reconnect();
}
});
return client_id;
}
runHandler(req, data, res)
// eslint-disable-next-line no-unused-vars
async runHandler(req, data, res)
{
// v3/kv/txn
// v3/kv/range
@ -173,10 +242,22 @@ class AntiEtcd
// v3/lease/keepalive
// v3/lease/revoke O_o
// v3/kv/lease/revoke O_o
if (this.stopped)
{
throw new RequestError(502, 'Server is stopping');
}
const requestUrl = new URL(req.url, 'http://'+(req.headers.host || 'localhost'));
if (requestUrl.pathname.substr(0, 4) == '/v3/')
{
const path = requestUrl.pathname.substr(4).replace(/\/+$/, '').replace(/\/+/g, '_');
if (this.cluster)
{
const res = await this.cluster.checkRaftState(path, requestUrl, data);
if (res)
{
return res;
}
}
const cb = this['handle_'+path];
if (cb)
{
@ -184,33 +265,42 @@ class AntiEtcd
{
throw new RequestError(405, 'Please use POST method');
}
return cb.call(this, data);
const res = cb.call(this, data);
if (res instanceof Promise)
{
return await res;
}
return res;
}
}
else if (requestUrl.pathname == '/dump')
{
return this.handle_dump(data);
}
throw new RequestError(404, 'Supported APIs: /v3/kv/txn, /v3/kv/range, /v3/kv/put, /v3/kv/deleterange, '+
'/v3/lease/grant, /v3/lease/revoke, /v3/kv/lease/revoke, /v3/lease/keepalive');
}
handle_kv_txn(data)
async handle_kv_txn(data)
{
return this.etctree.api_txn(data);
return await this.etctree.api_txn(data);
}
handle_kv_range(data)
async handle_kv_range(data)
{
const r = this.etctree.api_txn({ success: [ { request_range: data } ] });
const r = await this.etctree.api_txn({ success: [ { request_range: data } ] });
return { header: r.header, ...r.responses[0].response_range };
}
handle_kv_put(data)
async handle_kv_put(data)
{
const r = this.etctree.api_txn({ success: [ { request_put: data } ] });
const r = await this.etctree.api_txn({ success: [ { request_put: data } ] });
return { header: r.header, ...r.responses[0].response_put };
}
handle_kv_deleterange(data)
async handle_kv_deleterange(data)
{
const r = this.etctree.api_txn({ success: [ { request_delete_range: data } ] });
const r = await this.etctree.api_txn({ success: [ { request_delete_range: data } ] });
return { header: r.header, ...r.responses[0].response_delete_range };
}
@ -234,13 +324,20 @@ class AntiEtcd
return this.etctree.api_keepalive_lease(data);
}
// eslint-disable-next-line no-unused-vars
handle_dump(data)
{
return { ...this.etctree.dump(), term: this.stored_term };
}
handleMessage(client_id, msg, socket)
{
const client = this.clients[client_id];
console.log(new Date().toISOString()+' '+client.addr+' '+(client.raft_node_id || '-')+' -> '+JSON.stringify(msg));
if (msg.create_request)
{
// FIXME progress_notify, filters, prev_kv
const create_request = msg.create_request;
if (!create_request.watch_id || !this.clients[client_id].watches[create_request.watch_id])
if (!create_request.watch_id || !client.watches[create_request.watch_id])
{
const watch = this.etctree.api_create_watch(
{ ...create_request, watch_id: null }, (msg) => socket.send(JSON.stringify(msg))
@ -251,19 +348,19 @@ class AntiEtcd
}
else
{
create_request.watch_id ||= watch.watch_id;
this.clients[client_id].watches[create_request.watch_id] = watch.watch_id;
create_request.watch_id = create_request.watch_id || watch.watch_id;
client.watches[create_request.watch_id] = watch.watch_id;
socket.send(JSON.stringify({ result: { header: { revision: this.etctree.mod_revision }, watch_id: create_request.watch_id, created: true } }));
}
}
}
else if (msg.cancel_request)
{
const mapped_id = this.clients[client_id].watches[msg.cancel_request.watch_id];
const mapped_id = client.watches[msg.cancel_request.watch_id];
if (mapped_id)
{
this.etctree.api_cancel_watch({ watch_id: mapped_id });
delete this.clients[client_id].watches[msg.cancel_request.watch_id];
delete client.watches[msg.cancel_request.watch_id];
socket.send(JSON.stringify({ result: { header: { revision: this.etctree.mod_revision }, watch_id: msg.cancel_request.watch_id, canceled: true } }));
}
}
@ -271,12 +368,22 @@ class AntiEtcd
{
socket.send(JSON.stringify({ result: { header: { revision: this.etctree.mod_revision } } }));
}
else
{
if (!this.cluster)
{
return;
}
this.cluster.handleWsMsg(client, msg);
}
}
unsubscribeClient(client_id)
{
if (!this.clients[client_id])
{
return;
}
for (const watch_id in this.clients[client_id].watches)
{
const mapped_id = this.clients[client_id].watches[watch_id];
@ -285,4 +392,81 @@ class AntiEtcd
}
}
new AntiEtcd({ port: 12379 }).run();
function vitastor_persist_filter(prefix)
{
return (key, value) =>
{
if (key.substr(0, prefix.length+'/osd/stats/'.length) == prefix+'/osd/stats/')
{
if (value)
{
try
{
value = JSON.parse(value);
value = JSON.stringify({
bitmap_granularity: value.bitmap_granularity || undefined,
data_block_size: value.data_block_size || undefined,
host: value.host || undefined,
immediate_commit: value.immediate_commit || undefined,
});
}
catch (e)
{
console.error('invalid JSON in '+key+' = '+value+': '+e);
value = {};
}
}
else
{
value = undefined;
}
return value;
}
else if (key.substr(0, prefix.length+'/osd/'.length) == prefix+'/osd/' ||
key.substr(0, prefix.length+'/inode/stats/'.length) == prefix+'/inode/stats/' ||
key.substr(0, prefix.length+'/pg/stats/'.length) == prefix+'/pg/stats/' ||
key.substr(0, prefix.length+'/pool/stats/'.length) == prefix+'/pool/stats/' ||
key == prefix+'/stats')
{
return undefined;
}
return value;
};
}
function parse()
{
const options = {
persist_filter: vitastor_persist_filter('/vitastor'),
};
for (let i = 2; i < process.argv.length; i++)
{
const arg = process.argv[i].toLowerCase().replace(/^--(.+)$/, (m, m1) => '--'+m1.replace(/-/g, '_'));
if (arg === '-h' || arg === '--help')
{
console.error(
'USAGE:\n '+process.argv[0]+' '+process.argv[1]+' [OPTIONS]\n'+
'OPTIONS:\n'+
' [--cert ssl.crt] [--key ssl.key] [--port 12379]\n'+
' [--filename data.gz] [--vitastor-persist-filter /vitastor] [--no-persist-filter] [--persist_interval 500]\n'+
' [--node_id node1 --cluster_key abcdef --cluster node1=http://localhost:12379,node2=http://localhost:12380,node3=http://localhost:12381]'
);
process.exit();
}
else if (arg == '--no_persist_filter')
{
options['persist_filter'] = null;
}
else if (arg == '--vitastor_persist_filter')
{
options['persist_filter'] = vitastor_persist_filter(process.argv[++i]||'');
}
else if (arg.substr(0, 2) == '--' && arg != '--persist_filter')
{
options[arg.substr(2)] = process.argv[++i];
}
}
return options;
}
new AntiEtcd(parse()).run().catch(console.error);

134
antipersistence.js Normal file
View File

@ -0,0 +1,134 @@
const fs = require('fs');
const fsp = require('fs').promises;
const zlib = require('zlib');
const EtcTree = require('./etctree.js');
const { de64, runCallbacks } = require('./common.js');
class AntiPersistence
{
constructor(antietcd)
{
this.cfg = antietcd.cfg;
this.antietcd = antietcd;
this.prev_value = {};
this.persist_timer = null;
this.wait_persist = null;
}
async load()
{
// eslint-disable-next-line no-unused-vars
const [ err, stat ] = await new Promise(ok => fs.stat(this.cfg.filename, (err, stat) => ok([ err, stat ])));
if (!err)
{
let data = await fsp.readFile(this.cfg.filename);
data = await new Promise((ok, no) => zlib.gunzip(data, (err, res) => err ? no(err) : ok(res)));
data = JSON.parse(data);
this.loading = true;
this.antietcd.etctree.load(data);
this.loading = false;
this.antietcd.stored_term = data['term'];
}
else if (err.code != 'ENOENT')
{
throw err;
}
}
async persistChange(msg)
{
if (this.loading)
{
return;
}
if (!msg.events || !msg.events.length)
{
// lease-only changes don't need to be persisted
return;
}
if (this.cfg.persist_filter)
{
let changed = false;
for (const ev of msg.events)
{
if (ev.kv.lease)
{
// Values with lease are never persisted
const key = de64(ev.kv.key);
if (this.prev_value[key] !== undefined)
{
delete this.prev_value[key];
changed = true;
}
}
else
{
const key = de64(ev.kv.key);
const filtered = this.cfg.persist_filter(key, ev.type === 'DELETE' ? undefined : de64(ev.kv.value));
if (!EtcTree.eq(filtered, this.prev_value[key]))
{
this.prev_value[key] = filtered;
changed = true;
}
}
changed = true;
}
if (!changed)
{
return;
}
}
await this.schedulePersist();
}
async schedulePersist()
{
if (!this.cfg.persist_interval)
{
await this.persist();
return;
}
if (!this.persist_timer)
{
this.persist_timer = setTimeout(() =>
{
this.persist_timer = null;
this.persist().catch(console.error);
}, this.cfg.persist_interval);
}
}
async persist()
{
if (!this.cfg.filename)
{
return;
}
while (this.wait_persist)
{
await new Promise(ok => this.wait_persist.push(ok));
}
this.wait_persist = [];
try
{
let dump = this.antietcd.etctree.dump(true);
dump['term'] = this.antietcd.stored_term;
dump = JSON.stringify(dump);
dump = await new Promise((ok, no) => zlib.gzip(dump, (err, res) => err ? no(err) : ok(res)));
const fh = await fsp.open(this.cfg.filename+'.tmp', 'w');
await fh.writeFile(dump);
await fh.sync();
await fh.close();
await fsp.rename(this.cfg.filename+'.tmp', this.cfg.filename);
}
catch (e)
{
console.error(e);
process.exit(1);
}
runCallbacks(this, 'wait_persist', null);
}
}
module.exports = AntiPersistence;

34
common.js Normal file
View File

@ -0,0 +1,34 @@
class RequestError
{
constructor(code, text)
{
this.code = code;
this.message = text;
}
}
function de64(k)
{
if (k == null) // null or undefined
return k;
return Buffer.from(k, 'base64').toString();
}
function runCallbacks(obj, key, new_value)
{
const cbs = obj[key];
obj[key] = new_value;
if (cbs)
{
for (const cb of cbs)
{
cb();
}
}
}
module.exports = {
RequestError,
de64,
runCallbacks,
};

View File

@ -22,8 +22,27 @@ class EtcTree
this.watcher_id = 0;
this.mod_revision = 0;
this.use_base64 = use_base64;
this.replicate = null;
this.paused = false;
this.on_expire_lease = null;
this.active_immediate = [];
}
destroy()
{
this.pause_leases();
for (const imm of this.active_immediate)
{
clearImmediate(imm);
}
}
set_replicate_watcher(replicate)
{
// Replication watcher is special:
// It should be an async function and it is called BEFORE notifying all
// other watchers about any change.
// It may also throw to prevent notifying at all if replication fails.
this.replicate = replicate;
}
de64(k)
@ -40,10 +59,10 @@ class EtcTree
return this.use_base64 ? Buffer.from(k).toString('base64') : k;
}
check(chk)
_check(chk)
{
const parts = this.key_parts(this.de64(chk.key));
const { cur } = this.get_subtree(parts, false, false);
const parts = this._key_parts(this.de64(chk.key));
const { cur } = this._get_subtree(parts, false, false);
let check_value, ref_value;
if (chk.target === 'MOD')
{
@ -80,13 +99,13 @@ class EtcTree
return check_value == ref_value;
}
key_parts(key)
_key_parts(key)
{
const parts = key.replace(/\/\/+/g, '/').replace(/\/$/g, ''); // trim beginning?
return parts === '' ? [] : parts.split('/');
}
get_range(req)
_get_range(req)
{
const key = this.de64(req.key);
const end = this.de64(req.range_end);
@ -95,11 +114,11 @@ class EtcTree
{
throw new Error('Non-directory range queries are unsupported');
}
const parts = this.key_parts(key);
const parts = this._key_parts(key);
return { parts, all: end != null };
}
get_subtree(parts, create, notify)
_get_subtree(parts, create, notify)
{
let cur = this.state;
let watchers = notify ? [] : null;
@ -135,10 +154,10 @@ class EtcTree
}
// create a snapshot of all data including leases
dump(persistent_only)
dump(persistent_only, value_filter)
{
const snapshot = {
state: this._copy_tree(this.state, persistent_only) || {},
state: this._copy_tree(this.state, persistent_only, value_filter) || {},
mod_revision: this.mod_revision,
};
if (!persistent_only)
@ -153,17 +172,27 @@ class EtcTree
return snapshot;
}
_copy_tree(cur, no_lease)
_copy_tree(cur, no_lease, value_filter)
{
const nonempty = cur.value != null && (!no_lease || !copy.lease);
let nonempty = cur.value != null && (!no_lease || !cur.lease);
let filtered;
if (nonempty && value_filter)
{
filtered = value_filter(cur.value);
nonempty = nonempty && filtered != null;
}
const copy = (nonempty ? { ...cur } : {});
copy.children = {};
if (nonempty && value_filter)
{
copy.value = filtered;
}
delete copy.watchers;
delete copy.key_watchers;
let has_children = false;
for (const k in cur.children)
{
const child = this._copy_tree(cur.children[k]);
const child = this._copy_tree(cur.children[k], no_lease, value_filter);
if (child)
{
copy.children[k] = child;
@ -182,73 +211,59 @@ class EtcTree
}
// load snapshot of all data including leases
load(snapshot)
load(snapshot, update_only)
{
this.mod_revision = snapshot.mod_revision;
// First apply leases
for (const id in this.leases)
if (!update_only || this.mod_revision < snapshot.mod_revision)
{
if (!snapshot.leases[id])
this.mod_revision = snapshot.mod_revision;
}
// First apply leases
const notifications = [];
if (!update_only && snapshot.leases)
{
for (const id in this.leases)
{
this.api_revoke_lease(id);
if (!snapshot.leases[id])
{
// Revoke without replicating and notifying
this._sync_revoke_lease(id, notifications);
}
}
}
for (const id in snapshot.leases)
for (const id in snapshot.leases||{})
{
if (!this.leases[id])
{
this.leases[id] = { ...snapshot.leases[id], timer_id: null, keys: {} };
}
else if (this.leases[id].ttl != snapshot.leases[id].ttl ||
this.leases[id].expires != snapshot.leases[id].expires)
{
this.leases[id].ttl = snapshot.leases[id].ttl;
this.leases[id].expires = snapshot.leases[id].expires;
}
else
{
continue;
}
if (this.leases[id].timer_id)
{
clearTimeout(this.leases[id].timer_id);
this.leases[id].timer_id = null;
}
if (!this.paused)
{
this.leases[id].timer_id = setTimeout(() => this.expire_lease(id), this.leases[id].expires - Date.now());
}
this.load_lease({ id, ...snapshot.leases[id] });
}
// Then find and apply the difference in data
const notifications = [];
this._restore_diff(this.state, snapshot.state, null, this.state.watchers || [], notifications);
this.notify(notifications);
this._restore_diff(update_only, this.state, snapshot.state, null, this.state.watchers || [], notifications);
this._notify(notifications);
}
_restore_diff(cur_old, cur_new, prefix, watchers, notifications)
_restore_diff(update_only, cur_old, cur_new, prefix, watchers, notifications)
{
const key = prefix === null ? '' : prefix;
if (!eq(cur_old.lease, cur_new.lease))
if (!update_only || !cur_old.mod_revision || cur_old.mod_revision < cur_new.mod_revision)
{
if (cur_old.lease && this.leases[cur_old.lease])
const key = prefix === null ? '' : prefix;
if (!eq(cur_old.lease, cur_new.lease))
{
delete this.leases[cur_old.lease].keys[key];
if (cur_old.lease && this.leases[cur_old.lease])
{
delete this.leases[cur_old.lease].keys[key];
}
cur_old.lease = cur_new.lease;
if (cur_new.lease && this.leases[cur_new.lease])
{
this.leases[cur_new.lease].keys[key] = true;
}
}
cur_old.lease = cur_new.lease;
if (cur_new.lease && this.leases[cur_new.lease])
cur_old.mod_revision = cur_new.mod_revision;
cur_old.create_revision = cur_new.create_revision;
cur_old.version = cur_new.version;
if (!eq(cur_old.value, cur_new.value))
{
this.leases[cur_new.lease].keys[key] = true;
}
}
cur_old.mod_revision = cur_new.mod_revision;
cur_old.create_revision = cur_new.create_revision;
cur_old.version = cur_new.version;
if (!eq(cur_old.value, cur_new.value))
{
cur_old.value = cur_new.value;
for (const w of (cur_old.key_watchers ? [ ...watchers, ...(cur_old.key_watchers||[]) ] : watchers))
{
const notify = { watchers, key, value: cur_old.value, mod_revision: cur_old.mod_revision };
cur_old.value = cur_new.value;
const key_watchers = (cur_old.key_watchers ? [ ...watchers, ...(cur_old.key_watchers||[]) ] : watchers);
const notify = { watchers: key_watchers, key, value: cur_old.value, mod_revision: cur_old.mod_revision };
if (cur_old.lease)
{
notify.lease = cur_old.lease;
@ -256,7 +271,7 @@ class EtcTree
notifications.push(notify);
}
}
cur_old.children ||= {};
cur_old.children = cur_old.children || {};
for (const k in cur_new.children)
{
if (!cur_old.children[k])
@ -266,25 +281,28 @@ class EtcTree
else
{
this._restore_diff(
cur_old.children[k], cur_new.children[k],
update_only, cur_old.children[k], cur_new.children[k],
prefix === null ? k : prefix+'/'+k,
cur_old.children[k].watchers ? [ ...watchers, ...cur_old.children[k].watchers ] : watchers,
notifications
);
}
}
for (const k in cur_old.children)
if (!update_only)
{
if (!cur_new.children[k])
for (const k in cur_old.children)
{
// Delete subtree
this.delete_all(
notifications,
cur_old.children[k].watchers ? [ ...watchers, ...cur_old.children[k].watchers ] : watchers,
cur_old.children[k], true,
prefix === null ? k : prefix+'/'+k,
this.mod_revision
);
if (!cur_new.children[k])
{
// Delete subtree
this._delete_all(
notifications,
cur_old.children[k].watchers ? [ ...watchers, ...cur_old.children[k].watchers ] : watchers,
cur_old.children[k], true,
prefix === null ? k : prefix+'/'+k,
this.mod_revision
);
}
}
}
}
@ -292,6 +310,10 @@ class EtcTree
// slave/follower nodes don't expire leases themselves, they listen for the leader instead
pause_leases()
{
if (this.paused)
{
return;
}
this.paused = true;
for (const id in this.leases)
{
@ -306,23 +328,30 @@ class EtcTree
resume_leases()
{
if (!this.paused)
{
return;
}
this.paused = false;
for (const id in this.leases)
{
this._set_expire(id);
}
}
_set_expire(id)
{
if (!this.paused)
{
const lease = this.leases[id];
if (!lease.timer_id)
{
lease.timer_id = setTimeout(() => this.expire_lease(id), lease.expires - Date.now());
lease.timer_id = setTimeout(() => this.api_revoke_lease({ ID: id }).catch(console.error), lease.expires - Date.now());
}
}
}
set_on_expire_lease(cb)
{
this.on_expire_lease = cb;
}
api_grant_lease(req)
async api_grant_lease(req)
{
let id;
while (!id || this.leases[id])
@ -330,12 +359,16 @@ class EtcTree
id = crypto.randomBytes(8).toString('hex');
}
const expires = Date.now() + req.TTL*1000;
const timer_id = this.paused ? null : setTimeout(() => this.expire_lease(id), req.TTL*1000);
this.leases[id] = { ttl: req.TTL, expires, timer_id, keys: {} };
this.leases[id] = { ttl: req.TTL, expires, timer_id: null, keys: {} };
this._set_expire(id);
if (this.replicate)
{
await this.replicate({ header: { revision: this.mod_revision }, leases: [ { id, ttl: req.TTL, expires } ] });
}
return { header: { revision: this.mod_revision }, ID: id, TTL: req.TTL };
}
api_keepalive_lease(req)
async api_keepalive_lease(req)
{
const id = req.ID;
if (!this.leases[id])
@ -350,42 +383,126 @@ class EtcTree
}
const ttl = this.leases[id].ttl;
lease.expires = Date.now() + ttl*1000;
if (!this.paused)
this._set_expire(id);
if (this.replicate)
{
lease.timer_id = setTimeout(() => this.expire_lease(id), ttl*1000);
await this.replicate({ header: { revision: this.mod_revision }, leases: [ { id, ttl, expires: lease.expires } ] });
}
// extra wrapping in { result: ... }
return { result: { header: { revision: this.mod_revision }, ID: id, TTL: ''+ttl } };
}
expire_lease(id)
load_lease(lease)
{
this.api_revoke_lease({ ID: id })
if (this.on_expire_lease)
const id = lease.id;
if (!this.leases[id])
{
this.on_expire_lease(id);
this.leases[id] = { ...lease, timer_id: null, keys: {} };
}
else if (this.leases[id].ttl != lease.ttl ||
this.leases[id].expires != lease.expires)
{
this.leases[id].ttl = lease.ttl;
this.leases[id].expires = lease.expires;
}
else
{
return false;
}
if (this.leases[id].timer_id)
{
clearTimeout(this.leases[id].timer_id);
this.leases[id].timer_id = null;
}
this._set_expire(id);
return true;
}
api_revoke_lease(req)
_sync_revoke_lease(id, notifications)
{
if (!this.leases[req.ID])
if (!this.leases[id])
{
throw new Error('unknown lease');
}
const next_revision = this.mod_revision + 1;
const notifications = [];
for (const key in this.leases[req.ID].keys)
for (const key in this.leases[id].keys)
{
this.txn_action({ request_delete_range: { key } }, next_revision, notifications);
this._delete_range({ key }, next_revision, notifications);
}
this.notify(notifications);
delete this.leases[id];
}
async api_revoke_lease(req, no_throw)
{
const notifications = [];
if (!this.leases[req.ID])
{
if (no_throw)
return null;
throw new Error('unknown lease');
}
this._sync_revoke_lease(req.ID, notifications);
if (this.replicate)
{
await this.notify_replicator(notifications, [ { id: req.ID } ]);
}
this._notify(notifications);
return { header: { revision: this.mod_revision } };
}
async notify_replicator(notifications, leases)
{
// First replicate the change and then notify watchers about it
const all_changes = {};
for (const chg of notifications)
{
all_changes[chg.key] = { ...chg };
delete all_changes[chg.key].watchers;
}
await this.replicate({ header: { revision: this.mod_revision }, events: Object.values(all_changes), leases });
}
async apply_replication(msg)
{
const notifications = [];
if ((msg.leases||[]).length)
{
for (const lease of msg.leases)
{
if (lease.ttl)
{
this.load_lease(lease);
}
else
{
this._sync_revoke_lease(lease.id, notifications);
}
}
}
if ((msg.events||[]).length)
{
for (const ev of msg.events)
{
if (ev.value == null)
{
this._delete_range({ key: ev.key }, ev.mod_revision, notifications);
}
else
{
this._put({ key: ev.key, value: ev.value }, ev.mod_revision, notifications);
}
}
}
if (this.replicate)
{
await this.notify_replicator(notifications, msg.leases);
}
this._notify(notifications);
}
api_create_watch(req, send)
{
const { parts, all } = this.get_range(req);
const { parts, all } = this._get_range(req);
if (req.start_revision && this.compact_revision && this.compact_revision > req.start_revision)
{
// Deletions up to this.compact_revision are forgotten
@ -408,7 +525,7 @@ class EtcTree
};
}
this.watchers[watch_id].paths.push(parts);
const { cur } = this.get_subtree(parts, true, false);
const { cur } = this._get_subtree(parts, true, false);
if (all)
{
cur.watchers = cur.watchers || [];
@ -422,34 +539,42 @@ class EtcTree
if (req.start_revision && req.start_revision < this.mod_revision)
{
// Send initial changes
setImmediate(() =>
const imm = setImmediate(() =>
{
this.active_immediate = this.active_immediate.filter(i => i !== imm);
const events = [];
const { cur } = this.get_subtree([], false, false);
this.get_modified(events, cur, null, req.start_revision);
const { cur } = this._get_subtree([], false, false);
this._get_modified(events, cur, null, req.start_revision);
send({ result: { header: { revision: this.mod_revision }, events } });
});
this.active_immediate.push(imm);
}
return { watch_id, created: true };
}
get_modified(events, cur, prefix, min_rev)
_get_modified(events, cur, prefix, min_rev)
{
if (cur.mod_revision >= min_rev)
{
events.push({
const ev = {
type: cur.value == null ? 'DELETE' : 'PUT',
kv: cur.value == null ? { key: this.b64(prefix === null ? '' : prefix) } : {
key: this.b64(prefix),
value: this.b64(cur.value),
mod_revision: cur.mod_revision,
},
});
};
if (cur.lease)
{
ev.kv.lease = cur.lease;
}
events.push(ev);
}
if (cur.children)
{
for (const k in cur.children)
{
this.get_modified(events, cur.children[k], prefix === null ? k : prefix+'/'+k, min_rev);
this._get_modified(events, cur.children[k], prefix === null ? k : prefix+'/'+k, min_rev);
}
}
}
@ -460,7 +585,7 @@ class EtcTree
{
for (const parts of this.watchers[watch_id].paths)
{
const { cur } = this.get_subtree(parts, false, false);
const { cur } = this._get_subtree(parts, false, false);
if (cur)
{
if (cur.watchers)
@ -483,21 +608,7 @@ class EtcTree
return { canceled: true };
}
api_txn({ compare, success, failure })
{
const failed = (compare || []).filter(chk => !this.check(chk)).length > 0;
const responses = [];
const notifications = [];
const next_revision = this.mod_revision + 1;
for (const req of (failed ? failure : success) || [])
{
responses.push(this.txn_action(req, next_revision, notifications));
}
this.notify(notifications);
return { header: { revision: this.mod_revision }, succeeded: !failed, responses };
}
notify(notifications)
_notify(notifications)
{
if (!notifications.length)
{
@ -525,87 +636,118 @@ class EtcTree
}
}
txn_action(req, cur_revision, notifications)
async api_txn({ compare, success, failure })
{
const failed = (compare || []).filter(chk => !this._check(chk)).length > 0;
const responses = [];
const notifications = [];
const next_revision = this.mod_revision + 1;
for (const req of (failed ? failure : success) || [])
{
responses.push(this._txn_action(req, next_revision, notifications));
}
if (this.replicate && notifications.length)
{
// First replicate the change and then notify watchers about it
await this.notify_replicator(notifications);
}
this._notify(notifications);
return { header: { revision: this.mod_revision }, succeeded: !failed, responses };
}
_txn_action(req, cur_revision, notifications)
{
if (req.request_range || req.requestRange)
{
const request_range = req.request_range || req.requestRange;
// FIXME: limit, revision(-), sort_order, sort_target, serializable(-),
// count_only, min_mod_revision, max_mod_revision, min_create_revision, max_create_revision
const { parts, all } = this.get_range(request_range);
const { cur } = this.get_subtree(parts, false, false);
const kvs = [];
if (cur)
{
this.get_all(kvs, cur, all, parts.join('/') || null, request_range);
}
return { response_range: { kvs } };
return { response_range: this._range(req.request_range || req.requestRange) };
}
else if (req.request_put || req.requestPut)
{
const request_put = req.request_put || req.requestPut;
// FIXME: prev_kv, ignore_value(?), ignore_lease(?)
const parts = this.key_parts(this.de64(request_put.key));
const key = parts.join('/');
const value = this.de64(request_put.value);
const { cur, watchers } = this.get_subtree(parts, true, true);
if (cur.key_watchers)
{
watchers.push.apply(watchers, cur.key_watchers);
}
if (!eq(cur.value, value) || cur.lease != request_put.lease)
{
if (cur.lease && this.leases[cur.lease])
{
delete this.leases[cur.lease].keys[key];
}
if (request_put.lease)
{
if (!this.leases[request_put.lease])
{
throw new Error('unknown lease: '+request_put.lease);
}
cur.lease = request_put.lease;
this.leases[request_put.lease].keys[key] = true;
}
else if (cur.lease)
{
cur.lease = null;
}
this.mod_revision = cur_revision;
cur.version = (cur.version||0) + 1;
cur.mod_revision = cur_revision;
if (cur.value == null)
{
cur.create_revision = cur_revision;
}
cur.value = value;
const notify = { watchers, key: this.b64(key), value: this.b64(value), mod_revision: cur.mod_revision };
if (cur.lease)
{
notify.lease = cur.lease;
}
notifications.push(notify);
}
return { response_put: {} };
return { response_put: this._put(req.request_put || req.requestPut, cur_revision, notifications) };
}
else if (req.request_delete_range || req.requestDeleteRange)
{
const request_delete_range = req.request_delete_range || req.requestDeleteRange;
// FIXME: prev_kv
const { parts, all } = this.get_range(request_delete_range);
const { cur, watchers } = this.get_subtree(parts, false, true);
const prevcount = notifications.length;
if (cur)
{
this.delete_all(notifications, watchers, cur, all, parts.join('/') || null, cur_revision);
}
return { response_delete_range: { deleted: notifications.length-prevcount } };
return { response_delete_range: this._delete_range(req.request_delete_range || req.requestDeleteRange, cur_revision, notifications) };
}
return {};
}
get_all(kvs, cur, all, prefix, req)
_range(request_range)
{
// FIXME: limit, revision(-), sort_order, sort_target, serializable(-),
// count_only, min_mod_revision, max_mod_revision, min_create_revision, max_create_revision
const { parts, all } = this._get_range(request_range);
const { cur } = this._get_subtree(parts, false, false);
const kvs = [];
if (cur)
{
this._get_all(kvs, cur, all, parts.join('/') || null, request_range);
}
return { kvs };
}
_put(request_put, cur_revision, notifications)
{
// FIXME: prev_kv, ignore_value(?), ignore_lease(?)
const parts = this._key_parts(this.de64(request_put.key));
const key = parts.join('/');
const value = this.de64(request_put.value);
const { cur, watchers } = this._get_subtree(parts, true, true);
if (cur.key_watchers)
{
watchers.push.apply(watchers, cur.key_watchers);
}
if (!eq(cur.value, value) || cur.lease != request_put.lease)
{
if (cur.lease && this.leases[cur.lease])
{
delete this.leases[cur.lease].keys[key];
}
if (request_put.lease)
{
if (!this.leases[request_put.lease])
{
throw new Error('unknown lease: '+request_put.lease);
}
cur.lease = request_put.lease;
this.leases[request_put.lease].keys[key] = true;
}
else if (cur.lease)
{
cur.lease = null;
}
this.mod_revision = cur_revision;
cur.version = (cur.version||0) + 1;
cur.mod_revision = cur_revision;
if (cur.value == null)
{
cur.create_revision = cur_revision;
}
cur.value = value;
const notify = { watchers, key: this.b64(key), value: this.b64(value), mod_revision: cur.mod_revision };
if (cur.lease)
{
notify.lease = cur.lease;
}
notifications.push(notify);
}
return {};
}
_delete_range(request_delete_range, cur_revision, notifications)
{
// FIXME: prev_kv
const { parts, all } = this._get_range(request_delete_range);
const { cur, watchers } = this._get_subtree(parts, false, true);
const prevcount = notifications.length;
if (cur)
{
this._delete_all(notifications, watchers, cur, all, parts.join('/') || null, cur_revision);
}
return { deleted: notifications.length-prevcount };
}
_get_all(kvs, cur, all, prefix, req)
{
if (req.limit && kvs.length > req.limit)
{
@ -631,12 +773,12 @@ class EtcTree
{
for (let k in cur.children)
{
this.get_all(kvs, cur.children[k], true, prefix === null ? k : prefix+'/'+k, req);
this._get_all(kvs, cur.children[k], true, prefix === null ? k : prefix+'/'+k, req);
}
}
}
delete_all(notifications, watchers, cur, all, prefix, cur_revision)
_delete_all(notifications, watchers, cur, all, prefix, cur_revision)
{
if (cur.value != null)
{
@ -662,7 +804,7 @@ class EtcTree
for (let k in cur.children)
{
const subw = cur.children[k].watchers ? [ ...watchers, ...cur.children[k].watchers ] : watchers;
this.delete_all(notifications, subw, cur.children[k], true, prefix === null ? k : prefix+'/'+k, cur_revision);
this._delete_all(notifications, subw, cur.children[k], true, prefix === null ? k : prefix+'/'+k, cur_revision);
}
}
}

View File

@ -17,27 +17,27 @@ tests['read/write'] = async () =>
{
const t = new EtcTree();
expect(
t.api_txn({ success: [ { request_put: { key: '/vitastor//config/global', value: { hello: 'world' } } } ] }),
await t.api_txn({ success: [ { request_put: { key: '/vitastor//config/global', value: { hello: 'world' } } } ] }),
{ header: { revision: 1 }, succeeded: true, responses: [ { response_put: {} } ] }
);
expect(
t.api_txn({ success: [ { request_range: { key: '/vitastor/config/global' } } ] }),
await t.api_txn({ success: [ { request_range: { key: '/vitastor/config/global' } } ] }),
{ header: { revision: 1 }, succeeded: true, responses: [ { response_range: {
kvs: [ { key: '/vitastor/config/global', mod_revision: 1, value: { hello: 'world' } } ],
} } ] }
);
expect(
t.api_txn({ success: [ { request_range: { key: '/vitastor/config/', range_end: '/vitastor/config0' } } ] }),
await t.api_txn({ success: [ { request_range: { key: '/vitastor/config/', range_end: '/vitastor/config0' } } ] }),
{ header: { revision: 1 }, succeeded: true, responses: [ { response_range: {
kvs: [ { key: '/vitastor/config/global', mod_revision: 1, value: { hello: 'world' } } ],
} } ] }
);
expect(
t.api_txn({ success: [ { request_range: { key: '/vitasto/', range_end: '/vitasto0' } } ] }),
await t.api_txn({ success: [ { request_range: { key: '/vitasto/', range_end: '/vitasto0' } } ] }),
{ header: { revision: 1 }, succeeded: true, responses: [ { response_range: { kvs: [] } } ] }
);
expect(
t.api_txn({
await t.api_txn({
compare: [ { key: '/vitastor/config/global', target: 'MOD', mod_revision: 1, result: 'LESS' } ],
success: [ { request_put: { key: '/vitastor//config/global', value: { hello: 'world' } } } ],
failure: [ { request_range: { key: '/vitastor/config/global' } } ],
@ -47,14 +47,14 @@ tests['read/write'] = async () =>
} } ] }
);
expect(
t.api_txn({
await t.api_txn({
compare: [ { key: '/vitastor/config/global', target: 'MOD', mod_revision: 2, result: 'LESS' } ],
success: [ { request_put: { key: '/vitastor//config/global', value: { hello: 'world2' } } } ]
}),
{ header: { revision: 2 }, succeeded: true, responses: [ { response_put: {} } ] }
);
expect(
t.api_txn({ success: [ { request_range: { key: '/vitastor/config/', range_end: '/vitastor/config0' } } ] }),
await t.api_txn({ success: [ { request_range: { key: '/vitastor/config/', range_end: '/vitastor/config0' } } ] }),
{ header: { revision: 2 }, succeeded: true, responses: [ { response_range: {
kvs: [ { key: '/vitastor/config/global', mod_revision: 2, value: { hello: 'world2' } } ],
} } ] }
@ -63,6 +63,7 @@ tests['read/write'] = async () =>
t.dump(false),
{"state":{"children":{"":{"children":{"vitastor":{"children":{"config":{"children":{"global":{"version":2,"mod_revision":2,"create_revision":1,"value":{"hello":"world2"}}}}}}}}}},"mod_revision":2,"leases":{}}
);
t.destroy();
};
tests['watch'] = async () =>
@ -71,7 +72,7 @@ tests['watch'] = async () =>
const sent = [];
const send = (event) => sent.push(event);
expect(
t.api_txn({ success: [ { request_put: { key: '/vitastor//config/global', value: { hello: 'world' } } } ] }),
await t.api_txn({ success: [ { request_put: { key: '/vitastor//config/global', value: { hello: 'world' } } } ] }),
{ header: { revision: 1 }, succeeded: true, responses: [ { response_put: {} } ] }
);
expect(
@ -80,10 +81,11 @@ tests['watch'] = async () =>
);
expect(sent, []);
expect(
t.api_txn({ success: [ { request_put: { key: '/vitastor/osd/state/1', value: { ip: '1.2.3.4' } } } ] }),
await t.api_txn({ success: [ { request_put: { key: '/vitastor/osd/state/1', value: { ip: '1.2.3.4' } } } ] }),
{ header: { revision: 2 }, succeeded: true, responses: [ { response_put: {} } ] }
);
expect(sent, [ { result: { header: { revision: 2 }, events: [ { type: 'PUT', kv: { key: '/vitastor/osd/state/1', value: { ip: '1.2.3.4' }, mod_revision: 2 } } ] } } ]);
t.destroy();
};
tests['lease'] = async () =>
@ -91,10 +93,10 @@ tests['lease'] = async () =>
const t = new EtcTree();
const sent = [];
const send = (event) => sent.push(event);
const leaseID = t.api_grant_lease({ TTL: 0.5 }).ID;
const leaseID = (await t.api_grant_lease({ TTL: 0.5 })).ID;
expect(leaseID != null, true);
expect(
t.api_txn({ success: [ { request_put: { key: '/vitastor/osd/state/1', lease: leaseID, value: { ip: '1.2.3.4' } } } ] }),
await t.api_txn({ success: [ { request_put: { key: '/vitastor/osd/state/1', lease: leaseID, value: { ip: '1.2.3.4' } } } ] }),
{ header: { revision: 1 }, succeeded: true, responses: [ { response_put: {} } ] }
);
expect(
@ -114,6 +116,57 @@ tests['lease'] = async () =>
t2.pause_leases();
t2.load(dump);
expect(t2.dump(false), dump);
t.destroy();
t2.destroy();
};
tests['update'] = async () =>
{
const t1 = new EtcTree();
const t2 = new EtcTree();
const leaseID = (await t1.api_grant_lease({ TTL: 0.5 })).ID;
expect(leaseID != null, true);
expect(
await t1.api_txn({ success: [ { request_put: { key: '/vitastor/osd/state/1', lease: leaseID, value: { ip: '1.2.3.4' } } } ] }),
{ header: { revision: 1 }, succeeded: true, responses: [ { response_put: {} } ] }
);
expect(
await t2.api_txn({ success: [ { request_put: { key: '/vitastor/osd/state/1', value: { ip: '1.2.3.6' } } } ] }),
{ header: { revision: 1 }, succeeded: true, responses: [ { response_put: {} } ] }
);
expect(
await t1.api_txn({ success: [ { request_put: { key: '/vitastor/osd/state/1', lease: leaseID, value: { ip: '1.2.3.5' } } } ] }),
{ header: { revision: 2 }, succeeded: true, responses: [ { response_put: {} } ] }
);
let dump2 = t2.dump();
t2.load(t1.dump(), true);
t1.load(dump2, true);
let dump = t2.dump(false);
let expires = dump.leases[leaseID].expires;
expect(dump, {"state":{"children":{"":{"children":{"vitastor":{"children":{"osd":{"children":{"state":{"children":{"1":{"lease":leaseID,"version":2,"mod_revision":2,"create_revision":1,"value":{"ip":"1.2.3.5"}}}}}}}}}}}},"mod_revision":2,"leases":{[leaseID]:{"ttl":0.5,"expires":expires}}});
expect(t1.dump(false), {"state":{"children":{"":{"children":{"vitastor":{"children":{"osd":{"children":{"state":{"children":{"1":{"lease":leaseID,"version":2,"mod_revision":2,"create_revision":1,"value":{"ip":"1.2.3.5"}}}}}}}}}}}},"mod_revision":2,"leases":{[leaseID]:{"ttl":0.5,"expires":expires}}});
t1.destroy();
t2.destroy();
};
tests['replicate watcher'] = async () =>
{
const t = new EtcTree();
t.set_replicate_watcher(async () =>
{
throw new Error('replication failed');
});
let thrown = false;
try
{
await t.api_txn({ success: [ { request_put: { key: '/vitastor/osd/state/1', value: { ip: '1.2.3.4' } } } ] });
}
catch (e)
{
thrown = e;
}
expect(thrown && thrown.message == 'replication failed', true);
t.destroy();
};
(async function()

View File

@ -238,7 +238,7 @@ class TinyRaft extends EventEmitter
if (this.state == CANDIDATE && this.term == msg.term && from == this.leader)
{
this.state = FOLLOWER;
this.emit('change', { state: this.state, term: this.term, leader: this.nodeId });
this.emit('change', { state: this.state, term: this.term, leader: this.leader });
}
if (this.state == FOLLOWER && from == this.leader)
{

View File

@ -200,7 +200,7 @@ async function testRestart()
checkQuorum(nodes, 5);
if (leaderChanges >= 3)
{
throw new Error("leaderChanges = "+leaderChanges+" (expected < 3)")
throw new Error("leaderChanges = "+leaderChanges+" (expected < 3)");
}
// Stop a follower
let restarted = 1 + (prevLeader % 5);