You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/06/24 13:02:04 UTC

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1116: MINIFICPP-1573 Make AppendHostInfo platform independent

szaszm commented on a change in pull request #1116:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1116#discussion_r657855861



##########
File path: libminifi/src/utils/NetworkInterfaceInfo.cpp
##########
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "utils/NetworkInterfaceInfo.h"
+
+#ifdef WIN32
+#include <Windows.h>
+#include <winsock2.h>
+#include <iphlpapi.h>
+#include <WS2tcpip.h>
+#pragma comment(lib, "IPHLPAPI.lib")
+#else
+#include <unistd.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
+#include <net/if.h>
+#include <arpa/inet.h>
+#include <ifaddrs.h>
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+#ifdef WIN32
+std::string utf8_encode(const std::wstring& wstr) {
+  if (wstr.empty())
+    return std::string();
+  int size_needed = WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), nullptr, 0, nullptr, nullptr);
+  std::string result_string(size_needed, 0);
+  WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), &result_string[0], size_needed, nullptr, nullptr);
+  return result_string;
+}

Review comment:
       Consider moving this into an anonymous namespace or to StringUtils.

##########
File path: libminifi/src/utils/NetworkInterfaceInfo.cpp
##########
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "utils/NetworkInterfaceInfo.h"
+
+#ifdef WIN32
+#include <Windows.h>
+#include <winsock2.h>
+#include <iphlpapi.h>
+#include <WS2tcpip.h>
+#pragma comment(lib, "IPHLPAPI.lib")
+#else
+#include <unistd.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
+#include <net/if.h>
+#include <arpa/inet.h>
+#include <ifaddrs.h>
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+#ifdef WIN32
+std::string utf8_encode(const std::wstring& wstr) {
+  if (wstr.empty())
+    return std::string();
+  int size_needed = WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), nullptr, 0, nullptr, nullptr);
+  std::string result_string(size_needed, 0);
+  WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), &result_string[0], size_needed, nullptr, nullptr);
+  return result_string;
+}
+
+NetworkInterfaceInfo::NetworkInterfaceInfo(const IP_ADAPTER_ADDRESSES* adapter) {
+  name_ = utf8_encode(adapter->FriendlyName);
+  for (auto unicast_address = adapter->FirstUnicastAddress; unicast_address != nullptr; unicast_address = unicast_address->Next) {
+    if (unicast_address->Address.lpSockaddr->sa_family == AF_INET) {
+      char address_buffer[INET_ADDRSTRLEN];
+      void* sin_address = &(reinterpret_cast<SOCKADDR_IN*>(unicast_address->Address.lpSockaddr)->sin_addr);
+      InetNtopA(AF_INET, sin_address, address_buffer, INET_ADDRSTRLEN);
+      ip_v4_addresses_.push_back(address_buffer);
+    } else if (unicast_address->Address.lpSockaddr->sa_family == AF_INET6) {
+      char address_buffer[INET6_ADDRSTRLEN];
+      void* sin_address = &(reinterpret_cast<SOCKADDR_IN*>(unicast_address->Address.lpSockaddr)->sin_addr);
+      InetNtopA(AF_INET6, sin_address, address_buffer, INET6_ADDRSTRLEN);
+      ip_v6_addresses_.push_back(address_buffer);
+    }

Review comment:
       This is already implemented in ClientSocket.cpp in an anonymous namespace, called `sockaddr_ntop`. I think it would be better to move it somewhere else and reuse the code.

##########
File path: libminifi/src/utils/NetworkInterfaceInfo.cpp
##########
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "utils/NetworkInterfaceInfo.h"
+
+#ifdef WIN32
+#include <Windows.h>
+#include <winsock2.h>
+#include <iphlpapi.h>
+#include <WS2tcpip.h>
+#pragma comment(lib, "IPHLPAPI.lib")
+#else
+#include <unistd.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
+#include <net/if.h>
+#include <arpa/inet.h>
+#include <ifaddrs.h>
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+#ifdef WIN32
+std::string utf8_encode(const std::wstring& wstr) {
+  if (wstr.empty())
+    return std::string();
+  int size_needed = WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), nullptr, 0, nullptr, nullptr);

Review comment:
       1. Please use `wstr.c_str()` instead of `&wstr[0]` to get a c string from a c++ string. Since this is windows-only code, you can also rely on [`std::basic_string::data`](https://en.cppreference.com/w/cpp/string/basic_string/data) returning a pointer to mutable buffer in the second call, because we're on /std:c++latest (full C++17, partial C++20) there.
   2. I'm also not sure about the correctness of `wstr.size()`, because it returns the number of `wchar_t` elements of the `wstring`, but [`WideCharToMultiByte`](https://docs.microsoft.com/en-us/windows/win32/api/stringapiset/nf-stringapiset-widechartomultibyte) expects the number of characters, and UTF-16 is a variable length encoding. Please change the parameter to `-1`, which processes the string until the null terminator, unless you are sure about its correctness.
   

##########
File path: libminifi/include/utils/NetworkInterfaceInfo.h
##########
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <vector>
+#include <unordered_map>
+#include "utils/OptionalUtils.h"
+
+#ifdef WIN32
+struct _IP_ADAPTER_ADDRESSES_LH;
+typedef _IP_ADAPTER_ADDRESSES_LH IP_ADAPTER_ADDRESSES_LH;
+typedef IP_ADAPTER_ADDRESSES_LH IP_ADAPTER_ADDRESSES;
+#else
+struct ifaddrs;
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+class NetworkInterfaceInfo {
+ public:
+  NetworkInterfaceInfo(NetworkInterfaceInfo&& src) = default;

Review comment:
       Please make this `noexcept` and include a move assignment operator as well. Consider also marking the getters `noexcept` as none of them can throw.

##########
File path: libminifi/src/utils/NetworkInterfaceInfo.cpp
##########
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "utils/NetworkInterfaceInfo.h"
+
+#ifdef WIN32
+#include <Windows.h>
+#include <winsock2.h>
+#include <iphlpapi.h>
+#include <WS2tcpip.h>
+#pragma comment(lib, "IPHLPAPI.lib")
+#else
+#include <unistd.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
+#include <net/if.h>
+#include <arpa/inet.h>
+#include <ifaddrs.h>
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+#ifdef WIN32
+std::string utf8_encode(const std::wstring& wstr) {
+  if (wstr.empty())
+    return std::string();
+  int size_needed = WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), nullptr, 0, nullptr, nullptr);
+  std::string result_string(size_needed, 0);
+  WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), &result_string[0], size_needed, nullptr, nullptr);
+  return result_string;
+}
+
+NetworkInterfaceInfo::NetworkInterfaceInfo(const IP_ADAPTER_ADDRESSES* adapter) {
+  name_ = utf8_encode(adapter->FriendlyName);
+  for (auto unicast_address = adapter->FirstUnicastAddress; unicast_address != nullptr; unicast_address = unicast_address->Next) {
+    if (unicast_address->Address.lpSockaddr->sa_family == AF_INET) {
+      char address_buffer[INET_ADDRSTRLEN];
+      void* sin_address = &(reinterpret_cast<SOCKADDR_IN*>(unicast_address->Address.lpSockaddr)->sin_addr);
+      InetNtopA(AF_INET, sin_address, address_buffer, INET_ADDRSTRLEN);
+      ip_v4_addresses_.push_back(address_buffer);
+    } else if (unicast_address->Address.lpSockaddr->sa_family == AF_INET6) {
+      char address_buffer[INET6_ADDRSTRLEN];
+      void* sin_address = &(reinterpret_cast<SOCKADDR_IN*>(unicast_address->Address.lpSockaddr)->sin_addr);
+      InetNtopA(AF_INET6, sin_address, address_buffer, INET6_ADDRSTRLEN);
+      ip_v6_addresses_.push_back(address_buffer);
+    }
+  }
+  running_ = adapter->OperStatus == IfOperStatusUp;
+  loopback_ = adapter->IfType == IF_TYPE_SOFTWARE_LOOPBACK;
+}
+#else
+NetworkInterfaceInfo::NetworkInterfaceInfo(const struct ifaddrs* ifa) {
+  name_ = ifa->ifa_name;
+  void* sin_address = &(reinterpret_cast<struct sockaddr_in*>(ifa->ifa_addr)->sin_addr);
+  if (ifa->ifa_addr->sa_family == AF_INET) {
+    char address_buffer[INET_ADDRSTRLEN];
+    inet_ntop(AF_INET, sin_address, address_buffer, INET_ADDRSTRLEN);
+    ip_v4_addresses_.push_back(address_buffer);
+  } else if (ifa->ifa_addr->sa_family == AF_INET6) {
+    char address_buffer[INET6_ADDRSTRLEN];
+    inet_ntop(AF_INET6, sin_address, address_buffer, INET6_ADDRSTRLEN);
+    ip_v6_addresses_.push_back(address_buffer);
+  }
+  running_ = (ifa->ifa_flags & IFF_RUNNING);
+  loopback_ = (ifa->ifa_flags & IFF_LOOPBACK);
+}
+#endif
+
+std::unordered_map<std::string, NetworkInterfaceInfo> NetworkInterfaceInfo::getNetworkInterfaceInfos(std::function<bool(const NetworkInterfaceInfo&)> filter,
+                                                                                                     const utils::optional<uint32_t> max_interfaces) {
+  std::unordered_map<std::string, NetworkInterfaceInfo> network_adapters;
+#ifdef WIN32
+  ULONG buffer_length = sizeof(IP_ADAPTER_ADDRESSES);
+  if (ERROR_BUFFER_OVERFLOW != GetAdaptersAddresses(0, 0, nullptr, nullptr, &buffer_length))
+    return network_adapters;
+  std::vector<uint8_t> bytes(buffer_length, 0);
+  IP_ADAPTER_ADDRESSES* adapter = reinterpret_cast<IP_ADAPTER_ADDRESSES*>(bytes.data());

Review comment:
       I know that `uint8_t` is commonly used for bytes in the codebase, but only `char`, `unsigned char` (but not `signed char`) and `std::byte` are allowed to alias other types. I'm not 100% sure about `char` itself, and [this part](http://eel.is/c++draft/intro.object) of the standard is a bit hard to translate to different scenarios.

##########
File path: extensions/standard-processors/processors/AppendHostInfo.cpp
##########
@@ -23,90 +23,88 @@
 #define __USE_POSIX
 #endif /* __USE_POSIX */
 
-#include <limits.h>
-#include <string.h>
 #include <memory>
 #include <string>
-#include <set>
+#include <regex>
+#include <algorithm>
 #include "core/ProcessContext.h"
 #include "core/Property.h"
 #include "core/ProcessSession.h"
 #include "core/FlowFile.h"
 #include "io/ClientSocket.h"
+#include "utils/NetworkInterfaceInfo.h"
+
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace processors {
 
-#ifndef WIN32
-#include <netdb.h>
-#include <netinet/in.h>
-#include <sys/socket.h>
-#include <sys/ioctl.h>
-#include <net/if.h>
-#include <arpa/inet.h>
-#endif
-
-#ifndef HOST_NAME_MAX
-#define HOST_NAME_MAX 255
-#endif
+core::Property AppendHostInfo::InterfaceNameFilter("Network Interface Filter", "A regular expression to filter ip addresses based on the name of the network interface", "");
+core::Property AppendHostInfo::HostAttribute("Hostname Attribute", "Flowfile attribute used to record the agent's hostname", "source.hostname");
+core::Property AppendHostInfo::IPAttribute("IP Attribute", "Flowfile attribute used to record the agent's IP addresses in a comma separated list", "source.ipv4");
+core::Property AppendHostInfo::RefreshPolicy(core::PropertyBuilder::createProperty("Refresh Policy")
+    ->withDescription("When to recalculate the host info")
+    ->withAllowableValues<std::string>({ REFRESH_POLICY_ON_SCHEDULE, REFRESH_POLICY_ON_TRIGGER })
+    ->withDefaultValue(REFRESH_POLICY_ON_SCHEDULE)->build());
 
-core::Property AppendHostInfo::InterfaceName("Network Interface Name", "Network interface from which to read an IP v4 address", "eth0");
-core::Property AppendHostInfo::HostAttribute("Hostname Attribute", "Flowfile attribute to used to record the agent's hostname", "source.hostname");
-core::Property AppendHostInfo::IPAttribute("IP Attribute", "Flowfile attribute to used to record the agent's IP address", "source.ipv4");
 core::Relationship AppendHostInfo::Success("success", "success operational on the flow record");
 
 void AppendHostInfo::initialize() {
-  // Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(InterfaceName);
-  properties.insert(HostAttribute);
-  properties.insert(IPAttribute);
-  setSupportedProperties(properties);
+  setSupportedProperties({InterfaceNameFilter, HostAttribute, IPAttribute, RefreshPolicy});
+  setSupportedRelationships({Success});
+}
 
-  // Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+void AppendHostInfo::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  context->getProperty(HostAttribute.getName(), hostname_attribute_name_);
+  context->getProperty(IPAttribute.getName(), ipaddress_attribute_name_);
+  context->getProperty(InterfaceNameFilter.getName(), interface_name_filter_);
+  std::string refresh_policy;
+  context->getProperty(RefreshPolicy.getName(), refresh_policy);
+  if (refresh_policy == REFRESH_POLICY_ON_TRIGGER)
+    refresh_on_trigger_ = true;
+
+  if (!refresh_on_trigger_)
+    refreshHostInfo();
 }
 
-void AppendHostInfo::onTrigger(core::ProcessContext *context, core::ProcessSession *session) {
+void AppendHostInfo::onTrigger(core::ProcessContext*, core::ProcessSession* session) {
   std::shared_ptr<core::FlowFile> flow = session->get();
   if (!flow)
     return;
 
-  // Get Hostname
-
-  std::string hostAttribute = "";
-  context->getProperty(HostAttribute.getName(), hostAttribute);
-  flow->addAttribute(hostAttribute.c_str(), org::apache::nifi::minifi::io::Socket::getMyHostName());
+  if (refresh_on_trigger_)
+    refreshHostInfo();
 
-  // Get IP address for the specified interface
-  std::string iface;
-  context->getProperty(InterfaceName.getName(), iface);
-  // Confirm the specified interface name exists on this device
-#ifndef WIN32
-  if (if_nametoindex(iface.c_str()) != 0) {
-    struct ifreq ifr;
-    int fd = socket(AF_INET, SOCK_DGRAM, 0);
-    // Type of address to retrieve - IPv4 IP address
-    ifr.ifr_addr.sa_family = AF_INET;
-    // Copy the interface name in the ifreq structure
-    strncpy(ifr.ifr_name, iface.c_str(), IFNAMSIZ - 1);
-    ioctl(fd, SIOCGIFADDR, &ifr);
-    close(fd);
-
-    std::string ipAttribute;
-    context->getProperty(IPAttribute.getName(), ipAttribute);
-    flow->addAttribute(ipAttribute.c_str(), inet_ntoa(((struct sockaddr_in *) &ifr.ifr_addr)->sin_addr));
+  flow->addAttribute(hostname_attribute_name_, hostname_);
+  if (ipaddresses_.has_value()) {
+    flow->addAttribute(ipaddress_attribute_name_, ipaddresses_.value());
   }
-#endif
 
-  // Transfer to the relationship
   session->transfer(flow, Success);
 }
 
+void AppendHostInfo::refreshHostInfo() {
+  hostname_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+  auto filter = [this](const utils::NetworkInterfaceInfo& interface_info) -> bool {
+    bool has_ipv4_address = interface_info.hasIpV4Address();
+    bool matches_regex_or_empty_regex = interface_name_filter_.empty() || std::regex_match(interface_info.getName(), std::regex(interface_name_filter_));

Review comment:
       It's not ideal to compile the regex on every filter call. Please compile the regex in `onSchedule` and reuse it in filter calls.

##########
File path: libminifi/src/utils/NetworkInterfaceInfo.cpp
##########
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "utils/NetworkInterfaceInfo.h"
+
+#ifdef WIN32
+#include <Windows.h>
+#include <winsock2.h>
+#include <iphlpapi.h>
+#include <WS2tcpip.h>
+#pragma comment(lib, "IPHLPAPI.lib")
+#else
+#include <unistd.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
+#include <net/if.h>
+#include <arpa/inet.h>
+#include <ifaddrs.h>
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+#ifdef WIN32
+std::string utf8_encode(const std::wstring& wstr) {
+  if (wstr.empty())
+    return std::string();
+  int size_needed = WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), nullptr, 0, nullptr, nullptr);
+  std::string result_string(size_needed, 0);
+  WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), &result_string[0], size_needed, nullptr, nullptr);
+  return result_string;
+}
+
+NetworkInterfaceInfo::NetworkInterfaceInfo(const IP_ADAPTER_ADDRESSES* adapter) {
+  name_ = utf8_encode(adapter->FriendlyName);
+  for (auto unicast_address = adapter->FirstUnicastAddress; unicast_address != nullptr; unicast_address = unicast_address->Next) {
+    if (unicast_address->Address.lpSockaddr->sa_family == AF_INET) {
+      char address_buffer[INET_ADDRSTRLEN];
+      void* sin_address = &(reinterpret_cast<SOCKADDR_IN*>(unicast_address->Address.lpSockaddr)->sin_addr);
+      InetNtopA(AF_INET, sin_address, address_buffer, INET_ADDRSTRLEN);
+      ip_v4_addresses_.push_back(address_buffer);
+    } else if (unicast_address->Address.lpSockaddr->sa_family == AF_INET6) {
+      char address_buffer[INET6_ADDRSTRLEN];
+      void* sin_address = &(reinterpret_cast<SOCKADDR_IN*>(unicast_address->Address.lpSockaddr)->sin_addr);
+      InetNtopA(AF_INET6, sin_address, address_buffer, INET6_ADDRSTRLEN);
+      ip_v6_addresses_.push_back(address_buffer);
+    }
+  }
+  running_ = adapter->OperStatus == IfOperStatusUp;
+  loopback_ = adapter->IfType == IF_TYPE_SOFTWARE_LOOPBACK;
+}
+#else
+NetworkInterfaceInfo::NetworkInterfaceInfo(const struct ifaddrs* ifa) {
+  name_ = ifa->ifa_name;
+  void* sin_address = &(reinterpret_cast<struct sockaddr_in*>(ifa->ifa_addr)->sin_addr);
+  if (ifa->ifa_addr->sa_family == AF_INET) {
+    char address_buffer[INET_ADDRSTRLEN];
+    inet_ntop(AF_INET, sin_address, address_buffer, INET_ADDRSTRLEN);
+    ip_v4_addresses_.push_back(address_buffer);
+  } else if (ifa->ifa_addr->sa_family == AF_INET6) {
+    char address_buffer[INET6_ADDRSTRLEN];
+    inet_ntop(AF_INET6, sin_address, address_buffer, INET6_ADDRSTRLEN);
+    ip_v6_addresses_.push_back(address_buffer);
+  }
+  running_ = (ifa->ifa_flags & IFF_RUNNING);
+  loopback_ = (ifa->ifa_flags & IFF_LOOPBACK);
+}
+#endif
+
+std::unordered_map<std::string, NetworkInterfaceInfo> NetworkInterfaceInfo::getNetworkInterfaceInfos(std::function<bool(const NetworkInterfaceInfo&)> filter,
+                                                                                                     const utils::optional<uint32_t> max_interfaces) {
+  std::unordered_map<std::string, NetworkInterfaceInfo> network_adapters;

Review comment:
       I think it's best to keep the network interfaces in the order they're returned by the system call. For this reason, I suggest changing the return type to `std::vector<network_interface_entry>` with `struct network_interface_entry { std::string interface_name; NetworkInterfaceInfo network_interface_info; };` or something along these lines.

##########
File path: libminifi/src/utils/NetworkInterfaceInfo.cpp
##########
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "utils/NetworkInterfaceInfo.h"
+
+#ifdef WIN32
+#include <Windows.h>
+#include <winsock2.h>
+#include <iphlpapi.h>
+#include <WS2tcpip.h>
+#pragma comment(lib, "IPHLPAPI.lib")
+#else
+#include <unistd.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
+#include <net/if.h>
+#include <arpa/inet.h>
+#include <ifaddrs.h>
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+#ifdef WIN32
+std::string utf8_encode(const std::wstring& wstr) {
+  if (wstr.empty())
+    return std::string();
+  int size_needed = WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), nullptr, 0, nullptr, nullptr);
+  std::string result_string(size_needed, 0);
+  WideCharToMultiByte(CP_UTF8, 0, &wstr[0], wstr.size(), &result_string[0], size_needed, nullptr, nullptr);
+  return result_string;
+}
+
+NetworkInterfaceInfo::NetworkInterfaceInfo(const IP_ADAPTER_ADDRESSES* adapter) {
+  name_ = utf8_encode(adapter->FriendlyName);
+  for (auto unicast_address = adapter->FirstUnicastAddress; unicast_address != nullptr; unicast_address = unicast_address->Next) {
+    if (unicast_address->Address.lpSockaddr->sa_family == AF_INET) {
+      char address_buffer[INET_ADDRSTRLEN];
+      void* sin_address = &(reinterpret_cast<SOCKADDR_IN*>(unicast_address->Address.lpSockaddr)->sin_addr);
+      InetNtopA(AF_INET, sin_address, address_buffer, INET_ADDRSTRLEN);
+      ip_v4_addresses_.push_back(address_buffer);
+    } else if (unicast_address->Address.lpSockaddr->sa_family == AF_INET6) {
+      char address_buffer[INET6_ADDRSTRLEN];
+      void* sin_address = &(reinterpret_cast<SOCKADDR_IN*>(unicast_address->Address.lpSockaddr)->sin_addr);
+      InetNtopA(AF_INET6, sin_address, address_buffer, INET6_ADDRSTRLEN);
+      ip_v6_addresses_.push_back(address_buffer);
+    }
+  }
+  running_ = adapter->OperStatus == IfOperStatusUp;
+  loopback_ = adapter->IfType == IF_TYPE_SOFTWARE_LOOPBACK;
+}
+#else
+NetworkInterfaceInfo::NetworkInterfaceInfo(const struct ifaddrs* ifa) {
+  name_ = ifa->ifa_name;
+  void* sin_address = &(reinterpret_cast<struct sockaddr_in*>(ifa->ifa_addr)->sin_addr);
+  if (ifa->ifa_addr->sa_family == AF_INET) {
+    char address_buffer[INET_ADDRSTRLEN];
+    inet_ntop(AF_INET, sin_address, address_buffer, INET_ADDRSTRLEN);
+    ip_v4_addresses_.push_back(address_buffer);
+  } else if (ifa->ifa_addr->sa_family == AF_INET6) {
+    char address_buffer[INET6_ADDRSTRLEN];
+    inet_ntop(AF_INET6, sin_address, address_buffer, INET6_ADDRSTRLEN);

Review comment:
       You can't use a `struct sockaddr_in*` to work with IPv6 addresses. `struct sockaddr*` is a generic pointer, `struct sockaddr_in*` is IPv4-specific and `struct sockaddr_in6*` is for IPv6 addresses. This pattern also encourages violating aliasing rules, but they didn't really care about those in the 90's when the sockets api was born. This is why `memcpy` is used in `sockaddr_ntop` in ClientSocket.cpp.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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