本文档主要介绍一个基于 jraft 的分布式计数器的例子。

场景

在多个节点(机器)组成的一个 raft group 中保存一个分布式计数器,该计数器可以递增和获取,并且在所有节点之间保持一致,任何少数节点的挂掉都不会影响对外提供的两个服务:

  1. incrmentAndGet(delta) 递增 delta 数值并返回递增后的值。
  2. get() 获取最新的值

RPC 请求

jraft 底层使用 bolt 作为通讯框架,定义两个请求

  • IncrementAndGetRequest,用于递增
  1. public class IncrementAndGetRequest implements Serializable {
  2. private static final long serialVersionUID = -5623664785560971849L;
  3. private long delta;
  4. public long getDelta() {
  5. return this.delta;
  6. }
  7. public void setDelta(long delta) {
  8. this.delta = delta;
  9. }
  10. }
  • GetValueRequest,用于获取最新值:
  1. public class GetValueRequest implements Serializable {
  2. private static final long serialVersionUID = 9218253805003988802L;
  3. public GetValueRequest() {
  4. super();
  5. }
  6. }

应答结果 ValueResponse,包括:

  1. success 是否成功
  2. value 成功情况下返回的最新值
  3. errorMsg 失败情况下的错误信息
  4. redirect 发生了重新选举,需要跳转的新的leader节点。
  1. public class ValueResponse implements Serializable {
  2. private static final long serialVersionUID = -4220017686727146773L;
  3. private long value;
  4. private boolean success;
  5. /**
  6. * redirect peer id
  7. */
  8. private String redirect;
  9. private String errorMsg;
  10. public String getErrorMsg() {
  11. return this.errorMsg;
  12. }
  13. public void setErrorMsg(String errorMsg) {
  14. this.errorMsg = errorMsg;
  15. }
  16. ......
  17. }
  • IncrementAndAddClosure 用于 Leader 服务端接收
  • IncrementAndGetRequest 请求后的回调处理:
  1. public class IncrementAndAddClosure implements Closure {
  2. private CounterServer counterServer;
  3. private IncrementAndGetRequest request;
  4. private ValueResponse response;
  5. private Closure done; // 网络应答callback
  6. public IncrementAndAddClosure(CounterServer counterServer, IncrementAndGetRequest request, ValueResponse response,
  7. Closure done) {
  8. super();
  9. this.counterServer = counterServer;
  10. this.request = request;
  11. this.response = response;
  12. this.done = done;
  13. }
  14. @Override
  15. public void run(Status status) {
  16. // 返回应答给客户端
  17. if (this.done != null) {
  18. done.run(status);
  19. }
  20. }
  21. public IncrementAndGetRequest getRequest() {
  22. return this.request;
  23. }
  24. public void setRequest(IncrementAndGetRequest request) {
  25. this.request = request;
  26. }
  27. public ValueResponse getResponse() {
  28. return this.response;
  29. }
  30. }

服务端

状态机 CounterStateMachine

首先持有一个初始值:

  1. public class CounterStateMachine extends StateMachineAdapter {
  2. /**
  3. * counter value
  4. */
  5. private AtomicLong value = new AtomicLong(0);

实现核心的 onApply(iterator) 方法,应用用户提交的请求到状态机:

  1. @Override
  2. public void onApply(Iterator iter) {
  3. // 遍历日志
  4. while (iter.hasNext()) {
  5. long delta = 0;
  6. IncrementAndAddClosure closure = null;
  7. // done 回调不为null,必须在应用日志后调用,如果不为 null,说明当前是leader。
  8. if (iter.done() != null) {
  9. // 当前是leader,可以直接从 IncrementAndAddClosure 中获取 delta,避免反序列化
  10. closure = (IncrementAndAddClosure) iter.done();
  11. delta = closure.getRequest().getDelta();
  12. } else {
  13. // 其他节点应用此日志,需要反序列化 IncrementAndGetRequest,获取 delta
  14. ByteBuffer data = iter.getData();
  15. try {
  16. IncrementAndGetRequest request = Codecs.getSerializer(Codecs.Hessian2).decode(data.array(),
  17. IncrementAndGetRequest.class.getName());
  18. delta = request.getDelta();
  19. } catch (CodecException e) {
  20. LOG.error("Fail to decode IncrementAndGetRequest", e);
  21. }
  22. }
  23. long prev = this.value.get();
  24. // 更新状态机
  25. long updated = value.addAndGet(delta);
  26. // 更新后,确保调用 done,返回应答给客户端。
  27. if (closure != null) {
  28. closure.getResponse().setValue(updated);
  29. closure.getResponse().setSuccess(true);
  30. closure.run(Status.OK());
  31. }
  32. LOG.info("Added value={} by delta={} at logIndex={}", prev, delta, iter.getIndex());
  33. iter.next();
  34. }
  35. }

CounterServer

启动一个 raft node节点,提供分布式计数器服务,内部使用 jraft 提供的 RaftGroupService 服务框架:

  1. public class CounterServer {
  2. // jraft 服务端服务框架
  3. private RaftGroupService raftGroupService;
  4. // raft 节点
  5. private Node node;
  6. // 业务状态机
  7. private CounterStateMachine fsm;
  8. public CounterServer(String dataPath, String groupId, PeerId serverId, NodeOptions nodeOptions) throws IOException {
  9. // 初始化路径
  10. FileUtils.forceMkdir(new File(dataPath));
  11. // 初始化全局定时器
  12. TimerManager.init(50);
  13. // 这里让 raft RPC 和业务 RPC 使用同一个 RPC server, 通常也可以分开.
  14. RpcServer rpcServer = new RpcServer(serverId.getPort());
  15. RaftRpcServerFactory.addRaftRequestProcessors(rpcServer);
  16. // 注册业务处理器
  17. rpcServer.registerUserProcessor(new GetValueRequestProcessor(this));
  18. rpcServer.registerUserProcessor(new IncrementAndGetRequestProcessor(this));
  19. // 初始化状态机
  20. this.fsm = new CounterStateMachine();
  21. // 设置状态机到启动参数
  22. nodeOptions.setFsm(this.fsm);
  23. // 设置存储路径
  24. // 日志, 必须
  25. nodeOptions.setLogUri(dataPath + File.separator + "log");
  26. // 元信息, 必须
  27. nodeOptions.setRaftMetaUri(dataPath + File.separator + "raft_meta");
  28. // snapshot, 可选, 一般都推荐
  29. nodeOptions.setSnapshotUri(dataPath + File.separator + "snapshot");
  30. // 初始化 raft group 服务框架
  31. this.raftGroupService = new RaftGroupService(groupId, serverId, nodeOptions, rpcServer);
  32. // 启动
  33. this.node = this.raftGroupService.start();
  34. }
  35. public CounterStateMachine getFsm() {
  36. return this.fsm;
  37. }
  38. public Node getNode() {
  39. return this.node;
  40. }
  41. public RaftGroupService RaftGroupService() {
  42. return this.raftGroupService;
  43. }
  44. /**
  45. * 生成重定向请求
  46. */
  47. public ValueResponse redirect() {
  48. ValueResponse response = new ValueResponse();
  49. response.setSuccess(false);
  50. if (node != null) {
  51. PeerId leader = node.getLeaderId();
  52. if (leader != null) {
  53. response.setRedirect(leader.toString());
  54. }
  55. }
  56. return response;
  57. }
  58. public static void main(String[] args) throws IOException {
  59. if (args.length != 4) {
  60. System.out
  61. .println("Useage : java com.alipay.jraft.example.counter.CounterServer {dataPath} {groupId} {serverId} {initConf}");
  62. System.out
  63. .println("Example: java com.alipay.jraft.example.counter.CounterServer /tmp/server1 counter 127.0.0.1:8081 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083");
  64. System.exit(1);
  65. }
  66. String dataPath = args[0];
  67. String groupId = args[1];
  68. String serverIdStr = args[2];
  69. String initConfStr = args[3];
  70. NodeOptions nodeOptions = new NodeOptions();
  71. // 为了测试, 调整 snapshot 间隔等参数
  72. nodeOptions.setElectionTimeoutMs(5000);
  73. nodeOptions.setDisableCli(false);
  74. nodeOptions.setSnapshotIntervalSecs(30);
  75. // 解析参数
  76. PeerId serverId = new PeerId();
  77. if (!serverId.parse(serverIdStr)) {
  78. throw new IllegalArgumentException("Fail to parse serverId:" + serverIdStr);
  79. }
  80. Configuration initConf = new Configuration();
  81. if (!initConf.parse(initConfStr)) {
  82. throw new IllegalArgumentException("Fail to parse initConf:" + initConfStr);
  83. }
  84. // 设置初始集群配置
  85. nodeOptions.setInitialConf(initConf);
  86. // 启动
  87. CounterServer counterServer = new CounterServer(dataPath, groupId, serverId, nodeOptions);
  88. System.out.println("Started counter server at port:"
  89. + counterServer.getNode().getNodeId().getPeerId().getPort());
  90. }
  91. }

启动三个节点的参数类似:

windows 用户请注意第一个参数的数据目录设置

  1. /tmp/server1 counter 127.0.0.1:8081 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083
  2. /tmp/server2 counter 127.0.0.1:8082 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083
  3. /tmp/server3 counter 127.0.0.1:8083 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083

分别为 server1/server2/server3三个目录,raft group名称为 counter,节点ip也分别为

  1. 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083

注册的网络请求处理器,我们看下 IncrementAndGetRequestProcessor 实现,一个普通的 bolt processor :

  1. public class IncrementAndGetRequestProcessor extends AsyncUserProcessor<IncrementAndGetRequest> {
  2. private static final Logger LOG = LoggerFactory.getLogger(IncrementAndGetRequestProcessor.class);
  3. private CounterServer counterServer;
  4. public IncrementAndGetRequestProcessor(CounterServer counterServer) {
  5. super();
  6. this.counterServer = counterServer;
  7. }
  8. @Override
  9. public void handleRequest(BizContext bizCtx, AsyncContext asyncCtx, IncrementAndGetRequest request) {
  10.      // 非leader,生成跳转请求
  11. if (!counterServer.getFsm().isLeader()) {
  12. asyncCtx.sendResponse(counterServer.redirect());
  13. return;
  14. }
  15. // 构建应答回调
  16. ValueResponse response = new ValueResponse();
  17. IncrementAndAddClosure closure = new IncrementAndAddClosure(counterServer, request, response, new Closure() {
  18. @Override
  19. public void run(Status status) {
  20. // 提交后处理
  21. if (!status.isOk()) {
  22. // 提交失败,返回错误信息
  23. response.setErrorMsg(status.getErrorMsg());
  24. response.setSuccess(false);
  25. }
  26. // 成功,返回ValueResponse应答
  27. asyncCtx.sendResponse(response);
  28. }
  29. });
  30. try {
  31. // 构建提交任务
  32. Task task = new Task();
  33. task.setDone(closure); // 设置回调
  34. // 填充数据,将请求用 hessian2序列化到 data 字段
  35. task.setData(ByteBuffer.wrap(Codecs.getSerializer(Codecs.Hessian2).encode(request)));
  36. // 提交到 raft group
  37. counterServer.getNode().apply(task);
  38. } catch (CodecException e) {
  39. // 处理序列化异常
  40. LOG.error("Fail to encode IncrementAndGetRequest", e);
  41. ValueResponse responseObject = response;
  42. responseObject.setSuccess(false);
  43. responseObject.setErrorMsg(e.getMessage());
  44. asyncCtx.sendResponse(responseObject);
  45. }
  46. }
  47. @Override
  48. public String interest() {
  49. return IncrementAndGetRequest.class.getName();
  50. }
  51. }

客户端

客户端 CounterClient 比较简单,主要使用 jraft 提供的 RouteTable 来刷新获取最新的 leader 节点,然后发送请求到 leader节点即可:

  1. public class CounterClient {
  2. public static void main(String[] args) throws Exception {
  3. if (args.length != 2) {
  4. System.out.println("Useage : java com.alipay.jraft.example.counter.CounterClient {groupId} {conf}");
  5. System.out
  6. .println("Example: java com.alipay.jraft.example.counter.CounterClient counter 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083");
  7. System.exit(1);
  8. }
  9. String groupId = args[0];
  10. String confStr = args[1];
  11. Configuration conf = new Configuration();
  12. if (!conf.parse(confStr)) {
  13. throw new IllegalArgumentException("Fail to parse conf:" + confStr);
  14. }
  15. // 更新raft group配置
  16. RouteTable.getInstance().updateConfiguration(groupId, conf);

接下来初始化 RPC 客户端并更新路由表:

  1. BoltCliClientService cliClientService = new BoltCliClientService();
  2. cliClientService.init(new CliOptions());
  3. if (!RouteTable.getInstance().refreshLeader(cliClientService, groupId, 1000).isOk()) {
  4. throw new IllegalStateException("Refresh leader failed");
  5. }

获取 leader 后发送请求:

  1. PeerId leader = RouteTable.getInstance().selectLeader(groupId);
  2. System.out.println("Leader is " + leader);
  3. int n = 1000;
  4. CountDownLatch latch = new CountDownLatch(n);
  5. long start = System.currentTimeMillis();
  6. for (int i = 0; i < n; i++) {
  7. incrementAndGet(cliClientService, leader, i, latch);
  8. }
  9. latch.await();
  10. System.out.println(n + " ops, cost : " + (System.currentTimeMillis() - start) + " ms.");
  11. System.exit(0);

incrementAndGet 方法实现比较简单了:

  1. private static void incrementAndGet(BoltCliClientService cliClientService, PeerId leader, long delta,
  2. CountDownLatch latch) throws RemotingException, InterruptedException {
  3. // 构建 IncrementAndGetRequest 请求并发送到 leader
  4. IncrementAndGetRequest request = new IncrementAndGetRequest();
  5. request.setDelta(delta);
  6. cliClientService.getRpcClient().invokeWithCallback(leader.getEndpoint().toString(), request,
  7. new InvokeCallback() {
  8. @Override
  9. public void onResponse(Object result) {
  10. latch.countDown();
  11. System.out.println("incrementAndGet result:" + result);
  12. }
  13. @Override
  14. public void onException(Throwable e) {
  15. e.printStackTrace();
  16. latch.countDown();
  17. }
  18. @Override
  19. public Executor getExecutor() {
  20. return null;
  21. }
  22. }, 5000);
  23. }

Snapshot 实现

为了避免每次节点重启的时候,重新应用一遍所有的日志,并且避免保存所有的日志,可以使用 snapshot 机制,也就是为状态机做一个 checkpoint,保存当时状态机的状态,删除在此之前的所有日志,核心是实现 StateMachine的两个方法:

  1. onSnapshotLoad,启动或者安装 snapshot 后加载 snapshot
  2. onSnapshotSave ,定期保存 snapshot

我们先为 Counter实现一个snapshot数据文件:

  1. public class CounterSnapshotFile {
  2. private static final Logger LOG = LoggerFactory.getLogger(CounterSnapshotFile.class);
  3. private String path;
  4. public CounterSnapshotFile(String path) {
  5. super();
  6. this.path = path;
  7. }
  8. public String getPath() {
  9. return this.path;
  10. }
  11. /**
  12. * Save value to snapshot file.
  13. * @param value
  14. * @return
  15. */
  16. public boolean save(long value) {
  17. try {
  18. FileUtils.writeStringToFile(new File(path), String.valueOf(value));
  19. return true;
  20. } catch (IOException e) {
  21. LOG.error("Fail to save snapshot", e);
  22. return false;
  23. }
  24. }
  25. public long load() throws IOException {
  26. String s = FileUtils.readFileToString(new File(path));
  27. if (!StringUtils.isBlank(s)) {
  28. return Long.parseLong(s);
  29. }
  30. throw new IOException("Fail to load snapshot from " + path + ",content: " + s);
  31. }
  32. }

保存到指定的 path

然后实现 StateMachine的两个方法:

  1. public boolean onSnapshotLoad(SnapshotReader reader) {
  2. // leader不用从 snapshot 加载,他不会接受 snapshot 安装请求
  3. if (isLeader()) {
  4. LOG.warn("Leader is not supposed to load snapshot");
  5. return false;
  6. }
  7. // 未找到数据文件,忽略
  8. if (reader.getFileMeta("data") == null) {
  9. LOG.error("Fail to find data file in {}", reader.getPath());
  10. return false;
  11. }
  12. // 将 snapshot 保存在 reader.getPath()/data 文件里
  13. CounterSnapshotFile snapshot = new CounterSnapshotFile(reader.getPath() + File.separator + "data");
  14. try {
  15. this.value.set(snapshot.load());
  16. return true;
  17. } catch (IOException e) {
  18. LOG.error("Fail to load snapshot from {}", snapshot.getPath());
  19. return false;
  20. }
  21. }
  22. public void onSnapshotSave(final SnapshotWriter writer, final Closure done) {
  23. // 获取此刻状态机状态
  24. final long currVal = this.value.get();
  25. // 异步保存,避免阻塞状态机
  26. Utils.runInThread(new Runnable() {
  27. @Override
  28. public void run() {
  29. CounterSnapshotFile snapshot = new CounterSnapshotFile(writer.getPath() + File.separator + "data");
  30. if (snapshot.save(currVal)) {
  31. if (writer.addFile("data")) {
  32. done.run(Status.OK());
  33. } else {
  34. done.run(new Status(RaftError.EIO, "Fail to add file to writer"));
  35. }
  36. } else {
  37. done.run(new Status(RaftError.EIO, "Fail to save counter snapshot %s", snapshot.getPath()));
  38. }
  39. }
  40. });
  41. }

snapshot 的间隔可以通过 NodeOptions 的 snapshotIntervalSecs 控制,默认一个小时。