|
@@ -21,11 +21,14 @@ package org.apache.ratis;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
import org.apache.ratis.client.RaftClient;
|
|
|
import org.apache.ratis.conf.RaftProperties;
|
|
|
import org.apache.ratis.grpc.GrpcConfigKeys;
|
|
|
+import org.apache.ratis.grpc.GrpcFactory;
|
|
|
+import org.apache.ratis.grpc.GrpcTlsConfig;
|
|
|
import org.apache.ratis.protocol.RaftGroup;
|
|
|
import org.apache.ratis.protocol.RaftGroupId;
|
|
|
import org.apache.ratis.protocol.RaftPeer;
|
|
@@ -33,6 +36,7 @@ import org.apache.ratis.protocol.RaftPeerId;
|
|
|
import org.apache.ratis.retry.RetryPolicies;
|
|
|
import org.apache.ratis.retry.RetryPolicy;
|
|
|
import org.apache.ratis.rpc.RpcType;
|
|
|
+import org.apache.ratis.rpc.SupportedRpcType;
|
|
|
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
|
|
|
import org.apache.ratis.proto.RaftProtos;
|
|
|
import org.apache.ratis.util.SizeInBytes;
|
|
@@ -128,38 +132,77 @@ public interface RatisHelper {
|
|
|
}
|
|
|
|
|
|
static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline,
|
|
|
- RetryPolicy retryPolicy) throws IOException {
|
|
|
+ RetryPolicy retryPolicy, int maxOutStandingRequest,
|
|
|
+ GrpcTlsConfig tlsConfig) throws IOException {
|
|
|
return newRaftClient(rpcType, toRaftPeerId(pipeline.getFirstNode()),
|
|
|
newRaftGroup(RaftGroupId.valueOf(pipeline.getId().getId()),
|
|
|
- pipeline.getNodes()), retryPolicy);
|
|
|
+ pipeline.getNodes()), retryPolicy, maxOutStandingRequest, tlsConfig);
|
|
|
}
|
|
|
|
|
|
static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
|
|
|
- RetryPolicy retryPolicy) {
|
|
|
+ RetryPolicy retryPolicy, int maxOutstandingRequests,
|
|
|
+ GrpcTlsConfig tlsConfig) {
|
|
|
return newRaftClient(rpcType, leader.getId(),
|
|
|
- newRaftGroup(new ArrayList<>(Arrays.asList(leader))), retryPolicy);
|
|
|
+ newRaftGroup(new ArrayList<>(Arrays.asList(leader))), retryPolicy,
|
|
|
+ maxOutstandingRequests, tlsConfig);
|
|
|
}
|
|
|
|
|
|
static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
|
|
|
- RaftGroup group, RetryPolicy retryPolicy) {
|
|
|
- return newRaftClient(rpcType, leader.getId(), group, retryPolicy);
|
|
|
+ RetryPolicy retryPolicy, int maxOutstandingRequests) {
|
|
|
+ return newRaftClient(rpcType, leader.getId(),
|
|
|
+ newRaftGroup(new ArrayList<>(Arrays.asList(leader))), retryPolicy,
|
|
|
+ maxOutstandingRequests, null);
|
|
|
}
|
|
|
|
|
|
static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
|
|
|
- RaftGroup group, RetryPolicy retryPolicy) {
|
|
|
+ RaftGroup group, RetryPolicy retryPolicy, int maxOutStandingRequest,
|
|
|
+ GrpcTlsConfig tlsConfig) {
|
|
|
LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, leader, group);
|
|
|
final RaftProperties properties = new RaftProperties();
|
|
|
RaftConfigKeys.Rpc.setType(properties, rpcType);
|
|
|
|
|
|
GrpcConfigKeys.setMessageSizeMax(properties,
|
|
|
SizeInBytes.valueOf(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE));
|
|
|
-
|
|
|
- return RaftClient.newBuilder()
|
|
|
+ GrpcConfigKeys.OutputStream.setOutstandingAppendsMax(properties,
|
|
|
+ maxOutStandingRequest);
|
|
|
+ RaftClient.Builder builder = RaftClient.newBuilder()
|
|
|
.setRaftGroup(group)
|
|
|
.setLeaderId(leader)
|
|
|
.setProperties(properties)
|
|
|
- .setRetryPolicy(retryPolicy)
|
|
|
- .build();
|
|
|
+ .setRetryPolicy(retryPolicy);
|
|
|
+
|
|
|
+ // TODO: GRPC TLS only for now, netty/hadoop RPC TLS support later.
|
|
|
+ if (tlsConfig != null && rpcType == SupportedRpcType.GRPC) {
|
|
|
+ builder.setParameters(GrpcFactory.newRaftParameters(tlsConfig));
|
|
|
+ }
|
|
|
+ return builder.build();
|
|
|
+ }
|
|
|
+
|
|
|
+ static GrpcTlsConfig createTlsClientConfig(SecurityConfig conf) {
|
|
|
+ if (conf.isGrpcTlsEnabled()) {
|
|
|
+ if (conf.isGrpcMutualTlsRequired()) {
|
|
|
+ return new GrpcTlsConfig(
|
|
|
+ null, null, conf.getTrustStoreFile(), false);
|
|
|
+ } else {
|
|
|
+ return new GrpcTlsConfig(conf.getClientPrivateKeyFile(),
|
|
|
+ conf.getClientCertChainFile(), conf.getTrustStoreFile(), true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ static GrpcTlsConfig createTlsServerConfig(SecurityConfig conf) {
|
|
|
+ if (conf.isGrpcTlsEnabled()) {
|
|
|
+ if (conf.isGrpcMutualTlsRequired()) {
|
|
|
+ return new GrpcTlsConfig(
|
|
|
+ conf.getServerPrivateKeyFile(), conf.getServerCertChainFile(), null,
|
|
|
+ false);
|
|
|
+ } else {
|
|
|
+ return new GrpcTlsConfig(conf.getServerPrivateKeyFile(),
|
|
|
+ conf.getServerCertChainFile(), conf.getClientCertChainFile(), true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return null;
|
|
|
}
|
|
|
|
|
|
static RetryPolicy createRetryPolicy(Configuration conf) {
|