You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by jd...@apache.org on 2017/03/08 18:25:15 UTC

[1/3] kudu git commit: env: Add support for getting FS capacity

Repository: kudu
Updated Branches:
  refs/heads/branch-1.3.x dae253dab -> 86055e295


env: Add support for getting FS capacity

We need a way to get FS capacity in a follow-up patch. This change adds
that capability and changes the Env API to allow for fetching both
capacity and free bytes with a single call. This also allows us to fetch
both values with a single syscall. This API is inspired by
boost::filesystem::space().

This patch also removes the difference in the "free" space returned when
this API is invoked as a superuser vs. a non-privileged user. Now, only
the space available to non-privileged processes is returned. This is
also consistent with the boost API and makes the API more predictable.

Change-Id: Id43275876d3352f5cf943e24ed4513b9f2c131aa
Reviewed-on: http://gerrit.cloudera.org:8080/6255
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
Reviewed-by: David Ribeiro Alves <dr...@apache.org>
Reviewed-on: http://gerrit.cloudera.org:8080/6277
Reviewed-by: Jean-Daniel Cryans <jd...@apache.org>


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

Branch: refs/heads/branch-1.3.x
Commit: a28d6f97b2805d8d7c28694a831463d071a24132
Parents: dae253d
Author: Mike Percy <mp...@apache.org>
Authored: Thu Mar 2 12:39:52 2017 -0800
Committer: Jean-Daniel Cryans <jd...@apache.org>
Committed: Wed Mar 8 17:59:37 2017 +0000

----------------------------------------------------------------------
 src/kudu/util/env-test.cc  | 41 +++++++++++++++++++++++++++--------------
 src/kudu/util/env.h        | 14 ++++++++++----
 src/kudu/util/env_posix.cc | 11 ++++-------
 src/kudu/util/env_util.cc  | 15 ++++++++-------
 4 files changed, 49 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a28d6f97/src/kudu/util/env-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env-test.cc b/src/kudu/util/env-test.cc
index 01e0db6..a8e5f70 100644
--- a/src/kudu/util/env-test.cc
+++ b/src/kudu/util/env-test.cc
@@ -25,6 +25,7 @@
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/bind.h"
+#include "kudu/gutil/strings/human_readable.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/util/env.h"
@@ -802,28 +803,40 @@ TEST_F(TestEnv, TestTempRWFile) {
   ASSERT_OK(env_->DeleteFile(path));
 }
 
-TEST_F(TestEnv, TestGetBytesFree) {
+// Test that when we write data to disk we see SpaceInfo.free_bytes go down.
+TEST_F(TestEnv, TestGetSpaceInfoFreeBytes) {
   const string kDataDir = GetTestPath("parent");
   const string kTestFilePath = JoinPathSegments(kDataDir, "testfile");
   const int kFileSizeBytes = 256;
-  int64_t orig_bytes_free;
-  int64_t cur_bytes_free;
   ASSERT_OK(env_->CreateDir(kDataDir));
 
-  // Loop several times in case there are concurrent tests running that are
-  // modifying the filesystem.
-  const int kIters = 100;
-  for (int i = 0; i < kIters; i++) {
+  // Loop in case there are concurrent tests running that are modifying the
+  // filesystem.
+  NO_FATALS(AssertEventually([&] {
     if (env_->FileExists(kTestFilePath)) {
-      ASSERT_OK(env_->DeleteFile(kTestFilePath));
+      ASSERT_OK(env_->DeleteFile(kTestFilePath)); // Clean up the previous iteration.
     }
-    ASSERT_OK(env_->GetBytesFree(kDataDir, &orig_bytes_free));
+    SpaceInfo before_space_info;
+    ASSERT_OK(env_->GetSpaceInfo(kDataDir, &before_space_info));
+
     NO_FATALS(WriteTestFile(env_, kTestFilePath, kFileSizeBytes));
-    ASSERT_OK(env_->GetBytesFree(kDataDir, &cur_bytes_free));
-    if (orig_bytes_free - cur_bytes_free >= kFileSizeBytes) break;
-  }
-  ASSERT_GE(orig_bytes_free - cur_bytes_free, kFileSizeBytes)
-      << "Failed after " << kIters << " attempts";
+
+    SpaceInfo after_space_info;
+    ASSERT_OK(env_->GetSpaceInfo(kDataDir, &after_space_info));
+    ASSERT_GE(before_space_info.free_bytes - after_space_info.free_bytes, kFileSizeBytes);
+  }));
+}
+
+// Basic sanity check for GetSpaceInfo().
+TEST_F(TestEnv, TestGetSpaceInfoBasicInvariants) {
+  string path = GetTestDataDirectory();
+  SpaceInfo space_info;
+  ASSERT_OK(env_->GetSpaceInfo(path, &space_info));
+  ASSERT_GT(space_info.capacity_bytes, 0);
+  ASSERT_LE(space_info.free_bytes, space_info.capacity_bytes);
+  VLOG(1) << "Path " << path << " has capacity "
+          << HumanReadableNumBytes::ToString(space_info.capacity_bytes)
+          << " (" << HumanReadableNumBytes::ToString(space_info.free_bytes) << " free)";
 }
 
 TEST_F(TestEnv, TestChangeDir) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/a28d6f97/src/kudu/util/env.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env.h b/src/kudu/util/env.h
index 442a451..bbf8f9c 100644
--- a/src/kudu/util/env.h
+++ b/src/kudu/util/env.h
@@ -36,6 +36,12 @@ struct RandomAccessFileOptions;
 struct RWFileOptions;
 struct WritableFileOptions;
 
+// Returned by Env::GetSpaceInfo().
+struct SpaceInfo {
+  int64_t capacity_bytes; // Capacity of a filesystem, in bytes.
+  int64_t free_bytes;     // Bytes available to non-privileged processes.
+};
+
 class Env {
  public:
   // Governs if/how the file is created.
@@ -191,10 +197,10 @@ class Env {
   // *block_size. fname must exist but it may be a file or a directory.
   virtual Status GetBlockSize(const std::string& fname, uint64_t* block_size) = 0;
 
-  // Determine the number of bytes free on the filesystem specified by 'path'.
-  // "Free space" accounting on the underlying filesystem may be more coarse
-  // than single bytes.
-  virtual Status GetBytesFree(const std::string& path, int64_t* bytes_free) = 0;
+  // Determine the capacity and number of bytes free on the filesystem
+  // specified by 'path'. "Free space" accounting on the underlying filesystem
+  // may be more coarse than single bytes.
+  virtual Status GetSpaceInfo(const std::string& path, SpaceInfo* space_info) = 0;
 
   // Rename file src to target.
   virtual Status RenameFile(const std::string& src,

http://git-wip-us.apache.org/repos/asf/kudu/blob/a28d6f97/src/kudu/util/env_posix.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index 6158bdc..4da3d47 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -1020,15 +1020,12 @@ class PosixEnv : public Env {
     return Status::OK();
   }
 
-  virtual Status GetBytesFree(const string& path, int64_t* bytes_free) OVERRIDE {
-    TRACE_EVENT1("io", "PosixEnv::GetBytesFree", "path", path);
+  virtual Status GetSpaceInfo(const string& path, SpaceInfo* space_info) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::GetSpaceInfo", "path", path);
     struct statvfs buf;
     RETURN_NOT_OK(StatVfs(path, &buf));
-    if (geteuid() == 0) {
-      *bytes_free = buf.f_frsize * buf.f_bfree;
-    } else {
-      *bytes_free = buf.f_frsize * buf.f_bavail;
-    }
+    space_info->capacity_bytes = buf.f_frsize * buf.f_blocks;
+    space_info->free_bytes = buf.f_frsize * buf.f_bavail;
     return Status::OK();
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a28d6f97/src/kudu/util/env_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.cc b/src/kudu/util/env_util.cc
index 178d391..4c7e93c 100644
--- a/src/kudu/util/env_util.cc
+++ b/src/kudu/util/env_util.cc
@@ -125,22 +125,23 @@ Status VerifySufficientDiskSpace(Env *env, const std::string& path,
                                  int64_t requested_bytes, int64_t reserved_bytes) {
   DCHECK_GE(requested_bytes, 0);
 
-  int64_t bytes_free;
-  RETURN_NOT_OK(env->GetBytesFree(path, &bytes_free));
+  SpaceInfo space_info;
+  RETURN_NOT_OK(env->GetSpaceInfo(path, &space_info));
+  int64_t available_bytes = space_info.free_bytes;
 
   // Allow overriding these values by tests.
   if (PREDICT_FALSE(FLAGS_disk_reserved_bytes_free_for_testing > -1)) {
-    bytes_free = FLAGS_disk_reserved_bytes_free_for_testing;
+    available_bytes = FLAGS_disk_reserved_bytes_free_for_testing;
   }
   if (PREDICT_FALSE(FLAGS_disk_reserved_override_prefix_1_bytes_free_for_testing != -1 ||
                     FLAGS_disk_reserved_override_prefix_2_bytes_free_for_testing != -1)) {
-    OverrideBytesFreeWithTestingFlags(path, &bytes_free);
+    OverrideBytesFreeWithTestingFlags(path, &available_bytes);
   }
 
-  if (bytes_free - requested_bytes < reserved_bytes) {
+  if (available_bytes - requested_bytes < reserved_bytes) {
     return Status::IOError(Substitute("Insufficient disk space to allocate $0 bytes under path $1 "
-                                      "($2 bytes free vs $3 bytes reserved)",
-                                      requested_bytes, path, bytes_free, reserved_bytes),
+                                      "($2 bytes available vs $3 bytes reserved)",
+                                      requested_bytes, path, available_bytes, reserved_bytes),
                            "", ENOSPC);
   }
   return Status::OK();


[3/3] kudu git commit: Reserve 1% of disk space by default

Posted by jd...@apache.org.
Reserve 1% of disk space by default

Many people are not aware that Kudu is able to reserve a certain amount
of space per disk for non-Kudu usage and they don't set the
corresponding flags. Let's bump the default from 0 to a special value
indicating one percent, which seems like a reasonable default in most
cases.

Change-Id: I578afeefd9a520fd56bfca597c5fcec8f0f3f98d
Reviewed-on: http://gerrit.cloudera.org:8080/6192
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Reviewed-on: http://gerrit.cloudera.org:8080/6278
Reviewed-by: Jean-Daniel Cryans <jd...@apache.org>


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

Branch: refs/heads/branch-1.3.x
Commit: 86055e2950a3cb606ea567c2235da434dae5a72b
Parents: 7395221
Author: Mike Percy <mp...@apache.org>
Authored: Tue Feb 28 17:23:09 2017 -0800
Committer: Jean-Daniel Cryans <jd...@apache.org>
Committed: Wed Mar 8 18:24:23 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/log.cc      | 10 ++++++++--
 src/kudu/fs/data_dirs.cc       | 10 ++++++++--
 src/kudu/util/env_util-test.cc | 32 +++++++++++++++++++++++++-------
 src/kudu/util/env_util.cc      |  6 ++++++
 src/kudu/util/env_util.h       |  3 ++-
 5 files changed, 49 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/86055e29/src/kudu/consensus/log.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log.cc b/src/kudu/consensus/log.cc
index 716baa0..d51e326 100644
--- a/src/kudu/consensus/log.cc
+++ b/src/kudu/consensus/log.cc
@@ -116,8 +116,14 @@ DEFINE_double(log_inject_io_error_on_preallocate_fraction, 0.0,
 TAG_FLAG(log_inject_io_error_on_preallocate_fraction, unsafe);
 TAG_FLAG(log_inject_io_error_on_preallocate_fraction, runtime);
 
-DEFINE_int64(fs_wal_dir_reserved_bytes, 0,
-             "Number of bytes to reserve on the log directory filesystem for non-Kudu usage");
+DEFINE_int64(fs_wal_dir_reserved_bytes, -1,
+             "Number of bytes to reserve on the log directory filesystem for "
+             "non-Kudu usage. The default, which is represented by -1, is that "
+             "1% of the disk space on each disk will be reserved. Any other "
+             "value specified represents the number of bytes reserved and must "
+             "be greater than or equal to 0. Explicit percentages to reserve "
+             "are not currently supported");
+DEFINE_validator(fs_wal_dir_reserved_bytes, [](const char* /*n*/, int64_t v) { return v >= -1; });
 TAG_FLAG(fs_wal_dir_reserved_bytes, runtime);
 TAG_FLAG(fs_wal_dir_reserved_bytes, evolving);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/86055e29/src/kudu/fs/data_dirs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index 8453378..44e9f1b 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -51,8 +51,14 @@
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/threadpool.h"
 
-DEFINE_int64(fs_data_dirs_reserved_bytes, 0,
-             "Number of bytes to reserve on each data directory filesystem for non-Kudu usage.");
+DEFINE_int64(fs_data_dirs_reserved_bytes, -1,
+             "Number of bytes to reserve on each data directory filesystem for "
+             "non-Kudu usage. The default, which is represented by -1, is that "
+             "1% of the disk space on each disk will be reserved. Any other "
+             "value specified represents the number of bytes reserved and must "
+             "be greater than or equal to 0. Explicit percentages to reserve "
+             "are not currently supported");
+DEFINE_validator(fs_data_dirs_reserved_bytes, [](const char* /*n*/, int64_t v) { return v >= -1; });
 TAG_FLAG(fs_data_dirs_reserved_bytes, runtime);
 TAG_FLAG(fs_data_dirs_reserved_bytes, evolving);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/86055e29/src/kudu/util/env_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util-test.cc b/src/kudu/util/env_util-test.cc
index 7c79068..715f03b 100644
--- a/src/kudu/util/env_util-test.cc
+++ b/src/kudu/util/env_util-test.cc
@@ -33,7 +33,6 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_int64(disk_reserved_bytes);
 DECLARE_int64(disk_reserved_bytes_free_for_testing);
 
 using std::string;
@@ -47,20 +46,39 @@ namespace env_util {
 class EnvUtilTest: public KuduTest {
 };
 
+// Assert that Status 's' indicates there is not enough space left on the
+// device for the request.
+static void AssertNoSpace(const Status& s) {
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_EQ(ENOSPC, s.posix_code());
+  ASSERT_STR_CONTAINS(s.ToString(), "Insufficient disk space");
+}
+
 TEST_F(EnvUtilTest, TestDiskSpaceCheck) {
-  const int64_t kRequestedBytes = 0;
+  const int64_t kZeroRequestedBytes = 0;
+  const int64_t kRequestOnePercentReservation = -1;
   int64_t reserved_bytes = 0;
-  ASSERT_OK(VerifySufficientDiskSpace(env_, test_dir_, kRequestedBytes, reserved_bytes));
+  ASSERT_OK(VerifySufficientDiskSpace(env_, test_dir_, kZeroRequestedBytes, reserved_bytes));
+
+  // Check 1% reservation logic. We loop this in case there are other FS
+  // operations happening concurrent with this test.
+  NO_FATALS(AssertEventually([&] {
+    SpaceInfo space_info;
+    ASSERT_OK(env_->GetSpaceInfo(test_dir_, &space_info));
+    // Try for 1 less byte than 1% free. This request should be rejected.
+    int64_t target_free_bytes = (space_info.capacity_bytes / 100) - 1;
+    int64_t bytes_to_request = std::max(0L, space_info.free_bytes - target_free_bytes);
+    NO_FATALS(AssertNoSpace(VerifySufficientDiskSpace(env_, test_dir_, bytes_to_request,
+                                                      kRequestOnePercentReservation)));
+  }));
 
   // Make it seem as if the disk is full and specify that we should have
   // reserved 200 bytes. Even asking for 0 bytes should return an error
   // indicating we are out of space.
   FLAGS_disk_reserved_bytes_free_for_testing = 0;
   reserved_bytes = 200;
-  Status s = VerifySufficientDiskSpace(env_, test_dir_, kRequestedBytes, reserved_bytes);
-  ASSERT_TRUE(s.IsIOError());
-  ASSERT_EQ(ENOSPC, s.posix_code());
-  ASSERT_STR_CONTAINS(s.ToString(), "Insufficient disk space");
+  NO_FATALS(AssertNoSpace(VerifySufficientDiskSpace(env_, test_dir_, kZeroRequestedBytes,
+                                                    reserved_bytes)));
 }
 
 // Ensure that we can recursively create directories using both absolute and

http://git-wip-us.apache.org/repos/asf/kudu/blob/86055e29/src/kudu/util/env_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.cc b/src/kudu/util/env_util.cc
index 4c7e93c..b471951 100644
--- a/src/kudu/util/env_util.cc
+++ b/src/kudu/util/env_util.cc
@@ -123,6 +123,7 @@ static void OverrideBytesFreeWithTestingFlags(const string& path, int64_t* bytes
 
 Status VerifySufficientDiskSpace(Env *env, const std::string& path,
                                  int64_t requested_bytes, int64_t reserved_bytes) {
+  const int64_t kOnePercentReservation = -1;
   DCHECK_GE(requested_bytes, 0);
 
   SpaceInfo space_info;
@@ -138,6 +139,11 @@ Status VerifySufficientDiskSpace(Env *env, const std::string& path,
     OverrideBytesFreeWithTestingFlags(path, &available_bytes);
   }
 
+  // If they requested a one percent reservation, calculate what that is in bytes.
+  if (reserved_bytes == kOnePercentReservation) {
+    reserved_bytes = space_info.capacity_bytes / 100;
+  }
+
   if (available_bytes - requested_bytes < reserved_bytes) {
     return Status::IOError(Substitute("Insufficient disk space to allocate $0 bytes under path $1 "
                                       "($2 bytes available vs $3 bytes reserved)",

http://git-wip-us.apache.org/repos/asf/kudu/blob/86055e29/src/kudu/util/env_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.h b/src/kudu/util/env_util.h
index 884b17c..a2f96a4 100644
--- a/src/kudu/util/env_util.h
+++ b/src/kudu/util/env_util.h
@@ -42,7 +42,8 @@ Status OpenFileForSequential(Env *env, const std::string &path,
 // Returns Status::IOError with POSIX code ENOSPC if there is not sufficient
 // disk space to write 'bytes' bytes to the file system represented by 'path'.
 // Otherwise returns OK.
-
+// If 'reserved_bytes' equals -1, it is interpreted as a 1% reservation. No
+// other values less than 0 are supported at this time.
 Status VerifySufficientDiskSpace(Env *env, const std::string& path,
                                  int64_t requested_bytes, int64_t reserved_bytes);
 


[2/3] kudu git commit: Ignore SIGPIPE earlier in startup process

Posted by jd...@apache.org.
Ignore SIGPIPE earlier in startup process

This change resolves a race during startup where we are not protected
from SIGPIPE from the time we start the process until the time we start
the squeasel web server, which sets the disposition of SIGPIPE to
SIG_IGN.

This also factors some of the signal-handling helper functions into a
new set of util files, signal.{h,cc}.

Change-Id: I040bd38ff31451ed9e25e7cf2127c869cf08a628
Reviewed-on: http://gerrit.cloudera.org:8080/6262
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
(cherry picked from commit f65feff685099d0b166c4faf219c528476df4578)
Reviewed-on: http://gerrit.cloudera.org:8080/6276
Reviewed-by: Jean-Daniel Cryans <jd...@apache.org>


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

Branch: refs/heads/branch-1.3.x
Commit: 739522161fff5e5fc6cce0c3abfb6cf1aa1825a9
Parents: a28d6f9
Author: Mike Percy <mp...@apache.org>
Authored: Fri Mar 3 17:15:35 2017 -0800
Committer: Jean-Daniel Cryans <jd...@apache.org>
Committed: Wed Mar 8 18:01:09 2017 +0000

----------------------------------------------------------------------
 src/kudu/util/CMakeLists.txt |  1 +
 src/kudu/util/logging.cc     |  5 +++++
 src/kudu/util/logging.h      |  3 ++-
 src/kudu/util/signal.cc      | 47 +++++++++++++++++++++++++++++++++++++++
 src/kudu/util/signal.h       | 42 ++++++++++++++++++++++++++++++++++
 src/kudu/util/subprocess.cc  | 43 +++++++----------------------------
 src/kudu/util/test_main.cc   |  5 +++++
 7 files changed, 110 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/73952216/src/kudu/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index 685f01f..7d049a9 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -175,6 +175,7 @@ set(UTIL_SRCS
   rw_mutex.cc
   rwc_lock.cc
   ${SEMAPHORE_CC}
+  signal.cc
   slice.cc
   spinlock_profiling.cc
   status.cc

http://git-wip-us.apache.org/repos/asf/kudu/blob/73952216/src/kudu/util/logging.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/logging.cc b/src/kudu/util/logging.cc
index f45285d..c62bfe7 100644
--- a/src/kudu/util/logging.cc
+++ b/src/kudu/util/logging.cc
@@ -41,6 +41,7 @@
 #include "kudu/util/env_util.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/minidump.h"
+#include "kudu/util/signal.h"
 #include "kudu/util/status.h"
 
 DEFINE_string(log_filename, "",
@@ -265,6 +266,10 @@ void InitGoogleLoggingSafe(const char* arg) {
   // Sink logging: off.
   initial_stderr_severity = FLAGS_stderrthreshold;
 
+  // Ignore SIGPIPE early in the startup process so that threads writing to TLS
+  // sockets do not crash when writing to a closed socket. See KUDU-1910.
+  IgnoreSigPipe();
+
   // For minidump support. Must be called before logging threads started.
   CHECK_OK(BlockSigUSR1());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/73952216/src/kudu/util/logging.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/logging.h b/src/kudu/util/logging.h
index 1640793..682431e 100644
--- a/src/kudu/util/logging.h
+++ b/src/kudu/util/logging.h
@@ -258,7 +258,8 @@ class Env;
 // glog doesn't allow multiple invocations of InitGoogleLogging. This method conditionally
 // calls InitGoogleLogging only if it hasn't been called before.
 //
-// It also takes care of installing the google failure signal handler.
+// It also takes care of installing the google failure signal handler and
+// setting the signal handler for SIGPIPE to SIG_IGN.
 void InitGoogleLoggingSafe(const char* arg);
 
 // Like InitGoogleLoggingSafe() but stripped down: no signal handlers are

http://git-wip-us.apache.org/repos/asf/kudu/blob/73952216/src/kudu/util/signal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/signal.cc b/src/kudu/util/signal.cc
new file mode 100644
index 0000000..2de3000
--- /dev/null
+++ b/src/kudu/util/signal.cc
@@ -0,0 +1,47 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/util/signal.h"
+
+#include "kudu/util/logging.h"
+
+namespace kudu {
+
+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);
+}
+
+void IgnoreSigPipe() {
+  SetSignalHandler(SIGPIPE, SIG_IGN);
+}
+
+void ResetSigPipeHandlerToDefault() {
+  SetSignalHandler(SIGPIPE, SIG_DFL);
+}
+
+// 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);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/73952216/src/kudu/util/signal.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/signal.h b/src/kudu/util/signal.h
new file mode 100644
index 0000000..0c88a80
--- /dev/null
+++ b/src/kudu/util/signal.h
@@ -0,0 +1,42 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <signal.h>
+
+namespace kudu {
+
+#if defined(__linux__)
+typedef sighandler_t SignalHandlerCallback;
+#else
+typedef sig_t SignalHandlerCallback;
+#endif
+
+// Set a process-wide signal handler.
+void SetSignalHandler(int signal, SignalHandlerCallback handler);
+
+// Set the disposition of SIGPIPE to SIG_IGN.
+void IgnoreSigPipe();
+
+// Set the disposition of SIGPIPE to SIG_DFL.
+void ResetSigPipeHandlerToDefault();
+
+// Unblock all signal masks.
+void ResetAllSignalMasksToUnblocked();
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/73952216/src/kudu/util/subprocess.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/subprocess.cc b/src/kudu/util/subprocess.cc
index f69abe8..73d9672 100644
--- a/src/kudu/util/subprocess.cc
+++ b/src/kudu/util/subprocess.cc
@@ -46,6 +46,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/debug-util.h"
 #include "kudu/util/errno.h"
+#include "kudu/util/signal.h"
 #include "kudu/util/status.h"
 
 using std::string;
@@ -72,42 +73,11 @@ static const char* kProcSelfFd =
 #if defined(__linux__)
 #define READDIR readdir64
 #define DIRENT dirent64
-typedef sighandler_t SignalHandlerCallback;
 #else
 #define READDIR readdir
 #define DIRENT dirent
-typedef sig_t SignalHandlerCallback;
 #endif
 
-// Convenience wrapper for sigaction().
-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);
-}
-
-void IgnoreSigPipe() {
-  SetSignalHandler(SIGPIPE, SIG_IGN);
-}
-
-void ResetSigPipeHandlerToDefault() {
-  SetSignalHandler(SIGPIPE, SIG_DFL);
-}
-
-void EnsureSigPipeIgnored() {
-  static GoogleOnceType once = GOOGLE_ONCE_INIT;
-  GoogleOnceInit(&once, &IgnoreSigPipe);
-}
-
-// 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);
-}
-
 // Since opendir() calls malloc(), this must be called before fork().
 // This function is not async-signal-safe.
 Status OpenProcFdDir(DIR** dir) {
@@ -339,7 +309,9 @@ Status Subprocess::Start() {
   if (argv_.empty()) {
     return Status::InvalidArgument("argv must have at least one elem");
   }
-  EnsureSigPipeIgnored();
+
+  // We explicitly set SIGPIPE to SIG_IGN here because we are using UNIX pipes.
+  IgnoreSigPipe();
 
   vector<char*> argv_ptrs;
   for (const string& arg : argv_) {
@@ -429,9 +401,10 @@ Status Subprocess::Start() {
 
     // Ensure we are not ignoring or blocking signals in the child process.
     ResetAllSignalMasksToUnblocked();
-    // Reset SIGPIPE to its default disposition because we previously set it to
-    // SIG_IGN via EnsureSigPipeIgnored(). At the time of writing, we don't
-    // explicitly ignore any other signals in Kudu.
+
+    // Reset the disposition of SIGPIPE to SIG_DFL because we routinely set its
+    // disposition to SIG_IGN via IgnoreSigPipe(). At the time of writing, we
+    // don't explicitly ignore any other signals in Kudu.
     ResetSigPipeHandlerToDefault();
 
     // Set the environment for the subprocess. This is more portable than

http://git-wip-us.apache.org/repos/asf/kudu/blob/73952216/src/kudu/util/test_main.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_main.cc b/src/kudu/util/test_main.cc
index 74fe4e9..aa64387 100644
--- a/src/kudu/util/test_main.cc
+++ b/src/kudu/util/test_main.cc
@@ -25,6 +25,7 @@
 #include "kudu/util/pstack_watcher.h"
 #include "kudu/util/flags.h"
 #include "kudu/util/minidump.h"
+#include "kudu/util/signal.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_util.h"
 
@@ -71,6 +72,10 @@ int main(int argc, char **argv) {
   // need to block SIGUSR1 explicitly in order to test minidump generation.
   CHECK_OK(kudu::BlockSigUSR1());
 
+  // Ignore SIGPIPE for all tests so that threads writing to TLS
+  // sockets do not crash when writing to a closed socket. See KUDU-1910.
+  kudu::IgnoreSigPipe();
+
   // InitGoogleTest() must precede ParseCommandLineFlags(), as the former
   // removes gtest-related flags from argv that would trip up the latter.
   ::testing::InitGoogleTest(&argc, argv);