You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by na...@apache.org on 2023/03/23 09:08:22 UTC

[ozone] branch master updated: HDDS-8241. Transfer leader command doesn't work in secure cluster (#4454)

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

nanda pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new e38e4721a2 HDDS-8241. Transfer leader command doesn't work in secure cluster (#4454)
e38e4721a2 is described below

commit e38e4721a22f98b6a2ae61af24c20d6a5382195b
Author: Sammi Chen <sa...@apache.org>
AuthorDate: Thu Mar 23 17:08:16 2023 +0800

    HDDS-8241. Transfer leader command doesn't work in secure cluster (#4454)
---
 .../org/apache/hadoop/hdds/ratis/RatisHelper.java  |  4 +-
 .../hdds/scm/server/SCMClientProtocolServer.java   |  9 ++-
 .../src/main/compose/ozonesecure-ha/docker-config  | 20 +++---
 .../dist/src/main/compose/ozonesecure-ha/test.sh   |  5 +-
 .../admincli/{scmha.robot => scmrole.robot}        |  2 +-
 .../main/smoketest/omha/om-leader-transfer.robot   | 73 ++++++++++++++++++++++
 .../dist/src/main/smoketest/omha/testOMHA.robot    | 61 +-----------------
 .../main/smoketest/scmha/scm-leader-transfer.robot | 48 ++++++++++++++
 .../org/apache/hadoop/ozone/om/OzoneManager.java   |  7 ++-
 .../ozone/om/ratis/OzoneManagerRatisServer.java    | 16 +----
 .../om/ratis/utils/OzoneManagerRatisUtils.java     | 19 ++++++
 11 files changed, 175 insertions(+), 89 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
index b24ffddeec..1ad339b7dd 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
@@ -501,11 +501,11 @@ public final class RatisHelper {
    * @throws IOException
    */
   public static void transferRatisLeadership(ConfigurationSource conf,
-      RaftGroup raftGroup, RaftPeerId targetPeerId)
+      RaftGroup raftGroup, RaftPeerId targetPeerId, GrpcTlsConfig tlsConfig)
       throws IOException {
     // TODO: need a common raft client related conf.
     try (RaftClient raftClient = newRaftClient(SupportedRpcType.GRPC, null,
-        null, raftGroup, createRetryPolicy(conf), null, conf)) {
+        null, raftGroup, createRetryPolicy(conf), tlsConfig, conf)) {
       if (raftGroup.getPeer(targetPeerId) == null) {
         throw new IOException("Cannot choose the target leader. The expected " +
             "leader RaftPeerId is " + targetPeerId + " and the peers are " +
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
index ccd4153698..6ee278b7a7 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
@@ -72,6 +72,7 @@ import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
 import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
 import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.hdds.utils.HddsServerUtil;
 import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
 import org.apache.hadoop.io.IOUtils;
@@ -89,6 +90,7 @@ import org.apache.hadoop.ozone.audit.SCMAction;
 import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.ratis.grpc.GrpcTlsConfig;
 import org.apache.ratis.protocol.RaftGroup;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
@@ -112,6 +114,7 @@ import java.util.stream.Stream;
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StorageContainerLocationProtocolService.newReflectiveBlockingService;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdds.scm.ha.HASecurityUtils.createSCMRatisTLSConfig;
 import static org.apache.hadoop.hdds.scm.server.StorageContainerManager.startRpcServer;
 import static org.apache.hadoop.hdds.server.ServerUtils.getRemoteUserName;
 import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenAddress;
@@ -833,8 +836,12 @@ public class SCMClientProtocolServer implements
       } else {
         targetPeerId = RaftPeerId.valueOf(newLeaderId);
       }
+      final GrpcTlsConfig tlsConfig =
+          createSCMRatisTLSConfig(new SecurityConfig(scm.getConfiguration()),
+          scm.getScmCertificateClient());
+
       RatisHelper.transferRatisLeadership(scm.getConfiguration(), group,
-          targetPeerId);
+          targetPeerId, tlsConfig);
     } catch (Exception ex) {
       auditSuccess = false;
       AUDIT.logReadFailure(buildAuditMessageForFailure(
diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/docker-config b/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/docker-config
index dce3cf4460..db0e68d51b 100644
--- a/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/docker-config
@@ -18,18 +18,18 @@
 CORE-SITE.XML_hadoop.proxyuser.httpfs.hosts=*
 CORE-SITE.XML_hadoop.proxyuser.httpfs.groups=*
 
-CORE-SITE.XML_fs.defaultFS=ofs://id1
+CORE-SITE.XML_fs.defaultFS=ofs://omservice
 CORE-SITE.XML_fs.trash.interval=1
 
-OZONE-SITE.XML_ozone.om.service.ids=id1
-OZONE-SITE.XML_ozone.om.internal.service.id=id1
-OZONE-SITE.XML_ozone.om.nodes.id1=om1,om2,om3
-OZONE-SITE.XML_ozone.om.address.id1.om1=om1
-OZONE-SITE.XML_ozone.om.address.id1.om2=om2
-OZONE-SITE.XML_ozone.om.address.id1.om3=om3
-OZONE-SITE.XML_ozone.om.http-address.id1.om1=om1
-OZONE-SITE.XML_ozone.om.http-address.id1.om2=om2
-OZONE-SITE.XML_ozone.om.http-address.id1.om3=om3
+OZONE-SITE.XML_ozone.om.service.ids=omservice
+OZONE-SITE.XML_ozone.om.internal.service.id=omservice
+OZONE-SITE.XML_ozone.om.nodes.omservice=om1,om2,om3
+OZONE-SITE.XML_ozone.om.address.omservice.om1=om1
+OZONE-SITE.XML_ozone.om.address.omservice.om2=om2
+OZONE-SITE.XML_ozone.om.address.omservice.om3=om3
+OZONE-SITE.XML_ozone.om.http-address.omservice.om1=om1
+OZONE-SITE.XML_ozone.om.http-address.omservice.om2=om2
+OZONE-SITE.XML_ozone.om.http-address.omservice.om3=om3
 OZONE-SITE.XML_ozone.om.ratis.enable=true
 
 OZONE-SITE.XML_ozone.scm.service.ids=scmservice
diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/test.sh b/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/test.sh
index c87e6dce23..9e1fcba1f1 100755
--- a/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/test.sh
@@ -21,7 +21,7 @@ COMPOSE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
 export COMPOSE_DIR
 
 export SECURITY_ENABLED=true
-export OM_SERVICE_ID="id1"
+export OM_SERVICE_ID="omservice"
 export SCM=scm1.org
 
 : ${OZONE_BUCKET_KEY_NAME:=key1}
@@ -50,6 +50,9 @@ done
 
 execute_robot_test ${SCM} admincli
 
+execute_robot_test ${SCM} omha/om-leader-transfer.robot
+execute_robot_test ${SCM} scmha/scm-leader-transfer.robot
+
 execute_robot_test ${SCM} httpfs
 
 export SCM=scm2.org
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/scmha.robot b/hadoop-ozone/dist/src/main/smoketest/admincli/scmrole.robot
similarity index 98%
rename from hadoop-ozone/dist/src/main/smoketest/admincli/scmha.robot
rename to hadoop-ozone/dist/src/main/smoketest/admincli/scmrole.robot
index c02333bcb4..b7bb96aef7 100644
--- a/hadoop-ozone/dist/src/main/smoketest/admincli/scmha.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/scmrole.robot
@@ -25,4 +25,4 @@ Test Timeout        5 minutes
 *** Test Cases ***
 Run scm roles
     ${output} =         Execute          ozone admin scm roles
-                        Should Match Regexp   ${output}  [scm:9894(:LEADER|)]
+                        Should Match Regexp   ${output}  [scm:9894(:LEADER|)]
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/smoketest/omha/om-leader-transfer.robot b/hadoop-ozone/dist/src/main/smoketest/omha/om-leader-transfer.robot
new file mode 100644
index 0000000000..e1895b1db4
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/omha/om-leader-transfer.robot
@@ -0,0 +1,73 @@
+# 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.
+
+*** Settings ***
+Documentation       Smoketest ozone cluster startup
+Library             OperatingSystem
+Library             BuiltIn
+Resource            ../commonlib.robot
+Test Timeout        5 minutes
+
+** Keywords ***
+Get OM Leader Node
+    ${result} =             Execute                 ozone admin om roles --service-id=omservice
+                            LOG                     ${result}
+                            Should Contain          ${result}               LEADER              1
+                            Should Contain          ${result}               FOLLOWER            2
+    ${omLine} =             Get Lines Containing String                     ${result}           LEADER
+    ${split1}               ${split2}               Split String            ${omLine}           :
+    ${leaderOM} =           Strip String            ${split1}
+                            LOG                     Leader OM: ${leaderOM}
+    [return]                ${leaderOM}
+
+Get One OM Follower Node
+    ${result} =             Execute                 ozone admin om roles --service-id=omservice
+                            LOG                     ${result}
+                            Should Contain          ${result}               LEADER              1
+                            Should Contain          ${result}               FOLLOWER            2
+    ${omLines} =            Get Lines Containing String                     ${result}           FOLLOWER
+    ${omLine} =             Get Line                ${omLines}              0
+    ${split1}               ${split2}               Split String            ${omLine}           :
+    ${followerOM} =         Strip String            ${split1}
+                            LOG                     Follower OM: ${followerOM}
+    [return]                ${followerOM}
+
+
+** Test Cases ***
+Transfer Leadership for OM
+    # Find Leader OM and one Follower OM
+    ${leaderOM} =           Get OM Leader Node
+                            LOG                     Leader OM: ${leaderOM}
+    ${followerOM} =         Get One OM Follower Node
+                            LOG                     Follower OM: ${followerOM}
+    # Transfer leadership to the Follower OM
+    ${result} =             Execute                 ozone admin om transfer --service-id=omservice -n ${followerOM}
+                            LOG                     ${result}
+                            Should Contain          ${result}               Transfer leadership successfully
+
+    ${newLeaderOM} =        Get OM Leader Node
+                            Should be Equal         ${followerOM}           ${newLeaderOM}
+
+Transfer Leadership for OM randomly
+    # Find Leader OM and one Follower OM
+    ${leaderOM} =           Get OM Leader Node
+                            LOG                     Leader OM: ${leaderOM}
+    # Transfer leadership to the Follower OM
+    ${result} =             Execute                 ozone admin om transfer -r
+                            LOG                     ${result}
+                            Should Contain          ${result}               Transfer leadership successfully
+
+    ${newLeaderOM} =        Get OM Leader Node
+                            Should Not be Equal     ${leaderOM}             ${newLeaderOM}
diff --git a/hadoop-ozone/dist/src/main/smoketest/omha/testOMHA.robot b/hadoop-ozone/dist/src/main/smoketest/omha/testOMHA.robot
index 9793a31149..70fc66a228 100644
--- a/hadoop-ozone/dist/src/main/smoketest/omha/testOMHA.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/omha/testOMHA.robot
@@ -91,29 +91,6 @@ Put Multiple Keys
             Put Key         ${FILE}                 ${tmpKey}
     END
 
-Get OM Leader Node
-    ${result} =             Execute                 ozone admin om roles --service-id=omservice
-                            LOG                     ${result}
-                            Should Contain          ${result}               LEADER              1
-                            Should Contain          ${result}               FOLLOWER            2
-    ${omLine} =             Get Lines Containing String                     ${result}           LEADER
-    ${split1}               ${split2}               Split String            ${omLine}           :
-    ${leaderOM} =           Strip String            ${split1}
-                            LOG                     Leader OM: ${leaderOM}
-    [return]                ${leaderOM}
-
-Get One OM Follower Node
-    ${result} =             Execute                 ozone admin om roles --service-id=omservice
-                            LOG                     ${result}
-                            Should Contain          ${result}               LEADER              1
-                            Should Contain          ${result}               FOLLOWER            2
-    ${omLines} =            Get Lines Containing String                     ${result}           FOLLOWER
-    ${omLine} =             Get Line                ${omLines}              0
-    ${split1}               ${split2}               Split String            ${omLine}           :
-    ${followerOM} =         Strip String            ${split1}
-                            LOG                     Follower OM: ${followerOM}
-    [return]                ${followerOM}
-
 Get Ratis Logs
     [arguments]             ${OM_HOST}
                             Set Global Variable     ${HOST}                 ${OM_HOST}
@@ -192,40 +169,4 @@ Restart OM and Verify Ratis Logs
     Should Be True                  ${numLogsAfter} >= ${numLogsLeader}         Restarted OM did not catch up with Leader OM
 
     # Verify that the logs match with the Leader OMs logs
-    List Should Contain Sub List    ${logsAfter}        ${logsLeader}
-
-Transfer Leadership for OM
-    # Check OM write operation before failover
-    Create volume and bucket
-    Write Test File
-
-    # Find Leader OM and one Follower OM
-    ${leaderOM} =           Get OM Leader Node
-                            LOG                     Leader OM: ${leaderOM}
-    ${followerOM} =         Get One OM Follower Node
-                            LOG                     Follower OM: ${followerOM}
-    # Transfer leadership to the Follower OM
-    ${result} =             Execute                 ozone admin om transfer --service-id=omservice -n ${followerOM}
-                            LOG                     ${result}
-                            Should Contain          ${result}               Transfer leadership successfully
-
-    ${newLeaderOM} =        Get OM Leader Node
-                            Should be Equal         ${followerOM}           ${newLeaderOM}
-    Write Test File
-
-Transfer Leadership for OM randomly
-    # Check OM write operation before failover
-    Create volume and bucket
-    Write Test File
-
-    # Find Leader OM and one Follower OM
-    ${leaderOM} =           Get OM Leader Node
-                            LOG                     Leader OM: ${leaderOM}
-    # Transfer leadership to the Follower OM
-    ${result} =             Execute                 ozone admin om transfer -r
-                            LOG                     ${result}
-                            Should Contain          ${result}               Transfer leadership successfully
-
-    ${newLeaderOM} =        Get OM Leader Node
-                            Should Not be Equal     ${leaderOM}             ${newLeaderOM}
-    Write Test File
+    List Should Contain Sub List    ${logsAfter}        ${logsLeader}
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/smoketest/scmha/scm-leader-transfer.robot b/hadoop-ozone/dist/src/main/smoketest/scmha/scm-leader-transfer.robot
new file mode 100644
index 0000000000..80be127ef0
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/scmha/scm-leader-transfer.robot
@@ -0,0 +1,48 @@
+# 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.
+
+*** Settings ***
+Documentation       Smoketest ozone cluster startup
+Library             OperatingSystem
+Library             BuiltIn
+Resource            ../commonlib.robot
+Test Timeout        5 minutes
+
+*** Variables ***
+
+** Keywords ***
+Get SCM Leader Node
+    ${result} =             Execute                 ozone admin scm roles --service-id=scmservice
+                            LOG                     ${result}
+                            Should Contain          ${result}               LEADER              1
+                            Should Contain          ${result}               FOLLOWER            2
+    ${scmLine} =            Get Lines Containing String                     ${result}           LEADER
+    ${splits} =             Split String            ${scmLine}          :
+    ${leaderSCM} =          Strip String            ${splits[4]}
+                            LOG                     Leader SCM: ${leaderSCM}
+    [return]                ${leaderSCM}
+
+*** Test Cases ***
+Transfer Leadership randomly
+    # Find Leader SCM and one Follower SCM
+    ${leaderSCM} =          Get SCM Leader Node
+                            LOG                     Leader SCM: ${leaderSCM}
+    # Transfer leadership to the Follower SCM
+    ${result} =             Execute                 ozone admin scm transfer --service-id=scmservice -r
+                            LOG                     ${result}
+                            Should Contain          ${result}               Transfer leadership successfully
+
+    ${newLeaderSCM} =       Get SCM Leader Node
+                            Should Not be Equal     ${leaderSCM}            ${newLeaderSCM}
\ No newline at end of file
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 228a56ec65..eb6cd7fe03 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
@@ -288,6 +288,7 @@ import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerInterServicePro
 import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneManagerService;
 import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PrepareStatusResponse.PrepareStatus;
 
+import org.apache.ratis.grpc.GrpcTlsConfig;
 import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.protocol.RaftPeer;
@@ -3092,8 +3093,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
       } else {
         targetPeerId = RaftPeerId.valueOf(newLeaderId);
       }
+
+      final GrpcTlsConfig tlsConfig = OzoneManagerRatisUtils.createTlsConfig(
+          secConfig, certClient, true);
+
       RatisHelper.transferRatisLeadership(configuration, division.getGroup(),
-          targetPeerId);
+          targetPeerId, tlsConfig);
     } catch (IOException ex) {
       auditSuccess = false;
       AUDIT.logReadFailure(
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java
index 3b4e899910..0cab292025 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java
@@ -30,7 +30,6 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.nio.charset.StandardCharsets;
-import java.security.cert.X509Certificate;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -49,7 +48,6 @@ import org.apache.hadoop.hdds.ratis.RatisHelper;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
-import org.apache.hadoop.hdds.utils.HAUtils;
 import org.apache.hadoop.ipc.ProtobufRpcEngine.Server;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OzoneManager;
@@ -97,6 +95,7 @@ import org.slf4j.LoggerFactory;
 import static org.apache.hadoop.ipc.RpcConstants.DUMMY_CLIENT_ID;
 import static org.apache.hadoop.ipc.RpcConstants.INVALID_CALL_ID;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HA_PREFIX;
+import static org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils.createTlsConfig;
 
 /**
  * Creates a Ratis server endpoint for OM.
@@ -878,16 +877,7 @@ public final class OzoneManagerRatisServer {
 
   private static Parameters createServerTlsParameters(SecurityConfig conf,
       CertificateClient caClient) throws IOException {
-    if (conf.isSecurityEnabled() && conf.isGrpcTlsEnabled()) {
-      List<X509Certificate> caList = HAUtils.buildCAX509List(caClient,
-          conf.getConfiguration());
-      GrpcTlsConfig config = new GrpcTlsConfig(
-          caClient.getPrivateKey(), caClient.getCertificate(),
-          caList, true);
-      return RatisHelper.setServerTlsConf(config);
-    }
-
-    return null;
+    GrpcTlsConfig config = createTlsConfig(conf, caClient, true);
+    return config == null ? null : RatisHelper.setServerTlsConf(config);
   }
-
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
index 512c8cb92b..eac05157b2 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
@@ -25,6 +25,8 @@ import java.nio.file.Paths;
 
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
 import org.apache.hadoop.hdds.server.ServerUtils;
 import org.apache.hadoop.hdds.utils.HAUtils;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
@@ -89,10 +91,13 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMReque
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneObj.ObjectType;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.grpc.GrpcTlsConfig;
 import org.rocksdb.RocksDBException;
 
 import java.io.IOException;
 import java.nio.file.Path;
+import java.security.cert.X509Certificate;
+import java.util.List;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -457,4 +462,18 @@ public final class OzoneManagerRatisUtils {
       throw new ServiceException(e);
     }
   }
+
+  public static GrpcTlsConfig createTlsConfig(SecurityConfig conf,
+      CertificateClient caClient, boolean mutualTls) throws IOException {
+    if (conf.isSecurityEnabled() && conf.isGrpcTlsEnabled()) {
+      List<X509Certificate> caList = HAUtils.buildCAX509List(caClient,
+          conf.getConfiguration());
+      GrpcTlsConfig config = new GrpcTlsConfig(
+          caClient.getPrivateKey(), caClient.getCertificate(),
+          caList, mutualTls);
+      return config;
+    }
+
+    return null;
+  }
 }


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