This is an automated email from the ASF dual-hosted git repository.
alexey pushed a commit to branch branch-1.9.x
in repository https://gitbox.apache.org/repos/asf/kudu.git
commit 4e31b964c99b584dfe35bb6b416c481ce8280501
Author: Alexey Serbin <alexey@apache.org>
AuthorDate: Tue Mar 12 19:51:13 2019 -0700
KUDU-2743 [subprocess] use RAW_LOG() in child process
For a newly fork()-ed, but not yet exec()-ed process, only async-signal
safe functions should be called. At least, glog's LOG() should not be
called since it involves acquiring a lock while flushing the message
into the log sink. Otherwise, the child process can deadlock on attempt
to write a log message if glog's mutex has been copied from the parent's
address space in a locked state.
Change-Id: Ic9dca4ca8b1a6d72c9fc818ea41109c80ace3e39
Reviewed-on: http://gerrit.cloudera.org:8080/12739
Reviewed-by: Adar Dembo <adar@cloudera.com>
Tested-by: Alexey Serbin <aserbin@cloudera.com>
(cherry picked from commit d56930fbcdceff911efc8ec30b5c769901888d3d)
(cherry picked from commit 3270d67669e5cf0038fad655178c8c28082425d4)
Reviewed-on: http://gerrit.cloudera.org:8080/12787
Tested-by: Kudu Jenkins
Reviewed-by: Andrew Wong <awong@cloudera.com>
---
src/kudu/util/signal.cc | 28 ++++++++++++++---
src/kudu/util/subprocess-test.cc | 4 ++-
src/kudu/util/subprocess.cc | 66 +++++++++++++++++++++++++++++++---------
3 files changed, 78 insertions(+), 20 deletions(-)
diff --git a/src/kudu/util/signal.cc b/src/kudu/util/signal.cc
index e8b6e79..37e78f6 100644
--- a/src/kudu/util/signal.cc
+++ b/src/kudu/util/signal.cc
@@ -17,16 +17,28 @@
#include "kudu/util/signal.h"
-#include <glog/logging.h>
+#include <cerrno>
+
+#include <glog/logging.h> // IWYU pragma: keep
+#include <glog/raw_logging.h>
namespace kudu {
+// Using RAW_LOG() and logging directly into stderr instead of using more
+// advanced LOG() to keep all these function as async-signal-safe as possible.
+//
+// NOTE: RAW_LOG() uses vsnprintf(), and it's not async-signal-safe
+// strictly speaking (in some cases may call malloc(), etc.)
+
void SetSignalHandler(int signal, SignalHandlerCallback handler) {
struct sigaction act;
act.sa_handler = handler;
sigemptyset(&act.sa_mask);
act.sa_flags = 0;
- PCHECK(sigaction(signal, &act, nullptr) == 0);
+ if (sigaction(signal, &act, nullptr) != 0) {
+ int err = errno;
+ RAW_LOG(FATAL, "sigaction() failed: [%d]", err);
+ }
}
void IgnoreSigPipe() {
@@ -39,9 +51,15 @@ void ResetSigPipeHandlerToDefault() {
// We unblock all signal masks since they are inherited.
void ResetAllSignalMasksToUnblocked() {
- sigset_t signals;
- PCHECK(sigfillset(&signals) == 0);
- PCHECK(sigprocmask(SIG_UNBLOCK, &signals, nullptr) == 0);
+ sigset_t sigset;
+ if (sigfillset(&sigset) != 0) {
+ int err = errno;
+ RAW_LOG(FATAL, "sigfillset() failed: [%d]", err);
+ }
+ if (sigprocmask(SIG_UNBLOCK, &sigset, nullptr) != 0) {
+ int err = errno;
+ RAW_LOG(FATAL, "sigprocmask() failed: [%d]", err);
+ }
}
} // namespace kudu
diff --git a/src/kudu/util/subprocess-test.cc b/src/kudu/util/subprocess-test.cc
index a9b7aac..99c2994 100644
--- a/src/kudu/util/subprocess-test.cc
+++ b/src/kudu/util/subprocess-test.cc
@@ -353,7 +353,9 @@ TEST_F(SubprocessTest, TestSubprocessInterruptionHandling) {
}
}
-TEST_F(SubprocessTest, DISABLED_TestSubprocessDeadlockOnLogging) {
+// Test to make sure the spawned child process does not deadlock while running
+// the code between fork() and exec().
+TEST_F(SubprocessTest, TestSubprocessDeadlockOnLogging) {
int kNumLoggingThreads = 8;
// Participants are the logging threads and the main test thread.
Barrier barrier(kNumLoggingThreads + 1);
diff --git a/src/kudu/util/subprocess.cc b/src/kudu/util/subprocess.cc
index d68cb7f..f992ba4 100644
--- a/src/kudu/util/subprocess.cc
+++ b/src/kudu/util/subprocess.cc
@@ -39,6 +39,7 @@
#include <ev++.h>
#include <glog/logging.h>
+#include <glog/raw_logging.h>
#include <glog/stl_logging.h>
#include "kudu/gutil/basictypes.h"
@@ -126,7 +127,7 @@ void CloseNonStandardFDs(DIR* fd_dir) {
// make it as lean and mean as possible -- this runs in the subprocess
// after a fork, so there's some possibility that various global locks
// inside malloc() might be held, so allocating memory is a no-no.
- PCHECK(fd_dir != nullptr);
+ RAW_CHECK(fd_dir != nullptr, "fd_dir is null");
int dir_fd = dirfd(fd_dir);
struct DIRENT* ent;
@@ -165,12 +166,16 @@ void RedirectToDevNull(int fd) {
// file descriptors later on.
int dev_null;
RETRY_ON_EINTR(dev_null, open("/dev/null", O_WRONLY));
- if (dev_null < 0) {
- PLOG(WARNING) << "failed to open /dev/null";
+ if (dev_null == -1) {
+ int err = errno;
+ RAW_LOG(WARNING, "failed to open /dev/null: [%d]", err);
} else {
int ret;
RETRY_ON_EINTR(ret, dup2(dev_null, fd));
- PCHECK(ret);
+ if (ret == -1) {
+ int err = errno;
+ RAW_LOG(FATAL, "dup2() on /dev/null failed: [%d]", err);
+ }
}
}
@@ -369,6 +374,23 @@ Status Subprocess::Start() {
return Status::RuntimeError("Unable to fork", ErrnoToString(errno), errno);
}
if (ret == 0) { // We are the child
+ // As a general note, it's not safe to call non-async-signal-safe functions
+ // in the child process between fork() and exec(). Surprisingly, a call to
+ // LOG() locks a mutex that may have been copied from the parent's address
+ // space in an already locked state, so it is not async-signal-safe and
+ // can deadlock the child if called. So, in this vulnerable state the child
+ // outputs log messages using RAW_LOG() instead directly into stderr.
+ // RAW_LOG() uses vsnprintf() under the hood: it's not async-signal-safe
+ // strictly speaking (might call malloc() and getenv() in some cases which
+ // might acquire locks themselves), but it's much better than using LOG()
+ // where it can simply deadlock on glog's mutex. BTW, some allocators like
+ // tcmalloc install pthread_atfork() handlers, so with tcmalloc we have
+ // more safety with vsnprintf().
+ //
+ // An alternative approach might be to use some additional functionality
+ // in glog library (once implemented) to establish thread_atfork() handlers;
+ // see https://github.com/robi56/google-glog/issues/101 for details.
+
// Send the child a SIGTERM when the parent dies. This is done as early
// as possible in the child's life to prevent any orphaning whatsoever
// (e.g. from KUDU-402).
@@ -382,9 +404,13 @@ Status Subprocess::Start() {
if (fd_state_[STDIN_FILENO] == PIPED) {
int dup2_ret;
RETRY_ON_EINTR(dup2_ret, dup2(child_stdin[0], STDIN_FILENO));
- PCHECK(dup2_ret == STDIN_FILENO);
+ if (dup2_ret != STDIN_FILENO) {
+ int err = errno;
+ RAW_LOG(FATAL, "dup2() failed (STDIN): [%d]", err);
+ }
} else {
- DCHECK_EQ(SHARED, fd_state_[STDIN_FILENO]);
+ RAW_DCHECK(SHARED == fd_state_[STDIN_FILENO],
+ "unexpected state of STDIN");
}
// stdout
@@ -392,7 +418,10 @@ Status Subprocess::Start() {
case PIPED: {
int dup2_ret;
RETRY_ON_EINTR(dup2_ret, dup2(child_stdout[1], STDOUT_FILENO));
- PCHECK(dup2_ret == STDOUT_FILENO);
+ if (dup2_ret != STDOUT_FILENO) {
+ int err = errno;
+ RAW_LOG(FATAL, "dup2() failed (STDOUT): [%d]", err);
+ }
break;
}
case DISABLED: {
@@ -400,7 +429,8 @@ Status Subprocess::Start() {
break;
}
default:
- DCHECK_EQ(SHARED, fd_state_[STDOUT_FILENO]);
+ RAW_DCHECK(SHARED == fd_state_[STDOUT_FILENO],
+ "unexpected state of STDOUT");
break;
}
@@ -409,7 +439,10 @@ Status Subprocess::Start() {
case PIPED: {
int dup2_ret;
RETRY_ON_EINTR(dup2_ret, dup2(child_stderr[1], STDERR_FILENO));
- PCHECK(dup2_ret == STDERR_FILENO);
+ if (dup2_ret != STDERR_FILENO) {
+ int err = errno;
+ RAW_LOG(FATAL, "dup2() failed (STDERR): [%d]", err);
+ }
break;
}
case DISABLED: {
@@ -417,7 +450,8 @@ Status Subprocess::Start() {
break;
}
default:
- DCHECK_EQ(SHARED, fd_state_[STDERR_FILENO]);
+ RAW_DCHECK(SHARED == fd_state_[STDERR_FILENO],
+ "unexpected state of STDERR");
break;
}
@@ -425,7 +459,10 @@ Status Subprocess::Start() {
// the write side should be closed upon execvp().
int close_ret;
RETRY_ON_EINTR(close_ret, close(sync_pipe[0]));
- PCHECK(close_ret == 0);
+ if (close_ret == -1) {
+ int err = errno;
+ RAW_LOG(FATAL, "close() on the read side of sync pipe failed: [%d]", err);
+ }
CloseNonStandardFDs(fd_dir);
@@ -438,8 +475,9 @@ Status Subprocess::Start() {
ResetSigPipeHandlerToDefault();
// Set the current working directory of the subprocess.
- if (!cwd_.empty()) {
- PCHECK(chdir(cwd_.c_str()) == 0);
+ if (!cwd_.empty() && chdir(cwd_.c_str()) == -1) {
+ int err = errno;
+ RAW_LOG(FATAL, "chdir() to '%s' failed: [%d]", cwd_.c_str(), err);
}
// Set the environment for the subprocess. This is more portable than
@@ -452,7 +490,7 @@ Status Subprocess::Start() {
execvp(program_.c_str(), &argv_ptrs[0]);
int err = errno;
- PLOG(ERROR) << "Couldn't exec " << program_;
+ RAW_LOG(ERROR, "could not exec '%s': [%d]", program_.c_str(), err);
_exit(err);
} else {
// We are the parent
|