public RaftPeerId getId() { return server.isDone()? server.join().getId(): null; }
static void startServers(Iterable<? extends RaftServer> servers) throws IOException { for(RaftServer s : servers) { s.start(); } }
@Override public GroupInfoReplyProto groupInfo(RpcController controller, GroupInfoRequestProto proto) throws ServiceException { final GroupInfoRequest request; try { request = ClientProtoUtils.toGroupInfoRequest(proto); final GroupInfoReply reply = impl.getGroupInfo(request); return ClientProtoUtils.toGroupInfoReplyProto(reply); } catch (IOException ioe) { throw new ServiceException(ioe); } } }
@Override public void onNext(AppendEntriesRequestProto request) { final CompletableFuture<Void> current = new CompletableFuture<>(); final CompletableFuture<Void> previous = previousOnNext.getAndSet(current); try { server.appendEntriesAsync(request).thenCombine(previous, (reply, v) -> { if (!isClosed.get()) { if (LOG.isDebugEnabled()) { LOG.debug(server.getId() + ": reply " + ServerProtoUtils.toString(reply)); } responseObserver.onNext(reply); } current.complete(null); return null; }); } catch (Throwable e) { GrpcUtil.warn(LOG, () -> getId() + ": Failed appendEntries " + ProtoUtils.toString(request.getServerRequest()), e); responseObserver.onError(GrpcUtil.wrapException(e, request.getServerRequest().getCallId())); current.completeExceptionally(e); } }
final RequestVoteRequestProto request = proto.getRequestVoteRequest(); rpcRequest = request.getServerRequest(); final RequestVoteReplyProto reply = server.requestVote(request); return RaftNettyServerReplyProto.newBuilder() .setRequestVoteReply(reply) final AppendEntriesRequestProto request = proto.getAppendEntriesRequest(); rpcRequest = request.getServerRequest(); final AppendEntriesReplyProto reply = server.appendEntries(request); return RaftNettyServerReplyProto.newBuilder() .setAppendEntriesReply(reply) final InstallSnapshotRequestProto request = proto.getInstallSnapshotRequest(); rpcRequest = request.getServerRequest(); final InstallSnapshotReplyProto reply = server.installSnapshot(request); return RaftNettyServerReplyProto.newBuilder() .setInstallSnapshotReply(reply) final RaftClientRequestProto request = proto.getRaftClientRequest(); rpcRequest = request.getRpcRequest(); final RaftClientReply reply = server.submitClientRequest( ClientProtoUtils.toRaftClientRequest(request)); return RaftNettyServerReplyProto.newBuilder() final SetConfigurationRequestProto request = proto.getSetConfigurationRequest(); rpcRequest = request.getRpcRequest(); final RaftClientReply reply = server.setConfiguration( ClientProtoUtils.toSetConfigurationRequest(request)); return RaftNettyServerReplyProto.newBuilder()
@Override public void run() throws Exception { RaftPeerId peerId = RaftPeerId.valueOf(id); RaftProperties properties = new RaftProperties(); RaftPeer[] peers = getPeers(); final int port = NetUtils.createSocketAddr(getPeer(peerId).getAddress()).getPort(); GrpcConfigKeys.Server.setPort(properties, port); properties.setInt(GrpcConfigKeys.OutputStream.RETRY_TIMES_KEY, Integer.MAX_VALUE); RaftServerConfigKeys.setStorageDir(properties, storageDir); StateMachine stateMachine = new ArithmeticStateMachine(); RaftGroup raftGroup = new RaftGroup(RaftGroupId.valueOf(ByteString.copyFromUtf8(raftGroupId)), peers); RaftServer raftServer = RaftServer.newBuilder() .setServerId(RaftPeerId.valueOf(id)) .setStateMachine(stateMachine).setProperties(properties) .setGroup(raftGroup) .build(); raftServer.start(); }
@Override public void run() throws Exception { RaftPeerId peerId = RaftPeerId.valueOf(id); RaftProperties properties = new RaftProperties(); RaftPeer[] peers = getPeers(); final int port = NetUtils.createSocketAddr(getPeer(peerId).getAddress()).getPort(); GrpcConfigKeys.Server.setPort(properties, port); properties.setInt(GrpcConfigKeys.OutputStream.RETRY_TIMES_KEY, Integer.MAX_VALUE); RaftServerConfigKeys.setStorageDirs(properties, Collections.singletonList(storageDir)); StateMachine stateMachine = new ArithmeticStateMachine(); final RaftGroup raftGroup = RaftGroup.valueOf(RaftGroupId.valueOf(ByteString.copyFromUtf8(raftGroupId)), peers); RaftServer raftServer = RaftServer.newBuilder() .setServerId(RaftPeerId.valueOf(id)) .setStateMachine(stateMachine).setProperties(properties) .setGroup(raftGroup) .build(); raftServer.start(); for(; raftServer.getLifeCycleState() != LifeCycle.State.CLOSED;) { TimeUnit.SECONDS.sleep(1); } }
private GrpcService(RaftServer server, GrpcTlsConfig tlsConfig) { this(server, server::getId, GrpcConfigKeys.Server.port(server.getProperties()), GrpcConfigKeys.messageSizeMax(server.getProperties(), LOG::info), RaftServerConfigKeys.Log.Appender.bufferByteLimit(server.getProperties()), GrpcConfigKeys.flowControlWindow(server.getProperties(), LOG::info), RaftServerConfigKeys.Rpc.requestTimeout(server.getProperties()), tlsConfig); }
@Override public GroupListReplyProto groupList( RpcController controller, GroupListRequestProto proto) throws ServiceException { final GroupListRequest request; try { request = ClientProtoUtils.toGroupListRequest(proto); final GroupListReply reply = impl.getGroupList(request); return ClientProtoUtils.toGroupListReplyProto(reply); } catch (IOException ioe) { throw new ServiceException(ioe); } }
@Override public RaftClientReplyProto groupManagement(RpcController controller, GroupManagementRequestProto proto) throws ServiceException { final GroupManagementRequest request; try { request = ClientProtoUtils.toGroupManagementRequest(proto); final RaftClientReply reply = impl.groupManagement(request); return ClientProtoUtils.toRaftClientReplyProto(reply); } catch(IOException ioe) { throw new ServiceException(ioe); } }
@Override public RaftClientReplyProto setConfiguration( RpcController unused, SetConfigurationRequestProto proto) throws ServiceException { final SetConfigurationRequest request; try { request = ClientProtoUtils.toSetConfigurationRequest(proto); final RaftClientReply reply = impl.setConfiguration(request); return ClientProtoUtils.toRaftClientReplyProto(reply); } catch(IOException ioe) { throw new ServiceException(ioe); } }
@Override public void onNext(InstallSnapshotRequestProto request) { try { final InstallSnapshotReplyProto reply = server.installSnapshot(request); responseObserver.onNext(reply); } catch (Throwable e) { GrpcUtil.warn(LOG, () -> getId() + ": Failed installSnapshot " + ProtoUtils.toString(request.getServerRequest()), e); responseObserver.onError(GrpcUtil.wrapException(e)); } }
@Override public void requestVote(RequestVoteRequestProto request, StreamObserver<RequestVoteReplyProto> responseObserver) { try { final RequestVoteReplyProto reply = server.requestVote(request); responseObserver.onNext(reply); responseObserver.onCompleted(); } catch (Throwable e) { GrpcUtil.warn(LOG, () -> getId() + ": Failed requestVote " + ProtoUtils.toString(request.getServerRequest()), e); responseObserver.onError(GrpcUtil.wrapException(e)); } }
public void start() throws IOException { final ServerOpts opts = getServerOpts(); if (opts.getHost() == null) { opts.setHost(LogServiceUtils.getHostName()); } this.lifeCycle = new LifeCycle(this.id); RaftProperties properties = new RaftProperties(); if(opts.getWorkingDir() != null) { RaftServerConfigKeys.setStorageDirs(properties, Collections.singletonList(new File(opts.getWorkingDir()))); } GrpcConfigKeys.Server.setPort(properties, opts.getPort()); NettyConfigKeys.Server.setPort(properties, opts.getPort()); Set<RaftPeer> peers = getPeersFromQuorum(opts.getMetaQuorum()); RaftGroupId raftMetaGroupId = RaftGroupId.valueOf(opts.getMetaGroupId()); RaftGroup metaGroup = RaftGroup.valueOf(raftMetaGroupId, peers); metaStateMachine = new MetaStateMachine(raftMetaGroupId, RaftGroupId.valueOf(opts.getLogServerGroupId())); server = RaftServer.newBuilder() .setGroup(metaGroup) .setServerId(RaftPeerId.valueOf(id)) .setStateMachineRegistry(raftGroupId -> { if(raftGroupId.equals(META_GROUP_ID)) { return metaStateMachine; } return null; }) .setProperties(properties).build(); lifeCycle.startAndTransition(() -> { server.start(); }, IOException.class); }
@Override public void run() throws Exception { RaftPeerId peerId = RaftPeerId.valueOf(id); RaftProperties properties = new RaftProperties(); RaftPeer[] peers = getPeers(); final int port = NetUtils.createSocketAddr(getPeer(peerId).getAddress()).getPort(); GrpcConfigKeys.Server.setPort(properties, port); properties.setInt(GrpcConfigKeys.OutputStream.RETRY_TIMES_KEY, Integer.MAX_VALUE); RaftServerConfigKeys.setStorageDirs(properties, Collections.singletonList(storageDir)); ConfUtils.setFile(properties::setFile, FileStoreCommon.STATEMACHINE_DIR_KEY, storageDir); StateMachine stateMachine = new FileStoreStateMachine(properties); final RaftGroup raftGroup = RaftGroup.valueOf(RaftGroupId.valueOf(ByteString.copyFromUtf8(raftGroupId)), peers); RaftServer raftServer = RaftServer.newBuilder() .setServerId(RaftPeerId.valueOf(id)) .setStateMachine(stateMachine).setProperties(properties) .setGroup(raftGroup) .build(); raftServer.start(); for(; raftServer.getLifeCycleState() != LifeCycle.State.CLOSED;) { TimeUnit.SECONDS.sleep(1); } }
/** Constructs a netty server with the given port. */ private NettyRpcService(RaftServer server) { super(server::getId, id -> new NettyRpcProxy.PeerMap(id.toString())); this.server = server; final ChannelInitializer<SocketChannel> initializer = new ChannelInitializer<SocketChannel>() { @Override protected void initChannel(SocketChannel ch) throws Exception { final ChannelPipeline p = ch.pipeline(); p.addLast(new ProtobufVarint32FrameDecoder()); p.addLast(new ProtobufDecoder(RaftNettyServerRequestProto.getDefaultInstance())); p.addLast(new ProtobufVarint32LengthFieldPrepender()); p.addLast(new ProtobufEncoder()); p.addLast(new InboundHandler()); } }; final int port = NettyConfigKeys.Server.port(server.getProperties()); channelFuture = new ServerBootstrap() .group(bossGroup, workerGroup) .channel(NioServerSocketChannel.class) .handler(new LoggingHandler(LogLevel.INFO)) .childHandler(initializer) .bind(port); }
RaftGroup all = RaftGroup.valueOf(logServerGroupId, peer); RaftGroup meta = RaftGroup.valueOf(RaftGroupId.valueOf(opts.getMetaGroupId()), peers); raftServer = RaftServer.newBuilder() .setStateMachineRegistry(new StateMachine.Registry() { private final StateMachine managementMachine = new ManagementStateMachine(); .setGroup(all) .build(); raftServer.start();
static <SERVER extends RaftServer> void blockQueueAndSetDelay( Collection<SERVER> servers, DelayLocalExecutionInjection injection, String leaderId, int delayMs, long maxTimeout) throws InterruptedException { // block reqeusts sent to leader if delayMs > 0 final boolean block = delayMs > 0; LOG.debug("{} requests sent to leader {} and set {}ms delay for the others", block? "Block": "Unblock", leaderId, delayMs); if (block) { BlockRequestHandlingInjection.getInstance().blockReplier(leaderId); } else { BlockRequestHandlingInjection.getInstance().unblockReplier(leaderId); } // delay RaftServerRequest for other servers servers.stream().filter(s -> !s.getId().toString().equals(leaderId)) .forEach(s -> { if (block) { injection.setDelayMs(s.getId().toString(), delayMs); } else { injection.removeDelay(s.getId().toString()); } }); Thread.sleep(3 * maxTimeout); }
protected void startServer(RaftServer server, boolean startService) { if (startService) { server.start(); } }
SimulatedServerRpc(RaftServer server, SimulatedRequestReply<RaftServerRequest, RaftServerReply> serverRequestReply, SimulatedRequestReply<RaftClientRequest, RaftClientReply> clientRequestReply) { this.server = (RaftServerProxy)server; final Supplier<String> id = () -> server.getId().toString(); this.serverHandler = new RequestHandler<>(id, "serverHandler", serverRequestReply, serverHandlerImpl, 3); this.clientHandler = new RequestHandler<>(id, "clientHandler", clientRequestReply, clientHandlerImpl, 3); }