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/08 00:13:42 UTC

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

bharatviswa504 opened a new 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
 
 
   ## What changes were proposed in this pull request?
   
   1. Remove getValByRegex usage and regex usage.
   2. Use getByPrefix.
   3. Use prefix for Ratis Client, GRPC and Serve Conf.
   
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-2988
   
   ## How was this patch tested?
   
   UT's are modified to test the behavior.
   

----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.
URL: https://github.com/apache/hadoop-ozone/pull/536#issuecomment-584742986
 
 
   > > The reason for server, client and grpc prefix is to group the configurations by the group
   > 
   > Thanks the answer. I understand that grouping is useful. I don't understand why do we need double grouping.
   > 
   > > Do we really need the server and client part here? Why don't we use `datanode.ratis.raft.client.*` and `datanode.ratis.raft.server.*` instead of `datanode.ratis.server.raft.server`? Is there any use case where we need to configure the client configs on the server side? (`datanode.ratis.server.raft.client.*` for example?)
   > 
   > If we already have a grouping on the ratis side why do we introduce an other one on the ozone side?
   
   The reasons for a prefix "datanode" is in future if OM also uses this similar approach, we need a way to distinguish which config is for which component. So, that is the reason for prefix. So, created 3 config classes, one for ratis server with datanode.ratis.server, similarly for others. I understand the ratis.server is duplicated again with this approach. If any other thoughts how to handle this?
   
   

----------------------------------------------------------------
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


[GitHub] [hadoop-ozone] elek commented on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.

Posted by GitBox <gi...@apache.org>.
elek commented on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.
URL: https://github.com/apache/hadoop-ozone/pull/536#issuecomment-584662251
 
 
   > The reason for server, client and grpc prefix is to group the configurations by the group
   
   Thanks the answer. I understand that grouping is useful. I don't understand why do we need double grouping. 
   
   > Do we really need the server and client part here? Why don't we use `datanode.ratis.raft.client.*` and `datanode.ratis.raft.server.*` instead of `datanode.ratis.server.raft.server`? Is there any use case where we need to configure the client configs on the server side? (`datanode.ratis.server.raft.client.*` for example?)
   
   If we already have a grouping on the ratis side why do we introduce an other one on the ozone side?

----------------------------------------------------------------
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


[GitHub] [hadoop-ozone] elek closed pull request #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.

Posted by GitBox <gi...@apache.org>.
elek closed 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
 
 
   

----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.
URL: https://github.com/apache/hadoop-ozone/pull/536#issuecomment-584223834
 
 
   > Thanks the patch @bharatviswa504
   > 
   > Overall I like the idea to use string manipulation instead of regexp but it's a little hard to read the structure of the configuration. Maybe because the duplication of the "server/grpc" words in `datanode.ratis.grpc.raft.grpc.flow.control` (As I asked in an other comment why do we need the first grpc?).
   
   The reason for server, client and grpc prefix is to group the configurations by the group. So, that it will be easy to add these configurations in different classes(like have a class for Ratis Server, One for Ratis Client ..) instead of just having all the configuration in a single class.

----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
elek 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_r377105756
 
 

 ##########
 File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
 ##########
 @@ -54,29 +57,34 @@
 import org.apache.ratis.retry.RetryPolicy;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.rpc.SupportedRpcType;
+import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.TimeDuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.ozone.conf.DatanodeRatisServerConfig.DATANODE_RATIS_SERVER_CONFIG_PREFIX;
-
 /**
  * Ratis helper methods.
  */
 public interface RatisHelper {
   Logger LOG = LoggerFactory.getLogger(RatisHelper.class);
 
-  // Ratis Client and Grpc header regex filters.
-  String RATIS_CLIENT_HEADER_REGEX = "raft\\.client\\.([a-z\\.]+)";
-  String RATIS_GRPC_CLIENT_HEADER_REGEX = "raft\\.grpc\\.(?!server|tls)" +
-      "([a-z\\.]+)";
+  // Prefix for Ratis Server GRPC and Ratis client conf.
+  String HDDS_DATANODE_RATIS_PREFIX_KEY = "datanode.ratis.";
+  String HDDS_DATANODE_RATIS_SERVER_PREFIX_KEY = "datanode.ratis.server";
+  String HDDS_DATANODE_RATIS_CLIENT_PREFIX_KEY = "datanode.ratis.client";
+  String HDDS_DATANODE_RATIS_GRPC_PREFIX_KEY = "datanode.ratis.grpc";
+
 
-  // Ratis Server header regex filter.
-  String RATIS_SERVER_HEADER_REGEX = "datanode\\.ratis\\.raft\\.server\\" +
-      ".([a-z\\.]+)";
-  String RATIS_SERVER_GRPC_HEADER_REGEX = "datanode\\.ratis\\.raft\\.grpc\\" +
-      ".([a-z\\.]+)";
+  String RATIS_SERVER_PREFIX_KEY =
 
 Review comment:
   Do we really need the `server` and `client` part here? Why don't we use `datanode.ratis.raft.client.*` and `datanode.ratis.raft.server.*` instead of `datanode.ratis.server.raft.server`? Is there any use case where we need to configure the client configs on the server side? (`datanode.ratis.server.raft.client.*`)
   
   BTW, this is just a matter of taste but I would prefer to avoid substring. This seems to be more readable for me:
   
   ```
     String HDDS_DATANODE_RATIS_SERVER_TAG = "server";
     
   String HDDS_DATANODE_RATIS_PREFIX_KEY = "datanode.ratis.";
   
     String HDDS_DATANODE_RATIS_SERVER_PREFIX_KEY =
         HDDS_DATANODE_RATIS_PREFIX_KEY + "." HDDS_DATANODE_RATIS_SERVER_TAG;
   
   
   String RATIS_SERVER_PREFIX_KEY =  RATIS_SERVER_PREFIX_KEY +"." + RaftServerConfigKeys.PREFIX;
   ```
    

----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
elek 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_r377107994
 
 

 ##########
 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:
   Is there any specific reason to limit this to `server`, `client` and `grpc`? (BTW, can we use constants to here? It's not clear if they are ratis segments or ozone segments...)

----------------------------------------------------------------
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


[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.

Posted by GitBox <gi...@apache.org>.
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


[GitHub] [hadoop-ozone] elek commented on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.

Posted by GitBox <gi...@apache.org>.
elek commented on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.
URL: https://github.com/apache/hadoop-ozone/pull/536#issuecomment-585117102
 
 
   >  we can use "datanode.raft.server" for RatisServer config and for client "datanode.raft.client" and for grpc "datanode.raft.grpc". In this way we can use directly ratis grouping. Is this is what you are suggesting here?
   
   Exactly. Sorry if was not clear. It's a very small suggestion, and not a big deal. 
   
   My impression was that it simplifies the whole code as you don't need to do all the magic with the middle `[server|grpc]` tag in the `datanode.ratis.server.raft.server`. You can use the configuration based on the ratis groups.
   
    

----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.
URL: https://github.com/apache/hadoop-ozone/pull/536#issuecomment-585471740
 
 
   > > we can use "datanode.raft.server" for RatisServer config and for client "datanode.raft.client" and for grpc "datanode.raft.grpc". In this way we can use directly ratis grouping. Is this is what you are suggesting here?
   > 
   > Exactly. Sorry if was not clear. It's a very small suggestion, and not a big deal.
   > 
   > My impression was that it simplifies the whole code as you don't need to do all the magic with the middle `[server|grpc]` tag in the `datanode.ratis.server.raft.server`. You can use the configuration based on the ratis groups.
   
   Thank You @elek for review. Addressed review comments.
   
   (With this approach, not able to use direct Ratis variables. (But I think that is okay)

----------------------------------------------------------------
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


[GitHub] [hadoop-ozone] bharatviswa504 edited a comment on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.

Posted by GitBox <gi...@apache.org>.
bharatviswa504 edited a comment on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.
URL: https://github.com/apache/hadoop-ozone/pull/536#issuecomment-585471740
 
 
   > > we can use "datanode.raft.server" for RatisServer config and for client "datanode.raft.client" and for grpc "datanode.raft.grpc". In this way we can use directly ratis grouping. Is this is what you are suggesting here?
   > 
   > Exactly. Sorry if was not clear. It's a very small suggestion, and not a big deal.
   > 
   > My impression was that it simplifies the whole code as you don't need to do all the magic with the middle `[server|grpc]` tag in the `datanode.ratis.server.raft.server`. You can use the configuration based on the ratis groups.
   
   Thank You @elek for review. Addressed review comments.
   
   (With this approach, not able to use direct Ratis variables. (But I think that is okay, we have lost the advantage of if Ratis changes configs, we need to update ozone code accordingly)

----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
elek 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_r377102805
 
 

 ##########
 File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
 ##########
 @@ -54,29 +57,34 @@
 import org.apache.ratis.retry.RetryPolicy;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.rpc.SupportedRpcType;
+import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.TimeDuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.ozone.conf.DatanodeRatisServerConfig.DATANODE_RATIS_SERVER_CONFIG_PREFIX;
-
 /**
  * Ratis helper methods.
  */
 public interface RatisHelper {
   Logger LOG = LoggerFactory.getLogger(RatisHelper.class);
 
-  // Ratis Client and Grpc header regex filters.
-  String RATIS_CLIENT_HEADER_REGEX = "raft\\.client\\.([a-z\\.]+)";
-  String RATIS_GRPC_CLIENT_HEADER_REGEX = "raft\\.grpc\\.(?!server|tls)" +
-      "([a-z\\.]+)";
+  // Prefix for Ratis Server GRPC and Ratis client conf.
+  String HDDS_DATANODE_RATIS_PREFIX_KEY = "datanode.ratis.";
 
 Review comment:
   Nit: `HDDS_DATANODE_RATIS_PREFIX_KEY` can be reused.
   
   ```
     String HDDS_DATANODE_RATIS_PREFIX_KEY = "datanode.ratis.";
     String HDDS_DATANODE_RATIS_SERVER_PREFIX_KEY =
         HDDS_DATANODE_RATIS_PREFIX_KEY + ".server";
     String HDDS_DATANODE_RATIS_CLIENT_PREFIX_KEY =
         HDDS_DATANODE_RATIS_PREFIX_KEY + ".client";
     String HDDS_DATANODE_RATIS_GRPC_PREFIX_KEY =
         HDDS_DATANODE_RATIS_PREFIX_KEY + ".grpc";
   ```

----------------------------------------------------------------
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


[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.

Posted by GitBox <gi...@apache.org>.
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.
   
   And also as we have grouped these configs to "datanode.ratis.server", "datanode.ratis.client" and "datanode.ratis.grpc", so that is also one of the reasons for this.

----------------------------------------------------------------
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


[GitHub] [hadoop-ozone] bharatviswa504 edited a comment on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.

Posted by GitBox <gi...@apache.org>.
bharatviswa504 edited a comment on issue #536: HDDS-2988. Use getPropertiesByPrefix instead of regex in matching ratis client and server properties.
URL: https://github.com/apache/hadoop-ozone/pull/536#issuecomment-584742986
 
 
   > > The reason for server, client and grpc prefix is to group the configurations by the group
   > 
   > Thanks the answer. I understand that grouping is useful. I don't understand why do we need double grouping.
   > 
   > > Do we really need the server and client part here? Why don't we use `datanode.ratis.raft.client.*` and `datanode.ratis.raft.server.*` instead of `datanode.ratis.server.raft.server`? Is there any use case where we need to configure the client configs on the server side? (`datanode.ratis.server.raft.client.*` for example?)
   > 
   > If we already have a grouping on the ratis side why do we introduce an other one on the ozone side?
   
   The reasons for a prefix "datanode" is in future if OM also uses this similar approach, we need a way to distinguish which config is for which component. So, that is the reason for prefix. So, created 3 config classes, one for ratis server with datanode.ratis.server, similarly for others. I understand the ratis.server is duplicated again with this approach. If any other thoughts how to handle this?
   
    Main reasons for doing this way.
   1. Distinguish these properties for each component. (As OM also uses OM ratis Server, sooner or later, we might do similar work for OM also)
   
   One way I think we can remove this is.
   
    So, thinking more again, we can use  "datanode.raft.server" for RatisServer config and for client "datanode.raft.client" and for grpc "datanode.raft.grpc". In this way we can use directly ratis grouping. Is this is what you are suggesting 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