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 2022/10/14 11:05:58 UTC

[GitHub] [ozone] xBis7 opened a new pull request, #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

xBis7 opened a new pull request, #3835:
URL: https://github.com/apache/ozone/pull/3835

   ## What changes were proposed in this pull request?
   
   In order to make the CLI and `ozone admin datanode` subcommands, more consistent, added the hostname as a parameter to the commands that didn't accept it. These commands were `usageinfo` and `list info`.
   
   There is an hdfs flag `dfs.datanode.use.datanode.hostname` checked by the SCMNodeManager during start up and if it's true then we can get the datanode info by providing the hostname with the --ip pamameter, while ip will not be available anymore. The default case is keeping the flag's value to false where the ip is available but not the hostname.
   
   If we add the --hostname parameter and keep the flag, then it's confusing for the user. 
   In such a case we could get the usage info by specifying the hostname in both cases:
   
   ```
   $ ozone admin datanode usageinfo --ip=<hostname>
   $ ozone admin datanode usageinfo --hostname=<hostname>
   ```
   
   We deprecated the flag and in every case that it is checked, kept the default usage, which is having value false. This way, --ip will always require the ip and if the user needs to get the info with the hostname, he can use the new --hostname parameter.
   
   Currrently, `dfs.datanode.use.datanode.hostname` is only checked in `RatisHelper.java`. Should we remove it from there as well? It is left there, because it is accessed from the external jar and not `DFSConfigKeysLegacy`.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-7329
   
   ## How was this patch tested?
   
   Modified the existing tests and they are all passing. Tested manually in a docker cluster.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] sokui commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
sokui commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1003967992


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -373,6 +375,8 @@ public RegisteredCommand register(
 
     String dnsName;
     String networkLocation;
+    String ipAddress = datanodeDetails.getIpAddress();
+    String hostName = datanodeDetails.getHostName();

Review Comment:
   Please also be very careful. In my impression, in some of my test, datanodeDetails.getHostName() might give you IP instead of host name. Please test if your approach does not have bugs for this case as well.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1292028433

   @Xushaohong  
   
   > Kill/ Restart one DN pod could change the IP.
   
   This works fine with the current approach. Any particular scenarios or test cases? Commands to execute? I'm referring to anything more complex than getting the datanode info.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] sokui commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
sokui commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1067839498


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -373,6 +375,8 @@ public RegisteredCommand register(
 
     String dnsName;
     String networkLocation;
+    String ipAddress = datanodeDetails.getIpAddress();
+    String hostName = datanodeDetails.getHostName();

Review Comment:
   @kerneltime it has been so long time, I could not remember. But my testing environment is Kubernetes. I think when we deployed ozone, I see this happens a couple of times. But not sure what is the exact condition.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] adoroszlai commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1279817380

   @sokui @Xushaohong please review


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] neils-dev commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
neils-dev commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1290654133

   Thanks @sokui for your comments.  On the errors you mention that may happen when using the map for both ip and hostnames mapping to UUIDs of DNs,
   
   > If the map holds both IPs and Hostnames at all times, as it maps hostname / IP to the DNs, there will be the same UUID occurs in different keys' value sets causing a sequence of unknown errors.
   
   Can you give some detail on this?  Where this can cause problems?


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] Xushaohong commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1292875686

   > > Kill/ Restart one DN pod could change the IP.
   > 
   > This works fine with the current approach. Any particular scenarios or test cases? Commands to execute? I'm referring to anything more complex than getting the datanode info.
   
   Maybe you could try on the metal environment and adjust the hostname manually which is the case for hostname change.
   If the patch is able to handle the IP-changing and hostname-changing cases, that would be great~


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] sokui commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
sokui commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1003964853


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -124,6 +124,8 @@ public class SCMNodeManager implements NodeManager {
   private final boolean useHostname;
   private final ConcurrentHashMap<String, Set<String>> dnsToUuidMap =
       new ConcurrentHashMap<>();
+  private final Map<String, Set<String>> addressToUuidMap =

Review Comment:
   If this map is used to keep both ip and host names, probably we can deprecate the old dnsToUuidMap. It would be nice if you can clean up the code.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r998077754


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java:
##########
@@ -342,6 +342,24 @@ int getNodeQueuedCommandCount(DatanodeDetails datanodeDetails,
    */
   List<DatanodeDetails> getNodesByAddress(String address);

Review Comment:
   This method uses a map that either has the `ip` addresses or the `hostnames`. We would like to have both of them available in the CLI at all times.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] adoroszlai commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r996334888


##########
hadoop-hdds/interface-admin/src/main/resources/proto.lock:
##########
@@ -888,6 +888,11 @@
                 "id": 4,
                 "name": "count",
                 "type": "uint32"
+              },
+              {
+                "id": 5,
+                "name": "hostname",
+                "type": "string"

Review Comment:
   Please don't change `proto.lock`.  This file is needed to allow checking backwards compatibility.  If we change both proto definitions and the lock file in tandem, the check cannot flag incompatible changes.



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java:
##########
@@ -1704,7 +1703,7 @@ public void testScmRegisterNodeWithIpAddress()
   @Test
   public void testScmRegisterNodeWithHostname()
       throws IOException, InterruptedException, AuthenticationException {
-    testScmRegisterNodeWithNetworkTopology(true);
+    testScmRegisterNodeWithNetworkTopology();
   }

Review Comment:
   These tests used to exercise the same use case with different config setting (use hostname true/false).  Since the config is being removed, they are now duplicates.  Calling the same underlying method from two different test methods without any difference is unnecessary.  Please remove the methods `...WithIpAddress` and `...WithHostname`, move the `@Test` annotation to the delegate methods, and remove the `private` modifier from those methods.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -365,21 +363,14 @@ public RegisteredCommand register(
     InetAddress dnAddress = Server.getRemoteIp();
     if (dnAddress != null) {
       // Mostly called inside an RPC, update ip
-      if (!useHostname) {
-        datanodeDetails.setHostName(dnAddress.getHostName());
-      }
+      datanodeDetails.setHostName(dnAddress.getHostName());
       datanodeDetails.setIpAddress(dnAddress.getHostAddress());
     }
 
-    String dnsName;
-    String networkLocation;
+    String dnsName = datanodeDetails.getIpAddress();
+    String hostName = datanodeDetails.getHostName();

Review Comment:
   `dnsName` and `hostName` roughly mean the same thing.  Please use `ipAddress` for IP address.



##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java:
##########
@@ -386,18 +386,20 @@ StartContainerBalancerResponseProto startContainerBalancer(
   boolean getContainerBalancerStatus() throws IOException;
 
   /**
-   * Get Datanode usage information by ip or uuid.
+   * Get Datanode usage information by ip or uuid or hostname.
    *
    * @param ipaddress datanode IP address String
    * @param uuid datanode UUID String
+   * @param hostname datanode hostname address String
    * @param clientVersion Client's version number
    * @return List of DatanodeUsageInfoProto. Each element contains info such as
    * capacity, SCMused, and remaining space.
    * @throws IOException
    * @see org.apache.hadoop.ozone.ClientVersion
    */
   List<HddsProtos.DatanodeUsageInfoProto> getDatanodeUsageInfo(
-      String ipaddress, String uuid, int clientVersion) throws IOException;
+      String ipaddress, String uuid,
+      String hostname, int clientVersion) throws IOException;

Review Comment:
   Please also keep the method with existing signature for backward compatibility.  I think it can be a `default` method calling the new one, passing `null` for `hostname`.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -121,9 +121,10 @@ public class SCMNodeManager implements NodeManager {
   private final SCMStorageConfig scmStorageConfig;
   private final NetworkTopology clusterMap;
   private final DNSToSwitchMapping dnsToSwitchMapping;
-  private final boolean useHostname;
   private final ConcurrentHashMap<String, Set<String>> dnsToUuidMap =
       new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String, Set<String>> hostNmToUuidMap =

Review Comment:
   Prefer interface for variable type and please avoid obscure abbreviation that only saves 2 characters:
   
   ```suggestion
     private final Map<String, Set<String>> hostNameToUuidMap =
   ```



##########
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java:
##########
@@ -46,11 +46,16 @@ public class ListInfoSubcommand extends ScmSubcommand {
       defaultValue = "")
   private String ipaddress;
 
-  @CommandLine.Option(names = {"--id"},
+  @CommandLine.Option(names = {"--uuid"},

Review Comment:
   Please keep the old name as an alias for backwards compatibility.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -468,6 +458,17 @@ private synchronized void addEntryToDnsToUuidMap(
     dnList.add(uuid);
   }
 
+  @SuppressFBWarnings(value = "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION")
+  private synchronized void addEntryToHostNmToUuidMap(
+      String hostName, String uuid) {
+    Set<String> dnList = hostNmToUuidMap.get(hostName);
+    if (dnList == null) {
+      dnList = ConcurrentHashMap.newKeySet();
+      hostNmToUuidMap.put(hostName, dnList);
+    }
+    dnList.add(uuid);

Review Comment:
   We can simplify the code and avoid having to add a suppression:
   
   ```suggestion
     private void addEntryToHostNameToUuidMap(
         String hostName, String uuid) {
       hostNmToUuidMap
           .computeIfAbsent(hostName, any -> ConcurrentHashMap.newKeySet())
           .add(uuid);
   ```



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] Xushaohong commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1280788993

   > The goal here is to have a consistent CLI, so we keep the code as it is but we won't expose that behavior to the user. We need the --ip to accept only an IP address and add the --hostname parameter for accepting a hostname. We could create two new maps, one for mapping IP to UUID and one for mapping hostname to UUID and we could use those maps for accessing datanode info from the CLI. @adoroszlai @sokui What do you think about this approach?
   
   This sounds feasible, but some worry occurs that how to make sure the IP - UUID - hostname mapping is always consistent. If the hostname changes at some DN, we may need the retry logic with the IP-UUID mapping and need to update the corresponding hostname.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1280860946

   > If the hostname changes at some DN, we may need the retry logic with the IP-UUID mapping and need to update the corresponding hostname.
   
   @Xushaohong Can you provide an example in the code of how we are currently handling such a case?


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1289190907

   @sokui @Xushaohong If this [Map](https://github.com/apache/ozone/blob/4c65a07b124b21aa85aa3711035fd8616e13db38/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java#L125) holds both IPs and Hostnames at all times, will this be an issue? In case we are in an ip changing environment and the ip doesn't work, hostname will still be available for the user. What do you think?


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] kerneltime commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
kerneltime commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1013714193


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java:
##########
@@ -101,18 +98,16 @@ public static UUID toDatanodeId(RaftProtos.RaftPeerProto peerId) {
   }
 
   private static String toRaftPeerAddress(DatanodeDetails id, Port.Name port) {
-    if (datanodeUseHostName()) {
-      final String address =
-              id.getHostName() + ":" + id.getPort(port).getValue();
+    final String address;
+    if (Strings.isNullOrEmpty(id.getIpAddress())) {

Review Comment:
   We are changing the behavior of this call. Using hostnames if provided should be the preferred way to evaluate the address, not sure why IP address is preferred here (if both are available)



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] kerneltime commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
kerneltime commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1282936477

   Let's discuss this in the next community sync @neils-dev 


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter [ozone]

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai merged PR #3835:
URL: https://github.com/apache/ozone/pull/3835


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter [ozone]

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1820669027

   @xBis7 I plan to finish this, are you OK with that?


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter [ozone]

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1822179170

   > As far as I can recall, these were the unresolved issues that blocked this ticket
   
   Thanks @xBis7, that's very useful for the future.  However, here I don't intend to address all hostname-related problems, just trying to wrap it up considering your latest comment:
   
   > Went back to using the flag for hostnames and now everything works and we still have the desired outcome, that the ip and hostname are both available at the client.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1013762964


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java:
##########
@@ -101,18 +98,16 @@ public static UUID toDatanodeId(RaftProtos.RaftPeerProto peerId) {
   }
 
   private static String toRaftPeerAddress(DatanodeDetails id, Port.Name port) {
-    if (datanodeUseHostName()) {
-      final String address =
-              id.getHostName() + ":" + id.getPort(port).getValue();
+    final String address;
+    if (Strings.isNullOrEmpty(id.getIpAddress())) {

Review Comment:
   IP address was preferred here because that's the default right now, in master. I will change it to hostname.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] kerneltime commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
kerneltime commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1065034099


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -373,6 +375,8 @@ public RegisteredCommand register(
 
     String dnsName;
     String networkLocation;
+    String ipAddress = datanodeDetails.getIpAddress();
+    String hostName = datanodeDetails.getHostName();

Review Comment:
   @sokui can you elaborate when you see an IP address being returned for hostname?



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1280060368

   Hi @sokui, I wasn't aware of this. I was told that this flag is not used by the community and that we can deprecate it. We must keep the existing code as it is. 
   
   The goal here is to have a consistent CLI, so we keep the code as it is but we won't expose that behavior to the user. We need the --ip to accept only an IP address and add the --hostname parameter for accepting a hostname. We could create two new maps, one for mapping IP to UUID and one for mapping hostname to UUID and we could use those maps for accessing datanode info from the CLI. @adoroszlai @sokui What do you think about this approach?  


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter [ozone]

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1406370234


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -379,34 +380,31 @@ public RegisteredCommand register(
       datanodeDetails.setIpAddress(dnAddress.getHostAddress());
     }
 
-    String dnsName;
-    String networkLocation;
+    final String ipAddress = datanodeDetails.getIpAddress();
+    final String hostName = datanodeDetails.getHostName();
     datanodeDetails.setNetworkName(datanodeDetails.getUuidString());
-    if (useHostname) {
-      dnsName = datanodeDetails.getHostName();
-    } else {
-      dnsName = datanodeDetails.getIpAddress();
-    }
-    networkLocation = nodeResolve(dnsName);
+    String networkLocation = nodeResolve(useHostname ? hostName : ipAddress);
     if (networkLocation != null) {
       datanodeDetails.setNetworkLocation(networkLocation);
     }
 
+    final UUID uuid = datanodeDetails.getUuid();
     if (!isNodeRegistered(datanodeDetails)) {
       try {
         clusterMap.add(datanodeDetails);
         nodeStateManager.addNode(datanodeDetails, layoutInfo);
         // Check that datanode in nodeStateManager has topology parent set
         DatanodeDetails dn = nodeStateManager.getNode(datanodeDetails);
         Preconditions.checkState(dn.getParent() != null);
-        addToDnsToUuidMap(dnsName, datanodeDetails.getUuid());
+        addToDnsToUuidMap(ipAddress, uuid);

Review Comment:
   Thanks @sadanand48, updated.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1013762964


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java:
##########
@@ -101,18 +98,16 @@ public static UUID toDatanodeId(RaftProtos.RaftPeerProto peerId) {
   }
 
   private static String toRaftPeerAddress(DatanodeDetails id, Port.Name port) {
-    if (datanodeUseHostName()) {
-      final String address =
-              id.getHostName() + ":" + id.getPort(port).getValue();
+    final String address;
+    if (Strings.isNullOrEmpty(id.getIpAddress())) {

Review Comment:
   IP address is preferred here because that's the default right now, in master. I will change it to hostname.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1304467588

   @kerneltime After some testing, it seems that making the use of hostname the default option breaks the system except for a kubernetes environment due to pipeline allocation to datanodes. To make this work we would have to make a lot of changes and this PR would grow out of hand unnecessarily. Went back to using the flag for hostnames and now everything works and we still have the desired outcome, that the ip and hostname are both available at the client. If you could take another look and let me know what you think.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1298533055

   > Please add a end to end robot test which should help close the loop for the feature.
   
   @kerneltime I added the robot test you requested.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1281195827

   Moved everything back to how it was and added these [two methods](https://github.com/xBis7/ozone/blob/HDDS-7329/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java#L1280-L1302) used only by the CLI. This achieves the wanted behavior and I think it doesn't affect the system in the cases mentioned above. Let me know what you all think. 


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter [ozone]

Posted by "sadanand48 (via GitHub)" <gi...@apache.org>.
sadanand48 commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1405881819


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -379,34 +380,31 @@ public RegisteredCommand register(
       datanodeDetails.setIpAddress(dnAddress.getHostAddress());
     }
 
-    String dnsName;
-    String networkLocation;
+    final String ipAddress = datanodeDetails.getIpAddress();
+    final String hostName = datanodeDetails.getHostName();
     datanodeDetails.setNetworkName(datanodeDetails.getUuidString());
-    if (useHostname) {
-      dnsName = datanodeDetails.getHostName();
-    } else {
-      dnsName = datanodeDetails.getIpAddress();
-    }
-    networkLocation = nodeResolve(dnsName);
+    String networkLocation = nodeResolve(useHostname ? hostName : ipAddress);
     if (networkLocation != null) {
       datanodeDetails.setNetworkLocation(networkLocation);
     }
 
+    final UUID uuid = datanodeDetails.getUuid();
     if (!isNodeRegistered(datanodeDetails)) {
       try {
         clusterMap.add(datanodeDetails);
         nodeStateManager.addNode(datanodeDetails, layoutInfo);
         // Check that datanode in nodeStateManager has topology parent set
         DatanodeDetails dn = nodeStateManager.getNode(datanodeDetails);
         Preconditions.checkState(dn.getParent() != null);
-        addToDnsToUuidMap(dnsName, datanodeDetails.getUuid());
+        addToDnsToUuidMap(ipAddress, uuid);

Review Comment:
   Since these are synchronized methods, I think we should do both updates (ip & hostname) in a single method call.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -418,28 +416,24 @@ public RegisteredCommand register(
       try {
         final DatanodeInfo datanodeInfo =
                 nodeStateManager.getNode(datanodeDetails);
-        if (!datanodeInfo.getIpAddress().equals(datanodeDetails.getIpAddress())
-                || !datanodeInfo.getHostName()
-                .equals(datanodeDetails.getHostName())) {
-          LOG.info("Updating data node {} from {} to {}",
+        final String oldIpAddress = datanodeInfo.getIpAddress();
+        final String oldHostName = datanodeInfo.getHostName();
+        if (!Objects.equals(oldIpAddress, ipAddress)
+            || !Objects.equals(oldHostName, hostName)) {
+          LOG.info("Updating datanode {} from {} to {}",
                   datanodeDetails.getUuidString(),
                   datanodeInfo,
                   datanodeDetails);
           clusterMap.update(datanodeInfo, datanodeDetails);
 
-          String oldDnsName;
-          if (useHostname) {
-            oldDnsName = datanodeInfo.getHostName();
-          } else {
-            oldDnsName = datanodeInfo.getIpAddress();
-          }
-          updateDnsToUuidMap(oldDnsName, dnsName, datanodeDetails.getUuid());
+          updateDnsToUuidMap(oldIpAddress, ipAddress, uuid);
+          updateDnsToUuidMap(oldHostName, hostName, uuid);

Review Comment:
   Same comment as above



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] sokui commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
sokui commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r1067839498


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -373,6 +375,8 @@ public RegisteredCommand register(
 
     String dnsName;
     String networkLocation;
+    String ipAddress = datanodeDetails.getIpAddress();
+    String hostName = datanodeDetails.getHostName();

Review Comment:
   @kerneltime it has been so long time, I could not remember. But my testing environment was Kubernetes. I think when we deployed ozone, I saw this happens a couple of times. But not sure what was the exact condition.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r998075539


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -124,6 +124,10 @@ public class SCMNodeManager implements NodeManager {
   private final boolean useHostname;
   private final ConcurrentHashMap<String, Set<String>> dnsToUuidMap =
       new ConcurrentHashMap<>();
+  private final Map<String, Set<String>> ipAddressToUuidMap =

Review Comment:
   We could have a map that holds both IP and hostnames.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] kerneltime commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
kerneltime commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1297354935

   Please add a end to end robot test which should help close the loop for the feature.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter [ozone]

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1820741758

   @adoroszlai Please, feel free to take this over. 
   
   As far as I can recall, these were the unresolved issues that blocked this ticket
   
   - We need to use hostnames with Kubernetes because it keeps changing Node IPs dynamically
   - It would be nice to migrate Ozone entirely to using hostnames
   - Ozone is too dependent on IPs for topology and therefore moving entirely to hostnames is impossible for now
   - Based on this flag `dfs.datanode.use.datanode.hostname` we can either have IPs available or hostnames. `SCMNodeManager` needs refactoring to have both of them available


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter [ozone]

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1828323059

   @adoroszlai Thanks for finishing this. I've gone through the changes. LGTM!


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1282286052

   @kerneltime Let me explain in more details the purpose of this PR. The issue is that you can either have `ip` or `hostname` and use only one of them to get the datanode info.
   
   from master the default behavior
   
   ```
   bash-4.2$ ozone admin datanode usageinfo --ip=172.21.0.5
   Usage Information (1 Datanodes)
   
   UUID         : 08c48611-7121-46a5-a047-129f573c78a6 
   IP Address   : 172.21.0.5 (ozone_datanode_1.ozone_default) 
   Capacity     : 182775984128 B (170.22 GB) 
   Total Used   : 168680075264 B (157.10 GB) 
   Total Used % : 92.29% 
   Ozone Used   : 4096 B (4 KB) 
   Ozone Used % : 0.00% 
   Remaining    : 14095908864 B (13.13 GB) 
   Remaining %  : 7.71% 
   
   bash-4.2$ ozone admin datanode usageinfo --ip=ozone_datanode_1.ozone_default
   Usage Information (0 Datanodes)
   ```
   
   from master with `dfs.datanode.use.datanode.hostname=true`
   ```
   bash-4.2$ ozone admin datanode usageinfo --ip=172.19.0.5
   Usage Information (0 Datanodes)
   
   bash-4.2$ ozone admin datanode usageinfo --ip=d416d3fec041
   Usage Information (1 Datanodes)
   
   UUID         : 809c1065-2594-4208-bc57-4442b71022e2 
   IP Address   : 172.19.0.5 (d416d3fec041) 
   Capacity     : 182775984128 B (170.22 GB) 
   Total Used   : 168679792640 B (157.10 GB) 
   Total Used % : 92.29% 
   Ozone Used   : 4096 B (4 KB) 
   Ozone Used % : 0.00% 
   Remaining    : 14096191488 B (13.13 GB) 
   Remaining %  : 7.71% 
   ```
   
   We want to have a `--hostname` parameter and also we want the CLI to be more consistent, meaning that the `--ip` accepts only ip and the new `--hostname` accepts only hostname. If we just add the `--hostname` parameter then in the above second case we will have `--hostname` that accepts hostname and `--ip` that also accepts only hostname. 
   
   As pointed out to me, we can't deprecate the flag or change the current logic in `SCMNodeManager` because it's necessary for environments like Kubernetes where the datanode can restart and end up with a new Ip address. That's why I creating specific APIs for the CLI.
   
   The current code achieves the desired behavior.
   
   from this branch with `dfs.datanode.use.datanode.hostname=true`
   ```
   bash-4.2$ ozone admin datanode usageinfo --ip=172.22.0.5
   Usage Information (1 Datanodes)
   
   UUID         : 961511cf-38ec-412c-b805-381549ce03cf 
   IP Address   : 172.22.0.5 
   Hostname     : 16cbb539ec92 
   Capacity     : 182775984128 B (170.22 GB) 
   Total Used   : 168680542208 B (157.10 GB) 
   Total Used % : 92.29% 
   Ozone Used   : 4096 B (4 KB) 
   Ozone Used % : 0.00% 
   Remaining    : 14095441920 B (13.13 GB) 
   Remaining %  : 7.71% 
   
   bash-4.2$ ozone admin datanode usageinfo --ip=16cbb539ec92
   Usage Information (0 Datanodes)
   
   bash-4.2$ ozone admin datanode usageinfo --hostname=16cbb539ec92
   Usage Information (1 Datanodes)
   
   UUID         : 961511cf-38ec-412c-b805-381549ce03cf 
   IP Address   : 172.22.0.5 
   Hostname     : 16cbb539ec92 
   Capacity     : 182775984128 B (170.22 GB) 
   Total Used   : 169011408896 B (157.40 GB) 
   Total Used % : 92.47% 
   Ozone Used   : 4096 B (4 KB) 
   Ozone Used % : 0.00% 
   Remaining    : 13764575232 B (12.82 GB) 
   Remaining %  : 7.53% 
   ```
   


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] kerneltime commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
kerneltime commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r997596076


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -124,6 +124,10 @@ public class SCMNodeManager implements NodeManager {
   private final boolean useHostname;
   private final ConcurrentHashMap<String, Set<String>> dnsToUuidMap =
       new ConcurrentHashMap<>();
+  private final Map<String, Set<String>> ipAddressToUuidMap =

Review Comment:
   I think we should only keep a single map and iterate and filter vs maintaining each way to list datanodes in memory all the time. It seems wasteful and prone to bugs on updates. 



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] kerneltime commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
kerneltime commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r997596555


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java:
##########
@@ -342,6 +342,24 @@ int getNodeQueuedCommandCount(DatanodeDetails datanodeDetails,
    */
   List<DatanodeDetails> getNodesByAddress(String address);

Review Comment:
   Why is address not sufficient to cover both `hostname` as well as `ip`



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] kerneltime commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
kerneltime commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1281638345

   I think we need to revisit this PR. Ideally, the listing of nodes should work the same for IP and hostname and should not require opening up specific APIs in SCM protocol. 


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] Xushaohong commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
Xushaohong commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1289902890

   > If this [Map](https://github.com/apache/ozone/blob/4c65a07b124b21aa85aa3711035fd8616e13db38/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java#L125) holds both IPs and Hostnames at all times, will this be an issue? In case we are in an ip changing environment and the ip doesn't work, hostname will still be available for the user. What do you think? Also, if you could provide a way for me to test it.
   
   This map is fine with  IP changing environment since **we either use IP or hostname, not both of them simultaneously.** The prerequisite is the config ```dfs.datanode.use.datanode.hostname```. The maintainer should set the right config according to the environment. 
   
   The simple test environment could be found in the ```kubernetes``` directory. 
   E.g ozone/hadoop-ozone/dist/target/ozone-1.3.0-SNAPSHOT/kubernetes
   There is a README.md to teach how to set up an ozone k8s cluster.
   Kill/ Restart one DN pod could change the IP.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] sokui commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
sokui commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1291541939

   > Thanks @sokui for your comments. On the errors you mention that may happen when using the map for both ip and hostnames mapping to UUIDs of DNs,
   > 
   > > If the map holds both IPs and Hostnames at all times, as it maps hostname / IP to the DNs, there will be the same UUID occurs in different keys' value sets causing a sequence of unknown errors.
   > 
   > Can you give some detail on this? Where this can cause problems?
   
   @neils-dev , I did not mean your solution will have problems in this case. I just try to reminder you to be careful of this case. When the variable/method named as hostname, people may assume it is just the hostname not IP, but in some case it is not true. So far in this PR, I do not think this is an issue.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3835:
URL: https://github.com/apache/ozone/pull/3835#discussion_r998075539


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -124,6 +124,10 @@ public class SCMNodeManager implements NodeManager {
   private final boolean useHostname;
   private final ConcurrentHashMap<String, Set<String>> dnsToUuidMap =
       new ConcurrentHashMap<>();
+  private final Map<String, Set<String>> ipAddressToUuidMap =

Review Comment:
   We could have a map that holds both IP and hostnames. But if we use that map with the CLI, then the user will be able to get the datanode info even by providing a hostname in `--ip` parameter. That's the current issue we want to solve.



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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] sokui commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
sokui commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1279823072

   Hi @xBis7 ,  we should not deprecate dfs.datanode.use.datanode.hostname and always make it as false. This flag is also used by the users who deploy ozone to ip changing environment such as k8s. In those environments, the ip of the ozone components can be changed, and thus we should use the hostname for communication. 
   
   This PR is to make the datanodes to be adaptive to k8s environments by letting it use hostname. Please check it out. Thanks. https://github.com/apache/ozone/pull/3186


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] kerneltime commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
kerneltime commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1292392080

   This looks much better, will complete my review this week. 


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] sokui commented on pull request #3835: HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter

Posted by GitBox <gi...@apache.org>.
sokui commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1292928259

   > > > Kill/ Restart one DN pod could change the IP.
   > > 
   > > 
   > > This works fine with the current approach. Any particular scenarios or test cases? Commands to execute? I'm referring to anything more complex than getting the datanode info.
   > 
   > Maybe you could try on the metal environment and adjust the hostname manually which is the case for hostname change. If the patch is able to handle the IP-changing and hostname-changing cases, that would be great~
   
   For k8s, if you deploy datanodes as deployment instead of statefulset, when you delete a node, it will restart with a different hostname, I think.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter [ozone]

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1828243803

   @xBis7 please let me know if you are OK with the latest update to be merged


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] HDDS-7329. Extend ozone admin datanode usageinfo and list info to accept hostname parameter [ozone]

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #3835:
URL: https://github.com/apache/ozone/pull/3835#issuecomment-1828327056

   Thanks @xBis7 for the patch, @kerneltime, @sadanand48, @sokui, @Xushaohong for the review.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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