You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2020/12/08 21:16:11 UTC

[kudu] 05/05: txn_participant: return TXN_ILLEGAL_STATE code on illegal state errors

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

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

commit 57c757d084956710143b8d85fddb4d6f1fc87211
Author: Andrew Wong <aw...@cloudera.com>
AuthorDate: Mon Dec 7 19:00:30 2020 -0800

    txn_participant: return TXN_ILLEGAL_STATE code on illegal state errors
    
    I have a patch coming up that sends RPCs that submit participant ops to
    the TxnParticipant and retries if necessary. One of the criteria to
    retry an RPC is checking if the Raft application returned an
    IllegalState error, e.g. in RaftConsensus::CheckLeadershipAndBindTerm.
    
    This is problematic, as TxnParticipants may also return IllegalState if
    the transaction is not in an appropriate state (e.g. an RPC tries to
    abort an already-committed transaction). To disambiguate such cases,
    this patch updates the TxnParticipants to also set a TabletServerErrorPB
    code when such an error occurs, allowing for more specific error
    handling to be determined.
    
    Change-Id: Ibfb8400666855c694b78b1425b1c121597ec7ccf
    Reviewed-on: http://gerrit.cloudera.org:8080/16831
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Andrew Wong <aw...@cloudera.com>
---
 .../integration-tests/txn_participant-itest.cc     | 188 +++++++++++++--------
 src/kudu/tablet/ops/participant_op.cc              |  27 ++-
 src/kudu/tablet/ops/participant_op.h               |   5 +-
 src/kudu/tablet/txn_participant-test.cc            |   3 +
 src/kudu/tablet/txn_participant.h                  |  24 ++-
 5 files changed, 159 insertions(+), 88 deletions(-)

diff --git a/src/kudu/integration-tests/txn_participant-itest.cc b/src/kudu/integration-tests/txn_participant-itest.cc
index 0373b69..b962571 100644
--- a/src/kudu/integration-tests/txn_participant-itest.cc
+++ b/src/kudu/integration-tests/txn_participant-itest.cc
@@ -18,6 +18,7 @@
 #include <atomic>
 #include <cstdint>
 #include <functional>
+#include <initializer_list>
 #include <memory>
 #include <string>
 #include <thread>
@@ -93,6 +94,7 @@ using kudu::tserver::ParticipantOpPB;
 using kudu::tserver::ParticipantRequestPB;
 using kudu::tserver::ParticipantResponsePB;
 using kudu::tserver::TabletServerAdminServiceProxy;
+using kudu::tserver::TabletServerErrorPB;
 using kudu::tserver::WriteRequestPB;
 using std::string;
 using std::thread;
@@ -129,10 +131,14 @@ Status ParticipateInTransaction(TabletServerAdminServiceProxy* admin_proxy,
 
 Status ParticipateInTransactionCheckResp(TabletServerAdminServiceProxy* admin_proxy,
                                          const string& tablet_id, int64_t txn_id,
-                                         ParticipantOpPB::ParticipantOpType type) {
+                                         ParticipantOpPB::ParticipantOpType type,
+                                         TabletServerErrorPB::Code* code = nullptr) {
   ParticipantResponsePB resp;
   RETURN_NOT_OK(ParticipateInTransaction(admin_proxy, tablet_id, txn_id, type, &resp));
   if (resp.has_error()) {
+    if (code) {
+      *code = resp.error().code();
+    }
     return StatusFromPB(resp.error().status());
   }
   return Status::OK();
@@ -506,7 +512,7 @@ TEST_F(TxnParticipantITest, TestProxyBasicCalls) {
   }
 }
 
-TEST_F(TxnParticipantITest, TestProxyInvalidStatesInCommitSequence) {
+TEST_F(TxnParticipantITest, TestProxyIllegalStatesInCommitSequence) {
   constexpr const int kLeaderIdx = 0;
   constexpr const int kTxnId = 0;
   vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
@@ -515,95 +521,135 @@ TEST_F(TxnParticipantITest, TestProxyInvalidStatesInCommitSequence) {
 
   // Begin after already beginning.
   const auto tablet_id = replicas[kLeaderIdx]->tablet_id();
-  ASSERT_OK(ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN));
-  // TODO(awong): IllegalState error codes may be easily misconstrued with
-  // errors coming from Raft. We should make the participants return something
-  // like InvalidArgument if there's an unexpected state.
-  // TODO(awong): make repeated, idempotent return OK instead of an error.
-  Status s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    ASSERT_OK(ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN, &code));
+    ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
+  }
+  // TODO(awong): make repeated, idempotent return OK instead of an error? Or
+  // return a different error code so callers can distinguish between a benign
+  // error.
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
 
   // We can't finalize the commit without beginning to commit first.
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::FINALIZE_COMMIT);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::FINALIZE_COMMIT, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
 
   // Start commititng and ensure we can't start another transaction.
-  ASSERT_OK(ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_COMMIT));
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_COMMIT);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    ASSERT_OK(ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_COMMIT, &code));
+    ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
+  }
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_COMMIT, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
 
   // Finalize the commit and ensure we can do nothing else.
-  ASSERT_OK(ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::FINALIZE_COMMIT));
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::FINALIZE_COMMIT);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_COMMIT);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::ABORT_TXN);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    ASSERT_OK(ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::FINALIZE_COMMIT, &code));
+    ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
+  }
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::FINALIZE_COMMIT, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_COMMIT, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::ABORT_TXN, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
 }
 
-TEST_F(TxnParticipantITest, TestProxyInvalidStatesInAbortSequence) {
+TEST_F(TxnParticipantITest, TestProxyIllegalStatesInAbortSequence) {
   constexpr const int kLeaderIdx = 0;
   constexpr const int kTxnId = 0;
   vector<TabletReplica*> replicas = SetUpLeaderGetReplicas(kLeaderIdx);
   ASSERT_OK(replicas[kLeaderIdx]->consensus()->WaitUntilLeaderForTests(MonoDelta::FromSeconds(10)));
   auto admin_proxy = cluster_->tserver_admin_proxy(kLeaderIdx);
 
-  // Begin after already beginning.
+  // Try our illegal ops when our transaction is open.
   const auto tablet_id = replicas[kLeaderIdx]->tablet_id();
   ASSERT_OK(ParticipateInTransactionCheckResp(
       admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN));
-  // TODO(awong): IllegalState error codes may be easily misconstrued with
-  // errors coming from Raft. We should make the participants return something
-  // like InvalidArgument if there's an unexpected state.
   // TODO(awong): make repeated, idempotent return OK instead of an error.
-  Status s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::FINALIZE_COMMIT);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::FINALIZE_COMMIT, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
 
   // Abort the transaction and ensure we can do nothing else.
-  ASSERT_OK(ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::ABORT_TXN));
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::ABORT_TXN);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::FINALIZE_COMMIT);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_TXN);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
-
-  s = ParticipateInTransactionCheckResp(
-      admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::BEGIN_COMMIT);
-  ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+  {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    ASSERT_OK(ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, ParticipantOpPB::ABORT_TXN, &code));
+    ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, code);
+  }
+  for (auto type : { ParticipantOpPB::ABORT_TXN,
+                     ParticipantOpPB::FINALIZE_COMMIT,
+                     ParticipantOpPB::BEGIN_TXN,
+                     ParticipantOpPB::BEGIN_COMMIT}) {
+    TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+    Status s = ParticipateInTransactionCheckResp(
+        admin_proxy.get(), tablet_id, kTxnId, type, &code);
+    ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
+    ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, code);
+  }
 }
 
 TEST_F(TxnParticipantITest, TestProxyNonLeader) {
diff --git a/src/kudu/tablet/ops/participant_op.cc b/src/kudu/tablet/ops/participant_op.cc
index fb5295b..621f24e 100644
--- a/src/kudu/tablet/ops/participant_op.cc
+++ b/src/kudu/tablet/ops/participant_op.cc
@@ -36,6 +36,7 @@
 #include "kudu/tablet/tablet.h"
 #include "kudu/tablet/tablet_replica.h"
 #include "kudu/tablet/txn_participant.h"
+#include "kudu/tserver/tserver.pb.h"
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
@@ -48,6 +49,7 @@ using kudu::consensus::OpId;
 using kudu::pb_util::SecureShortDebugString;
 using kudu::tablet::TabletReplica;
 using kudu::tserver::ParticipantOpPB;
+using kudu::tserver::TabletServerErrorPB;
 using std::string;
 using std::unique_ptr;
 using strings::Substitute;
@@ -91,20 +93,31 @@ string ParticipantOpState::ToString() const {
       ParticipantOpPB::ParticipantOpType_Name(request_->op().type()));
 }
 
-Status ParticipantOpState::ValidateOp() const {
+Status ParticipantOpState::ValidateOp() {
   const auto& op = request()->op();
   DCHECK(txn_);
+  TabletServerErrorPB::Code code = TabletServerErrorPB::UNKNOWN_ERROR;
+  Status s;
   switch (op.type()) {
     case ParticipantOpPB::BEGIN_TXN:
-      return txn_->ValidateBeginTransaction();
+      s = txn_->ValidateBeginTransaction(&code);
+      break;
     case ParticipantOpPB::BEGIN_COMMIT:
-      return txn_->ValidateBeginCommit();
+      s = txn_->ValidateBeginCommit(&code);
+      break;
     case ParticipantOpPB::FINALIZE_COMMIT:
-      return txn_->ValidateFinalize();
+      s = txn_->ValidateFinalize(&code);
+      break;
     case ParticipantOpPB::ABORT_TXN:
-      return txn_->ValidateAbort();
-    case ParticipantOpPB::UNKNOWN:
-      return Status::InvalidArgument("unknown op type");
+      s = txn_->ValidateAbort(&code);
+      break;
+    default:
+      s = Status::InvalidArgument("unknown op type");
+      break;
+  }
+  if (PREDICT_FALSE(!s.ok())) {
+    completion_callback()->set_error(s, code);
+    return s;
   }
   return Status::OK();
 }
diff --git a/src/kudu/tablet/ops/participant_op.h b/src/kudu/tablet/ops/participant_op.h
index 7c8bbd1..5c7a281 100644
--- a/src/kudu/tablet/ops/participant_op.h
+++ b/src/kudu/tablet/ops/participant_op.h
@@ -103,8 +103,9 @@ class ParticipantOpState : public OpState {
   friend class ParticipantOp;
 
   // Returns an error if the transaction is not in an appropriate state for
-  // the state change requested by this op.
-  Status ValidateOp() const;
+  // the state change requested by this op, also setting the OpState's callback
+  // error with an appropriate error code.
+  Status ValidateOp();
 
   // The particpant being mutated. This may differ from the one we'd get from
   // TabletReplica if, for instance, we're bootstrapping a new Tablet.
diff --git a/src/kudu/tablet/txn_participant-test.cc b/src/kudu/tablet/txn_participant-test.cc
index 868b0a4..3e24d07 100644
--- a/src/kudu/tablet/txn_participant-test.cc
+++ b/src/kudu/tablet/txn_participant-test.cc
@@ -75,6 +75,7 @@ using kudu::pb_util::SecureShortDebugString;
 using kudu::tserver::ParticipantRequestPB;
 using kudu::tserver::ParticipantResponsePB;
 using kudu::tserver::ParticipantOpPB;
+using kudu::tserver::TabletServerErrorPB;
 using kudu::tserver::WriteRequestPB;
 using std::map;
 using std::string;
@@ -254,6 +255,7 @@ TEST_F(TxnParticipantTest, TestTransactionNotFound) {
       SCOPED_TRACE(SecureShortDebugString(resp));
       ASSERT_TRUE(resp.has_error());
       ASSERT_TRUE(resp.error().has_status());
+      ASSERT_EQ(TabletServerErrorPB::UNKNOWN_ERROR, resp.error().code());
       ASSERT_EQ(AppStatusPB::NOT_FOUND, resp.error().status().code());
       ASSERT_FALSE(resp.has_timestamp());
     }
@@ -285,6 +287,7 @@ TEST_F(TxnParticipantTest, TestIllegalTransitions) {
       ASSERT_TRUE(resp.has_error());
       ASSERT_TRUE(resp.error().has_status());
       ASSERT_EQ(AppStatusPB::ILLEGAL_STATE, resp.error().status().code());
+      ASSERT_EQ(TabletServerErrorPB::TXN_ILLEGAL_STATE, resp.error().code());
       ASSERT_FALSE(resp.has_timestamp());
     }
   };
diff --git a/src/kudu/tablet/txn_participant.h b/src/kudu/tablet/txn_participant.h
index 3cea176..480b6c0 100644
--- a/src/kudu/tablet/txn_participant.h
+++ b/src/kudu/tablet/txn_participant.h
@@ -34,6 +34,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/mvcc.h"
 #include "kudu/tablet/tablet_metadata.h"
+#include "kudu/tserver/tserver.pb.h"
 #include "kudu/util/locks.h"
 #include "kudu/util/rw_semaphore.h"
 #include "kudu/util/status.h"
@@ -110,45 +111,50 @@ class Txn : public RefCountedThreadSafe<Txn> {
   // Validates that the transaction is in the appropriate state to perform the
   // given operation. Should be called while holding the state lock before
   // replicating a participant op.
-  Status ValidateBeginTransaction() const {
+  Status ValidateBeginTransaction(tserver::TabletServerErrorPB::Code* code) const {
     DCHECK(state_lock_.is_locked());
     if (PREDICT_FALSE(tablet_metadata_->HasTxnMetadata(txn_id_))) {
+      *code = tserver::TabletServerErrorPB::TXN_ILLEGAL_STATE;
       return Status::IllegalState(
           strings::Substitute("Transaction metadata for transaction $0 already exists",
                               txn_id_));
     }
     if (PREDICT_FALSE(state_ != kInitializing)) {
+      *code = tserver::TabletServerErrorPB::TXN_ILLEGAL_STATE;
       return Status::IllegalState(
           strings::Substitute("Cannot begin transaction in state: $0",
                               StateToString(state_)));
     }
     return Status::OK();
   }
-  Status ValidateBeginCommit() const {
+  Status ValidateBeginCommit(tserver::TabletServerErrorPB::Code* code) const {
     DCHECK(state_lock_.is_locked());
-    RETURN_NOT_OK(CheckFinishedInitializing());
+    RETURN_NOT_OK(CheckFinishedInitializing(code));
     if (PREDICT_FALSE(state_ != kOpen)) {
+      *code = tserver::TabletServerErrorPB::TXN_ILLEGAL_STATE;
       return Status::IllegalState(
           strings::Substitute("Cannot begin committing transaction in state: $0",
                               StateToString(state_)));
     }
     return Status::OK();
   }
-  Status ValidateFinalize() const {
+  Status ValidateFinalize(tserver::TabletServerErrorPB::Code* code) const {
     DCHECK(state_lock_.is_locked());
-    RETURN_NOT_OK(CheckFinishedInitializing());
+    RETURN_NOT_OK(CheckFinishedInitializing(code));
     if (PREDICT_FALSE(state_ != kCommitInProgress)) {
+      *code = tserver::TabletServerErrorPB::TXN_ILLEGAL_STATE;
       return Status::IllegalState(
           strings::Substitute("Cannot finalize transaction in state: $0",
                               StateToString(state_)));
     }
     return Status::OK();
   }
-  Status ValidateAbort() const {
+  Status ValidateAbort(tserver::TabletServerErrorPB::Code* code) const {
     DCHECK(state_lock_.is_locked());
-    RETURN_NOT_OK(CheckFinishedInitializing());
+    RETURN_NOT_OK(CheckFinishedInitializing(code));
     if (PREDICT_FALSE(state_ != kOpen &&
                       state_ != kCommitInProgress)) {
+      *code = tserver::TabletServerErrorPB::TXN_ILLEGAL_STATE;
       return Status::IllegalState(
           strings::Substitute("Cannot abort transaction in state: $0",
                               StateToString(state_)));
@@ -207,15 +213,17 @@ class Txn : public RefCountedThreadSafe<Txn> {
   }
 
   // Returns an error if the transaction has not finished initializing.
-  Status CheckFinishedInitializing() const {
+  Status CheckFinishedInitializing(tserver::TabletServerErrorPB::Code* code) const {
     if (PREDICT_FALSE(state_ == kInitializing)) {
       if (tablet_metadata_->HasTxnMetadata(txn_id_)) {
         // We created this transaction as a part of this op (i.e. it was not
         // already in flight), and there is existing metadata for it.
+        *code = tserver::TabletServerErrorPB::TXN_ILLEGAL_STATE;
         return Status::IllegalState(
             strings::Substitute("Transaction metadata for transaction $0 already exists",
                                 txn_id_));
       }
+      // TODO(awong): add another code for this?
       return Status::NotFound("Transaction hasn't been successfully started");
     }
     return Status::OK();