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

[1/2] kudu git commit: env: rename writable_file_use_fsync to env_file_use_fsync

Repository: kudu
Updated Branches:
  refs/heads/master 11f4121a9 -> c681f9348


env: rename writable_file_use_fsync to env_file_use_fsync

It's not just used for writable_file. And I doubt anyone has set this thing;
we copied it from rocksdb to work around buggy ext3 implementations.

Still, it's a backwards incompatible change, so maybe I should leave it be?

Change-Id: I66f9b448869885e855558dfc9194aabd17a8644d
Reviewed-on: http://gerrit.cloudera.org:8080/5458
Tested-by: Kudu Jenkins
Reviewed-by: Mike Percy <mp...@apache.org>


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

Branch: refs/heads/master
Commit: d5567724a210cbbd5425f2de683b413e01b533a4
Parents: 11f4121
Author: Adar Dembo <ad...@cloudera.com>
Authored: Fri Dec 9 18:47:11 2016 -0800
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Mon Dec 12 20:46:02 2016 +0000

----------------------------------------------------------------------
 src/kudu/util/env_posix.cc | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/d5567724/src/kudu/util/env_posix.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index 984e032..d1d9a23 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -95,10 +95,11 @@
 } while ((nread) == 0 && ferror(stream) == EINTR)
 
 // See KUDU-588 for details.
-DEFINE_bool(writable_file_use_fsync, false,
+DEFINE_bool(env_use_fsync, false,
             "Use fsync(2) instead of fdatasync(2) for synchronizing dirty "
             "data to disk.");
-TAG_FLAG(writable_file_use_fsync, advanced);
+TAG_FLAG(env_use_fsync, advanced);
+TAG_FLAG(env_use_fsync, evolving);
 
 DEFINE_bool(suicide_on_eio, true,
             "Kill the process if an I/O operation results in EIO");
@@ -207,7 +208,7 @@ static Status IOError(const std::string& context, int err_number) {
 static Status DoSync(int fd, const string& filename) {
   ThreadRestrictions::AssertIOAllowed();
   if (FLAGS_never_fsync) return Status::OK();
-  if (FLAGS_writable_file_use_fsync) {
+  if (FLAGS_env_use_fsync) {
     TRACE_COUNTER_SCOPE_LATENCY_US("fsync_us");
     TRACE_COUNTER_INCREMENT("fsync", 1);
     if (fsync(fd) < 0) {


[2/2] kudu git commit: tests: set never_fsync for every test

Posted by ad...@apache.org.
tests: set never_fsync for every test

None of our tests rely on unplugging machines, forcefully unmounting a
filesystem, or any other equivalent activity where fsync()'s durability
guarantees might be necessary. Moreover, we were already disabling fsync()
in a bunch of tests; this patch just centralizes it into KuduTest so all
tests get it for free. Doing so means tests wil run faster and we can remove
some duplicated test code.

Some tests used enable_data_block_fsync=false instead. I think it predates
never_fsync, and it's not nearly as good of a choice, as it leads to
slightly different code paths taken (and thus different coverage) in the
block managers.

I don't think the change to compaction-test will regress the bug fixed by
commit 7d92ed6; other tests have since begun using
Env::GetFileSizeOnDiskRecursively() without issue, and I don't think our
test environments are so messed up that they're mounting ext4 as ext3.

I ran ctest serially on a debug build (fast tests) and here's what I got:

Vanilla
-------
Total Test time (real) = 957.84 sec

real	15m57.866s
user	13m47.340s
sys	2m24.148s

Modified
--------
Total Test time (real) = 816.72 sec

real	13m36.820s
user	14m34.812s
sys	2m27.704s

So a pretty nice improvement already.

Change-Id: Ia8c2574eb3cb76fba1bce0202d4335d8b7035a93
Reviewed-on: http://gerrit.cloudera.org:8080/5459
Tested-by: Kudu Jenkins
Reviewed-by: Mike Percy <mp...@apache.org>


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

Branch: refs/heads/master
Commit: c681f93480f23914a0c45c5fa6a3829838d81643
Parents: d556772
Author: Adar Dembo <ad...@cloudera.com>
Authored: Fri Dec 9 18:56:35 2016 -0800
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Mon Dec 12 22:03:11 2016 +0000

----------------------------------------------------------------------
 src/kudu/cfile/encoding-test.cc                 | 14 ++++++-------
 src/kudu/client/client-test.cc                  |  3 ---
 src/kudu/common/encoded_key-test.cc             |  2 +-
 src/kudu/consensus/consensus_queue-test.cc      |  2 --
 src/kudu/fs/block_manager-stress-test.cc        |  5 -----
 src/kudu/fs/block_manager-test.cc               | 11 ----------
 .../alter_table-randomized-test.cc              |  4 ----
 src/kudu/integration-tests/alter_table-test.cc  |  2 --
 src/kudu/integration-tests/consistency-itest.cc |  3 ---
 .../integration-tests/create-table-itest.cc     |  5 +----
 .../exactly_once_writes-itest.cc                |  1 -
 .../external_mini_cluster-itest-base.cc         |  2 --
 .../integration-tests/external_mini_cluster.cc  | 22 ++++++++++++--------
 .../integration-tests/master-stress-test.cc     |  6 ------
 .../integration-tests/raft_consensus-itest.cc   |  1 -
 src/kudu/integration-tests/ts_recovery-itest.cc |  2 --
 .../integration-tests/version_migration-test.cc |  1 -
 .../integration-tests/webserver-stress-itest.cc |  2 --
 src/kudu/tablet/compaction-test.cc              | 13 ------------
 src/kudu/tablet/deltafile-test.cc               |  3 ---
 src/kudu/tablet/memrowset-test.cc               |  4 +---
 src/kudu/tablet/tablet-test-util.h              |  7 -------
 src/kudu/tools/kudu-tool-test.cc                |  3 +--
 src/kudu/tserver/tablet_server-test-base.h      |  7 -------
 src/kudu/util/atomic-test.cc                    |  7 +++++--
 src/kudu/util/file_cache-stress-test.cc         |  6 ------
 src/kudu/util/flag_tags-test.cc                 |  8 +++++++
 src/kudu/util/flags-test.cc                     |  9 ++++++--
 src/kudu/util/test_util.cc                      |  5 +++++
 29 files changed, 49 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/cfile/encoding-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/encoding-test.cc b/src/kudu/cfile/encoding-test.cc
index 0a2e368..6f5d7e1 100644
--- a/src/kudu/cfile/encoding-test.cc
+++ b/src/kudu/cfile/encoding-test.cc
@@ -37,26 +37,26 @@
 #include "kudu/util/hexdump.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/random.h"
-#include "kudu/util/test_macros.h"
-#include <kudu/util/test_util.h>
 #include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
 
 using std::unique_ptr;
 using std::vector;
 
-namespace kudu { namespace cfile {
+namespace kudu {
 
-extern void DumpSSETable();
+namespace cfile {
 
-class TestEncoding : public ::testing::Test {
+class TestEncoding : public KuduTest {
  public:
   TestEncoding()
-    : ::testing::Test(),
-      arena_(1024, 1024*1024) {
+    : arena_(1024, 1024*1024) {
   }
 
  protected:
   virtual void SetUp() OVERRIDE {
+    KuduTest::SetUp();
     arena_.Reset();
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index e467164..4e24595 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -69,7 +69,6 @@
 #include "kudu/util/test_util.h"
 #include "kudu/util/thread.h"
 
-DECLARE_bool(enable_data_block_fsync);
 DECLARE_bool(fail_dns_resolution);
 DECLARE_bool(log_inject_latency);
 DECLARE_bool(allow_unsafe_replication_factor);
@@ -129,8 +128,6 @@ class ClientTest : public KuduTest {
     b.AddColumn("non_null_with_default")->Type(KuduColumnSchema::INT32)->NotNull()
       ->Default(KuduValue::FromInt(12345));
     CHECK_OK(b.Build(&schema_));
-
-    FLAGS_enable_data_block_fsync = false; // Keep unit tests fast.
   }
 
   void SetUp() override {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/common/encoded_key-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/encoded_key-test.cc b/src/kudu/common/encoded_key-test.cc
index 25dd122..cc3122e 100644
--- a/src/kudu/common/encoded_key-test.cc
+++ b/src/kudu/common/encoded_key-test.cc
@@ -41,7 +41,7 @@ namespace kudu {
     EXPECT_NO_FATAL_FAILURE(ExpectDecodedKeyEq<(type)>((expected), (encoded_form), (val))); \
   } while (0)
 
-class EncodedKeyTest : public ::testing::Test {
+class EncodedKeyTest : public KuduTest {
  public:
   EncodedKeyTest()
   : schema_(CreateSchema()),

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/consensus/consensus_queue-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue-test.cc b/src/kudu/consensus/consensus_queue-test.cc
index bcf98af..6d4ef7e 100644
--- a/src/kudu/consensus/consensus_queue-test.cc
+++ b/src/kudu/consensus/consensus_queue-test.cc
@@ -34,7 +34,6 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_bool(enable_data_block_fsync);
 DECLARE_int32(consensus_max_batch_size_bytes);
 
 METRIC_DECLARE_entity(tablet);
@@ -52,7 +51,6 @@ class ConsensusQueueTest : public KuduTest {
       : schema_(GetSimpleTestSchema()),
         metric_entity_(METRIC_ENTITY_tablet.Instantiate(&metric_registry_, "queue-test")),
         registry_(new log::LogAnchorRegistry) {
-    FLAGS_enable_data_block_fsync = false; // Keep unit tests fast.
   }
 
   virtual void SetUp() OVERRIDE {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/fs/block_manager-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/block_manager-stress-test.cc b/src/kudu/fs/block_manager-stress-test.cc
index 7d591ca..d9d1ee7 100644
--- a/src/kudu/fs/block_manager-stress-test.cc
+++ b/src/kudu/fs/block_manager-stress-test.cc
@@ -37,7 +37,6 @@
 DECLARE_int64(block_manager_max_open_files);
 DECLARE_uint64(log_container_max_size);
 DECLARE_uint64(log_container_preallocate_bytes);
-DECLARE_bool(never_fsync);
 
 DEFINE_int32(test_duration_secs, 2, "Number of seconds to run the test");
 DEFINE_int32(num_writer_threads, 4, "Number of writer threads to run");
@@ -91,10 +90,6 @@ class BlockManagerStressTest : public KuduTest {
     total_bytes_read_(0),
     total_blocks_deleted_(0) {
 
-    // Increases total number of blocks manipulated, which is the right kind
-    // of stress for this test.
-    FLAGS_never_fsync = true;
-
     // Increase the number of containers created.
     FLAGS_log_container_max_size = 1 * 1024 * 1024;
     FLAGS_log_container_preallocate_bytes = 1 * 1024 * 1024;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/fs/block_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/block_manager-test.cc b/src/kudu/fs/block_manager-test.cc
index 4e7cf6a..5e8bd1e 100644
--- a/src/kudu/fs/block_manager-test.cc
+++ b/src/kudu/fs/block_manager-test.cc
@@ -48,8 +48,6 @@ using std::unordered_set;
 using std::vector;
 using strings::Substitute;
 
-DECLARE_bool(never_fsync);
-
 DECLARE_uint64(log_container_preallocate_bytes);
 DECLARE_uint64(log_container_max_size);
 
@@ -1153,9 +1151,6 @@ TYPED_TEST(BlockManagerTest, TestMetadataOkayDespiteFailedWrites) {
   const int kNumAppends = 4;
   const string kTestData = "asdf";
 
-  // Speed up the test.
-  FLAGS_never_fsync = true;
-
   // Since we're appending so little data, reconfigure these to ensure quite a
   // few containers and a good amount of preallocating.
   FLAGS_log_container_max_size = 256 * 1024;
@@ -1269,9 +1264,6 @@ TEST_F(LogBlockManagerTest, TestContainerWithManyHoles) {
 
   ASSERT_GE(kNumBlocks, last_interior_node_block_number);
 
-  // Speed up the test.
-  FLAGS_never_fsync = true;
-
   // Create a bunch of blocks. They should all go in one container (unless
   // the container becomes full).
   LOG(INFO) << Substitute("Creating $0 blocks", kNumBlocks);
@@ -1345,9 +1337,6 @@ TEST_F(LogBlockManagerTest, TestContainerBlockLimiting) {
 
   const int kNumBlocks = 1000;
 
-  // Speed up the test.
-  FLAGS_never_fsync = true;
-
   // Creates 'kNumBlocks' blocks with minimal data.
   auto create_some_blocks = [&]() -> Status {
     for (int i = 0; i < kNumBlocks; i++) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/alter_table-randomized-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/alter_table-randomized-test.cc b/src/kudu/integration-tests/alter_table-randomized-test.cc
index 55b5318..14a592c 100644
--- a/src/kudu/integration-tests/alter_table-randomized-test.cc
+++ b/src/kudu/integration-tests/alter_table-randomized-test.cc
@@ -69,10 +69,6 @@ class AlterTableRandomized : public KuduTest {
 
     ExternalMiniClusterOptions opts;
     opts.num_tablet_servers = 3;
-    // Because this test performs a lot of alter tables, we end up flushing
-    // and rewriting metadata files quite a bit. Globally disabling fsync
-    // speeds the test runtime up dramatically.
-    opts.extra_tserver_flags.push_back("--never_fsync");
     // This test produces tables with lots of columns. With container preallocation,
     // we end up using quite a bit of disk space. So, we disable it.
     opts.extra_tserver_flags.push_back("--log_container_preallocate_bytes=0");

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/alter_table-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc
index 2d0b8eb..9a08065 100644
--- a/src/kudu/integration-tests/alter_table-test.cc
+++ b/src/kudu/integration-tests/alter_table-test.cc
@@ -49,7 +49,6 @@
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_bool(enable_data_block_fsync);
 DECLARE_bool(enable_maintenance_manager);
 DECLARE_int32(heartbeat_interval_ms);
 DECLARE_int32(flush_threshold_mb);
@@ -95,7 +94,6 @@ class AlterTableTest : public KuduTest {
     b.AddColumn("c1")->Type(KuduColumnSchema::INT32)->NotNull();
     CHECK_OK(b.Build(&schema_));
 
-    FLAGS_enable_data_block_fsync = false; // Keep unit tests fast.
     FLAGS_use_hybrid_clock = false;
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/consistency-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/consistency-itest.cc b/src/kudu/integration-tests/consistency-itest.cc
index 10c0574..23a1f58 100644
--- a/src/kudu/integration-tests/consistency-itest.cc
+++ b/src/kudu/integration-tests/consistency-itest.cc
@@ -48,7 +48,6 @@
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_bool(enable_data_block_fsync);
 DECLARE_int32(heartbeat_interval_ms);
 DECLARE_int32(max_clock_sync_error_usec);
 DECLARE_int32(scanner_gc_check_interval_us);
@@ -79,8 +78,6 @@ class ConsistencyITest : public MiniClusterITestBase {
         table_name_("timestamp_propagation_test_table"),
         key_column_name_("key"),
         key_split_value_(8) {
-    // Keep unit tests fast.
-    FLAGS_enable_data_block_fsync = false;
 
     // Using the mock clock: need to advance the clock for tablet servers.
     FLAGS_use_mock_wall_clock = true;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/create-table-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/create-table-itest.cc b/src/kudu/integration-tests/create-table-itest.cc
index 81c1f7f..8267dff 100644
--- a/src/kudu/integration-tests/create-table-itest.cc
+++ b/src/kudu/integration-tests/create-table-itest.cc
@@ -127,10 +127,7 @@ TEST_F(CreateTableITest, TestCreateWhenMajorityOfReplicasFailCreation) {
 TEST_F(CreateTableITest, TestSpreadReplicasEvenly) {
   const int kNumServers = 10;
   const int kNumTablets = 20;
-  vector<string> ts_flags;
-  vector<string> master_flags;
-  ts_flags.push_back("--never_fsync"); // run faster on slow disks
-  NO_FATALS(StartCluster(ts_flags, master_flags, kNumServers));
+  NO_FATALS(StartCluster({}, {}, kNumServers));
 
   gscoped_ptr<client::KuduTableCreator> table_creator(client_->NewTableCreator());
   client::KuduSchema client_schema(client::KuduSchemaFromSchema(GetSimpleTestSchema()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/exactly_once_writes-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/exactly_once_writes-itest.cc b/src/kudu/integration-tests/exactly_once_writes-itest.cc
index 8841267..6b2acf5 100644
--- a/src/kudu/integration-tests/exactly_once_writes-itest.cc
+++ b/src/kudu/integration-tests/exactly_once_writes-itest.cc
@@ -283,7 +283,6 @@ TEST_F(ExactlyOnceSemanticsITest, TestWritesWithExactlyOnceSemanticsWithChurnyEl
 #endif
   ts_flags.push_back("--leader_failure_monitor_check_mean_ms=2");
   ts_flags.push_back("--leader_failure_monitor_check_stddev_ms=1");
-  ts_flags.push_back("--never_fsync");
 
   int num_batches = 200;
   if (AllowSlowTests()) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/external_mini_cluster-itest-base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster-itest-base.cc b/src/kudu/integration-tests/external_mini_cluster-itest-base.cc
index 657d4df..62a18a1 100644
--- a/src/kudu/integration-tests/external_mini_cluster-itest-base.cc
+++ b/src/kudu/integration-tests/external_mini_cluster-itest-base.cc
@@ -45,8 +45,6 @@ void ExternalMiniClusterITestBase::StartCluster(
   opts.num_tablet_servers = num_tablet_servers;
   opts.extra_master_flags = extra_master_flags;
   opts.extra_tserver_flags = extra_ts_flags;
-  // fsync causes flakiness on EC2.
-  opts.extra_tserver_flags.push_back("--never_fsync");
   cluster_.reset(new ExternalMiniCluster(opts));
   ASSERT_OK(cluster_->Start());
   inspect_.reset(new itest::ExternalMiniClusterFsInspector(cluster_.get()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/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 f4be5ad..ff26d1b 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -624,6 +624,10 @@ Status ExternalDaemon::StartProcess(const vector<string>& user_flags) {
   // Then all the flags coming from the minicluster framework.
   argv.insert(argv.end(), user_flags.begin(), user_flags.end());
 
+  // Disable fsync to dramatically speed up runtime. This is safe as no tests
+  // rely on forcefully cutting power to a machine or equivalent.
+  argv.push_back("--never_fsync");
+
   // Enable metrics logging.
   argv.push_back("--metrics_log_interval_ms=1000");
 
@@ -642,11 +646,6 @@ Status ExternalDaemon::StartProcess(const vector<string>& user_flags) {
     argv.push_back("--log_dir=" + data_dir_);
   }
 
-  // Then the "extra flags" passed into the ctor (from the ExternalMiniCluster
-  // options struct). These come at the end so they can override things like
-  // web port or RPC bind address if necessary.
-  argv.insert(argv.end(), extra_flags_.begin(), extra_flags_.end());
-
   // Tell the server to dump its port information so we can pick it up.
   string info_path = JoinPathSegments(data_dir_, "info.pb");
   argv.push_back("--server_dump_info_path=" + info_path);
@@ -656,15 +655,20 @@ Status ExternalDaemon::StartProcess(const vector<string>& user_flags) {
   // in unit tests.
   argv.push_back("--rpc_server_allow_ephemeral_ports");
 
-  // A previous instance of the daemon may have run in the same directory. So, remove
-  // the previous info file if it's there.
-  ignore_result(Env::Default()->DeleteFile(info_path));
-
   // Allow unsafe and experimental flags from tests, since we often use
   // fault injection, etc.
   argv.push_back("--unlock_experimental_flags");
   argv.push_back("--unlock_unsafe_flags");
 
+  // Then the "extra flags" passed into the ctor (from the ExternalMiniCluster
+  // options struct). These come at the end so they can override things like
+  // web port or RPC bind address if necessary.
+  argv.insert(argv.end(), extra_flags_.begin(), extra_flags_.end());
+
+  // A previous instance of the daemon may have run in the same directory. So, remove
+  // the previous info file if it's there.
+  ignore_result(Env::Default()->DeleteFile(info_path));
+
   gscoped_ptr<Subprocess> p(new Subprocess(exe_, argv));
   p->ShareParentStdout(false);
   p->SetEnvVars(extra_env_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/master-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master-stress-test.cc b/src/kudu/integration-tests/master-stress-test.cc
index a9b362c..b99ea5e 100644
--- a/src/kudu/integration-tests/master-stress-test.cc
+++ b/src/kudu/integration-tests/master-stress-test.cc
@@ -92,12 +92,6 @@ class MasterStressTest : public KuduTest {
     opts.num_masters = opts.master_rpc_ports.size();
     opts.num_tablet_servers = 3;
 
-    // Because this test performs a lot of DDL operations, we end up flushing
-    // and rewriting metadata files quite a bit. Globally disabling fsync
-    // speeds the test runtime up dramatically.
-    opts.extra_master_flags.push_back("--never_fsync");
-    opts.extra_tserver_flags.push_back("--never_fsync");
-
     // Don't preallocate log segments, since we're creating many tablets here.
     // If each preallocates 64M or so, we use a ton of disk space in this
     // test, and it fails on normal sized /tmp dirs.

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/raft_consensus-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc b/src/kudu/integration-tests/raft_consensus-itest.cc
index cf83164..231fbcb 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -960,7 +960,6 @@ void RaftConsensusITest::CreateClusterForChurnyElectionsTests(
 #endif
   ts_flags.push_back("--leader_failure_monitor_check_mean_ms=1");
   ts_flags.push_back("--leader_failure_monitor_check_stddev_ms=1");
-  ts_flags.push_back("--never_fsync");
 
   ts_flags.insert(ts_flags.end(), extra_ts_flags.cbegin(), extra_ts_flags.cend());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/ts_recovery-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/ts_recovery-itest.cc b/src/kudu/integration-tests/ts_recovery-itest.cc
index ffed6b8..6f73f6c 100644
--- a/src/kudu/integration-tests/ts_recovery-itest.cc
+++ b/src/kudu/integration-tests/ts_recovery-itest.cc
@@ -334,8 +334,6 @@ TEST_P(Kudu969Test, Test) {
   // concurrency bugs where a compaction and a flush might be happening
   // at the same time during the crash.
   opts.extra_tserver_flags.push_back("--maintenance_manager_num_threads=3");
-  // Speed up test by not fsyncing.
-  opts.extra_tserver_flags.push_back("--never_fsync");
   cluster_.reset(new ExternalMiniCluster(opts));
   ASSERT_OK(cluster_->Start());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/version_migration-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/version_migration-test.cc b/src/kudu/integration-tests/version_migration-test.cc
index 1d98468..87876f0 100644
--- a/src/kudu/integration-tests/version_migration-test.cc
+++ b/src/kudu/integration-tests/version_migration-test.cc
@@ -81,7 +81,6 @@ void VersionMigrationTest::StartCluster(const vector<string>& extra_ts_flags,
   opts.extra_master_flags = extra_master_flags;
   opts.extra_master_flags.push_back("--undefok=unlock_experimental_flags,unlock_unsafe_flags");
   opts.extra_tserver_flags = extra_ts_flags;
-  opts.extra_tserver_flags.push_back("--never_fsync"); // fsync causes flakiness on EC2.
   opts.extra_tserver_flags.push_back("--undefok=unlock_experimental_flags,unlock_unsafe_flags");
   cluster_.reset(new ExternalMiniCluster(opts));
   verifier_.reset(new LogVerifier(cluster_.get()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/integration-tests/webserver-stress-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/webserver-stress-itest.cc b/src/kudu/integration-tests/webserver-stress-itest.cc
index d108993..a473d02 100644
--- a/src/kudu/integration-tests/webserver-stress-itest.cc
+++ b/src/kudu/integration-tests/webserver-stress-itest.cc
@@ -45,8 +45,6 @@ TEST_F(KuduTest, TestWebUIDoesNotCrashCluster) {
   const int kNumTablets = 50;
 
   ExternalMiniClusterOptions opts;
-  opts.extra_master_flags.push_back("--never_fsync");
-  opts.extra_tserver_flags.push_back("--never_fsync");
   ExternalMiniCluster cluster(opts);
   ASSERT_OK(cluster.Start());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/tablet/compaction-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/compaction-test.cc b/src/kudu/tablet/compaction-test.cc
index dca1d80..55035d7 100644
--- a/src/kudu/tablet/compaction-test.cc
+++ b/src/kudu/tablet/compaction-test.cc
@@ -49,7 +49,6 @@ DEFINE_int32(merge_benchmark_num_rows_per_rowset, 500000,
              "Number of rowsets as input to the merge");
 
 DECLARE_string(block_manager);
-DECLARE_bool(enable_data_block_fsync);
 
 using std::shared_ptr;
 
@@ -1022,18 +1021,6 @@ TEST_F(TestCompaction, BenchmarkMergeWithOverlap) {
 #endif
 
 TEST_F(TestCompaction, TestCompactionFreesDiskSpace) {
-  // On RHEL 6.4 with an ext4 filesystem mounted as ext3, it was observed
-  // that freshly created files report st_blocks=0 via stat(2) for several
-  // seconds. This appears to be some buggy interaction with ext4 delalloc.
-  //
-  // Enabling data block fsync appears to work around the problem. We do
-  // that here and not for all tests because:
-  // 1. fsync is expensive, and
-  // 2. This is the only test that cares about disk space usage and can't
-  //    explicitly fsync() after writing new files.
-
-  FLAGS_enable_data_block_fsync = true;
-
   {
     // We must force the LocalTabletWriter out of scope before measuring
     // disk space usage. Otherwise some deleted blocks are kept open for

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/tablet/deltafile-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltafile-test.cc b/src/kudu/tablet/deltafile-test.cc
index f58754a..a550bbe 100644
--- a/src/kudu/tablet/deltafile-test.cc
+++ b/src/kudu/tablet/deltafile-test.cc
@@ -34,7 +34,6 @@ DEFINE_int32(first_row_to_update, 10000, "the first row to update");
 DEFINE_int32(last_row_to_update, 100000, "the last row to update");
 DEFINE_int32(n_verify, 1, "number of times to verify the updates"
              "(useful for benchmarks");
-DECLARE_bool(never_fsync);
 
 using std::is_sorted;
 using std::shared_ptr;
@@ -52,8 +51,6 @@ class TestDeltaFile : public KuduTest {
   TestDeltaFile() :
     schema_(CreateSchema()),
     arena_(1024, 1024) {
-    // Speed up unit tests.
-    FLAGS_never_fsync = true;
   }
 
  public:

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/tablet/memrowset-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/memrowset-test.cc b/src/kudu/tablet/memrowset-test.cc
index a5fd09c..2b223b6 100644
--- a/src/kudu/tablet/memrowset-test.cc
+++ b/src/kudu/tablet/memrowset-test.cc
@@ -29,7 +29,6 @@
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_macros.h"
 
-DECLARE_bool(enable_data_block_fsync);
 DEFINE_int32(roundtrip_num_rows, 10000,
              "Number of rows to use for the round-trip test");
 DEFINE_int32(num_scan_passes, 1,
@@ -42,7 +41,7 @@ using consensus::OpId;
 using log::LogAnchorRegistry;
 using std::shared_ptr;
 
-class TestMemRowSet : public ::testing::Test {
+class TestMemRowSet : public KuduTest {
  public:
   TestMemRowSet()
     : op_id_(consensus::MaximumOpId()),
@@ -50,7 +49,6 @@ class TestMemRowSet : public ::testing::Test {
       schema_(CreateSchema()),
       key_schema_(schema_.CreateKeyProjection()),
       clock_(server::LogicalClock::CreateStartingAt(Timestamp::kInitialTimestamp)) {
-    FLAGS_enable_data_block_fsync = false; // Keep unit tests fast.
   }
 
   static Schema CreateSchema() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/tablet/tablet-test-util.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet-test-util.h b/src/kudu/tablet/tablet-test-util.h
index 4db2113..8adc3ad 100644
--- a/src/kudu/tablet/tablet-test-util.h
+++ b/src/kudu/tablet/tablet-test-util.h
@@ -18,7 +18,6 @@
 #define KUDU_TABLET_TABLET_TEST_UTIL_H
 
 #include <algorithm>
-#include <gflags/gflags.h>
 #include <memory>
 #include <string>
 #include <vector>
@@ -34,8 +33,6 @@
 #include "kudu/util/metrics.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_bool(enable_data_block_fsync);
-
 namespace kudu {
 namespace tablet {
 
@@ -51,10 +48,6 @@ class KuduTabletTest : public KuduTest {
     : schema_(schema.CopyWithColumnIds()),
       client_schema_(schema),
       clock_type_(clock_type) {
-    // Keep unit tests fast, but only if no one has set the flag explicitly.
-    if (google::GetCommandLineFlagInfoOrDie("enable_data_block_fsync").is_default) {
-      FLAGS_enable_data_block_fsync = false;
-    }
   }
 
   virtual void SetUp() OVERRIDE {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/tools/kudu-tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 163ef63..ef2d8a8 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -854,8 +854,7 @@ TEST_F(ToolTest, TestLocalReplicaOps) {
 void ToolTest::RunLoadgen(int num_tservers,
                           const vector<string>& tool_args,
                           const string& table_name) {
-  // fsync causes flakiness on EC2
-  NO_FATALS(StartExternalMiniCluster({}, {"--never_fsync"}, num_tservers));
+  NO_FATALS(StartExternalMiniCluster({}, {}, num_tservers));
   if (!table_name.empty()) {
     static const string kKeyColumnName = "key";
     static const Schema kSchema = Schema(

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/tserver/tablet_server-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server-test-base.h b/src/kudu/tserver/tablet_server-test-base.h
index e245c3f..585e8bd 100644
--- a/src/kudu/tserver/tablet_server-test-base.h
+++ b/src/kudu/tserver/tablet_server-test-base.h
@@ -57,9 +57,7 @@
 
 DEFINE_int32(rpc_timeout, 1000, "Timeout for RPC calls, in seconds");
 DEFINE_int32(num_updater_threads, 1, "Number of updating threads to launch");
-DECLARE_bool(log_force_fsync_all);
 DECLARE_bool(enable_maintenance_manager);
-DECLARE_bool(enable_data_block_fsync);
 DECLARE_int32(heartbeat_rpc_timeout_ms);
 
 METRIC_DEFINE_entity(test);
@@ -85,11 +83,6 @@ class TabletServerTestBase : public KuduTest {
     // the hearbeat timeout to 1 second speeds up unit tests which
     // purposefully specify non-running Master servers.
     FLAGS_heartbeat_rpc_timeout_ms = 1000;
-
-    // Keep unit tests fast, but only if no one has set the flag explicitly.
-    if (google::GetCommandLineFlagInfoOrDie("enable_data_block_fsync").is_default) {
-      FLAGS_enable_data_block_fsync = false;
-    }
   }
 
   // Starts the tablet server, override to start it later.

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/util/atomic-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/atomic-test.cc b/src/kudu/util/atomic-test.cc
index ad754f3..9834d09 100644
--- a/src/kudu/util/atomic-test.cc
+++ b/src/kudu/util/atomic-test.cc
@@ -17,10 +17,13 @@
 
 #include "kudu/util/atomic.h"
 
-#include <gtest/gtest.h>
 #include <limits>
 #include <vector>
 
+#include <gtest/gtest.h>
+
+#include "kudu/util/test_util.h"
+
 namespace kudu {
 
 using std::numeric_limits;
@@ -32,7 +35,7 @@ using std::vector;
 // invariants are preserved in a multi-threaded environment.
 
 template<typename T>
-class AtomicIntTest : public ::testing::Test {
+class AtomicIntTest : public KuduTest {
  public:
 
   AtomicIntTest()

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/util/file_cache-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/file_cache-stress-test.cc b/src/kudu/util/file_cache-stress-test.cc
index 387936a..1abec57 100644
--- a/src/kudu/util/file_cache-stress-test.cc
+++ b/src/kudu/util/file_cache-stress-test.cc
@@ -63,8 +63,6 @@
     CHECK(_s.ok()) << "Bad status: " << _s.ToString();                    \
   } while (0);
 
-DECLARE_bool(never_fsync);
-
 // This default value is friendly to many n-CPU configurations.
 DEFINE_int32(test_max_open_files, 192, "Maximum number of open files enforced "
              "by the cache. Should be a multiple of the number of CPUs on the "
@@ -96,10 +94,6 @@ class FileCacheStressTest : public KuduTest {
                scoped_refptr<MetricEntity>()),
         rand_(SeedRandom()),
         running_(1) {
-
-    // Increases total number of files manipulated, which is the right kind
-    // of stress for this test.
-    FLAGS_never_fsync = true;
   }
 
   void SetUp() override {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/util/flag_tags-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/flag_tags-test.cc b/src/kudu/util/flag_tags-test.cc
index 59d7437..3f6849e 100644
--- a/src/kudu/util/flag_tags-test.cc
+++ b/src/kudu/util/flag_tags-test.cc
@@ -25,6 +25,8 @@
 #include "kudu/util/logging_test_util.h"
 #include "kudu/util/test_util.h"
 
+DECLARE_bool(never_fsync);
+
 DEFINE_int32(flag_with_no_tags, 0, "test flag that has no tags");
 
 DEFINE_int32(flag_with_one_tag, 0, "test flag that has 1 tag");
@@ -46,6 +48,12 @@ using std::unordered_set;
 namespace kudu {
 
 class FlagTagsTest : public KuduTest {
+ public:
+  FlagTagsTest() {
+    // Set to true via KuduTest, but explicitly unset here as this test deals
+    // with unsafe flags.
+    FLAGS_never_fsync = false;
+  }
 };
 
 TEST_F(FlagTagsTest, TestTags) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/util/flags-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/flags-test.cc b/src/kudu/util/flags-test.cc
index 440c9f4..1bbfc38 100644
--- a/src/kudu/util/flags-test.cc
+++ b/src/kudu/util/flags-test.cc
@@ -33,12 +33,17 @@ DEFINE_string(test_default_ff, "default",
              "Check if we track defaults from flagfile");
 DEFINE_string(test_default_explicit, "default",
              "Check if we track explicitly set defaults");
-DECLARE_string(flagfile);
-
+DECLARE_bool(never_fsync);
 
 namespace kudu {
 
 class FlagsTest : public KuduTest {
+ public:
+  FlagsTest() {
+    // Set to true via KuduTest, but explicitly unset here as this test parses
+    // command line flags and fails if an unsafe flag is set.
+    FLAGS_never_fsync = false;
+  }
 };
 
 TEST_F(FlagsTest, TestNonDefaultFlags) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c681f934/src/kudu/util/test_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/test_util.cc b/src/kudu/util/test_util.cc
index 4027872..855330e 100644
--- a/src/kudu/util/test_util.cc
+++ b/src/kudu/util/test_util.cc
@@ -33,6 +33,8 @@
 #include "kudu/util/random.h"
 #include "kudu/util/spinlock_profiling.h"
 
+DECLARE_bool(never_fsync);
+
 DEFINE_string(test_leave_files, "on_failure",
               "Whether to leave test files around after the test run. "
               " Valid values are 'always', 'on_failure', or 'never'");
@@ -64,6 +66,9 @@ bool g_is_gtest = true;
 KuduTest::KuduTest()
   : env_(Env::Default()),
     test_dir_(GetTestDataDirectory()) {
+  // Disabling fsync() speeds up tests dramatically, and it's safe to do as no
+  // tests rely on cutting power to a machine or equivalent.
+  FLAGS_never_fsync = true;
 }
 
 KuduTest::~KuduTest() {