You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ad...@apache.org on 2022/04/11 18:23:26 UTC

[ozone] 07/08: HDDS-5545. Enable TLS for GRPC OmTransport implementation (#2945)

This is an automated email from the ASF dual-hosted git repository.

adoroszlai pushed a commit to branch HDDS-4440-s3-performance
in repository https://gitbox.apache.org/repos/asf/ozone.git

commit 47e457697e57551e01545b2a92f71913e6146c3a
Author: Neil Joshi <ne...@gmail.com>
AuthorDate: Tue Mar 29 12:11:14 2022 -0600

    HDDS-5545. Enable TLS for GRPC OmTransport implementation (#2945)
---
 hadoop-ozone/common/pom.xml                        | 11 +++
 .../org/apache/hadoop/ozone/om/OMConfigKeys.java   |  1 -
 .../ozone/om/protocolPB/GrpcOmTransport.java       | 39 ++++++++-
 .../ozone/om/protocolPB/OmTransportFactory.java    |  8 +-
 ...OzoneManagerProtocolClientSideTranslatorPB.java |  2 +-
 hadoop-ozone/dist/src/main/license/bin/LICENSE.txt |  2 +
 hadoop-ozone/dist/src/main/license/jar-report.txt  |  2 +
 .../src/main/proto/OmClientProtocol.proto          |  2 +-
 hadoop-ozone/ozone-manager/pom.xml                 | 11 +++
 .../hadoop/ozone/om/GrpcOzoneManagerServer.java    | 46 +++++++++--
 .../org/apache/hadoop/ozone/om/OzoneManager.java   |  3 +-
 .../ozone/om/TestGrpcOzoneManagerServer.java       |  3 +-
 .../apache/hadoop/ozone/s3/OzoneClientCache.java   | 96 +++++++++++++++++++++-
 .../hadoop/ozone/s3/endpoint/EndpointBase.java     |  2 +-
 .../ozone/protocolPB/TestGrpcOmTransport.java      | 13 +++
 pom.xml                                            |  3 +
 16 files changed, 222 insertions(+), 22 deletions(-)

diff --git a/hadoop-ozone/common/pom.xml b/hadoop-ozone/common/pom.xml
index 7c9518ab53..701e6d5e6a 100644
--- a/hadoop-ozone/common/pom.xml
+++ b/hadoop-ozone/common/pom.xml
@@ -48,6 +48,17 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
     <dependency>
       <groupId>io.netty</groupId>
       <artifactId>netty-handler-proxy</artifactId>
+    </dependency>
+      <dependency>
+        <groupId>io.netty</groupId>
+        <artifactId>netty-tcnative-boringssl-static</artifactId>
+        <version>${tcnative.version}</version>
+        <scope>runtime</scope>
+      </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-tcnative</artifactId>
+      <version>${tcnative.version}</version>
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
index 6ebd7e11ad..e679404eba 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
@@ -307,5 +307,4 @@ public final class OMConfigKeys {
   public static final String OZONE_OM_TRANSPORT_CLASS_DEFAULT =
       "org.apache.hadoop.ozone.om.protocolPB"
           + ".Hadoop3OmTransportFactory";
-
 }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/GrpcOmTransport.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/GrpcOmTransport.java
index 72c29f0cc6..764f51b2f6 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/GrpcOmTransport.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/GrpcOmTransport.java
@@ -19,11 +19,12 @@ package org.apache.hadoop.ozone.om.protocolPB;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
+import java.security.cert.X509Certificate;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import com.google.common.net.HostAndPort;
@@ -35,6 +36,7 @@ import org.apache.hadoop.hdds.conf.Config;
 import org.apache.hadoop.hdds.conf.ConfigGroup;
 import org.apache.hadoop.hdds.conf.ConfigTag;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -48,7 +50,9 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.ozone.om.ha.GrpcOMFailoverProxyProvider;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerServiceGrpc;
 import io.grpc.ManagedChannel;
+import io.grpc.netty.GrpcSslContexts;
 import io.grpc.netty.NettyChannelBuilder;
+import io.netty.handler.ssl.SslContextBuilder;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
@@ -70,6 +74,8 @@ public class GrpcOmTransport implements OmTransport {
   private final AtomicBoolean isRunning = new AtomicBoolean(false);
 
   // gRPC specific
+  private static List<X509Certificate> caCerts = null;
+
   private OzoneManagerServiceGrpc.OzoneManagerServiceBlockingStub client;
   private Map<String,
       OzoneManagerServiceGrpc.OzoneManagerServiceBlockingStub> clients;
@@ -77,9 +83,13 @@ public class GrpcOmTransport implements OmTransport {
   private int lastVisited = -1;
   private ConfigurationSource conf;
 
-  //private String host = "om";
   private AtomicReference<String> host;
   private int maxSize;
+  private SecurityConfig secConfig;
+
+  public static void setCaCerts(List<X509Certificate> x509Certificates) {
+    caCerts = x509Certificates;
+  }
 
   private List<String> oms;
   private RetryPolicy retryPolicy;
@@ -96,6 +106,7 @@ public class GrpcOmTransport implements OmTransport {
     this.conf = conf;
     this.host = new AtomicReference();
 
+    secConfig =  new SecurityConfig(conf);
     maxSize = conf.getInt(OZONE_OM_GRPC_MAXIMUM_RESPONSE_LENGTH,
         OZONE_OM_GRPC_MAXIMUM_RESPONSE_LENGTH_DEFAULT);
 
@@ -127,6 +138,29 @@ public class GrpcOmTransport implements OmTransport {
           NettyChannelBuilder.forAddress(hp.getHost(), hp.getPort())
               .usePlaintext()
               .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE);
+
+      if (secConfig.isGrpcTlsEnabled()) {
+        try {
+          SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient();
+          if (secConfig.isSecurityEnabled()) {
+            if (caCerts != null) {
+              sslContextBuilder.trustManager(caCerts);
+            } else {
+              LOG.error("x509Certicates empty");
+            }
+            channelBuilder.useTransportSecurity().
+                sslContext(sslContextBuilder.build());
+          } else {
+            LOG.error("ozone.security not enabled when TLS specified," +
+                " using plaintext");
+          }
+        } catch (Exception ex) {
+          LOG.error("cannot establish TLS for grpc om transport client");
+        }
+      } else {
+        channelBuilder.usePlaintext();
+      }
+
       channels.put(hostaddr, channelBuilder.build());
       clients.put(hostaddr,
           OzoneManagerServiceGrpc
@@ -136,7 +170,6 @@ public class GrpcOmTransport implements OmTransport {
         OzoneConfigKeys.OZONE_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
         OzoneConfigKeys.OZONE_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
 
-
     retryPolicy = omFailoverProxyProvider.getRetryPolicy(maxFailovers);
     LOG.info("{}: started", CLIENT_NAME);
   }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OmTransportFactory.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OmTransportFactory.java
index 2eb11d0320..2ba8536e18 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OmTransportFactory.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OmTransportFactory.java
@@ -46,8 +46,8 @@ public interface OmTransportFactory {
       throws IOException {
     try {
       // if configured transport class is different than the default
-      // Hadoop3OmTransportFactory, then check service loader for
-      // transport class and instantiate it
+      // OmTransportFactory (Hadoop3OmTransportFactory), then
+      // check service loader for transport class and instantiate it
       if (conf
           .get(OZONE_OM_TRANSPORT_CLASS,
               OZONE_OM_TRANSPORT_CLASS_DEFAULT) !=
@@ -61,9 +61,7 @@ public interface OmTransportFactory {
         }
       }
       return OmTransportFactory.class.getClassLoader()
-          .loadClass(
-              "org.apache.hadoop.ozone.om.protocolPB"
-                  + ".Hadoop3OmTransportFactory")
+          .loadClass(OZONE_OM_TRANSPORT_CLASS_DEFAULT)
           .asSubclass(OmTransportFactory.class)
           .newInstance();
     } catch (Exception ex) {
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
index be758a06d2..e368f10732 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
@@ -178,7 +178,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
   private OmTransport transport;
   private ThreadLocal<S3Auth> threadLocalS3Auth
       = new ThreadLocal<>();
-
+    
   private boolean s3AuthCheck;
   public OzoneManagerProtocolClientSideTranslatorPB(OmTransport omTransport,
       String clientId) {
diff --git a/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt b/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt
index 3439317598..20281ea0c5 100644
--- a/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt
+++ b/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt
@@ -309,6 +309,8 @@ Apache License
    io.netty:netty-handler
    io.netty:netty-handler-proxy
    io.netty:netty-resolver
+   io.netty:netty-tcnative-boringssl-static
+   io.netty:netty-tcnative
    io.netty:netty-transport
    io.netty:netty-transport-native-epoll
    io.netty:netty-transport-native-unix-common
diff --git a/hadoop-ozone/dist/src/main/license/jar-report.txt b/hadoop-ozone/dist/src/main/license/jar-report.txt
index ec949fdf33..a337955656 100644
--- a/hadoop-ozone/dist/src/main/license/jar-report.txt
+++ b/hadoop-ozone/dist/src/main/license/jar-report.txt
@@ -172,6 +172,8 @@ share/ozone/lib/netty-common.Final.jar
 share/ozone/lib/netty-handler.Final.jar
 share/ozone/lib/netty-handler-proxy.Final.jar
 share/ozone/lib/netty-resolver.Final.jar
+share/ozone/lib/netty-tcnative-boringssl-static.Final.jar
+share/ozone/lib/netty-tcnative.Final.jar
 share/ozone/lib/netty-transport.Final.jar
 share/ozone/lib/netty-transport-native-epoll.Final.jar
 share/ozone/lib/netty-transport-native-unix-common.Final.jar
diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
index 694d7df6b3..bbb4267a12 100644
--- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
+++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
@@ -1350,7 +1350,7 @@ message UpdateGetS3SecretRequest {
 }
 
 /**
-  This will be used by OM to authenicate S3 gateway requests on a per request basis.
+  This will be used by OM to authenticate S3 gateway requests on a per request basis.
 */
 message S3Authentication {
     required string stringToSign = 1;
diff --git a/hadoop-ozone/ozone-manager/pom.xml b/hadoop-ozone/ozone-manager/pom.xml
index cea140b122..bc2909e686 100644
--- a/hadoop-ozone/ozone-manager/pom.xml
+++ b/hadoop-ozone/ozone-manager/pom.xml
@@ -87,6 +87,17 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.bouncycastle</groupId>
       <artifactId>bcprov-jdk15on</artifactId>
     </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-tcnative</artifactId>
+      <version>${tcnative.version}</version>
+    </dependency>
+      <dependency>
+        <groupId>io.netty</groupId>
+        <artifactId>netty-tcnative-boringssl-static</artifactId>
+        <version>${tcnative.version}</version>
+        <scope>runtime</scope>
+      </dependency>
 
     <dependency>
       <groupId>org.mockito</groupId>
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/GrpcOzoneManagerServer.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/GrpcOzoneManagerServer.java
index 7fe338c83e..b083378fab 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/GrpcOzoneManagerServer.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/GrpcOzoneManagerServer.java
@@ -30,11 +30,23 @@ import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.ha.ConfUtils;
 import org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB;
 import org.apache.hadoop.ozone.security.OzoneDelegationTokenSecretManager;
-import io.grpc.Server;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import io.grpc.netty.GrpcSslContexts;
 import io.grpc.netty.NettyServerBuilder;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+import io.grpc.Server;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_GRPC_TLS_PROVIDER;
+import static org.apache.hadoop.hdds.HddsConfigKeys
+    .HDDS_GRPC_TLS_PROVIDER_DEFAULT;
+
 /**
  * Separated network server for gRPC transport OzoneManagerService s3g->OM.
  */
@@ -49,7 +61,8 @@ public class GrpcOzoneManagerServer {
                                 OzoneManagerProtocolServerSideTranslatorPB
                                     omTranslator,
                                 OzoneDelegationTokenSecretManager
-                                    delegationTokenMgr) {
+                                    delegationTokenMgr,
+                                CertificateClient caClient) {
     OptionalInt haPort = HddsUtils.getNumberFromConfigKeys(config,
         ConfUtils.addKeySuffixes(
             OMConfigKeys.OZONE_OM_GRPC_PORT_KEY,
@@ -63,21 +76,44 @@ public class GrpcOzoneManagerServer {
               GrpcOzoneManagerServerConfig.class).
           getPort();
     }
-
+    
     init(omTranslator,
         delegationTokenMgr,
-        config);
+        config,
+        caClient);
   }
 
   public void init(OzoneManagerProtocolServerSideTranslatorPB omTranslator,
                    OzoneDelegationTokenSecretManager delegationTokenMgr,
-                   OzoneConfiguration omServerConfig) {
+                   OzoneConfiguration omServerConfig,
+                   CertificateClient caClient) {
     NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forPort(port)
         .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
         .addService(new OzoneManagerServiceGrpc(omTranslator,
             delegationTokenMgr,
             omServerConfig));
 
+    SecurityConfig secConf = new SecurityConfig(omServerConfig);
+    if (secConf.isGrpcTlsEnabled()) {
+      try {
+        if (secConf.isSecurityEnabled()) {
+          SslContextBuilder sslClientContextBuilder =
+              SslContextBuilder.forServer(caClient.getPrivateKey(),
+                  caClient.getCertificate());
+          SslContextBuilder sslContextBuilder = GrpcSslContexts.configure(
+              sslClientContextBuilder,
+              SslProvider.valueOf(omServerConfig.get(HDDS_GRPC_TLS_PROVIDER,
+                  HDDS_GRPC_TLS_PROVIDER_DEFAULT)));
+          nettyServerBuilder.sslContext(sslContextBuilder.build());
+        } else {
+          LOG.error("ozone.security not enabled when TLS specified," +
+                            " creating Om S3g GRPC channel using plaintext");
+        }
+      } catch (Exception ex) {
+        LOG.error("Unable to setup TLS for secure Om S3g GRPC channel.", ex);
+      }
+    }
+
     server = nettyServerBuilder.build();
   }
 
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 6297199c27..a02b3197a0 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -1107,7 +1107,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
           throws IOException {
     return new GrpcOzoneManagerServer(conf,
             this.omServerProtocol,
-            this.delegationTokenMgr);
+            this.delegationTokenMgr,
+            this.certClient);
   }
 
   private static boolean isOzoneSecurityEnabled() {
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestGrpcOzoneManagerServer.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestGrpcOzoneManagerServer.java
index b75a651eea..e58b7a47ea 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestGrpcOzoneManagerServer.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestGrpcOzoneManagerServer.java
@@ -49,7 +49,8 @@ public class TestGrpcOzoneManagerServer {
 
     server = new GrpcOzoneManagerServer(conf,
         omServerProtocol,
-        ozoneManager.getDelegationTokenMgr());
+        ozoneManager.getDelegationTokenMgr(),
+        ozoneManager.getCertificateClient());
 
     try {
       server.start();
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientCache.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientCache.java
index ab88b18e7e..80f0b643fc 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientCache.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientCache.java
@@ -19,15 +19,25 @@ package org.apache.hadoop.ozone.s3;
 
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.om.protocol.S3Auth;
+import org.apache.hadoop.ozone.om.helpers.ServiceInfoEx;
+import org.apache.hadoop.ozone.om.protocolPB.GrpcOmTransport;
+import org.apache.hadoop.ozone.OzoneSecurityUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.PreDestroy;
 import javax.enterprise.context.ApplicationScoped;
 import java.io.IOException;
+import java.security.cert.CertificateException;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_TRANSPORT_CLASS;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_TRANSPORT_CLASS_DEFAULT;
 
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_CLIENT_PROTOCOL_VERSION;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_CLIENT_PROTOCOL_VERSION_KEY;
@@ -43,16 +53,29 @@ public final class OzoneClientCache {
   // for s3g gRPC OmTransport, OmRequest - OmResponse channel
   private static OzoneClientCache instance;
   private OzoneClient client;
+  private SecurityConfig secConfig;
 
   private OzoneClientCache(OzoneConfiguration ozoneConfiguration)
       throws IOException {
-    // S3 Gateway should always set the S3 Auth.
-    ozoneConfiguration.setBoolean(S3Auth.S3_AUTH_CHECK, true);
     // Set the expected OM version if not set via config.
     ozoneConfiguration.setIfUnset(OZONE_OM_CLIENT_PROTOCOL_VERSION_KEY,
         OZONE_OM_CLIENT_PROTOCOL_VERSION);
     String omServiceID = OmUtils.getOzoneManagerServiceId(ozoneConfiguration);
+    secConfig = new SecurityConfig(ozoneConfiguration);
+    client = null;
     try {
+      if (secConfig.isGrpcTlsEnabled()) {
+        if (ozoneConfiguration
+            .get(OZONE_OM_TRANSPORT_CLASS,
+                OZONE_OM_TRANSPORT_CLASS_DEFAULT) !=
+            OZONE_OM_TRANSPORT_CLASS_DEFAULT) {
+          // Grpc transport selected
+          // need to get certificate for TLS through
+          // hadoop rpc first via ServiceInfo
+          setCertificate(omServiceID,
+              ozoneConfiguration);
+        }
+      }
       if (omServiceID == null) {
         client = OzoneClientFactory.getRpcClient(ozoneConfiguration);
       } else {
@@ -64,6 +87,8 @@ public final class OzoneClientCache {
       LOG.warn("cannot create OzoneClient", e);
       throw e;
     }
+    // S3 Gateway should always set the S3 Auth.
+    ozoneConfiguration.setBoolean(S3Auth.S3_AUTH_CHECK, true);
   }
 
   public static OzoneClient getOzoneClientInstance(OzoneConfiguration
@@ -75,8 +100,73 @@ public final class OzoneClientCache {
     return instance.client;
   }
 
+  public static void closeClient() throws IOException {
+    if (instance != null) {
+      instance.client.close();
+      instance = null;
+    }
+  }
+
+  private void setCertificate(String omServiceID,
+                              OzoneConfiguration conf)
+      throws IOException {
+
+    // create local copy of config incase exception occurs
+    // with certificate OmRequest
+    OzoneConfiguration config = new OzoneConfiguration(conf);
+    OzoneClient certClient;
+
+    if (secConfig.isGrpcTlsEnabled()) {
+      // set OmTransport to hadoop rpc to securely,
+      // get certificates with service list request
+      config.set(OZONE_OM_TRANSPORT_CLASS,
+          OZONE_OM_TRANSPORT_CLASS_DEFAULT);
+
+      if (omServiceID == null) {
+        certClient = OzoneClientFactory.getRpcClient(config);
+      } else {
+        // As in HA case, we need to pass om service ID.
+        certClient = OzoneClientFactory.getRpcClient(omServiceID,
+            config);
+      }
+      try {
+        ServiceInfoEx serviceInfoEx = certClient
+            .getObjectStore()
+            .getClientProxy()
+            .getOzoneManagerClient()
+            .getServiceInfo();
+
+        if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
+          String caCertPem = null;
+          List<String> caCertPems = null;
+          caCertPem = serviceInfoEx.getCaCertificate();
+          caCertPems = serviceInfoEx.getCaCertPemList();
+          if (caCertPems == null || caCertPems.isEmpty()) {
+            if (caCertPem == null) {
+              LOG.error("S3g received empty caCertPems from serviceInfo");
+              throw new CertificateException("No caCerts found; caCertPem can" +
+                  " not be null when caCertPems is empty or null");
+            }
+            caCertPems = Collections.singletonList(caCertPem);
+          }
+          GrpcOmTransport.setCaCerts(OzoneSecurityUtil
+              .convertToX509(caCertPems));
+        }
+      } catch (CertificateException ce) {
+        throw new IOException(ce);
+      } catch (IOException e) {
+        throw e;
+      } finally {
+        if (certClient != null) {
+          certClient.close();
+        }
+      }
+    }
+  }
+
+
   @PreDestroy
   public void destroy() throws IOException {
-    client.close();
+    OzoneClientCache.closeClient();
   }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java
index 248ee92cfd..162d0b55aa 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java
@@ -58,7 +58,7 @@ public abstract class EndpointBase {
     } catch (OMException ex) {
       if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
         throw S3ErrorTable.newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName);
-      } else if (ex.getResult() == ResultCodes.S3_SECRET_NOT_FOUND) {
+      } else if (ex.getResult() == ResultCodes.INVALID_TOKEN) {
         throw S3ErrorTable.newError(S3ErrorTable.ACCESS_DENIED,
             s3Auth.getAccessID());
       } else if (ex.getResult() == ResultCodes.TIMEOUT ||
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/protocolPB/TestGrpcOmTransport.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/protocolPB/TestGrpcOmTransport.java
index a158e0212e..a28f47a809 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/protocolPB/TestGrpcOmTransport.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/protocolPB/TestGrpcOmTransport.java
@@ -58,6 +58,19 @@ public class TestGrpcOmTransport {
 
   }
 
+  @Test
+  public void testHrpcOmTransportFactory() throws Exception {
+    String omServiceId = "";
+    OzoneConfiguration conf = new OzoneConfiguration();
+
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    OmTransport omTransport = OmTransportFactory.create(conf, ugi, omServiceId);
+    // OmTransport should be Hadoop Rpc and
+    // fail equality GrpcOmTransport equality test
+    Assert.assertNotEquals(GrpcOmTransport.class.getSimpleName(),
+        omTransport.getClass().getSimpleName());
+  }
+
   @Test
   public void testStartStop() throws Exception {
     String omServiceId = "";
diff --git a/pom.xml b/pom.xml
index 11a0ad1ee9..9b282a68f3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -194,6 +194,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
 
     <netty.version>4.1.63.Final</netty.version>
     <io.grpc.version>1.38.0</io.grpc.version>
+    <tcnative.version>2.0.38.Final</tcnative.version> <!-- See table for correct version -->
+    <!-- Table for netty, grpc & tcnative version combinations  -->
+    <!-- https://github.com/grpc/grpc-java/blob/master/SECURITY.md#netty -->
 
     <!-- define the Java language version used by the compiler -->
     <javac.version>1.8</javac.version>


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org