You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2018/07/24 04:51:31 UTC

[1/2] kudu git commit: KUDU-2509 fix use-after-free in case of WAL replay error

Repository: kudu
Updated Branches:
  refs/heads/master ec1e47f41 -> 5b09a693d


KUDU-2509 fix use-after-free in case of WAL replay error

Fixed use-after-free mistake in case of a failure to apply a pending
commit message from the WAL while bootstrapping a tablet.

Also, a repro scenario to expose the use-after-free condition is added.
Prior to the fix, the repro scenario would crash with SIGSEGV on Linux
or with SIGBUS on OS X (at least for DEBUG builds).

Change-Id: I11373b1cc34d9e2e0181bee2d3841b49022218ed
Reviewed-on: http://gerrit.cloudera.org:8080/10997
Tested-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: 6b429e8a42ad9fb12a97cc26e33ca19ac2626533
Parents: ec1e47f
Author: Alexey Serbin <as...@cloudera.com>
Authored: Thu Jul 19 21:05:54 2018 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Tue Jul 24 04:40:20 2018 +0000

----------------------------------------------------------------------
 src/kudu/tablet/tablet_bootstrap-test.cc | 75 ++++++++++++++++++++++++++-
 src/kudu/tablet/tablet_bootstrap.cc      |  2 +-
 2 files changed, 74 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/6b429e8a/src/kudu/tablet/tablet_bootstrap-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap-test.cc b/src/kudu/tablet/tablet_bootstrap-test.cc
index 0b80c20..f5b2668 100644
--- a/src/kudu/tablet/tablet_bootstrap-test.cc
+++ b/src/kudu/tablet/tablet_bootstrap-test.cc
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "kudu/consensus/log-test-base.h"
+#include "kudu/tablet/tablet_bootstrap.h"
 
 #include <cstddef>
 #include <cstdint>
@@ -35,7 +35,9 @@
 #include "kudu/clock/logical_clock.h"
 #include "kudu/common/common.pb.h"
 #include "kudu/common/iterator.h"
+#include "kudu/common/partial_row.h"
 #include "kudu/common/partition.h"
+#include "kudu/common/row_operations.h"
 #include "kudu/common/schema.h"
 #include "kudu/common/timestamp.h"
 #include "kudu/common/wire_protocol-test-util.h"
@@ -45,6 +47,7 @@
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/consensus/consensus_meta.h"
 #include "kudu/consensus/consensus_meta_manager.h"
+#include "kudu/consensus/log-test-base.h"
 #include "kudu/consensus/log.h"
 #include "kudu/consensus/log_anchor_registry.h"
 #include "kudu/consensus/log_reader.h"
@@ -67,7 +70,6 @@
 #include "kudu/tablet/tablet-test-util.h"
 #include "kudu/tablet/tablet.h"
 #include "kudu/tablet/tablet.pb.h"
-#include "kudu/tablet/tablet_bootstrap.h"
 #include "kudu/tablet/tablet_metadata.h"
 #include "kudu/tablet/tablet_replica.h"
 #include "kudu/tserver/tserver.pb.h"
@@ -675,5 +677,74 @@ TEST_F(BootstrapTest, TestConsensusOnlyOperationOutOfOrderTimestamp) {
   ASSERT_EQ(1, results.size());
 }
 
+// Regression test for KUDU-2509. There was a use-after-free bug that sometimes
+// lead to SIGSEGV while replaying the WAL. This scenario would crash or
+// at least UB sanitizer would report a warning if such condition exists.
+TEST_F(BootstrapTest, TestKudu2509) {
+  ASSERT_OK(BuildLog());
+
+  consensus::ReplicateRefPtr replicate = consensus::make_scoped_refptr_replicate(
+      new consensus::ReplicateMsg());
+  replicate->get()->set_op_type(consensus::WRITE_OP);
+  tserver::WriteRequestPB* batch_request = replicate->get()->mutable_write_request();
+  ASSERT_OK(SchemaToPB(schema_, batch_request->mutable_schema()));
+  batch_request->set_tablet_id(log::kTestTablet);
+
+  // This appends Insert(1) with op 10.10
+  const OpId insert_opid = MakeOpId(10, 10);
+  replicate->get()->mutable_id()->CopyFrom(insert_opid);
+  replicate->get()->set_timestamp(clock_->Now().ToUint64());
+  AddTestRowToPB(RowOperationsPB::INSERT, schema_, 10, 1,
+                 "this is a test insert", batch_request->mutable_row_operations());
+  NO_FATALS(AppendReplicateBatch(replicate));
+
+  // This appends Mutate(1) with op 10.11. The operation would try to update
+  // a row having an extra column. This should fail since there hasn't been
+  // corresponding DDL operation committed yet.
+  const OpId mutate_opid = MakeOpId(10, 11);
+  batch_request->mutable_row_operations()->Clear();
+  replicate->get()->mutable_id()->CopyFrom(mutate_opid);
+  replicate->get()->set_timestamp(clock_->Now().ToUint64());
+  {
+    // Modify the existing schema to add an extra row.
+    SchemaBuilder builder(schema_);
+    ASSERT_OK(builder.AddNullableColumn("string_val_extra", STRING));
+    const auto schema = builder.BuildWithoutIds();
+    ASSERT_OK(SchemaToPB(schema, batch_request->mutable_schema()));
+
+    KuduPartialRow row(&schema);
+    ASSERT_OK(row.SetInt32("key", 100));
+    ASSERT_OK(row.SetInt32("int_val", 200));
+    ASSERT_OK(row.SetStringCopy("string_val", "300"));
+    ASSERT_OK(row.SetStringCopy("string_val_extra", "100500"));
+    RowOperationsPBEncoder enc(batch_request->mutable_row_operations());
+    enc.Add(RowOperationsPB::UPDATE, row);
+  }
+  NO_FATALS(AppendReplicateBatch(replicate));
+
+  // Now commit the mutate before the insert (in the log).
+  gscoped_ptr<consensus::CommitMsg> mutate_commit(new consensus::CommitMsg);
+  mutate_commit->set_op_type(consensus::WRITE_OP);
+  mutate_commit->mutable_commited_op_id()->CopyFrom(mutate_opid);
+  mutate_commit->mutable_result()->add_ops()->add_mutated_stores()->set_mrs_id(1);
+  NO_FATALS(AppendCommit(std::move(mutate_commit)));
+
+  gscoped_ptr<consensus::CommitMsg> insert_commit(new consensus::CommitMsg);
+  insert_commit->set_op_type(consensus::WRITE_OP);
+  insert_commit->mutable_commited_op_id()->CopyFrom(insert_opid);
+  insert_commit->mutable_result()->add_ops()->add_mutated_stores()->set_mrs_id(1);
+  NO_FATALS(AppendCommit(std::move(insert_commit)));
+
+  ConsensusBootstrapInfo boot_info;
+  shared_ptr<Tablet> tablet;
+  const auto s = BootstrapTestTablet(-1, -1, &tablet, &boot_info);
+  const auto& status_msg = s.ToString();
+  ASSERT_TRUE(s.IsInvalidArgument()) << status_msg;
+  ASSERT_STR_CONTAINS(status_msg,
+      "Unable to bootstrap test tablet: Failed log replay.");
+  ASSERT_STR_CONTAINS(status_msg,
+      "column string_val_extra[string NULLABLE] not present in tablet");
+}
+
 } // namespace tablet
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/6b429e8a/src/kudu/tablet/tablet_bootstrap.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc
index 58f9f54..fcc472a 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -946,7 +946,6 @@ Status TabletBootstrap::HandleCommitMessage(ReplayState* state, LogEntryPB* comm
   // ... if it does, we apply it and all the commits that immediately follow in the sequence.
   OpId last_applied = commit_entry->commit().commited_op_id();
   RETURN_NOT_OK(ApplyCommitMessage(state, commit_entry));
-  delete commit_entry;
 
   auto iter = state->pending_commits.begin();
   while (iter != state->pending_commits.end()) {
@@ -960,6 +959,7 @@ Status TabletBootstrap::HandleCommitMessage(ReplayState* state, LogEntryPB* comm
     break;
   }
 
+  delete commit_entry;
   return Status::OK();
 }
 


[2/2] kudu git commit: [log-test] clean-up on WAL entry test operations

Posted by al...@apache.org.
[log-test] clean-up on WAL entry test operations

Change-Id: I01ef7798c63c18cb03df5969fc73bbb0ec65111b
Reviewed-on: http://gerrit.cloudera.org:8080/11028
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 5b09a693dd651203512abf9faaddcc21aad92e59
Parents: 6b429e8
Author: Alexey Serbin <as...@cloudera.com>
Authored: Mon Jul 23 18:20:47 2018 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Tue Jul 24 04:42:49 2018 +0000

----------------------------------------------------------------------
 src/kudu/consensus/log-test-base.h       | 97 ++++++++++++++-------------
 src/kudu/consensus/log-test.cc           | 22 +++---
 src/kudu/tablet/tablet_bootstrap-test.cc | 69 ++++++++++---------
 3 files changed, 95 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/5b09a693/src/kudu/consensus/log-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log-test-base.h b/src/kudu/consensus/log-test-base.h
index 5239ea2..987b769 100644
--- a/src/kudu/consensus/log-test-base.h
+++ b/src/kudu/consensus/log-test-base.h
@@ -65,10 +65,10 @@ constexpr bool APPEND_ASYNC = false;
 // If 'size' is not NULL, increments it by the expected increase in log size.
 // Increments 'op_id''s index once for each operation logged.
 inline Status AppendNoOpsToLogSync(const scoped_refptr<clock::Clock>& clock,
-                            Log* log,
-                            consensus::OpId* op_id,
-                            int count,
-                            int* size = NULL) {
+                                   Log* log,
+                                   consensus::OpId* op_id,
+                                   int count,
+                                   int* size = nullptr) {
 
   std::vector<consensus::ReplicateRefPtr> replicates;
   for (int i = 0; i < count; i++) {
@@ -103,9 +103,9 @@ inline Status AppendNoOpsToLogSync(const scoped_refptr<clock::Clock>& clock,
 }
 
 inline Status AppendNoOpToLogSync(const scoped_refptr<clock::Clock>& clock,
-                           Log* log,
-                           consensus::OpId* op_id,
-                           int* size = NULL) {
+                                  Log* log,
+                                  consensus::OpId* op_id,
+                                  int* size = nullptr) {
   return AppendNoOpsToLogSync(clock, log, op_id, 1, size);
 }
 
@@ -145,11 +145,11 @@ class LogTestBase : public KuduTest {
   typedef std::pair<int, int> DeltaId;
 
   LogTestBase()
-    : schema_(GetSimpleTestSchema()),
-      log_anchor_registry_(new LogAnchorRegistry()) {
+      : schema_(GetSimpleTestSchema()),
+        log_anchor_registry_(new LogAnchorRegistry) {
   }
 
-  virtual void SetUp() OVERRIDE {
+  void SetUp() override {
     KuduTest::SetUp();
     current_index_ = kStartIndex;
     fs_manager_.reset(new FsManager(env_, GetTestPath("fs_root")));
@@ -162,7 +162,7 @@ class LogTestBase : public KuduTest {
     ASSERT_OK(clock_->Init());
   }
 
-  virtual void TearDown() OVERRIDE {
+  void TearDown() override {
     KuduTest::TearDown();
     STLDeleteElements(&entries_);
   }
@@ -209,14 +209,15 @@ class LogTestBase : public KuduTest {
   }
 
   // Appends a batch with size 2 (1 insert, 1 mutate) to the log.
-  void AppendReplicateBatch(const consensus::OpId& opid, bool sync = APPEND_SYNC) {
+  Status AppendReplicateBatch(const consensus::OpId& opid,
+                              bool sync = APPEND_SYNC) {
     consensus::ReplicateRefPtr replicate =
         make_scoped_refptr_replicate(new consensus::ReplicateMsg());
     replicate->get()->set_op_type(consensus::WRITE_OP);
     replicate->get()->mutable_id()->CopyFrom(opid);
     replicate->get()->set_timestamp(clock_->Now().ToUint64());
     tserver::WriteRequestPB* batch_request = replicate->get()->mutable_write_request();
-    ASSERT_OK(SchemaToPB(schema_, batch_request->mutable_schema()));
+    RETURN_NOT_OK(SchemaToPB(schema_, batch_request->mutable_schema()));
     AddTestRowToPB(RowOperationsPB::INSERT, schema_,
                    opid.index(),
                    0,
@@ -228,22 +229,21 @@ class LogTestBase : public KuduTest {
                    "this is a test mutate",
                    batch_request->mutable_row_operations());
     batch_request->set_tablet_id(kTestTablet);
-    AppendReplicateBatch(replicate, sync);
+    return AppendReplicateBatch(replicate, sync);
   }
 
   // Appends the provided batch to the log.
-  void AppendReplicateBatch(const consensus::ReplicateRefPtr& replicate,
-                            bool sync = APPEND_SYNC) {
+  Status AppendReplicateBatch(const consensus::ReplicateRefPtr& replicate,
+                              bool sync = APPEND_SYNC) {
     if (sync) {
       Synchronizer s;
-      ASSERT_OK(log_->AsyncAppendReplicates({ replicate }, s.AsStatusCallback()));
-      ASSERT_OK(s.Wait());
-    } else {
-      // AsyncAppendReplicates does not free the ReplicateMsg on completion, so we
-      // need to pass it through to our callback.
-      ASSERT_OK(log_->AsyncAppendReplicates({ replicate },
-                                            Bind(&LogTestBase::CheckReplicateResult, replicate)));
+      RETURN_NOT_OK(log_->AsyncAppendReplicates({ replicate }, s.AsStatusCallback()));
+      return s.Wait();
     }
+    // AsyncAppendReplicates does not free the ReplicateMsg on completion, so we
+    // need to pass it through to our callback.
+    return log_->AsyncAppendReplicates(
+        { replicate }, Bind(&LogTestBase::CheckReplicateResult, replicate));
   }
 
   static void CheckCommitResult(const Status& s) {
@@ -252,21 +252,23 @@ class LogTestBase : public KuduTest {
 
   // Append a commit log entry containing one entry for the insert and one
   // for the mutate.
-  void AppendCommit(const consensus::OpId& original_opid,
-                    bool sync = APPEND_SYNC) {
+  Status AppendCommit(const consensus::OpId& original_opid,
+                      bool sync = APPEND_SYNC) {
     // The mrs id for the insert.
-    const int kTargetMrsId = 1;
+    constexpr int kTargetMrsId = 1;
 
     // The rs and delta ids for the mutate.
-    const int kTargetRsId = 0;
-    const int kTargetDeltaId = 0;
+    constexpr int kTargetRsId = 0;
+    constexpr int kTargetDeltaId = 0;
 
-    AppendCommit(original_opid, kTargetMrsId, kTargetRsId, kTargetDeltaId, sync);
+    return AppendCommit(original_opid, kTargetMrsId, kTargetRsId, kTargetDeltaId, sync);
   }
 
-  void AppendCommit(const consensus::OpId& original_opid,
-                    int mrs_id, int rs_id, int dms_id,
-                    bool sync = APPEND_SYNC) {
+  Status AppendCommit(const consensus::OpId& original_opid,
+                      int mrs_id,
+                      int rs_id,
+                      int dms_id,
+                      bool sync = APPEND_SYNC) {
     gscoped_ptr<consensus::CommitMsg> commit(new consensus::CommitMsg);
     commit->set_op_type(consensus::WRITE_OP);
 
@@ -281,16 +283,15 @@ class LogTestBase : public KuduTest {
     tablet::MemStoreTargetPB* target = mutate->add_mutated_stores();
     target->set_dms_id(dms_id);
     target->set_rs_id(rs_id);
-    AppendCommit(std::move(commit), sync);
+    return AppendCommit(std::move(commit), sync);
   }
 
   // Append a COMMIT message for 'original_opid', but with results
   // indicating that the associated writes failed due to
   // "NotFound" errors.
-  void AppendCommitWithNotFoundOpResults(const consensus::OpId& original_opid) {
+  Status AppendCommitWithNotFoundOpResults(const consensus::OpId& original_opid) {
     gscoped_ptr<consensus::CommitMsg> commit(new consensus::CommitMsg);
     commit->set_op_type(consensus::WRITE_OP);
-
     commit->mutable_commited_op_id()->CopyFrom(original_opid);
 
     tablet::TxResultPB* result = commit->mutable_result();
@@ -300,34 +301,36 @@ class LogTestBase : public KuduTest {
     tablet::OperationResultPB* mutate = result->add_ops();
     StatusToPB(Status::NotFound("fake failed write"), mutate->mutable_failed_status());
 
-    AppendCommit(std::move(commit));
+    return AppendCommit(std::move(commit));
   }
 
-  void AppendCommit(gscoped_ptr<consensus::CommitMsg> commit, bool sync = APPEND_SYNC) {
+  Status AppendCommit(gscoped_ptr<consensus::CommitMsg> commit,
+                      bool sync = APPEND_SYNC) {
     if (sync) {
       Synchronizer s;
-      ASSERT_OK(log_->AsyncAppendCommit(std::move(commit), s.AsStatusCallback()));
-      ASSERT_OK(s.Wait());
-    } else {
-      ASSERT_OK(log_->AsyncAppendCommit(std::move(commit),
-                                               Bind(&LogTestBase::CheckCommitResult)));
+      RETURN_NOT_OK(log_->AsyncAppendCommit(std::move(commit), s.AsStatusCallback()));
+      return s.Wait();
     }
+    return log_->AsyncAppendCommit(std::move(commit),
+                                   Bind(&LogTestBase::CheckCommitResult));
   }
 
     // Appends 'count' ReplicateMsgs and the corresponding CommitMsgs to the log
-  void AppendReplicateBatchAndCommitEntryPairsToLog(int count, bool sync = true) {
+  Status AppendReplicateBatchAndCommitEntryPairsToLog(int count,
+                                                      bool sync = APPEND_SYNC) {
     for (int i = 0; i < count; i++) {
       consensus::OpId opid = consensus::MakeOpId(1, current_index_);
-      AppendReplicateBatch(opid);
-      AppendCommit(opid, sync);
+      RETURN_NOT_OK(AppendReplicateBatch(opid));
+      RETURN_NOT_OK(AppendCommit(opid, sync));
       current_index_ += 1;
     }
+    return Status::OK();
   }
 
   // Append a single NO_OP entry. Increments op_id by one.
   // If non-NULL, and if the write is successful, 'size' is incremented
   // by the size of the written operation.
-  Status AppendNoOp(consensus::OpId* op_id, int* size = NULL) {
+  Status AppendNoOp(consensus::OpId* op_id, int* size = nullptr) {
     return AppendNoOpToLogSync(clock_, log_.get(), op_id, size);
   }
 
@@ -335,7 +338,7 @@ class LogTestBase : public KuduTest {
   // Increments op_id's index by the number of records written.
   // If non-NULL, 'size' keeps track of the size of the operations
   // successfully written.
-  Status AppendNoOps(consensus::OpId* op_id, int num, int* size = NULL) {
+  Status AppendNoOps(consensus::OpId* op_id, int num, int* size = nullptr) {
     for (int i = 0; i < num; i++) {
       RETURN_NOT_OK(AppendNoOp(op_id, size));
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/5b09a693/src/kudu/consensus/log-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log-test.cc b/src/kudu/consensus/log-test.cc
index 24fcc29..e92f112 100644
--- a/src/kudu/consensus/log-test.cc
+++ b/src/kudu/consensus/log-test.cc
@@ -15,9 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/consensus/log.h"
+
 #include <algorithm>
 #include <cerrno>
-#include <cstddef>
 #include <cstdint>
 #include <limits>
 #include <memory>
@@ -36,7 +37,6 @@
 #include "kudu/common/wire_protocol.h"
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/consensus/log-test-base.h"
-#include "kudu/consensus/log.h"
 #include "kudu/consensus/log.pb.h"
 #include "kudu/consensus/log_anchor_registry.h"
 #include "kudu/consensus/log_index.h"
@@ -408,7 +408,7 @@ TEST_P(LogTestOptionalCompression, TestSegmentRollover) {
   ASSERT_OK(log_->Close());
 
   shared_ptr<LogReader> reader;
-  ASSERT_OK(LogReader::Open(fs_manager_.get(), NULL, kTestTablet, NULL, &reader));
+  ASSERT_OK(LogReader::Open(fs_manager_.get(), nullptr, kTestTablet, nullptr, &reader));
   ASSERT_OK(reader->GetSegmentsSnapshot(&segments));
 
   ASSERT_TRUE(segments.back()->HasFooter());
@@ -589,7 +589,7 @@ TEST_P(LogTestOptionalCompression, TestGCWithLogRunning) {
   }
 
   ASSERT_OK(log_->Close());
-  CheckRightNumberOfSegmentFiles(2);
+  NO_FATALS(CheckRightNumberOfSegmentFiles(2));
 
   // We skip the first three, since we unregistered them above.
   for (int i = 3; i < kNumTotalSegments; i++) {
@@ -644,7 +644,7 @@ TEST_P(LogTestOptionalCompression, TestGCOfIndexChunks) {
 TEST_P(LogTestOptionalCompression, TestWaitUntilAllFlushed) {
   ASSERT_OK(BuildLog());
   // Append 2 replicate/commit pairs asynchronously
-  AppendReplicateBatchAndCommitEntryPairsToLog(2, APPEND_ASYNC);
+  ASSERT_OK(AppendReplicateBatchAndCommitEntryPairsToLog(2, APPEND_ASYNC));
 
   ASSERT_OK(log_->WaitUntilAllFlushed());
 
@@ -715,7 +715,7 @@ TEST_P(LogTestOptionalCompression, TestLogReopenAndGC) {
   retention.for_peers = 0;
   ASSERT_OK(log_->GC(retention, &num_gced_segments));
   ASSERT_EQ(0, num_gced_segments);
-  CheckRightNumberOfSegmentFiles(4);
+  NO_FATALS(CheckRightNumberOfSegmentFiles(4));
 
   // Set the max segments to retain so that, even though we have peers who need
   // the segments, we'll GC them.
@@ -729,7 +729,7 @@ TEST_P(LogTestOptionalCompression, TestLogReopenAndGC) {
   ASSERT_EQ(2, segments.size()) << DumpSegmentsToString(segments);
   ASSERT_OK(log_->Close());
 
-  CheckRightNumberOfSegmentFiles(2);
+  NO_FATALS(CheckRightNumberOfSegmentFiles(2));
 
   // Unregister the final anchor.
   ASSERT_OK(log_anchor_registry_->Unregister(anchors[3]));
@@ -745,7 +745,7 @@ TEST_P(LogTestOptionalCompression, TestWriteManyBatches) {
 
   LOG(INFO)<< "Starting to write " << num_batches << " to log";
   LOG_TIMING(INFO, "Wrote all batches to log") {
-    AppendReplicateBatchAndCommitEntryPairsToLog(num_batches);
+    ASSERT_OK(AppendReplicateBatchAndCommitEntryPairsToLog(num_batches));
   }
   ASSERT_OK(log_->Close());
   LOG(INFO) << "Done writing";
@@ -757,7 +757,7 @@ TEST_P(LogTestOptionalCompression, TestWriteManyBatches) {
     vector<scoped_refptr<ReadableLogSegment> > segments;
 
     shared_ptr<LogReader> reader;
-    ASSERT_OK(LogReader::Open(fs_manager_.get(), NULL, kTestTablet, NULL, &reader));
+    ASSERT_OK(LogReader::Open(fs_manager_.get(), nullptr, kTestTablet, nullptr, &reader));
     ASSERT_OK(reader->GetSegmentsSnapshot(&segments));
 
     for (const scoped_refptr<ReadableLogSegment>& entry : segments) {
@@ -810,8 +810,8 @@ TEST_P(LogTestOptionalCompression, TestLogReaderReturnsLatestSegmentIfIndexEmpty
   ASSERT_OK(BuildLog());
 
   OpId opid = MakeOpId(1, 1);
-  AppendCommit(opid, APPEND_ASYNC);
-  AppendReplicateBatch(opid, APPEND_SYNC);
+  ASSERT_OK(AppendCommit(opid, APPEND_ASYNC));
+  ASSERT_OK(AppendReplicateBatch(opid, APPEND_SYNC));
 
   SegmentSequence segments;
   ASSERT_OK(log_->reader()->GetSegmentsSnapshot(&segments));

http://git-wip-us.apache.org/repos/asf/kudu/blob/5b09a693/src/kudu/tablet/tablet_bootstrap-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap-test.cc b/src/kudu/tablet/tablet_bootstrap-test.cc
index f5b2668..05a55a0 100644
--- a/src/kudu/tablet/tablet_bootstrap-test.cc
+++ b/src/kudu/tablet/tablet_bootstrap-test.cc
@@ -17,7 +17,6 @@
 
 #include "kudu/tablet/tablet_bootstrap.h"
 
-#include <cstddef>
 #include <cstdint>
 #include <limits>
 #include <memory>
@@ -172,7 +171,7 @@ class BootstrapTest : public LogTestBase {
         scoped_refptr<Clock>(LogicalClock::CreateStartingAt(Timestamp::kInitialTimestamp)),
         shared_ptr<MemTracker>(),
         scoped_refptr<rpc::ResultTracker>(),
-        NULL,
+        nullptr,
         nullptr, // no status listener
         tablet,
         &log_,
@@ -220,10 +219,10 @@ class BootstrapTest : public LogTestBase {
 TEST_F(BootstrapTest, TestBootstrap) {
   ASSERT_OK(BuildLog());
 
-  AppendReplicateBatch(MakeOpId(1, current_index_));
+  ASSERT_OK(AppendReplicateBatch(MakeOpId(1, current_index_)));
   ASSERT_OK(RollLog());
 
-  AppendCommit(MakeOpId(1, current_index_));
+  ASSERT_OK(AppendCommit(MakeOpId(1, current_index_)));
 
   shared_ptr<Tablet> tablet;
   ConsensusBootstrapInfo boot_info;
@@ -299,11 +298,11 @@ TEST_F(BootstrapTest, TestOrphanCommit) {
   OpId opid = MakeOpId(1, current_index_);
 
   // Step 1) Write a REPLICATE to the log, and roll it.
-  AppendReplicateBatch(opid);
+  ASSERT_OK(AppendReplicateBatch(opid));
   ASSERT_OK(RollLog());
 
   // Step 2) Write the corresponding COMMIT in the second segment.
-  AppendCommit(opid);
+  ASSERT_OK(AppendCommit(opid));
 
   scoped_refptr<TabletMetadata> meta;
   ASSERT_OK(LoadTestTabletMetadata(/*mrs_id=*/ -1, /*delta_id=*/ -1, &meta));
@@ -324,7 +323,7 @@ TEST_F(BootstrapTest, TestOrphanCommit) {
     // Step 4) Create an orphaned commit by first adding a commit to
     // the newly rolled logfile, and then by removing the previous
     // commits.
-    AppendCommit(opid);
+    ASSERT_OK(AppendCommit(opid));
     log::SegmentSequence segments;
     ASSERT_OK(log_->reader()->GetSegmentsSnapshot(&segments));
     fs_manager_->env()->DeleteFile(segments[0]->path());
@@ -362,9 +361,9 @@ TEST_F(BootstrapTest, TestPendingFailedCommit) {
 
   // Step 2) Write the corresponding COMMIT in the second segment,
   // with a status indicating that the writes had 'NotFound' results.
-  AppendReplicateBatch(opid_1);
-  AppendReplicateBatch(opid_2);
-  AppendCommitWithNotFoundOpResults(opid_2);
+  ASSERT_OK(AppendReplicateBatch(opid_1));
+  ASSERT_OK(AppendReplicateBatch(opid_2));
+  ASSERT_OK(AppendCommitWithNotFoundOpResults(opid_2));
 
   {
     shared_ptr<Tablet> tablet;
@@ -388,10 +387,10 @@ TEST_F(BootstrapTest, TestNonOrphansAfterOrphanCommit) {
 
   OpId opid = MakeOpId(1, current_index_);
 
-  AppendReplicateBatch(opid);
+  ASSERT_OK(AppendReplicateBatch(opid));
   ASSERT_OK(RollLog());
 
-  AppendCommit(opid);
+  ASSERT_OK(AppendCommit(opid));
 
   log::SegmentSequence segments;
   ASSERT_OK(log_->reader()->GetSegmentsSnapshot(&segments));
@@ -401,8 +400,8 @@ TEST_F(BootstrapTest, TestNonOrphansAfterOrphanCommit) {
 
   opid = MakeOpId(1, current_index_);
 
-  AppendReplicateBatch(opid);
-  AppendCommit(opid, 2, 1, 0);
+  ASSERT_OK(AppendReplicateBatch(opid));
+  ASSERT_OK(AppendCommit(opid, 2, 1, 0));
 
   shared_ptr<Tablet> tablet;
   ConsensusBootstrapInfo boot_info;
@@ -430,7 +429,7 @@ TEST_F(BootstrapTest, TestOrphanedReplicate) {
 
   OpId opid = MakeOpId(1, replicate_index);
 
-  AppendReplicateBatch(opid);
+  ASSERT_OK(AppendReplicateBatch(opid));
 
   // Bootstrap the tablet. It shouldn't replay anything.
   ConsensusBootstrapInfo boot_info;
@@ -472,18 +471,18 @@ TEST_F(BootstrapTest, TestOperationOverwriting) {
   OpId opid = MakeOpId(1, 1);
 
   // Append a replicate in term 1
-  AppendReplicateBatch(opid);
+  ASSERT_OK(AppendReplicateBatch(opid));
 
   // Append a commit for op 1.1
-  AppendCommit(opid);
+  ASSERT_OK(AppendCommit(opid));
 
   // Now append replicates for 4.2 and 4.3
-  AppendReplicateBatch(MakeOpId(4, 2));
-  AppendReplicateBatch(MakeOpId(4, 3));
+  ASSERT_OK(AppendReplicateBatch(MakeOpId(4, 2)));
+  ASSERT_OK(AppendReplicateBatch(MakeOpId(4, 3)));
 
   ASSERT_OK(RollLog());
   // And overwrite with 3.2
-  AppendReplicateBatch(MakeOpId(3, 2), true);
+  ASSERT_OK(AppendReplicateBatch(MakeOpId(3, 2)));
 
   // When bootstrapping we should apply ops 1.1 and get 3.2 as pending.
   ConsensusBootstrapInfo boot_info;
@@ -520,7 +519,7 @@ TEST_F(BootstrapTest, TestOutOfOrderCommits) {
   replicate->get()->set_timestamp(clock_->Now().ToUint64());
   AddTestRowToPB(RowOperationsPB::INSERT, schema_, 10, 1,
                  "this is a test insert", batch_request->mutable_row_operations());
-  AppendReplicateBatch(replicate, true);
+  ASSERT_OK(AppendReplicateBatch(replicate));
 
   // This appends Mutate(1) with op 10.11
   OpId mutate_opid = MakeOpId(10, 11);
@@ -530,7 +529,7 @@ TEST_F(BootstrapTest, TestOutOfOrderCommits) {
   AddTestRowToPB(RowOperationsPB::UPDATE, schema_,
                  10, 2, "this is a test mutate",
                  batch_request->mutable_row_operations());
-  AppendReplicateBatch(replicate, true);
+  ASSERT_OK(AppendReplicateBatch(replicate));
 
   // Now commit the mutate before the insert (in the log).
   gscoped_ptr<consensus::CommitMsg> mutate_commit(new consensus::CommitMsg);
@@ -541,7 +540,7 @@ TEST_F(BootstrapTest, TestOutOfOrderCommits) {
   MemStoreTargetPB* target = mutate->add_mutated_stores();
   target->set_mrs_id(1);
 
-  AppendCommit(std::move(mutate_commit));
+  ASSERT_OK(AppendCommit(std::move(mutate_commit)));
 
   gscoped_ptr<consensus::CommitMsg> insert_commit(new consensus::CommitMsg);
   insert_commit->set_op_type(consensus::WRITE_OP);
@@ -551,7 +550,7 @@ TEST_F(BootstrapTest, TestOutOfOrderCommits) {
   target = insert->add_mutated_stores();
   target->set_mrs_id(1);
 
-  AppendCommit(std::move(insert_commit));
+  ASSERT_OK(AppendCommit(std::move(insert_commit)));
 
   ConsensusBootstrapInfo boot_info;
   shared_ptr<Tablet> tablet;
@@ -584,7 +583,7 @@ TEST_F(BootstrapTest, TestMissingCommitMessage) {
   replicate->get()->set_timestamp(clock_->Now().ToUint64());
   AddTestRowToPB(RowOperationsPB::INSERT, schema_, 10, 1,
                  "this is a test insert", batch_request->mutable_row_operations());
-  AppendReplicateBatch(replicate, true);
+  ASSERT_OK(AppendReplicateBatch(replicate));
 
   // This appends Mutate(1) with op 10.11
   OpId mutate_opid = MakeOpId(10, 11);
@@ -594,7 +593,7 @@ TEST_F(BootstrapTest, TestMissingCommitMessage) {
   AddTestRowToPB(RowOperationsPB::UPDATE, schema_,
                  10, 2, "this is a test mutate",
                  batch_request->mutable_row_operations());
-  AppendReplicateBatch(replicate, true);
+  ASSERT_OK(AppendReplicateBatch(replicate));
 
   // Now commit the mutate before the insert (in the log).
   gscoped_ptr<consensus::CommitMsg> mutate_commit(new consensus::CommitMsg);
@@ -605,7 +604,7 @@ TEST_F(BootstrapTest, TestMissingCommitMessage) {
   MemStoreTargetPB* target = mutate->add_mutated_stores();
   target->set_mrs_id(1);
 
-  AppendCommit(std::move(mutate_commit));
+  ASSERT_OK(AppendCommit(std::move(mutate_commit)));
 
   ConsensusBootstrapInfo boot_info;
   shared_ptr<Tablet> tablet;
@@ -631,7 +630,7 @@ TEST_F(BootstrapTest, TestConsensusOnlyOperationOutOfOrderTimestamp) {
   *noop_replicate->get()->mutable_id() = MakeOpId(1, 1);
   noop_replicate->get()->set_timestamp(2);
 
-  AppendReplicateBatch(noop_replicate, true);
+  ASSERT_OK(AppendReplicateBatch(noop_replicate));
 
   // Append WRITE_OP with higher OpId and lower timestamp.
   ReplicateRefPtr write_replicate = make_scoped_refptr_replicate(new ReplicateMsg());
@@ -644,7 +643,7 @@ TEST_F(BootstrapTest, TestConsensusOnlyOperationOutOfOrderTimestamp) {
   AddTestRowToPB(RowOperationsPB::INSERT, schema_, 1, 1, "foo",
                  batch_request->mutable_row_operations());
 
-  AppendReplicateBatch(write_replicate, true);
+  ASSERT_OK(AppendReplicateBatch(write_replicate));
 
   // Now commit in OpId order.
   // NO_OP...
@@ -652,7 +651,7 @@ TEST_F(BootstrapTest, TestConsensusOnlyOperationOutOfOrderTimestamp) {
   mutate_commit->set_op_type(consensus::NO_OP);
   *mutate_commit->mutable_commited_op_id() = noop_replicate->get()->id();
 
-  AppendCommit(std::move(mutate_commit));
+  ASSERT_OK(AppendCommit(std::move(mutate_commit)));
 
   // ...and WRITE_OP...
   mutate_commit = gscoped_ptr<consensus::CommitMsg>(new consensus::CommitMsg);
@@ -663,7 +662,7 @@ TEST_F(BootstrapTest, TestConsensusOnlyOperationOutOfOrderTimestamp) {
   MemStoreTargetPB* target = mutate->add_mutated_stores();
   target->set_mrs_id(1);
 
-  AppendCommit(std::move(mutate_commit));
+  ASSERT_OK(AppendCommit(std::move(mutate_commit)));
 
   ConsensusBootstrapInfo boot_info;
   shared_ptr<Tablet> tablet;
@@ -696,7 +695,7 @@ TEST_F(BootstrapTest, TestKudu2509) {
   replicate->get()->set_timestamp(clock_->Now().ToUint64());
   AddTestRowToPB(RowOperationsPB::INSERT, schema_, 10, 1,
                  "this is a test insert", batch_request->mutable_row_operations());
-  NO_FATALS(AppendReplicateBatch(replicate));
+  ASSERT_OK(AppendReplicateBatch(replicate));
 
   // This appends Mutate(1) with op 10.11. The operation would try to update
   // a row having an extra column. This should fail since there hasn't been
@@ -720,20 +719,20 @@ TEST_F(BootstrapTest, TestKudu2509) {
     RowOperationsPBEncoder enc(batch_request->mutable_row_operations());
     enc.Add(RowOperationsPB::UPDATE, row);
   }
-  NO_FATALS(AppendReplicateBatch(replicate));
+  ASSERT_OK(AppendReplicateBatch(replicate));
 
   // Now commit the mutate before the insert (in the log).
   gscoped_ptr<consensus::CommitMsg> mutate_commit(new consensus::CommitMsg);
   mutate_commit->set_op_type(consensus::WRITE_OP);
   mutate_commit->mutable_commited_op_id()->CopyFrom(mutate_opid);
   mutate_commit->mutable_result()->add_ops()->add_mutated_stores()->set_mrs_id(1);
-  NO_FATALS(AppendCommit(std::move(mutate_commit)));
+  ASSERT_OK(AppendCommit(std::move(mutate_commit)));
 
   gscoped_ptr<consensus::CommitMsg> insert_commit(new consensus::CommitMsg);
   insert_commit->set_op_type(consensus::WRITE_OP);
   insert_commit->mutable_commited_op_id()->CopyFrom(insert_opid);
   insert_commit->mutable_result()->add_ops()->add_mutated_stores()->set_mrs_id(1);
-  NO_FATALS(AppendCommit(std::move(insert_commit)));
+  ASSERT_OK(AppendCommit(std::move(insert_commit)));
 
   ConsensusBootstrapInfo boot_info;
   shared_ptr<Tablet> tablet;