diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/RaftGroupService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/RaftGroupService.java index 2d3e09b8a..df9f9d1be 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/RaftGroupService.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/RaftGroupService.java @@ -20,12 +20,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.option.NodeOptions; import com.alipay.sofa.jraft.option.RpcOptions; import com.alipay.sofa.jraft.rpc.ProtobufMsgFactory; import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; import com.alipay.sofa.jraft.util.Endpoint; import com.alipay.sofa.jraft.util.Utils; @@ -128,7 +128,7 @@ public synchronized Node start(final boolean startRpcServer) { this.node = RaftServiceFactory.createAndInitRaftNode(this.groupId, this.serverId, this.nodeOptions); if (startRpcServer) { - this.rpcServer.startup(); + this.rpcServer.init(null); } else { LOG.warn("RPC server is not started in RaftGroupService."); } @@ -249,7 +249,7 @@ public void setRpcServer(final RpcServer rpcServer) { if (this.serverId == null) { throw new IllegalStateException("Please set serverId at first"); } - if (rpcServer.port() != this.serverId.getPort()) { + if (rpcServer.boundPort() != this.serverId.getPort()) { throw new IllegalArgumentException("RPC server port mismatch"); } this.rpcServer = rpcServer; diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java index bdb855431..0d299f116 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java @@ -57,7 +57,7 @@ import com.alipay.sofa.jraft.rpc.CliRequests.SnapshotRequest; import com.alipay.sofa.jraft.rpc.CliRequests.TransferLeaderRequest; import com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse; -import com.alipay.sofa.jraft.rpc.impl.cli.BoltCliClientService; +import com.alipay.sofa.jraft.rpc.impl.cli.CliClientServiceImpl; import com.alipay.sofa.jraft.util.Requires; import com.alipay.sofa.jraft.util.Utils; import com.google.protobuf.Message; @@ -67,8 +67,7 @@ * Cli service implementation. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 4:12:06 PM + * @author jiachun.fjc */ public class CliServiceImpl implements CliService { @@ -85,7 +84,7 @@ public synchronized boolean init(final CliOptions opts) { return true; } this.cliOptions = opts; - this.cliClientService = new BoltCliClientService(); + this.cliClientService = new CliClientServiceImpl(); return this.cliClientService.init(this.cliOptions); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/error/InvokeTimeoutException.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/error/InvokeTimeoutException.java new file mode 100644 index 000000000..68dba43f5 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/error/InvokeTimeoutException.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.error; + +/** + * @author jiachun.fjc + */ +public class InvokeTimeoutException extends RemotingException { + + private static final long serialVersionUID = -4710810309766380565L; + + public InvokeTimeoutException() { + } + + public InvokeTimeoutException(String message) { + super(message); + } + + public InvokeTimeoutException(String message, Throwable cause) { + super(message, cause); + } + + public InvokeTimeoutException(Throwable cause) { + super(cause); + } + + public InvokeTimeoutException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/error/RemotingException.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/error/RemotingException.java new file mode 100644 index 000000000..29381ac46 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/error/RemotingException.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.error; + +/** + * Exception for default remoting problems. + * + * @author jiachun.fjc + */ +public class RemotingException extends Exception { + + private static final long serialVersionUID = -6326244159775972292L; + + public RemotingException() { + } + + public RemotingException(String message) { + super(message); + } + + public RemotingException(String message, Throwable cause) { + super(message, cause); + } + + public RemotingException(Throwable cause) { + super(cause); + } + + public RemotingException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/ClientService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/ClientService.java index ef3a4a857..86c6df7d9 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/ClientService.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/ClientService.java @@ -38,7 +38,7 @@ public interface ClientService extends Lifecycle { * @param endpoint server address * @return true on connect success */ - boolean connect(Endpoint endpoint); + boolean connect(final Endpoint endpoint); /** * Disconnect from endpoint. @@ -46,7 +46,7 @@ public interface ClientService extends Lifecycle { * @param endpoint server address * @return true on disconnect success */ - boolean disconnect(Endpoint endpoint); + boolean disconnect(final Endpoint endpoint); /** * Returns true when the endpoint's connection is active. @@ -54,7 +54,7 @@ public interface ClientService extends Lifecycle { * @param endpoint server address * @return true on connection is active */ - boolean isConnected(Endpoint endpoint); + boolean isConnected(final Endpoint endpoint); /** * Send a requests and waits for response with callback, returns the request future. @@ -65,6 +65,6 @@ public interface ClientService extends Lifecycle { * @param timeoutMs timeout millis * @return a future with operation result */ - Future invokeWithDone(Endpoint endpoint, Message request, RpcResponseClosure done, - int timeoutMs); + Future invokeWithDone(final Endpoint endpoint, final Message request, + final RpcResponseClosure done, final int timeoutMs); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/Connection.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/Connection.java new file mode 100644 index 000000000..74d9bdbf2 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/Connection.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc; + +/** + * @author jiachun.fjc + */ +public interface Connection { + + /** + * Get the attribute that bound to the connection. + * + * @param key the attribute key + * @return the attribute value + */ + Object getAttribute(final String key); + + /** + * Set the attribute to the connection. + * + * @param key the attribute key + * @param value the attribute value + */ + void setAttribute(final String key, final Object value); + + /** + * Close the connection. + */ + void close(); +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/InvokeCallback.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/InvokeCallback.java new file mode 100644 index 000000000..d760aabf8 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/InvokeCallback.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc; + +import java.util.concurrent.Executor; + +/** + * @author jiachun.fjc + */ +public interface InvokeCallback { + + void complete(final Object result, final Throwable err); + + Executor executor(); +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/InvokeContext.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/InvokeContext.java new file mode 100644 index 000000000..06ad64ee8 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/InvokeContext.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** + * RPC invoke context. + * + * @author jiachun.fjc + */ +public class InvokeContext { + + public final static String CRC_SWITCH = "invoke.crc.switch"; + + private final ConcurrentMap ctx = new ConcurrentHashMap<>(); + + public Object put(final String key, final Object value) { + return this.ctx.put(key, value); + } + + @SuppressWarnings("unchecked") + public T get(final String key) { + return (T) this.ctx.get(key); + } + + @SuppressWarnings("unchecked") + public T getOrDefault(final String key, final T defaultValue) { + return (T) this.ctx.getOrDefault(key, defaultValue); + } + + public void clear() { + this.ctx.clear(); + } + + public Set> entrySet() { + return this.ctx.entrySet(); + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/ProtobufMsgFactory.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/ProtobufMsgFactory.java index 2febdc135..003b13320 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/ProtobufMsgFactory.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/ProtobufMsgFactory.java @@ -25,9 +25,9 @@ import org.apache.commons.lang.SerializationException; -import com.alipay.remoting.CustomSerializerManager; import com.alipay.sofa.jraft.error.MessageClassNotFoundException; import com.alipay.sofa.jraft.storage.io.ProtoBufFile; +import com.alipay.sofa.jraft.util.RpcFactoryHelper; import com.google.protobuf.DescriptorProtos.FileDescriptorProto; import com.google.protobuf.DescriptorProtos.FileDescriptorSet; import com.google.protobuf.Descriptors.Descriptor; @@ -50,31 +50,30 @@ public class ProtobufMsgFactory { static { try { - FileDescriptorSet descriptorSet = FileDescriptorSet.parseFrom(ProtoBufFile.class + final FileDescriptorSet descriptorSet = FileDescriptorSet.parseFrom(ProtoBufFile.class .getResourceAsStream("/raft.desc")); - List resolveFDs = new ArrayList<>(); - for (FileDescriptorProto fdp : descriptorSet.getFileList()) { + final List resolveFDs = new ArrayList<>(); + for (final FileDescriptorProto fdp : descriptorSet.getFileList()) { - FileDescriptor[] dependencies = new FileDescriptor[resolveFDs.size()]; + final FileDescriptor[] dependencies = new FileDescriptor[resolveFDs.size()]; resolveFDs.toArray(dependencies); - FileDescriptor fd = FileDescriptor.buildFrom(fdp, dependencies); + final FileDescriptor fd = FileDescriptor.buildFrom(fdp, dependencies); resolveFDs.add(fd); - for (Descriptor descriptor : fd.getMessageTypes()) { + for (final Descriptor descriptor : fd.getMessageTypes()) { - String className = fdp.getOptions().getJavaPackage() + "." - + fdp.getOptions().getJavaOuterClassname() + "$" + descriptor.getName(); - Class clazz = Class.forName(className); - MethodHandle methodHandle = MethodHandles.lookup().findStatic(clazz, "parseFrom", + final String className = fdp.getOptions().getJavaPackage() + "." + + fdp.getOptions().getJavaOuterClassname() + "$" + descriptor.getName(); + final Class clazz = Class.forName(className); + final MethodHandle methodHandle = MethodHandles.lookup().findStatic(clazz, "parseFrom", methodType(clazz, byte[].class)); PARSE_METHODS_4PROTO.put(descriptor.getFullName(), methodHandle); PARSE_METHODS_4J.put(className, methodHandle); - CustomSerializerManager.registerCustomSerializer(className, ProtobufSerializer.INSTANCE); + RpcFactoryHelper.getRpcFactory().registerProtobufSerializer(className); } } - - } catch (Exception e) { + } catch (final Exception e) { e.printStackTrace(); // NOPMD } } @@ -86,8 +85,8 @@ public static void load() { } @SuppressWarnings("unchecked") - public static T newMessageByJavaClassName(String className, byte[] bs) { - MethodHandle handle = PARSE_METHODS_4J.get(className); + public static T newMessageByJavaClassName(final String className, final byte[] bs) { + final MethodHandle handle = PARSE_METHODS_4J.get(className); if (handle == null) { throw new MessageClassNotFoundException(className + " not found"); } @@ -99,8 +98,8 @@ public static T newMessageByJavaClassName(String className, } @SuppressWarnings("unchecked") - public static T newMessageByProtoClassName(String className, byte[] bs) { - MethodHandle handle = PARSE_METHODS_4PROTO.get(className); + public static T newMessageByProtoClassName(final String className, final byte[] bs) { + final MethodHandle handle = PARSE_METHODS_4PROTO.get(className); if (handle == null) { throw new MessageClassNotFoundException(className + " not found"); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcFactory.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcFactory.java new file mode 100644 index 000000000..6f8286f86 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcFactory.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc; + +import com.alipay.sofa.jraft.option.RpcOptions; +import com.alipay.sofa.jraft.util.Endpoint; + +/** + * Raft RPC service factory. + * + * @author jiachun.fjc + */ +public interface RaftRpcFactory { + + /** + * Register serializer with class name. + * + * @param className class name + */ + void registerProtobufSerializer(final String className); + + /** + * Creates a raft RPC client. + * + * @return a new rpc client instance + */ + default RpcClient createRpcClient() { + return createRpcClient(null); + } + + /** + * Creates a raft RPC client. + * + * @param helper config helper for rpc client impl + * @return a new rpc client instance + */ + RpcClient createRpcClient(final ConfigHelper helper); + + /** + * Creates a raft RPC server. + * + * @param endpoint server address to bind + * @return a new rpc server instance + */ + default RpcServer createRpcServer(final Endpoint endpoint) { + return createRpcServer(endpoint, null); + } + + /** + * Creates a raft RPC server. + * + * @param endpoint server address to bind + * @param helper config helper for rpc server impl + * @return a new rpc server instance + */ + RpcServer createRpcServer(final Endpoint endpoint, final ConfigHelper helper); + + @SuppressWarnings("unused") + default ConfigHelper defaultJRaftClientConfigHelper(final RpcOptions opts) { + return null; + } + + @SuppressWarnings("unused") + default ConfigHelper defaultJRaftServerConfigHelper(final RpcOptions opts) { + return null; + } + + interface ConfigHelper { + + void config(final T instance); + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcServerFactory.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcServerFactory.java index c6ab480c4..7cf41e134 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcServerFactory.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcServerFactory.java @@ -18,11 +18,6 @@ import java.util.concurrent.Executor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.alipay.remoting.ConnectionEventType; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.rpc.impl.PingRequestProcessor; import com.alipay.sofa.jraft.rpc.impl.cli.AddLearnersRequestProcessor; import com.alipay.sofa.jraft.rpc.impl.cli.AddPeerRequestProcessor; @@ -42,18 +37,16 @@ import com.alipay.sofa.jraft.rpc.impl.core.RequestVoteRequestProcessor; import com.alipay.sofa.jraft.rpc.impl.core.TimeoutNowRequestProcessor; import com.alipay.sofa.jraft.util.Endpoint; +import com.alipay.sofa.jraft.util.RpcFactoryHelper; /** * Raft RPC server factory. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-08 7:56:40 PM + * @author jiachun.fjc */ public class RaftRpcServerFactory { - public static final Logger LOG = LoggerFactory.getLogger(RaftRpcServerFactory.class); - /** * Creates a raft RPC server with default request executors. * @@ -74,7 +67,7 @@ public static RpcServer createRaftRpcServer(final Endpoint endpoint) { */ public static RpcServer createRaftRpcServer(final Endpoint endpoint, final Executor raftExecutor, final Executor cliExecutor) { - final RpcServer rpcServer = new RpcServer(endpoint.getPort(), true, true); + final RpcServer rpcServer = RpcFactoryHelper.getRpcFactory().createRpcServer(endpoint); addRaftRequestProcessors(rpcServer, raftExecutor, cliExecutor); return rpcServer; } @@ -91,35 +84,35 @@ public static void addRaftRequestProcessors(final RpcServer rpcServer) { /** * Adds RAFT and CLI service request processors. * - * @param rpcServer rpc server instance - * @param raftExecutor executor to handle RAFT requests. - * @param cliExecutor executor to handle CLI service requests. + * @param rpcServer rpc server instance + * @param raftExecutor executor to handle RAFT requests. + * @param cliExecutor executor to handle CLI service requests. */ public static void addRaftRequestProcessors(final RpcServer rpcServer, final Executor raftExecutor, final Executor cliExecutor) { // raft core processors final AppendEntriesRequestProcessor appendEntriesRequestProcessor = new AppendEntriesRequestProcessor( raftExecutor); - rpcServer.addConnectionEventProcessor(ConnectionEventType.CLOSE, appendEntriesRequestProcessor); - rpcServer.registerUserProcessor(appendEntriesRequestProcessor); - rpcServer.registerUserProcessor(new GetFileRequestProcessor(raftExecutor)); - rpcServer.registerUserProcessor(new InstallSnapshotRequestProcessor(raftExecutor)); - rpcServer.registerUserProcessor(new RequestVoteRequestProcessor(raftExecutor)); - rpcServer.registerUserProcessor(new PingRequestProcessor()); - rpcServer.registerUserProcessor(new TimeoutNowRequestProcessor(raftExecutor)); - rpcServer.registerUserProcessor(new ReadIndexRequestProcessor(raftExecutor)); + rpcServer.registerConnectionClosedEventListener(appendEntriesRequestProcessor); + rpcServer.registerProcessor(appendEntriesRequestProcessor); + rpcServer.registerProcessor(new GetFileRequestProcessor(raftExecutor)); + rpcServer.registerProcessor(new InstallSnapshotRequestProcessor(raftExecutor)); + rpcServer.registerProcessor(new RequestVoteRequestProcessor(raftExecutor)); + rpcServer.registerProcessor(new PingRequestProcessor()); + rpcServer.registerProcessor(new TimeoutNowRequestProcessor(raftExecutor)); + rpcServer.registerProcessor(new ReadIndexRequestProcessor(raftExecutor)); // raft cli service - rpcServer.registerUserProcessor(new AddPeerRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new RemovePeerRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new ResetPeerRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new ChangePeersRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new GetLeaderRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new SnapshotRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new TransferLeaderRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new GetPeersRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new AddLearnersRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new RemoveLearnersRequestProcessor(cliExecutor)); - rpcServer.registerUserProcessor(new ResetLearnersRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new AddPeerRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new RemovePeerRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new ResetPeerRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new ChangePeersRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new GetLeaderRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new SnapshotRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new TransferLeaderRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new GetPeersRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new AddLearnersRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new RemoveLearnersRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new ResetLearnersRequestProcessor(cliExecutor)); } /** @@ -135,15 +128,15 @@ public static RpcServer createAndStartRaftRpcServer(final Endpoint endpoint) { /** * Creates a raft RPC server and starts it. * - * @param endpoint server address to bind - * @param raftExecutor executor to handle RAFT requests. - * @param cliExecutor executor to handle CLI service requests. + * @param endpoint server address to bind + * @param raftExecutor executor to handle RAFT requests. + * @param cliExecutor executor to handle CLI service requests. * @return a rpc server instance */ public static RpcServer createAndStartRaftRpcServer(final Endpoint endpoint, final Executor raftExecutor, final Executor cliExecutor) { final RpcServer server = createRaftRpcServer(endpoint, raftExecutor, cliExecutor); - server.startup(); + server.init(null); return server; } } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcClient.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcClient.java new file mode 100644 index 000000000..6a57d9636 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcClient.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc; + +import com.alipay.sofa.jraft.Lifecycle; +import com.alipay.sofa.jraft.ReplicatorGroup; +import com.alipay.sofa.jraft.error.RemotingException; +import com.alipay.sofa.jraft.option.RpcOptions; +import com.alipay.sofa.jraft.util.Endpoint; + +/** + * + * @author jiachun.fjc + */ +public interface RpcClient extends Lifecycle { + + /** + * Check connection for given address. + * + * @param endpoint target address + * @return true if there is a connection adn the connection is active adn writable. + */ + boolean checkConnection(final Endpoint endpoint); + + /** + * Close all connections of a address. + * + * @param endpoint target address + */ + void closeConnection(final Endpoint endpoint); + + /** + * Register a connect event listener for the replicator group. + * + * @param replicatorGroup replicator group + */ + void registerConnectEventListener(final ReplicatorGroup replicatorGroup); + + /** + * Synchronous invocation. + * + * @param endpoint target address + * @param request request object + * @param timeoutMs timeout millisecond + * @return invoke result + */ + default Object invokeSync(final Endpoint endpoint, final Object request, final long timeoutMs) + throws InterruptedException, RemotingException { + return invokeSync(endpoint, request, null, timeoutMs); + } + + /** + * Synchronous invocation using a invoke context. + * + * @param endpoint target address + * @param request request object + * @param ctx invoke context + * @param timeoutMs timeout millisecond + * @return invoke result + */ + Object invokeSync(final Endpoint endpoint, final Object request, final InvokeContext ctx, + final long timeoutMs) throws InterruptedException, RemotingException; + + /** + * Asynchronous invocation with a callback. + * + * @param endpoint target address + * @param request request object + * @param callback invoke callback + * @param timeoutMs timeout millisecond + */ + default void invokeAsync(final Endpoint endpoint, final Object request, final InvokeCallback callback, + final long timeoutMs) throws InterruptedException, RemotingException { + invokeAsync(endpoint, request, null, callback, timeoutMs); + } + + /** + * Asynchronous invocation with a callback. + * + * @param endpoint target address + * @param request request object + * @param ctx invoke context + * @param callback invoke callback + * @param timeoutMs timeout millisecond + */ + void invokeAsync(final Endpoint endpoint, final Object request, final InvokeContext ctx, final InvokeCallback callback, + final long timeoutMs) throws InterruptedException, RemotingException; +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcContext.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcContext.java new file mode 100644 index 000000000..bfb54a7ec --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcContext.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc; + +/** + * @author jiachun.fjc + */ +public interface RpcContext { + + /** + * Send a response back. + * + * @param responseObj the response object + */ + void sendResponse(final Object responseObj); + + /** + * Get current connection. + * + * @return current connection + */ + Connection getConnection(); + + /** + * GFet the remote address. + * + * @return remote address + */ + String getRemoteAddress(); +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcProcessor.java new file mode 100644 index 000000000..fdd8a5e08 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcProcessor.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc; + +import java.util.concurrent.Executor; + +/** + * Defined functions for process user defined request. + * + * @author jiachun.fjc + */ +public interface RpcProcessor { + + /** + * Async to handle request with {@link RpcContext}. + * + * @param rpcCtx the rpc context + * @param request the request + */ + void handleRequest(final RpcContext rpcCtx, final T request); + + /** + * The class name of user request. + * Use String type to avoid loading class. + * + * @return interested request's class name + */ + String interest(); + + /** + * Get user's executor. + * + * @return executor + */ + default Executor executor() { + return null; + } + + /** + * + * @return the executor selector + */ + default ExecutorSelector executorSelector() { + return null; + } + + /** + * Executor selector interface. + */ + interface ExecutorSelector { + + /** + * Select a executor. + * + * @param reqClass request class name + * @param reqHeader request header + * @return a executor + */ + Executor select(final String reqClass, final Object reqHeader); + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcRequestClosure.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcRequestClosure.java index 14c493db1..a71858fc8 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcRequestClosure.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcRequestClosure.java @@ -16,8 +16,6 @@ */ package com.alipay.sofa.jraft.rpc; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; import com.alipay.sofa.jraft.Closure; import com.alipay.sofa.jraft.Status; import com.google.protobuf.Message; @@ -26,40 +24,33 @@ * RPC request Closure encapsulates the RPC contexts. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Mar-28 4:55:24 PM + * @author jiachun.fjc */ public class RpcRequestClosure implements Closure { - private final BizContext bizContext; - private final AsyncContext asyncContext; - private boolean respond; + private final RpcContext rpcCtx; + private boolean respond; - public RpcRequestClosure(BizContext bizContext, AsyncContext asyncContext) { + public RpcRequestClosure(RpcContext rpcCtx) { super(); - this.bizContext = bizContext; - this.asyncContext = asyncContext; + this.rpcCtx = rpcCtx; this.respond = false; } - public BizContext getBizContext() { - return this.bizContext; - } - - public AsyncContext getAsyncContext() { - return this.asyncContext; + public RpcContext getRpcCtx() { + return rpcCtx; } - public synchronized void sendResponse(Message msg) { + public synchronized void sendResponse(final Message msg) { if (this.respond) { return; } - this.asyncContext.sendResponse(msg); + this.rpcCtx.sendResponse(msg); this.respond = true; } @Override - public void run(Status status) { + public void run(final Status status) { sendResponse(RpcResponseFactory.newResponse(status)); } } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcRequestProcessor.java index 9a69d4162..2bce7691c 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcRequestProcessor.java @@ -21,26 +21,23 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.rpc.protocol.AsyncUserProcessor; import com.google.protobuf.Message; /** * Abstract AsyncUserProcessor for RPC processors. * - * @author boyan (boyan@alibaba-inc.com) + * @param Message * - * 2018-Apr-08 5:55:39 PM - * @param + * @author boyan (boyan@alibaba-inc.com) + * @author jiachun.fjc */ -public abstract class RpcRequestProcessor extends AsyncUserProcessor { +public abstract class RpcRequestProcessor implements RpcProcessor { protected static final Logger LOG = LoggerFactory.getLogger(RpcRequestProcessor.class); private final Executor executor; - public abstract Message processRequest(T request, RpcRequestClosure done); + public abstract Message processRequest(final T request, final RpcRequestClosure done); public RpcRequestProcessor(Executor executor) { super(); @@ -48,20 +45,20 @@ public RpcRequestProcessor(Executor executor) { } @Override - public void handleRequest(BizContext bizCtx, AsyncContext asyncCtx, T request) { + public void handleRequest(final RpcContext rpcCtx, final T request) { try { - final Message msg = this.processRequest(request, new RpcRequestClosure(bizCtx, asyncCtx)); + final Message msg = processRequest(request, new RpcRequestClosure(rpcCtx)); if (msg != null) { - asyncCtx.sendResponse(msg); + rpcCtx.sendResponse(msg); } } catch (final Throwable t) { LOG.error("handleRequest {} failed", request, t); - asyncCtx.sendResponse(RpcResponseFactory.newResponse(-1, "handleRequest internal error")); + rpcCtx.sendResponse(RpcResponseFactory.newResponse(-1, "handleRequest internal error")); } } @Override - public Executor getExecutor() { - return executor; + public Executor executor() { + return this.executor; } } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcServer.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcServer.java new file mode 100644 index 000000000..98407776b --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RpcServer.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc; + +import com.alipay.sofa.jraft.Lifecycle; +import com.alipay.sofa.jraft.rpc.impl.ConnectionClosedEventListener; + +/** + * + * @author jiachun.fjc + */ +public interface RpcServer extends Lifecycle { + + /** + * Register a conn closed event listener. + * + * @param listener the event listener. + */ + void registerConnectionClosedEventListener(final ConnectionClosedEventListener listener); + + /** + * Register user processor. + * + * @param processor the user processor which has a interest + */ + void registerProcessor(final RpcProcessor processor); + + /** + * + * @return bound port + */ + int boundPort(); +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/AbstractBoltClientService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/AbstractClientService.java similarity index 70% rename from jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/AbstractBoltClientService.java rename to jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/AbstractClientService.java index 1626359b3..00d52c099 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/AbstractBoltClientService.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/AbstractClientService.java @@ -24,48 +24,45 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.InvokeCallback; -import com.alipay.remoting.InvokeContext; -import com.alipay.remoting.Url; -import com.alipay.remoting.exception.RemotingException; -import com.alipay.remoting.rpc.RpcClient; -import com.alipay.remoting.rpc.exception.InvokeTimeoutException; import com.alipay.sofa.jraft.Status; +import com.alipay.sofa.jraft.error.InvokeTimeoutException; import com.alipay.sofa.jraft.error.RaftError; +import com.alipay.sofa.jraft.error.RemotingException; import com.alipay.sofa.jraft.option.RpcOptions; import com.alipay.sofa.jraft.rpc.ClientService; +import com.alipay.sofa.jraft.rpc.InvokeCallback; +import com.alipay.sofa.jraft.rpc.InvokeContext; import com.alipay.sofa.jraft.rpc.ProtobufMsgFactory; +import com.alipay.sofa.jraft.rpc.RaftRpcFactory; +import com.alipay.sofa.jraft.rpc.RpcClient; import com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse; import com.alipay.sofa.jraft.rpc.RpcRequests.PingRequest; import com.alipay.sofa.jraft.rpc.RpcResponseClosure; -import com.alipay.sofa.jraft.rpc.impl.core.JRaftRpcAddressParser; import com.alipay.sofa.jraft.util.Endpoint; import com.alipay.sofa.jraft.util.NamedThreadFactory; +import com.alipay.sofa.jraft.util.RpcFactoryHelper; import com.alipay.sofa.jraft.util.ThreadPoolMetricSet; import com.alipay.sofa.jraft.util.ThreadPoolUtil; import com.alipay.sofa.jraft.util.Utils; import com.google.protobuf.Message; /** - * Abstract RPC client service based on bolt. + * Abstract RPC client service based. * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 3:27:33 PM + * @author jiachun.fjc */ -public abstract class AbstractBoltClientService implements ClientService { +public abstract class AbstractClientService implements ClientService { - protected static final Logger LOG = LoggerFactory.getLogger(AbstractBoltClientService.class); + protected static final Logger LOG = LoggerFactory.getLogger(AbstractClientService.class); static { ProtobufMsgFactory.load(); } - protected volatile RpcClient rpcClient; - protected ThreadPoolExecutor rpcExecutor; - protected RpcOptions rpcOptions; - protected JRaftRpcAddressParser rpcAddressParser; - protected InvokeContext defaultInvokeCtx; + protected volatile RpcClient rpcClient; + protected ThreadPoolExecutor rpcExecutor; + protected RpcOptions rpcOptions; public RpcClient getRpcClient() { return this.rpcClient; @@ -78,7 +75,7 @@ public boolean isConnected(final Endpoint endpoint) { } private static boolean isConnected(final RpcClient rpcClient, final Endpoint endpoint) { - return rpcClient.checkConnection(endpoint.toString()); + return rpcClient.checkConnection(endpoint); } @Override @@ -87,9 +84,6 @@ public synchronized boolean init(final RpcOptions rpcOptions) { return true; } this.rpcOptions = rpcOptions; - this.rpcAddressParser = new JRaftRpcAddressParser(); - this.defaultInvokeCtx = new InvokeContext(); - this.defaultInvokeCtx.put(InvokeContext.BOLT_CRC_SWITCH, this.rpcOptions.isEnableRpcChecksum()); return initRpcClient(this.rpcOptions.getRpcProcessorThreadPoolSize()); } @@ -98,9 +92,10 @@ protected void configRpcClient(final RpcClient rpcClient) { } protected boolean initRpcClient(final int rpcProcessorThreadPoolSize) { - this.rpcClient = new RpcClient(); + final RaftRpcFactory factory = RpcFactoryHelper.getRpcFactory(); + this.rpcClient = factory.createRpcClient(factory.defaultJRaftClientConfigHelper(this.rpcOptions)); configRpcClient(this.rpcClient); - this.rpcClient.startup(); + this.rpcClient.init(null); this.rpcExecutor = ThreadPoolUtil.newBuilder() // .poolName("JRaft-RPC-Processor") // .enableMetric(true) // @@ -140,7 +135,7 @@ public boolean connect(final Endpoint endpoint) { final PingRequest req = PingRequest.newBuilder() // .setSendTimestamp(System.currentTimeMillis()) // .build(); - final ErrorResponse resp = (ErrorResponse) rc.invokeSync(endpoint.toString(), req, this.defaultInvokeCtx, + final ErrorResponse resp = (ErrorResponse) rc.invokeSync(endpoint, req, this.rpcOptions.getRpcConnectTimeoutMs()); return resp.getErrorCode() == 0; } catch (final InterruptedException e) { @@ -159,20 +154,20 @@ public boolean disconnect(final Endpoint endpoint) { return true; } LOG.info("Disconnect from {}.", endpoint); - rc.closeConnection(endpoint.toString()); + rc.closeConnection(endpoint); return true; } @Override public Future invokeWithDone(final Endpoint endpoint, final Message request, final RpcResponseClosure done, final int timeoutMs) { - return invokeWithDone(endpoint, request, this.defaultInvokeCtx, done, timeoutMs, this.rpcExecutor); + return invokeWithDone(endpoint, request, done, timeoutMs, this.rpcExecutor); } public Future invokeWithDone(final Endpoint endpoint, final Message request, final RpcResponseClosure done, final int timeoutMs, final Executor rpcExecutor) { - return invokeWithDone(endpoint, request, this.defaultInvokeCtx, done, timeoutMs, rpcExecutor); + return invokeWithDone(endpoint, request, null, done, timeoutMs, rpcExecutor); } public Future invokeWithDone(final Endpoint endpoint, final Message request, @@ -186,7 +181,6 @@ public Future invokeWithDone(final Endpoint endpoin final RpcResponseClosure done, final int timeoutMs, final Executor rpcExecutor) { final RpcClient rc = this.rpcClient; - final FutureImpl future = new FutureImpl<>(); try { if (rc == null) { @@ -195,63 +189,59 @@ public Future invokeWithDone(final Endpoint endpoin Utils.runClosureInThread(done, new Status(RaftError.EINTERNAL, "Client service is uninitialized.")); return future; } - final Url rpcUrl = this.rpcAddressParser.parse(endpoint.toString()); - rc.invokeWithCallback(rpcUrl, request, ctx, new InvokeCallback() { + + rc.invokeAsync(endpoint, request, ctx, new InvokeCallback() { @SuppressWarnings("unchecked") @Override - public void onResponse(final Object result) { + public void complete(final Object result, final Throwable err) { if (future.isCancelled()) { onCanceled(request, done); return; } - Status status = Status.OK(); - if (result instanceof ErrorResponse) { - final ErrorResponse eResp = (ErrorResponse) result; - status = new Status(); - status.setCode(eResp.getErrorCode()); - if (eResp.hasErrorMsg()) { - status.setErrorMsg(eResp.getErrorMsg()); + + if (err == null) { + Status status = Status.OK(); + if (result instanceof ErrorResponse) { + final ErrorResponse eResp = (ErrorResponse) result; + status = new Status(); + status.setCode(eResp.getErrorCode()); + if (eResp.hasErrorMsg()) { + status.setErrorMsg(eResp.getErrorMsg()); + } + } else { + if (done != null) { + done.setResponse((T) result); + } } - } else { if (done != null) { - done.setResponse((T) result); + try { + done.run(status); + } catch (final Throwable t) { + LOG.error("Fail to run RpcResponseClosure, the request is {}.", request, t); + } } - } - if (done != null) { - try { - done.run(status); - } catch (final Throwable t) { - LOG.error("Fail to run RpcResponseClosure, the request is {}.", request, t); + if (!future.isDone()) { + future.setResult((Message) result); } - } - if (!future.isDone()) { - future.setResult((Message) result); - } - } - - @Override - public void onException(final Throwable e) { - if (future.isCancelled()) { - onCanceled(request, done); - return; - } - if (done != null) { - try { - done.run(new Status(e instanceof InvokeTimeoutException ? RaftError.ETIMEDOUT - : RaftError.EINTERNAL, "RPC exception:" + e.getMessage())); - } catch (final Throwable t) { - LOG.error("Fail to run RpcResponseClosure, the request is {}.", request, t); + } else { + if (done != null) { + try { + done.run(new Status(err instanceof InvokeTimeoutException ? RaftError.ETIMEDOUT + : RaftError.EINTERNAL, "RPC exception:" + err.getMessage())); + } catch (final Throwable t) { + LOG.error("Fail to run RpcResponseClosure, the request is {}.", request, t); + } + } + if (!future.isDone()) { + future.failure(err); } - } - if (!future.isDone()) { - future.failure(e); } } @Override - public Executor getExecutor() { - return rpcExecutor != null ? rpcExecutor : AbstractBoltClientService.this.rpcExecutor; + public Executor executor() { + return rpcExecutor != null ? rpcExecutor : AbstractClientService.this.rpcExecutor; } }, timeoutMs <= 0 ? this.rpcOptions.getRpcDefaultTimeout() : timeoutMs); } catch (final InterruptedException e) { @@ -266,6 +256,7 @@ public Executor getExecutor() { new Status(RaftError.EINTERNAL, "Fail to send a RPC request:" + e.getMessage())); } + return future; } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/BoltRaftRpcFactory.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/BoltRaftRpcFactory.java new file mode 100644 index 000000000..35cea06b8 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/BoltRaftRpcFactory.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc.impl; + +import com.alipay.remoting.CustomSerializerManager; +import com.alipay.remoting.InvokeContext; +import com.alipay.sofa.jraft.option.RpcOptions; +import com.alipay.sofa.jraft.rpc.ProtobufSerializer; +import com.alipay.sofa.jraft.rpc.RaftRpcFactory; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.rpc.RpcServer; +import com.alipay.sofa.jraft.rpc.impl.core.JRaftRpcAddressParser; +import com.alipay.sofa.jraft.util.Endpoint; +import com.alipay.sofa.jraft.util.Requires; +import com.alipay.sofa.jraft.util.SPI; + +/** + * + * @author jiachun.fjc + */ +@SPI +public class BoltRaftRpcFactory implements RaftRpcFactory { + + @Override + public void registerProtobufSerializer(final String className) { + CustomSerializerManager.registerCustomSerializer(className, ProtobufSerializer.INSTANCE); + } + + @Override + public RpcClient createRpcClient(final ConfigHelper helper) { + final com.alipay.remoting.rpc.RpcClient boltImpl = new com.alipay.remoting.rpc.RpcClient(); + final RpcClient rpcClient = new BoltRpcClient(boltImpl); + if (helper != null) { + helper.config(rpcClient); + } + return rpcClient; + } + + @Override + public RpcServer createRpcServer(final Endpoint endpoint, final ConfigHelper helper) { + final int port = Requires.requireNonNull(endpoint, "endpoint").getPort(); + Requires.requireTrue(port > 0 && port < 0xFFFF, "port out of range:" + port); + final com.alipay.remoting.rpc.RpcServer boltImpl = new com.alipay.remoting.rpc.RpcServer(port, true, false); + final RpcServer rpcServer = new BoltRpcServer(boltImpl); + if (helper != null) { + helper.config(rpcServer); + } + return rpcServer; + } + + @Override + public ConfigHelper defaultJRaftClientConfigHelper(final RpcOptions opts) { + return instance -> { + final BoltRpcClient client = (BoltRpcClient) instance; + final InvokeContext ctx = new InvokeContext(); + ctx.put(InvokeContext.BOLT_CRC_SWITCH, opts.isEnableRpcChecksum()); + client.setDefaultInvokeCtx(ctx); + client.setDefaultAddressParser(new JRaftRpcAddressParser()); + }; + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/BoltRpcClient.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/BoltRpcClient.java new file mode 100644 index 000000000..29f96b3de --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/BoltRpcClient.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc.impl; + +import java.util.Map; +import java.util.concurrent.Executor; + +import com.alipay.remoting.ConnectionEventType; +import com.alipay.remoting.Url; +import com.alipay.remoting.config.switches.GlobalSwitch; +import com.alipay.remoting.rpc.RpcAddressParser; +import com.alipay.sofa.jraft.ReplicatorGroup; +import com.alipay.sofa.jraft.error.InvokeTimeoutException; +import com.alipay.sofa.jraft.error.RemotingException; +import com.alipay.sofa.jraft.option.RpcOptions; +import com.alipay.sofa.jraft.rpc.InvokeCallback; +import com.alipay.sofa.jraft.rpc.InvokeContext; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.rpc.impl.core.ClientServiceConnectionEventProcessor; +import com.alipay.sofa.jraft.util.Endpoint; +import com.alipay.sofa.jraft.util.Requires; + +/** + * Bolt rpc client impl. + * + * @author jiachun.fjc + */ +public class BoltRpcClient implements RpcClient { + + public static final String BOLT_ADDRESS_PARSER = "BOLT_ADDRESS_PARSER"; + public static final String BOLT_CTX = "BOLT_CTX"; + + private final com.alipay.remoting.rpc.RpcClient rpcClient; + + private com.alipay.remoting.InvokeContext defaultInvokeCtx; + private RpcAddressParser defaultAddressParser = new RpcAddressParser(); + + public BoltRpcClient(com.alipay.remoting.rpc.RpcClient rpcClient) { + this.rpcClient = Requires.requireNonNull(rpcClient, "rpcClient"); + } + + @Override + public boolean init(final RpcOptions opts) { + this.rpcClient.switches().turnOn(GlobalSwitch.CODEC_FLUSH_CONSOLIDATION); + this.rpcClient.startup(); + return true; + } + + @Override + public void shutdown() { + this.rpcClient.shutdown(); + } + + @Override + public boolean checkConnection(final Endpoint endpoint) { + Requires.requireNonNull(endpoint, "endpoint"); + return this.rpcClient.checkConnection(endpoint.toString()); + } + + @Override + public void closeConnection(final Endpoint endpoint) { + Requires.requireNonNull(endpoint, "endpoint"); + this.rpcClient.closeConnection(endpoint.toString()); + } + + @Override + public void registerConnectEventListener(final ReplicatorGroup replicatorGroup) { + this.rpcClient.addConnectionEventProcessor(ConnectionEventType.CONNECT, + new ClientServiceConnectionEventProcessor(replicatorGroup)); + } + + @Override + public Object invokeSync(final Endpoint endpoint, final Object request, final InvokeContext ctx, + final long timeoutMs) throws InterruptedException, RemotingException { + Requires.requireNonNull(endpoint, "endpoint"); + final RpcAddressParser addressParser = getAddressParser(ctx); + try { + final Url url = addressParser.parse(endpoint.toString()); + return this.rpcClient.invokeSync(url, request, getBoltInvokeCtx(ctx), (int) timeoutMs); + } catch (final com.alipay.remoting.rpc.exception.InvokeTimeoutException e) { + throw new InvokeTimeoutException(e); + } catch (final com.alipay.remoting.exception.RemotingException e) { + throw new RemotingException(e); + } + } + + @Override + public void invokeAsync(final Endpoint endpoint, final Object request, final InvokeContext ctx, + final InvokeCallback callback, final long timeoutMs) throws InterruptedException, + RemotingException { + Requires.requireNonNull(endpoint, "endpoint"); + final RpcAddressParser addressParser = getAddressParser(ctx); + try { + final Url url = addressParser.parse(endpoint.toString()); + this.rpcClient.invokeWithCallback(url, request, getBoltInvokeCtx(ctx), getBoltCallback(callback), + (int) timeoutMs); + } catch (final com.alipay.remoting.rpc.exception.InvokeTimeoutException e) { + throw new InvokeTimeoutException(e); + } catch (final com.alipay.remoting.exception.RemotingException e) { + throw new RemotingException(e); + } + } + + public com.alipay.remoting.rpc.RpcClient getRpcClient() { + return rpcClient; + } + + public com.alipay.remoting.InvokeContext getDefaultInvokeCtx() { + return defaultInvokeCtx; + } + + public void setDefaultInvokeCtx(com.alipay.remoting.InvokeContext defaultInvokeCtx) { + this.defaultInvokeCtx = defaultInvokeCtx; + } + + public RpcAddressParser getDefaultAddressParser() { + return defaultAddressParser; + } + + public void setDefaultAddressParser(RpcAddressParser defaultAddressParser) { + this.defaultAddressParser = defaultAddressParser; + } + + private RpcAddressParser getAddressParser(final InvokeContext ctx) { + return ctx == null ? this.defaultAddressParser : ctx.getOrDefault(BOLT_ADDRESS_PARSER, + this.defaultAddressParser); + } + + private com.alipay.remoting.InvokeContext getBoltInvokeCtx(final InvokeContext ctx) { + if (ctx == null) { + return this.defaultInvokeCtx; + } + + com.alipay.remoting.InvokeContext boltCtx = ctx.get(BOLT_CTX); + if (boltCtx != null) { + return boltCtx; + } + + boltCtx = new com.alipay.remoting.InvokeContext(); + for (Map.Entry entry : ctx.entrySet()) { + boltCtx.put(entry.getKey(), entry.getValue()); + } + final Boolean crcSwitch = ctx.get(InvokeContext.CRC_SWITCH); + if (crcSwitch != null) { + boltCtx.put(com.alipay.remoting.InvokeContext.BOLT_CRC_SWITCH, crcSwitch); + } + return boltCtx; + } + + private BoltCallback getBoltCallback(final InvokeCallback callback) { + return new BoltCallback(callback); + } + + private static class BoltCallback implements com.alipay.remoting.InvokeCallback { + + private final InvokeCallback callback; + + private BoltCallback(final InvokeCallback callback) { + this.callback = callback; + } + + @Override + public void onResponse(final Object result) { + this.callback.complete(result, null); + } + + @Override + public void onException(final Throwable err) { + this.callback.complete(null, err); + } + + @Override + public Executor getExecutor() { + return this.callback.executor(); + } + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/BoltRpcServer.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/BoltRpcServer.java new file mode 100644 index 000000000..428501080 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/BoltRpcServer.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc.impl; + +import com.alipay.remoting.AsyncContext; +import com.alipay.remoting.BizContext; +import com.alipay.remoting.ConnectionEventType; +import com.alipay.remoting.config.switches.GlobalSwitch; +import com.alipay.remoting.rpc.protocol.AsyncUserProcessor; +import com.alipay.sofa.jraft.rpc.Connection; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; +import com.alipay.sofa.jraft.rpc.RpcServer; +import com.alipay.sofa.jraft.util.Requires; + +/** + * Bolt RPC server impl. + * + * @author jiachun.fjc + */ +public class BoltRpcServer implements RpcServer { + + private final com.alipay.remoting.rpc.RpcServer rpcServer; + + public BoltRpcServer(com.alipay.remoting.rpc.RpcServer rpcServer) { + this.rpcServer = Requires.requireNonNull(rpcServer, "rpcServer"); + } + + @Override + public boolean init(final Void opts) { + this.rpcServer.switches().turnOn(GlobalSwitch.CODEC_FLUSH_CONSOLIDATION); + this.rpcServer.startup(); + return this.rpcServer.isStarted(); + } + + @Override + public void shutdown() { + this.rpcServer.shutdown(); + } + + @Override + public void registerConnectionClosedEventListener(final ConnectionClosedEventListener listener) { + this.rpcServer.addConnectionEventProcessor(ConnectionEventType.CLOSE, (remoteAddress, conn) -> { + final Connection proxyConn = conn == null ? null : new Connection() { + + @Override + public Object getAttribute(final String key) { + return conn.getAttribute(key); + } + + @Override + public void setAttribute(final String key, final Object value) { + conn.setAttribute(key, value); + } + + @Override + public void close() { + conn.close(); + } + }; + + listener.onClosed(remoteAddress, proxyConn); + }); + } + + @Override + public int boundPort() { + return this.rpcServer.port(); + } + + @Override + public void registerProcessor(final RpcProcessor processor) { + this.rpcServer.registerUserProcessor(new AsyncUserProcessor() { + + @SuppressWarnings("unchecked") + @Override + public void handleRequest(final BizContext bizCtx, final AsyncContext asyncCtx, final Object request) { + final RpcContext rpcCtx = new RpcContext() { + + @Override + public void sendResponse(final Object responseObj) { + asyncCtx.sendResponse(responseObj); + } + + @Override + public Connection getConnection() { + com.alipay.remoting.Connection conn = bizCtx.getConnection(); + if (conn == null) { + return null; + } + return new BoltConnection(conn); + } + + @Override + public String getRemoteAddress() { + return bizCtx.getRemoteAddress(); + } + }; + + processor.handleRequest(rpcCtx, request); + } + + @Override + public String interest() { + return processor.interest(); + } + + @Override + public ExecutorSelector getExecutorSelector() { + final RpcProcessor.ExecutorSelector realSelector = processor.executorSelector(); + if (realSelector == null) { + return null; + } + return realSelector::select; + } + }); + } + + private static class BoltConnection implements Connection { + + private final com.alipay.remoting.Connection conn; + + private BoltConnection(com.alipay.remoting.Connection conn) { + this.conn = Requires.requireNonNull(conn, "conn"); + } + + @Override + public Object getAttribute(final String key) { + return this.conn.getAttribute(key); + } + + @Override + public void setAttribute(final String key, final Object value) { + this.conn.setAttribute(key, value); + } + + @Override + public void close() { + this.conn.close(); + } + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/ConnectionClosedEventListener.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/ConnectionClosedEventListener.java new file mode 100644 index 000000000..fe069d0a8 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/ConnectionClosedEventListener.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc.impl; + +import com.alipay.sofa.jraft.rpc.Connection; + +/** + * + * @author jiachun.fjc + */ +public interface ConnectionClosedEventListener { + + void onClosed(final String remoteAddress, final Connection conn); +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/FutureImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/FutureImpl.java index 029f67ab3..6c7e6e71d 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/FutureImpl.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/FutureImpl.java @@ -90,7 +90,7 @@ public FutureImpl() { public FutureImpl(ReentrantLock lock) { this.lock = lock; - latch = new CountDownLatch(1); + this.latch = new CountDownLatch(1); } /** @@ -99,20 +99,20 @@ public FutureImpl(ReentrantLock lock) { * @return current result value without any blocking. */ public R getResult() { + this.lock.lock(); try { - lock.lock(); - return result; + return this.result; } finally { - lock.unlock(); + this.lock.unlock(); } } public Throwable getFailure() { + this.lock.lock(); try { - lock.lock(); return this.failure; } finally { - lock.unlock(); + this.lock.unlock(); } } @@ -123,12 +123,12 @@ public Throwable getFailure() { * the result value */ public void setResult(R result) { + this.lock.lock(); try { - lock.lock(); this.result = result; notifyHaveResult(); } finally { - lock.unlock(); + this.lock.unlock(); } } @@ -137,13 +137,13 @@ public void setResult(R result) { */ @Override public boolean cancel(boolean mayInterruptIfRunning) { + this.lock.lock(); try { - lock.lock(); - isCancelled = true; + this.isCancelled = true; notifyHaveResult(); return true; } finally { - lock.unlock(); + this.lock.unlock(); } } @@ -153,10 +153,10 @@ public boolean cancel(boolean mayInterruptIfRunning) { @Override public boolean isCancelled() { try { - lock.lock(); - return isCancelled; + this.lock.lock(); + return this.isCancelled; } finally { - lock.unlock(); + this.lock.unlock(); } } @@ -165,11 +165,11 @@ public boolean isCancelled() { */ @Override public boolean isDone() { + this.lock.lock(); try { - lock.lock(); - return isDone; + return this.isDone; } finally { - lock.unlock(); + this.lock.unlock(); } } @@ -178,19 +178,18 @@ public boolean isDone() { */ @Override public R get() throws InterruptedException, ExecutionException { - latch.await(); - + this.latch.await(); + this.lock.lock(); try { - lock.lock(); - if (isCancelled) { + if (this.isCancelled) { throw new CancellationException(); - } else if (failure != null) { - throw new ExecutionException(failure); + } else if (this.failure != null) { + throw new ExecutionException(this.failure); } - return result; + return this.result; } finally { - lock.unlock(); + this.lock.unlock(); } } @@ -198,23 +197,24 @@ public R get() throws InterruptedException, ExecutionException { * {@inheritDoc} */ @Override - public R get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + public R get(final long timeout, final TimeUnit unit) throws InterruptedException, ExecutionException, + TimeoutException { final boolean isTimeOut = !latch.await(timeout, unit); + this.lock.lock(); try { - lock.lock(); if (!isTimeOut) { - if (isCancelled) { + if (this.isCancelled) { throw new CancellationException(); - } else if (failure != null) { - throw new ExecutionException(failure); + } else if (this.failure != null) { + throw new ExecutionException(this.failure); } - return result; + return this.result; } else { throw new TimeoutException(); } } finally { - lock.unlock(); + this.lock.unlock(); } } @@ -222,13 +222,13 @@ public R get(long timeout, TimeUnit unit) throws InterruptedException, Execution * Notify about the failure, occured during asynchronous operation * execution. */ - public void failure(Throwable failure) { + public void failure(final Throwable failure) { + this.lock.lock(); try { - lock.lock(); this.failure = failure; notifyHaveResult(); } finally { - lock.unlock(); + this.lock.unlock(); } } @@ -236,7 +236,7 @@ public void failure(Throwable failure) { * Notify blocked listeners threads about operation completion. */ protected void notifyHaveResult() { - isDone = true; - latch.countDown(); + this.isDone = true; + this.latch.countDown(); } } \ No newline at end of file diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/PingRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/PingRequestProcessor.java index 56d958c7e..faf2538d4 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/PingRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/PingRequestProcessor.java @@ -16,8 +16,8 @@ */ package com.alipay.sofa.jraft.rpc.impl; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.rpc.protocol.SyncUserProcessor; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; import com.alipay.sofa.jraft.rpc.RpcRequests.PingRequest; import com.alipay.sofa.jraft.rpc.RpcResponseFactory; @@ -25,14 +25,13 @@ * Ping request processor. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-08 7:33:30 PM + * @author jiachun.fjc */ -public class PingRequestProcessor extends SyncUserProcessor { +public class PingRequestProcessor implements RpcProcessor { @Override - public Object handleRequest(BizContext bizCtx, PingRequest request) throws Exception { - return RpcResponseFactory.newResponse(0, "OK"); + public void handleRequest(final RpcContext rpcCtx, final PingRequest request) { + rpcCtx.sendResponse(RpcResponseFactory.newResponse(0, "OK")); } @Override diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/AddLearnersRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/AddLearnersRequestProcessor.java index 38b2cf8db..783d5fe3a 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/AddLearnersRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/AddLearnersRequestProcessor.java @@ -32,7 +32,7 @@ * AddLearners request processor. * * @author boyan (boyan@alibaba-inc.com) - * + * @author jiachun.fjc */ public class AddLearnersRequestProcessor extends BaseCliRequestProcessor { @@ -64,8 +64,8 @@ protected Message processRequest0(final CliRequestContext ctx, final AddLearners addingLearners.add(peer); } - LOG.info("Receive AddLearnersRequest to {} from {}, adding {}", ctx.node.getNodeId(), - done.getBizContext().getRemoteAddress(), addingLearners); + LOG.info("Receive AddLearnersRequest to {} from {}, adding {}.", ctx.node.getNodeId(), + done.getRpcCtx().getRemoteAddress(), addingLearners); ctx.node.addLearners(addingLearners, status -> { if (!status.isOk()) { done.run(status); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/AddPeerRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/AddPeerRequestProcessor.java index 5437c1e51..166304cb7 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/AddPeerRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/AddPeerRequestProcessor.java @@ -31,8 +31,7 @@ * AddPeer request processor. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 11:33:50 AM + * @author jiachun.fjc */ public class AddPeerRequestProcessor extends BaseCliRequestProcessor { @@ -41,30 +40,30 @@ public AddPeerRequestProcessor(Executor executor) { } @Override - protected String getPeerId(AddPeerRequest request) { + protected String getPeerId(final AddPeerRequest request) { return request.getLeaderId(); } @Override - protected String getGroupId(AddPeerRequest request) { + protected String getGroupId(final AddPeerRequest request) { return request.getGroupId(); } @Override - protected Message processRequest0(CliRequestContext ctx, AddPeerRequest request, RpcRequestClosure done) { - List oldPeers = ctx.node.listPeers(); - String addingPeerIdStr = request.getPeerId(); - PeerId addingPeer = new PeerId(); + protected Message processRequest0(final CliRequestContext ctx, final AddPeerRequest request, final RpcRequestClosure done) { + final List oldPeers = ctx.node.listPeers(); + final String addingPeerIdStr = request.getPeerId(); + final PeerId addingPeer = new PeerId(); if (addingPeer.parse(addingPeerIdStr)) { - LOG.info("Receive AddPeerRequest to {} from {}, adding {}", ctx.node.getNodeId(), done.getBizContext() + LOG.info("Receive AddPeerRequest to {} from {}, adding {}", ctx.node.getNodeId(), done.getRpcCtx() .getRemoteAddress(), addingPeerIdStr); ctx.node.addPeer(addingPeer, status -> { if (!status.isOk()) { done.run(status); } else { - AddPeerResponse.Builder rb = AddPeerResponse.newBuilder(); + final AddPeerResponse.Builder rb = AddPeerResponse.newBuilder(); boolean alreadyExists = false; - for (PeerId oldPeer : oldPeers) { + for (final PeerId oldPeer : oldPeers) { rb.addOldPeers(oldPeer.toString()); rb.addNewPeers(oldPeer.toString()); if (oldPeer.equals(addingPeer)) { diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ChangePeersRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ChangePeersRequestProcessor.java index c2a889b4a..b52549d8f 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ChangePeersRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ChangePeersRequestProcessor.java @@ -32,8 +32,7 @@ * Change peers request processor. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 3:09:23 PM + * @author jiachun.fjc */ public class ChangePeersRequestProcessor extends BaseCliRequestProcessor { @@ -42,39 +41,39 @@ public ChangePeersRequestProcessor(Executor executor) { } @Override - protected String getPeerId(ChangePeersRequest request) { + protected String getPeerId(final ChangePeersRequest request) { return request.getLeaderId(); } @Override - protected String getGroupId(ChangePeersRequest request) { + protected String getGroupId(final ChangePeersRequest request) { return request.getGroupId(); } @Override - protected Message processRequest0(CliRequestContext ctx, ChangePeersRequest request, RpcRequestClosure done) { - List oldConf = ctx.node.listPeers(); + protected Message processRequest0(final CliRequestContext ctx, final ChangePeersRequest request, final RpcRequestClosure done) { + final List oldConf = ctx.node.listPeers(); - Configuration conf = new Configuration(); - for (String peerIdStr : request.getNewPeersList()) { - PeerId peer = new PeerId(); + final Configuration conf = new Configuration(); + for (final String peerIdStr : request.getNewPeersList()) { + final PeerId peer = new PeerId(); if (peer.parse(peerIdStr)) { conf.addPeer(peer); } else { return RpcResponseFactory.newResponse(RaftError.EINVAL, "Fail to parse peer id %s", peerIdStr); } } - LOG.info("Receive ChangePeersRequest to {} from {}, new conf is {}", ctx.node.getNodeId(), done.getBizContext() + LOG.info("Receive ChangePeersRequest to {} from {}, new conf is {}", ctx.node.getNodeId(), done.getRpcCtx() .getRemoteAddress(), conf); ctx.node.changePeers(conf, status -> { if (!status.isOk()) { done.run(status); } else { ChangePeersResponse.Builder rb = ChangePeersResponse.newBuilder(); - for (PeerId peer : oldConf) { + for (final PeerId peer : oldConf) { rb.addOldPeers(peer.toString()); } - for (PeerId peer : conf) { + for (final PeerId peer : conf) { rb.addNewPeers(peer.toString()); } done.sendResponse(rb.build()); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/BoltCliClientService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/CliClientServiceImpl.java similarity index 80% rename from jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/BoltCliClientService.java rename to jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/CliClientServiceImpl.java index 37ce130b7..679c96301 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/BoltCliClientService.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/CliClientServiceImpl.java @@ -39,17 +39,16 @@ import com.alipay.sofa.jraft.rpc.CliRequests.TransferLeaderRequest; import com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse; import com.alipay.sofa.jraft.rpc.RpcResponseClosure; -import com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService; +import com.alipay.sofa.jraft.rpc.impl.AbstractClientService; import com.alipay.sofa.jraft.util.Endpoint; import com.google.protobuf.Message; /** * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 3:14:55 PM + * @author jiachun.fjc */ -public class BoltCliClientService extends AbstractBoltClientService implements CliClientService { +public class CliClientServiceImpl extends AbstractClientService implements CliClientService { private CliOptions cliOptions; @@ -65,67 +64,66 @@ public synchronized boolean init(final RpcOptions rpcOptions) { @Override public Future addPeer(final Endpoint endpoint, final AddPeerRequest request, final RpcResponseClosure done) { - - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future removePeer(final Endpoint endpoint, final RemovePeerRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future resetPeer(final Endpoint endpoint, final ResetPeerRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future snapshot(final Endpoint endpoint, final SnapshotRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future changePeers(final Endpoint endpoint, final ChangePeersRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future addLearners(final Endpoint endpoint, final AddLearnersRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future removeLearners(final Endpoint endpoint, final RemoveLearnersRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future resetLearners(final Endpoint endpoint, final ResetLearnersRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future getLeader(final Endpoint endpoint, final GetLeaderRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future transferLeader(final Endpoint endpoint, final TransferLeaderRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } @Override public Future getPeers(final Endpoint endpoint, final CliRequests.GetPeersRequest request, final RpcResponseClosure done) { - return this.invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); } } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/GetLeaderRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/GetLeaderRequestProcessor.java index 53b716424..7619ee954 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/GetLeaderRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/GetLeaderRequestProcessor.java @@ -35,8 +35,7 @@ * Process get leader request. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 2:43:20 PM + * @author jiachun.fjc */ public class GetLeaderRequestProcessor extends BaseCliRequestProcessor { @@ -45,30 +44,31 @@ public GetLeaderRequestProcessor(Executor executor) { } @Override - protected String getPeerId(GetLeaderRequest request) { + protected String getPeerId(final GetLeaderRequest request) { return request.getPeerId(); } @Override - protected String getGroupId(GetLeaderRequest request) { + protected String getGroupId(final GetLeaderRequest request) { return request.getGroupId(); } @Override - protected Message processRequest0(CliRequestContext ctx, GetLeaderRequest request, RpcRequestClosure done) { - //ignore + protected Message processRequest0(final CliRequestContext ctx, final GetLeaderRequest request, + final RpcRequestClosure done) { + // ignore return null; } @Override - public Message processRequest(GetLeaderRequest request, RpcRequestClosure done) { + public Message processRequest(final GetLeaderRequest request, final RpcRequestClosure done) { List nodes = new ArrayList<>(); - String groupId = getGroupId(request); + final String groupId = getGroupId(request); if (request.hasPeerId()) { - String peerIdStr = getPeerId(request); - PeerId peer = new PeerId(); + final String peerIdStr = getPeerId(request); + final PeerId peer = new PeerId(); if (peer.parse(peerIdStr)) { - Status st = new Status(); + final Status st = new Status(); nodes.add(getNode(groupId, peer, st)); if (!st.isOk()) { return RpcResponseFactory.newResponse(st); @@ -82,8 +82,8 @@ public Message processRequest(GetLeaderRequest request, RpcRequestClosure done) if (nodes == null || nodes.isEmpty()) { return RpcResponseFactory.newResponse(RaftError.ENOENT, "No nodes in group %s", groupId); } - for (Node node : nodes) { - PeerId leader = node.getLeaderId(); + for (final Node node : nodes) { + final PeerId leader = node.getLeaderId(); if (leader != null && !leader.isEmpty()) { return GetLeaderResponse.newBuilder().setLeaderId(leader.toString()).build(); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/RemoveLearnersRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/RemoveLearnersRequestProcessor.java index 14f0df8a1..f27c4b32f 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/RemoveLearnersRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/RemoveLearnersRequestProcessor.java @@ -64,8 +64,8 @@ protected Message processRequest0(final CliRequestContext ctx, final RemoveLearn removeingLearners.add(peer); } - LOG.info("Receive RemoveLearnersRequest to {} from {}, removing {}", ctx.node.getNodeId(), - done.getBizContext().getRemoteAddress(), removeingLearners); + LOG.info("Receive RemoveLearnersRequest to {} from {}, removing {}.", ctx.node.getNodeId(), + done.getRpcCtx().getRemoteAddress(), removeingLearners); ctx.node.removeLearners(removeingLearners, status -> { if (!status.isOk()) { done.run(status); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/RemovePeerRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/RemovePeerRequestProcessor.java index a3063b60d..41e76f122 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/RemovePeerRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/RemovePeerRequestProcessor.java @@ -31,8 +31,7 @@ * Remove peer request processor. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 2:23:40 PM + * @author jiachun.fjc */ public class RemovePeerRequestProcessor extends BaseCliRequestProcessor { @@ -41,29 +40,29 @@ public RemovePeerRequestProcessor(Executor executor) { } @Override - protected String getPeerId(RemovePeerRequest request) { + protected String getPeerId(final RemovePeerRequest request) { return request.getLeaderId(); } @Override - protected String getGroupId(RemovePeerRequest request) { + protected String getGroupId(final RemovePeerRequest request) { return request.getGroupId(); } @Override - protected Message processRequest0(CliRequestContext ctx, RemovePeerRequest request, RpcRequestClosure done) { - List oldPeers = ctx.node.listPeers(); - String removingPeerIdStr = request.getPeerId(); - PeerId removingPeer = new PeerId(); + protected Message processRequest0(final CliRequestContext ctx, final RemovePeerRequest request, final RpcRequestClosure done) { + final List oldPeers = ctx.node.listPeers(); + final String removingPeerIdStr = request.getPeerId(); + final PeerId removingPeer = new PeerId(); if (removingPeer.parse(removingPeerIdStr)) { - LOG.info("Receive RemovePeerRequest to {} from {}, removing {}", ctx.node.getNodeId(), done.getBizContext() + LOG.info("Receive RemovePeerRequest to {} from {}, removing {}", ctx.node.getNodeId(), done.getRpcCtx() .getRemoteAddress(), removingPeerIdStr); ctx.node.removePeer(removingPeer, status -> { if (!status.isOk()) { done.run(status); } else { - RemovePeerResponse.Builder rb = RemovePeerResponse.newBuilder(); - for (PeerId oldPeer : oldPeers) { + final RemovePeerResponse.Builder rb = RemovePeerResponse.newBuilder(); + for (final PeerId oldPeer : oldPeers) { rb.addOldPeers(oldPeer.toString()); if (!oldPeer.equals(removingPeer)) { rb.addNewPeers(oldPeer.toString()); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetLearnersRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetLearnersRequestProcessor.java index fe29bb2ba..e0b4583f1 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetLearnersRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetLearnersRequestProcessor.java @@ -64,8 +64,8 @@ protected Message processRequest0(final CliRequestContext ctx, final ResetLearne newLearners.add(peer); } - LOG.info("Receive ResetLearnersRequest to {} from {}, resetting into {}", ctx.node.getNodeId(), - done.getBizContext().getRemoteAddress(), newLearners); + LOG.info("Receive ResetLearnersRequest to {} from {}, resetting into {}.", ctx.node.getNodeId(), + done.getRpcCtx().getRemoteAddress(), newLearners); ctx.node.resetLearners(newLearners, status -> { if (!status.isOk()) { done.run(status); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetPeerRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetPeerRequestProcessor.java index 72702eb38..2efd388c4 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetPeerRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetPeerRequestProcessor.java @@ -31,8 +31,7 @@ * Reset peer request processor. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 2:38:32 PM + * @author jiachun.fjc */ public class ResetPeerRequestProcessor extends BaseCliRequestProcessor { @@ -41,29 +40,30 @@ public ResetPeerRequestProcessor(Executor executor) { } @Override - protected String getPeerId(ResetPeerRequest request) { + protected String getPeerId(final ResetPeerRequest request) { return request.getPeerId(); } @Override - protected String getGroupId(ResetPeerRequest request) { + protected String getGroupId(final ResetPeerRequest request) { return request.getGroupId(); } @Override - protected Message processRequest0(CliRequestContext ctx, ResetPeerRequest request, RpcRequestClosure done) { - Configuration newConf = new Configuration(); - for (String peerIdStr : request.getNewPeersList()) { - PeerId peer = new PeerId(); + protected Message processRequest0(final CliRequestContext ctx, final ResetPeerRequest request, + final RpcRequestClosure done) { + final Configuration newConf = new Configuration(); + for (final String peerIdStr : request.getNewPeersList()) { + final PeerId peer = new PeerId(); if (peer.parse(peerIdStr)) { newConf.addPeer(peer); } else { return RpcResponseFactory.newResponse(RaftError.EINVAL, "Fail to parse peer id %s", peerIdStr); } } - LOG.info("Receive ResetPeerRequest to {} from {}, new conf is {}", ctx.node.getNodeId(), done.getBizContext() + LOG.info("Receive ResetPeerRequest to {} from {}, new conf is {}", ctx.node.getNodeId(), done.getRpcCtx() .getRemoteAddress(), newConf); - Status st = ctx.node.resetPeers(newConf); + final Status st = ctx.node.resetPeers(newConf); return RpcResponseFactory.newResponse(st); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/SnapshotRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/SnapshotRequestProcessor.java index 65be41f7c..a6e5481ff 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/SnapshotRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/SnapshotRequestProcessor.java @@ -26,8 +26,7 @@ * Snapshot request processor. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 2:41:27 PM + * @author jiachun.fjc */ public class SnapshotRequestProcessor extends BaseCliRequestProcessor { @@ -36,17 +35,18 @@ public SnapshotRequestProcessor(Executor executor) { } @Override - protected String getPeerId(SnapshotRequest request) { + protected String getPeerId(final SnapshotRequest request) { return request.getPeerId(); } @Override - protected String getGroupId(SnapshotRequest request) { + protected String getGroupId(final SnapshotRequest request) { return request.getGroupId(); } @Override - protected Message processRequest0(CliRequestContext ctx, SnapshotRequest request, RpcRequestClosure done) { + protected Message processRequest0(final CliRequestContext ctx, final SnapshotRequest request, + final RpcRequestClosure done) { LOG.info("Receive SnapshotRequest to {} from {}", ctx.node.getNodeId(), request.getPeerId()); ctx.node.snapshot(done); return null; diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/TransferLeaderRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/TransferLeaderRequestProcessor.java index ef5ecd6b2..67fb591b7 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/TransferLeaderRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/TransferLeaderRequestProcessor.java @@ -30,8 +30,7 @@ * Snapshot request processor. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-09 2:41:27 PM + * @author jiachun.fjc */ public class TransferLeaderRequestProcessor extends BaseCliRequestProcessor { @@ -40,24 +39,25 @@ public TransferLeaderRequestProcessor(Executor executor) { } @Override - protected String getPeerId(TransferLeaderRequest request) { + protected String getPeerId(final TransferLeaderRequest request) { return request.getLeaderId(); } @Override - protected String getGroupId(TransferLeaderRequest request) { + protected String getGroupId(final TransferLeaderRequest request) { return request.getGroupId(); } @Override - protected Message processRequest0(CliRequestContext ctx, TransferLeaderRequest request, RpcRequestClosure done) { - PeerId peer = new PeerId(); + protected Message processRequest0(final CliRequestContext ctx, final TransferLeaderRequest request, + final RpcRequestClosure done) { + final PeerId peer = new PeerId(); if (request.hasPeerId() && !peer.parse(request.getPeerId())) { return RpcResponseFactory.newResponse(RaftError.EINVAL, "Fail to parse peer id %s", request.getPeerId()); } - LOG.info("Receive TransferLeaderRequest to {} from {} , newLeader will be {}", ctx.node.getNodeId(), done - .getBizContext().getRemoteAddress(), peer); - Status st = ctx.node.transferLeadershipTo(peer); + LOG.info("Receive TransferLeaderRequest to {} from {}, newLeader will be {}.", ctx.node.getNodeId(), done + .getRpcCtx().getRemoteAddress(), peer); + final Status st = ctx.node.transferLeadershipTo(peer); return RpcResponseFactory.newResponse(st); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/AppendEntriesRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/AppendEntriesRequestProcessor.java index 182fbe306..c91cd3f8c 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/AppendEntriesRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/AppendEntriesRequestProcessor.java @@ -24,18 +24,18 @@ import org.apache.commons.lang.StringUtils; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.Connection; -import com.alipay.remoting.ConnectionEventProcessor; import com.alipay.sofa.jraft.JRaftUtils; import com.alipay.sofa.jraft.Node; import com.alipay.sofa.jraft.NodeManager; import com.alipay.sofa.jraft.entity.PeerId; +import com.alipay.sofa.jraft.rpc.Connection; import com.alipay.sofa.jraft.rpc.RaftServerService; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; import com.alipay.sofa.jraft.rpc.RpcRequestClosure; import com.alipay.sofa.jraft.rpc.RpcRequests.AppendEntriesRequest; import com.alipay.sofa.jraft.rpc.RpcRequests.AppendEntriesRequestHeader; +import com.alipay.sofa.jraft.rpc.impl.ConnectionClosedEventListener; import com.alipay.sofa.jraft.util.Utils; import com.alipay.sofa.jraft.util.concurrent.MpscSingleThreadExecutor; import com.alipay.sofa.jraft.util.concurrent.SingleThreadExecutor; @@ -49,7 +49,7 @@ * 2018-Apr-04 3:00:13 PM */ public class AppendEntriesRequestProcessor extends NodeRequestProcessor implements - ConnectionEventProcessor { + ConnectionClosedEventListener { static final String PEER_ATTR = "jraft-peer"; @@ -57,28 +57,28 @@ public class AppendEntriesRequestProcessor extends NodeRequestProcessor { - public final Message msg; - private final int sequence; - private final AsyncContext asyncContext; + public final Message msg; + private final int sequence; + private final RpcContext rpcCtx; - public SequenceMessage(AsyncContext asyncContext, Message msg, int sequence) { + public SequenceMessage(RpcContext rpcCtx, Message msg, int sequence) { super(); - this.asyncContext = asyncContext; + this.rpcCtx = rpcCtx; this.msg = msg; this.sequence = sequence; } @@ -134,7 +134,7 @@ public SequenceMessage(AsyncContext asyncContext, Message msg, int sequence) { * Send the response. */ void sendResponse() { - this.asyncContext.sendResponse(this.msg); + this.rpcCtx.sendResponse(this.msg); } /** @@ -149,15 +149,15 @@ public int compareTo(final SequenceMessage o) { /** * Send request in pipeline mode. */ - void sendSequenceResponse(final String groupId, final String peerId, final int seq, - final AsyncContext asyncContext, final BizContext bizContext, final Message msg) { - final Connection connection = bizContext.getConnection(); + void sendSequenceResponse(final String groupId, final String peerId, final int seq, final RpcContext rpcCtx, + final Message msg) { + final Connection connection = rpcCtx.getConnection(); final PeerRequestContext ctx = getPeerRequestContext(groupId, peerId, connection); final PriorityQueue respQueue = ctx.responseQueue; assert (respQueue != null); synchronized (Utils.withLockObject(respQueue)) { - respQueue.add(new SequenceMessage(asyncContext, msg, seq)); + respQueue.add(new SequenceMessage(rpcCtx, msg, seq)); if (!ctx.hasTooManyPendingResponses()) { while (!respQueue.isEmpty()) { @@ -346,12 +346,11 @@ public Message processRequest0(final RaftServerService service, final AppendEntr final String groupId = request.getGroupId(); final String peerId = request.getPeerId(); - final int reqSequence = getAndIncrementSequence(groupId, peerId, done.getBizContext().getConnection()); + final int reqSequence = getAndIncrementSequence(groupId, peerId, done.getRpcCtx().getConnection()); final Message response = service.handleAppendEntriesRequest(request, new SequenceRpcRequestClosure(done, reqSequence, groupId, peerId)); if (response != null) { - sendSequenceResponse(groupId, peerId, reqSequence, done.getAsyncContext(), done.getBizContext(), - response); + sendSequenceResponse(groupId, peerId, reqSequence, done.getRpcCtx(), response); } return null; } else { @@ -365,7 +364,7 @@ public String interest() { } @Override - public ExecutorSelector getExecutorSelector() { + public ExecutorSelector executorSelector() { return this.executorSelector; } @@ -379,7 +378,7 @@ public void destroy() { } @Override - public void onEvent(final String remoteAddr, final Connection conn) { + public void onClosed(final String remoteAddress, final Connection conn) { final PeerId peer = new PeerId(); final String peerAttr = (String) conn.getAttribute(PEER_ATTR); @@ -396,7 +395,7 @@ public void onEvent(final String remoteAddr, final Connection conn) { } } } else { - LOG.info("Connection disconnected: {}", remoteAddr); + LOG.info("Connection disconnected: {}", remoteAddress); } } } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/BoltRaftClientService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/BoltRaftClientService.java index 9b2e63193..142b02316 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/BoltRaftClientService.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/BoltRaftClientService.java @@ -21,13 +21,12 @@ import java.util.concurrent.Executor; import java.util.concurrent.Future; -import com.alipay.remoting.ConnectionEventType; -import com.alipay.remoting.InvokeContext; -import com.alipay.remoting.rpc.RpcClient; import com.alipay.sofa.jraft.ReplicatorGroup; import com.alipay.sofa.jraft.option.NodeOptions; import com.alipay.sofa.jraft.option.RpcOptions; +import com.alipay.sofa.jraft.rpc.InvokeContext; import com.alipay.sofa.jraft.rpc.RaftClientService; +import com.alipay.sofa.jraft.rpc.RpcClient; import com.alipay.sofa.jraft.rpc.RpcRequests.AppendEntriesRequest; import com.alipay.sofa.jraft.rpc.RpcRequests.AppendEntriesResponse; import com.alipay.sofa.jraft.rpc.RpcRequests.GetFileRequest; @@ -41,7 +40,7 @@ import com.alipay.sofa.jraft.rpc.RpcRequests.TimeoutNowRequest; import com.alipay.sofa.jraft.rpc.RpcRequests.TimeoutNowResponse; import com.alipay.sofa.jraft.rpc.RpcResponseClosure; -import com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService; +import com.alipay.sofa.jraft.rpc.impl.AbstractClientService; import com.alipay.sofa.jraft.util.Endpoint; import com.alipay.sofa.jraft.util.Utils; import com.alipay.sofa.jraft.util.concurrent.DefaultFixedThreadsExecutorGroupFactory; @@ -52,10 +51,9 @@ * Raft rpc service based bolt. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Mar-28 6:07:05 PM + * @author jiachun.fjc */ -public class BoltRaftClientService extends AbstractBoltClientService implements RaftClientService { +public class BoltRaftClientService extends AbstractClientService implements RaftClientService { private static final FixedThreadsExecutorGroup APPEND_ENTRIES_EXECUTORS = DefaultFixedThreadsExecutorGroupFactory.INSTANCE .newExecutorGroup( @@ -72,8 +70,7 @@ public class BoltRaftClientService extends AbstractBoltClientService implements @Override protected void configRpcClient(final RpcClient rpcClient) { - rpcClient.addConnectionEventProcessor(ConnectionEventType.CONNECT, new ClientServiceConnectionEventProcessor( - this.rgGroup)); + rpcClient.registerConnectEventListener(this.rgGroup); } public BoltRaftClientService(final ReplicatorGroup rgGroup) { @@ -113,7 +110,7 @@ public Future getFile(final Endpoint endpoint, final GetFileRequest req final RpcResponseClosure done) { // open checksum final InvokeContext ctx = new InvokeContext(); - ctx.put(InvokeContext.BOLT_CRC_SWITCH, true); + ctx.put(InvokeContext.CRC_SWITCH, true); return invokeWithDone(endpoint, request, ctx, done, timeoutMs); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/GetFileRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/GetFileRequestProcessor.java index 3763d8108..639b80636 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/GetFileRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/GetFileRequestProcessor.java @@ -38,7 +38,7 @@ public GetFileRequestProcessor(Executor executor) { } @Override - public Message processRequest(GetFileRequest request, RpcRequestClosure done) { + public Message processRequest(final GetFileRequest request, final RpcRequestClosure done) { return FileService.getInstance().handleGetFile(request, done); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/InstallSnapshotRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/InstallSnapshotRequestProcessor.java index a73f5c2d0..571385c57 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/InstallSnapshotRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/InstallSnapshotRequestProcessor.java @@ -37,17 +37,18 @@ public InstallSnapshotRequestProcessor(Executor executor) { } @Override - protected String getPeerId(InstallSnapshotRequest request) { + protected String getPeerId(final InstallSnapshotRequest request) { return request.getPeerId(); } @Override - protected String getGroupId(InstallSnapshotRequest request) { + protected String getGroupId(final InstallSnapshotRequest request) { return request.getGroupId(); } @Override - public Message processRequest0(RaftServerService service, InstallSnapshotRequest request, RpcRequestClosure done) { + public Message processRequest0(final RaftServerService service, final InstallSnapshotRequest request, + final RpcRequestClosure done) { return service.handleInstallSnapshot(request, done); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/NodeRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/NodeRequestProcessor.java index 13432fe73..3a54cc34a 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/NodeRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/NodeRequestProcessor.java @@ -31,10 +31,10 @@ /** * Node handle requests processor template. * - * @author boyan (boyan@alibaba-inc.com) + * @param Message * - * 2018-Apr-08 6:03:25 PM - * @param + * @author boyan (boyan@alibaba-inc.com) + * @author jiachun.fjc */ public abstract class NodeRequestProcessor extends RpcRequestProcessor { @@ -50,7 +50,7 @@ protected abstract Message processRequest0(final RaftServerService serviceServic protected abstract String getGroupId(final T request); @Override - public Message processRequest(T request, RpcRequestClosure done) { + public Message processRequest(final T request, final RpcRequestClosure done) { final PeerId peer = new PeerId(); final String peerIdStr = getPeerId(request); if (peer.parse(peerIdStr)) { diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/ReadIndexRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/ReadIndexRequestProcessor.java index 2e881f071..7d8c536ff 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/ReadIndexRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/ReadIndexRequestProcessor.java @@ -30,8 +30,7 @@ * Handle read index request. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-08 6:09:34 PM + * @author jiachun.fjc */ public class ReadIndexRequestProcessor extends NodeRequestProcessor { @@ -40,21 +39,22 @@ public ReadIndexRequestProcessor(Executor executor) { } @Override - protected String getPeerId(ReadIndexRequest request) { + protected String getPeerId(final ReadIndexRequest request) { return request.getPeerId(); } @Override - protected String getGroupId(ReadIndexRequest request) { + protected String getGroupId(final ReadIndexRequest request) { return request.getGroupId(); } @Override - public Message processRequest0(RaftServerService service, ReadIndexRequest request, RpcRequestClosure done) { + public Message processRequest0(final RaftServerService service, final ReadIndexRequest request, + final RpcRequestClosure done) { service.handleReadIndexRequest(request, new RpcResponseClosureAdapter() { @Override - public void run(Status status) { + public void run(final Status status) { if (getResponse() != null) { done.sendResponse(getResponse()); } else { diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/RequestVoteRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/RequestVoteRequestProcessor.java index 3f53fc56c..036ac169f 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/RequestVoteRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/RequestVoteRequestProcessor.java @@ -37,17 +37,18 @@ public RequestVoteRequestProcessor(Executor executor) { } @Override - protected String getPeerId(RequestVoteRequest request) { + protected String getPeerId(final RequestVoteRequest request) { return request.getPeerId(); } @Override - protected String getGroupId(RequestVoteRequest request) { + protected String getGroupId(final RequestVoteRequest request) { return request.getGroupId(); } @Override - public Message processRequest0(RaftServerService service, RequestVoteRequest request, RpcRequestClosure done) { + public Message processRequest0(final RaftServerService service, final RequestVoteRequest request, + final RpcRequestClosure done) { if (request.getPreVote()) { return service.handlePreVoteRequest(request); } else { diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/TimeoutNowRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/TimeoutNowRequestProcessor.java index 542b7018c..4994666c6 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/TimeoutNowRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/core/TimeoutNowRequestProcessor.java @@ -27,8 +27,7 @@ * TimeoutNow request processor. * * @author boyan (boyan@alibaba-inc.com) - * - * 2018-Apr-23 11:12:48 AM + * @author jiachun.fjc */ public class TimeoutNowRequestProcessor extends NodeRequestProcessor { @@ -37,17 +36,18 @@ public TimeoutNowRequestProcessor(Executor executor) { } @Override - protected String getPeerId(TimeoutNowRequest request) { + protected String getPeerId(final TimeoutNowRequest request) { return request.getPeerId(); } @Override - protected String getGroupId(TimeoutNowRequest request) { + protected String getGroupId(final TimeoutNowRequest request) { return request.getGroupId(); } @Override - public Message processRequest0(RaftServerService service, TimeoutNowRequest request, RpcRequestClosure done) { + public Message processRequest0(final RaftServerService service, final TimeoutNowRequest request, + final RpcRequestClosure done) { return service.handleTimeoutNowRequest(request, done); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/FileService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/FileService.java index ac3d5f1a9..d7ae30a68 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/FileService.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/FileService.java @@ -91,9 +91,8 @@ public Message handleGetFile(final GetFileRequest request, final RpcRequestClosu } if (LOG.isDebugEnabled()) { - LOG.debug("GetFile from {} path={} filename={} offset={} count={}", - done.getBizContext().getRemoteAddress(), reader.getPath(), request.getFilename(), request.getOffset(), - request.getCount()); + LOG.debug("GetFile from {} path={} filename={} offset={} count={}", done.getRpcCtx().getRemoteAddress(), + reader.getPath(), request.getFilename(), request.getOffset(), request.getCount()); } final ByteBufferCollector dataBuffer = ByteBufferCollector.allocate(); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/BoltSession.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/CopySession.java similarity index 98% rename from jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/BoltSession.java rename to jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/CopySession.java index 16de70098..4cca27445 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/BoltSession.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/CopySession.java @@ -50,15 +50,15 @@ import com.google.protobuf.Message; /** - * Copy session based on bolt framework. + * Copy session. * @author boyan (boyan@alibaba-inc.com) * * 2018-Apr-08 12:01:23 PM */ @ThreadSafe -public class BoltSession implements Session { +public class CopySession implements Session { - private static final Logger LOG = LoggerFactory.getLogger(BoltSession.class); + private static final Logger LOG = LoggerFactory.getLogger(CopySession.class); private final Lock lock = new ReentrantLock(); private final Status st = Status.OK(); @@ -123,7 +123,7 @@ public void close() throws IOException { } } - public BoltSession(final RaftClientService rpcService, final TimerManager timerManager, + public CopySession(final RaftClientService rpcService, final TimerManager timerManager, final SnapshotThrottle snapshotThrottle, final RaftOptions raftOptions, final GetFileRequest.Builder rb, final Endpoint ep) { super(); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/RemoteFileCopier.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/RemoteFileCopier.java index 4dce8e155..51a273d50 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/RemoteFileCopier.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/snapshot/remote/RemoteFileCopier.java @@ -139,7 +139,7 @@ public void close() throws IOException { super.close(); } }); - final BoltSession session = newBoltSession(source); + final CopySession session = newCopySession(source); session.setOutputStream(out); session.setDestPath(destPath); session.setDestBuf(null); @@ -150,11 +150,11 @@ public void close() throws IOException { return session; } - private BoltSession newBoltSession(final String source) { + private CopySession newCopySession(final String source) { final GetFileRequest.Builder reqBuilder = GetFileRequest.newBuilder() // .setFilename(source) // .setReaderId(this.readId); - return new BoltSession(this.rpcService, this.timerManager, this.snapshotThrottle, this.raftOptions, reqBuilder, + return new CopySession(this.rpcService, this.timerManager, this.snapshotThrottle, this.raftOptions, reqBuilder, this.endpoint); } @@ -180,7 +180,7 @@ public boolean copy2IoBuffer(final String source, final ByteBufferCollector dest } public Session startCopy2IoBuffer(final String source, final ByteBufferCollector destBuf, final CopyOptions opts) { - final BoltSession session = newBoltSession(source); + final CopySession session = newCopySession(source); session.setOutputStream(null); session.setDestBuf(destBuf); if (opts != null) { diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/util/RpcFactoryHelper.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/util/RpcFactoryHelper.java new file mode 100644 index 000000000..017349068 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/util/RpcFactoryHelper.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.util; + +import com.alipay.sofa.jraft.rpc.RaftRpcFactory; + +/** + * @author jiachun.fjc + */ +public class RpcFactoryHelper { + + private static final RaftRpcFactory RPC_FACTORY = JRaftServiceLoader.load(RaftRpcFactory.class) // + .first(); + + public static RaftRpcFactory getRpcFactory() { + return RPC_FACTORY; + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/util/Utils.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/util/Utils.java index c2fc21824..3e4bf986b 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/util/Utils.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/util/Utils.java @@ -41,7 +41,6 @@ import com.alipay.sofa.jraft.Closure; import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.error.RaftError; -import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory; import com.codahale.metrics.MetricRegistry; /** @@ -317,7 +316,7 @@ public static byte[] getBytes(final String s) { public static void ensureBoltPipeline() { if (RpcConfigManager.dispatch_msg_list_in_default_executor()) { System.setProperty(RpcConfigs.DISPATCH_MSG_LIST_IN_DEFAULT_EXECUTOR, "false"); - RaftRpcServerFactory.LOG.warn("JRaft SET {} to be false for replicator pipeline optimistic.", + LOG.warn("JRaft SET {} to be false for replicator pipeline optimistic.", RpcConfigs.DISPATCH_MSG_LIST_IN_DEFAULT_EXECUTOR); } } diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/ThrowUtil.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/util/internal/ThrowUtil.java similarity index 91% rename from jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/ThrowUtil.java rename to jraft-core/src/main/java/com/alipay/sofa/jraft/util/internal/ThrowUtil.java index 13218c4cf..c179a89ad 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/ThrowUtil.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/util/internal/ThrowUtil.java @@ -14,11 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.alipay.sofa.jraft.rhea.util; - -import com.alipay.sofa.jraft.util.internal.ReferenceFieldUpdater; -import com.alipay.sofa.jraft.util.internal.UnsafeUtil; -import com.alipay.sofa.jraft.util.internal.Updaters; +package com.alipay.sofa.jraft.util.internal; /** * Throwing tool. diff --git a/jraft-core/src/main/resources/META-INF/services/com.alipay.sofa.jraft.rpc.RaftRpcFactory b/jraft-core/src/main/resources/META-INF/services/com.alipay.sofa.jraft.rpc.RaftRpcFactory new file mode 100644 index 000000000..8416bc165 --- /dev/null +++ b/jraft-core/src/main/resources/META-INF/services/com.alipay.sofa.jraft.rpc.RaftRpcFactory @@ -0,0 +1 @@ +com.alipay.sofa.jraft.rpc.impl.BoltRaftRpcFactory \ No newline at end of file diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/RouteTableTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/RouteTableTest.java index 0367ea64a..1995d3bf0 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/RouteTableTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/RouteTableTest.java @@ -33,7 +33,7 @@ import com.alipay.sofa.jraft.core.TestCluster; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.option.CliOptions; -import com.alipay.sofa.jraft.rpc.impl.cli.BoltCliClientService; +import com.alipay.sofa.jraft.rpc.impl.cli.CliClientServiceImpl; import com.alipay.sofa.jraft.test.TestUtils; import static org.junit.Assert.assertArrayEquals; @@ -52,11 +52,11 @@ public class RouteTableTest { private TestCluster cluster; private final String groupId = "RouteTableTest"; - BoltCliClientService cliClientService; + CliClientServiceImpl cliClientService; @Before public void setup() throws Exception { - cliClientService = new BoltCliClientService(); + cliClientService = new CliClientServiceImpl(); cliClientService.init(new CliOptions()); this.dataPath = TestUtils.mkTempDir(); FileUtils.forceMkdir(new File(this.dataPath)); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/CliServiceTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/CliServiceTest.java index bb84b353b..377f0962f 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/CliServiceTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/CliServiceTest.java @@ -211,7 +211,7 @@ public void testLearnerServices() throws Exception { // Stop learner3 this.cluster.stop(learner3.getEndpoint()); - Thread.sleep(100); + Thread.sleep(1000); assertEquals(Arrays.asList(learner3), this.cliService.getLearners(this.groupId, this.conf)); assertTrue(this.cliService.getAliveLearners(this.groupId, this.conf).isEmpty()); } diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java index d0571317f..f54ee2cce 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java @@ -44,7 +44,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.Iterator; import com.alipay.sofa.jraft.JRaftUtils; import com.alipay.sofa.jraft.Node; @@ -69,6 +68,7 @@ import com.alipay.sofa.jraft.option.NodeOptions; import com.alipay.sofa.jraft.option.RaftOptions; import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; import com.alipay.sofa.jraft.storage.SnapshotThrottle; import com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage; import com.alipay.sofa.jraft.storage.snapshot.SnapshotReader; diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/TestCluster.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/TestCluster.java index 099962b94..aa1bf5e26 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/TestCluster.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/TestCluster.java @@ -33,7 +33,6 @@ import org.apache.commons.io.FileUtils; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.JRaftServiceFactory; import com.alipay.sofa.jraft.Node; import com.alipay.sofa.jraft.RaftGroupService; @@ -42,6 +41,7 @@ import com.alipay.sofa.jraft.option.NodeOptions; import com.alipay.sofa.jraft.option.RaftOptions; import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; import com.alipay.sofa.jraft.storage.SnapshotThrottle; import com.alipay.sofa.jraft.util.Endpoint; diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/AbstractBoltClientServiceTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/AbstractClientServiceTest.java similarity index 72% rename from jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/AbstractBoltClientServiceTest.java rename to jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/AbstractClientServiceTest.java index 541725b32..bd2eded9d 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/AbstractBoltClientServiceTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/AbstractClientServiceTest.java @@ -28,19 +28,14 @@ import org.mockito.Mockito; import org.mockito.runners.MockitoJUnitRunner; -import com.alipay.remoting.InvokeCallback; -import com.alipay.remoting.InvokeContext; -import com.alipay.remoting.Url; -import com.alipay.remoting.exception.RemotingException; -import com.alipay.remoting.rpc.RpcClient; -import com.alipay.remoting.rpc.exception.InvokeTimeoutException; import com.alipay.sofa.jraft.Status; +import com.alipay.sofa.jraft.error.InvokeTimeoutException; import com.alipay.sofa.jraft.error.RaftError; +import com.alipay.sofa.jraft.error.RemotingException; import com.alipay.sofa.jraft.option.RpcOptions; import com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse; import com.alipay.sofa.jraft.rpc.RpcRequests.PingRequest; -import com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService; -import com.alipay.sofa.jraft.rpc.impl.core.JRaftRpcAddressParser; +import com.alipay.sofa.jraft.rpc.impl.AbstractClientService; import com.alipay.sofa.jraft.test.TestUtils; import com.alipay.sofa.jraft.util.Endpoint; import com.google.protobuf.Message; @@ -55,24 +50,23 @@ import static org.mockito.Matchers.eq; @RunWith(value = MockitoJUnitRunner.class) -public class AbstractBoltClientServiceTest { - static class MockBoltClientService extends AbstractBoltClientService { +public class AbstractClientServiceTest { + static class MockClientService extends AbstractClientService { public void setRpcClient(final RpcClient rpcClient) { this.rpcClient = rpcClient; } } - private RpcOptions rpcOptions; - private MockBoltClientService clientService; + private RpcOptions rpcOptions; + private MockClientService clientService; @Mock - private RpcClient rpcClient; - private final JRaftRpcAddressParser rpcAddressParser = new JRaftRpcAddressParser(); - private final Endpoint endpoint = new Endpoint("localhost", 8081); + private RpcClient rpcClient; + private final Endpoint endpoint = new Endpoint("localhost", 8081); @Before public void setup() { this.rpcOptions = new RpcOptions(); - this.clientService = new MockBoltClientService(); + this.clientService = new MockClientService(); assertTrue(this.clientService.init(this.rpcOptions)); this.clientService.setRpcClient(this.rpcClient); @@ -81,32 +75,34 @@ public void setup() { @Test public void testConnect() throws Exception { Mockito.when( - this.rpcClient.invokeSync(eq(this.endpoint.toString()), Mockito.any(), Mockito.any(), - eq(this.rpcOptions.getRpcConnectTimeoutMs()))).thenReturn(RpcResponseFactory.newResponse(Status.OK())); + this.rpcClient.invokeSync(eq(this.endpoint), Mockito.any(), + eq((long) this.rpcOptions.getRpcConnectTimeoutMs()))) // + .thenReturn(RpcResponseFactory.newResponse(Status.OK())); assertTrue(this.clientService.connect(this.endpoint)); } @Test public void testConnectFailure() throws Exception { Mockito.when( - this.rpcClient.invokeSync(eq(this.endpoint.toString()), Mockito.any(), Mockito.any(), - eq(this.rpcOptions.getRpcConnectTimeoutMs()))).thenReturn( - RpcResponseFactory.newResponse(new Status(-1, "test"))); + this.rpcClient.invokeSync(eq(this.endpoint), Mockito.any(), + eq((long) this.rpcOptions.getRpcConnectTimeoutMs()))) // + .thenReturn(RpcResponseFactory.newResponse(new Status(-1, "test"))); assertFalse(this.clientService.connect(this.endpoint)); } @Test public void testConnectException() throws Exception { Mockito.when( - this.rpcClient.invokeSync(eq(this.endpoint.toString()), Mockito.any(), Mockito.any(), - eq(this.rpcOptions.getRpcConnectTimeoutMs()))).thenThrow(new RemotingException("test")); + this.rpcClient.invokeSync(eq(this.endpoint), Mockito.any(), + eq((long) this.rpcOptions.getRpcConnectTimeoutMs()))) // + .thenThrow(new RemotingException("test")); assertFalse(this.clientService.connect(this.endpoint)); } @Test public void testDisconnect() { this.clientService.disconnect(this.endpoint); - Mockito.verify(this.rpcClient).closeConnection(this.endpoint.toString()); + Mockito.verify(this.rpcClient).closeConnection(this.endpoint); } static class MockRpcResponseClosure extends RpcResponseClosureAdapter { @@ -130,9 +126,8 @@ public void testCancel() throws Exception { MockRpcResponseClosure done = new MockRpcResponseClosure<>(); Future future = this.clientService.invokeWithDone(this.endpoint, request, done, -1); - Url rpcUrl = this.rpcAddressParser.parse(this.endpoint.toString()); - Mockito.verify(this.rpcClient).invokeWithCallback(eq(rpcUrl), eq(request), Mockito.any(), - callbackArg.capture(), eq(this.rpcOptions.getRpcDefaultTimeout())); + Mockito.verify(this.rpcClient).invokeAsync(eq(this.endpoint), eq(request), Mockito.any(), + callbackArg.capture(), eq((long) this.rpcOptions.getRpcDefaultTimeout())); InvokeCallback cb = callbackArg.getValue(); assertNotNull(cb); assertNotNull(future); @@ -143,7 +138,7 @@ public void testCancel() throws Exception { future.cancel(true); ErrorResponse response = RpcResponseFactory.newResponse(Status.OK()); - cb.onResponse(response); + cb.complete(response, null); // The closure should be notified with ECANCELED error code. done.latch.await(); @@ -158,9 +153,8 @@ public void testInvokeWithDoneOK() throws Exception { MockRpcResponseClosure done = new MockRpcResponseClosure<>(); Future future = this.clientService.invokeWithDone(this.endpoint, request, done, -1); - Url rpcUrl = this.rpcAddressParser.parse(this.endpoint.toString()); - Mockito.verify(this.rpcClient).invokeWithCallback(eq(rpcUrl), eq(request), Mockito.any(), - callbackArg.capture(), eq(this.rpcOptions.getRpcDefaultTimeout())); + Mockito.verify(this.rpcClient).invokeAsync(eq(this.endpoint), eq(request), Mockito.any(), + callbackArg.capture(), eq((long) this.rpcOptions.getRpcDefaultTimeout())); InvokeCallback cb = callbackArg.getValue(); assertNotNull(cb); assertNotNull(future); @@ -170,7 +164,7 @@ public void testInvokeWithDoneOK() throws Exception { assertFalse(future.isDone()); ErrorResponse response = RpcResponseFactory.newResponse(Status.OK()); - cb.onResponse(response); + cb.complete(response, null); Message msg = future.get(); assertNotNull(msg); @@ -185,16 +179,15 @@ public void testInvokeWithDoneOK() throws Exception { @Test public void testInvokeWithDoneException() throws Exception { InvokeContext invokeCtx = new InvokeContext(); - invokeCtx.put(InvokeContext.BOLT_CRC_SWITCH, false); + invokeCtx.put(InvokeContext.CRC_SWITCH, false); ArgumentCaptor callbackArg = ArgumentCaptor.forClass(InvokeCallback.class); PingRequest request = TestUtils.createPingRequest(); - Url rpcUrl = this.rpcAddressParser.parse(this.endpoint.toString()); Mockito .doThrow(new RemotingException()) .when(this.rpcClient) - .invokeWithCallback(eq(rpcUrl), eq(request), eq(invokeCtx), callbackArg.capture(), - eq(this.rpcOptions.getRpcDefaultTimeout())); + .invokeAsync(eq(this.endpoint), eq(request), eq(invokeCtx), callbackArg.capture(), + eq((long) this.rpcOptions.getRpcDefaultTimeout())); MockRpcResponseClosure done = new MockRpcResponseClosure<>(); Future future = this.clientService.invokeWithDone(this.endpoint, request, invokeCtx, done, -1); @@ -219,15 +212,14 @@ public void testInvokeWithDoneException() throws Exception { @Test public void testInvokeWithDoneOnException() throws Exception { InvokeContext invokeCtx = new InvokeContext(); - invokeCtx.put(InvokeContext.BOLT_CRC_SWITCH, false); + invokeCtx.put(InvokeContext.CRC_SWITCH, false); ArgumentCaptor callbackArg = ArgumentCaptor.forClass(InvokeCallback.class); PingRequest request = TestUtils.createPingRequest(); MockRpcResponseClosure done = new MockRpcResponseClosure<>(); Future future = this.clientService.invokeWithDone(this.endpoint, request, invokeCtx, done, -1); - Url rpcUrl = this.rpcAddressParser.parse(this.endpoint.toString()); - Mockito.verify(this.rpcClient).invokeWithCallback(eq(rpcUrl), eq(request), eq(invokeCtx), - callbackArg.capture(), eq(this.rpcOptions.getRpcDefaultTimeout())); + Mockito.verify(this.rpcClient).invokeAsync(eq(this.endpoint), eq(request), eq(invokeCtx), + callbackArg.capture(), eq((long) this.rpcOptions.getRpcDefaultTimeout())); InvokeCallback cb = callbackArg.getValue(); assertNotNull(cb); assertNotNull(future); @@ -236,7 +228,7 @@ public void testInvokeWithDoneOnException() throws Exception { assertNull(done.status); assertFalse(future.isDone()); - cb.onException(new InvokeTimeoutException()); + cb.complete(null, new InvokeTimeoutException()); try { future.get(); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/PingRequestProcessorTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/PingRequestProcessorTest.java index ee931708f..e7d137e43 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/PingRequestProcessorTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/PingRequestProcessorTest.java @@ -19,6 +19,7 @@ import org.junit.Test; import com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse; +import com.alipay.sofa.jraft.test.MockAsyncContext; import com.alipay.sofa.jraft.test.TestUtils; import static org.junit.Assert.assertEquals; @@ -28,7 +29,9 @@ public class PingRequestProcessorTest { @Test public void testHandlePing() throws Exception { PingRequestProcessor processor = new PingRequestProcessor(); - ErrorResponse response = (ErrorResponse) processor.handleRequest(null, TestUtils.createPingRequest()); + MockAsyncContext ctx = new MockAsyncContext(); + processor.handleRequest(ctx, TestUtils.createPingRequest()); + ErrorResponse response = (ErrorResponse) ctx.getResponseObject(); assertEquals(0, response.getErrorCode()); } } diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/cli/AbstractCliRequestProcessorTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/cli/AbstractCliRequestProcessorTest.java index 6f9bda2d5..c1217bb45 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/cli/AbstractCliRequestProcessorTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/cli/AbstractCliRequestProcessorTest.java @@ -28,7 +28,6 @@ import org.mockito.Mockito; import org.mockito.runners.MockitoJUnitRunner; -import com.alipay.remoting.BizContext; import com.alipay.sofa.jraft.Closure; import com.alipay.sofa.jraft.JRaftUtils; import com.alipay.sofa.jraft.Node; @@ -46,8 +45,6 @@ public abstract class AbstractCliRequestProcessorTest { private final String groupId = "test"; private final String peerIdStr = "localhost:8081"; protected MockAsyncContext asyncContext; - @Mock - protected BizContext bizContext; public abstract T createRequest(String groupId, PeerId peerId); @@ -71,7 +68,6 @@ public void mockNodes(final int n) { @Before public void setup() { this.asyncContext = new MockAsyncContext(); - Mockito.when(this.bizContext.getRemoteAddress()).thenReturn("localhost:12345"); } @After @@ -91,7 +87,7 @@ public void testHandleRequest() { NodeManager.getInstance().add(this.node); BaseCliRequestProcessor processor = newProcessor(); - processor.handleRequest(this.bizContext, this.asyncContext, createRequest(this.groupId, peerId)); + processor.handleRequest(this.asyncContext, createRequest(this.groupId, peerId)); ArgumentCaptor doneArg = ArgumentCaptor.forClass(Closure.class); verify(processor.interest(), this.node, doneArg); } diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/cli/BaseCliRequestProcessorTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/cli/BaseCliRequestProcessorTest.java index 817daceb0..7428c0d65 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/cli/BaseCliRequestProcessorTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/cli/BaseCliRequestProcessorTest.java @@ -103,7 +103,7 @@ public void teardown() { public void testOK() { Node node = mockNode(false); - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(this.processor.done); assertSame(this.processor.ctx.node, node); @@ -115,7 +115,7 @@ public void testOK() { public void testDisableCli() { mockNode(true); - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(resp); assertEquals(RaftError.EACCES.getNumber(), resp.getErrorCode()); @@ -137,7 +137,7 @@ private Node mockNode(boolean disableCli) { @Test public void testInvalidPeerId() { this.processor = new MockCliRequestProcessor("localhost", "test"); - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(resp); assertEquals(RaftError.EINVAL.getNumber(), resp.getErrorCode()); @@ -147,7 +147,7 @@ public void testInvalidPeerId() { @Test public void testEmptyNodes() { this.processor = new MockCliRequestProcessor(null, "test"); - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(resp); assertEquals(RaftError.ENOENT.getNumber(), resp.getErrorCode()); @@ -172,7 +172,7 @@ public void testManyNodes() { NodeManager.getInstance().add(node2); this.processor = new MockCliRequestProcessor(null, "test"); - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(resp); assertEquals(RaftError.EINVAL.getNumber(), resp.getErrorCode()); @@ -183,7 +183,7 @@ public void testManyNodes() { public void testSingleNode() { Node node = this.mockNode(false); this.processor = new MockCliRequestProcessor(null, "test"); - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(resp); assertSame(this.processor.ctx.node, node); @@ -193,7 +193,7 @@ public void testSingleNode() { @Test public void testPeerIdNotFound() { - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(resp); assertEquals(RaftError.ENOENT.getNumber(), resp.getErrorCode()); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/AppendEntriesRequestProcessorTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/AppendEntriesRequestProcessorTest.java index 0166ccfba..9463a18f2 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/AppendEntriesRequestProcessorTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/AppendEntriesRequestProcessorTest.java @@ -23,15 +23,15 @@ import org.mockito.Mock; import org.mockito.Mockito; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.Connection; import com.alipay.sofa.jraft.NodeManager; import com.alipay.sofa.jraft.entity.PeerId; +import com.alipay.sofa.jraft.rpc.Connection; import com.alipay.sofa.jraft.rpc.RaftServerService; +import com.alipay.sofa.jraft.rpc.RpcContext; import com.alipay.sofa.jraft.rpc.RpcRequests.AppendEntriesRequest; import com.alipay.sofa.jraft.rpc.RpcRequests.PingRequest; import com.alipay.sofa.jraft.rpc.impl.core.AppendEntriesRequestProcessor.PeerRequestContext; +import com.alipay.sofa.jraft.test.MockAsyncContext; import com.alipay.sofa.jraft.test.TestUtils; import static org.junit.Assert.assertEquals; @@ -63,7 +63,12 @@ public AppendEntriesRequest createRequest(String groupId, PeerId peerId) { @Override public void setup() { super.setup(); - Mockito.when(this.bizContext.getConnection()).thenReturn(this.conn); + this.asyncContext = new MockAsyncContext() { + @Override + public Connection getConnection() { + return conn; + } + }; Mockito.when(this.conn.getAttribute(AppendEntriesRequestProcessor.PEER_ATTR)).thenReturn(this.peerIdStr); } @@ -124,14 +129,13 @@ public void testGetPeerRequestContextRemovePeerRequestContext() { public void testSendSequenceResponse() { mockNode(); - final AsyncContext asyncContext = Mockito.mock(AsyncContext.class); - final BizContext bizContext = Mockito.mock(BizContext.class); + final RpcContext asyncContext = Mockito.mock(RpcContext.class); final AppendEntriesRequestProcessor processor = (AppendEntriesRequestProcessor) newProcessor(); final PingRequest msg = TestUtils.createPingRequest(); - processor.sendSequenceResponse(groupId, peerIdStr, 1, asyncContext, bizContext, msg); + processor.sendSequenceResponse(groupId, peerIdStr, 1, asyncContext, msg); Mockito.verify(asyncContext, Mockito.never()).sendResponse(msg); - processor.sendSequenceResponse(groupId, peerIdStr, 0, asyncContext, bizContext, msg); + processor.sendSequenceResponse(groupId, peerIdStr, 0, asyncContext, msg); Mockito.verify(asyncContext, Mockito.times(2)).sendResponse(msg); } @@ -140,17 +144,16 @@ public void testTooManyPendingResponses() { final PeerId peer = this.mockNode(); NodeManager.getInstance().get(groupId, peer).getRaftOptions().setMaxReplicatorInflightMsgs(2); - final AsyncContext asyncContext = Mockito.mock(AsyncContext.class); - final BizContext bizContext = Mockito.mock(BizContext.class); + final RpcContext asyncContext = Mockito.mock(RpcContext.class); final AppendEntriesRequestProcessor processor = (AppendEntriesRequestProcessor) newProcessor(); final PingRequest msg = TestUtils.createPingRequest(); final Connection conn = Mockito.mock(Connection.class); - Mockito.when(bizContext.getConnection()).thenReturn(conn); + Mockito.when(asyncContext.getConnection()).thenReturn(conn); final PeerRequestContext ctx = processor.getPeerRequestContext(groupId, peerIdStr, conn); assertNotNull(ctx); - processor.sendSequenceResponse(groupId, peerIdStr, 1, asyncContext, bizContext, msg); - processor.sendSequenceResponse(groupId, peerIdStr, 2, asyncContext, bizContext, msg); - processor.sendSequenceResponse(groupId, peerIdStr, 3, asyncContext, bizContext, msg); + processor.sendSequenceResponse(groupId, peerIdStr, 1, asyncContext, msg); + processor.sendSequenceResponse(groupId, peerIdStr, 2, asyncContext, msg); + processor.sendSequenceResponse(groupId, peerIdStr, 3, asyncContext, msg); Mockito.verify(asyncContext, Mockito.never()).sendResponse(msg); Mockito.verify(conn).close(); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/BaseNodeRequestProcessorTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/BaseNodeRequestProcessorTest.java index 37857a886..c0161e7ae 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/BaseNodeRequestProcessorTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/BaseNodeRequestProcessorTest.java @@ -24,7 +24,6 @@ import org.mockito.Mockito; import org.mockito.runners.MockitoJUnitRunner; -import com.alipay.remoting.BizContext; import com.alipay.sofa.jraft.Node; import com.alipay.sofa.jraft.NodeManager; import com.alipay.sofa.jraft.entity.NodeId; @@ -37,12 +36,10 @@ @RunWith(value = MockitoJUnitRunner.class) public abstract class BaseNodeRequestProcessorTest { @Mock(extraInterfaces = { RaftServerService.class }) - private Node node; - protected final String groupId = "test"; - protected final String peerIdStr = "localhost:8081"; - private MockAsyncContext asyncContext; - @Mock - protected BizContext bizContext; + private Node node; + protected final String groupId = "test"; + protected final String peerIdStr = "localhost:8081"; + protected MockAsyncContext asyncContext; public abstract T createRequest(String groupId, PeerId peerId); @@ -52,7 +49,6 @@ public abstract class BaseNodeRequestProcessorTest { @Before public void setup() { - this.asyncContext = new MockAsyncContext(); Mockito.when(node.getRaftOptions()).thenReturn(new RaftOptions()); } @@ -66,7 +62,7 @@ public void testHandleRequest() { final PeerId peerId = mockNode(); final NodeRequestProcessor processor = newProcessor(); - processor.handleRequest(bizContext, asyncContext, createRequest(groupId, peerId)); + processor.handleRequest(asyncContext, createRequest(groupId, peerId)); verify(processor.interest(), (RaftServerService) this.node, processor); } diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/NodeRequestProcessorTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/NodeRequestProcessorTest.java index b1565cb12..b461af7d9 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/NodeRequestProcessorTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/rpc/impl/core/NodeRequestProcessorTest.java @@ -98,7 +98,7 @@ public void testOK() { NodeManager.getInstance().addAddress(peerId.getEndpoint()); NodeManager.getInstance().add(node); - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(resp); assertEquals(0, resp.getErrorCode()); @@ -107,7 +107,7 @@ public void testOK() { @Test public void testInvalidPeerId() { this.processor = new MockRequestProcessor("localhost", "test"); - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(resp); assertEquals(RaftError.EINVAL.getNumber(), resp.getErrorCode()); @@ -116,7 +116,7 @@ public void testInvalidPeerId() { @Test public void testPeerIdNotFound() { - this.processor.handleRequest(null, asyncContext, TestUtils.createPingRequest()); + this.processor.handleRequest(asyncContext, TestUtils.createPingRequest()); ErrorResponse resp = (ErrorResponse) asyncContext.getResponseObject(); assertNotNull(resp); assertEquals(RaftError.ENOENT.getNumber(), resp.getErrorCode()); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/FileServiceTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/FileServiceTest.java index cc1d157e2..5ce272645 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/FileServiceTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/FileServiceTest.java @@ -26,9 +26,8 @@ import org.junit.Test; import org.mockito.Mockito; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; import com.alipay.sofa.jraft.error.RaftError; +import com.alipay.sofa.jraft.rpc.RpcContext; import com.alipay.sofa.jraft.rpc.RpcRequestClosure; import com.alipay.sofa.jraft.rpc.RpcRequests; import com.alipay.sofa.jraft.storage.io.LocalDirReader; @@ -67,9 +66,8 @@ public void testAddRemove() { public void testGetFileNotFoundReader() { RpcRequests.GetFileRequest request = RpcRequests.GetFileRequest.newBuilder().setCount(Integer.MAX_VALUE) .setFilename("data").setOffset(0).setReaderId(1).build(); - BizContext bizContext = Mockito.mock(BizContext.class); - AsyncContext asyncContext = Mockito.mock(AsyncContext.class); - Message msg = FileService.getInstance().handleGetFile(request, new RpcRequestClosure(bizContext, asyncContext)); + RpcContext asyncContext = Mockito.mock(RpcContext.class); + Message msg = FileService.getInstance().handleGetFile(request, new RpcRequestClosure(asyncContext)); assertTrue(msg instanceof RpcRequests.ErrorResponse); RpcRequests.ErrorResponse response = (RpcRequests.ErrorResponse) msg; Assert.assertEquals(RaftError.ENOENT.getNumber(), response.getErrorCode()); @@ -81,9 +79,8 @@ public void testGetFileNotFound() { long readerId = FileService.getInstance().addReader(this.fileReader); RpcRequests.GetFileRequest request = RpcRequests.GetFileRequest.newBuilder().setCount(Integer.MAX_VALUE) .setFilename("data").setOffset(0).setReaderId(readerId).build(); - BizContext bizContext = Mockito.mock(BizContext.class); - AsyncContext asyncContext = Mockito.mock(AsyncContext.class); - Message msg = FileService.getInstance().handleGetFile(request, new RpcRequestClosure(bizContext, asyncContext)); + RpcContext asyncContext = Mockito.mock(RpcContext.class); + Message msg = FileService.getInstance().handleGetFile(request, new RpcRequestClosure(asyncContext)); assertTrue(msg instanceof RpcRequests.ErrorResponse); RpcRequests.ErrorResponse response = (RpcRequests.ErrorResponse) msg; assertEquals(RaftError.EIO.getNumber(), response.getErrorCode()); @@ -103,9 +100,8 @@ public void testGetFileData() throws IOException { long readerId = FileService.getInstance().addReader(this.fileReader); RpcRequests.GetFileRequest request = RpcRequests.GetFileRequest.newBuilder().setCount(Integer.MAX_VALUE) .setFilename("data").setOffset(0).setReaderId(readerId).build(); - BizContext bizContext = Mockito.mock(BizContext.class); - AsyncContext asyncContext = Mockito.mock(AsyncContext.class); - Message msg = FileService.getInstance().handleGetFile(request, new RpcRequestClosure(bizContext, asyncContext)); + RpcContext asyncContext = Mockito.mock(RpcContext.class); + Message msg = FileService.getInstance().handleGetFile(request, new RpcRequestClosure(asyncContext)); assertTrue(msg instanceof RpcRequests.GetFileResponse); RpcRequests.GetFileResponse response = (RpcRequests.GetFileResponse) msg; assertTrue(response.getEof()); @@ -133,10 +129,9 @@ public void testGetLargeFileData() throws IOException { .setOffset(fileOffset) // .setReaderId(readerId) // .build(); - final BizContext bizContext = Mockito.mock(BizContext.class); - final AsyncContext asyncContext = Mockito.mock(AsyncContext.class); + final RpcContext asyncContext = Mockito.mock(RpcContext.class); final Message msg = FileService.getInstance() // - .handleGetFile(request, new RpcRequestClosure(bizContext, asyncContext)); + .handleGetFile(request, new RpcRequestClosure(asyncContext)); assertTrue(msg instanceof RpcRequests.GetFileResponse); final RpcRequests.GetFileResponse response = (RpcRequests.GetFileResponse) msg; final byte[] sourceArray = data.getBytes(); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/SnapshotExecutorTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/SnapshotExecutorTest.java index 94547c631..3afe8dd56 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/SnapshotExecutorTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/SnapshotExecutorTest.java @@ -27,8 +27,6 @@ import org.mockito.Mockito; import org.mockito.runners.MockitoJUnitRunner; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; import com.alipay.sofa.jraft.FSMCaller; import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.closure.LoadSnapshotClosure; @@ -44,6 +42,7 @@ import com.alipay.sofa.jraft.option.RaftOptions; import com.alipay.sofa.jraft.option.SnapshotExecutorOptions; import com.alipay.sofa.jraft.rpc.RaftClientService; +import com.alipay.sofa.jraft.rpc.RpcContext; import com.alipay.sofa.jraft.rpc.RpcRequestClosure; import com.alipay.sofa.jraft.rpc.RpcRequests; import com.alipay.sofa.jraft.rpc.RpcResponseClosure; @@ -76,9 +75,7 @@ public class SnapshotExecutorTest extends BaseStorageTest { private LogManager logManager; private Endpoint addr; @Mock - private BizContext bizCtx; - @Mock - private AsyncContext asyncCtx; + private RpcContext asyncCtx; @Mock private RaftClientService raftClientService; @@ -166,8 +163,7 @@ public void testInstallSnapshot() throws Exception { @Override public void run() { SnapshotExecutorTest.this.executor.installSnapshot(irb.build(), RpcRequests.InstallSnapshotResponse - .newBuilder(), new RpcRequestClosure(SnapshotExecutorTest.this.bizCtx, - SnapshotExecutorTest.this.asyncCtx)); + .newBuilder(), new RpcRequestClosure(SnapshotExecutorTest.this.asyncCtx)); } }); @@ -235,8 +231,7 @@ public void testInterruptInstallaling() throws Exception { @Override public void run() { SnapshotExecutorTest.this.executor.installSnapshot(irb.build(), RpcRequests.InstallSnapshotResponse - .newBuilder(), new RpcRequestClosure(SnapshotExecutorTest.this.bizCtx, - SnapshotExecutorTest.this.asyncCtx)); + .newBuilder(), new RpcRequestClosure(SnapshotExecutorTest.this.asyncCtx)); } }); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/snapshot/remote/BoltSessionTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/snapshot/remote/CopySessionTest.java similarity index 97% rename from jraft-core/src/test/java/com/alipay/sofa/jraft/storage/snapshot/remote/BoltSessionTest.java rename to jraft-core/src/test/java/com/alipay/sofa/jraft/storage/snapshot/remote/CopySessionTest.java index f2e9ec1de..38b6c1b03 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/snapshot/remote/BoltSessionTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/snapshot/remote/CopySessionTest.java @@ -46,8 +46,8 @@ import static org.junit.Assert.assertSame; @RunWith(value = MockitoJUnitRunner.class) -public class BoltSessionTest { - private BoltSession session; +public class CopySessionTest { + private CopySession session; @Mock private RaftClientService rpcService; private RpcRequests.GetFileRequest.Builder rb; @@ -65,7 +65,7 @@ public void setup() { this.rb.setReaderId(99); this.rb.setFilename("data"); this.raftOpts = new RaftOptions(); - this.session = new BoltSession(rpcService, timerManager, null, raftOpts, rb, address); + this.session = new CopySession(rpcService, timerManager, null, raftOpts, rb, address); this.session.setCopyOptions(copyOpts); } diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/test/MockAsyncContext.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/test/MockAsyncContext.java index 73d0c2147..0f07dae31 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/test/MockAsyncContext.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/test/MockAsyncContext.java @@ -16,7 +16,8 @@ */ package com.alipay.sofa.jraft.test; -import com.alipay.remoting.AsyncContext; +import com.alipay.sofa.jraft.rpc.Connection; +import com.alipay.sofa.jraft.rpc.RpcContext; import com.google.protobuf.Message; /** @@ -25,7 +26,7 @@ * * 2018-Apr-19 6:05:46 PM */ -public class MockAsyncContext implements AsyncContext { +public class MockAsyncContext implements RpcContext { private Object responseObject; public Object getResponseObject() { @@ -47,4 +48,14 @@ public void sendResponse(Object responseObject) { } + @Override + public Connection getConnection() { + return null; + } + + @Override + public String getRemoteAddress() { + return "localhost:12345"; + } + } diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterClient.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterClient.java index c58a210b0..4fbfb2eff 100644 --- a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterClient.java +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterClient.java @@ -19,14 +19,14 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; -import com.alipay.remoting.InvokeCallback; -import com.alipay.remoting.exception.RemotingException; import com.alipay.sofa.jraft.RouteTable; import com.alipay.sofa.jraft.conf.Configuration; import com.alipay.sofa.jraft.entity.PeerId; +import com.alipay.sofa.jraft.error.RemotingException; import com.alipay.sofa.jraft.example.counter.rpc.IncrementAndGetRequest; import com.alipay.sofa.jraft.option.CliOptions; -import com.alipay.sofa.jraft.rpc.impl.cli.BoltCliClientService; +import com.alipay.sofa.jraft.rpc.InvokeCallback; +import com.alipay.sofa.jraft.rpc.impl.cli.CliClientServiceImpl; public class CounterClient { @@ -47,7 +47,7 @@ public static void main(final String[] args) throws Exception { RouteTable.getInstance().updateConfiguration(groupId, conf); - final BoltCliClientService cliClientService = new BoltCliClientService(); + final CliClientServiceImpl cliClientService = new CliClientServiceImpl(); cliClientService.init(new CliOptions()); if (!RouteTable.getInstance().refreshLeader(cliClientService, groupId, 1000).isOk()) { @@ -67,32 +67,29 @@ public static void main(final String[] args) throws Exception { System.exit(0); } - private static void incrementAndGet(final BoltCliClientService cliClientService, final PeerId leader, + private static void incrementAndGet(final CliClientServiceImpl cliClientService, final PeerId leader, final long delta, CountDownLatch latch) throws RemotingException, InterruptedException { final IncrementAndGetRequest request = new IncrementAndGetRequest(); request.setDelta(delta); - cliClientService.getRpcClient().invokeWithCallback(leader.getEndpoint().toString(), request, - new InvokeCallback() { + cliClientService.getRpcClient().invokeAsync(leader.getEndpoint(), request, new InvokeCallback() { - @Override - public void onResponse(Object result) { + @Override + public void complete(Object result, Throwable err) { + if (err == null) { latch.countDown(); System.out.println("incrementAndGet result:" + result); - } - - @Override - public void onException(Throwable e) { - e.printStackTrace(); + } else { + err.printStackTrace(); latch.countDown(); - } + } - @Override - public Executor getExecutor() { - return null; - } - }, 5000); + @Override + public Executor executor() { + return null; + } + }, 5000); } } diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterServer.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterServer.java index 04491e252..9be0cdaae 100644 --- a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterServer.java +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterServer.java @@ -21,7 +21,6 @@ import org.apache.commons.io.FileUtils; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.Node; import com.alipay.sofa.jraft.RaftGroupService; import com.alipay.sofa.jraft.conf.Configuration; @@ -31,6 +30,7 @@ import com.alipay.sofa.jraft.example.counter.rpc.ValueResponse; import com.alipay.sofa.jraft.option.NodeOptions; import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; /** * Counter server that keeps a counter value in a raft group. @@ -51,12 +51,11 @@ public CounterServer(final String dataPath, final String groupId, final PeerId s FileUtils.forceMkdir(new File(dataPath)); // 这里让 raft RPC 和业务 RPC 使用同一个 RPC server, 通常也可以分开 - final RpcServer rpcServer = new RpcServer(serverId.getPort()); - RaftRpcServerFactory.addRaftRequestProcessors(rpcServer); + final RpcServer rpcServer = RaftRpcServerFactory.createRaftRpcServer(serverId.getEndpoint()); // 注册业务处理器 CounterService counterService = new CounterServiceImpl(this); - rpcServer.registerUserProcessor(new GetValueRequestProcessor(counterService)); - rpcServer.registerUserProcessor(new IncrementAndGetRequestProcessor(counterService)); + rpcServer.registerProcessor(new GetValueRequestProcessor(counterService)); + rpcServer.registerProcessor(new IncrementAndGetRequestProcessor(counterService)); // 初始化状态机 this.fsm = new CounterStateMachine(); // 设置状态机到启动参数 diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/rpc/GetValueRequestProcessor.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/rpc/GetValueRequestProcessor.java index 92cddb2b1..273330c15 100644 --- a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/rpc/GetValueRequestProcessor.java +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/rpc/GetValueRequestProcessor.java @@ -16,18 +16,11 @@ */ package com.alipay.sofa.jraft.example.counter.rpc; -import java.util.concurrent.CompletableFuture; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.rpc.protocol.AsyncUserProcessor; import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.example.counter.CounterClosure; import com.alipay.sofa.jraft.example.counter.CounterService; -import com.alipay.sofa.jraft.rhea.client.FutureHelper; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; /** * GetValueRequest processor. @@ -36,9 +29,7 @@ * * 2018-Apr-09 5:48:33 PM */ -public class GetValueRequestProcessor extends AsyncUserProcessor { - - private static final Logger LOG = LoggerFactory.getLogger(GetValueRequestProcessor.class); +public class GetValueRequestProcessor implements RpcProcessor { private final CounterService counterService; @@ -48,11 +39,11 @@ public GetValueRequestProcessor(CounterService counterService) { } @Override - public void handleRequest(BizContext bizCtx, AsyncContext asyncCtx, GetValueRequest request) { + public void handleRequest(final RpcContext rpcCtx, final GetValueRequest request) { final CounterClosure closure = new CounterClosure() { @Override public void run(Status status) { - asyncCtx.sendResponse(getValueResponse()); + rpcCtx.sendResponse(getValueResponse()); } }; diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/rpc/IncrementAndGetRequestProcessor.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/rpc/IncrementAndGetRequestProcessor.java index 90cff6c21..c99573a4f 100644 --- a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/rpc/IncrementAndGetRequestProcessor.java +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/rpc/IncrementAndGetRequestProcessor.java @@ -16,15 +16,11 @@ */ package com.alipay.sofa.jraft.example.counter.rpc; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.rpc.protocol.AsyncUserProcessor; import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.example.counter.CounterClosure; import com.alipay.sofa.jraft.example.counter.CounterService; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; /** * IncrementAndGetRequest processor. @@ -33,9 +29,7 @@ * * 2018-Apr-09 5:43:57 PM */ -public class IncrementAndGetRequestProcessor extends AsyncUserProcessor { - - private static final Logger LOG = LoggerFactory.getLogger(IncrementAndGetRequestProcessor.class); +public class IncrementAndGetRequestProcessor implements RpcProcessor { private final CounterService counterService; @@ -45,11 +39,11 @@ public IncrementAndGetRequestProcessor(CounterService counterService) { } @Override - public void handleRequest(final BizContext bizCtx, final AsyncContext asyncCtx, final IncrementAndGetRequest request) { + public void handleRequest(final RpcContext rpcCtx, final IncrementAndGetRequest request) { final CounterClosure closure = new CounterClosure() { @Override public void run(Status status) { - asyncCtx.sendResponse(getValueResponse()); + rpcCtx.sendResponse(getValueResponse()); } }; diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/election/ElectionNode.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/election/ElectionNode.java index 89e3084c9..447f1efec 100644 --- a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/election/ElectionNode.java +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/election/ElectionNode.java @@ -26,15 +26,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.Lifecycle; import com.alipay.sofa.jraft.Node; import com.alipay.sofa.jraft.RaftGroupService; import com.alipay.sofa.jraft.conf.Configuration; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.option.NodeOptions; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; /** * @@ -89,8 +89,7 @@ public boolean init(final ElectionNodeOptions opts) { if (!serverId.parse(opts.getServerAddress())) { throw new IllegalArgumentException("Fail to parse serverId: " + opts.getServerAddress()); } - final RpcServer rpcServer = new RpcServer(serverId.getPort()); - RaftRpcServerFactory.addRaftRequestProcessors(rpcServer); + final RpcServer rpcServer = RaftRpcServerFactory.createRaftRpcServer(serverId.getEndpoint()); this.raftGroupService = new RaftGroupService(groupId, serverId, nodeOpts, rpcServer); this.node = this.raftGroupService.start(); if (this.node != null) { diff --git a/jraft-extension/pom.xml b/jraft-extension/pom.xml new file mode 100644 index 000000000..8474c66ca --- /dev/null +++ b/jraft-extension/pom.xml @@ -0,0 +1,17 @@ + + + 4.0.0 + + jraft-parent + com.alipay.sofa + 1.3.0 + + + jraft-extension + pom + jraft-extension ${project.version} + + rpc-grpc-impl + + + diff --git a/jraft-extension/rpc-grpc-impl/pom.xml b/jraft-extension/rpc-grpc-impl/pom.xml new file mode 100644 index 000000000..5c04c86e3 --- /dev/null +++ b/jraft-extension/rpc-grpc-impl/pom.xml @@ -0,0 +1,34 @@ + + + 4.0.0 + + jraft-extension + com.alipay.sofa + 1.3.0 + + + rpc-grpc-impl + + + + ${project.groupId} + jraft-core + + + + io.grpc + grpc-netty-shaded + 1.17.0 + + + io.grpc + grpc-protobuf + 1.17.0 + + + io.grpc + grpc-stub + 1.17.0 + + + diff --git a/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/GrpcClient.java b/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/GrpcClient.java new file mode 100644 index 000000000..5e63a7aa1 --- /dev/null +++ b/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/GrpcClient.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc.impl; + +import com.alipay.sofa.jraft.ReplicatorGroup; +import com.alipay.sofa.jraft.error.RemotingException; +import com.alipay.sofa.jraft.option.RpcOptions; +import com.alipay.sofa.jraft.rpc.InvokeCallback; +import com.alipay.sofa.jraft.rpc.InvokeContext; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.util.Endpoint; + +/** + * @author jiachun.fjc + */ +public class GrpcClient implements RpcClient { + + @Override + public boolean init(final RpcOptions opts) { + return false; + } + + @Override + public void shutdown() { + + } + + @Override + public boolean checkConnection(final Endpoint endpoint) { + return false; + } + + @Override + public void closeConnection(final Endpoint endpoint) { + + } + + @Override + public void registerConnectEventListener(final ReplicatorGroup replicatorGroup) { + + } + + @Override + public Object invokeSync(final Endpoint endpoint, final Object request, final InvokeContext ctx, + final long timeoutMs) throws InterruptedException, RemotingException { + return null; + } + + @Override + public void invokeAsync(final Endpoint endpoint, final Object request, final InvokeContext ctx, + final InvokeCallback callback, final long timeoutMs) throws InterruptedException, + RemotingException { + + } +} diff --git a/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/GrpcRaftRpcFactory.java b/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/GrpcRaftRpcFactory.java new file mode 100644 index 000000000..c77fd4cc7 --- /dev/null +++ b/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/GrpcRaftRpcFactory.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc.impl; + +import io.grpc.Server; +import io.grpc.ServerBuilder; +import io.grpc.util.MutableHandlerRegistry; + +import com.alipay.sofa.jraft.rpc.RaftRpcFactory; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.rpc.RpcServer; +import com.alipay.sofa.jraft.util.Endpoint; +import com.alipay.sofa.jraft.util.Requires; +import com.alipay.sofa.jraft.util.SPI; + +/** + * + * @author jiachun.fjc + */ +@SPI(priority = 1) +public class GrpcRaftRpcFactory implements RaftRpcFactory { + + @Override + public void registerProtobufSerializer(final String className) { + + } + + @Override + public RpcClient createRpcClient(final ConfigHelper helper) { + return null; + } + + @Override + public RpcServer createRpcServer(final Endpoint endpoint, final ConfigHelper helper) { + final int port = Requires.requireNonNull(endpoint, "endpoint").getPort(); + Requires.requireTrue(port > 0 && port < 0xFFFF, "port out of range:" + port); + final MutableHandlerRegistry handlerRegistry = new MutableHandlerRegistry(); + final Server server = ServerBuilder.forPort(port) // + .fallbackHandlerRegistry(handlerRegistry) // + .build(); + final RpcServer rpcServer = new GrpcServer(server, handlerRegistry); + if (helper != null) { + helper.config(rpcServer); + } + return rpcServer; + } +} diff --git a/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/GrpcServer.java b/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/GrpcServer.java new file mode 100644 index 000000000..ee4cc31dc --- /dev/null +++ b/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/GrpcServer.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc.impl; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +import io.grpc.Server; +import io.grpc.util.MutableHandlerRegistry; + +import com.alipay.sofa.jraft.rpc.RpcProcessor; +import com.alipay.sofa.jraft.rpc.RpcServer; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; + +/** + * @author jiachun.fjc + */ +public class GrpcServer implements RpcServer { + + private final Server server; + private final MutableHandlerRegistry handlerRegistry; + private final AtomicBoolean start = new AtomicBoolean(false); + + public GrpcServer(Server server, MutableHandlerRegistry handlerRegistry) { + this.server = server; + this.handlerRegistry = handlerRegistry; + } + + @Override + public boolean init(final Void opts) { + if (!this.start.compareAndSet(false, true)) { + throw new IllegalStateException("grpc server has started"); + } + + try { + this.server.start(); + } catch (final IOException e) { + ThrowUtil.throwException(e); + } + return true; + } + + @Override + public void shutdown() { + if (!this.start.compareAndSet(true, false)) { + return; + } + this.server.shutdown(); + } + + @Override + public void registerConnectionClosedEventListener(final ConnectionClosedEventListener listener) { + + } + + @Override + public void registerProcessor(final RpcProcessor processor) { + // final ServerServiceDefinition definition = ServerServiceDefinition.builder(processor.interest()).addMethod() + // .build(); + } + + @Override + public int boundPort() { + return this.server.getPort(); + } +} diff --git a/jraft-extension/rpc-grpc-impl/src/main/resources/META-INF/services/com.alipay.sofa.jraft.rpc.RaftRpcFactory b/jraft-extension/rpc-grpc-impl/src/main/resources/META-INF/services/com.alipay.sofa.jraft.rpc.RaftRpcFactory new file mode 100644 index 000000000..35e827138 --- /dev/null +++ b/jraft-extension/rpc-grpc-impl/src/main/resources/META-INF/services/com.alipay.sofa.jraft.rpc.RaftRpcFactory @@ -0,0 +1 @@ +com.alipay.sofa.jraft.rpc.impl.GrpcRaftRpcFactory \ No newline at end of file diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/KVCommandProcessor.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/KVCommandProcessor.java index 27e042299..228e0fef9 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/KVCommandProcessor.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/KVCommandProcessor.java @@ -18,9 +18,6 @@ import java.util.concurrent.Executor; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.rpc.protocol.AsyncUserProcessor; import com.alipay.sofa.jraft.rhea.cmd.store.BaseRequest; import com.alipay.sofa.jraft.rhea.cmd.store.BaseResponse; import com.alipay.sofa.jraft.rhea.cmd.store.BatchDeleteRequest; @@ -45,6 +42,8 @@ import com.alipay.sofa.jraft.rhea.cmd.store.ScanRequest; import com.alipay.sofa.jraft.rhea.errors.Errors; import com.alipay.sofa.jraft.rhea.errors.RheaRuntimeException; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; import com.alipay.sofa.jraft.util.Requires; /** @@ -52,7 +51,7 @@ * * @author jiachun.fjc */ -public class KVCommandProcessor extends AsyncUserProcessor { +public class KVCommandProcessor implements RpcProcessor { private final Class reqClazz; private final StoreEngine storeEngine; @@ -63,10 +62,9 @@ public KVCommandProcessor(Class reqClazz, StoreEngine storeEngine) { } @Override - public void handleRequest(final BizContext bizCtx, final AsyncContext asyncCtx, final T request) { + public void handleRequest(final RpcContext rpcCtx, final T request) { Requires.requireNonNull(request, "request"); - final RequestProcessClosure> closure = new RequestProcessClosure<>(request, - bizCtx, asyncCtx); + final RequestProcessClosure> closure = new RequestProcessClosure<>(request, rpcCtx); final RegionKVService regionKVService = this.storeEngine.getRegionKVService(request.getRegionId()); if (regionKVService == null) { final NoRegionFoundResponse noRegion = new NoRegionFoundResponse(); @@ -145,7 +143,7 @@ public String interest() { } @Override - public Executor getExecutor() { + public Executor executor() { return this.storeEngine.getKvRpcExecutor(); } } diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/RegionEngine.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/RegionEngine.java index 0f7e2fca1..046c00e91 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/RegionEngine.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/RegionEngine.java @@ -27,7 +27,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.Lifecycle; import com.alipay.sofa.jraft.Node; import com.alipay.sofa.jraft.RaftGroupService; @@ -43,9 +42,10 @@ import com.alipay.sofa.jraft.rhea.storage.RaftRawKVStore; import com.alipay.sofa.jraft.rhea.storage.RawKVStore; import com.alipay.sofa.jraft.rhea.util.Strings; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.rpc.RpcServer; import com.alipay.sofa.jraft.util.Endpoint; import com.alipay.sofa.jraft.util.Requires; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.ScheduledReporter; import com.codahale.metrics.Slf4jReporter; diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/RequestProcessClosure.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/RequestProcessClosure.java index 823558c5c..d7650bcc2 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/RequestProcessClosure.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/RequestProcessClosure.java @@ -16,10 +16,9 @@ */ package com.alipay.sofa.jraft.rhea; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; import com.alipay.sofa.jraft.Closure; import com.alipay.sofa.jraft.Status; +import com.alipay.sofa.jraft.rpc.RpcContext; /** * RPC request processor closure wraps request/response and network biz context. @@ -29,25 +28,19 @@ */ public class RequestProcessClosure implements Closure { - private final REQ request; - private final BizContext bizContext; - private final AsyncContext asyncContext; + private final REQ request; + private final RpcContext rpcCtx; - private RSP response; + private RSP response; - public RequestProcessClosure(REQ request, BizContext bizContext, AsyncContext asyncContext) { + public RequestProcessClosure(REQ request, RpcContext rpcCtx) { super(); this.request = request; - this.bizContext = bizContext; - this.asyncContext = asyncContext; + this.rpcCtx = rpcCtx; } - public AsyncContext getAsyncContext() { - return asyncContext; - } - - public BizContext getBizContext() { - return bizContext; + public RpcContext getRpcCtx() { + return rpcCtx; } public REQ getRequest() { @@ -68,6 +61,6 @@ public void sendResponse(RSP response) { */ @Override public void run(final Status status) { - this.asyncContext.sendResponse(this.response); + this.rpcCtx.sendResponse(this.response); } } diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/StoreEngine.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/StoreEngine.java index f144fd66a..ec457067c 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/StoreEngine.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/StoreEngine.java @@ -30,7 +30,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.Lifecycle; import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.entity.Task; @@ -64,6 +63,7 @@ import com.alipay.sofa.jraft.rhea.util.NetUtil; import com.alipay.sofa.jraft.rhea.util.Strings; import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; import com.alipay.sofa.jraft.util.BytesUtil; import com.alipay.sofa.jraft.util.Describer; import com.alipay.sofa.jraft.util.Endpoint; @@ -199,11 +199,10 @@ public synchronized boolean init(final StoreEngineOptions opts) { // init metrics startMetricReporters(metricsReportPeriod); // init rpc server - this.rpcServer = new RpcServer(port, true, false); - RaftRpcServerFactory.addRaftRequestProcessors(this.rpcServer, this.raftRpcExecutor, this.cliRpcExecutor); + this.rpcServer = RaftRpcServerFactory.createRaftRpcServer(serverAddress, this.raftRpcExecutor, + this.cliRpcExecutor); StoreEngineHelper.addKvStoreRequestProcessor(this.rpcServer, this); - this.rpcServer.startup(); - if (!this.rpcServer.isStarted()) { + if (!this.rpcServer.init(null)) { LOG.error("Fail to init [RpcServer]."); return false; } diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/StoreEngineHelper.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/StoreEngineHelper.java index 3e7fc9a64..b7a456d6a 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/StoreEngineHelper.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/StoreEngineHelper.java @@ -25,7 +25,6 @@ import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.rhea.cmd.store.BatchDeleteRequest; import com.alipay.sofa.jraft.rhea.cmd.store.BatchPutRequest; import com.alipay.sofa.jraft.rhea.cmd.store.CompareAndPutRequest; @@ -47,6 +46,7 @@ import com.alipay.sofa.jraft.rhea.cmd.store.ScanRequest; import com.alipay.sofa.jraft.rhea.util.concurrent.CallerRunsPolicyWithReport; import com.alipay.sofa.jraft.rhea.util.concurrent.NamedThreadFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; import com.alipay.sofa.jraft.util.ThreadPoolUtil; /** @@ -90,25 +90,25 @@ public static ScheduledExecutorService createMetricsScheduler() { } public static void addKvStoreRequestProcessor(final RpcServer rpcServer, final StoreEngine engine) { - rpcServer.registerUserProcessor(new KVCommandProcessor<>(GetRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(MultiGetRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(ContainsKeyRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(GetSequenceRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(ResetSequenceRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(ScanRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(PutRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(GetAndPutRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(CompareAndPutRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(MergeRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(PutIfAbsentRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(KeyLockRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(KeyUnlockRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(BatchPutRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(DeleteRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(DeleteRangeRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(BatchDeleteRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(NodeExecuteRequest.class, engine)); - rpcServer.registerUserProcessor(new KVCommandProcessor<>(RangeSplitRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(GetRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(MultiGetRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(ContainsKeyRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(GetSequenceRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(ResetSequenceRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(ScanRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(PutRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(GetAndPutRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(CompareAndPutRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(MergeRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(PutIfAbsentRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(KeyLockRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(KeyUnlockRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(BatchPutRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(DeleteRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(DeleteRangeRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(BatchDeleteRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(NodeExecuteRequest.class, engine)); + rpcServer.registerProcessor(new KVCommandProcessor<>(RangeSplitRequest.class, engine)); } private static ExecutorService newPool(final int coreThreads, final int maxThreads, final String name) { diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultDistributedLock.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultDistributedLock.java index 7c7c1fb10..7e0751087 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultDistributedLock.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultDistributedLock.java @@ -26,7 +26,7 @@ import com.alipay.sofa.jraft.rhea.errors.InvalidLockAcquirerException; import com.alipay.sofa.jraft.rhea.util.StackTraceUtil; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.alipay.sofa.jraft.rhea.util.concurrent.DistributedLock; /** diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultRheaKVCliService.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultRheaKVCliService.java index ee8a76619..096cb6cab 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultRheaKVCliService.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultRheaKVCliService.java @@ -19,7 +19,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.rpc.RpcClient; import com.alipay.sofa.jraft.CliService; import com.alipay.sofa.jraft.RaftServiceFactory; import com.alipay.sofa.jraft.Status; @@ -31,7 +30,8 @@ import com.alipay.sofa.jraft.rhea.cmd.store.RangeSplitRequest; import com.alipay.sofa.jraft.rhea.util.StackTraceUtil; import com.alipay.sofa.jraft.rpc.CliClientService; -import com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.rpc.impl.AbstractClientService; import com.alipay.sofa.jraft.util.Requires; /** @@ -79,8 +79,8 @@ public Status rangeSplit(final long regionId, final long newRegionId, final Stri request.setRegionId(regionId); request.setNewRegionId(newRegionId); try { - final BaseResponse response = (BaseResponse) this.rpcClient.invokeSync(leaderId.getEndpoint() - .toString(), request, this.opts.getTimeoutMs()); + final BaseResponse response = (BaseResponse) this.rpcClient.invokeSync(leaderId.getEndpoint(), + request, this.opts.getTimeoutMs()); if (response.isSuccess()) { return Status.OK(); } @@ -101,6 +101,6 @@ private void initCli(CliOptions cliOpts) { this.cliService = RaftServiceFactory.createAndInitCliService(cliOpts); final CliClientService cliClientService = ((CliServiceImpl) this.cliService).getCliClientService(); Requires.requireNonNull(cliClientService, "cliClientService"); - this.rpcClient = ((AbstractBoltClientService) cliClientService).getRpcClient(); + this.rpcClient = ((AbstractClientService) cliClientService).getRpcClient(); } } diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultRheaKVRpcService.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultRheaKVRpcService.java index 379cef265..6f5a8efea 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultRheaKVRpcService.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/DefaultRheaKVRpcService.java @@ -24,9 +24,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.InvokeCallback; -import com.alipay.remoting.InvokeContext; -import com.alipay.remoting.rpc.RpcClient; import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.rhea.client.failover.FailoverClosure; import com.alipay.sofa.jraft.rhea.client.pd.AbstractPlacementDriverClient; @@ -39,6 +36,10 @@ import com.alipay.sofa.jraft.rhea.rpc.ExtSerializerSupports; import com.alipay.sofa.jraft.rhea.util.concurrent.CallerRunsPolicyWithReport; import com.alipay.sofa.jraft.rhea.util.concurrent.NamedThreadFactory; +import com.alipay.sofa.jraft.rpc.InvokeCallback; +import com.alipay.sofa.jraft.rpc.InvokeContext; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.rpc.impl.BoltRpcClient; import com.alipay.sofa.jraft.util.Endpoint; import com.alipay.sofa.jraft.util.ExecutorServiceHelper; import com.alipay.sofa.jraft.util.Requires; @@ -122,34 +123,34 @@ public Endpoint getRpcEndpoint(final long regionId, final boolean forceRefresh, private void internalCallAsyncWithRpc(final Endpoint endpoint, final BaseRequest request, final FailoverClosure closure) { - final String address = endpoint.toString(); - final InvokeContext invokeCtx = ExtSerializerSupports.getInvokeContext(); + final InvokeContext invokeCtx = new InvokeContext(); + invokeCtx.put(BoltRpcClient.BOLT_CTX, ExtSerializerSupports.getInvokeContext()); final InvokeCallback invokeCallback = new InvokeCallback() { @Override - public void onResponse(final Object result) { - final BaseResponse response = (BaseResponse) result; - if (response.isSuccess()) { - closure.setData(response.getValue()); - closure.run(Status.OK()); + public void complete(final Object result, final Throwable err) { + if (err == null) { + final BaseResponse response = (BaseResponse) result; + if (response.isSuccess()) { + closure.setData(response.getValue()); + closure.run(Status.OK()); + } else { + closure.setError(response.getError()); + closure.run(new Status(-1, "RPC failed with address: %s, response: %s", endpoint, response)); + } } else { - closure.setError(response.getError()); - closure.run(new Status(-1, "RPC failed with address: %s, response: %s", address, response)); + closure.failure(err); } } @Override - public void onException(final Throwable t) { - closure.failure(t); - } - - @Override - public Executor getExecutor() { + public Executor executor() { return rpcCallbackExecutor; } }; + try { - this.rpcClient.invokeWithCallback(address, request, invokeCtx, invokeCallback, this.rpcTimeoutMillis); + this.rpcClient.invokeAsync(endpoint, request, invokeCtx, invokeCallback, this.rpcTimeoutMillis); } catch (final Throwable t) { closure.failure(t); } diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/FutureHelper.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/FutureHelper.java index fea6cb6a4..6904b0319 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/FutureHelper.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/FutureHelper.java @@ -26,7 +26,7 @@ import com.alipay.sofa.jraft.rhea.errors.NeverGetHereException; import com.alipay.sofa.jraft.rhea.util.Lists; import com.alipay.sofa.jraft.rhea.util.Maps; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.alipay.sofa.jraft.util.SystemPropertyUtil; /** diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/AbstractPlacementDriverClient.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/AbstractPlacementDriverClient.java index feaac663b..2bf28f5ad 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/AbstractPlacementDriverClient.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/AbstractPlacementDriverClient.java @@ -23,7 +23,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.rpc.RpcClient; import com.alipay.sofa.jraft.CliService; import com.alipay.sofa.jraft.RaftServiceFactory; import com.alipay.sofa.jraft.RouteTable; @@ -47,11 +46,12 @@ import com.alipay.sofa.jraft.rhea.storage.KVEntry; import com.alipay.sofa.jraft.rhea.util.StackTraceUtil; import com.alipay.sofa.jraft.rhea.util.Strings; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; import com.alipay.sofa.jraft.rpc.CliClientService; -import com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.rpc.impl.AbstractClientService; import com.alipay.sofa.jraft.util.Endpoint; import com.alipay.sofa.jraft.util.Requires; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; /** * @@ -409,7 +409,7 @@ protected void initCli(CliOptions cliOpts) { this.cliService = RaftServiceFactory.createAndInitCliService(cliOpts); this.cliClientService = ((CliServiceImpl) this.cliService).getCliClientService(); Requires.requireNonNull(this.cliClientService, "cliClientService"); - this.rpcClient = ((AbstractBoltClientService) this.cliClientService).getRpcClient(); + this.rpcClient = ((AbstractClientService) this.cliClientService).getRpcClient(); } protected abstract void refreshRouteTable(); diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/DefaultPlacementDriverRpcService.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/DefaultPlacementDriverRpcService.java index 1b3ad3148..98f6f4034 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/DefaultPlacementDriverRpcService.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/DefaultPlacementDriverRpcService.java @@ -24,9 +24,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.InvokeCallback; -import com.alipay.remoting.InvokeContext; -import com.alipay.remoting.rpc.RpcClient; import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.rhea.client.failover.FailoverClosure; import com.alipay.sofa.jraft.rhea.cmd.pd.BaseRequest; @@ -37,6 +34,10 @@ import com.alipay.sofa.jraft.rhea.rpc.ExtSerializerSupports; import com.alipay.sofa.jraft.rhea.util.concurrent.CallerRunsPolicyWithReport; import com.alipay.sofa.jraft.rhea.util.concurrent.NamedThreadFactory; +import com.alipay.sofa.jraft.rpc.InvokeCallback; +import com.alipay.sofa.jraft.rpc.InvokeContext; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.rpc.impl.BoltRpcClient; import com.alipay.sofa.jraft.util.Endpoint; import com.alipay.sofa.jraft.util.ExecutorServiceHelper; import com.alipay.sofa.jraft.util.Requires; @@ -93,34 +94,34 @@ public CompletableFuture callPdServerWithRpc(final BaseRequest request, f private void internalCallPdWithRpc(final Endpoint endpoint, final BaseRequest request, final FailoverClosure closure) { - final String address = endpoint.toString(); - final InvokeContext invokeCtx = ExtSerializerSupports.getInvokeContext(); + final InvokeContext invokeCtx = new InvokeContext(); + invokeCtx.put(BoltRpcClient.BOLT_CTX, ExtSerializerSupports.getInvokeContext()); final InvokeCallback invokeCallback = new InvokeCallback() { @Override - public void onResponse(final Object result) { - final BaseResponse response = (BaseResponse) result; - if (response.isSuccess()) { - closure.setData(response.getValue()); - closure.run(Status.OK()); + public void complete(final Object result, final Throwable err) { + if (err == null) { + final BaseResponse response = (BaseResponse) result; + if (response.isSuccess()) { + closure.setData(response.getValue()); + closure.run(Status.OK()); + } else { + closure.setError(response.getError()); + closure.run(new Status(-1, "RPC failed with address: %s, response: %s", endpoint, response)); + } } else { - closure.setError(response.getError()); - closure.run(new Status(-1, "RPC failed with address: %s, response: %s", address, response)); + closure.failure(err); } } @Override - public void onException(final Throwable t) { - closure.failure(t); - } - - @Override - public Executor getExecutor() { + public Executor executor() { return rpcCallbackExecutor; } }; + try { - this.rpcClient.invokeWithCallback(address, request, invokeCtx, invokeCallback, this.rpcTimeoutMillis); + this.rpcClient.invokeAsync(endpoint, request, invokeCtx, invokeCallback, this.rpcTimeoutMillis); } catch (final Throwable t) { closure.failure(t); } diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/HeartbeatSender.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/HeartbeatSender.java index 1cde7bb97..b2e607fe3 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/HeartbeatSender.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/client/pd/HeartbeatSender.java @@ -25,9 +25,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.InvokeCallback; -import com.alipay.remoting.InvokeContext; -import com.alipay.remoting.rpc.RpcClient; import com.alipay.sofa.jraft.Lifecycle; import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.rhea.StoreEngine; @@ -49,6 +46,9 @@ import com.alipay.sofa.jraft.rhea.util.StackTraceUtil; import com.alipay.sofa.jraft.rhea.util.concurrent.DiscardOldPolicyWithReport; import com.alipay.sofa.jraft.rhea.util.concurrent.NamedThreadFactory; +import com.alipay.sofa.jraft.rpc.InvokeCallback; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.rpc.impl.BoltRpcClient; import com.alipay.sofa.jraft.util.Endpoint; import com.alipay.sofa.jraft.util.ExecutorServiceHelper; import com.alipay.sofa.jraft.util.ThreadPoolUtil; @@ -206,36 +206,35 @@ public void run(final Status status) { private void callAsyncWithRpc(final Endpoint endpoint, final BaseRequest request, final HeartbeatClosure closure) { - final String address = endpoint.toString(); - final InvokeContext invokeCtx = ExtSerializerSupports.getInvokeContext(); + final com.alipay.sofa.jraft.rpc.InvokeContext invokeCtx = new com.alipay.sofa.jraft.rpc.InvokeContext(); + invokeCtx.put(BoltRpcClient.BOLT_CTX, ExtSerializerSupports.getInvokeContext()); final InvokeCallback invokeCallback = new InvokeCallback() { @SuppressWarnings("unchecked") @Override - public void onResponse(final Object result) { - final BaseResponse response = (BaseResponse) result; - if (response.isSuccess()) { - closure.setResult((V) response.getValue()); - closure.run(Status.OK()); + public void complete(final Object result, final Throwable err) { + if (err == null) { + final BaseResponse response = (BaseResponse) result; + if (response.isSuccess()) { + closure.setResult((V) response.getValue()); + closure.run(Status.OK()); + } else { + closure.setError(response.getError()); + closure.run(new Status(-1, "RPC failed with address: %s, response: %s", endpoint, response)); + } } else { - closure.setError(response.getError()); - closure.run(new Status(-1, "RPC failed with address: %s, response: %s", address, response)); + closure.run(new Status(-1, err.getMessage())); } } @Override - public void onException(final Throwable t) { - closure.run(new Status(-1, t.getMessage())); - } - - @Override - public Executor getExecutor() { + public Executor executor() { return heartbeatRpcCallbackExecutor; } }; + try { - this.rpcClient.invokeWithCallback(address, request, invokeCtx, invokeCallback, - this.heartbeatRpcTimeoutMillis); + this.rpcClient.invokeAsync(endpoint, request, invokeCtx, invokeCallback, this.heartbeatRpcTimeoutMillis); } catch (final Throwable t) { closure.run(new Status(-1, t.getMessage())); } diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/rocks/support/RocksStatistics.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/rocks/support/RocksStatistics.java index 6a9deb818..28cf3e9d8 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/rocks/support/RocksStatistics.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/rocks/support/RocksStatistics.java @@ -24,7 +24,7 @@ import org.rocksdb.TickerType; import com.alipay.sofa.jraft.rhea.storage.RocksRawKVStore; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.alipay.sofa.jraft.util.DebugStatistics; import com.alipay.sofa.jraft.util.internal.ReferenceFieldUpdater; import com.alipay.sofa.jraft.util.internal.Updaters; diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/rpc/ProtostuffSerializer.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/rpc/ProtostuffSerializer.java index be6578ea3..97b5fe1b8 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/rpc/ProtostuffSerializer.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/rpc/ProtostuffSerializer.java @@ -22,7 +22,7 @@ import com.alipay.remoting.serialization.Serializer; import com.alipay.sofa.jraft.rhea.serialization.Serializers; import com.alipay.sofa.jraft.rhea.util.Maps; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; /** * @author jiachun.fjc diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/ProtoStuffSerializer.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/ProtoStuffSerializer.java index eae9ad5ad..ee2ca279f 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/ProtoStuffSerializer.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/ProtoStuffSerializer.java @@ -31,7 +31,7 @@ import com.alipay.sofa.jraft.rhea.serialization.impl.protostuff.io.Outputs; import com.alipay.sofa.jraft.rhea.serialization.io.InputBuf; import com.alipay.sofa.jraft.rhea.serialization.io.OutputBuf; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.alipay.sofa.jraft.util.SystemPropertyUtil; /** diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/io/NioBufInput.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/io/NioBufInput.java index 2092a6c35..b519bd30c 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/io/NioBufInput.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/io/NioBufInput.java @@ -29,7 +29,7 @@ import io.protostuff.StringSerializer; import io.protostuff.UninitializedMessageException; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.alipay.sofa.jraft.util.internal.UnsafeUtf8Util; import com.alipay.sofa.jraft.util.internal.UnsafeUtil; diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/io/UnsafeNioBufInput.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/io/UnsafeNioBufInput.java index e91e4bc1d..543a73cfe 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/io/UnsafeNioBufInput.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/serialization/impl/protostuff/io/UnsafeNioBufInput.java @@ -28,7 +28,7 @@ import io.protostuff.Schema; import io.protostuff.UninitializedMessageException; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.alipay.sofa.jraft.rhea.util.internal.UnsafeDirectBufferUtil; import com.alipay.sofa.jraft.util.internal.UnsafeUtf8Util; import com.alipay.sofa.jraft.util.internal.UnsafeUtil; diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/storage/KVStoreStateMachine.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/storage/KVStoreStateMachine.java index a03b70be4..481a0dd25 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/storage/KVStoreStateMachine.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/storage/KVStoreStateMachine.java @@ -39,7 +39,7 @@ import com.alipay.sofa.jraft.rhea.serialization.Serializer; import com.alipay.sofa.jraft.rhea.serialization.Serializers; import com.alipay.sofa.jraft.rhea.util.StackTraceUtil; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.alipay.sofa.jraft.storage.snapshot.SnapshotReader; import com.alipay.sofa.jraft.storage.snapshot.SnapshotWriter; import com.alipay.sofa.jraft.util.BytesUtil; diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/NetUtil.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/NetUtil.java index 434c1911d..88f2c8be1 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/NetUtil.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/NetUtil.java @@ -22,6 +22,8 @@ import java.util.Enumeration; import java.util.regex.Pattern; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; + /** * * @author jiachun.fjc diff --git a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/concurrent/DistributedLock.java b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/concurrent/DistributedLock.java index ec9925a9a..ddc3e99f9 100644 --- a/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/concurrent/DistributedLock.java +++ b/jraft-rheakv/rheakv-core/src/main/java/com/alipay/sofa/jraft/rhea/util/concurrent/DistributedLock.java @@ -21,7 +21,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.alipay.sofa.jraft.rhea.util.UniqueIdUtil; import com.alipay.sofa.jraft.util.BytesUtil; import com.alipay.sofa.jraft.util.Requires; diff --git a/jraft-rheakv/rheakv-core/src/test/java/com/alipay/sofa/jraft/rhea/storage/KVStateMachineTest.java b/jraft-rheakv/rheakv-core/src/test/java/com/alipay/sofa/jraft/rhea/storage/KVStateMachineTest.java index 755e13820..949030195 100644 --- a/jraft-rheakv/rheakv-core/src/test/java/com/alipay/sofa/jraft/rhea/storage/KVStateMachineTest.java +++ b/jraft-rheakv/rheakv-core/src/test/java/com/alipay/sofa/jraft/rhea/storage/KVStateMachineTest.java @@ -45,7 +45,7 @@ import com.alipay.sofa.jraft.rhea.StoreEngine; import com.alipay.sofa.jraft.rhea.client.pd.FakePlacementDriverClient; import com.alipay.sofa.jraft.rhea.metadata.Region; -import com.alipay.sofa.jraft.rhea.util.ThrowUtil; +import com.alipay.sofa.jraft.util.internal.ThrowUtil; import com.alipay.sofa.jraft.util.BytesUtil; import com.alipay.sofa.jraft.util.Endpoint; diff --git a/jraft-rheakv/rheakv-pd/src/main/java/com/alipay/sofa/jraft/rhea/PlacementDriverProcessor.java b/jraft-rheakv/rheakv-pd/src/main/java/com/alipay/sofa/jraft/rhea/PlacementDriverProcessor.java index fb74125da..6dc83772e 100644 --- a/jraft-rheakv/rheakv-pd/src/main/java/com/alipay/sofa/jraft/rhea/PlacementDriverProcessor.java +++ b/jraft-rheakv/rheakv-pd/src/main/java/com/alipay/sofa/jraft/rhea/PlacementDriverProcessor.java @@ -18,9 +18,6 @@ import java.util.concurrent.Executor; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.rpc.protocol.AsyncUserProcessor; import com.alipay.sofa.jraft.rhea.cmd.pd.BaseRequest; import com.alipay.sofa.jraft.rhea.cmd.pd.BaseResponse; import com.alipay.sofa.jraft.rhea.cmd.pd.CreateRegionIdRequest; @@ -31,6 +28,8 @@ import com.alipay.sofa.jraft.rhea.cmd.pd.SetStoreInfoRequest; import com.alipay.sofa.jraft.rhea.cmd.pd.StoreHeartbeatRequest; import com.alipay.sofa.jraft.rhea.errors.RheaRuntimeException; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; import com.alipay.sofa.jraft.util.Requires; /** @@ -53,7 +52,7 @@ * * @author jiachun.fjc */ -public class PlacementDriverProcessor extends AsyncUserProcessor { +public class PlacementDriverProcessor implements RpcProcessor { private final Class reqClazz; private final PlacementDriverService placementDriverService; @@ -66,10 +65,9 @@ public PlacementDriverProcessor(Class reqClazz, PlacementDriverService placem } @Override - public void handleRequest(BizContext bizCtx, AsyncContext asyncCtx, T request) { + public void handleRequest(final RpcContext rpcCtx, final T request) { Requires.requireNonNull(request, "request"); - final RequestProcessClosure closure = new RequestProcessClosure<>(request, bizCtx, - asyncCtx); + final RequestProcessClosure closure = new RequestProcessClosure<>(request, rpcCtx); switch (request.magic()) { case BaseRequest.STORE_HEARTBEAT: this.placementDriverService.handleStoreHeartbeatRequest((StoreHeartbeatRequest) request, closure); @@ -103,7 +101,7 @@ public String interest() { } @Override - public Executor getExecutor() { + public Executor executor() { return this.executor; } } diff --git a/jraft-rheakv/rheakv-pd/src/main/java/com/alipay/sofa/jraft/rhea/PlacementDriverServer.java b/jraft-rheakv/rheakv-pd/src/main/java/com/alipay/sofa/jraft/rhea/PlacementDriverServer.java index cd006b63d..d24d8193a 100644 --- a/jraft-rheakv/rheakv-pd/src/main/java/com/alipay/sofa/jraft/rhea/PlacementDriverServer.java +++ b/jraft-rheakv/rheakv-pd/src/main/java/com/alipay/sofa/jraft/rhea/PlacementDriverServer.java @@ -23,7 +23,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.Lifecycle; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.rhea.client.DefaultRheaKVStore; @@ -40,6 +39,7 @@ import com.alipay.sofa.jraft.rhea.options.RheaKVStoreOptions; import com.alipay.sofa.jraft.rhea.util.concurrent.CallerRunsPolicyWithReport; import com.alipay.sofa.jraft.rhea.util.concurrent.NamedThreadFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; import com.alipay.sofa.jraft.util.Endpoint; import com.alipay.sofa.jraft.util.ExecutorServiceHelper; import com.alipay.sofa.jraft.util.Requires; @@ -201,19 +201,19 @@ public boolean awaitReady(final long timeoutMillis) { } private void addPlacementDriverProcessor(final RpcServer rpcServer) { - rpcServer.registerUserProcessor(new PlacementDriverProcessor<>(RegionHeartbeatRequest.class, + rpcServer.registerProcessor(new PlacementDriverProcessor<>(RegionHeartbeatRequest.class, this.placementDriverService, this.pdExecutor)); - rpcServer.registerUserProcessor(new PlacementDriverProcessor<>(StoreHeartbeatRequest.class, + rpcServer.registerProcessor(new PlacementDriverProcessor<>(StoreHeartbeatRequest.class, this.placementDriverService, this.pdExecutor)); - rpcServer.registerUserProcessor(new PlacementDriverProcessor<>(GetClusterInfoRequest.class, + rpcServer.registerProcessor(new PlacementDriverProcessor<>(GetClusterInfoRequest.class, this.placementDriverService, this.pdExecutor)); - rpcServer.registerUserProcessor(new PlacementDriverProcessor<>(GetStoreIdRequest.class, + rpcServer.registerProcessor(new PlacementDriverProcessor<>(GetStoreIdRequest.class, this.placementDriverService, this.pdExecutor)); - rpcServer.registerUserProcessor(new PlacementDriverProcessor<>(GetStoreInfoRequest.class, + rpcServer.registerProcessor(new PlacementDriverProcessor<>(GetStoreInfoRequest.class, this.placementDriverService, this.pdExecutor)); - rpcServer.registerUserProcessor(new PlacementDriverProcessor<>(SetStoreInfoRequest.class, + rpcServer.registerProcessor(new PlacementDriverProcessor<>(SetStoreInfoRequest.class, this.placementDriverService, this.pdExecutor)); - rpcServer.registerUserProcessor(new PlacementDriverProcessor<>(CreateRegionIdRequest.class, + rpcServer.registerProcessor(new PlacementDriverProcessor<>(CreateRegionIdRequest.class, this.placementDriverService, this.pdExecutor)); } diff --git a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/client/AtomicClient.java b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/client/AtomicClient.java index 043b93d29..d1b3eeaac 100644 --- a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/client/AtomicClient.java +++ b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/client/AtomicClient.java @@ -25,13 +25,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.exception.RemotingException; -import com.alipay.remoting.rpc.RpcClient; import com.alipay.sofa.jraft.RouteTable; import com.alipay.sofa.jraft.conf.Configuration; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.option.CliOptions; -import com.alipay.sofa.jraft.rpc.impl.cli.BoltCliClientService; +import com.alipay.sofa.jraft.rpc.RpcClient; +import com.alipay.sofa.jraft.rpc.impl.cli.CliClientServiceImpl; import com.alipay.sofa.jraft.test.atomic.HashUtils; import com.alipay.sofa.jraft.test.atomic.KeyNotFoundException; import com.alipay.sofa.jraft.test.atomic.command.BooleanCommand; @@ -54,7 +53,7 @@ public class AtomicClient { static final Logger LOG = LoggerFactory.getLogger(AtomicClient.class); private final Configuration conf; - private final BoltCliClientService cliClientService; + private final CliClientServiceImpl cliClientService; private RpcClient rpcClient; private CliOptions cliOptions; private TreeMap groups = new TreeMap<>(); @@ -62,7 +61,7 @@ public class AtomicClient { public AtomicClient(String groupId, Configuration conf) { super(); this.conf = conf; - this.cliClientService = new BoltCliClientService(); + this.cliClientService = new CliClientServiceImpl(); } public void shutdown() { @@ -80,16 +79,16 @@ public void start() throws InterruptedException, TimeoutException { final Set peers = conf.getPeerSet(); for (final PeerId peer : peers) { try { - final BooleanCommand cmd = (BooleanCommand) this.rpcClient.invokeSync( - peer.getEndpoint().toString(), new GetSlotsCommand(), cliOptions.getRpcDefaultTimeout()); + final BooleanCommand cmd = (BooleanCommand) this.rpcClient.invokeSync(peer.getEndpoint(), + new GetSlotsCommand(), cliOptions.getRpcDefaultTimeout()); if (cmd instanceof SlotsResponseCommand) { groups = ((SlotsResponseCommand) cmd).getMap(); break; } else { LOG.warn("Fail to get slots from peer {}, error: {}", peer, cmd.getErrorMsg()); } - } catch (final RemotingException e) { - LOG.warn("Fail to get slots from peer {}, error: {}", peer, e.getMessage()); + } catch (final Throwable t) { + LOG.warn("Fail to get slots from peer {}, error: {}", peer, t.getMessage()); //continue; } } @@ -158,7 +157,7 @@ public long get(PeerId peer, String key, boolean readFromQuorum, boolean readByS final GetCommand request = new GetCommand(key); request.setReadFromQuorum(readFromQuorum); request.setReadByStateMachine(readByStateMachine); - final Object response = this.rpcClient.invokeSync(peer.getEndpoint().toString(), request, + final Object response = this.rpcClient.invokeSync(peer.getEndpoint(), request, cliOptions.getRpcDefaultTimeout()); final BooleanCommand cmd = (BooleanCommand) response; if (cmd.isSuccess()) { @@ -170,9 +169,9 @@ public long get(PeerId peer, String key, boolean readFromQuorum, boolean readByS throw new IllegalStateException("Server error:" + cmd.getErrorMsg()); } } - } catch (final RemotingException e) { - e.printStackTrace(); - throw new IllegalStateException("Remoting error:" + e.getMessage()); + } catch (final Throwable t) { + t.printStackTrace(); + throw new IllegalStateException("Remoting error:" + t.getMessage()); } } @@ -185,7 +184,7 @@ public long addAndGet(PeerId peer, String key, long delta) throws InterruptedExc final IncrementAndGetCommand request = new IncrementAndGetCommand(); request.setKey(key); request.setDetal(delta); - final Object response = this.rpcClient.invokeSync(peer.getEndpoint().toString(), request, + final Object response = this.rpcClient.invokeSync(peer.getEndpoint(), request, cliOptions.getRpcDefaultTimeout()); final BooleanCommand cmd = (BooleanCommand) response; if (cmd.isSuccess()) { @@ -193,8 +192,8 @@ public long addAndGet(PeerId peer, String key, long delta) throws InterruptedExc } else { throw new IllegalStateException("Server error:" + cmd.getErrorMsg()); } - } catch (final RemotingException e) { - throw new IllegalStateException("Remoting error:" + e.getMessage()); + } catch (final Throwable t) { + throw new IllegalStateException("Remoting error:" + t.getMessage()); } } @@ -207,12 +206,12 @@ public boolean set(PeerId peer, String key, long value) throws InterruptedExcept final SetCommand request = new SetCommand(); request.setKey(key); request.setValue(value); - final Object response = this.rpcClient.invokeSync(peer.getEndpoint().toString(), request, + final Object response = this.rpcClient.invokeSync(peer.getEndpoint(), request, cliOptions.getRpcDefaultTimeout()); final BooleanCommand cmd = (BooleanCommand) response; return cmd.isSuccess(); - } catch (final RemotingException e) { - throw new IllegalStateException("Remoting error:" + e.getMessage()); + } catch (final Throwable t) { + throw new IllegalStateException("Remoting error:" + t.getMessage()); } } @@ -226,12 +225,12 @@ public boolean compareAndSet(PeerId peer, String key, long expect, long newVal) request.setKey(key); request.setNewValue(newVal); request.setExpect(expect); - final Object response = this.rpcClient.invokeSync(peer.getEndpoint().toString(), request, + final Object response = this.rpcClient.invokeSync(peer.getEndpoint(), request, cliOptions.getRpcDefaultTimeout()); final BooleanCommand cmd = (BooleanCommand) response; return cmd.isSuccess(); - } catch (final RemotingException e) { - throw new IllegalStateException("Remoting error:" + e.getMessage()); + } catch (final Throwable t) { + throw new IllegalStateException("Remoting error:" + t.getMessage()); } } diff --git a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/client/AtomicClientTest.java b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/client/AtomicClientTest.java index f62b366be..5787a42a7 100644 --- a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/client/AtomicClientTest.java +++ b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/client/AtomicClientTest.java @@ -23,7 +23,7 @@ import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.option.CliOptions; -import com.alipay.sofa.jraft.rpc.impl.cli.BoltCliClientService; +import com.alipay.sofa.jraft.rpc.impl.cli.CliClientServiceImpl; public class AtomicClientTest { @@ -31,7 +31,7 @@ public static void main(String[] args) throws Exception { final RouteTable table = RouteTable.getInstance(); table.updateConfiguration("atomic_0", JRaftUtils.getConfiguration("127.0.0.1:8609,127.0.0.1:8610,127.0.0.1:8611")); - final BoltCliClientService cliClientService = new BoltCliClientService(); + final CliClientServiceImpl cliClientService = new CliClientServiceImpl(); cliClientService.init(new CliOptions()); final Status st = table.refreshLeader(cliClientService, "atomic_0", 10000); System.out.println(st); diff --git a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/AtomicRangeGroup.java b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/AtomicRangeGroup.java index 830658e5c..dec68b2ba 100644 --- a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/AtomicRangeGroup.java +++ b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/AtomicRangeGroup.java @@ -25,8 +25,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.Node; import com.alipay.sofa.jraft.RaftGroupService; import com.alipay.sofa.jraft.Status; @@ -35,6 +33,8 @@ import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.option.NodeOptions; import com.alipay.sofa.jraft.option.ReadOnlyOption; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcServer; import com.alipay.sofa.jraft.test.atomic.KeyNotFoundException; import com.alipay.sofa.jraft.test.atomic.command.BooleanCommand; import com.alipay.sofa.jraft.test.atomic.command.ValueCommand; @@ -101,7 +101,7 @@ public AtomicRangeGroup(String dataPath, String groupId, PeerId serverId, long m } - public void readFromQuorum(final String key, AsyncContext asyncContext) { + public void readFromQuorum(final String key, RpcContext asyncContext) { final byte[] reqContext = new byte[4]; Bits.putInt(reqContext, 0, requestId.incrementAndGet()); this.node.readIndex(reqContext, new ReadIndexClosure() { diff --git a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/AtomicServer.java b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/AtomicServer.java index 907c767ee..f58e90ff7 100644 --- a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/AtomicServer.java +++ b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/AtomicServer.java @@ -24,9 +24,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.alipay.remoting.rpc.RpcServer; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; import com.alipay.sofa.jraft.test.atomic.HashUtils; import com.alipay.sofa.jraft.test.atomic.server.processor.CompareAndSetCommandProcessor; import com.alipay.sofa.jraft.test.atomic.server.processor.GetCommandProcessor; @@ -74,16 +74,14 @@ public void start() throws IOException { } FileUtils.forceMkdir(new File(conf.getDataPath())); - //同一个进程内 raft group 共用同一个 RPC Server. - RpcServer rpcServer = new RpcServer(serverId.getPort()); - //注册 raft 处理器 - RaftRpcServerFactory.addRaftRequestProcessors(rpcServer); - //注册业务处理器 - rpcServer.registerUserProcessor(new GetSlotsCommandProcessor(this)); - rpcServer.registerUserProcessor(new GetCommandProcessor(this)); - rpcServer.registerUserProcessor(new IncrementAndGetCommandProcessor(this)); - rpcServer.registerUserProcessor(new CompareAndSetCommandProcessor(this)); - rpcServer.registerUserProcessor(new SetCommandProcessor(this)); + // The same in-process raft group shares the same RPC Server. + RpcServer rpcServer = RaftRpcServerFactory.createRaftRpcServer(serverId.getEndpoint()); + // Register biz handler + rpcServer.registerProcessor(new GetSlotsCommandProcessor(this)); + rpcServer.registerProcessor(new GetCommandProcessor(this)); + rpcServer.registerProcessor(new IncrementAndGetCommandProcessor(this)); + rpcServer.registerProcessor(new CompareAndSetCommandProcessor(this)); + rpcServer.registerProcessor(new SetCommandProcessor(this)); long step = conf.getMaxSlot() / totalSlots; if (conf.getMaxSlot() % totalSlots > 0) { diff --git a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/BaseAsyncUserProcessor.java b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/BaseAsyncUserProcessor.java index 4e1f2f0e1..8474fd769 100644 --- a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/BaseAsyncUserProcessor.java +++ b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/BaseAsyncUserProcessor.java @@ -18,12 +18,9 @@ import java.nio.ByteBuffer; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.rpc.protocol.AsyncUserProcessor; -import com.alipay.sofa.jraft.Closure; -import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.entity.Task; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; import com.alipay.sofa.jraft.test.atomic.command.BaseRequestCommand; import com.alipay.sofa.jraft.test.atomic.command.BooleanCommand; import com.alipay.sofa.jraft.test.atomic.command.CommandCodec; @@ -32,7 +29,7 @@ import com.alipay.sofa.jraft.test.atomic.server.CommandType; import com.alipay.sofa.jraft.test.atomic.server.LeaderTaskClosure; -public abstract class BaseAsyncUserProcessor extends AsyncUserProcessor { +public abstract class BaseAsyncUserProcessor implements RpcProcessor { protected AtomicServer server; public BaseAsyncUserProcessor(AtomicServer server) { @@ -41,33 +38,29 @@ public BaseAsyncUserProcessor(AtomicServer server) { } @Override - public void handleRequest(BizContext bizCtx, AsyncContext asyncCtx, T request) { + public void handleRequest(final RpcContext rpcCtx, final T request) { final AtomicRangeGroup group = server.getGroupBykey(request.getKey()); if (!group.getFsm().isLeader()) { - asyncCtx.sendResponse(group.redirect()); + rpcCtx.sendResponse(group.redirect()); return; } final CommandType cmdType = getCmdType(); - final Task task = createTask(asyncCtx, request, cmdType); + final Task task = createTask(rpcCtx, request, cmdType); group.getNode().apply(task); } protected abstract CommandType getCmdType(); - private Task createTask(AsyncContext asyncCtx, T request, CommandType cmdType) { + private Task createTask(RpcContext asyncCtx, T request, CommandType cmdType) { final LeaderTaskClosure closure = new LeaderTaskClosure(); closure.setCmd(request); closure.setCmdType(cmdType); - closure.setDone(new Closure() { - - @Override - public void run(Status status) { - if (status.isOk()) { - asyncCtx.sendResponse(closure.getResponse()); - } else { - asyncCtx.sendResponse(new BooleanCommand(false, status.getErrorMsg())); - } + closure.setDone(status -> { + if (status.isOk()) { + asyncCtx.sendResponse(closure.getResponse()); + } else { + asyncCtx.sendResponse(new BooleanCommand(false, status.getErrorMsg())); } }); final byte[] cmdBytes = CommandCodec.encodeCommand(request); @@ -75,8 +68,6 @@ public void run(Status status) { data.put(cmdType.toByte()); data.put(cmdBytes); data.flip(); - final Task task = new Task(data, closure); - return task; + return new Task(data, closure); } - } diff --git a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/GetCommandProcessor.java b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/GetCommandProcessor.java index 175fc123d..cdef71ec9 100644 --- a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/GetCommandProcessor.java +++ b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/GetCommandProcessor.java @@ -16,8 +16,7 @@ */ package com.alipay.sofa.jraft.test.atomic.server.processor; -import com.alipay.remoting.AsyncContext; -import com.alipay.remoting.BizContext; +import com.alipay.sofa.jraft.rpc.RpcContext; import com.alipay.sofa.jraft.test.atomic.KeyNotFoundException; import com.alipay.sofa.jraft.test.atomic.command.GetCommand; import com.alipay.sofa.jraft.test.atomic.command.ValueCommand; @@ -42,19 +41,19 @@ public GetCommandProcessor(AtomicServer server) { } @Override - public void handleRequest(BizContext bizCtx, AsyncContext asyncCtx, GetCommand request) { + public void handleRequest(RpcContext rpcCtx, GetCommand request) { if (request.isReadByStateMachine()) { - super.handleRequest(bizCtx, asyncCtx, request); + super.handleRequest(rpcCtx, request); } else { try { final AtomicRangeGroup group = server.getGroupBykey(request.getKey()); if (!request.isReadFromQuorum()) { - asyncCtx.sendResponse(new ValueCommand(group.getFsm().getValue(request.getKey()))); + rpcCtx.sendResponse(new ValueCommand(group.getFsm().getValue(request.getKey()))); } else { - group.readFromQuorum(request.getKey(), asyncCtx); + group.readFromQuorum(request.getKey(), rpcCtx); } } catch (final KeyNotFoundException e) { - asyncCtx.sendResponse(createKeyNotFoundResponse()); + rpcCtx.sendResponse(createKeyNotFoundResponse()); } } } diff --git a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/GetSlotsCommandProcessor.java b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/GetSlotsCommandProcessor.java index 4bce83233..a2162bf3a 100644 --- a/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/GetSlotsCommandProcessor.java +++ b/jraft-test/src/main/java/com/alipay/sofa/jraft/test/atomic/server/processor/GetSlotsCommandProcessor.java @@ -16,13 +16,13 @@ */ package com.alipay.sofa.jraft.test.atomic.server.processor; -import com.alipay.remoting.BizContext; -import com.alipay.remoting.rpc.protocol.SyncUserProcessor; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; import com.alipay.sofa.jraft.test.atomic.command.GetSlotsCommand; import com.alipay.sofa.jraft.test.atomic.command.SlotsResponseCommand; import com.alipay.sofa.jraft.test.atomic.server.AtomicServer; -public class GetSlotsCommandProcessor extends SyncUserProcessor { +public class GetSlotsCommandProcessor implements RpcProcessor { private AtomicServer server; public GetSlotsCommandProcessor(AtomicServer server) { @@ -31,10 +31,10 @@ public GetSlotsCommandProcessor(AtomicServer server) { } @Override - public Object handleRequest(BizContext bizCtx, GetSlotsCommand request) throws Exception { - SlotsResponseCommand response = new SlotsResponseCommand(); + public void handleRequest(final RpcContext rpcCtx, final GetSlotsCommand request) { + final SlotsResponseCommand response = new SlotsResponseCommand(); response.setMap(this.server.getGroups()); - return response; + rpcCtx.sendResponse(response); } @Override diff --git a/pom.xml b/pom.xml index b5edc458d..da6779962 100644 --- a/pom.xml +++ b/pom.xml @@ -38,6 +38,7 @@ jraft-test jraft-example jraft-rheakv + jraft-extension