You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2016/12/15 01:54:40 UTC

kudu git commit: external_mini_cluster: check for leaks before killing servers

Repository: kudu
Updated Branches:
  refs/heads/master a38977ec5 -> 94719b558


external_mini_cluster: check for leaks before killing servers

This improves the ExternalMiniCluster when running with LeakSanitizer
enabled so that, before killing subprocesses, it runs a leak check.
Without this, it's hard for us to know if there are any leaks in code
paths that are covered only by EMC-based tests.

Tested by manually inserting a leak into the tablet server and verifying
that an EMC test failed.

Change-Id: I9b6c38281e35e8300e849f92e91d6955a8d735bf
Reviewed-on: http://gerrit.cloudera.org:8080/5501
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: 94719b558357a692615d86e5daf180702ef188a7
Parents: a38977e
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Dec 14 16:07:39 2016 +0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Dec 15 01:54:09 2016 +0000

----------------------------------------------------------------------
 .../integration-tests/external_mini_cluster.cc  | 42 ++++++++++++++++----
 .../integration-tests/external_mini_cluster.h   |  5 +++
 src/kudu/server/generic_service.cc              | 24 ++++++++++-
 src/kudu/server/generic_service.h               |  4 ++
 src/kudu/server/server_base.proto               | 12 ++++++
 src/kudu/util/debug/leak_annotations.h          | 24 ++++++++---
 6 files changed, 97 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/94719b55/src/kudu/integration-tests/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.cc b/src/kudu/integration-tests/external_mini_cluster.cc
index ff26d1b..1497ad2 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -725,12 +725,14 @@ void ExternalDaemon::SetExePath(string exe) {
 Status ExternalDaemon::Pause() {
   if (!process_) return Status::OK();
   VLOG(1) << "Pausing " << exe_ << " with pid " << process_->pid();
+  paused_ = true;
   return process_->Kill(SIGSTOP);
 }
 
 Status ExternalDaemon::Resume() {
   if (!process_) return Status::OK();
   VLOG(1) << "Resuming " << exe_ << " with pid " << process_->pid();
+  paused_ = false;
   return process_->Kill(SIGCONT);
 }
 
@@ -815,15 +817,20 @@ void ExternalDaemon::Shutdown() {
   }
 
   if (IsProcessAlive()) {
-    // In coverage builds, ask the process nicely to flush coverage info
-    // before we kill -9 it. Otherwise, we never get any coverage from
-    // external clusters.
-    FlushCoverage();
+    if (!paused_) {
+      // In coverage builds, ask the process nicely to flush coverage info
+      // before we kill -9 it. Otherwise, we never get any coverage from
+      // external clusters.
+      FlushCoverage();
+      // Similarly, check for leaks in LSAN builds before killing.
+      CheckForLeaks();
+    }
 
     LOG(INFO) << "Killing " << exe_ << " with pid " << process_->pid();
     ignore_result(process_->Kill(SIGKILL));
   }
   WARN_NOT_OK(process_->Wait(), "Waiting on " + exe_);
+  paused_ = false;
   process_.reset();
 }
 
@@ -838,9 +845,7 @@ void ExternalDaemon::FlushCoverage() {
   server::FlushCoverageResponsePB resp;
   rpc::RpcController rpc;
 
-  // Set a reasonably short timeout, since some of our tests kill servers which
-  // are kill -STOPed.
-  rpc.set_timeout(MonoDelta::FromMilliseconds(100));
+  rpc.set_timeout(MonoDelta::FromMilliseconds(1000));
   Status s = proxy.FlushCoverage(req, &resp, &rpc);
   if (s.ok() && !resp.success()) {
     s = Status::RemoteError("Server does not appear to be running a coverage build");
@@ -849,6 +854,29 @@ void ExternalDaemon::FlushCoverage() {
 #endif
 }
 
+void ExternalDaemon::CheckForLeaks() {
+#if defined(__has_feature)
+#  if __has_feature(address_sanitizer)
+  LOG(INFO) << "Attempting to check leaks for " << exe_ << " pid " << process_->pid();
+  server::GenericServiceProxy proxy(messenger_, bound_rpc_addr());
+  server::CheckLeaksRequestPB req;
+  server::CheckLeaksResponsePB resp;
+  rpc::RpcController rpc;
+
+  rpc.set_timeout(MonoDelta::FromMilliseconds(1000));
+  Status s = proxy.CheckLeaks(req, &resp, &rpc);
+  if (s.ok()) {
+    if (!resp.success()) {
+      s = Status::RemoteError("Server does not appear to be running an LSAN build");
+    } else {
+      CHECK(!resp.found_leaks()) << "Found leaks in " << exe_ << " pid " << process_->pid();
+    }
+  }
+  WARN_NOT_OK(s, Substitute("Unable to check leaks on $0 pid $1", exe_, process_->pid()));
+#  endif
+#endif
+}
+
 HostPort ExternalDaemon::bound_rpc_hostport() const {
   CHECK(status_);
   CHECK_GE(status_->bound_rpc_addresses_size(), 1);

http://git-wip-us.apache.org/repos/asf/kudu/blob/94719b55/src/kudu/integration-tests/external_mini_cluster.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.h b/src/kudu/integration-tests/external_mini_cluster.h
index 48979f0..287d57d 100644
--- a/src/kudu/integration-tests/external_mini_cluster.h
+++ b/src/kudu/integration-tests/external_mini_cluster.h
@@ -436,6 +436,10 @@ class ExternalDaemon : public RefCountedThreadSafe<ExternalDaemon> {
   // In a non-coverage build, this does nothing.
   void FlushCoverage();
 
+  // In an LSAN build, ask the daemon to check for leaked memory, and
+  // LOG(FATAL) if there are any leaks.
+  void CheckForLeaks();
+
   // Get/Set rpc_bind_addresses for daemon.
   virtual const std::string& get_rpc_bind_address() const {
     return rpc_bind_address_;
@@ -452,6 +456,7 @@ class ExternalDaemon : public RefCountedThreadSafe<ExternalDaemon> {
   std::map<std::string, std::string> extra_env_;
 
   gscoped_ptr<Subprocess> process_;
+  bool paused_ = false;
 
   gscoped_ptr<server::ServerStatusPB> status_;
   std::string rpc_bind_address_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/94719b55/src/kudu/server/generic_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/generic_service.cc b/src/kudu/server/generic_service.cc
index 466f5e1..42cb88e 100644
--- a/src/kudu/server/generic_service.cc
+++ b/src/kudu/server/generic_service.cc
@@ -27,6 +27,7 @@
 #include "kudu/server/hybrid_clock.h"
 #include "kudu/server/server_base.h"
 #include "kudu/util/debug-util.h"
+#include "kudu/util/debug/leak_annotations.h"
 #include "kudu/util/flag_tags.h"
 
 DECLARE_bool(use_mock_wall_clock);
@@ -94,7 +95,28 @@ void GenericServiceImpl::SetFlag(const SetFlagRequestPB* req,
   rpc->RespondSuccess();
 }
 
-void GenericServiceImpl::FlushCoverage(const FlushCoverageRequestPB* req,
+void GenericServiceImpl::CheckLeaks(const CheckLeaksRequestPB* /*req*/,
+                                    CheckLeaksResponsePB* resp,
+                                    rpc::RpcContext* rpc) {
+  // We have to use these nested #if statements rather than an && to avoid
+  // a preprocessor error with GCC which doesn't know about __has_feature.
+#if defined(__has_feature)
+#  if __has_feature(address_sanitizer)
+#    define LSAN_ENABLED
+#  endif
+#endif
+#ifndef LSAN_ENABLED
+  resp->set_success(false);
+#else
+  LOG(INFO) << "Checking for leaks (request via RPC)";
+  resp->set_success(true);
+  resp->set_found_leaks(__lsan_do_recoverable_leak_check());
+#endif
+#undef LSAN_ENABLED
+  rpc->RespondSuccess();
+}
+
+void GenericServiceImpl::FlushCoverage(const FlushCoverageRequestPB* /*req*/,
                                        FlushCoverageResponsePB* resp,
                                        rpc::RpcContext* rpc) {
   if (IsCoverageBuild()) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/94719b55/src/kudu/server/generic_service.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/generic_service.h b/src/kudu/server/generic_service.h
index 74c4994..1ef5423 100644
--- a/src/kudu/server/generic_service.h
+++ b/src/kudu/server/generic_service.h
@@ -38,6 +38,10 @@ class GenericServiceImpl : public GenericServiceIf {
                              FlushCoverageResponsePB* resp,
                              rpc::RpcContext* rpc) OVERRIDE;
 
+  virtual void CheckLeaks(const CheckLeaksRequestPB* req,
+                          CheckLeaksResponsePB* resp,
+                          rpc::RpcContext* rpc) OVERRIDE;
+
   virtual void ServerClock(const ServerClockRequestPB* req,
                            ServerClockResponsePB* resp,
                            rpc::RpcContext* rpc) OVERRIDE;

http://git-wip-us.apache.org/repos/asf/kudu/blob/94719b55/src/kudu/server/server_base.proto
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.proto b/src/kudu/server/server_base.proto
index 6ad9f48..ed031fc 100644
--- a/src/kudu/server/server_base.proto
+++ b/src/kudu/server/server_base.proto
@@ -84,6 +84,15 @@ message FlushCoverageResponsePB {
   optional bool success = 1;
 }
 
+// Run leak checks in an LSAN-instrumented build.
+message CheckLeaksRequestPB {
+}
+message CheckLeaksResponsePB {
+  // If the current build is not an LSAN-instrumented build, returns false.
+  optional bool success = 1;
+  optional bool found_leaks = 2;
+}
+
 // Requests the server's current timestamp.
 message ServerClockRequestPB {
 }
@@ -120,6 +129,9 @@ service GenericService {
   rpc FlushCoverage(FlushCoverageRequestPB)
     returns (FlushCoverageResponsePB);
 
+  rpc CheckLeaks(CheckLeaksRequestPB)
+    returns (CheckLeaksResponsePB);
+
   rpc ServerClock(ServerClockRequestPB)
     returns (ServerClockResponsePB);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/94719b55/src/kudu/util/debug/leak_annotations.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/debug/leak_annotations.h b/src/kudu/util/debug/leak_annotations.h
index c13b7e8..642aa6a 100644
--- a/src/kudu/util/debug/leak_annotations.h
+++ b/src/kudu/util/debug/leak_annotations.h
@@ -31,13 +31,25 @@ extern "C" {
   // function must be defined as returning a constant value; any behavior beyond
   // that is unsupported.
   int __lsan_is_turned_off();
-  // Calling this function makes LSan enter the leak checking phase immediately.
-  // Use this if normal end-of-process leak checking happens too late (e.g. if
-  // you have intentional memory leaks in your shutdown code). Calling this
-  // function overrides end-of-process leak checking; it must be called at
-  // most once per process. This function will terminate the process if there
-  // are memory leaks and the exit_code flag is non-zero.
+
+  // Check for leaks now. This function behaves identically to the default
+  // end-of-process leak check. In particular, it will terminate the process if
+  // leaks are found and the exitcode runtime flag is non-zero.
+  // Subsequent calls to this function will have no effect and end-of-process
+  // leak check will not run. Effectively, end-of-process leak check is moved to
+  // the time of first invocation of this function.
+  // By calling this function early during process shutdown, you can instruct
+  // LSan to ignore shutdown-only leaks which happen later on.
   void __lsan_do_leak_check();
+
+  // Check for leaks now. Returns zero if no leaks have been found or if leak
+  // detection is disabled, non-zero otherwise.
+  // This function may be called repeatedly, e.g. to periodically check a
+  // long-running process. It prints a leak report if appropriate, but does not
+  // terminate the process. It does not affect the behavior of
+  // __lsan_do_leak_check() or the end-of-process leak check, and is not
+  // affected by them.
+  int __lsan_do_recoverable_leak_check();
 }  // extern "C"
 
 namespace kudu {