You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2020/06/26 00:21:26 UTC

[kudu] branch master updated (396b70b -> 5190073)

This is an automated email from the ASF dual-hosted git repository.

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


    from 396b70b  KUDU-2612 p1: add initial transaction status storage
     new 27cf7d8  [test_util] multiple bind address for WaitFor{Tcp,Udp}Bind()
     new 5190073  [mini_ranger] search for bound port at loopback and wildcard addresses

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/hms/mini_hms.cc           |  2 +-
 src/kudu/postgres/mini_postgres.cc |  2 +-
 src/kudu/ranger/mini_ranger.cc     |  7 ++--
 src/kudu/security/test/mini_kdc.cc |  4 +-
 src/kudu/util/test_util.cc         | 85 ++++++++++++++++++++------------------
 src/kudu/util/test_util.h          | 13 +++---
 6 files changed, 60 insertions(+), 53 deletions(-)


[kudu] 02/02: [mini_ranger] search for bound port at loopback and wildcard addresses

Posted by al...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 51900735075fca92e67cd9dd290b304604d70a84
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Thu Jun 25 00:12:38 2020 -0700

    [mini_ranger] search for bound port at loopback and wildcard addresses
    
    This patch updates the set of candidate bind addresses for MiniRanger.
    There is an evidence that in some environments it's bound to the
    wildcard IP address, sometimes on a loopback IP address.  The easiest
    way to address the issue is to specify both loopback and wildcard
    addresses as candidates when searching for the bound Ranger's port.
    
    The motivation for this patch was seeing the following test failures
    when running RangerClientTestBase.TestLogging test scenario on
    Ubuntu 14.04:
    
      src/kudu/ranger/ranger_client-test.cc:345: Failure
      Failed
      Bad status: Runtime error: unexpected lsof output: p158341
      f58
      n127.0.0.1:60001
      f243
      n127.0.0.1:47896->127.0.0.1:44711
      f244
      n127.0.0.1:47894->127.0.0.1:44711
      f245
      n127.0.0.1:47898->127.0.0.1:44711
      f246
      n127.0.0.1:47920->127.0.0.1:44711
      f247
      n127.0.0.1:47922->127.0.0.1:44711
    
    I verified that with this patch the RangerClientTestBase.TestLogging
    test scenario successfully completes.
    
    Change-Id: Idcc6835f035fa1af1aeed171a14197e03a60f547
    Reviewed-on: http://gerrit.cloudera.org:8080/16111
    Tested-by: Kudu Jenkins
    Reviewed-by: Attila Bukor <ab...@apache.org>
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/ranger/mini_ranger.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/kudu/ranger/mini_ranger.cc b/src/kudu/ranger/mini_ranger.cc
index 8defe47..713cfb1 100644
--- a/src/kudu/ranger/mini_ranger.cc
+++ b/src/kudu/ranger/mini_ranger.cc
@@ -229,7 +229,7 @@ Status MiniRanger::StartRanger() {
     uint16_t port;
     RETURN_NOT_OK(WaitForTcpBind(process_->pid(),
                                  &port,
-                                 {},
+                                 { "0.0.0.0", "127.0.0.1", },
                                  MonoDelta::FromMilliseconds(kRangerStartTimeoutMs)));
     LOG(INFO) << "Ranger bound to " << port;
     LOG(INFO) << "Ranger admin URL: " << ranger_admin_url_;


[kudu] 01/02: [test_util] multiple bind address for WaitFor{Tcp, Udp}Bind()

Posted by al...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 27cf7d881f1ab68b7c022d88c5f28160a0d0e518
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Wed Jun 24 21:04:50 2020 -0700

    [test_util] multiple bind address for WaitFor{Tcp,Udp}Bind()
    
    This patch introduces a way to specify multiple candidate bind addresses
    for WaitForTcpBind() and WaitForUdpBind() test utility functions.  The
    newly introduced functionality will be used in a follow-up changelist.
    
    Change-Id: Iacda85624cb31995adc5910302468d80d5177237
    Reviewed-on: http://gerrit.cloudera.org:8080/16110
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/hms/mini_hms.cc           |  2 +-
 src/kudu/postgres/mini_postgres.cc |  2 +-
 src/kudu/ranger/mini_ranger.cc     |  7 ++--
 src/kudu/security/test/mini_kdc.cc |  4 +-
 src/kudu/util/test_util.cc         | 85 ++++++++++++++++++++------------------
 src/kudu/util/test_util.h          | 13 +++---
 6 files changed, 60 insertions(+), 53 deletions(-)

diff --git a/src/kudu/hms/mini_hms.cc b/src/kudu/hms/mini_hms.cc
index fefe53c..1158e30 100644
--- a/src/kudu/hms/mini_hms.cc
+++ b/src/kudu/hms/mini_hms.cc
@@ -168,7 +168,7 @@ Status MiniHms::Start() {
   // Wait for HMS to start listening on its ports and commencing operation
   // with a wildcard binding.
   VLOG(1) << "Waiting for HMS ports";
-  Status wait = WaitForTcpBind(hms_process_->pid(), &port_, /*addr=*/none,
+  Status wait = WaitForTcpBind(hms_process_->pid(), &port_, {},
                                MonoDelta::FromMilliseconds(kHmsStartTimeoutMs));
   if (!wait.ok()) {
     WARN_NOT_OK(hms_process_->Kill(SIGQUIT), "failed to send SIGQUIT to HMS");
diff --git a/src/kudu/postgres/mini_postgres.cc b/src/kudu/postgres/mini_postgres.cc
index ab739d5..8146cef 100644
--- a/src/kudu/postgres/mini_postgres.cc
+++ b/src/kudu/postgres/mini_postgres.cc
@@ -86,7 +86,7 @@ Status MiniPostgres::Start() {
   });
   RETURN_NOT_OK(process_->Start());
 
-  Status wait = WaitForTcpBind(process_->pid(), &port_, host_,
+  Status wait = WaitForTcpBind(process_->pid(), &port_, { host_ },
                                MonoDelta::FromMilliseconds(kPgStartTimeoutMs));
   if (!wait.ok()) {
     // TODO(abukor): implement retry with a different port if it can't bind
diff --git a/src/kudu/ranger/mini_ranger.cc b/src/kudu/ranger/mini_ranger.cc
index fbf9089..8defe47 100644
--- a/src/kudu/ranger/mini_ranger.cc
+++ b/src/kudu/ranger/mini_ranger.cc
@@ -22,7 +22,6 @@
 #include <string>
 #include <vector>
 
-#include <boost/optional/optional.hpp>
 #include <glog/logging.h>
 
 #include "kudu/gutil/map-util.h"
@@ -228,8 +227,10 @@ Status MiniRanger::StartRanger() {
     });
     RETURN_NOT_OK(process_->Start());
     uint16_t port;
-    RETURN_NOT_OK(WaitForTcpBind(process_->pid(), &port, boost::none,
-                  MonoDelta::FromMilliseconds(kRangerStartTimeoutMs)));
+    RETURN_NOT_OK(WaitForTcpBind(process_->pid(),
+                                 &port,
+                                 {},
+                                 MonoDelta::FromMilliseconds(kRangerStartTimeoutMs)));
     LOG(INFO) << "Ranger bound to " << port;
     LOG(INFO) << "Ranger admin URL: " << ranger_admin_url_;
   }
diff --git a/src/kudu/security/test/mini_kdc.cc b/src/kudu/security/test/mini_kdc.cc
index da11d07..727bfe4 100644
--- a/src/kudu/security/test/mini_kdc.cc
+++ b/src/kudu/security/test/mini_kdc.cc
@@ -156,8 +156,8 @@ Status MiniKdc::Start() {
   const bool need_config_update = (options_.port == 0);
   // Wait for KDC to start listening on its ports and commencing operation
   // with a wildcard binding.
-  RETURN_NOT_OK(WaitForUdpBind(kdc_process_->pid(), &options_.port,
-                               /*addr=*/none, MonoDelta::FromSeconds(1)));
+  RETURN_NOT_OK(WaitForUdpBind(
+      kdc_process_->pid(), &options_.port, {}, MonoDelta::FromSeconds(1)));
 
   if (need_config_update) {
     // If we asked for an ephemeral port, grab the actual ports and
diff --git a/src/kudu/util/test_util.cc b/src/kudu/util/test_util.cc
index 693123e..bce1693 100644
--- a/src/kudu/util/test_util.cc
+++ b/src/kudu/util/test_util.cc
@@ -37,7 +37,6 @@
 #include <sys/param.h> // for MAXPATHLEN
 #endif
 
-#include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 #include <gtest/gtest-spi.h>
@@ -68,7 +67,6 @@ DEFINE_int32(test_random_seed, 0, "Random seed to use for randomized tests");
 
 DECLARE_string(time_source);
 
-using boost::optional;
 using std::string;
 using std::vector;
 using strings::Substitute;
@@ -406,8 +404,11 @@ int CountOpenFds(Env* env, const string& path_pattern) {
 
 namespace {
 Status WaitForBind(pid_t pid, uint16_t* port,
-                   const optional<const string&>& addr,
-                   const char* kind, MonoDelta timeout) {
+                   const vector<string>& addresses,
+                   const char* kind,
+                   MonoDelta timeout) {
+  static const vector<string> kWildcard = { "0.0.0.0" };
+
   // In general, processes do not expose the port they bind to, and
   // reimplementing lsof involves parsing a lot of files in /proc/. So,
   // requiring lsof for tests and parsing its output seems more
@@ -442,59 +443,63 @@ Status WaitForBind(pid_t pid, uint16_t* port,
   // to be the primary service port. When searching, we use the provided bind
   // address if there is any, otherwise we use '*' (same as '0.0.0.0') which
   // matches all addresses on the local machine.
-  string addr_pattern = Substitute("n$0:", (!addr || *addr == "0.0.0.0") ? "*" : *addr);
-  MonoTime deadline = MonoTime::Now() + timeout;
-  string lsof_out;
-  int32_t p = -1;
-
-  for (int64_t i = 1; ; i++) {
-    lsof_out.clear();
-    Status s = Subprocess::Call(cmd, "", &lsof_out).AndThen([&] () {
-      StripTrailingNewline(&lsof_out);
-      vector<string> lines = strings::Split(lsof_out, "\n");
-      for (int index = 2; index < lines.size(); index += 2) {
-        StringPiece cur_line(lines[index]);
-        if (HasPrefixString(cur_line.ToString(), addr_pattern) &&
-            !cur_line.contains("->")) {
-          cur_line.remove_prefix(addr_pattern.size());
-          if (!safe_strto32(cur_line.data(), cur_line.size(), &p)) {
-            return Status::RuntimeError("unexpected lsof output", lsof_out);
+  const MonoTime deadline = MonoTime::Now() + timeout;
+  const auto& addresses_to_check = addresses.empty() ? kWildcard : addresses;
+  for (int64_t i = 1; ; ++i) {
+    for (const auto& addr : addresses_to_check) {
+      string addr_pattern = Substitute("n$0:", addr == "0.0.0.0" ? "*" : addr);
+      string lsof_out;
+      int32_t p = -1;
+      Status s = Subprocess::Call(cmd, "", &lsof_out).AndThen([&] () {
+        StripTrailingNewline(&lsof_out);
+        vector<string> lines = strings::Split(lsof_out, "\n");
+        for (int index = 2; index < lines.size(); index += 2) {
+          StringPiece cur_line(lines[index]);
+          if (HasPrefixString(cur_line.ToString(), addr_pattern) &&
+              !cur_line.contains("->")) {
+            cur_line.remove_prefix(addr_pattern.size());
+            if (!safe_strto32(cur_line.data(), cur_line.size(), &p)) {
+              return Status::RuntimeError("unexpected lsof output", lsof_out);
+            }
+
+            return Status::OK();
           }
-
-          return Status::OK();
         }
-      }
 
-      return Status::RuntimeError("unexpected lsof output", lsof_out);
-    });
+        return Status::RuntimeError("unexpected lsof output", lsof_out);
+      });
 
-    if (s.ok()) {
-      break;
-    }
-    if (deadline < MonoTime::Now()) {
-      return s;
-    }
+      if (s.ok()) {
+        CHECK(p > 0 && p < std::numeric_limits<uint16_t>::max())
+            << "parsed invalid port: " << p;
+        VLOG(1) << "Determined bound port: " << p;
+        *port = static_cast<uint16_t>(p);
 
+        return Status::OK();
+      }
+      if (deadline < MonoTime::Now()) {
+        return s;
+      }
+    }
     SleepFor(MonoDelta::FromMilliseconds(i * 10));
   }
 
-  CHECK(p > 0 && p < std::numeric_limits<uint16_t>::max()) << "parsed invalid port: " << p;
-  VLOG(1) << "Determined bound port: " << p;
-  *port = p;
-  return Status::OK();
+  // Should not reach here.
+  LOG(FATAL) << "could not determine bound port the process";
+  __builtin_unreachable();
 }
 } // anonymous namespace
 
 Status WaitForTcpBind(pid_t pid, uint16_t* port,
-                      const optional<const string&>& addr,
+                      const vector<string>& addresses,
                       MonoDelta timeout) {
-  return WaitForBind(pid, port, addr, "4TCP", timeout);
+  return WaitForBind(pid, port, addresses, "4TCP", timeout);
 }
 
 Status WaitForUdpBind(pid_t pid, uint16_t* port,
-                      const optional<const string&>& addr,
+                      const vector<string>& addresses,
                       MonoDelta timeout) {
-  return WaitForBind(pid, port, addr, "4UDP", timeout);
+  return WaitForBind(pid, port, addresses, "4UDP", timeout);
 }
 
 Status FindHomeDir(const string& name, const string& bin_dir, string* home_dir) {
diff --git a/src/kudu/util/test_util.h b/src/kudu/util/test_util.h
index d320e3e..9f60aef 100644
--- a/src/kudu/util/test_util.h
+++ b/src/kudu/util/test_util.h
@@ -25,8 +25,8 @@
 #include <functional>
 #include <memory>
 #include <string>
+#include <vector>
 
-#include <boost/optional/optional.hpp>
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/port.h"
@@ -144,16 +144,17 @@ void AssertEventually(const std::function<void(void)>& f,
 // unlike the usual behavior of path globs.
 int CountOpenFds(Env* env, const std::string& path_pattern);
 
-// Waits for the subprocess to bind to any listening TCP port on the provided
-// IP address (if the address is not provided, it is a wildcard binding), and
-// returns the port.
+// Waits for the subprocess to bind to any listening TCP port at least on one
+// of the provided IP addresses and returns the port number. As for values
+// for the 'addresses' parameter, {} (an empty vector) and { "0.0.0.0" }
+// semantically mean the same.
 Status WaitForTcpBind(pid_t pid, uint16_t* port,
-                      const boost::optional<const std::string&>& addr,
+                      const std::vector<std::string>& addresses,
                       MonoDelta timeout) WARN_UNUSED_RESULT;
 
 // Similar to above but binds to any listening UDP port.
 Status WaitForUdpBind(pid_t pid, uint16_t* port,
-                      const boost::optional<const std::string&>& addr,
+                      const std::vector<std::string>& addresses,
                       MonoDelta timeout) WARN_UNUSED_RESULT;
 
 // Find the home directory of a Java-style application, e.g. JAVA_HOME or