Skip to content

Commit

Permalink
HDDS-2020. Remove mTLS from Ozone GRPC. Contributed by Xiaoyu Yao.
Browse files Browse the repository at this point in the history
Signed-off-by: Anu Engineer <aengineer@apache.org>
  • Loading branch information
xiaoyuyao authored and anuengineer committed Sep 20, 2019
1 parent 9be448b commit d072d33
Show file tree
Hide file tree
Showing 48 changed files with 407 additions and 413 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -52,8 +52,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.io.IOException;
import java.security.cert.X509Certificate;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
Expand All @@ -80,15 +80,18 @@ public class XceiverClientGrpc extends XceiverClientSpi {
private boolean closed = false;
private SecurityConfig secConfig;
private final boolean topologyAwareRead;
private X509Certificate caCert;

/**
* Constructs a client that can communicate with the Container framework on
* data nodes.
*
* @param pipeline - Pipeline that defines the machines.
* @param config -- Ozone Config
* @param caCert - SCM ca certificate.
*/
public XceiverClientGrpc(Pipeline pipeline, Configuration config) {
public XceiverClientGrpc(Pipeline pipeline, Configuration config,
X509Certificate caCert) {
super();
Preconditions.checkNotNull(pipeline);
Preconditions.checkNotNull(config);
Expand All @@ -103,6 +106,18 @@ public XceiverClientGrpc(Pipeline pipeline, Configuration config) {
this.topologyAwareRead = config.getBoolean(
OzoneConfigKeys.OZONE_NETWORK_TOPOLOGY_AWARE_READ_KEY,
OzoneConfigKeys.OZONE_NETWORK_TOPOLOGY_AWARE_READ_DEFAULT);
this.caCert = caCert;
}

/**
* Constructs a client that can communicate with the Container framework on
* data nodes.
*
* @param pipeline - Pipeline that defines the machines.
* @param config -- Ozone Config
*/
public XceiverClientGrpc(Pipeline pipeline, Configuration config) {
this(pipeline, config, null);
}

/**
Expand Down Expand Up @@ -151,19 +166,10 @@ private void connectToDatanode(DatanodeDetails dn, String encodedToken)
.intercept(new ClientCredentialInterceptor(userName, encodedToken),
new GrpcClientInterceptor());
if (secConfig.isGrpcTlsEnabled()) {
File trustCertCollectionFile = secConfig.getTrustStoreFile(COMPONENT);
File privateKeyFile = secConfig.getClientPrivateKeyFile(COMPONENT);
File clientCertChainFile = secConfig.getClientCertChainFile(COMPONENT);

SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient();
if (trustCertCollectionFile != null) {
sslContextBuilder.trustManager(trustCertCollectionFile);
}
if (secConfig.isGrpcMutualTlsRequired() && clientCertChainFile != null
&& privateKeyFile != null) {
sslContextBuilder.keyManager(clientCertChainFile, privateKeyFile);
if (caCert != null) {
sslContextBuilder.trustManager(caCert);
}

if (secConfig.useTestCert()) {
channelBuilder.overrideAuthority("localhost");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneSecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
Expand All @@ -39,6 +41,8 @@

import java.io.Closeable;
import java.io.IOException;
import java.security.cert.CertificateException;
import java.security.cert.X509Certificate;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;

Expand All @@ -65,6 +69,7 @@ public class XceiverClientManager implements Closeable {
private final Configuration conf;
private final Cache<String, XceiverClientSpi> clientCache;
private final boolean useRatis;
private X509Certificate caCert;

private static XceiverClientMetrics metrics;
private boolean isSecurityEnabled;
Expand All @@ -74,11 +79,13 @@ public class XceiverClientManager implements Closeable {
*
* @param conf configuration
*/
public XceiverClientManager(Configuration conf) {
this(conf, OzoneConfiguration.of(conf).getObject(ScmClientConfig.class));
public XceiverClientManager(Configuration conf) throws IOException {
this(conf, OzoneConfiguration.of(conf).getObject(ScmClientConfig.class),
null);
}

public XceiverClientManager(Configuration conf, ScmClientConfig clientConf) {
public XceiverClientManager(Configuration conf, ScmClientConfig clientConf,
String caCertPem) throws IOException {
Preconditions.checkNotNull(clientConf);
Preconditions.checkNotNull(conf);
long staleThresholdMs = clientConf.getStaleThreshold(MILLISECONDS);
Expand All @@ -87,6 +94,16 @@ public XceiverClientManager(Configuration conf, ScmClientConfig clientConf) {
ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
this.conf = conf;
this.isSecurityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf);
if (isSecurityEnabled) {
Preconditions.checkNotNull(caCertPem);
try {
this.caCert = CertificateCodec.getX509Cert(caCertPem);
} catch (CertificateException ex) {
throw new SCMSecurityException("Error: Fail to get SCM CA certificate",
ex);
}
}

this.clientCache = CacheBuilder.newBuilder()
.expireAfterAccess(staleThresholdMs, MILLISECONDS)
.maximumSize(clientConf.getMaxSize())
Expand Down Expand Up @@ -211,11 +228,12 @@ public XceiverClientSpi call() throws Exception {
XceiverClientSpi client = null;
switch (type) {
case RATIS:
client = XceiverClientRatis.newXceiverClientRatis(pipeline, conf);
client = XceiverClientRatis.newXceiverClientRatis(pipeline, conf,
caCert);
client.connect();
break;
case STAND_ALONE:
client = new XceiverClientGrpc(pipeline, conf);
client = new XceiverClientGrpc(pipeline, conf, caCert);
break;
case CHAINED:
default:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.scm;

import java.io.IOException;
import java.security.cert.X509Certificate;
import java.util.Collection;
import java.util.List;
import java.util.Objects;
Expand Down Expand Up @@ -78,6 +79,12 @@ public final class XceiverClientRatis extends XceiverClientSpi {
public static XceiverClientRatis newXceiverClientRatis(
org.apache.hadoop.hdds.scm.pipeline.Pipeline pipeline,
Configuration ozoneConf) {
return newXceiverClientRatis(pipeline, ozoneConf, null);
}

public static XceiverClientRatis newXceiverClientRatis(
org.apache.hadoop.hdds.scm.pipeline.Pipeline pipeline,
Configuration ozoneConf, X509Certificate caCert) {
final String rpcType = ozoneConf
.get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
Expand All @@ -87,7 +94,7 @@ public static XceiverClientRatis newXceiverClientRatis(
HddsClientUtils.getMaxOutstandingRequests(ozoneConf);
final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(ozoneConf);
final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig(new
SecurityConfig(ozoneConf));
SecurityConfig(ozoneConf), caCert);
return new XceiverClientRatis(pipeline,
SupportedRpcType.valueOfIgnoreCase(rpcType), maxOutstandingRequests,
retryPolicy, tlsConfig, clientRequestTimeout);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -176,34 +176,18 @@ public final class HddsConfigKeys {
private HddsConfigKeys() {
}

// Enable TLS for GRPC clients/server in ozone.
public static final String HDDS_GRPC_TLS_ENABLED = "hdds.grpc.tls.enabled";
public static final boolean HDDS_GRPC_TLS_ENABLED_DEFAULT = false;

public static final String HDDS_GRPC_MUTUAL_TLS_REQUIRED =
"hdds.grpc.mutual.tls.required";
public static final boolean HDDS_GRPC_MUTUAL_TLS_REQUIRED_DEFAULT = false;

// Choose TLS provider the default is set to OPENSSL for better performance.
public static final String HDDS_GRPC_TLS_PROVIDER = "hdds.grpc.tls.provider";
public static final String HDDS_GRPC_TLS_PROVIDER_DEFAULT = "OPENSSL";

public static final String HDDS_TRUST_STORE_FILE_NAME =
"hdds.trust.cert.collection.file.name";
public static final String HDDS_TRUST_STORE_FILE_NAME_DEFAULT = "ca.crt";

public static final String
HDDS_SERVER_CERTIFICATE_CHAIN_FILE_NAME =
"hdds.server.cert.chain.file.name";
public static final String
HDDS_SERVER_CERTIFICATE_CHAIN_FILE_NAME_DEFAULT = "server.crt";

public static final String
HDDS_CLIENT_CERTIFICATE_CHAIN_FILE_NAME =
"hdds.client.cert.chain.file.name";
public static final String
HDDS_CLIENT_CERTIFICATE_CHAIN_FILE_NAME_DEFAULT = "client.crt";

// Test only settings for using test signed certificate, authority assume to
// be localhost.
public static final String HDDS_GRPC_TLS_TEST_CERT = "hdds.grpc.tls" +
".test_cert";
".test.cert";
public static final boolean HDDS_GRPC_TLS_TEST_CERT_DEFAULT = false;

// Comma separated acls (users, groups) allowing clients accessing
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
package org.apache.hadoop.hdds.ratis;

import java.io.IOException;
import java.security.cert.CertificateException;
import java.security.cert.X509Certificate;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
Expand All @@ -31,7 +33,11 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;

Expand Down Expand Up @@ -200,29 +206,47 @@ static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
return builder.build();
}

static GrpcTlsConfig createTlsClientConfig(SecurityConfig conf) {
if (conf.isGrpcTlsEnabled()) {
if (conf.isGrpcMutualTlsRequired()) {
return new GrpcTlsConfig(conf.getClientPrivateKeyFile(),
conf.getClientCertChainFile(), conf.getTrustStoreFile(), true);
} else {
return new GrpcTlsConfig(
null, null, conf.getTrustStoreFile(), false);
// For External gRPC client to server with gRPC TLS.
// No mTLS for external client as SCM CA does not issued certificates for them
static GrpcTlsConfig createTlsClientConfig(SecurityConfig conf,
X509Certificate caCert) {
GrpcTlsConfig tlsConfig = null;
if (conf.isSecurityEnabled() && conf.isGrpcTlsEnabled()) {
tlsConfig = new GrpcTlsConfig(null, null,
caCert, false);
}
return tlsConfig;
}

// For Internal gRPC client from SCM to DN with gRPC TLS
static GrpcTlsConfig createTlsClientConfigForSCM(SecurityConfig conf,
CertificateServer certificateServer) throws IOException {
if (conf.isSecurityEnabled() && conf.isGrpcTlsEnabled()) {
try {
X509Certificate caCert =
CertificateCodec.getX509Certificate(
certificateServer.getCACertificate());
return new GrpcTlsConfig(null, null,
caCert, false);
} catch (CertificateException ex) {
throw new SCMSecurityException("Fail to find SCM CA certificate.", ex);
}
}
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);
}
// For gRPC server running DN container service with gPRC TLS
// No mTLS as the channel is shared for for external client, which
// does not have SCM CA issued certificates.
// In summary:
// authenticate from server to client is via TLS.
// authenticate from client to server is via block token (or container token).
static GrpcTlsConfig createTlsServerConfigForDN(SecurityConfig conf,
CertificateClient caClient) {
if (conf.isSecurityEnabled() && conf.isGrpcTlsEnabled()) {
return new GrpcTlsConfig(
caClient.getPrivateKey(), caClient.getCertificate(),
null, false);
}
return null;
}
Expand Down
Loading

0 comments on commit d072d33

Please sign in to comment.