You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kvrocks.apache.org by "zevin02 (via GitHub)" <gi...@apache.org> on 2023/04/17 15:01:22 UTC

[GitHub] [incubator-kvrocks] zevin02 opened a new pull request, #1392: feat: implement local IP address retrieval function

zevin02 opened a new pull request, #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392

   fixes: #1381
   
   - Add a function to get all local IP addresses


-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1171397705


##########
src/cluster/cluster.cc:
##########
@@ -20,9 +20,11 @@
 
 #include "cluster.h"
 
+#include <arpa/inet.h>
 #include <config/config_util.h>
+#include <ifaddrs.h>
+#include <sys/types.h>

Review Comment:
   Seems these newly added headers are no longer needed in this file.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] git-hulk commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169454116


##########
src/cluster/cluster.cc:
##########
@@ -157,7 +159,34 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> getLocalIPAddresses() {
+  std::vector<std::string> ipAddresses;
+
+  struct ifaddrs *ifAddrStruct = NULL;
+  struct ifaddrs *ifa = NULL;
+  void *tmpAddrPtr = NULL;
+
+  getifaddrs(&ifAddrStruct);
 
+  for (ifa = ifAddrStruct; ifa != NULL; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmpAddrPtr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char addressBuffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmpAddrPtr, addressBuffer, INET_ADDRSTRLEN);
+      std::string ipAddress(addressBuffer);
+      ipAddresses.push_back(move(ipAddress));

Review Comment:
   The implementation of `getLocalIPAddresses` looks good. My point is that we should use all IP addresses to match only if the listening address is `0.0.0.0`.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] enjoy-binbin commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "enjoy-binbin (via GitHub)" <gi...@apache.org>.
enjoy-binbin commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169857603


##########
src/cluster/cluster.cc:
##########
@@ -201,14 +230,20 @@ Status Cluster::SetClusterNodes(const std::string &nodes_str, int64_t version, b
   }
 
   // Find myself
+  // Get all local IP addresses.
+  std::vector<std::string> localhost = getLocalIPAddresses();
   if (myid_.empty() || force) {
     for (auto &n : nodes_) {
-      if (n.second->port == port_ && std::find(binds_.begin(), binds_.end(), n.second->host) != binds_.end()) {
-        myid_ = n.first;
+      //If the IP being listened on is 0.0.0.0, we need to compare the node's IP address with all local IP addresses.

Review Comment:
   ```suggestion
         // If the IP being listened on is 0.0.0.0, we need to compare the node's IP address with all local IP addresses.
   ```



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169944683


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {

Review Comment:
   I think it would be better to put this function in `common/io_util.h/cc`.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1171397705


##########
src/cluster/cluster.cc:
##########
@@ -20,9 +20,11 @@
 
 #include "cluster.h"
 
+#include <arpa/inet.h>
 #include <config/config_util.h>
+#include <ifaddrs.h>
+#include <sys/types.h>

Review Comment:
   Seems these newly added headers are no logger needed.



##########
src/cluster/cluster.cc:
##########
@@ -20,9 +20,11 @@
 
 #include "cluster.h"
 
+#include <arpa/inet.h>
 #include <config/config_util.h>
+#include <ifaddrs.h>
+#include <sys/types.h>

Review Comment:
   Seems these newly added headers are no longer needed.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on pull request #1392: Convert 0.0.0.0 to local ip addresses when matching the cluster node

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#issuecomment-1516002962

   Thanks for your contribution! Merging...


-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1168897278


##########
src/cluster/cluster.cc:
##########
@@ -157,7 +159,34 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> getLocalIPAddresses() {
+  std::vector<std::string> ipAddresses;
+
+  struct ifaddrs *ifAddrStruct = NULL;
+  struct ifaddrs *ifa = NULL;
+  void *tmpAddrPtr = NULL;
+
+  getifaddrs(&ifAddrStruct);
 
+  for (ifa = ifAddrStruct; ifa != NULL; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmpAddrPtr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char addressBuffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmpAddrPtr, addressBuffer, INET_ADDRSTRLEN);
+      std::string ipAddress(addressBuffer);
+      ipAddresses.push_back(move(ipAddress));

Review Comment:
   I would add `std::` prefix to the `move` function.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] zevin02 commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "zevin02 (via GitHub)" <gi...@apache.org>.
zevin02 commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169454807


##########
src/cluster/cluster.cc:
##########
@@ -157,7 +159,34 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> getLocalIPAddresses() {
+  std::vector<std::string> ipAddresses;
+
+  struct ifaddrs *ifAddrStruct = NULL;
+  struct ifaddrs *ifa = NULL;
+  void *tmpAddrPtr = NULL;
+
+  getifaddrs(&ifAddrStruct);
 
+  for (ifa = ifAddrStruct; ifa != NULL; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmpAddrPtr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char addressBuffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmpAddrPtr, addressBuffer, INET_ADDRSTRLEN);
+      std::string ipAddress(addressBuffer);
+      ipAddresses.push_back(move(ipAddress));

Review Comment:
   Got it, thanks



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: Convert 0.0.0.0 to local ip addresses when matching the cluster node

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1172022259


##########
src/common/io_util.cc:
##########
@@ -312,6 +314,53 @@ int AeWait(int fd, int mask, int timeout) {
   }
 }
 
+bool MatchListeningIP(std::vector<std::string> &binds, const std::string &ip) {
+  if (std::find(binds.begin(), binds.end(), ip) != binds.end()) {
+    return true;
+  }
+
+  // If binds contains 0.0.0.0, we should resolve ip addresses and check it
+  if (std::find(binds.begin(), binds.end(), "0.0.0.0") != binds.end()) {
+    auto local_ip_addresses = GetLocalIPAddresses();
+    return std::find(local_ip_addresses.begin(), local_ip_addresses.end(), ip) != local_ip_addresses.end();
+  }
+  return false;
+}
+
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIPAddresses() {
+  std::vector<std::string> ip_addresses;
+  ifaddrs *if_addr_struct = nullptr;
+  // Use unique_ptr for if_addr_struct to avoid manually free
+  std::unique_ptr<ifaddrs, decltype(&freeifaddrs)> ifaddrs_ptr(nullptr, &freeifaddrs);
+  if (getifaddrs(&if_addr_struct) == -1) {
+    return ip_addresses;
+  }
+  ifaddrs_ptr.reset(if_addr_struct);
+
+  for (ifaddrs *ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    void *tmp_addr_ptr = nullptr;
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address

Review Comment:
   The comment should be what the code does, otherwise it must be `TODO` or `FIXME`.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1392: Convert 0.0.0.0 to local ip addresses when matching the cluster node

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1171517929


##########
src/common/io_util.cc:
##########
@@ -312,6 +314,53 @@ int AeWait(int fd, int mask, int timeout) {
   }
 }
 
+bool MatchListeningIP(std::vector<std::string> &binds, const std::string &ip) {
+  if (std::find(binds.begin(), binds.end(), ip) != binds.end()) {
+    return true;
+  }
+
+  // If binds contains 0.0.0.0, we should resolve ip addresses and check it
+  if (std::find(binds.begin(), binds.end(), "0.0.0.0") != binds.end()) {
+    auto local_ip_addresses = GetLocalIPAddresses();
+    return std::find(local_ip_addresses.begin(), local_ip_addresses.end(), ip) != local_ip_addresses.end();
+  }
+  return false;
+}
+
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIPAddresses() {
+  std::vector<std::string> ip_addresses;
+  ifaddrs *if_addr_struct = nullptr;
+  // Use unique_ptr for if_addr_struct to avoid manually free

Review Comment:
   Also here we can omit the comment in my opinion.



##########
src/common/io_util.cc:
##########
@@ -312,6 +314,53 @@ int AeWait(int fd, int mask, int timeout) {
   }
 }
 
+bool MatchListeningIP(std::vector<std::string> &binds, const std::string &ip) {
+  if (std::find(binds.begin(), binds.end(), ip) != binds.end()) {
+    return true;
+  }
+
+  // If binds contains 0.0.0.0, we should resolve ip addresses and check it
+  if (std::find(binds.begin(), binds.end(), "0.0.0.0") != binds.end()) {
+    auto local_ip_addresses = GetLocalIPAddresses();
+    return std::find(local_ip_addresses.begin(), local_ip_addresses.end(), ip) != local_ip_addresses.end();
+  }
+  return false;
+}
+
+// Get all local IP addresses.

Review Comment:
   I believe the function's name is so descriptive that we can omit this comment.



##########
src/common/io_util.cc:
##########
@@ -312,6 +314,53 @@ int AeWait(int fd, int mask, int timeout) {
   }
 }
 
+bool MatchListeningIP(std::vector<std::string> &binds, const std::string &ip) {
+  if (std::find(binds.begin(), binds.end(), ip) != binds.end()) {
+    return true;
+  }
+
+  // If binds contains 0.0.0.0, we should resolve ip addresses and check it
+  if (std::find(binds.begin(), binds.end(), "0.0.0.0") != binds.end()) {
+    auto local_ip_addresses = GetLocalIPAddresses();
+    return std::find(local_ip_addresses.begin(), local_ip_addresses.end(), ip) != local_ip_addresses.end();
+  }
+  return false;
+}
+
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIPAddresses() {
+  std::vector<std::string> ip_addresses;
+  ifaddrs *if_addr_struct = nullptr;
+  // Use unique_ptr for if_addr_struct to avoid manually free
+  std::unique_ptr<ifaddrs, decltype(&freeifaddrs)> ifaddrs_ptr(nullptr, &freeifaddrs);
+  if (getifaddrs(&if_addr_struct) == -1) {
+    return ip_addresses;
+  }
+  ifaddrs_ptr.reset(if_addr_struct);
+
+  for (ifaddrs *ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    void *tmp_addr_ptr = nullptr;
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address

Review Comment:
   The comment states that there is a check that it's not a loopback address, but I don't see any checks. Is everything OK here?
   The same for the `else` branch. 



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] zevin02 commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "zevin02 (via GitHub)" <gi...@apache.org>.
zevin02 commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169452119


##########
src/cluster/cluster.cc:
##########
@@ -157,7 +159,34 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> getLocalIPAddresses() {
+  std::vector<std::string> ipAddresses;
+
+  struct ifaddrs *ifAddrStruct = NULL;
+  struct ifaddrs *ifa = NULL;
+  void *tmpAddrPtr = NULL;
+
+  getifaddrs(&ifAddrStruct);
 
+  for (ifa = ifAddrStruct; ifa != NULL; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmpAddrPtr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char addressBuffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmpAddrPtr, addressBuffer, INET_ADDRSTRLEN);
+      std::string ipAddress(addressBuffer);
+      ipAddresses.push_back(move(ipAddress));

Review Comment:
   So is there a problem with my above operation? If it is 0.0.0.0, get all local ip and cluster node ip for matching



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] zevin02 commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "zevin02 (via GitHub)" <gi...@apache.org>.
zevin02 commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1170054751


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;
+  struct ifaddrs *ifa = nullptr;
+  void *tmp_addr_ptr = nullptr;
+
+  getifaddrs(&if_addr_struct);
+
+  for (ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmp_addr_ptr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char address_buffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmp_addr_ptr, address_buffer, INET_ADDRSTRLEN);
+      ip_addresses.emplace_back(std::string(address_buffer));
+    }
+  }
+
+  if (if_addr_struct) freeifaddrs(if_addr_struct);

Review Comment:
   I tried a bit using smart pointers to avoid manual freeing, it seems that there is some difficulty for me, can you give me some advice



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#issuecomment-1511603694

   Hi @zevin02, thanks for your contribution!
   
   We setup lots of code checks in our CI, which requires you to pass for better code quality.
   
   For example, from a glance to your code, maybe these checks will fail: 
   - local variables must be lower_case with underscore,
   - NULL must be replaced with nullptr
   
   You can check the CI log for more details and for how to modify your code to pass the CI.


-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] git-hulk commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169454116


##########
src/cluster/cluster.cc:
##########
@@ -157,7 +159,34 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> getLocalIPAddresses() {
+  std::vector<std::string> ipAddresses;
+
+  struct ifaddrs *ifAddrStruct = NULL;
+  struct ifaddrs *ifa = NULL;
+  void *tmpAddrPtr = NULL;
+
+  getifaddrs(&ifAddrStruct);
 
+  for (ifa = ifAddrStruct; ifa != NULL; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmpAddrPtr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char addressBuffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmpAddrPtr, addressBuffer, INET_ADDRSTRLEN);
+      std::string ipAddress(addressBuffer);
+      ipAddresses.push_back(move(ipAddress));

Review Comment:
   The implementation of `getLocalIPAddresses` looks good. My point is that we can use all IP addresses only if the listening address is `0.0.0.0`.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] git-hulk commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169414962


##########
src/cluster/cluster.cc:
##########
@@ -157,7 +159,34 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> getLocalIPAddresses() {
+  std::vector<std::string> ipAddresses;
+
+  struct ifaddrs *ifAddrStruct = NULL;
+  struct ifaddrs *ifa = NULL;
+  void *tmpAddrPtr = NULL;
+
+  getifaddrs(&ifAddrStruct);
 
+  for (ifa = ifAddrStruct; ifa != NULL; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmpAddrPtr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char addressBuffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmpAddrPtr, addressBuffer, INET_ADDRSTRLEN);
+      std::string ipAddress(addressBuffer);
+      ipAddresses.push_back(move(ipAddress));

Review Comment:
   It looks unnecessary to get IP addresses, we can do it once if finds the listening address was on 0.0.0.0 when starting.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] git-hulk commented on a diff in pull request #1392: Convert 0.0.0.0 to local ip addresses when matching the cluster node

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1172021444


##########
src/common/io_util.cc:
##########
@@ -312,6 +314,53 @@ int AeWait(int fd, int mask, int timeout) {
   }
 }
 
+bool MatchListeningIP(std::vector<std::string> &binds, const std::string &ip) {
+  if (std::find(binds.begin(), binds.end(), ip) != binds.end()) {
+    return true;
+  }
+
+  // If binds contains 0.0.0.0, we should resolve ip addresses and check it
+  if (std::find(binds.begin(), binds.end(), "0.0.0.0") != binds.end()) {
+    auto local_ip_addresses = GetLocalIPAddresses();
+    return std::find(local_ip_addresses.begin(), local_ip_addresses.end(), ip) != local_ip_addresses.end();
+  }
+  return false;
+}
+
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIPAddresses() {
+  std::vector<std::string> ip_addresses;
+  ifaddrs *if_addr_struct = nullptr;
+  // Use unique_ptr for if_addr_struct to avoid manually free
+  std::unique_ptr<ifaddrs, decltype(&freeifaddrs)> ifaddrs_ptr(nullptr, &freeifaddrs);
+  if (getifaddrs(&if_addr_struct) == -1) {
+    return ip_addresses;
+  }
+  ifaddrs_ptr.reset(if_addr_struct);
+
+  for (ifaddrs *ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    void *tmp_addr_ptr = nullptr;
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address

Review Comment:
   Yes, the comment should be removed if we didn't check that. It looks no necessary to check if it's a loopback address.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] zevin02 commented on pull request #1392: feat: implement local IP address retrieval function

Posted by "zevin02 (via GitHub)" <gi...@apache.org>.
zevin02 commented on PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#issuecomment-1514472933

   @PragmaTwice I think I solved all the problems, can you give me a 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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] zevin02 commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "zevin02 (via GitHub)" <gi...@apache.org>.
zevin02 commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169430253


##########
src/cluster/cluster.cc:
##########
@@ -157,7 +159,34 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> getLocalIPAddresses() {
+  std::vector<std::string> ipAddresses;
+
+  struct ifaddrs *ifAddrStruct = NULL;
+  struct ifaddrs *ifa = NULL;
+  void *tmpAddrPtr = NULL;
+
+  getifaddrs(&ifAddrStruct);
 
+  for (ifa = ifAddrStruct; ifa != NULL; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmpAddrPtr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char addressBuffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmpAddrPtr, addressBuffer, INET_ADDRSTRLEN);
+      std::string ipAddress(addressBuffer);
+      ipAddresses.push_back(move(ipAddress));

Review Comment:
   Can I understand that if the server listens to 0.0.0.0, any cluster node with any ip on the same port needs to be matched, or what?



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1168894659


##########
src/cluster/cluster.cc:
##########
@@ -157,7 +159,34 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> getLocalIPAddresses() {
+  std::vector<std::string> ipAddresses;

Review Comment:
   Hi @zevin02 , thank you for your contribution.
   Please, rename the function's local variables like `ip_addresses`.
   Also, you can use the `nullptr` keyword instead of `NULL` and simplify checks like `if (pointer != NULL)` by using `if (pointer)`
   
   



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] torwig commented on pull request #1392: feat: implement local IP address retrieval function

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#issuecomment-1511593247

   @zevin02 It's very handy to run `./x.py format` before committing the changes - it will format the C++ source code automatically.


-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] git-hulk commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169446250


##########
src/cluster/cluster.cc:
##########
@@ -157,7 +159,34 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> getLocalIPAddresses() {
+  std::vector<std::string> ipAddresses;
+
+  struct ifaddrs *ifAddrStruct = NULL;
+  struct ifaddrs *ifa = NULL;
+  void *tmpAddrPtr = NULL;
+
+  getifaddrs(&ifAddrStruct);
 
+  for (ifa = ifAddrStruct; ifa != NULL; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmpAddrPtr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char addressBuffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmpAddrPtr, addressBuffer, INET_ADDRSTRLEN);
+      std::string ipAddress(addressBuffer);
+      ipAddresses.push_back(move(ipAddress));

Review Comment:
   No actually, we can only match the IP addresses that the current node is listening. And 0.0.0.0 is a special case since it represents listening on all IP addresses in this node, we need to convert to the real IP(like 192.168.1.4) + loopback IP(127.0.0.1) before matching.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] git-hulk commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169963116


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;
+  struct ifaddrs *ifa = nullptr;
+  void *tmp_addr_ptr = nullptr;
+
+  getifaddrs(&if_addr_struct);
+
+  for (ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmp_addr_ptr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char address_buffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmp_addr_ptr, address_buffer, INET_ADDRSTRLEN);
+      ip_addresses.emplace_back(std::string(address_buffer));
+    }
+  }
+
+  if (if_addr_struct) freeifaddrs(if_addr_struct);

Review Comment:
   @zevin02 can just append fix commits directly without rebasing, then reviewers can review the new changes only. 



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169949845


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;
+  struct ifaddrs *ifa = nullptr;
+  void *tmp_addr_ptr = nullptr;
+
+  getifaddrs(&if_addr_struct);
+
+  for (ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmp_addr_ptr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char address_buffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmp_addr_ptr, address_buffer, INET_ADDRSTRLEN);
+      ip_addresses.emplace_back(std::string(address_buffer));

Review Comment:
   ```suggestion
         ip_addresses.emplace_back(address_buffer);
   ```



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] zevin02 commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "zevin02 (via GitHub)" <gi...@apache.org>.
zevin02 commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169958575


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;
+  struct ifaddrs *ifa = nullptr;
+  void *tmp_addr_ptr = nullptr;
+
+  getifaddrs(&if_addr_struct);
+
+  for (ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmp_addr_ptr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char address_buffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmp_addr_ptr, address_buffer, INET_ADDRSTRLEN);
+      ip_addresses.emplace_back(std::string(address_buffer));
+    }
+  }
+
+  if (if_addr_struct) freeifaddrs(if_addr_struct);

Review Comment:
   got it



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169946799


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;

Review Comment:
   Seems the `struct` prefix is useless.



##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;

Review Comment:
   Seems the `struct` prefix is useless 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.

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

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#issuecomment-1514478214

   > @PragmaTwice I think I solved all the problems, can you give me a review
   
   Sure. But as I mentioned in https://github.com/apache/incubator-kvrocks/pull/1392#issuecomment-1511603694, you need first to make the CI pass, which currently failed.


-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] zevin02 commented on a diff in pull request #1392: Convert 0.0.0.0 to local ip addresses when matching the cluster node

Posted by "zevin02 (via GitHub)" <gi...@apache.org>.
zevin02 commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1172020042


##########
src/common/io_util.cc:
##########
@@ -312,6 +314,53 @@ int AeWait(int fd, int mask, int timeout) {
   }
 }
 
+bool MatchListeningIP(std::vector<std::string> &binds, const std::string &ip) {
+  if (std::find(binds.begin(), binds.end(), ip) != binds.end()) {
+    return true;
+  }
+
+  // If binds contains 0.0.0.0, we should resolve ip addresses and check it
+  if (std::find(binds.begin(), binds.end(), "0.0.0.0") != binds.end()) {
+    auto local_ip_addresses = GetLocalIPAddresses();
+    return std::find(local_ip_addresses.begin(), local_ip_addresses.end(), ip) != local_ip_addresses.end();
+  }
+  return false;
+}
+
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIPAddresses() {
+  std::vector<std::string> ip_addresses;
+  ifaddrs *if_addr_struct = nullptr;
+  // Use unique_ptr for if_addr_struct to avoid manually free
+  std::unique_ptr<ifaddrs, decltype(&freeifaddrs)> ifaddrs_ptr(nullptr, &freeifaddrs);
+  if (getifaddrs(&if_addr_struct) == -1) {
+    return ip_addresses;
+  }
+  ifaddrs_ptr.reset(if_addr_struct);
+
+  for (ifaddrs *ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    void *tmp_addr_ptr = nullptr;
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address

Review Comment:
   I rechecked this code, I didn't check the local loopback address, I think it's ok, so I should delete the extra comment after that, right? or should I check if it's the local loopback address?



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169945902


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;
+  struct ifaddrs *ifa = nullptr;
+  void *tmp_addr_ptr = nullptr;
+
+  getifaddrs(&if_addr_struct);
+
+  for (ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmp_addr_ptr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char address_buffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmp_addr_ptr, address_buffer, INET_ADDRSTRLEN);
+      ip_addresses.emplace_back(std::string(address_buffer));
+    }

Review Comment:
   It seems the IPv6 part is missing, which is supported by kvrocks in other components.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169952174


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;
+  struct ifaddrs *ifa = nullptr;
+  void *tmp_addr_ptr = nullptr;
+
+  getifaddrs(&if_addr_struct);
+
+  for (ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmp_addr_ptr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char address_buffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmp_addr_ptr, address_buffer, INET_ADDRSTRLEN);
+      ip_addresses.emplace_back(std::string(address_buffer));
+    }
+  }
+
+  if (if_addr_struct) freeifaddrs(if_addr_struct);

Review Comment:
   You can try to use `std::unique_ptr` or `ScopeExit` for `if_addr_struct` to avoid manually free.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] zevin02 commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "zevin02 (via GitHub)" <gi...@apache.org>.
zevin02 commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1170054751


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;
+  struct ifaddrs *ifa = nullptr;
+  void *tmp_addr_ptr = nullptr;
+
+  getifaddrs(&if_addr_struct);
+
+  for (ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmp_addr_ptr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char address_buffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmp_addr_ptr, address_buffer, INET_ADDRSTRLEN);
+      ip_addresses.emplace_back(std::string(address_buffer));
+    }
+  }
+
+  if (if_addr_struct) freeifaddrs(if_addr_struct);

Review Comment:
   I tried a bit using smart pointers to avoid manual freeing, it seems that there is some difficulty for me, can you give me some advice



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1392: feat: implement local IP address retrieval function

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#issuecomment-1514901225

   > The rest looks good to me. cc @git-hulk
   
   @PragmaTwice I do a minor refactor, can help to take a look again.


-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice merged pull request #1392: Convert 0.0.0.0 to local ip addresses when matching the cluster node

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice merged PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392


-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1171411783


##########
src/cluster/cluster.cc:
##########
@@ -199,16 +201,34 @@ Status Cluster::SetClusterNodes(const std::string &nodes_str, int64_t version, b
       size_++;
     }
   }
+  // Used to check if listening on "0.0.0.0“
+  bool is_listen_all_ip = false;
+  // Store all the local ip
+  std::unique_ptr<std::vector<std::string>> local_hosts(nullptr);

Review Comment:
   ```suggestion
     std::unique_ptr<std::vector<std::string>> local_hosts;
   ```



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1171410597


##########
src/cluster/cluster.cc:
##########
@@ -199,16 +201,34 @@ Status Cluster::SetClusterNodes(const std::string &nodes_str, int64_t version, b
       size_++;
     }
   }
+  // Used to check if listening on "0.0.0.0“
+  bool is_listen_all_ip = false;
+  // Store all the local ip
+  std::unique_ptr<std::vector<std::string>> local_hosts(nullptr);
+
+  if (std::find(binds_.begin(), binds_.end(), "0.0.0.0") != binds_.end()) {

Review Comment:
   Seems the ipv6 address INADDR6_ANY (`::`) is missing.



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1171400891


##########
src/common/io_util.h:
##########
@@ -20,7 +20,9 @@
 
 #pragma once
 
+#include <ifaddrs.h>
 #include <netinet/in.h>
+#include <sys/types.h>

Review Comment:
   These headers can be put into `.cc`, rather than `.h`.



##########
src/cluster/cluster.h:
##########
@@ -30,6 +30,7 @@
 #include <vector>
 
 #include "commands/commander.h"
+#include "common/io_util.h"

Review Comment:
   ditto



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] zevin02 commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "zevin02 (via GitHub)" <gi...@apache.org>.
zevin02 commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169969677


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;
+  struct ifaddrs *ifa = nullptr;
+  void *tmp_addr_ptr = nullptr;
+
+  getifaddrs(&if_addr_struct);
+
+  for (ifa = if_addr_struct; ifa; ifa = ifa->ifa_next) {
+    if (!ifa->ifa_addr) {
+      continue;
+    }
+    if (ifa->ifa_addr->sa_family == AF_INET) {
+      // check it is IP4 and not a loopback address
+      tmp_addr_ptr = &((struct sockaddr_in *)ifa->ifa_addr)->sin_addr;
+      char address_buffer[INET_ADDRSTRLEN];
+      inet_ntop(AF_INET, tmp_addr_ptr, address_buffer, INET_ADDRSTRLEN);
+      ip_addresses.emplace_back(std::string(address_buffer));
+    }
+  }
+
+  if (if_addr_struct) freeifaddrs(if_addr_struct);

Review Comment:
   OK, I will submit PR like this in the future



-- 
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@kvrocks.apache.org

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


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1392: feat: implement local IP address retrieval function

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1392:
URL: https://github.com/apache/incubator-kvrocks/pull/1392#discussion_r1169948319


##########
src/cluster/cluster.cc:
##########
@@ -157,6 +159,33 @@ Status Cluster::SetSlot(int slot, const std::string &node_id, int64_t new_versio
 
   return Status::OK();
 }
+// Get all local IP addresses.
+std::vector<std::string> GetLocalIpAddresses() {
+  std::vector<std::string> ip_addresses;
+
+  struct ifaddrs *if_addr_struct = nullptr;
+  struct ifaddrs *ifa = nullptr;
+  void *tmp_addr_ptr = nullptr;

Review Comment:
   Please define these variables closer to where they are used.



-- 
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@kvrocks.apache.org

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