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/10/19 22:16:05 UTC

[2/7] kudu git commit: [util] added Subprocess::GetExitStatus()

[util] added Subprocess::GetExitStatus()

The Subprocess::GetExitStatus() method allows to retrive exit status
and exit information in printable form.  That facilitates reporting
proper exit status for a sub-processes if using Subprocess::Wait().

The motivation for this change was to free the users of the Subprocess
interface of calling WIFEXITED/WEXITSTATUS and friends, duplicating
that code over and over.  In some cases, callers did not use
WEXITSTATUS or WTERMSIG and mistakenly interpreted wait status
as sub-process exit status.  In short, most users of the Subprocess
interface need exit status of a sub-process, not wait status.

Also, addressed the race when a signal arrives to the child process
before it calls execve().  Todd suggested to use the trick with
a pipe and O_CLOEXEC option.

Change-Id: Ic2b16e2a2a53a01982f816b9ee41cc61fd93d4bf
Reviewed-on: http://gerrit.cloudera.org:8080/4648
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-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/bd56e021
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/bd56e021
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/bd56e021

Branch: refs/heads/master
Commit: bd56e021c401fcc0d08ae52d1ba0800dc9bc32d3
Parents: 0b0cc92
Author: Alexey Serbin <as...@cloudera.com>
Authored: Thu Oct 6 02:24:24 2016 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Oct 19 00:05:59 2016 +0000

----------------------------------------------------------------------
 src/kudu/benchmarks/tpch/tpch_real_world.cc     |  16 +-
 .../integration-tests/external_mini_cluster.cc  |  16 +-
 .../full_stack-insert-scan-test.cc              |  12 +-
 src/kudu/util/pstack_watcher.cc                 |  24 ++-
 src/kudu/util/subprocess-test.cc                |  55 ++++-
 src/kudu/util/subprocess.cc                     | 205 ++++++++++++++-----
 src/kudu/util/subprocess.h                      |  30 +--
 7 files changed, 257 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/bd56e021/src/kudu/benchmarks/tpch/tpch_real_world.cc
----------------------------------------------------------------------
diff --git a/src/kudu/benchmarks/tpch/tpch_real_world.cc b/src/kudu/benchmarks/tpch/tpch_real_world.cc
index c9f03c6..562a6fb 100644
--- a/src/kudu/benchmarks/tpch/tpch_real_world.cc
+++ b/src/kudu/benchmarks/tpch/tpch_real_world.cc
@@ -290,22 +290,26 @@ void TpchRealWorld::LoadLineItemsThread(int i) {
 void TpchRealWorld::MonitorDbgenThread(int i) {
   Subprocess* dbgen_proc = dbgen_processes_[i];
   while (!stop_threads_.Load()) {
-    int ret;
-    Status s = dbgen_proc->WaitNoBlock(&ret);
+    Status s = dbgen_proc->WaitNoBlock();
     if (s.ok()) {
-      CHECK(ret == 0) << "dbgen exited with a non-zero return code: " << ret;
+      int exit_status;
+      string exit_info;
+      CHECK_OK(dbgen_proc->GetExitStatus(&exit_status, &exit_info));
+      if (exit_status != 0) {
+        LOG(FATAL) << exit_info;
+      }
       LOG(INFO) << "dbgen finished inserting data";
       dbgen_processes_finished_.CountDown();
       return;
-    } else {
-      SleepFor(MonoDelta::FromMilliseconds(100));
     }
+    CHECK(s.IsTimedOut()) << "Unexpected wait status: " << s.ToString();
+    SleepFor(MonoDelta::FromMilliseconds(100));
   }
   Status s = dbgen_proc->Kill(SIGKILL);
   if (!s.ok()) {
     LOG(FATAL) << "Failed to send SIGKILL to dbgen: " << s.ToString();
   }
-  s = dbgen_proc->Wait(nullptr);
+  s = dbgen_proc->Wait();
   if (!s.ok()) {
     LOG(FATAL) << "Failed to await for dbgen exit: " << s.ToString();
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/bd56e021/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 9786f59..63eaff6 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -607,16 +607,15 @@ Status ExternalDaemon::StartProcess(const vector<string>& user_flags) {
       break;
     }
     SleepFor(MonoDelta::FromMilliseconds(10));
-    int rc;
-    Status s = p->WaitNoBlock(&rc);
+    Status s = p->WaitNoBlock();
     if (s.IsTimedOut()) {
       // The process is still running.
       continue;
     }
     RETURN_NOT_OK_PREPEND(s, Substitute("Failed waiting on $0", exe_));
-    return Status::RuntimeError(
-      Substitute("Process exited with rc=$0", rc),
-      exe_);
+    string exit_info;
+    RETURN_NOT_OK(p->GetExitStatus(nullptr, &exit_info));
+    return Status::RuntimeError(exit_info);
   }
 
   if (!success) {
@@ -661,9 +660,7 @@ bool ExternalDaemon::IsProcessAlive() const {
   if (IsShutdown()) {
     return false;
   }
-
-  int rc = 0;
-  Status s = process_->WaitNoBlock(&rc);
+  Status s = process_->WaitNoBlock();
   // If the non-blocking Wait "times out", that means the process
   // is running.
   return s.IsTimedOut();
@@ -702,8 +699,7 @@ void ExternalDaemon::Shutdown() {
     LOG(INFO) << "Killing " << exe_ << " with pid " << process_->pid();
     ignore_result(process_->Kill(SIGKILL));
   }
-  int ret;
-  WARN_NOT_OK(process_->Wait(&ret), "Waiting on " + exe_);
+  WARN_NOT_OK(process_->Wait(), "Waiting on " + exe_);
   process_.reset();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/bd56e021/src/kudu/integration-tests/full_stack-insert-scan-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/full_stack-insert-scan-test.cc b/src/kudu/integration-tests/full_stack-insert-scan-test.cc
index dc2e3dc..11d3d40 100644
--- a/src/kudu/integration-tests/full_stack-insert-scan-test.cc
+++ b/src/kudu/integration-tests/full_stack-insert-scan-test.cc
@@ -231,12 +231,14 @@ gscoped_ptr<Subprocess> MakePerfRecord() {
 
 void InterruptNotNull(gscoped_ptr<Subprocess> sub) {
   if (!sub) return;
+
   ASSERT_OK(sub->Kill(SIGINT));
-  int exit_status = 0;
-  ASSERT_OK(sub->Wait(&exit_status));
-  if (!exit_status) {
-    LOG(WARNING) << "Subprocess returned " << exit_status
-                 << ": " << ErrnoToString(exit_status);
+  ASSERT_OK(sub->Wait());
+  int exit_status;
+  string exit_info_str;
+  ASSERT_OK(sub->GetExitStatus(&exit_status, &exit_info_str));
+  if (exit_status != 0) {
+    LOG(WARNING) << exit_info_str;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/bd56e021/src/kudu/util/pstack_watcher.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/pstack_watcher.cc b/src/kudu/util/pstack_watcher.cc
index c769f07..ec24a11 100644
--- a/src/kudu/util/pstack_watcher.cc
+++ b/src/kudu/util/pstack_watcher.cc
@@ -92,15 +92,14 @@ Status PstackWatcher::HasProgram(const char* progname) {
   proc.DisableStdout();
   RETURN_NOT_OK_PREPEND(proc.Start(),
       Substitute("HasProgram($0): error running 'which'", progname));
-  int wait_status = 0;
-  RETURN_NOT_OK(proc.Wait(&wait_status));
-  if ((WIFEXITED(wait_status)) && (0 == WEXITSTATUS(wait_status))) {
+  RETURN_NOT_OK(proc.Wait());
+  int exit_status;
+  string exit_info;
+  RETURN_NOT_OK(proc.GetExitStatus(&exit_status, &exit_info));
+  if (exit_status == 0) {
     return Status::OK();
   }
-  return Status::NotFound(Substitute("can't find $0: exited?=$1, status=$2",
-                                     progname,
-                                     static_cast<bool>(WIFEXITED(wait_status)),
-                                     WEXITSTATUS(wait_status)));
+  return Status::NotFound(Substitute("can't find $0: $1", progname, exit_info));
 }
 
 Status PstackWatcher::DumpStacks(int flags) {
@@ -173,10 +172,13 @@ Status PstackWatcher::RunStackDump(const string& prog, const vector<string>& arg
   if (::close(pstack_proc.ReleaseChildStdinFd()) == -1) {
     return Status::IOError("Unable to close child stdin", ErrnoToString(errno), errno);
   }
-  int ret;
-  RETURN_NOT_OK_PREPEND(pstack_proc.Wait(&ret), "RunStackDump proc.Wait() failed");
-  if (ret == -1) {
-    return Status::RuntimeError("RunStackDump proc.Wait() error", ErrnoToString(errno), errno);
+  RETURN_NOT_OK_PREPEND(pstack_proc.Wait(), "RunStackDump proc.Wait() failed");
+  int exit_code;
+  string exit_info;
+  RETURN_NOT_OK_PREPEND(pstack_proc.GetExitStatus(&exit_code, &exit_info),
+                        "RunStackDump proc.GetExitStatus() failed");
+  if (exit_code != 0) {
+    return Status::RuntimeError("RunStackDump proc.Wait() error", exit_info);
   }
   printf("************************* END STACKS ***************************\n");
   if (fflush(stdout) == EOF) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/bd56e021/src/kudu/util/subprocess-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/subprocess-test.cc b/src/kudu/util/subprocess-test.cc
index a1da506..76778c6 100644
--- a/src/kudu/util/subprocess-test.cc
+++ b/src/kudu/util/subprocess-test.cc
@@ -29,6 +29,7 @@
 
 using std::string;
 using std::vector;
+using strings::Substitute;
 
 namespace kudu {
 
@@ -100,6 +101,7 @@ TEST_F(SubprocessTest, TestKill) {
 
   int wait_status = 0;
   ASSERT_OK(p.Wait(&wait_status));
+  ASSERT_TRUE(WIFSIGNALED(wait_status));
   ASSERT_EQ(SIGKILL, WTERMSIG(wait_status));
 
   // Test that calling Wait() a second time returns the same
@@ -107,6 +109,7 @@ TEST_F(SubprocessTest, TestKill) {
   // that was assigned the same pid.
   wait_status = 0;
   ASSERT_OK(p.Wait(&wait_status));
+  ASSERT_TRUE(WIFSIGNALED(wait_status));
   ASSERT_EQ(SIGKILL, WTERMSIG(wait_status));
 }
 
@@ -135,7 +138,7 @@ TEST_F(SubprocessTest, TestReadFromStdoutAndStderr) {
 TEST_F(SubprocessTest, TestReadSingleFD) {
   string stderr;
   const string str = "ApacheKudu";
-  const string cmd_str = strings::Substitute("/bin/echo -n $0 1>&2", str);
+  const string cmd_str = Substitute("/bin/echo -n $0 1>&2", str);
   ASSERT_OK(Subprocess::Call({"/bin/sh", "-c", cmd_str}, nullptr, &stderr));
   ASSERT_EQ(stderr, str);
 
@@ -147,4 +150,54 @@ TEST_F(SubprocessTest, TestReadSingleFD) {
   ASSERT_OK(Subprocess::Call({"/bin/ls", "/dev/zero"}, nullptr, nullptr));
 }
 
+TEST_F(SubprocessTest, TestGetExitStatusExitSuccess) {
+  Subprocess p("/bin/sh", { "/bin/sh", "-c", "exit 0" });
+  ASSERT_OK(p.Start());
+  ASSERT_OK(p.Wait());
+  int exit_status;
+  string exit_info;
+  ASSERT_OK(p.GetExitStatus(&exit_status, &exit_info));
+  ASSERT_EQ(0, exit_status);
+  ASSERT_STR_CONTAINS(exit_info, "process successfully exited");
+}
+
+TEST_F(SubprocessTest, TestGetExitStatusExitFailure) {
+  static const vector<int> kStatusCodes = { 1, 255 };
+  for (auto code : kStatusCodes) {
+    vector<string> argv = { "/bin/sh", "-c", Substitute("exit $0", code)};
+    Subprocess p("/bin/sh", argv);
+    ASSERT_OK(p.Start());
+    ASSERT_OK(p.Wait());
+    int exit_status;
+    string exit_info;
+    ASSERT_OK(p.GetExitStatus(&exit_status, &exit_info));
+    ASSERT_EQ(code, exit_status);
+    ASSERT_STR_CONTAINS(exit_info,
+                        Substitute("process exited with non-zero status $0",
+                                   exit_status));
+  }
+}
+
+TEST_F(SubprocessTest, TestGetExitStatusSignaled) {
+  static const vector<int> kSignals = {
+    SIGHUP,
+    SIGABRT,
+    SIGKILL,
+    SIGTERM,
+    SIGUSR1,
+  };
+  for (auto signum : kSignals) {
+    Subprocess p("/bin/cat", { "cat" });
+    ASSERT_OK(p.Start());
+    ASSERT_OK(p.Kill(signum));
+    ASSERT_OK(p.Wait());
+    int exit_status;
+    string exit_info;
+    ASSERT_OK(p.GetExitStatus(&exit_status, &exit_info));
+    EXPECT_EQ(signum, exit_status);
+    ASSERT_STR_CONTAINS(exit_info, Substitute("process exited on signal $0",
+                                              signum));
+  }
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/bd56e021/src/kudu/util/subprocess.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/subprocess.cc b/src/kudu/util/subprocess.cc
index db8315f..89854cb 100644
--- a/src/kudu/util/subprocess.cc
+++ b/src/kudu/util/subprocess.cc
@@ -51,6 +51,7 @@ using std::unique_ptr;
 using std::vector;
 using strings::Split;
 using strings::Substitute;
+using strings::SubstituteAndAppend;
 
 namespace kudu {
 
@@ -262,8 +263,7 @@ Subprocess::~Subprocess() {
                  << "(" << JoinStrings(argv_, " ") << ") "
                  << " was orphaned. Sending SIGKILL...";
     WARN_NOT_OK(Kill(SIGKILL), "Failed to send SIGKILL");
-    int junk = 0;
-    WARN_NOT_OK(Wait(&junk), "Failed to Wait()");
+    WARN_NOT_OK(Wait(), "Failed to Wait()");
   }
 
   for (int i = 0; i < 3; ++i) {
@@ -273,12 +273,6 @@ Subprocess::~Subprocess() {
   }
 }
 
-void Subprocess::SetFdShared(int stdfd, bool share) {
-  CHECK_EQ(state_, kNotStarted);
-  CHECK_NE(fd_state_[stdfd], DISABLED);
-  fd_state_[stdfd] = share? SHARED : PIPED;
-}
-
 void Subprocess::DisableStderr() {
   CHECK_EQ(state_, kNotStarted);
   fd_state_[STDERR_FILENO] = DISABLED;
@@ -315,8 +309,9 @@ static int pipe2(int pipefd[2], int flags) {
 
 Status Subprocess::Start() {
   if (state_ != kNotStarted) {
-    return Status::IllegalState(
-        Substitute("$0: illegal sub-process state", state_));
+    const string err_str = Substitute("$0: illegal sub-process state", state_);
+    LOG(DFATAL) << err_str;
+    return Status::IllegalState(err_str);
   }
   if (argv_.empty()) {
     return Status::InvalidArgument("argv must have at least one elem");
@@ -347,6 +342,10 @@ Status Subprocess::Start() {
   if (fd_state_[STDERR_FILENO] == PIPED) {
     PCHECK(pipe2(child_stderr, O_CLOEXEC) == 0);
   }
+  // The synchronization pipe: this trick is to make sure the parent returns
+  // control only after the child process has invoked execve().
+  int sync_pipe[2];
+  PCHECK(pipe2(sync_pipe, O_CLOEXEC) == 0);
 
   DIR* fd_dir = nullptr;
   RETURN_NOT_OK_PREPEND(OpenProcFdDir(&fd_dir), "Unable to open fd dir");
@@ -394,12 +393,16 @@ Status Subprocess::Start() {
     }
     default: break;
     }
+    // Close the read side of the sync pipe;
+    // the write side should be closed upon execvp().
+    PCHECK(close(sync_pipe[0]) == 0);
 
     CloseNonStandardFDs(fd_dir);
 
     execvp(program_.c_str(), &argv_ptrs[0]);
-    PLOG(WARNING) << "Couldn't exec " << program_;
-    _exit(errno);
+    int err = errno;
+    PLOG(ERROR) << "Couldn't exec " << program_;
+    _exit(err);
   } else {
     // We are the parent
     child_pid_ = ret;
@@ -411,48 +414,61 @@ Status Subprocess::Start() {
     child_fds_[STDIN_FILENO]  = child_stdin[1];
     child_fds_[STDOUT_FILENO] = child_stdout[0];
     child_fds_[STDERR_FILENO] = child_stderr[0];
+
+    // Wait for the child process to invoke execve(). The trick involves
+    // a pipe with O_CLOEXEC option for its descriptors. The parent process
+    // performs blocking read from the pipe while the write side of the pipe
+    // is kept open by the child (it does not write any data, though). The write
+    // side of the pipe is closed when the child invokes execvp(). At that
+    // point, the parent should receive EOF, i.e. read() should return 0.
+    {
+      // Close the write side of the sync pipe. It's crucial to make sure
+      // it succeeds otherwise the blocking read() below might wait forever
+      // even if the child process has closed the pipe.
+      PCHECK(close(sync_pipe[1]) == 0);
+      while (true) {
+        uint8_t buf;
+        int err = 0;
+        const int rc = read(sync_pipe[0], &buf, 1);
+        if (rc == -1) {
+          err = errno;
+          if (err == EINTR) {
+            // Retry in case of a signal.
+            continue;
+          }
+        }
+        PCHECK(close(sync_pipe[0]) == 0);
+        if (rc == 0) {
+          // That's OK -- expecting EOF from the other side of the pipe.
+          break;
+        } else if (rc == -1) {
+          // Other errors besides EINTR are not expected.
+          return Status::RuntimeError("Unexpected error from the sync pipe",
+                                      ErrnoToString(err), err);
+        }
+        // No data is expected from the sync pipe.
+        LOG(FATAL) << Substitute("$0: unexpected data from the sync pipe", rc);
+      }
+    }
   }
 
   state_ = kRunning;
   return Status::OK();
 }
 
-Status Subprocess::DoWait(int* ret, int options) {
-  if (state_ == kExited) {
-    if (ret != nullptr) {
-      *ret = cached_rc_;
-    }
-    return Status::OK();
-  }
-  if (state_ != kRunning) {
-    return Status::IllegalState(
-        Substitute("$0: illegal sub-process state", state_));
-  }
-
-  int proc_exit_info;
-  int rc = waitpid(child_pid_, &proc_exit_info, options);
-  if (rc == -1) {
-    return Status::RuntimeError("Unable to wait on child",
-                                ErrnoToString(errno),
-                                errno);
-  }
-  if ((options & WNOHANG) && rc == 0) {
-    return Status::TimedOut("");
-  }
+Status Subprocess::Wait(int* wait_status) {
+  return DoWait(wait_status, BLOCKING);
+}
 
-  CHECK_EQ(rc, child_pid_);
-  child_pid_ = -1;
-  cached_rc_ = proc_exit_info;
-  state_ = kExited;
-  if (ret != nullptr) {
-    *ret = proc_exit_info;
-  }
-  return Status::OK();
+Status Subprocess::WaitNoBlock(int* wait_status) {
+  return DoWait(wait_status, NON_BLOCKING);
 }
 
 Status Subprocess::Kill(int signal) {
   if (state_ != kRunning) {
-    return Status::IllegalState("Sub-process is not running");
+    const string err_str = "Sub-process is not running";
+    LOG(DFATAL) << err_str;
+    return Status::IllegalState(err_str);
   }
   if (kill(child_pid_, signal) != 0) {
     return Status::RuntimeError("Unable to kill",
@@ -462,6 +478,46 @@ Status Subprocess::Kill(int signal) {
   return Status::OK();
 }
 
+Status Subprocess::GetExitStatus(int* exit_status, string* info_str) const {
+  if (state_ != kExited) {
+    const string err_str = "Sub-process termination hasn't yet been detected";
+    LOG(DFATAL) << err_str;
+    return Status::IllegalState(err_str);
+  }
+  string info;
+  int status;
+  if (WIFEXITED(wait_status_)) {
+    status = WEXITSTATUS(wait_status_);
+    if (status == 0) {
+      info = Substitute("$0: process successfully exited", program_);
+    } else {
+      info = Substitute("$0: process exited with non-zero status $1",
+                        program_, status);
+    }
+  } else if (WIFSIGNALED(wait_status_)) {
+    // Using signal number as exit status.
+    status = WTERMSIG(wait_status_);
+    info = Substitute("$0: process exited on signal $1", program_, status);
+#if defined(WCOREDUMP)
+    if (WCOREDUMP(wait_status_)) {
+      SubstituteAndAppend(&info, " (core dumped)");
+    }
+#endif
+  } else {
+    status = -1;
+    info = Substitute("$0: process reported unexpected wait status $1",
+                      program_, wait_status_);
+    LOG(DFATAL) << info;
+  }
+  if (exit_status) {
+    *exit_status = status;
+  }
+  if (info_str) {
+    *info_str = info;
+  }
+  return Status::OK();
+}
+
 Status Subprocess::Call(const string& arg_str) {
   vector<string> argv = Split(arg_str, " ");
   return Call(argv, nullptr, nullptr);
@@ -507,18 +563,62 @@ Status Subprocess::Call(const vector<string>& argv,
     *stderr_out = std::move(outv.back());
   }
 
-  int retcode;
-  RETURN_NOT_OK_PREPEND(p.Wait(&retcode), "Unable to wait() for " + argv[0]);
+  RETURN_NOT_OK_PREPEND(p.Wait(), "Unable to wait() for " + argv[0]);
+  int exit_status;
+  string exit_info_str;
+  RETURN_NOT_OK(p.GetExitStatus(&exit_status, &exit_info_str));
+  if (exit_status != 0) {
+    return Status::RuntimeError(exit_info_str);
+  }
+  return Status::OK();
+}
+
+pid_t Subprocess::pid() const {
+  CHECK_EQ(state_, kRunning);
+  return child_pid_;
+}
 
-  if (PREDICT_FALSE(retcode != 0)) {
-    return Status::RuntimeError(Substitute(
-        "Subprocess '$0' terminated with non-zero exit status $1",
-        argv[0],
-        retcode));
+Status Subprocess::DoWait(int* wait_status, WaitMode mode) {
+  if (state_ == kExited) {
+    if (wait_status) {
+      *wait_status = wait_status_;
+    }
+    return Status::OK();
+  }
+  if (state_ != kRunning) {
+    const string err_str = Substitute("$0: illegal sub-process state", state_);
+    LOG(DFATAL) << err_str;
+    return Status::IllegalState(err_str);
+  }
+
+  const int options = (mode == NON_BLOCKING) ? WNOHANG : 0;
+  int status;
+  const int rc = waitpid(child_pid_, &status, options);
+  if (rc == -1) {
+    return Status::RuntimeError("Unable to wait on child",
+                                ErrnoToString(errno), errno);
+  }
+  if (mode == NON_BLOCKING && rc == 0) {
+    return Status::TimedOut("");
+  }
+  CHECK_EQ(rc, child_pid_);
+  CHECK(WIFEXITED(status) || WIFSIGNALED(status));
+
+  child_pid_ = -1;
+  wait_status_ = status;
+  state_ = kExited;
+  if (wait_status) {
+    *wait_status = status;
   }
   return Status::OK();
 }
 
+void Subprocess::SetFdShared(int stdfd, bool share) {
+  CHECK_EQ(state_, kNotStarted);
+  CHECK_NE(fd_state_[stdfd], DISABLED);
+  fd_state_[stdfd] = share? SHARED : PIPED;
+}
+
 int Subprocess::CheckAndOffer(int stdfd) const {
   CHECK_EQ(state_, kRunning);
   CHECK_EQ(fd_state_[stdfd], PIPED);
@@ -534,9 +634,4 @@ int Subprocess::ReleaseChildFd(int stdfd) {
   return ret;
 }
 
-pid_t Subprocess::pid() const {
-  CHECK_EQ(state_, kRunning);
-  return child_pid_;
-}
-
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/bd56e021/src/kudu/util/subprocess.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/subprocess.h b/src/kudu/util/subprocess.h
index 1b70b1a..640f4aa 100644
--- a/src/kudu/util/subprocess.h
+++ b/src/kudu/util/subprocess.h
@@ -73,7 +73,7 @@ class Subprocess {
   // NOTE: unlike the standard wait(2) call, this may be called multiple
   // times. If the process has exited, it will repeatedly return the same
   // exit code.
-  Status Wait(int* ret) { return DoWait(ret, 0); }
+  Status Wait(int* wait_status = nullptr);
 
   // Like the above, but does not block. This returns Status::TimedOut
   // immediately if the child has not exited. Otherwise returns Status::OK
@@ -82,13 +82,17 @@ class Subprocess {
   // NOTE: unlike the standard wait(2) call, this may be called multiple
   // times. If the process has exited, it will repeatedly return the same
   // exit code.
-  Status WaitNoBlock(int* ret) { return DoWait(ret, WNOHANG); }
+  Status WaitNoBlock(int* wait_status = nullptr);
 
   // Send a signal to the subprocess.
   // Note that this does not reap the process -- you must still Wait()
   // in order to reap it. Only call after starting.
   Status Kill(int signal);
 
+  // Retrieve exit status of the process awaited by Wait() and/or WaitNoBlock()
+  // methods. Must be called only after calling Wait()/WaitNoBlock().
+  Status GetExitStatus(int* exit_status, std::string* info_str = nullptr) const;
+
   // Helper method that creates a Subprocess, issues a Start() then a Wait().
   // Expects a blank-separated list of arguments, with the first being the
   // full path to the executable.
@@ -122,29 +126,29 @@ class Subprocess {
   pid_t pid() const;
 
  private:
+  enum State {
+    kNotStarted,
+    kRunning,
+    kExited
+  };
+  enum StreamMode {SHARED, DISABLED, PIPED};
+  enum WaitMode {BLOCKING, NON_BLOCKING};
+
+  Status DoWait(int* wait_status, WaitMode mode);
   void SetFdShared(int stdfd, bool share);
   int CheckAndOffer(int stdfd) const;
   int ReleaseChildFd(int stdfd);
-  Status DoWait(int* ret, int options);
-
-  enum StreamMode {SHARED, DISABLED, PIPED};
 
   std::string program_;
   std::vector<std::string> argv_;
-
-  enum State {
-    kNotStarted,
-    kRunning,
-    kExited
-  };
   State state_;
   int child_pid_;
   enum StreamMode fd_state_[3];
   int child_fds_[3];
 
-  // The cached exit result code if Wait() has been called.
+  // The cached wait status if Wait()/WaitNoBlock() has been called.
   // Only valid if state_ == kExited.
-  int cached_rc_;
+  int wait_status_;
 
   DISALLOW_COPY_AND_ASSIGN(Subprocess);
 };