You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2017/07/11 00:06:16 UTC

kudu git commit: MiniKdc: move common logic into test-utils

Repository: kudu
Updated Branches:
  refs/heads/master 2c0237469 -> 5b18be1a7


MiniKdc: move common logic into test-utils

Change-Id: Ie3f50e315f3e7ea16b3c73f058248c3dbd47d72e
Reviewed-on: http://gerrit.cloudera.org:8080/7356
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/5b18be1a
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/5b18be1a
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/5b18be1a

Branch: refs/heads/master
Commit: 5b18be1a707d76abedc0f6542015a9dfa3d1387f
Parents: 2c02374
Author: Dan Burkert <da...@apache.org>
Authored: Wed Jul 5 16:11:09 2017 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Mon Jul 10 23:39:57 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/test/mini_kdc.cc | 98 +--------------------------------
 src/kudu/security/test/mini_kdc.h  |  4 --
 src/kudu/util/test_util.cc         | 95 ++++++++++++++++++++++++++++++++
 src/kudu/util/test_util.h          | 13 +++++
 4 files changed, 110 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/5b18be1a/src/kudu/security/test/mini_kdc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/mini_kdc.cc b/src/kudu/security/test/mini_kdc.cc
index b297d9b..bae2278 100644
--- a/src/kudu/security/test/mini_kdc.cc
+++ b/src/kudu/security/test/mini_kdc.cc
@@ -20,7 +20,6 @@
 #include <csignal>
 #include <stdlib.h>
 
-#include <limits>
 #include <map>
 #include <memory>
 #include <string>
@@ -28,9 +27,6 @@
 
 #include <glog/logging.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
-#include "kudu/gutil/strings/numbers.h"
-#include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/env.h"
@@ -98,34 +94,6 @@ vector<string> MiniKdc::MakeArgv(const vector<string>& in_argv) {
 
 namespace {
 // Attempts to find the path to the specified Kerberos binary, storing it in 'path'.
-Status GetBinaryPath(const string& binary,
-                     const vector<string>& search,
-                     string* path) {
-  string p;
-
-  // First, check specified locations which are sometimes not on the PATH.
-  // This is necessary to check first so that the system Heimdal kerberos
-  // binaries won't be found first on OS X.
-  for (const auto& location : search) {
-    p = JoinPathSegments(location, binary);
-    if (Env::Default()->FileExists(p)) {
-      *path = p;
-      return Status::OK();
-    }
-  }
-
-  // Next check if the binary is on the PATH.
-  Status s = Subprocess::Call({ "which", binary }, "", &p);
-  if (s.ok()) {
-    StripTrailingNewline(&p);
-    *path = p;
-    return Status::OK();
-  }
-
-  return Status::NotFound("Unable to find binary", binary);
-}
-
-// Attempts to find the path to the specified Kerberos binary, storing it in 'path'.
 Status GetBinaryPath(const string& binary, string* path) {
   static const vector<string> kCommonLocations = {
     "/usr/local/opt/krb5/sbin", // Homebrew
@@ -135,11 +103,10 @@ Status GetBinaryPath(const string& binary, string* path) {
     "/usr/lib/mit/sbin", // SLES
     "/usr/sbin", // Linux
   };
-  return GetBinaryPath(binary, kCommonLocations, path);
+  return GetExecutablePath(binary, kCommonLocations, path);
 }
 } // namespace
 
-
 Status MiniKdc::Start() {
   SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, "starting KDC");
   CHECK(!kdc_process_);
@@ -178,7 +145,7 @@ Status MiniKdc::Start() {
 
   const bool need_config_update = (options_.port == 0);
   // Wait for KDC to start listening on its ports and commencing operation.
-  RETURN_NOT_OK(WaitForKdcPorts());
+  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
@@ -261,67 +228,6 @@ Status MiniKdc::CreateKrb5Conf() const {
                            JoinPathSegments(options_.data_root, "krb5.conf"));
 }
 
-Status MiniKdc::WaitForKdcPorts() {
-  SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("waiting for KDC ports"));
-  // We have to use 'lsof' to figure out which ports the KDC bound to if we
-  // requested ephemeral ones. The KDC doesn't log the bound port or expose it
-  // in any other fashion, and re-implementing lsof involves parsing a lot of
-  // files in /proc/. So, requiring lsof for tests and parsing its output seems
-  // more straight-forward. We call lsof in a loop in case the kdc is slow to
-  // bind to the ports.
-
-  string lsof;
-  RETURN_NOT_OK(GetBinaryPath("lsof", {"/sbin", "/usr/sbin"}, &lsof));
-
-  const vector<string> cmd = {
-    lsof, "-wbnP", "-Ffn",
-    "-p", std::to_string(kdc_process_->pid()),
-    "-a", "-i", "4UDP"};
-
-  string lsof_out;
-  for (int i = 1; ; i++) {
-    lsof_out.clear();
-    Status s = Subprocess::Call(cmd, "", &lsof_out);
-
-    if (s.ok()) {
-      StripTrailingNewline(&lsof_out);
-      break;
-    } else if (i > 10) {
-      return s;
-    }
-
-    SleepFor(MonoDelta::FromMilliseconds(i * i));
-  }
-
-  // The '-Ffn' flag gets lsof to output something like:
-  //   p19730
-  //   f123
-  //   n*:41254
-  // The first line is the pid. We ignore it.
-  // The second line is the file descriptor number. We ignore it.
-  // The third line has the bind address and port.
-  vector<string> lines = strings::Split(lsof_out, "\n");
-  int32_t port = -1;
-  if (lines.size() != 3 ||
-      lines[2].substr(0, 3) != "n*:" ||
-      !safe_strto32(lines[2].substr(3), &port) ||
-      port <= 0) {
-    return Status::RuntimeError("unexpected lsof output", lsof_out);
-  }
-  CHECK(port > 0 && port < std::numeric_limits<uint16_t>::max())
-      << "parsed invalid port: " << port;
-  VLOG(1) << "Determined bound KDC port: " << port;
-  if (options_.port == 0) {
-    options_.port = port;
-  } else {
-    // Sanity check: if KDC's port is already established, it's supposed to be
-    // written into the configuration files, so the process must bind to the
-    // already established port.
-    CHECK(options_.port == port);
-  }
-  return Status::OK();
-}
-
 Status MiniKdc::CreateUserPrincipal(const string& username) {
   SCOPED_LOG_SLOW_EXECUTION(WARNING, 100, Substitute("creating user principal $0", username));
   string kadmin;

http://git-wip-us.apache.org/repos/asf/kudu/blob/5b18be1a/src/kudu/security/test/mini_kdc.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/mini_kdc.h b/src/kudu/security/test/mini_kdc.h
index 5f8ee4d..760c74b 100644
--- a/src/kudu/security/test/mini_kdc.h
+++ b/src/kudu/security/test/mini_kdc.h
@@ -122,10 +122,6 @@ class MiniKdc {
   // Creates a krb5.conf in the data root.
   Status CreateKdcConf() const WARN_UNUSED_RESULT;
 
-  // Determine the ports that the KDC bound to. Will wait for the KDC if it is
-  // still initializing.
-  Status WaitForKdcPorts() WARN_UNUSED_RESULT;
-
   std::unique_ptr<Subprocess> kdc_process_;
   MiniKdcOptions options_;
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/5b18be1a/src/kudu/util/test_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_util.cc b/src/kudu/util/test_util.cc
index af809a7..9180a7b 100644
--- a/src/kudu/util/test_util.cc
+++ b/src/kudu/util/test_util.cc
@@ -16,6 +16,7 @@
 // under the License.
 #include "kudu/util/test_util.h"
 
+#include <limits>
 #include <map>
 #include <string>
 #include <vector>
@@ -25,7 +26,9 @@
 #include <glog/stl_logging.h>
 #include <gtest/gtest-spi.h>
 
+#include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/strcat.h"
+#include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/gutil/walltime.h"
@@ -34,6 +37,7 @@
 #include "kudu/util/random.h"
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/spinlock_profiling.h"
+#include "kudu/util/subprocess.h"
 
 DEFINE_string(test_leave_files, "on_failure",
               "Whether to leave test files around after the test run. "
@@ -293,4 +297,95 @@ int CountOpenFds(Env* env) {
   return num_fds - 1;
 }
 
+Status GetExecutablePath(const string& binary,
+                         const vector<string>& search,
+                         string* path) {
+  string p;
+
+  // First, check specified locations. This is necessary to check first so that
+  // the system binaries won't be found before the specified search locations.
+  for (const auto& location : search) {
+    p = JoinPathSegments(location, binary);
+    if (Env::Default()->FileExists(p)) {
+      *path = p;
+      return Status::OK();
+    }
+  }
+
+  // Next check if the binary is on the PATH.
+  Status s = Subprocess::Call({ "which", binary }, "", &p);
+  if (s.ok()) {
+    StripTrailingNewline(&p);
+    *path = p;
+    return Status::OK();
+  }
+
+  return Status::NotFound("Unable to find binary", binary);
+}
+
+namespace {
+Status WaitForBind(pid_t pid, uint16_t* port, const char* kind, MonoDelta timeout) {
+  // 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
+  // straight-forward. We call lsof in a loop since it typically takes a long
+  // time for it to initialize and bind a port.
+
+  string lsof;
+  RETURN_NOT_OK(GetExecutablePath("lsof", {"/sbin", "/usr/sbin"}, &lsof));
+
+  const vector<string> cmd = {
+    lsof, "-wbnP", "-Ffn",
+    "-p", std::to_string(pid),
+    "-a", "-i", kind
+  };
+
+  MonoTime deadline = MonoTime::Now() + timeout;
+  string lsof_out;
+
+  for (int64_t i = 1; ; i++) {
+    lsof_out.clear();
+    Status s = Subprocess::Call(cmd, "", &lsof_out);
+
+    if (s.ok()) {
+      StripTrailingNewline(&lsof_out);
+      break;
+    }
+    if (deadline < MonoTime::Now()) {
+      return s;
+    }
+
+    SleepFor(MonoDelta::FromMilliseconds(i * 10));
+  }
+
+  // The '-Ffn' flag gets lsof to output something like:
+  //   p19730
+  //   f123
+  //   n*:41254
+  // The first line is the pid. We ignore it.
+  // The second line is the file descriptor number. We ignore it.
+  // The third line has the bind address and port.
+  vector<string> lines = strings::Split(lsof_out, "\n");
+  int32_t p = -1;
+  if (lines.size() != 3 ||
+      lines[2].substr(0, 3) != "n*:" ||
+      !safe_strto32(lines[2].substr(3), &p) ||
+      p <= 0) {
+    return Status::RuntimeError("unexpected lsof output", lsof_out);
+  }
+  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();
+}
+} // anonymous namespace
+
+Status WaitForTcpBind(pid_t pid, uint16_t* port, MonoDelta timeout) {
+  return WaitForBind(pid, port, "4TCP", timeout);
+}
+
+Status WaitForUdpBind(pid_t pid, uint16_t* port, MonoDelta timeout) {
+  return WaitForBind(pid, port, "4UDP", timeout);
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/5b18be1a/src/kudu/util/test_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_util.h b/src/kudu/util/test_util.h
index 061fb3a..1eb8486 100644
--- a/src/kudu/util/test_util.h
+++ b/src/kudu/util/test_util.h
@@ -22,6 +22,7 @@
 #include <functional>
 #include <gtest/gtest.h>
 #include <string>
+#include <vector>
 
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/util/env.h"
@@ -109,5 +110,17 @@ void AssertEventually(const std::function<void(void)>& f,
 // Count the number of open file descriptors in use by this process.
 int CountOpenFds(Env* env);
 
+// Attempts to find the path to the executable, searching the provided locations
+// as well as the $PATH environment variable.
+Status GetExecutablePath(const std::string& binary,
+                         const std::vector<std::string>& search,
+                         std::string* path) WARN_UNUSED_RESULT;
+
+// Waits for the subprocess to bind to any listening TCP port, and returns the port.
+Status WaitForTcpBind(pid_t pid, uint16_t* port, MonoDelta timeout) WARN_UNUSED_RESULT;
+
+// Waits for the subprocess to bind to any listening UDP port, and returns the port.
+Status WaitForUdpBind(pid_t pid, uint16_t* port, MonoDelta timeout) WARN_UNUSED_RESULT;
+
 } // namespace kudu
 #endif