You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2017/03/06 19:54:28 UTC

[1/2] kudu git commit: KUDU-1905 - Allow reinserts on pk only tables

Repository: kudu
Updated Branches:
  refs/heads/master 7f2624ae4 -> f65feff68


KUDU-1905 - Allow reinserts on pk only tables

Doing a reinsert to a table that has only primary key columns
results in an empty change list. We're currently crashing whenever
we see a empty changelist that is not a delete.

The fix is just to allow empty changelists for reinserts.
This also adds a new flavor of fuzz tests to fuzz-itest.cc
that only have pk-only operations, as well as a directed
regression test that would trigger the problem deterministically.

Ran fuzz-itest in dist-tests with the new tests and the following
command:
KUDU_ALLOW_SLOW_TESTS=1 build-support/dist_test.py --collect-tmpdir \
loop -n 1000 build/debug/bin/fuzz-itest --gtest_repeat=10 \
--gtest_break_on_failure

Tests passed 1000/1000. Results:
http://dist-test.cloudera.org//job?job_id=david.alves.1488580839.22665

Change-Id: I7ce03378c7b97fac8ad8cb7783dec4a1b0277344
Reviewed-on: http://gerrit.cloudera.org:8080/6258
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: fff5cbda80b6cc2016c4bce012d2e183d3c3e2bb
Parents: 7f2624a
Author: David Alves <dr...@apache.org>
Authored: Fri Mar 3 14:17:58 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Mar 6 18:44:59 2017 +0000

----------------------------------------------------------------------
 src/kudu/common/row_changelist-test.cc   |   6 --
 src/kudu/common/row_changelist.cc        |   6 +-
 src/kudu/integration-tests/fuzz-itest.cc | 137 +++++++++++++++++++++-----
 3 files changed, 118 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/fff5cbda/src/kudu/common/row_changelist-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_changelist-test.cc b/src/kudu/common/row_changelist-test.cc
index 9523322..6b934a2 100644
--- a/src/kudu/common/row_changelist-test.cc
+++ b/src/kudu/common/row_changelist-test.cc
@@ -199,12 +199,6 @@ TEST_F(TestRowChangeList, TestInvalid_TooLongDelete) {
                       "Corruption: DELETE changelist too long");
 }
 
-TEST_F(TestRowChangeList, TestInvalid_TooShortReinsert) {
-  RowChangeListDecoder decoder(RowChangeList(Slice("\x03")));
-  ASSERT_STR_CONTAINS(decoder.Init().ToString(),
-                      "Corruption: empty changelist - expected column updates");
-}
-
 TEST_F(TestRowChangeList, TestInvalid_SetNullForNonNullableColumn) {
   faststring buf;
   RowChangeListEncoder rcl(&buf);

http://git-wip-us.apache.org/repos/asf/kudu/blob/fff5cbda/src/kudu/common/row_changelist.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/row_changelist.cc b/src/kudu/common/row_changelist.cc
index 82b6cc3..cd73d46 100644
--- a/src/kudu/common/row_changelist.cc
+++ b/src/kudu/common/row_changelist.cc
@@ -158,9 +158,11 @@ Status RowChangeListDecoder::Init() {
 
   remaining_.remove_prefix(1);
 
-  // We should discard empty REINSERT/UPDATE RowChangeLists, so if after getting
+  // We should discard empty UPDATE RowChangeLists, so if after getting
   // the type remaining_ is empty() return an error.
-  if (!is_delete() && remaining_.empty()) {
+  // Note that REINSERTs might have empty changelists when reinserting a row on a tablet that
+  // has only primary key columns.
+  if (is_update() && remaining_.empty()) {
     return Status::Corruption("empty changelist - expected column updates");
   }
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/fff5cbda/src/kudu/integration-tests/fuzz-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/fuzz-itest.cc b/src/kudu/integration-tests/fuzz-itest.cc
index b88148d..e98afb5 100644
--- a/src/kudu/integration-tests/fuzz-itest.cc
+++ b/src/kudu/integration-tests/fuzz-itest.cc
@@ -53,6 +53,7 @@ DECLARE_bool(use_hybrid_clock);
 // the fuzz test.
 enum TestOpType {
   TEST_INSERT,
+  TEST_INSERT_PK_ONLY,
   TEST_UPSERT,
   TEST_UPSERT_PK_ONLY,
   TEST_UPDATE,
@@ -101,6 +102,7 @@ namespace tablet {
 
 const char* TestOpType_names[] = {
   "TEST_INSERT",
+  "TEST_INSERT_PK_ONLY",
   "TEST_UPSERT",
   "TEST_UPSERT_PK_ONLY",
   "TEST_UPDATE",
@@ -130,6 +132,33 @@ struct TestOp {
   }
 };
 
+const vector<TestOpType> kAllOps {TEST_INSERT,
+                                  TEST_INSERT_PK_ONLY,
+                                  TEST_UPSERT,
+                                  TEST_UPSERT_PK_ONLY,
+                                  TEST_UPDATE,
+                                  TEST_DELETE,
+                                  TEST_FLUSH_OPS,
+                                  TEST_FLUSH_TABLET,
+                                  TEST_FLUSH_DELTAS,
+                                  TEST_MINOR_COMPACT_DELTAS,
+                                  TEST_MAJOR_COMPACT_DELTAS,
+                                  TEST_COMPACT_TABLET,
+                                  TEST_RESTART_TS,
+                                  TEST_SCAN_AT_TIMESTAMP};
+
+const vector<TestOpType> kPkOnlyOps {TEST_INSERT_PK_ONLY,
+                                     TEST_UPSERT_PK_ONLY,
+                                     TEST_DELETE,
+                                     TEST_FLUSH_OPS,
+                                     TEST_FLUSH_TABLET,
+                                     TEST_FLUSH_DELTAS,
+                                     TEST_MINOR_COMPACT_DELTAS,
+                                     TEST_MAJOR_COMPACT_DELTAS,
+                                     TEST_COMPACT_TABLET,
+                                     TEST_RESTART_TS,
+                                     TEST_SCAN_AT_TIMESTAMP};
+
 // Test which does only random operations against a tablet, including update and random
 // get (ie scans with equal lower and upper bounds).
 //
@@ -142,11 +171,10 @@ class FuzzTest : public KuduTest {
     FLAGS_enable_maintenance_manager = false;
     FLAGS_use_hybrid_clock = false;
     FLAGS_scanner_allow_snapshot_scans_with_logical_timestamps = true;
-
-    schema_ = client::KuduSchemaFromSchema(CreateKeyValueTestSchema());
   }
 
-  void SetUp() override {
+  void CreateTabletAndStartClusterWithSchema(const Schema& schema) {
+    schema_ =  client::KuduSchemaFromSchema(schema);
     KuduTest::SetUp();
 
     MiniClusterOptions opts;
@@ -175,8 +203,8 @@ class FuzzTest : public KuduTest {
   }
 
   void TearDown() override {
-    tablet_peer_.reset();
-    cluster_->Shutdown();
+    if (tablet_peer_) tablet_peer_.reset();
+    if (cluster_) cluster_->Shutdown();
   }
 
   scoped_refptr<TabletPeer> LookupTabletPeer() {
@@ -211,7 +239,7 @@ class FuzzTest : public KuduTest {
                                         TestOpType type) {
     ExpectedKeyValueRow ret;
     unique_ptr<KuduWriteOperation> op;
-    if (type == TEST_INSERT) {
+    if (type == TEST_INSERT || type == TEST_INSERT_PK_ONLY) {
       op.reset(table_->NewInsert());
     } else {
       op.reset(table_->NewUpsert());
@@ -219,17 +247,26 @@ class FuzzTest : public KuduTest {
     KuduPartialRow* row = op->mutable_row();
     CHECK_OK(row->SetInt32(0, key));
     ret.key = key;
-    if (type != TEST_UPSERT_PK_ONLY) {
-      if (val & 1) {
-        CHECK_OK(row->SetNull(1));
-      } else {
-        CHECK_OK(row->SetInt32(1, val));
-        ret.val = val;
+    switch (type) {
+      case TEST_INSERT:
+      case TEST_UPSERT: {
+        if (val & 1) {
+          CHECK_OK(row->SetNull(1));
+        } else {
+          CHECK_OK(row->SetInt32(1, val));
+          ret.val = val;
+        }
+        break;
       }
-    } else {
-      // For "upsert PK only", we expect the row to keep its old value
-      // the row existed, or NULL if there was no old row.
-      ret.val = old_row ? old_row->val : boost::none;
+      case TEST_INSERT_PK_ONLY:
+        break;
+      case TEST_UPSERT_PK_ONLY: {
+        // For "upsert PK only", we expect the row to keep its old value if
+        // the row existed, or NULL if there was no old row.
+        ret.val = old_row ? old_row->val : boost::none;
+        break;
+      }
+      default: LOG(FATAL) << "Invalid test op type: " << TestOpType_names[type];
     }
     CHECK_OK(session_->Apply(op.release()));
     return ret;
@@ -276,7 +313,7 @@ class FuzzTest : public KuduTest {
       for (KuduScanBatch::RowPtr row : batch) {
         ExpectedKeyValueRow ret;
         CHECK_OK(row.GetInt32(0, &ret.key));
-        if (!row.IsNull(1)) {
+        if (schema_.num_columns() > 1 && !row.IsNull(1)) {
           ret.val = 0;
           CHECK_OK(row.GetInt32(1, ret.val.get_ptr()));
         }
@@ -356,7 +393,7 @@ class FuzzTest : public KuduTest {
       for (KuduScanBatch::RowPtr row : batch) {
         ExpectedKeyValueRow ret;
         ASSERT_OK(row.GetInt32(0, &ret.key));
-        if (!row.IsNull(1)) {
+        if (schema_.num_columns() > 1 && !row.IsNull(1)) {
           ret.val = 0;
           ASSERT_OK(row.GetInt32(1, ret.val.get_ptr()));
         }
@@ -393,9 +430,25 @@ class FuzzTest : public KuduTest {
   scoped_refptr<TabletPeer> tablet_peer_;
 };
 
+// The set of ops to draw from.
+enum TestOpSets {
+  ALL, // Pick an operation at random from all possible operations.
+  PK_ONLY // Pick an operation at random from the set of operations that apply only to the
+          // primary key (or that are now row-specific, like flushes or compactions).
+};
+
+TestOpType PickOpAtRandom(TestOpSets sets) {
+  switch (sets) {
+    case ALL:
+      return kAllOps[rand() % kAllOps.size()];
+    case PK_ONLY:
+      return kPkOnlyOps[rand() % kPkOnlyOps.size()];
+  }
+}
+
 // Generate a random valid sequence of operations for use as a
 // fuzz test.
-void GenerateTestCase(vector<TestOp>* ops, int len) {
+void GenerateTestCase(vector<TestOp>* ops, int len, TestOpSets sets = ALL) {
   vector<bool> exists(FLAGS_keyspace_size);
   int op_timestamps = 0;
   bool ops_pending = false;
@@ -404,12 +457,13 @@ void GenerateTestCase(vector<TestOp>* ops, int len) {
   bool data_in_dms = false;
   ops->clear();
   while (ops->size() < len) {
-    TestOpType r = tight_enum_cast<TestOpType>(rand() % enum_limits<TestOpType>::max_enumerator);
+    TestOpType r = PickOpAtRandom(sets);
     int row_key = rand() % FLAGS_keyspace_size;
     switch (r) {
       case TEST_INSERT:
+      case TEST_INSERT_PK_ONLY:
         if (exists[row_key]) continue;
-        ops->push_back({TEST_INSERT, row_key});
+        ops->push_back({r, row_key});
         exists[row_key] = true;
         ops_pending = true;
         data_in_mrs = true;
@@ -502,7 +556,7 @@ void GenerateTestCase(vector<TestOp>* ops, int len) {
         break;
       }
       default:
-        LOG(FATAL);
+        LOG(FATAL) << "Invalid op type: " << r;
     }
   }
 }
@@ -543,6 +597,7 @@ void FuzzTest::RunFuzzCase(const vector<TestOp>& test_ops,
   for (const TestOp& test_op : test_ops) {
     switch (test_op.type) {
       case TEST_INSERT:
+      case TEST_INSERT_PK_ONLY:
       case TEST_UPSERT:
       case TEST_UPSERT_PK_ONLY:
       case TEST_UPDATE:
@@ -555,6 +610,7 @@ void FuzzTest::RunFuzzCase(const vector<TestOp>& test_ops,
     LOG(INFO) << test_op.ToString();
     switch (test_op.type) {
       case TEST_INSERT:
+      case TEST_INSERT_PK_ONLY:
       case TEST_UPSERT:
       case TEST_UPSERT_PK_ONLY: {
         pending_val[test_op.val] = InsertOrUpsertRow(
@@ -612,7 +668,19 @@ void FuzzTest::RunFuzzCase(const vector<TestOp>& test_ops,
 // Generates a random test sequence and runs it.
 // The logs of this test are designed to easily be copy-pasted and create
 // more specific test cases like TestFuzz<N> below.
+TEST_F(FuzzTest, TestRandomFuzzPksOnly) {
+  CreateTabletAndStartClusterWithSchema(Schema({ColumnSchema("key", INT32)}, 1));
+  SeedRandom();
+  vector<TestOp> test_ops;
+  GenerateTestCase(&test_ops, AllowSlowTests() ? 1000 : 50, TestOpSets::PK_ONLY);
+  RunFuzzCase(test_ops);
+}
+
+// Generates a random test sequence and runs it.
+// The logs of this test are designed to easily be copy-pasted and create
+// more specific test cases like TestFuzz<N> below.
 TEST_F(FuzzTest, TestRandomFuzz) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   SeedRandom();
   vector<TestOp> test_ops;
   GenerateTestCase(&test_ops, AllowSlowTests() ? 1000 : 50);
@@ -623,6 +691,7 @@ TEST_F(FuzzTest, TestRandomFuzz) {
 // This results in very large batches which are likely to span multiple delta blocks
 // when flushed.
 TEST_F(FuzzTest, TestRandomFuzzHugeBatches) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   SeedRandom();
   vector<TestOp> test_ops;
   GenerateTestCase(&test_ops, AllowSlowTests() ? 500 : 50);
@@ -637,6 +706,7 @@ TEST_F(FuzzTest, TestRandomFuzzHugeBatches) {
 }
 
 TEST_F(FuzzTest, TestFuzz1) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   vector<TestOp> test_ops = {
       // Get an inserted row in a DRS.
       {TEST_INSERT, 0},
@@ -662,6 +732,7 @@ TEST_F(FuzzTest, TestFuzz1) {
 
 // A particular test case which previously failed TestFuzz.
 TEST_F(FuzzTest, TestFuzz2) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   vector<TestOp> test_ops = {
     {TEST_INSERT, 0},
     {TEST_DELETE, 0},
@@ -694,6 +765,7 @@ TEST_F(FuzzTest, TestFuzz2) {
 
 // A particular test case which previously failed TestFuzz.
 TEST_F(FuzzTest, TestFuzz3) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   vector<TestOp> test_ops = {
     {TEST_INSERT, 0},
     {TEST_FLUSH_OPS, 0},
@@ -728,6 +800,7 @@ TEST_F(FuzzTest, TestFuzz3) {
 
 // A particular test case which previously failed TestFuzz.
 TEST_F(FuzzTest, TestFuzz4) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   vector<TestOp> test_ops = {
     {TEST_INSERT, 0},
     {TEST_FLUSH_OPS, 0},
@@ -763,6 +836,7 @@ TEST_F(FuzzTest, TestFuzz4) {
 //  Actual: "()"
 //  Expected: "(" + cur_val + ")"
 TEST_F(FuzzTest, TestFuzzWithRestarts1) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   RunFuzzCase({
       {TEST_INSERT, 1},
       {TEST_FLUSH_OPS, 0},
@@ -789,6 +863,7 @@ TEST_F(FuzzTest, TestFuzzWithRestarts1) {
 // insert undo deltas in sorted order (ascending key, then descending ts):
 // got key (row 1@tx5965182714017464320) after (row 1@tx5965182713875046400)
 TEST_F(FuzzTest, TestFuzzWithRestarts2) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   RunFuzzCase({
       {TEST_INSERT, 0},
       {TEST_FLUSH_OPS, 0},
@@ -823,6 +898,7 @@ TEST_F(FuzzTest, TestFuzzWithRestarts2) {
 // Regression test for KUDU-1467: a sequence involving
 // UPSERT which failed to replay properly upon bootstrap.
 TEST_F(FuzzTest, TestUpsertSeq) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   RunFuzzCase({
       {TEST_INSERT, 1},
       {TEST_UPSERT, 1},
@@ -841,7 +917,8 @@ TEST_F(FuzzTest, TestUpsertSeq) {
 
 // Regression test for KUDU-1623: updates without primary key
 // columns specified can cause crashes and issues at restart.
-TEST_F(FuzzTest, TestUpsert_PKOnly) {
+TEST_F(FuzzTest, TestUpsert_PKOnlyOps) {
+  CreateTabletAndStartClusterWithSchema(CreateKeyValueTestSchema());
   RunFuzzCase({
       {TEST_INSERT, 1},
       {TEST_FLUSH_OPS, 0},
@@ -851,5 +928,19 @@ TEST_F(FuzzTest, TestUpsert_PKOnly) {
     });
 }
 
+// Regression test for KUDU-1905: reinserts to a tablet that has
+// only primary keys end up as empty change lists. We were previously
+// crashing when a changelist was empty.
+TEST_F(FuzzTest, TestUpsert_PKOnlySchema) {
+  CreateTabletAndStartClusterWithSchema(Schema({ColumnSchema("key", INT32)}, 1));
+  RunFuzzCase({
+      {TEST_UPSERT_PK_ONLY, 1},
+      {TEST_DELETE, 1},
+      {TEST_UPSERT_PK_ONLY, 1},
+      {TEST_UPSERT_PK_ONLY, 1},
+      {TEST_FLUSH_OPS, 0}
+     });
+}
+
 } // namespace tablet
 } // namespace kudu


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

Posted by mp...@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>


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

Branch: refs/heads/master
Commit: f65feff685099d0b166c4faf219c528476df4578
Parents: fff5cbd
Author: Mike Percy <mp...@apache.org>
Authored: Fri Mar 3 17:15:35 2017 -0800
Committer: Mike Percy <mp...@apache.org>
Committed: Mon Mar 6 19:52:37 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/f65feff6/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/f65feff6/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/f65feff6/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/f65feff6/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/f65feff6/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/f65feff6/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/f65feff6/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);