You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2020/08/28 07:06:38 UTC

[kudu] branch master updated: tablet: replace MVCC "commit" with "apply"

This is an automated email from the ASF dual-hosted git repository.

awong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new e4c2cb9  tablet: replace MVCC "commit" with "apply"
e4c2cb9 is described below

commit e4c2cb91020d67a97195e0b7745cc2a98562e4c4
Author: Andrew Wong <aw...@cloudera.com>
AuthorDate: Thu Aug 27 18:03:56 2020 -0700

    tablet: replace MVCC "commit" with "apply"
    
    As multi-op transaction work progresses, the distinction between "apply"
    and "commit" will widen, so let's clarify the terminology up front.
    This is a first step, replacing the MvccManager's "commit" terminology
    with "apply" terminology.
    
    As far as MVCC is concerned, there is no concept of "transaction
    commit" today, only "Raft committed and applied". Thus, all current
    usages of MVCC's "commit" have been replaced with "apply". This is
    somewhat heavy-handed, but correct for now, and may be adjusted as
    needed as "transaction commit" becomes more clearly specified.
    
    For example:
    - committed --> applied
    - commit --> finish applying
    - uncommitted --> nonapplied
    
    As "Raft commit" and "commit messages" (logical Raft committing entails
    updating a commit index on Raft leaders) differ from MVCC's usage of
    commit (i.e. an op is "MVCC committed" once it physically applies the op
    to underlying stores), I left such terminology alone.
    
    Change-Id: I4da2ed45daf1d4bf74ea12c2616a4a550c45fc65
    Reviewed-on: http://gerrit.cloudera.org:8080/16378
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 .../raft_consensus_election-itest.cc               |   4 +-
 src/kudu/tablet/compaction-test.cc                 |   8 +-
 src/kudu/tablet/compaction.cc                      |   6 +-
 src/kudu/tablet/compaction.h                       |   2 +-
 src/kudu/tablet/delta_relevancy.h                  |  24 +-
 src/kudu/tablet/deltafile-test.cc                  |   2 +-
 src/kudu/tablet/deltamemstore-test.cc              |  19 +-
 src/kudu/tablet/diff_scan-test.cc                  |   2 +-
 src/kudu/tablet/diskrowset-test-base.h             |   2 +-
 src/kudu/tablet/diskrowset-test.cc                 |   4 +-
 src/kudu/tablet/local_tablet_writer.h              |   2 +-
 src/kudu/tablet/memrowset-test.cc                  |  14 +-
 src/kudu/tablet/memrowset.cc                       |   8 +-
 src/kudu/tablet/mvcc-test.cc                       | 414 ++++++++++-----------
 src/kudu/tablet/mvcc.cc                            | 162 ++++----
 src/kudu/tablet/mvcc.h                             | 174 +++++----
 src/kudu/tablet/ops/alter_schema_op.cc             |   2 +-
 src/kudu/tablet/ops/op.h                           |   6 +-
 src/kudu/tablet/ops/op_driver.cc                   |   4 +-
 src/kudu/tablet/ops/participant_op.cc              |   4 +-
 src/kudu/tablet/ops/write_op.cc                    |  12 +-
 src/kudu/tablet/ops/write_op.h                     |   4 +-
 src/kudu/tablet/tablet.cc                          |   6 +-
 src/kudu/tablet/tablet_bootstrap.cc                |   4 +-
 src/kudu/tablet/tablet_replica.cc                  |   6 +-
 src/kudu/tserver/tablet_service.cc                 |   2 +-
 26 files changed, 449 insertions(+), 448 deletions(-)

diff --git a/src/kudu/integration-tests/raft_consensus_election-itest.cc b/src/kudu/integration-tests/raft_consensus_election-itest.cc
index 77b65b1..fed341f 100644
--- a/src/kudu/integration-tests/raft_consensus_election-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus_election-itest.cc
@@ -112,10 +112,10 @@ void RaftConsensusElectionITest::CreateClusterForChurnyElectionsTests(
   // On TSAN builds, we need to be a little bit less churny in order to make
   // any progress at all.
   ts_flags.push_back("--raft_heartbeat_interval_ms=5");
-  ts_flags.emplace_back("--inject_latency_ms_before_starting_txn=100");
+  ts_flags.emplace_back("--inject_latency_ms_before_starting_op=100");
 #else
   ts_flags.emplace_back("--raft_heartbeat_interval_ms=2");
-  ts_flags.emplace_back("--inject_latency_ms_before_starting_txn=1000");
+  ts_flags.emplace_back("--inject_latency_ms_before_starting_op=1000");
 #endif
 
   ts_flags.insert(ts_flags.end(), extra_ts_flags.cbegin(), extra_ts_flags.cend());
diff --git a/src/kudu/tablet/compaction-test.cc b/src/kudu/tablet/compaction-test.cc
index 07d4597..9d78ff5 100644
--- a/src/kudu/tablet/compaction-test.cc
+++ b/src/kudu/tablet/compaction-test.cc
@@ -139,7 +139,7 @@ class TestCompaction : public KuduRowSetTest {
     ScopedOp op(&mvcc_, clock_.Now());
     op.StartApplying();
     InsertRowInOp(mrs, op, row_key, val);
-    op.Commit();
+    op.FinishApplying();
   }
 
   void BuildRow(int row_key, int32_t val) {
@@ -190,7 +190,7 @@ class TestCompaction : public KuduRowSetTest {
     ScopedOp op(&mvcc_, clock_.Now());
     op.StartApplying();
     UpdateRowInOp(rowset, op, row_key, new_val);
-    op.Commit();
+    op.FinishApplying();
   }
 
   void UpdateRowInOp(RowSet *rowset,
@@ -246,7 +246,7 @@ class TestCompaction : public KuduRowSetTest {
     ScopedOp op(&mvcc_, clock_.Now());
     op.StartApplying();
     DeleteRowInOp(rowset, op, row_key);
-    op.Commit();
+    op.FinishApplying();
   }
 
   void DeleteRowInOp(RowSet *rowset, const ScopedOp& op, int row_key) {
@@ -868,7 +868,7 @@ TEST_F(TestCompaction, TestMRSCompactionDoesntOutputUnobservableRows) {
     DeleteRowInOp(mrs.get(), op, 1);
 
     InsertRowInOp(mrs.get(), op, 2, 0);
-    op.Commit();
+    op.FinishApplying();
     FlushMRSAndReopenNoRoll(*mrs, schema_, &rs2);
     NO_FATALS();
   }
diff --git a/src/kudu/tablet/compaction.cc b/src/kudu/tablet/compaction.cc
index 23eba03..42ef966 100644
--- a/src/kudu/tablet/compaction.cc
+++ b/src/kudu/tablet/compaction.cc
@@ -997,7 +997,7 @@ Status ApplyMutationsAndGenerateUndos(const MvccSnapshot& snap,
        redo_mut = redo_mut->acquire_next()) {
 
     // Skip anything not committed.
-    if (!snap.IsCommitted(redo_mut->timestamp())) {
+    if (!snap.IsApplied(redo_mut->timestamp())) {
       break;
     }
 
@@ -1240,7 +1240,7 @@ Status ReupdateMissedDeltas(const IOContext* io_context,
         RowChangeListDecoder decoder(mut->changelist());
         RETURN_NOT_OK(decoder.Init());
 
-        if (snap_to_exclude.IsCommitted(mut->timestamp())) {
+        if (snap_to_exclude.IsApplied(mut->timestamp())) {
           // This update was already taken into account in the first phase of the
           // compaction. We don't need to reapply it.
 
@@ -1278,7 +1278,7 @@ Status ReupdateMissedDeltas(const IOContext* io_context,
           << " row=" << schema->DebugRow(row.row)
           << " mutations=" << Mutation::StringifyMutationList(*schema, row.redo_head);
 
-        if (!snap_to_include.IsCommitted(mut->timestamp())) {
+        if (!snap_to_include.IsApplied(mut->timestamp())) {
           // The mutation was inserted after the DuplicatingRowSet was swapped in.
           // Therefore, it's already present in the output rowset, and we don't need
           // to copy it in.
diff --git a/src/kudu/tablet/compaction.h b/src/kudu/tablet/compaction.h
index 466bef5..d7060c3 100644
--- a/src/kudu/tablet/compaction.h
+++ b/src/kudu/tablet/compaction.h
@@ -100,7 +100,7 @@ class CompactionInput {
   // NOTE: For efficiency, this doesn't currently filter the mutations to only
   // include those committed in the given snapshot. It does, however, filter out
   // rows that weren't inserted prior to this snapshot. Users of this input still
-  // need to call snap.IsCommitted() on each mutation.
+  // need to call snap.IsApplied() on each mutation.
   //
   // TODO: can we make the above less messy?
   static Status Create(const DiskRowSet &rowset,
diff --git a/src/kudu/tablet/delta_relevancy.h b/src/kudu/tablet/delta_relevancy.h
index 67e2766..08883fe 100644
--- a/src/kudu/tablet/delta_relevancy.h
+++ b/src/kudu/tablet/delta_relevancy.h
@@ -76,10 +76,10 @@ inline bool IsDeltaRelevantForApply<REDO>(const MvccSnapshot& snap,
                                           const Timestamp& delta_ts,
                                           bool* finished_row) {
   *finished_row = false;
-  if (snap.IsCommitted(delta_ts)) {
+  if (snap.IsApplied(delta_ts)) {
     return true;
   }
-  if (!snap.MayHaveCommittedOpsAtOrAfter(delta_ts)) {
+  if (!snap.MayHaveAppliedOpsAtOrAfter(delta_ts)) {
     // REDO deltas are sorted first in ascending row ordinal order, then in
     // ascending timestamp order. Thus, if we know that there are no more
     // committed ops whose timestamps are >= 'delta_ts', we know that
@@ -96,10 +96,10 @@ inline bool IsDeltaRelevantForApply<UNDO>(const MvccSnapshot& snap,
                                           const Timestamp& delta_ts,
                                           bool* finished_row) {
   *finished_row = false;
-  if (!snap.IsCommitted(delta_ts)) {
+  if (!snap.IsApplied(delta_ts)) {
     return true;
   }
-  if (!snap.MayHaveUncommittedOpsAtOrBefore(delta_ts)) {
+  if (!snap.MayHaveNonAppliedOpsAtOrBefore(delta_ts)) {
     // UNDO deltas are sorted first in ascending row ordinal order, then in
     // descending timestamp order. Thus, if we know that there are no more
     // uncommitted ops whose timestamps are <= 'delta_ts', we know that
@@ -117,8 +117,8 @@ inline bool IsDeltaRelevantForSelect(const MvccSnapshot& snap_start,
                                      const MvccSnapshot& snap_end,
                                      const Timestamp& delta_ts_start,
                                      const Timestamp& delta_ts_end) {
-  return !snap_start.IsCommitted(delta_ts_end) &&
-      snap_end.IsCommitted(delta_ts_start);
+  return !snap_start.IsApplied(delta_ts_end) &&
+      snap_end.IsApplied(delta_ts_start);
 }
 
 // A variant of IsDeltaRelevantForSelect that operates on a single delta's
@@ -137,14 +137,14 @@ inline bool IsDeltaRelevantForSelect<REDO>(const MvccSnapshot& snap_start,
                                            const Timestamp& delta_ts,
                                            bool* finished_row) {
   *finished_row = false;
-  if (snap_start.IsCommitted(delta_ts)) {
+  if (snap_start.IsApplied(delta_ts)) {
     // No short-circuit available here; because REDO deltas for a given row are
     // sorted in ascending timestamp order, the next REDO may be uncommitted in
     // 'snap_start'.
     return false;
   }
-  if (!snap_end.IsCommitted(delta_ts)) {
-    if (!snap_end.MayHaveCommittedOpsAtOrAfter(delta_ts)) {
+  if (!snap_end.IsApplied(delta_ts)) {
+    if (!snap_end.MayHaveAppliedOpsAtOrAfter(delta_ts)) {
       // But if 'delta_ts' is not committed in 'snap_end', all future REDOs may
       // also be uncommitted in 'snap_end'.
       *finished_row = true;
@@ -160,14 +160,14 @@ inline bool IsDeltaRelevantForSelect<UNDO>(const MvccSnapshot& snap_start,
                                            const Timestamp& delta_ts,
                                            bool* finished_row) {
   *finished_row = false;
-  if (!snap_end.IsCommitted(delta_ts)) {
+  if (!snap_end.IsApplied(delta_ts)) {
     // No short-circuit available here; because UNDO deltas for a given row are
     // sorted in descending timestamp order, the next UNDO may be committed in
     // 'snap_end'.
     return false;
   }
-  if (snap_start.IsCommitted(delta_ts)) {
-    if (!snap_start.MayHaveUncommittedOpsAtOrBefore(delta_ts)) {
+  if (snap_start.IsApplied(delta_ts)) {
+    if (!snap_start.MayHaveNonAppliedOpsAtOrBefore(delta_ts)) {
       // But if 'delta_ts' is committed in 'snap_start', all future UNDOs may
       // also be committed in 'snap_start'.
       *finished_row = true;
diff --git a/src/kudu/tablet/deltafile-test.cc b/src/kudu/tablet/deltafile-test.cc
index b58b2fa..b31a229 100644
--- a/src/kudu/tablet/deltafile-test.cc
+++ b/src/kudu/tablet/deltafile-test.cc
@@ -347,7 +347,7 @@ TEST_F(TestDeltaFile, TestSkipsDeltasOutOfRange) {
 
   // should skip
   opts.snap_to_include = MvccSnapshot(Timestamp(9));
-  ASSERT_FALSE(opts.snap_to_include.MayHaveCommittedOpsAtOrAfter(Timestamp(10)));
+  ASSERT_FALSE(opts.snap_to_include.MayHaveAppliedOpsAtOrAfter(Timestamp(10)));
   unique_ptr<DeltaIterator> iter;
   Status s = reader->NewDeltaIterator(opts, &iter);
   ASSERT_TRUE(s.IsNotFound());
diff --git a/src/kudu/tablet/deltamemstore-test.cc b/src/kudu/tablet/deltamemstore-test.cc
index a0e637f..a6dc408 100644
--- a/src/kudu/tablet/deltamemstore-test.cc
+++ b/src/kudu/tablet/deltamemstore-test.cc
@@ -36,6 +36,7 @@
 
 #include "kudu/clock/logical_clock.h"
 #include "kudu/common/columnblock.h"
+#include "kudu/common/columnblock-test-util.h"
 #include "kudu/common/common.pb.h"
 #include "kudu/common/row_changelist.h"
 #include "kudu/common/rowblock.h"
@@ -121,7 +122,7 @@ class TestDeltaMemStore : public KuduTest {
                              schema_.column_id(kIntColumn), &new_val);
 
       CHECK_OK(dms_->Update(op.timestamp(), idx_to_update, RowChangeList(buf), op_id_));
-      op.Commit();
+      op.FinishApplying();
     }
   }
 
@@ -197,7 +198,7 @@ TEST_F(TestDeltaMemStore, TestUpdateCount) {
       update.AddColumnUpdate(schema_.column(kIntColumn),
                              schema_.column_id(kIntColumn), &new_val);
       ASSERT_OK_FAST(dms_->Update(op.timestamp(), idx, RowChangeList(update_buf), op_id_));
-      op.Commit();
+      op.FinishApplying();
     }
   }
 
@@ -272,7 +273,7 @@ TEST_F(TestDeltaMemStore, BenchmarkManyUpdatesToOneRow) {
     update.AddColumnUpdate(schema_.column(kStringColumn),
                            schema_.column_id(kStringColumn), &s);
     CHECK_OK(dms_->Update(op.timestamp(), kIdxToUpdate, RowChangeList(buf), op_id_));
-    op.Commit();
+    op.FinishApplying();
   }
   mvcc_.AdjustNewOpLowerBound(clock_.Now());
 
@@ -396,7 +397,7 @@ TEST_F(TestDeltaMemStore, TestReUpdateSlice) {
                            schema_.column_id(0), &s);
     ASSERT_OK_FAST(dms_->Update(op.timestamp(), 123, RowChangeList(update_buf), op_id_));
     memset(buf, 0xff, sizeof(buf));
-    op.Commit();
+    op.FinishApplying();
   }
   MvccSnapshot snapshot_after_first_update(mvcc_);
 
@@ -411,7 +412,7 @@ TEST_F(TestDeltaMemStore, TestReUpdateSlice) {
                            schema_.column_id(0), &s);
     ASSERT_OK_FAST(dms_->Update(op.timestamp(), 123, RowChangeList(update_buf), op_id_));
     memset(buf, 0xff, sizeof(buf));
-    op.Commit();
+    op.FinishApplying();
   }
   MvccSnapshot snapshot_after_second_update(mvcc_);
 
@@ -447,7 +448,7 @@ TEST_F(TestDeltaMemStore, TestOutOfOrderOps) {
     update.AddColumnUpdate(schema_.column(kStringColumn),
                            schema_.column_id(kStringColumn), &s);
     ASSERT_OK(dms_->Update(op2.timestamp(), 123, RowChangeList(update_buf), op_id_));
-    op2.Commit();
+    op2.FinishApplying();
 
 
     op1.StartApplying();
@@ -456,7 +457,7 @@ TEST_F(TestDeltaMemStore, TestOutOfOrderOps) {
     update.AddColumnUpdate(schema_.column(kStringColumn),
                            schema_.column_id(kStringColumn), &s);
     ASSERT_OK(dms_->Update(op1.timestamp(), 123, RowChangeList(update_buf), op_id_));
-    op1.Commit();
+    op1.FinishApplying();
   }
 
   // Ensure we end up two entries for the cell.
@@ -488,7 +489,7 @@ TEST_F(TestDeltaMemStore, TestDMSBasic) {
                            schema_.column_id(kStringColumn), &s);
 
     ASSERT_OK_FAST(dms_->Update(op.timestamp(), i, RowChangeList(update_buf), op_id_));
-    op.Commit();
+    op.FinishApplying();
   }
 
   ASSERT_EQ(1000, dms_->Count());
@@ -526,7 +527,7 @@ TEST_F(TestDeltaMemStore, TestDMSBasic) {
     update.AddColumnUpdate(schema_.column(kIntColumn),
                            schema_.column_id(kIntColumn), &val);
     ASSERT_OK_FAST(dms_->Update(op.timestamp(), i, RowChangeList(update_buf), op_id_));
-    op.Commit();
+    op.FinishApplying();
   }
 
   ASSERT_EQ(2000, dms_->Count());
diff --git a/src/kudu/tablet/diff_scan-test.cc b/src/kudu/tablet/diff_scan-test.cc
index 8a9026c..a1db260 100644
--- a/src/kudu/tablet/diff_scan-test.cc
+++ b/src/kudu/tablet/diff_scan-test.cc
@@ -88,7 +88,7 @@ TEST_P(DiffScanTest, TestDiffScan) {
   ASSERT_EQ("(int64 key=1, int32 key_idx=1, int32 val=2)", rows[0]);
 
   // 4. Do a diff scan from time snap1.
-  ASSERT_OK(tablet->mvcc_manager()->WaitForApplyingOpsToCommit());
+  ASSERT_OK(tablet->mvcc_manager()->WaitForApplyingOpsToApply());
   MvccSnapshot snap2(*tablet->mvcc_manager());
 
   RowIteratorOptions opts;
diff --git a/src/kudu/tablet/diskrowset-test-base.h b/src/kudu/tablet/diskrowset-test-base.h
index 9eef818..30949bf 100644
--- a/src/kudu/tablet/diskrowset-test-base.h
+++ b/src/kudu/tablet/diskrowset-test-base.h
@@ -189,7 +189,7 @@ class TestRowSet : public KuduRowSetTest {
     ScopedOp op(&mvcc_, clock_.Now());
     op.StartApplying();
     Status s = rs->MutateRow(op.timestamp(), probe, mutation, op_id_, nullptr, &stats, result);
-    op.Commit();
+    op.FinishApplying();
     return s;
   }
 
diff --git a/src/kudu/tablet/diskrowset-test.cc b/src/kudu/tablet/diskrowset-test.cc
index 3fcb2e4..0674008 100644
--- a/src/kudu/tablet/diskrowset-test.cc
+++ b/src/kudu/tablet/diskrowset-test.cc
@@ -437,7 +437,7 @@ TEST_F(TestRowSet, TestFlushedUpdatesRespectMVCC) {
       ASSERT_EQ(1, result.mutated_stores_size());
       ASSERT_EQ(0L, result.mutated_stores(0).rs_id());
       ASSERT_EQ(0L, result.mutated_stores(0).dms_id());
-      op.Commit();
+      op.FinishApplying();
     }
     snaps.emplace_back(mvcc_);
   }
@@ -862,7 +862,7 @@ TEST_P(DiffScanRowSetTest, TestFuzz) {
     OperationResultPB result;
     ASSERT_OK(rs->MutateRow(op.timestamp(), probe, enc.as_changelist(), op_id_,
                             &test_context, &stats, &result));
-    op.Commit();
+    op.FinishApplying();
   };
 
   Random prng(SeedRandom());
diff --git a/src/kudu/tablet/local_tablet_writer.h b/src/kudu/tablet/local_tablet_writer.h
index b59e681..ad6377a 100644
--- a/src/kudu/tablet/local_tablet_writer.h
+++ b/src/kudu/tablet/local_tablet_writer.h
@@ -114,7 +114,7 @@ class LocalTabletWriter {
         op_state_->pb_arena());
     op_state_->ReleaseTxResultPB(result_);
     tablet_->mvcc_manager()->AdjustNewOpLowerBound(op_state_->timestamp());
-    op_state_->CommitOrAbort(Op::COMMITTED);
+    op_state_->FinishApplyingOrAbort(Op::APPLIED);
 
     // Return the status of first failed op.
     int op_idx = 0;
diff --git a/src/kudu/tablet/memrowset-test.cc b/src/kudu/tablet/memrowset-test.cc
index 62afeb3..f26293d 100644
--- a/src/kudu/tablet/memrowset-test.cc
+++ b/src/kudu/tablet/memrowset-test.cc
@@ -154,7 +154,7 @@ class TestMemRowSet : public KuduTest {
     rb.AddUint32(val);
     op.StartApplying();
     Status s = mrs->Insert(op.timestamp(), rb.row(), op_id_);
-    op.Commit();
+    op.FinishApplying();
     return s;
   }
 
@@ -181,7 +181,7 @@ class TestMemRowSet : public KuduTest {
                               nullptr,
                               &stats,
                               result);
-    op.Commit();
+    op.FinishApplying();
     return s;
   }
 
@@ -205,7 +205,7 @@ class TestMemRowSet : public KuduTest {
                               nullptr,
                               &stats,
                               result);
-    op.Commit();
+    op.FinishApplying();
     return s;
   }
 
@@ -319,7 +319,7 @@ TEST_F(TestMemRowSet, TestInsertAndIterateCompoundKey) {
     rb.AddUint32(12345);
     Status row1 = mrs->Insert(op.timestamp(), rb.row(), op_id_);
     ASSERT_OK(row1);
-    op.Commit();
+    op.FinishApplying();
   }
 
   {
@@ -331,7 +331,7 @@ TEST_F(TestMemRowSet, TestInsertAndIterateCompoundKey) {
     rb.AddUint32(54321);
     Status row2 = mrs->Insert(op2.timestamp(), rb.row(), op_id_);
     ASSERT_OK(row2);
-    op2.Commit();
+    op2.FinishApplying();
   }
 
   {
@@ -343,7 +343,7 @@ TEST_F(TestMemRowSet, TestInsertAndIterateCompoundKey) {
     rb.AddUint32(12345);
     Status row3 = mrs->Insert(op3.timestamp(), rb.row(), op_id_);
     ASSERT_OK(row3);
-    op3.Commit();
+    op3.FinishApplying();
   }
 
   ASSERT_EQ(3, mrs->entry_count());
@@ -561,7 +561,7 @@ TEST_F(TestMemRowSet, TestInsertionMVCC) {
       rb.AddString(Slice(keybuf));
       rb.AddUint32(i);
       ASSERT_OK_FAST(mrs->Insert(op.timestamp(), rb.row(), op_id_));
-      op.Commit();
+      op.FinishApplying();
     }
 
     // Op is committed. Save the snapshot after this commit.
diff --git a/src/kudu/tablet/memrowset.cc b/src/kudu/tablet/memrowset.cc
index d4a4a05..11436e6 100644
--- a/src/kudu/tablet/memrowset.cc
+++ b/src/kudu/tablet/memrowset.cc
@@ -518,10 +518,10 @@ Status MemRowSet::Iterator::FetchRows(RowBlock* dst, size_t* fetched) {
     // insert was "excluded"). However, subsequent mutations may be inside the
     // time range, so we must still project the row and walk its mutation list.
     bool insert_excluded = opts_.snap_to_exclude &&
-                           opts_.snap_to_exclude->IsCommitted(row.insertion_timestamp());
+                           opts_.snap_to_exclude->IsApplied(row.insertion_timestamp());
     bool unset_in_sel_vector;
     ApplyStatus apply_status;
-    if (insert_excluded || opts_.snap_to_include.IsCommitted(row.insertion_timestamp())) {
+    if (insert_excluded || opts_.snap_to_include.IsApplied(row.insertion_timestamp())) {
       RETURN_NOT_OK(projector_->ProjectRowForRead(row, &dst_row, dst->arena()));
 
       // Roll-forward MVCC for committed updates.
@@ -592,7 +592,7 @@ Status MemRowSet::Iterator::ApplyMutationsToProjectedRow(
   for (const Mutation *mut = mutation_head;
        mut != nullptr;
        mut = mut->acquire_next()) {
-    if (!opts_.snap_to_include.IsCommitted(mut->timestamp_)) {
+    if (!opts_.snap_to_include.IsApplied(mut->timestamp_)) {
       // This mutation is too new and should be omitted.
       //
       // All subsequent mutations are also too new because their timestamps are
@@ -604,7 +604,7 @@ Status MemRowSet::Iterator::ApplyMutationsToProjectedRow(
     // values are correct if we see a relevant mutation later), but it doesn't
     // count towards the overall "application status".
     if (!opts_.snap_to_exclude ||
-        !opts_.snap_to_exclude->IsCommitted(mut->timestamp_)) {
+        !opts_.snap_to_exclude->IsApplied(mut->timestamp_)) {
 
       // This is the first mutation within the time range, so we may use it to
       // initialize 'is_deleted_start'.
diff --git a/src/kudu/tablet/mvcc-test.cc b/src/kudu/tablet/mvcc-test.cc
index dfa3f2a..6cf152e 100644
--- a/src/kudu/tablet/mvcc-test.cc
+++ b/src/kudu/tablet/mvcc-test.cc
@@ -53,7 +53,7 @@ class MvccTest : public KuduTest {
 
   void WaitForSnapshotAtTSThread(MvccManager* mgr, Timestamp ts) {
     MvccSnapshot s;
-    CHECK_OK(mgr->WaitForSnapshotWithAllCommitted(ts, &s, MonoTime::Max()));
+    CHECK_OK(mgr->WaitForSnapshotWithAllApplied(ts, &s, MonoTime::Max()));
     CHECK(s.is_clean()) << "verifying postcondition";
     std::lock_guard<simple_spinlock> lock(lock_);
     result_snapshot_.reset(new MvccSnapshot(s));
@@ -74,37 +74,37 @@ TEST_F(MvccTest, TestMvccBasic) {
   MvccManager mgr;
   MvccSnapshot snap;
 
-  // Initial state should not have any committed ops.
+  // Initial state should not have any applied ops.
   snap = MvccSnapshot(mgr);
-  ASSERT_EQ("MvccSnapshot[committed={T|T < 1}]", snap.ToString());
-  ASSERT_FALSE(snap.IsCommitted(Timestamp(1)));
-  ASSERT_FALSE(snap.IsCommitted(Timestamp(2)));
+  ASSERT_EQ("MvccSnapshot[applied={T|T < 1}]", snap.ToString());
+  ASSERT_FALSE(snap.IsApplied(Timestamp(1)));
+  ASSERT_FALSE(snap.IsApplied(Timestamp(2)));
 
   // Start timestamp 1
   Timestamp t = clock_.Now();
   ASSERT_EQ(1, t.value());
   ScopedOp op(&mgr, t);
 
-  // State should still have no committed ops, since 1 is in-flight.
+  // State should still have no applied ops, since 1 is in-flight.
   snap = MvccSnapshot(mgr);
-  ASSERT_EQ("MvccSnapshot[committed={T|T < 1}]", snap.ToString());
-  ASSERT_FALSE(snap.IsCommitted(Timestamp(1)));
-  ASSERT_FALSE(snap.IsCommitted(Timestamp(2)));
+  ASSERT_EQ("MvccSnapshot[applied={T|T < 1}]", snap.ToString());
+  ASSERT_FALSE(snap.IsApplied(Timestamp(1)));
+  ASSERT_FALSE(snap.IsApplied(Timestamp(2)));
 
   // Mark timestamp 1 as "applying"
   op.StartApplying();
 
-  // This should not change the set of committed ops.
-  ASSERT_FALSE(snap.IsCommitted(Timestamp(1)));
+  // This should not change the set of applied ops.
+  ASSERT_FALSE(snap.IsApplied(Timestamp(1)));
 
-  // Commit timestamp 1
-  op.Commit();
+  // Apply timestamp 1
+  op.FinishApplying();
 
-  // State should show 0 as committed, 1 as uncommitted.
+  // State should show 0 as applied, 1 as nonapplied.
   snap = MvccSnapshot(mgr);
-  ASSERT_EQ("MvccSnapshot[committed={T|T < 1 or (T in {1})}]", snap.ToString());
-  ASSERT_TRUE(snap.IsCommitted(Timestamp(1)));
-  ASSERT_FALSE(snap.IsCommitted(Timestamp(2)));
+  ASSERT_EQ("MvccSnapshot[applied={T|T < 1 or (T in {1})}]", snap.ToString());
+  ASSERT_TRUE(snap.IsApplied(Timestamp(1)));
+  ASSERT_FALSE(snap.IsApplied(Timestamp(2)));
 }
 
 TEST_F(MvccTest, TestMvccMultipleInFlight) {
@@ -118,68 +118,68 @@ TEST_F(MvccTest, TestMvccMultipleInFlight) {
   ASSERT_EQ(2, t2.value());
   ScopedOp op2(&mgr, t2);
 
-  // State should still have no committed ops, since both are in-flight.
+  // State should still have no applied ops, since both are in-flight.
 
   snap = MvccSnapshot(mgr);
-  ASSERT_EQ("MvccSnapshot[committed={T|T < 1}]", snap.ToString());
-  ASSERT_FALSE(snap.IsCommitted(t1));
-  ASSERT_FALSE(snap.IsCommitted(t2));
+  ASSERT_EQ("MvccSnapshot[applied={T|T < 1}]", snap.ToString());
+  ASSERT_FALSE(snap.IsApplied(t1));
+  ASSERT_FALSE(snap.IsApplied(t2));
 
-  // Commit timestamp 2
+  // Apply timestamp 2
   op2.StartApplying();
-  op2.Commit();
+  op2.FinishApplying();
 
-  // State should show 2 as committed, 1 as uncommitted.
+  // State should show 2 as applied, 1 as nonapplied.
   snap = MvccSnapshot(mgr);
-  ASSERT_EQ("MvccSnapshot[committed="
+  ASSERT_EQ("MvccSnapshot[applied="
             "{T|T < 1 or (T in {2})}]",
             snap.ToString());
-  ASSERT_FALSE(snap.IsCommitted(t1));
-  ASSERT_TRUE(snap.IsCommitted(t2));
+  ASSERT_FALSE(snap.IsApplied(t1));
+  ASSERT_TRUE(snap.IsApplied(t2));
 
   // Start another ops. This gets timestamp 3
   Timestamp t3 = clock_.Now();
   ASSERT_EQ(3, t3.value());
   ScopedOp op3(&mgr, t3);
 
-  // State should show 2 as committed, 1 and 4 as uncommitted.
+  // State should show 2 as applied, 1 and 4 as nonapplied.
   snap = MvccSnapshot(mgr);
-  ASSERT_EQ("MvccSnapshot[committed="
+  ASSERT_EQ("MvccSnapshot[applied="
             "{T|T < 1 or (T in {2})}]",
             snap.ToString());
-  ASSERT_FALSE(snap.IsCommitted(t1));
-  ASSERT_TRUE(snap.IsCommitted(t2));
-  ASSERT_FALSE(snap.IsCommitted(t3));
+  ASSERT_FALSE(snap.IsApplied(t1));
+  ASSERT_TRUE(snap.IsApplied(t2));
+  ASSERT_FALSE(snap.IsApplied(t3));
 
-  // Commit 3
+  // Apply 3
   op3.StartApplying();
-  op3.Commit();
+  op3.FinishApplying();
 
-  // 2 and 3 committed
+  // 2 and 3 applied
   snap = MvccSnapshot(mgr);
-  ASSERT_EQ("MvccSnapshot[committed="
+  ASSERT_EQ("MvccSnapshot[applied="
             "{T|T < 1 or (T in {2,3})}]",
             snap.ToString());
-  ASSERT_FALSE(snap.IsCommitted(t1));
-  ASSERT_TRUE(snap.IsCommitted(t2));
-  ASSERT_TRUE(snap.IsCommitted(t3));
+  ASSERT_FALSE(snap.IsApplied(t1));
+  ASSERT_TRUE(snap.IsApplied(t2));
+  ASSERT_TRUE(snap.IsApplied(t3));
 
-  // Commit 1
+  // Apply 1
   op1.StartApplying();
-  op1.Commit();
+  op1.FinishApplying();
 
-  // All ops are committed, adjust the new op lower bound.
+  // All ops are applied, adjust the new op lower bound.
   mgr.AdjustNewOpLowerBound(t3);
 
-  // all committed
+  // all applied
   snap = MvccSnapshot(mgr);
-  ASSERT_EQ("MvccSnapshot[committed={T|T < 3 or (T in {3})}]", snap.ToString());
-  ASSERT_TRUE(snap.IsCommitted(t1));
-  ASSERT_TRUE(snap.IsCommitted(t2));
-  ASSERT_TRUE(snap.IsCommitted(t3));
+  ASSERT_EQ("MvccSnapshot[applied={T|T < 3 or (T in {3})}]", snap.ToString());
+  ASSERT_TRUE(snap.IsApplied(t1));
+  ASSERT_TRUE(snap.IsApplied(t2));
+  ASSERT_TRUE(snap.IsApplied(t3));
 }
 
-TEST_F(MvccTest, TestOutOfOrderTxns) {
+TEST_F(MvccTest, TestOutOfOrderOps) {
   MetricRegistry metric_registry;
   auto metric_entity(METRIC_ENTITY_server.Instantiate(&metric_registry, "mvcc-test"));
   clock::HybridClock hybrid_clock(metric_entity);
@@ -191,40 +191,40 @@ TEST_F(MvccTest, TestOutOfOrderTxns) {
   ScopedOp first_op(&mgr, first_ts);
 
   // Take a snapshot that con
-  MvccSnapshot snap_with_nothing_committed(mgr);
+  MvccSnapshot snap_with_nothing_applied(mgr);
 
   // Start an op as if it were using commit-wait (i.e. started in future)
   Timestamp cw_ts = hybrid_clock.NowLatest();
   ScopedOp cw_op(&mgr, cw_ts);
 
-  // Commit the original op
+  // Apply the original op
   first_op.StartApplying();
-  first_op.Commit();
+  first_op.FinishApplying();
 
   // Start a new op
   Timestamp second_ts = hybrid_clock.Now();
   ScopedOp second_op(&mgr, second_ts);
 
   // The old snapshot should not have either op
-  EXPECT_FALSE(snap_with_nothing_committed.IsCommitted(first_ts));
-  EXPECT_FALSE(snap_with_nothing_committed.IsCommitted(second_ts));
+  EXPECT_FALSE(snap_with_nothing_applied.IsApplied(first_ts));
+  EXPECT_FALSE(snap_with_nothing_applied.IsApplied(second_ts));
 
   // A new snapshot should have only the first op
-  MvccSnapshot snap_with_first_committed(mgr);
-  EXPECT_TRUE(snap_with_first_committed.IsCommitted(first_ts));
-  EXPECT_FALSE(snap_with_first_committed.IsCommitted(second_ts));
+  MvccSnapshot snap_with_first_applied(mgr);
+  EXPECT_TRUE(snap_with_first_applied.IsApplied(first_ts));
+  EXPECT_FALSE(snap_with_first_applied.IsApplied(second_ts));
 
-  // Commit the commit-wait one once it is time.
+  // Apply the commit-wait one once it is time.
   ASSERT_OK(hybrid_clock.WaitUntilAfter(cw_ts, MonoTime::Max()));
   cw_op.StartApplying();
-  cw_op.Commit();
+  cw_op.FinishApplying();
 
-  // A new snapshot at this point should still think that normal_op_2 is uncommitted
-  MvccSnapshot snap_with_all_committed(mgr);
-  EXPECT_FALSE(snap_with_all_committed.IsCommitted(second_ts));
+  // A new snapshot at this point should still think that normal_op_2 is nonapplied
+  MvccSnapshot snap_with_all_applied(mgr);
+  EXPECT_FALSE(snap_with_all_applied.IsApplied(second_ts));
 }
 
-// Tests starting ops at a point-in-time in the past and committing them while
+// Tests starting ops at a point-in-time in the past and applying them while
 // adjusting the new op timestamp lower bound.
 TEST_F(MvccTest, TestSafeTimeWithOutOfOrderTxns) {
   MvccManager mgr;
@@ -239,27 +239,27 @@ TEST_F(MvccTest, TestSafeTimeWithOutOfOrderTxns) {
 
   ASSERT_EQ(Timestamp::kInitialTimestamp, mgr.GetCleanTimestamp());
 
-  // Committing 'op_in_the_past' should not advance the new op lower
+  // Applying 'op_in_the_past' should not advance the new op lower
   // bound or the clean time.
-  op_in_the_past.Commit();
+  op_in_the_past.FinishApplying();
 
   // Now take a snapshot.
   MvccSnapshot snap_with_first_op(mgr);
 
   // Because we did not advance the the new op lower bound or clean time, even
-  // though the only in-flight op was committed at time 50, an op at time 40
-  // should still be considered uncommitted.
-  ASSERT_FALSE(snap_with_first_op.IsCommitted(Timestamp(40)));
+  // though the only in-flight op was applied at time 50, an op at time 40
+  // should still be considered nonapplied.
+  ASSERT_FALSE(snap_with_first_op.IsApplied(Timestamp(40)));
 
   // Now advance the both clean and new op lower bound watermarks to the last
-  // committed op.
+  // applied op.
   mgr.AdjustNewOpLowerBound(Timestamp(50));
 
   ASSERT_EQ(ts_in_the_past, mgr.GetCleanTimestamp());
 
   MvccSnapshot snap_with_adjusted_clean_time(mgr);
 
-  ASSERT_TRUE(snap_with_adjusted_clean_time.IsCommitted(Timestamp(40)));
+  ASSERT_TRUE(snap_with_adjusted_clean_time.IsApplied(Timestamp(40)));
 }
 
 TEST_F(MvccTest, TestScopedOp) {
@@ -274,17 +274,17 @@ TEST_F(MvccTest, TestScopedOp) {
     ASSERT_EQ(2, t2.timestamp().value());
 
     t1.StartApplying();
-    t1.Commit();
+    t1.FinishApplying();
 
     snap = MvccSnapshot(mgr);
-    ASSERT_TRUE(snap.IsCommitted(t1.timestamp()));
-    ASSERT_FALSE(snap.IsCommitted(t2.timestamp()));
+    ASSERT_TRUE(snap.IsApplied(t1.timestamp()));
+    ASSERT_FALSE(snap.IsApplied(t2.timestamp()));
   }
 
   // t2 going out of scope aborts it.
   snap = MvccSnapshot(mgr);
-  ASSERT_TRUE(snap.IsCommitted(Timestamp(1)));
-  ASSERT_FALSE(snap.IsCommitted(Timestamp(2)));
+  ASSERT_TRUE(snap.IsApplied(Timestamp(1)));
+  ASSERT_FALSE(snap.IsApplied(Timestamp(2)));
 
   // Test that an applying scoped op does not crash if it goes out of
   // scope while the MvccManager is closed.
@@ -298,97 +298,97 @@ TEST_F(MvccTest, TestScopedOp) {
 TEST_F(MvccTest, TestPointInTimeSnapshot) {
   MvccSnapshot snap(Timestamp(10));
 
-  ASSERT_TRUE(snap.IsCommitted(Timestamp(1)));
-  ASSERT_TRUE(snap.IsCommitted(Timestamp(9)));
-  ASSERT_FALSE(snap.IsCommitted(Timestamp(10)));
-  ASSERT_FALSE(snap.IsCommitted(Timestamp(11)));
+  ASSERT_TRUE(snap.IsApplied(Timestamp(1)));
+  ASSERT_TRUE(snap.IsApplied(Timestamp(9)));
+  ASSERT_FALSE(snap.IsApplied(Timestamp(10)));
+  ASSERT_FALSE(snap.IsApplied(Timestamp(11)));
 }
 
-TEST_F(MvccTest, TestMayHaveCommittedOpsAtOrAfter) {
+TEST_F(MvccTest, TestMayHaveAppliedOpsAtOrAfter) {
   MvccSnapshot snap;
-  snap.all_committed_before_ = Timestamp(10);
-  snap.committed_timestamps_.push_back(11);
-  snap.committed_timestamps_.push_back(13);
-  snap.none_committed_at_or_after_ = Timestamp(14);
-
-  ASSERT_TRUE(snap.MayHaveCommittedOpsAtOrAfter(Timestamp(9)));
-  ASSERT_TRUE(snap.MayHaveCommittedOpsAtOrAfter(Timestamp(10)));
-  ASSERT_TRUE(snap.MayHaveCommittedOpsAtOrAfter(Timestamp(12)));
-  ASSERT_TRUE(snap.MayHaveCommittedOpsAtOrAfter(Timestamp(13)));
-  ASSERT_FALSE(snap.MayHaveCommittedOpsAtOrAfter(Timestamp(14)));
-  ASSERT_FALSE(snap.MayHaveCommittedOpsAtOrAfter(Timestamp(15)));
-
-  // Test for "all committed" snapshot
-  MvccSnapshot all_committed =
+  snap.all_applied_before_ = Timestamp(10);
+  snap.applied_timestamps_.push_back(11);
+  snap.applied_timestamps_.push_back(13);
+  snap.none_applied_at_or_after_ = Timestamp(14);
+
+  ASSERT_TRUE(snap.MayHaveAppliedOpsAtOrAfter(Timestamp(9)));
+  ASSERT_TRUE(snap.MayHaveAppliedOpsAtOrAfter(Timestamp(10)));
+  ASSERT_TRUE(snap.MayHaveAppliedOpsAtOrAfter(Timestamp(12)));
+  ASSERT_TRUE(snap.MayHaveAppliedOpsAtOrAfter(Timestamp(13)));
+  ASSERT_FALSE(snap.MayHaveAppliedOpsAtOrAfter(Timestamp(14)));
+  ASSERT_FALSE(snap.MayHaveAppliedOpsAtOrAfter(Timestamp(15)));
+
+  // Test for "all applied" snapshot
+  MvccSnapshot all_applied =
       MvccSnapshot::CreateSnapshotIncludingAllOps();
   ASSERT_TRUE(
-      all_committed.MayHaveCommittedOpsAtOrAfter(Timestamp(1)));
+      all_applied.MayHaveAppliedOpsAtOrAfter(Timestamp(1)));
   ASSERT_TRUE(
-      all_committed.MayHaveCommittedOpsAtOrAfter(Timestamp(12345)));
+      all_applied.MayHaveAppliedOpsAtOrAfter(Timestamp(12345)));
 
-  // And "none committed" snapshot
-  MvccSnapshot none_committed =
+  // And "none applied" snapshot
+  MvccSnapshot none_applied =
       MvccSnapshot::CreateSnapshotIncludingNoOps();
   ASSERT_FALSE(
-      none_committed.MayHaveCommittedOpsAtOrAfter(Timestamp(1)));
+      none_applied.MayHaveAppliedOpsAtOrAfter(Timestamp(1)));
   ASSERT_FALSE(
-      none_committed.MayHaveCommittedOpsAtOrAfter(Timestamp(12345)));
+      none_applied.MayHaveAppliedOpsAtOrAfter(Timestamp(12345)));
 
   // Test for a "clean" snapshot
   MvccSnapshot clean_snap(Timestamp(10));
-  ASSERT_TRUE(clean_snap.MayHaveCommittedOpsAtOrAfter(Timestamp(9)));
-  ASSERT_FALSE(clean_snap.MayHaveCommittedOpsAtOrAfter(Timestamp(10)));
+  ASSERT_TRUE(clean_snap.MayHaveAppliedOpsAtOrAfter(Timestamp(9)));
+  ASSERT_FALSE(clean_snap.MayHaveAppliedOpsAtOrAfter(Timestamp(10)));
 }
 
-TEST_F(MvccTest, TestMayHaveUncommittedOpsBefore) {
+TEST_F(MvccTest, TestMayHaveNonAppliedOpsBefore) {
   MvccSnapshot snap;
-  snap.all_committed_before_ = Timestamp(10);
-  snap.committed_timestamps_.push_back(11);
-  snap.committed_timestamps_.push_back(13);
-  snap.none_committed_at_or_after_ = Timestamp(14);
-
-  ASSERT_FALSE(snap.MayHaveUncommittedOpsAtOrBefore(Timestamp(9)));
-  ASSERT_TRUE(snap.MayHaveUncommittedOpsAtOrBefore(Timestamp(10)));
-  ASSERT_TRUE(snap.MayHaveUncommittedOpsAtOrBefore(Timestamp(11)));
-  ASSERT_TRUE(snap.MayHaveUncommittedOpsAtOrBefore(Timestamp(13)));
-  ASSERT_TRUE(snap.MayHaveUncommittedOpsAtOrBefore(Timestamp(14)));
-  ASSERT_TRUE(snap.MayHaveUncommittedOpsAtOrBefore(Timestamp(15)));
-
-  // Test for "all committed" snapshot
-  MvccSnapshot all_committed =
+  snap.all_applied_before_ = Timestamp(10);
+  snap.applied_timestamps_.push_back(11);
+  snap.applied_timestamps_.push_back(13);
+  snap.none_applied_at_or_after_ = Timestamp(14);
+
+  ASSERT_FALSE(snap.MayHaveNonAppliedOpsAtOrBefore(Timestamp(9)));
+  ASSERT_TRUE(snap.MayHaveNonAppliedOpsAtOrBefore(Timestamp(10)));
+  ASSERT_TRUE(snap.MayHaveNonAppliedOpsAtOrBefore(Timestamp(11)));
+  ASSERT_TRUE(snap.MayHaveNonAppliedOpsAtOrBefore(Timestamp(13)));
+  ASSERT_TRUE(snap.MayHaveNonAppliedOpsAtOrBefore(Timestamp(14)));
+  ASSERT_TRUE(snap.MayHaveNonAppliedOpsAtOrBefore(Timestamp(15)));
+
+  // Test for "all applied" snapshot
+  MvccSnapshot all_applied =
       MvccSnapshot::CreateSnapshotIncludingAllOps();
   ASSERT_FALSE(
-      all_committed.MayHaveUncommittedOpsAtOrBefore(Timestamp(1)));
+      all_applied.MayHaveNonAppliedOpsAtOrBefore(Timestamp(1)));
   ASSERT_FALSE(
-      all_committed.MayHaveUncommittedOpsAtOrBefore(Timestamp(12345)));
+      all_applied.MayHaveNonAppliedOpsAtOrBefore(Timestamp(12345)));
 
-  // And "none committed" snapshot
-  MvccSnapshot none_committed =
+  // And "none applied" snapshot
+  MvccSnapshot none_applied =
       MvccSnapshot::CreateSnapshotIncludingNoOps();
   ASSERT_TRUE(
-      none_committed.MayHaveUncommittedOpsAtOrBefore(Timestamp(1)));
+      none_applied.MayHaveNonAppliedOpsAtOrBefore(Timestamp(1)));
   ASSERT_TRUE(
-      none_committed.MayHaveUncommittedOpsAtOrBefore(
+      none_applied.MayHaveNonAppliedOpsAtOrBefore(
           Timestamp(12345)));
 
   // Test for a "clean" snapshot
   MvccSnapshot clean_snap(Timestamp(10));
-  ASSERT_FALSE(clean_snap.MayHaveUncommittedOpsAtOrBefore(Timestamp(9)));
-  ASSERT_TRUE(clean_snap.MayHaveUncommittedOpsAtOrBefore(Timestamp(10)));
+  ASSERT_FALSE(clean_snap.MayHaveNonAppliedOpsAtOrBefore(Timestamp(9)));
+  ASSERT_TRUE(clean_snap.MayHaveNonAppliedOpsAtOrBefore(Timestamp(10)));
 
   // Test for the case where we have a single op in flight. Since this is also
-  // the earliest op, all_committed_before_ is equal to the op's ts, but when
-  // it gets committed we can't advance all_committed_before_ past it because
+  // the earliest op, all_applied_before_ is equal to the op's ts, but when
+  // it gets applied we can't advance all_applied_before_ past it because
   // there is no other op to advance it to. In this case we should still report
-  // that there can't be any uncommitted ops before.
+  // that there can't be any nonapplied ops before.
   MvccSnapshot snap2;
-  snap2.all_committed_before_ = Timestamp(10);
-  snap2.committed_timestamps_.push_back(10);
+  snap2.all_applied_before_ = Timestamp(10);
+  snap2.applied_timestamps_.push_back(10);
 
-  ASSERT_FALSE(snap2.MayHaveUncommittedOpsAtOrBefore(Timestamp(10)));
+  ASSERT_FALSE(snap2.MayHaveNonAppliedOpsAtOrBefore(Timestamp(10)));
 }
 
-TEST_F(MvccTest, TestAreAllOpsCommittedForTests) {
+TEST_F(MvccTest, TestAreAllOpsAppliedForTests) {
   MvccManager mgr;
 
   // start several ops and take snapshots along the way
@@ -400,31 +400,31 @@ TEST_F(MvccTest, TestAreAllOpsCommittedForTests) {
   ScopedOp op3(&mgr, ts3);
   mgr.AdjustNewOpLowerBound(clock_.Now());
 
-  ASSERT_FALSE(mgr.AreAllOpsCommittedForTests(Timestamp(1)));
-  ASSERT_FALSE(mgr.AreAllOpsCommittedForTests(Timestamp(2)));
-  ASSERT_FALSE(mgr.AreAllOpsCommittedForTests(Timestamp(3)));
+  ASSERT_FALSE(mgr.AreAllOpsAppliedForTests(Timestamp(1)));
+  ASSERT_FALSE(mgr.AreAllOpsAppliedForTests(Timestamp(2)));
+  ASSERT_FALSE(mgr.AreAllOpsAppliedForTests(Timestamp(3)));
 
-  // commit op3, should all still report as having as having uncommitted ops.
+  // Apply op3, should all still report as having as having nonapplied ops.
   op3.StartApplying();
-  op3.Commit();
-  ASSERT_FALSE(mgr.AreAllOpsCommittedForTests(Timestamp(1)));
-  ASSERT_FALSE(mgr.AreAllOpsCommittedForTests(Timestamp(2)));
-  ASSERT_FALSE(mgr.AreAllOpsCommittedForTests(Timestamp(3)));
+  op3.FinishApplying();
+  ASSERT_FALSE(mgr.AreAllOpsAppliedForTests(Timestamp(1)));
+  ASSERT_FALSE(mgr.AreAllOpsAppliedForTests(Timestamp(2)));
+  ASSERT_FALSE(mgr.AreAllOpsAppliedForTests(Timestamp(3)));
 
-  // commit op1, first snap with in-flights should now report as all committed
-  // and remaining snaps as still having uncommitted ops
+  // Apply op1, first snap with in-flights should now report as all applied
+  // and remaining snaps as still having nonapplied ops
   op1.StartApplying();
-  op1.Commit();
-  ASSERT_TRUE(mgr.AreAllOpsCommittedForTests(Timestamp(1)));
-  ASSERT_FALSE(mgr.AreAllOpsCommittedForTests(Timestamp(2)));
-  ASSERT_FALSE(mgr.AreAllOpsCommittedForTests(Timestamp(3)));
+  op1.FinishApplying();
+  ASSERT_TRUE(mgr.AreAllOpsAppliedForTests(Timestamp(1)));
+  ASSERT_FALSE(mgr.AreAllOpsAppliedForTests(Timestamp(2)));
+  ASSERT_FALSE(mgr.AreAllOpsAppliedForTests(Timestamp(3)));
 
-  // Now they should all report as all committed.
+  // Now they should all report as all applied.
   op2.StartApplying();
-  op2.Commit();
-  ASSERT_TRUE(mgr.AreAllOpsCommittedForTests(Timestamp(1)));
-  ASSERT_TRUE(mgr.AreAllOpsCommittedForTests(Timestamp(2)));
-  ASSERT_TRUE(mgr.AreAllOpsCommittedForTests(Timestamp(3)));
+  op2.FinishApplying();
+  ASSERT_TRUE(mgr.AreAllOpsAppliedForTests(Timestamp(1)));
+  ASSERT_TRUE(mgr.AreAllOpsAppliedForTests(Timestamp(2)));
+  ASSERT_TRUE(mgr.AreAllOpsAppliedForTests(Timestamp(3)));
 }
 
 TEST_F(MvccTest, WaitForCleanSnapshotSnapWithNoInflights) {
@@ -450,7 +450,7 @@ TEST_F(MvccTest, WaitForCleanSnapshotSnapBeforeSafeTimeWithInFlights) {
 
   // Select a new op timestamp lower bound that is after all ops and after the
   // the timestamp we'll wait for.  This will cause "clean time" to move when
-  // op1 and op2 commit.
+  // op1 and op2 finish applying.
   Timestamp future_ts = clock_.Now();
   mgr.AdjustNewOpLowerBound(future_ts);
 
@@ -458,10 +458,10 @@ TEST_F(MvccTest, WaitForCleanSnapshotSnapBeforeSafeTimeWithInFlights) {
 
   ASSERT_FALSE(HasResultSnapshot());
   op1.StartApplying();
-  op1.Commit();
+  op1.FinishApplying();
   ASSERT_FALSE(HasResultSnapshot());
   op2.StartApplying();
-  op2.Commit();
+  op2.FinishApplying();
   waiting_thread.join();
   ASSERT_TRUE(HasResultSnapshot());
 }
@@ -475,13 +475,13 @@ TEST_F(MvccTest, WaitForCleanSnapshotSnapAfterSafeTimeWithInFlights) {
   mgr.AdjustNewOpLowerBound(ts2);
 
   // Wait should return immediately, since we have no ops "applying" yet.
-  ASSERT_OK(mgr.WaitForApplyingOpsToCommit());
+  ASSERT_OK(mgr.WaitForApplyingOpsToApply());
 
   op1.StartApplying();
 
   Status s;
   thread waiting_thread = thread([&] {
-    s = mgr.WaitForApplyingOpsToCommit();
+    s = mgr.WaitForApplyingOpsToApply();
   });
   while (mgr.GetNumWaitersForTests() == 0) {
     SleepFor(MonoDelta::FromMilliseconds(5));
@@ -492,8 +492,8 @@ TEST_F(MvccTest, WaitForCleanSnapshotSnapAfterSafeTimeWithInFlights) {
   op2.Abort();
   ASSERT_EQ(mgr.GetNumWaitersForTests(), 1);
 
-  // Committing our op should wake the waiter.
-  op1.Commit();
+  // Applying our op should wake the waiter.
+  op1.FinishApplying();
   ASSERT_EQ(mgr.GetNumWaitersForTests(), 0);
   waiting_thread.join();
   ASSERT_OK(s);
@@ -510,25 +510,25 @@ TEST_F(MvccTest, WaitForCleanSnapshotSnapAtTimestampWithInFlights) {
   Timestamp ts3 = clock_.Now();
   ScopedOp op3(&mgr, ts3);
 
-  // Start a thread waiting for ops with ts <= 2 to commit
+  // Start a thread waiting for ops with ts <= 2 to finish applying
   thread waiting_thread = thread(&MvccTest::WaitForSnapshotAtTSThread, this, &mgr, ts2);
   ASSERT_FALSE(HasResultSnapshot());
 
-  // Commit op 1 - thread should still wait.
+  // Apply op 1 - thread should still wait.
   op1.StartApplying();
-  op1.Commit();
+  op1.FinishApplying();
   SleepFor(MonoDelta::FromMilliseconds(1));
   ASSERT_FALSE(HasResultSnapshot());
 
-  // Commit op 3 - thread should still wait.
+  // Apply op 3 - thread should still wait.
   op3.StartApplying();
-  op3.Commit();
+  op3.FinishApplying();
   SleepFor(MonoDelta::FromMilliseconds(1));
   ASSERT_FALSE(HasResultSnapshot());
 
-  // Commit op 2 - thread should still wait.
+  // Apply op 2 - thread should still wait.
   op2.StartApplying();
-  op2.Commit();
+  op2.FinishApplying();
   ASSERT_FALSE(HasResultSnapshot());
 
   // Advance new op lower bound and the clean time, thread should continue.
@@ -537,7 +537,7 @@ TEST_F(MvccTest, WaitForCleanSnapshotSnapAtTimestampWithInFlights) {
   ASSERT_TRUE(HasResultSnapshot());
 }
 
-TEST_F(MvccTest, TestWaitForApplyingOpsToCommit) {
+TEST_F(MvccTest, TestWaitForApplyingOpsToApply) {
   MvccManager mgr;
 
   Timestamp ts1 = clock_.Now();
@@ -547,11 +547,11 @@ TEST_F(MvccTest, TestWaitForApplyingOpsToCommit) {
   mgr.AdjustNewOpLowerBound(ts2);
 
   // Wait should return immediately, since we have no ops "applying" yet.
-  ASSERT_OK(mgr.WaitForApplyingOpsToCommit());
+  ASSERT_OK(mgr.WaitForApplyingOpsToApply());
 
   op1.StartApplying();
 
-  thread waiting_thread = thread(&MvccManager::WaitForApplyingOpsToCommit, &mgr);
+  thread waiting_thread = thread(&MvccManager::WaitForApplyingOpsToApply, &mgr);
   while (mgr.GetNumWaitersForTests() == 0) {
     SleepFor(MonoDelta::FromMilliseconds(5));
   }
@@ -561,8 +561,8 @@ TEST_F(MvccTest, TestWaitForApplyingOpsToCommit) {
   op2.Abort();
   ASSERT_EQ(mgr.GetNumWaitersForTests(), 1);
 
-  // Committing our op should wake the waiter.
-  op1.Commit();
+  // Applying our op should wake the waiter.
+  op1.FinishApplying();
   ASSERT_EQ(mgr.GetNumWaitersForTests(), 0);
   waiting_thread.join();
 }
@@ -584,7 +584,7 @@ TEST_F(MvccTest, TestDontWaitAfterClose) {
   simple_spinlock status_lock;
   thread waiting_thread = thread([&] {
     std::lock_guard<simple_spinlock> l(status_lock);
-    s = mgr.WaitForApplyingOpsToCommit();
+    s = mgr.WaitForApplyingOpsToApply();
   });
 
   // Wait until the waiter actually gets registered.
@@ -599,13 +599,13 @@ TEST_F(MvccTest, TestDontWaitAfterClose) {
   ASSERT_TRUE(s.IsAborted());
 
   // New waiters should abort immediately.
-  s = mgr.WaitForApplyingOpsToCommit();
+  s = mgr.WaitForApplyingOpsToApply();
   ASSERT_STR_CONTAINS(s.ToString(), "closed");
   ASSERT_TRUE(s.IsAborted());
 }
 
 // Test that if we abort an op we don't advance the new op lower bound and
-// don't add the op to the committed set.
+// don't add the op to the applied set.
 TEST_F(MvccTest, TestTxnAbort) {
 
   MvccManager mgr;
@@ -620,28 +620,28 @@ TEST_F(MvccTest, TestTxnAbort) {
   mgr.AdjustNewOpLowerBound(ts3);
 
   // Now abort op1, this shouldn't move the clean time and the op shouldn't be
-  // reported as committed.
+  // reported as applied.
   op1.Abort();
   ASSERT_EQ(Timestamp::kInitialTimestamp, mgr.GetCleanTimestamp());
-  ASSERT_FALSE(mgr.cur_snap_.IsCommitted(ts1));
+  ASSERT_FALSE(mgr.cur_snap_.IsApplied(ts1));
 
-  // Committing op3 shouldn't advance the clean time since it is not the earliest
+  // Applying op3 shouldn't advance the clean time since it is not the earliest
   // in-flight, but it should advance 'new_op_timestamp_exc_lower_bound_' to 3.
   op3.StartApplying();
-  op3.Commit();
-  ASSERT_TRUE(mgr.cur_snap_.IsCommitted(ts3));
+  op3.FinishApplying();
+  ASSERT_TRUE(mgr.cur_snap_.IsApplied(ts3));
   ASSERT_EQ(ts3, mgr.new_op_timestamp_exc_lower_bound_);
 
-  // Committing op2 should advance the clean time to 3.
+  // Applying op2 should advance the clean time to 3.
   op2.StartApplying();
-  op2.Commit();
-  ASSERT_TRUE(mgr.cur_snap_.IsCommitted(ts2));
+  op2.FinishApplying();
+  ASSERT_TRUE(mgr.cur_snap_.IsApplied(ts2));
   ASSERT_EQ(ts3, mgr.GetCleanTimestamp());
 }
 
 // This tests for a bug we were observing, where a clean snapshot would not
 // coalesce to the latest timestamp.
-TEST_F(MvccTest, TestAutomaticCleanTimeMoveToSafeTimeOnCommit) {
+TEST_F(MvccTest, TestAutomaticCleanTimeMoveToSafeTimeOnApply) {
   MvccManager mgr;
   clock_.Update(Timestamp(20));
 
@@ -650,17 +650,17 @@ TEST_F(MvccTest, TestAutomaticCleanTimeMoveToSafeTimeOnCommit) {
   mgr.AdjustNewOpLowerBound(Timestamp(15));
 
   op2.StartApplying();
-  op2.Commit();
+  op2.FinishApplying();
 
   op1.StartApplying();
-  op1.Commit();
-  ASSERT_EQ(mgr.cur_snap_.ToString(), "MvccSnapshot[committed={T|T < 15 or (T in {15})}]");
+  op1.FinishApplying();
+  ASSERT_EQ(mgr.cur_snap_.ToString(), "MvccSnapshot[applied={T|T < 15 or (T in {15})}]");
 }
 
 // Various death tests which ensure that we can only transition in one of the following
 // valid ways:
 //
-// - Start() -> StartApplying() -> Commit()
+// - Start() -> StartApplying() -> FinishApplying()
 // - Start() -> Abort()
 //
 // Any other transition should fire a CHECK failure.
@@ -676,24 +676,24 @@ TEST_F(MvccTest, TestIllegalStateTransitionsCrash) {
   // could be different for this case -- the "future timestamp" check is only
   // run in DEBUG builds.
   EXPECT_DEATH({
-      mgr.CommitOp(Timestamp(1));
+      mgr.FinishApplyingOp(Timestamp(1));
     },
-    "Trying to commit an op with a future timestamp|"
+    "Trying to apply an op with a future timestamp|"
     "Trying to remove timestamp which isn't in the in-flight set: 1");
 
   clock_.Update(Timestamp(20));
 
   EXPECT_DEATH({
-      mgr.CommitOp(Timestamp(1));
+      mgr.FinishApplyingOp(Timestamp(1));
     }, "Trying to remove timestamp which isn't in the in-flight set: 1");
 
-  // Start an op, and try committing it without having moved to "Applying"
+  // Start an op, and try applying it without having moved to "Applying"
   // state.
   Timestamp t = clock_.Now();
   mgr.StartOp(t);
   EXPECT_DEATH({
-      mgr.CommitOp(t);
-    }, "Trying to commit an op which never entered APPLYING state");
+      mgr.FinishApplyingOp(t);
+    }, "Trying to apply an op which never entered APPLYING state");
 
   // Aborting should succeed, since we never moved to Applying.
   mgr.AbortOp(t);
@@ -719,8 +719,8 @@ TEST_F(MvccTest, TestIllegalStateTransitionsCrash) {
       mgr.AbortOp(t);
     }, "op with timestamp 22 cannot be aborted in state 1");
 
-  // We can commit it successfully.
-  mgr.CommitOp(t);
+  // We can apply it successfully.
+  mgr.FinishApplyingOp(t);
 }
 
 TEST_F(MvccTest, TestWaitUntilCleanDeadline) {
@@ -731,10 +731,10 @@ TEST_F(MvccTest, TestWaitUntilCleanDeadline) {
   ScopedOp op1(&mgr, ts1);
 
   // Wait until the 'op1' timestamp is clean -- this won't happen because the
-  // op isn't committed yet.
+  // op isn't applied yet.
   MonoTime deadline = MonoTime::Now() + MonoDelta::FromMilliseconds(10);
   MvccSnapshot snap;
-  Status s = mgr.WaitForSnapshotWithAllCommitted(ts1, &snap, deadline);
+  Status s = mgr.WaitForSnapshotWithAllApplied(ts1, &snap, deadline);
   ASSERT_TRUE(s.IsTimedOut()) << s.ToString();
 }
 
@@ -743,18 +743,18 @@ TEST_F(MvccTest, TestWaitUntilCleanDeadline) {
 // AdjustNewOpLowerBound().
 //
 // Prior to the fix we would advance clean time but not the
-// 'none_committed_at_or_after_' watermark, meaning the latter would become lower
+// 'none_applied_at_or_after_' watermark, meaning the latter would become lower
 // than clean time. This had the effect on compaction of culling delta files
 // even though they shouldn't be culled.
 // This test makes sure that watermarks are advanced correctly and that delta
 // files are culled correctly.
-TEST_F(MvccTest, TestCorrectInitWithNoTxns) {
+TEST_F(MvccTest, TestCorrectInitWithNoOps) {
   MvccManager mgr;
 
   MvccSnapshot snap(mgr);
-  EXPECT_EQ(snap.all_committed_before_, Timestamp::kInitialTimestamp);
-  EXPECT_EQ(snap.none_committed_at_or_after_, Timestamp::kInitialTimestamp);
-  EXPECT_EQ(snap.committed_timestamps_.size(), 0);
+  EXPECT_EQ(snap.all_applied_before_, Timestamp::kInitialTimestamp);
+  EXPECT_EQ(snap.none_applied_at_or_after_, Timestamp::kInitialTimestamp);
+  EXPECT_EQ(snap.applied_timestamps_.size(), 0);
 
   // Read the clock a few times to advance the timestamp
   for (int i = 0; i < 10; i++) {
@@ -766,19 +766,19 @@ TEST_F(MvccTest, TestCorrectInitWithNoTxns) {
   mgr.AdjustNewOpLowerBound(new_ts_lower_bound);
 
   // Test that the snapshot reports that a timestamp lower than the new op
-  // lower bound may have committed ops after that timestamp. Conversely, test
-  // that the snapshot reports that there are no committed ops at or after the
+  // lower bound may have applied ops after that timestamp. Conversely, test
+  // that the snapshot reports that there are no applied ops at or after the
   // new lower bound.
   MvccSnapshot snap2;
   snap2 = MvccSnapshot(mgr);
   Timestamp before_lb(new_ts_lower_bound.value() - 1);
   Timestamp after_lb(new_ts_lower_bound.value() + 1);
-  EXPECT_TRUE(snap2.MayHaveCommittedOpsAtOrAfter(before_lb));
-  EXPECT_FALSE(snap2.MayHaveCommittedOpsAtOrAfter(after_lb));
+  EXPECT_TRUE(snap2.MayHaveAppliedOpsAtOrAfter(before_lb));
+  EXPECT_FALSE(snap2.MayHaveAppliedOpsAtOrAfter(after_lb));
 
-  EXPECT_EQ(snap2.all_committed_before_, new_ts_lower_bound);
-  EXPECT_EQ(snap2.none_committed_at_or_after_, new_ts_lower_bound);
-  EXPECT_EQ(snap2.committed_timestamps_.size(), 0);
+  EXPECT_EQ(snap2.all_applied_before_, new_ts_lower_bound);
+  EXPECT_EQ(snap2.none_applied_at_or_after_, new_ts_lower_bound);
+  EXPECT_EQ(snap2.applied_timestamps_.size(), 0);
 }
 
 } // namespace tablet
diff --git a/src/kudu/tablet/mvcc.cc b/src/kudu/tablet/mvcc.cc
index 4c34cc0..8f58329 100644
--- a/src/kudu/tablet/mvcc.cc
+++ b/src/kudu/tablet/mvcc.cc
@@ -36,11 +36,11 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/monotime.h"
 
-DEFINE_int32(inject_latency_ms_before_starting_txn, 0,
+DEFINE_int32(inject_latency_ms_before_starting_op, 0,
              "Amount of latency in ms to inject before registering "
              "an op with MVCC.");
-TAG_FLAG(inject_latency_ms_before_starting_txn, advanced);
-TAG_FLAG(inject_latency_ms_before_starting_txn, hidden);
+TAG_FLAG(inject_latency_ms_before_starting_op, advanced);
+TAG_FLAG(inject_latency_ms_before_starting_op, hidden);
 
 namespace kudu {
 namespace tablet {
@@ -51,8 +51,8 @@ MvccManager::MvccManager()
   : new_op_timestamp_exc_lower_bound_(Timestamp::kMin),
     earliest_in_flight_(Timestamp::kMax),
     open_(true) {
-  cur_snap_.all_committed_before_ = Timestamp::kInitialTimestamp;
-  cur_snap_.none_committed_at_or_after_ = Timestamp::kInitialTimestamp;
+  cur_snap_.all_applied_before_ = Timestamp::kInitialTimestamp;
+  cur_snap_.none_applied_at_or_after_ = Timestamp::kInitialTimestamp;
 }
 
 Status MvccManager::CheckIsCleanTimeInitialized() const {
@@ -63,14 +63,14 @@ Status MvccManager::CheckIsCleanTimeInitialized() const {
 }
 
 void MvccManager::StartOp(Timestamp timestamp) {
-  MAYBE_INJECT_RANDOM_LATENCY(FLAGS_inject_latency_ms_before_starting_txn);
+  MAYBE_INJECT_RANDOM_LATENCY(FLAGS_inject_latency_ms_before_starting_op);
   std::lock_guard<LockType> l(lock_);
-  CHECK(!cur_snap_.IsCommitted(timestamp)) <<
-      Substitute("Trying to start a new txn at an already committed "
+  CHECK(!cur_snap_.IsApplied(timestamp)) <<
+      Substitute("Trying to start a new op at an already applied "
                  "timestamp: $0, current MVCC snapshot: $1",
                  timestamp.ToString(), cur_snap_.ToString());
   CHECK(InitOpUnlocked(timestamp)) <<
-      Substitute("There is already a txn with timestamp: $0 in flight, or "
+      Substitute("There is already a op with timestamp: $0 in flight, or "
                  "this timestamp is below or equal to the exclusive lower "
                  "bound for new op timestamps. Current lower bound: "
                  "$1, current MVCC snapshot: $2", timestamp.ToString(),
@@ -133,15 +133,15 @@ void MvccManager::AbortOp(Timestamp timestamp) {
   }
 }
 
-void MvccManager::CommitOp(Timestamp timestamp) {
+void MvccManager::FinishApplyingOp(Timestamp timestamp) {
   std::lock_guard<LockType> l(lock_);
 
-  // Commit the op, but do not adjust 'all_committed_before_', that will
+  // Commit the op, but do not adjust 'all_applied_before_', that will
   // be done with a separate OfflineAdjustCurSnap() call.
   bool was_earliest = false;
-  CommitOpUnlocked(timestamp, &was_earliest);
+  ApplyOpUnlocked(timestamp, &was_earliest);
 
-  // NOTE: we should have pushed the lower bound forward before committing, but
+  // NOTE: we should have pushed the lower bound forward before applying, but
   // we may not have in tests.
   if (was_earliest && new_op_timestamp_exc_lower_bound_ >= timestamp) {
 
@@ -164,21 +164,21 @@ MvccManager::TxnState MvccManager::RemoveInFlightAndGetStateUnlocked(Timestamp t
   return state;
 }
 
-void MvccManager::CommitOpUnlocked(Timestamp timestamp,
+void MvccManager::ApplyOpUnlocked(Timestamp timestamp,
                                    bool* was_earliest_in_flight) {
   *was_earliest_in_flight = earliest_in_flight_ == timestamp;
 
   // Remove from our in-flight list.
   TxnState old_state = RemoveInFlightAndGetStateUnlocked(timestamp);
   CHECK_EQ(old_state, APPLYING)
-    << "Trying to commit an op which never entered APPLYING state: "
+    << "Trying to apply an op which never entered APPLYING state: "
     << timestamp.ToString() << " state=" << old_state;
 
-  // Add to snapshot's committed list
-  cur_snap_.AddCommittedTimestamp(timestamp);
+  // Add to snapshot's applied list
+  cur_snap_.AddAppliedTimestamp(timestamp);
 
-  // If we're committing the earliest op that was in flight,
-  // update our cached value.
+  // If we're applying the earliest op that was in flight, update our cached
+  // value.
   if (*was_earliest_in_flight) {
     AdvanceEarliestInFlightTimestamp();
   }
@@ -254,29 +254,29 @@ void MvccManager::AdjustCleanTimeUnlocked() {
   //    NOTE: there may still be in-flight ops with future timestamps
   //    due to commit-wait ops which start in the future.
   //
-  // In either case, we have to add the newly committed ts only if it remains higher
-  // than the new watermark.
+  // In either case, we have to add the newly applied ts only if it remains
+  // higher than the new watermark.
 
   if (earliest_in_flight_ < new_op_timestamp_exc_lower_bound_) {
-    cur_snap_.all_committed_before_ = earliest_in_flight_;
+    cur_snap_.all_applied_before_ = earliest_in_flight_;
   } else {
-    cur_snap_.all_committed_before_ = new_op_timestamp_exc_lower_bound_;
+    cur_snap_.all_applied_before_ = new_op_timestamp_exc_lower_bound_;
   }
 
-  DVLOG(4) << "Adjusted clean time to: " << cur_snap_.all_committed_before_;
+  DVLOG(4) << "Adjusted clean time to: " << cur_snap_.all_applied_before_;
 
-  // Filter out any committed timestamps that now fall below the watermark
-  FilterTimestamps(&cur_snap_.committed_timestamps_, cur_snap_.all_committed_before_.value());
+  // Filter out any applied timestamps that now fall below the watermark
+  FilterTimestamps(&cur_snap_.applied_timestamps_, cur_snap_.all_applied_before_.value());
 
-  // If the current snapshot doesn't have any committed timestamps, then make sure we still
-  // advance the 'none_committed_at_or_after_' watermark so that it never falls below
-  // 'all_committed_before_'.
-  if (cur_snap_.committed_timestamps_.empty()) {
-    cur_snap_.none_committed_at_or_after_ = cur_snap_.all_committed_before_;
+  // If the current snapshot doesn't have any applied timestamps, then make sure we still
+  // advance the 'none_applied_at_or_after_' watermark so that it never falls below
+  // 'all_applied_before_'.
+  if (cur_snap_.applied_timestamps_.empty()) {
+    cur_snap_.none_applied_at_or_after_ = cur_snap_.all_applied_before_;
   }
 
   // it may also have unblocked some waiters.
-  // Check if someone is waiting for ops to be committed.
+  // Check if someone is waiting for ops to be applied.
   if (PREDICT_FALSE(!waiters_.empty())) {
     auto iter = waiters_.begin();
     while (iter != waiters_.end()) {
@@ -293,7 +293,7 @@ void MvccManager::AdjustCleanTimeUnlocked() {
 
 Status MvccManager::WaitUntil(WaitFor wait_for, Timestamp ts, const MonoTime& deadline) const {
   TRACE_EVENT2("tablet", "MvccManager::WaitUntil",
-               "wait_for", wait_for == ALL_COMMITTED ? "all_committed" : "none_applying",
+               "wait_for", wait_for == ALL_APPLIED ? "all_applied" : "none_applying",
                "ts", ts.ToUint64());
 
   // If MVCC is closed, there's no point in waiting.
@@ -322,16 +322,20 @@ Status MvccManager::WaitUntil(WaitFor wait_for, Timestamp ts, const MonoTime& de
     return CheckOpen();
   }
 
+  // TODO(awong): the distinction here seems nuanced. Do we actually need to
+  // wait for clean-time advancement, or can we wait to finish applying ops?
   waiters_.erase(std::find(waiters_.begin(), waiters_.end(), &waiting_state));
   return Status::TimedOut(Substitute("Timed out waiting for all ops with ts < $0 to $1",
                                      ts.ToString(),
-                                     wait_for == ALL_COMMITTED ? "commit" : "finish applying"));
+                                     wait_for == ALL_APPLIED ?
+                                     "finish applying and guarantee no earlier applying ops" :
+                                     "finish applying"));
 }
 
 bool MvccManager::IsDoneWaitingUnlocked(const WaitingState& waiter) const {
   switch (waiter.wait_for) {
-    case ALL_COMMITTED:
-      return AreAllOpsCommittedUnlocked(waiter.timestamp);
+    case ALL_APPLIED:
+      return AreAllOpsAppliedUnlocked(waiter.timestamp);
     case NONE_APPLYING:
       return !AnyApplyingAtOrBeforeUnlocked(waiter.timestamp);
   }
@@ -345,12 +349,12 @@ Status MvccManager::CheckOpen() const {
   return Status::Aborted("MVCC is closed");
 }
 
-bool MvccManager::AreAllOpsCommittedUnlocked(Timestamp ts) const {
-  // If ts is before the 'all_committed_before_' watermark on the current snapshot then
-  // all ops before it are committed.
-  if (ts < cur_snap_.all_committed_before_) return true;
+bool MvccManager::AreAllOpsAppliedUnlocked(Timestamp ts) const {
+  // If ts is before the 'all_applied_before_' watermark on the current snapshot then
+  // all ops before it are applied.
+  if (ts < cur_snap_.all_applied_before_) return true;
 
-  // We might not have moved 'cur_snap_.all_committed_before_' (the clean time) but 'ts'
+  // We might not have moved 'cur_snap_.all_applied_before_' (the clean time) but 'ts'
   // might still come before any possible in-flights.
   return ts < earliest_in_flight_;
 }
@@ -371,17 +375,17 @@ void MvccManager::TakeSnapshot(MvccSnapshot *snap) const {
   *snap = cur_snap_;
 }
 
-Status MvccManager::WaitForSnapshotWithAllCommitted(Timestamp timestamp,
+Status MvccManager::WaitForSnapshotWithAllApplied(Timestamp timestamp,
                                                     MvccSnapshot* snapshot,
                                                     const MonoTime& deadline) const {
-  TRACE_EVENT0("tablet", "MvccManager::WaitForSnapshotWithAllCommitted");
+  TRACE_EVENT0("tablet", "MvccManager::WaitForSnapshotWithAllApplied");
 
-  RETURN_NOT_OK(WaitUntil(ALL_COMMITTED, timestamp, deadline));
+  RETURN_NOT_OK(WaitUntil(ALL_APPLIED, timestamp, deadline));
   *snapshot = MvccSnapshot(timestamp);
   return Status::OK();
 }
 
-Status MvccManager::WaitForApplyingOpsToCommit() const {
+Status MvccManager::WaitForApplyingOpsToApply() const {
   TRACE_EVENT0("tablet", "MvccManager::WaitForApplyingOpsToCommit");
   RETURN_NOT_OK(CheckOpen());
 
@@ -408,7 +412,7 @@ Status MvccManager::WaitForApplyingOpsToCommit() const {
 
 Timestamp MvccManager::GetCleanTimestamp() const {
   std::lock_guard<LockType> l(lock_);
-  return cur_snap_.all_committed_before_;
+  return cur_snap_.all_applied_before_;
 }
 
 void MvccManager::GetApplyingOpsTimestamps(std::vector<Timestamp>* timestamps) const {
@@ -430,8 +434,8 @@ MvccManager::~MvccManager() {
 ////////////////////////////////////////////////////////////
 
 MvccSnapshot::MvccSnapshot()
-  : all_committed_before_(Timestamp::kInitialTimestamp),
-    none_committed_at_or_after_(Timestamp::kInitialTimestamp) {
+  : all_applied_before_(Timestamp::kInitialTimestamp),
+    none_applied_at_or_after_(Timestamp::kInitialTimestamp) {
 }
 
 MvccSnapshot::MvccSnapshot(const MvccManager &manager) {
@@ -439,8 +443,8 @@ MvccSnapshot::MvccSnapshot(const MvccManager &manager) {
 }
 
 MvccSnapshot::MvccSnapshot(const Timestamp& timestamp)
-  : all_committed_before_(timestamp),
-    none_committed_at_or_after_(timestamp) {
+  : all_applied_before_(timestamp),
+    none_applied_at_or_after_(timestamp) {
  }
 
 MvccSnapshot MvccSnapshot::CreateSnapshotIncludingAllOps() {
@@ -451,39 +455,39 @@ MvccSnapshot MvccSnapshot::CreateSnapshotIncludingNoOps() {
   return MvccSnapshot(Timestamp::kMin);
 }
 
-bool MvccSnapshot::IsCommittedFallback(const Timestamp& timestamp) const {
-  for (const Timestamp::val_type& v : committed_timestamps_) {
+bool MvccSnapshot::IsAppliedFallback(const Timestamp& timestamp) const {
+  for (const Timestamp::val_type& v : applied_timestamps_) {
     if (v == timestamp.value()) return true;
   }
 
   return false;
 }
 
-bool MvccSnapshot::MayHaveCommittedOpsAtOrAfter(const Timestamp& timestamp) const {
-  return timestamp < none_committed_at_or_after_;
+bool MvccSnapshot::MayHaveAppliedOpsAtOrAfter(const Timestamp& timestamp) const {
+  return timestamp < none_applied_at_or_after_;
 }
 
-bool MvccSnapshot::MayHaveUncommittedOpsAtOrBefore(const Timestamp& timestamp) const {
-  // The snapshot may have uncommitted ops before 'timestamp' if:
-  // - 'all_committed_before_' comes before 'timestamp'
-  // - 'all_committed_before_' is precisely 'timestamp' but 'timestamp' isn't in the
-  //   committed set.
-  return timestamp > all_committed_before_ ||
-      (timestamp == all_committed_before_ && !IsCommittedFallback(timestamp));
+bool MvccSnapshot::MayHaveNonAppliedOpsAtOrBefore(const Timestamp& timestamp) const {
+  // The snapshot may have nonapplied ops before 'timestamp' if:
+  // - 'all_applied_before_' comes before 'timestamp'
+  // - 'all_applied_before_' is precisely 'timestamp' but 'timestamp' isn't in the
+  //   applied set.
+  return timestamp > all_applied_before_ ||
+      (timestamp == all_applied_before_ && !IsAppliedFallback(timestamp));
 }
 
 std::string MvccSnapshot::ToString() const {
-  std::string ret("MvccSnapshot[committed={T|");
+  std::string ret("MvccSnapshot[applied={T|");
 
-  if (committed_timestamps_.size() == 0) {
-    StrAppend(&ret, "T < ", all_committed_before_.ToString(),"}]");
+  if (applied_timestamps_.size() == 0) {
+    StrAppend(&ret, "T < ", all_applied_before_.ToString(),"}]");
     return ret;
   }
-  StrAppend(&ret, "T < ", all_committed_before_.ToString(),
+  StrAppend(&ret, "T < ", all_applied_before_.ToString(),
             " or (T in {");
 
   bool first = true;
-  for (Timestamp::val_type t : committed_timestamps_) {
+  for (Timestamp::val_type t : applied_timestamps_) {
     if (!first) {
       ret.push_back(',');
     }
@@ -494,31 +498,31 @@ std::string MvccSnapshot::ToString() const {
   return ret;
 }
 
-void MvccSnapshot::AddCommittedTimestamps(const std::vector<Timestamp>& timestamps) {
+void MvccSnapshot::AddAppliedTimestamps(const std::vector<Timestamp>& timestamps) {
   for (const Timestamp& ts : timestamps) {
-    AddCommittedTimestamp(ts);
+    AddAppliedTimestamp(ts);
   }
 }
 
-void MvccSnapshot::AddCommittedTimestamp(Timestamp timestamp) {
-  if (IsCommitted(timestamp)) return;
+void MvccSnapshot::AddAppliedTimestamp(Timestamp timestamp) {
+  if (IsApplied(timestamp)) return;
 
-  committed_timestamps_.push_back(timestamp.value());
+  applied_timestamps_.push_back(timestamp.value());
 
-  // If this is a new upper bound commit mark, update it.
-  if (none_committed_at_or_after_ <= timestamp) {
-    none_committed_at_or_after_ = Timestamp(timestamp.value() + 1);
+  // If this is a new upper bound apply mark, update it.
+  if (none_applied_at_or_after_ <= timestamp) {
+    none_applied_at_or_after_ = Timestamp(timestamp.value() + 1);
   }
 }
 
 bool MvccSnapshot::Equals(const MvccSnapshot& other) const {
-  if (all_committed_before_ != other.all_committed_before_) {
+  if (all_applied_before_ != other.all_applied_before_) {
     return false;
   }
-  if (none_committed_at_or_after_ != other.none_committed_at_or_after_) {
+  if (none_applied_at_or_after_ != other.none_applied_at_or_after_) {
     return false;
   }
-  return committed_timestamps_ == other.committed_timestamps_;
+  return applied_timestamps_ == other.applied_timestamps_;
 }
 
 ////////////////////////////////////////////////////////////
@@ -541,8 +545,8 @@ void ScopedOp::StartApplying() {
   manager_->StartApplyingOp(timestamp_);
 }
 
-void ScopedOp::Commit() {
-  manager_->CommitOp(timestamp_);
+void ScopedOp::FinishApplying() {
+  manager_->FinishApplyingOp(timestamp_);
   done_ = true;
 }
 
diff --git a/src/kudu/tablet/mvcc.h b/src/kudu/tablet/mvcc.h
index 8c5da57..32d205b 100644
--- a/src/kudu/tablet/mvcc.h
+++ b/src/kudu/tablet/mvcc.h
@@ -50,66 +50,66 @@ class MvccSnapshot {
   // Create a snapshot at a specific Timestamp.
   //
   // This snapshot considers all ops with lower timestamps to
-  // be committed, and those with higher timestamps to be uncommitted.
+  // be applied, and those with higher timestamps to be nonapplied.
   explicit MvccSnapshot(const Timestamp& timestamp);
 
-  // Create a snapshot which considers all ops as committed. This is mostly
+  // Create a snapshot which considers all ops as applied. This is mostly
   // useful in test contexts.
   static MvccSnapshot CreateSnapshotIncludingAllOps();
 
-  // Creates a snapshot which considers no ops committed.
+  // Creates a snapshot which considers no ops applied.
   static MvccSnapshot CreateSnapshotIncludingNoOps();
 
-  // Return true if the given op timestamp should be considered committed in
+  // Return true if the given op timestamp should be considered applied in
   // this snapshot.
-  inline bool IsCommitted(const Timestamp& timestamp) const {
+  inline bool IsApplied(const Timestamp& timestamp) const {
     // Inline the most likely path, in which our watermarks determine whether
-    // an op is committed.
-    if (PREDICT_TRUE(timestamp < all_committed_before_)) {
+    // an op is applied.
+    if (PREDICT_TRUE(timestamp < all_applied_before_)) {
       return true;
     }
-    if (PREDICT_TRUE(timestamp >= none_committed_at_or_after_)) {
+    if (PREDICT_TRUE(timestamp >= none_applied_at_or_after_)) {
       return false;
     }
     // Out-of-line the unlikely case which involves more complex (loopy) code.
-    return IsCommittedFallback(timestamp);
+    return IsAppliedFallback(timestamp);
   }
 
-  // Returns true if this snapshot may have any committed ops with timestamp
+  // Returns true if this snapshot may have any applied ops with timestamp
   // equal to or higher than the provided 'timestamp'.
   // This is mostly useful to avoid scanning REDO deltas in certain cases.
-  // If MayHaveCommittedOpsAtOrAfter(delta_stats.min) returns true
+  // If MayHaveAppliedOpsAtOrAfter(delta_stats.min) returns true
   // it means that there might be ops that need to be applied in the context of
   // this snapshot; otherwise no scanning is necessary.
-  bool MayHaveCommittedOpsAtOrAfter(const Timestamp& timestamp) const;
+  bool MayHaveAppliedOpsAtOrAfter(const Timestamp& timestamp) const;
 
-  // Returns true if this snapshot may have any uncommitted ops with timestamp
+  // Returns true if this snapshot may have any nonapplied ops with timestamp
   // equal to or lower than the provided 'timestamp'.
   // This is mostly useful to avoid scanning UNDO deltas in certain cases.
-  // If MayHaveUncommittedOpsAtOrBefore(delta_stats.max) returns false it
-  // means that all UNDO delta ops are committed in the context of this
+  // If MayHaveNonAppliedOpsAtOrBefore(delta_stats.max) returns false it
+  // means that all UNDO delta ops are applied in the context of this
   // snapshot and no scanning is necessary; otherwise there might be some
   // ops that need to be undone.
-  bool MayHaveUncommittedOpsAtOrBefore(const Timestamp& timestamp) const;
+  bool MayHaveNonAppliedOpsAtOrBefore(const Timestamp& timestamp) const;
 
-  // Return a string representation of the set of committed ops in this
-  // snapshot, suitable for debug printouts.
+  // Return a string representation of the set of applied ops in this snapshot,
+  // suitable for debug printouts.
   std::string ToString() const;
 
   // Return true if the snapshot is considered 'clean'. A clean snapshot is one
   // which is determined only by a timestamp -- the snapshot considers all ops
-  // with timestamps less than some timestamp to be committed, and all other
-  // ops to be uncommitted.
+  // with timestamps less than some timestamp to be applied, and all other ops
+  // to be nonapplied.
   bool is_clean() const {
-    return committed_timestamps_.empty();
+    return applied_timestamps_.empty();
   }
 
-  // Consider the given list of timestamps to be committed in this snapshot,
+  // Consider the given list of timestamps to be applied in this snapshot,
   // even if they weren't when the snapshot was constructed.
-  // This is used in the flush path, where the set of commits going into a
+  // This is used in the flush path, where the set of applied ops going into a
   // flushed file may not be a consistent snapshot from the MVCC point of view,
   // yet we need to construct a scanner that accurately represents that set.
-  void AddCommittedTimestamps(const std::vector<Timestamp>& timestamps);
+  void AddAppliedTimestamps(const std::vector<Timestamp>& timestamps);
 
   // Returns true if 'other' represents the same set of timestamps as this
   // snapshot, false otherwise.
@@ -117,74 +117,71 @@ class MvccSnapshot {
 
  private:
   friend class MvccManager;
-  FRIEND_TEST(MvccTest, TestMayHaveCommittedOpsAtOrAfter);
-  FRIEND_TEST(MvccTest, TestMayHaveUncommittedOpsBefore);
-  FRIEND_TEST(MvccTest, TestWaitUntilAllCommitted_SnapAtTimestampWithInFlights);
-  FRIEND_TEST(MvccTest, TestCorrectInitWithNoTxns);
+  FRIEND_TEST(MvccTest, TestMayHaveAppliedOpsAtOrAfter);
+  FRIEND_TEST(MvccTest, TestMayHaveNonAppliedOpsBefore);
+  FRIEND_TEST(MvccTest, TestWaitUntilAllApplied_SnapAtTimestampWithInFlights);
+  FRIEND_TEST(MvccTest, TestCorrectInitWithNoOps);
 
-  bool IsCommittedFallback(const Timestamp& timestamp) const;
+  bool IsAppliedFallback(const Timestamp& timestamp) const;
 
-  void AddCommittedTimestamp(Timestamp timestamp);
+  void AddAppliedTimestamp(Timestamp timestamp);
 
   // Summary rule:
-  //   A op T is committed if and only if:
-  //      T < all_committed_before_ or
-  //   or committed_timestamps_.contains(T)
+  //   An op T is applied if and only if:
+  //      T < all_applied_before_ or
+  //   or applied_timestamps_.contains(T)
   //
-  // In ASCII form, where 'C' represents a committed op,
-  // and 'U' represents an uncommitted one:
+  // In ASCII form, where 'C' represents an applied op,
+  // and 'U' represents an nonapplied one:
   //
   //   CCCCCCCCCCCCCCCCCUUUUUCUUUCU
-  //                    |    \___\___ committed_timestamps_
+  //                    |    \___\___ applied_timestamps_
   //                    |
-  //                    \- all_committed_before_
+  //                    \- all_applied_before_
 
 
-  // An op timestamp below which all ops have been committed.
-  // For any timestamp X, if X < all_committed_timestamp_, then X is committed.
-  Timestamp all_committed_before_;
+  // An op timestamp below which all ops have been applied.
+  // For any timestamp X, if X < all_applied_before_, then X is applied.
+  Timestamp all_applied_before_;
 
-  // An op timestamp at or beyond which no ops have been committed.
-  // For any timestamp X, if X >= none_committed_after_, then X is uncommitted.
-  // This is equivalent to max(committed_timestamps_) + 1, but since
+  // An op timestamp at or beyond which no ops have been applied.
+  // For any timestamp X, if X >= none_applied_at_or_after_, then X is
+  // nonapplied. This is equivalent to max(applied_timestamps_) + 1, but since
   // that vector is unsorted, we cache it.
-  Timestamp none_committed_at_or_after_;
+  Timestamp none_applied_at_or_after_;
 
-  // The set of ops higher than all_committed_before_timestamp_ which are
-  // committed in this snapshot.
+  // The set of ops higher than all_applied_before_timestamp_ which are applied
+  // in this snapshot.
   // It might seem like using an unordered_set<> or a set<> would be faster here,
   // but in practice, this list tends to be stay pretty small, and is only
-  // rarely consulted (most data will be culled by 'all_committed_before_'
-  // or none_committed_at_or_after_. So, using the compact vector structure fits
+  // rarely consulted (most data will be culled by 'all_applied_before_'
+  // or none_applied_at_or_after_. So, using the compact vector structure fits
   // the whole thing on one or two cache lines, and it ends up going faster.
-  std::vector<Timestamp::val_type> committed_timestamps_;
+  std::vector<Timestamp::val_type> applied_timestamps_;
 
 };
 
-// TODO(awong): replace "commit" terminology with "applied" to disambiguate a
-// future implementation of multi-op transactions.
-//
 // Coordinator of MVCC ops. Threads wishing to make updates use
 // the MvccManager to obtain a unique timestamp, usually through the ScopedOp
 // class defined below.
 //
-// MVCC is used to defer updates until commit time, and allow iterators to
-// operate on a snapshot which contains only committed ops.
+// MVCC is used to defer updates until apply time, and allow iterators to
+// operate on a snapshot which contains only applied ops.
 //
 // There are two valid paths for an op:
 //
-// 1) StartOp() -> StartApplyingOp() -> CommitOp()
+// 1) StartOp() -> StartApplyingOp() -> FinishApplyingOp()
 //   or
 // 2) StartOp() -> AbortOp()
 //
 // When an op is ready to start making changes to in-memory data, it should
 // transition to APPLYING state by calling StartApplyingOp().  At this point,
-// the op should apply its in-memory operations and must commit in a
+// the op should apply its in-memory operations and must finish applying in a
 // bounded amount of time (i.e it should not wait on external input such as an
 // RPC from another host).
 //
 // NOTE: we do not support "rollback" of in-memory edits. Thus, once we call
-// StartApplyingOp(), the op _must_ commit.
+// StartApplyingOp(), the op _must_ finish applying.
 class MvccManager {
  public:
   MvccManager();
@@ -199,8 +196,8 @@ class MvccManager {
   // also now be 'timestamp' (see AdjustCleanTimeUnlocked() for more details).
   //
   // This must only called when we are guaranteed that there won't be new ops
-  // started at or below the given timestamp, e.g. the op is
-  // consensus committed and we're beginning to apply it.
+  // started at or below the given timestamp, e.g. the op is consensus
+  // committed and we're beginning to apply it.
   //
   // TODO(dralves): Until leader leases is implemented this should only be
   // called with the timestamps of consensus committed ops, not with the safe
@@ -215,20 +212,20 @@ class MvccManager {
   //
   // If 'timestamp' was marked safe before the call to this method (e.g. by TimeManager)
   // then the returned snapshot is repeatable.
-  Status WaitForSnapshotWithAllCommitted(Timestamp timestamp,
+  Status WaitForSnapshotWithAllApplied(Timestamp timestamp,
                                          MvccSnapshot* snapshot,
                                          const MonoTime& deadline) const WARN_UNUSED_RESULT;
 
-  // Wait for all operations that are currently APPLYING to commit.
+  // Wait for all operations that are currently APPLYING to finish applying.
   //
   // NOTE: this does _not_ guarantee that no ops are APPLYING upon return --
   // just that those that were APPLYING at call time are finished upon return.
   //
   // Returns Status::Aborted() if MVCC closed while waiting.
-  Status WaitForApplyingOpsToCommit() const WARN_UNUSED_RESULT;
+  Status WaitForApplyingOpsToApply() const WARN_UNUSED_RESULT;
 
-  // Returns the earliest possible timestamp for an uncommitted op. All
-  // timestamps before this one are guaranteed to be committed.
+  // Returns the earliest possible timestamp for an nonapplied op. All
+  // timestamps before this one are guaranteed to be applied.
   Timestamp GetCleanTimestamp() const;
 
   // Return the timestamps of all ops which are currently 'APPLYING' (i.e.
@@ -236,7 +233,7 @@ class MvccManager {
   // structures). Other ops may have reserved their timestamps via StartOp()
   // but not yet begun applying.
   //
-  // These ops are guaranteed to eventually Commit() -- i.e. they will never
+  // These ops are guaranteed to eventually FinishApplying() -- i.e. they will never
   // Abort().
   void GetApplyingOpsTimestamps(std::vector<Timestamp>* timestamps) const;
 
@@ -247,9 +244,9 @@ class MvccManager {
 
   ~MvccManager();
 
-  bool AreAllOpsCommittedForTests(Timestamp ts) const {
+  bool AreAllOpsAppliedForTests(Timestamp ts) const {
     std::lock_guard<LockType> l(lock_);
-    return AreAllOpsCommittedUnlocked(ts);
+    return AreAllOpsAppliedUnlocked(ts);
   }
 
   int GetNumWaitersForTests() const {
@@ -261,7 +258,7 @@ class MvccManager {
   friend class MvccSnapshot;
   friend class MvccTest;
   friend class ScopedOp;
-  FRIEND_TEST(MvccTest, TestAutomaticCleanTimeMoveToSafeTimeOnCommit);
+  FRIEND_TEST(MvccTest, TestAutomaticCleanTimeMoveToSafeTimeOnApply);
   FRIEND_TEST(MvccTest, TestIllegalStateTransitionsCrash);
   FRIEND_TEST(MvccTest, TestTxnAbort);
 
@@ -272,12 +269,12 @@ class MvccManager {
 
   // Begins a new op, which is assigned the provided timestamp.
   //
-  // Requires that 'timestamp' is not committed is greater than
+  // Requires that 'timestamp' is not applied is greater than
   // 'new_op_timestamp_exc_lower_bound_'.
   void StartOp(Timestamp timestamp);
 
   // Mark that the op with the given timestamp is starting to apply its writes
-  // to in-memory stores. This must be called before CommitOp().  If this is
+  // to in-memory stores. This must be called before FinishApplyingOp().  If this is
   // called, then AbortOp(timestamp) must never be called.
   void StartApplyingOp(Timestamp timestamp);
 
@@ -293,25 +290,25 @@ class MvccManager {
   // StartApplyingOp(), or else this logs a FATAL error.
   void AbortOp(Timestamp timestamp);
 
-  // Commit the given op.
+  // Finish applying the given op.
   //
   // If the op is not currently in-flight, this will trigger an assertion
-  // error. It is an error to commit the same op more than once.
+  // error. It is an error to finish applying the same op more than once.
   //
   // The op must already have been marked as 'APPLYING' by calling
   // StartApplyingOp(), or else this logs a FATAL error.
-  void CommitOp(Timestamp timestamp);
+  void FinishApplyingOp(Timestamp timestamp);
 
   // Take a snapshot of the current MVCC state, which indicates which ops have
-  // been committed at the time of this call.
+  // been applied at the time of this call.
   void TakeSnapshot(MvccSnapshot *snapshot) const;
 
   bool InitOpUnlocked(const Timestamp& timestamp);
 
-  // TODO(dralves) ponder merging these since the new ALL_COMMITTED path no longer
+  // TODO(awong) ponder merging these since the new ALL_APPLIED path no longer
   // waits for the clean timestamp.
   enum WaitFor {
-    ALL_COMMITTED,
+    ALL_APPLIED,
     NONE_APPLYING
   };
 
@@ -324,17 +321,17 @@ class MvccManager {
   // Returns an error if the MVCC manager is closed.
   Status CheckOpen() const;
 
-  // Returns true if all ops before the given timestamp are committed.
+  // Returns true if all ops before the given timestamp are applied.
   //
   // If 'ts' is not in the past, it's still possible that new ops could
   // start with a lower timestamp after this returns.
-  bool AreAllOpsCommittedUnlocked(Timestamp ts) const;
+  bool AreAllOpsAppliedUnlocked(Timestamp ts) const;
 
   // Return true if there is any APPLYING operation with a timestamp
   // less than or equal to 'ts'.
   bool AnyApplyingAtOrBeforeUnlocked(Timestamp ts) const;
 
-  // Waits until all ops before the given time are committed.
+  // Waits until all ops before the given time are applied.
   Status WaitUntil(WaitFor wait_for, Timestamp ts,
                    const MonoTime& deadline) const WARN_UNUSED_RESULT;
 
@@ -344,7 +341,7 @@ class MvccManager {
 
   // Commits the given op.
   // Sets *was_earliest to true if this was the earliest in-flight op.
-  void CommitOpUnlocked(Timestamp timestamp,
+  void ApplyOpUnlocked(Timestamp timestamp,
                         bool* was_earliest_in_flight);
 
   // Remove the timestamp 'ts' from the in-flight map.
@@ -353,7 +350,7 @@ class MvccManager {
   TxnState RemoveInFlightAndGetStateUnlocked(Timestamp ts);
 
   // Adjusts the clean time, i.e. the timestamp such that all ops with lower
-  // timestamps are committed or aborted, based on which ops are currently in
+  // timestamps are applied or aborted, based on which ops are currently in
   // flight and on what is the latest value of
   // 'new_op_timestamp_exc_lower_bound_'.
   //
@@ -362,7 +359,7 @@ class MvccManager {
 
   // Advances the earliest in-flight timestamp, based on which ops are
   // currently in-flight. Usually called when the previous earliest op
-  // commits or aborts.
+  // finishes applying or aborts.
   void AdvanceEarliestInFlightTimestamp();
 
   typedef simple_spinlock LockType;
@@ -384,7 +381,7 @@ class MvccManager {
 
   // The minimum timestamp in timestamps_in_flight_, or Timestamp::kMax
   // if that set is empty. This is cached in order to avoid having to iterate
-  // over timestamps_in_flight_ on every commit.
+  // over timestamps_in_flight_ on every apply.
   Timestamp earliest_in_flight_;
 
   mutable std::vector<WaitingState*> waiters_;
@@ -395,17 +392,16 @@ class MvccManager {
 };
 
 // A scoped handle to a running op.
-// When this object goes out of scope, the op is automatically
-// committed.
+// When this object goes out of scope, the op automatically finishes applying.
 class ScopedOp {
  public:
   // Create a new op from the given MvccManager.
   //
-  // When this op is committed it will use MvccManager::CommitOp().
+  // When this op is applied it will use MvccManager::FinishApplyingOp().
   ScopedOp(MvccManager* manager, Timestamp timestamp);
 
   // Commit the op referenced by this scoped object, if it hasn't
-  // already been committed.
+  // already been applied.
   ~ScopedOp();
 
   Timestamp timestamp() const {
@@ -415,14 +411,14 @@ class ScopedOp {
   // Mark that this op is about to begin applying its modifications to
   // in-memory stores.
   //
-  // This must be called before Commit(). Abort() may not be called after this
+  // This must be called before FinishApplying(). Abort() may not be called after this
   // method.
   void StartApplying();
 
   // Commit the in-flight op.
   //
   // Requires that StartApplying() has been called.
-  void Commit();
+  void FinishApplying();
 
   // Abort the in-flight op.
   //
diff --git a/src/kudu/tablet/ops/alter_schema_op.cc b/src/kudu/tablet/ops/alter_schema_op.cc
index 72b5833..1695216 100644
--- a/src/kudu/tablet/ops/alter_schema_op.cc
+++ b/src/kudu/tablet/ops/alter_schema_op.cc
@@ -160,7 +160,7 @@ void AlterSchemaOp::Finish(OpResult result) {
   // Tablet::AlterSchema().
   state()->ReleaseSchemaLock();
 
-  DCHECK_EQ(result, Op::COMMITTED);
+  DCHECK_EQ(result, Op::APPLIED);
   // Now that all of the changes have been applied and the commit is durable
   // make the changes visible to readers.
   TRACE("AlterSchemaCommitCallback: making alter schema visible");
diff --git a/src/kudu/tablet/ops/op.h b/src/kudu/tablet/ops/op.h
index 0f973f8..c9a6237 100644
--- a/src/kudu/tablet/ops/op.h
+++ b/src/kudu/tablet/ops/op.h
@@ -86,7 +86,7 @@ class Op {
   };
 
   enum OpResult {
-    COMMITTED,
+    APPLIED,
     ABORTED
   };
 
@@ -128,12 +128,12 @@ class Op {
   // method where data-structures are changed.
   virtual Status Apply(consensus::CommitMsg** commit_msg) = 0;
 
-  // Executed after the op has been applied and the commit message has
+  // Executed after the op has been applied and the Raft commit message has
   // been appended to the log (though it might not be durable yet), or if the
   // op was aborted.
   // Implementations are expected to perform cleanup on this method, the driver
   // will reply to the client after this method call returns.
-  // 'result' will be either COMMITTED or ABORTED, letting implementations
+  // 'result' will be either APPLIED or ABORTED, letting implementations
   // know what was the final status of the op.
   virtual void Finish(OpResult result) {}
 
diff --git a/src/kudu/tablet/ops/op_driver.cc b/src/kudu/tablet/ops/op_driver.cc
index 020d81c..eb8f630 100644
--- a/src/kudu/tablet/ops/op_driver.cc
+++ b/src/kudu/tablet/ops/op_driver.cc
@@ -507,7 +507,7 @@ void OpDriver::ApplyTask() {
   }
 #endif // #if DCHECK_IS_ON() ...
 
-  // We need to ref-count ourself, since Commit() may run very quickly
+  // We need to ref-count ourself, since FinishApplying() may run very quickly
   // and end up calling Finalize() while we're still in this code.
   scoped_refptr<OpDriver> ref(this);
 
@@ -574,7 +574,7 @@ void OpDriver::Finalize() {
   // object while we still hold the lock.
   scoped_refptr<OpDriver> ref(this);
   std::lock_guard<simple_spinlock> lock(lock_);
-  op_->Finish(Op::COMMITTED);
+  op_->Finish(Op::APPLIED);
   mutable_state()->completion_callback()->OpCompleted();
   op_tracker_->Release(this);
 }
diff --git a/src/kudu/tablet/ops/participant_op.cc b/src/kudu/tablet/ops/participant_op.cc
index c1a6d9e..cf9250d 100644
--- a/src/kudu/tablet/ops/participant_op.cc
+++ b/src/kudu/tablet/ops/participant_op.cc
@@ -179,10 +179,10 @@ void ParticipantOp::Finish(OpResult result) {
     TRACE("FINISH: Op aborted");
     return;
   }
-  DCHECK_EQ(result, Op::COMMITTED);
+  DCHECK_EQ(result, Op::APPLIED);
   // TODO(awong): when implementing transaction cleanup on participants, clean
   // up finalized and aborted transactions here.
-  TRACE("FINISH: Op committed");
+  TRACE("FINISH: Op applied");
 }
 
 string ParticipantOp::ToString() const {
diff --git a/src/kudu/tablet/ops/write_op.cc b/src/kudu/tablet/ops/write_op.cc
index f4e726c..87c9dc5 100644
--- a/src/kudu/tablet/ops/write_op.cc
+++ b/src/kudu/tablet/ops/write_op.cc
@@ -252,14 +252,14 @@ Status WriteOp::Apply(CommitMsg** commit_msg) {
 void WriteOp::Finish(OpResult result) {
   TRACE_EVENT0("op", "WriteOp::Finish");
 
-  state()->CommitOrAbort(result);
+  state()->FinishApplyingOrAbort(result);
 
   if (PREDICT_FALSE(result == Op::ABORTED)) {
     TRACE("FINISH: Op aborted.");
     return;
   }
 
-  DCHECK_EQ(result, Op::COMMITTED);
+  DCHECK_EQ(result, Op::APPLIED);
 
   TRACE("FINISH: Updating metrics.");
 
@@ -377,7 +377,7 @@ void WriteOpState::StartApplying() {
   CHECK_NOTNULL(mvcc_op_.get())->StartApplying();
 }
 
-void WriteOpState::CommitOrAbort(Op::OpResult result) {
+void WriteOpState::FinishApplyingOrAbort(Op::OpResult result) {
   ReleaseMvccTxn(result);
 
   TRACE("Releasing row and schema locks");
@@ -394,8 +394,8 @@ void WriteOpState::ReleaseMvccTxn(Op::OpResult result) {
   if (mvcc_op_) {
     // Commit the op.
     switch (result) {
-      case Op::COMMITTED:
-        mvcc_op_->Commit();
+      case Op::APPLIED:
+        mvcc_op_->FinishApplying();
         break;
       case Op::ABORTED:
         mvcc_op_->Abort();
@@ -469,7 +469,7 @@ WriteOpState::~WriteOpState() {
 }
 
 void WriteOpState::Reset() {
-  CommitOrAbort(Op::ABORTED);
+  FinishApplyingOrAbort(Op::ABORTED);
   op_metrics_.Reset();
   timestamp_ = Timestamp::kInvalidTimestamp;
   tablet_components_ = nullptr;
diff --git a/src/kudu/tablet/ops/write_op.h b/src/kudu/tablet/ops/write_op.h
index 2bb5201..3c69eac 100644
--- a/src/kudu/tablet/ops/write_op.h
+++ b/src/kudu/tablet/ops/write_op.h
@@ -185,7 +185,7 @@ class WriteOpState : public OpState {
   }
 
   // Notifies the MVCC manager that this operation is about to start applying
-  // its in-memory edits. After this method is called, the op _must_ Commit()
+  // its in-memory edits. After this method is called, the op _must_ FinishApplying()
   // within a bounded amount of time (there may be other threads blocked on
   // it).
   void StartApplying();
@@ -199,7 +199,7 @@ class WriteOpState : public OpState {
   // REQUIRES: StartApplying() was called.
   //
   // Note: request_ and response_ are set to NULL after this method returns.
-  void CommitOrAbort(Op::OpResult result);
+  void FinishApplyingOrAbort(Op::OpResult result);
 
   // Returns all the prepared row writes for this op. Usually called on the
   // apply phase to actually make changes to the tablet.
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index 90f744d..b633301 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -1188,7 +1188,7 @@ Status Tablet::FlushUnlocked() {
   // it.
   //
   // This may fail if the tablet has been stopped.
-  RETURN_NOT_OK(mvcc_.WaitForApplyingOpsToCommit());
+  RETURN_NOT_OK(mvcc_.WaitForApplyingOpsToApply());
 
   // Note: "input" should only contain old_mrs.
   return FlushInternal(input, old_mrs);
@@ -1718,11 +1718,11 @@ Status Tablet::DoMergeCompactionOrFlush(const RowSetsInCompaction &input,
   // for those ops in 'applying_during_swap', but MVCC doesn't implement the
   // ability to wait for a specific set. So instead we wait for all currently
   // applying -- a bit more than we need, but still correct.
-  RETURN_NOT_OK(mvcc_.WaitForApplyingOpsToCommit());
+  RETURN_NOT_OK(mvcc_.WaitForApplyingOpsToApply());
 
   // Then we want to consider all those ops that were in-flight when we did the
   // swap as committed in 'non_duplicated_ops_snap'.
-  non_duplicated_ops_snap.AddCommittedTimestamps(applying_during_swap);
+  non_duplicated_ops_snap.AddAppliedTimestamps(applying_during_swap);
 
   if (common_hooks_) {
     RETURN_NOT_OK_PREPEND(common_hooks_->PostSwapInDuplicatingRowSet(),
diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc
index ae53d0a..976db5d 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -1434,7 +1434,7 @@ Status TabletBootstrap::PlayWriteRequest(const IOContext* io_context,
   Status play_status;
   if (!all_flushed && write->has_row_operations()) {
     // Rather than RETURN_NOT_OK() here, we need to just save the status and do the
-    // RETURN_NOT_OK() down below the Commit() call below. Even though it seems wrong
+    // RETURN_NOT_OK() down below the FinishApplying() call below. Even though it seems wrong
     // to commit the op when in fact it failed to apply, we would throw a CHECK
     // failure if we attempted to 'Abort()' after entering the applying stage. Allowing it to
     // Commit isn't problematic because we don't expose the results anyway, and the bad
@@ -1447,7 +1447,7 @@ Status TabletBootstrap::PlayWriteRequest(const IOContext* io_context,
     }
   }
 
-  op_state.CommitOrAbort(Op::COMMITTED);
+  op_state.FinishApplyingOrAbort(Op::APPLIED);
 
   // If we failed to apply the operations, fail bootstrap before we write anything incorrect
   // to the recovery log.
diff --git a/src/kudu/tablet/tablet_replica.cc b/src/kudu/tablet/tablet_replica.cc
index 1716fc9..a41609e 100644
--- a/src/kudu/tablet/tablet_replica.cc
+++ b/src/kudu/tablet/tablet_replica.cc
@@ -931,10 +931,10 @@ Status FlushInflightsToLogCallback::WaitForInflightsAndFlushLog() {
   //
   // So, to enforce this property, we do two steps:
   //
-  // 1) Wait for any operations which are already mid-Apply() to Commit() in MVCC.
+  // 1) Wait for any operations which are already mid-Apply() to FinishApplying() in MVCC.
   //
   // Because the operations always enqueue their COMMIT message to the log
-  // before calling Commit(), this ensures that any in-flight operations have
+  // before calling FinishApplying(), this ensures that any in-flight operations have
   // their commit messages "en route".
   //
   // NOTE: we only wait for those operations that have started their Apply() phase.
@@ -957,7 +957,7 @@ Status FlushInflightsToLogCallback::WaitForInflightsAndFlushLog() {
   VLOG(1) << "T " << tablet_->metadata()->tablet_id()
       <<  ": Waiting for in-flight ops to commit.";
   LOG_SLOW_EXECUTION(WARNING, 200, "Committing in-flights took a long time.") {
-    RETURN_NOT_OK(tablet_->mvcc_manager()->WaitForApplyingOpsToCommit());
+    RETURN_NOT_OK(tablet_->mvcc_manager()->WaitForApplyingOpsToApply());
   }
   VLOG(1) << "T " << tablet_->metadata()->tablet_id()
       << ": Waiting for the log queue to be flushed.";
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index f1a6c74..dc99eca 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -3058,7 +3058,7 @@ Status TabletServiceImpl::HandleScanAtSnapshot(const NewScanRequestPB& scan_pb,
   if (PREDICT_TRUE(s.ok())) {
     // Wait for the in-flights in the snapshot to be finished.
     TRACE("Waiting for operations to commit");
-    s = tablet->mvcc_manager()->WaitForSnapshotWithAllCommitted(
+    s = tablet->mvcc_manager()->WaitForSnapshotWithAllApplied(
           tmp_snap_timestamp, &snap, client_deadline);
   }