You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2020/02/10 17:03:37 UTC

[GitHub] [hadoop-ozone] bharatviswa504 commented on a change in pull request #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.

bharatviswa504 commented on a change in pull request #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.
URL: https://github.com/apache/hadoop-ozone/pull/536#discussion_r377193798
 
 

 ##########
 File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
 ##########
 @@ -218,58 +223,74 @@ static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
   }
 
   /**
-   * Set all the properties matching with regex RATIS_CLIENT_HEADER_REGEX in
+   * Set all the properties matching with regex
+   * {@link RatisHelper#HDDS_DATANODE_RATIS_PREFIX_KEY} in
    * ozone configuration object and configure it to RaftProperties.
    * @param ozoneConf
    * @param raftProperties
    */
   static void createRaftClientProperties(Configuration ozoneConf,
       RaftProperties raftProperties) {
-    Map<String, String> ratisClientConf =
-        ozoneConf.getValByRegex(RATIS_CLIENT_HEADER_REGEX);
-    ratisClientConf.forEach((key, val) -> raftProperties.set(key, val));
-  }
 
-  /**
-   * Set all the properties matching with regex
-   * {@link RatisHelper#RATIS_GRPC_CLIENT_HEADER_REGEX} in ozone
-   * configuration object and configure it to RaftProperties.
-   * @param ozoneConf
-   * @param raftProperties
-   */
-  static void createRaftGrpcProperties(Configuration ozoneConf,
-      RaftProperties raftProperties) {
-    Map<String, String> ratisClientConf =
-        ozoneConf.getValByRegex(RATIS_GRPC_CLIENT_HEADER_REGEX);
-    ratisClientConf.forEach((key, val) -> raftProperties.set(key, val));
-  }
+    // As for client we do not require server and grpc server/tls. exclude them.
+    String grpcPrefix = HDDS_DATANODE_RATIS_GRPC_PREFIX_KEY
+        .substring(HDDS_DATANODE_RATIS_PREFIX_KEY.length());
+    String grpcTlsKey =
+        grpcPrefix + "." +GrpcConfigKeys.TLS.PREFIX;
+    String grpcServerKey = grpcPrefix + "." + GrpcConfigKeys.Server.PREFIX;
 
-  static void createRaftServerGrpcProperties(Configuration ozoneConf,
-      RaftProperties raftProperties) {
     Map<String, String> ratisClientConf =
-        ozoneConf.getValByRegex(RATIS_SERVER_GRPC_HEADER_REGEX);
-    ratisClientConf.forEach((key, val) -> raftProperties.set(
-        removeDatanodePrefix(key), val));
+        ozoneConf.getPropsWithPrefix(HDDS_DATANODE_RATIS_PREFIX_KEY);
+    ratisClientConf.forEach((key, val) -> {
+      if (!(key.startsWith(RATIS_SERVER_PREFIX_KEY) ||
+          key.startsWith(grpcServerKey) || key.startsWith(grpcTlsKey))) {
+        raftProperties.set(removeDatanodePrefix(key), val);
+      }
+    });
   }
 
 
   /**
-   * Set all the properties matching with regex
-   * {@link RatisHelper#RATIS_SERVER_HEADER_REGEX} in ozone configuration
-   * object and configure it to RaftProperties.
+   * Set all the properties matching with prefix
+   * {@link RatisHelper#HDDS_DATANODE_RATIS_PREFIX_KEY} in
+   * ozone configuration object and configure it to RaftProperties.
    * @param ozoneConf
    * @param raftProperties
    */
   static void createRaftServerProperties(Configuration ozoneConf,
        RaftProperties raftProperties) {
+
     Map<String, String> ratisServerConf =
-        ozoneConf.getValByRegex(RATIS_SERVER_HEADER_REGEX);
-    ratisServerConf.forEach((key, val) -> raftProperties.set(
-        removeDatanodePrefix(key), val));
+        getDatanodeRatisPrefixProps(ozoneConf);
+    ratisServerConf.forEach((key, val) -> {
+      // Exclude ratis client configuration.
+      if (!key.startsWith(RATIS_CLIENT_PREFIX_KEY)) {
+        raftProperties.set(removeDatanodePrefix(key), val);
+      }
+    });
   }
 
   static String removeDatanodePrefix(String key) {
-    return key.replaceFirst(DATANODE_RATIS_SERVER_CONFIG_PREFIX, "");
+
+    if (key.startsWith("server.")) {
 
 Review comment:
   Because first, we call getByPrefix is called with "datanode.ratis." and which it returns a map by removing 'datanode.ratis." from the key, so here we will be left with 3 prefixes server/client and grpc (As ratis currently has 3 groups one for Ratis Server, Ratis Client and other for Ratis Grpc). This is the reason for the check and then based on the prefix removing the prefix. Yes, I will use constants here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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