You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2018/05/14 18:35:58 UTC

kudu git commit: catalog-manager: refactor error handling

Repository: kudu
Updated Branches:
  refs/heads/master 05d594867 -> 1a707b991


catalog-manager: refactor error handling

This commit refactors how the SetupError helper works slightly in order
to reduce the prevelance of temporary Status locals. I've found the
Status locals to be error prone in the past due to shadowing.

Change-Id: I695a62a75a3b1979741e2d7105ed131eace64b6d
Reviewed-on: http://gerrit.cloudera.org:8080/10376
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 1a707b991785a5a7d4d316d721d813f49d0bb7e5
Parents: 05d5948
Author: Dan Burkert <da...@apache.org>
Authored: Thu May 10 11:11:10 2018 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Mon May 14 18:35:12 2018 +0000

----------------------------------------------------------------------
 src/kudu/client/client-test.cc     |  14 +-
 src/kudu/master/catalog_manager.cc | 244 ++++++++++++++------------------
 2 files changed, 117 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1a707b99/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index 28f12e2..6fad6e8 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -768,7 +768,7 @@ TEST_F(ClientTest, TestBadTable) {
   shared_ptr<KuduTable> t;
   Status s = client_->OpenTable("xxx-does-not-exist", &t);
   ASSERT_TRUE(s.IsNotFound());
-  ASSERT_STR_CONTAINS(s.ToString(), "Not found: The table does not exist");
+  ASSERT_STR_CONTAINS(s.ToString(), "Not found: the table does not exist");
 }
 
 // Test that, if the master is down, we experience a network error talking
@@ -3926,7 +3926,7 @@ TEST_F(ClientTest, TestDeleteTable) {
   // Try to open the deleted table
   Status s = client_->OpenTable(kTableName, &client_table_);
   ASSERT_TRUE(s.IsNotFound());
-  ASSERT_STR_CONTAINS(s.ToString(), "The table does not exist");
+  ASSERT_STR_CONTAINS(s.ToString(), "the table does not exist");
 
   // Create a new table with the same name. This is to ensure that the client
   // doesn't cache anything inappropriately by table name (see KUDU-1055).
@@ -3946,7 +3946,7 @@ TEST_F(ClientTest, TestGetTableSchema) {
   // Verify that a get schema request for a missing table throws not found
   Status s = client_->GetTableSchema("MissingTableName", &schema);
   ASSERT_TRUE(s.IsNotFound());
-  ASSERT_STR_CONTAINS(s.ToString(), "The table does not exist");
+  ASSERT_STR_CONTAINS(s.ToString(), "the table does not exist");
 }
 
 // Test creating and accessing a table which has multiple tablets,
@@ -4439,15 +4439,15 @@ TEST_F(ClientTest, TestCreateTableWithTooManyTablets) {
       .Create();
   ASSERT_TRUE(s.IsInvalidArgument());
   ASSERT_STR_CONTAINS(s.ToString(),
-                      "The requested number of tablets is over the "
+                      "the requested number of tablets is over the "
                       "maximum permitted at creation time (1)");
 }
 
 // Tests for too many replicas, too few replicas, even replica count, etc.
 TEST_F(ClientTest, TestCreateTableWithBadNumReplicas) {
   const vector<pair<int, string>> cases = {
-    {3, "Not enough live tablet servers to create a table with the requested "
-     "replication factor 3. 1 tablet servers are alive"},
+    {3, "not enough live tablet servers to create a table with the requested "
+     "replication factor 3; 1 tablet servers are alive"},
     {2, "illegal replication factor 2 (replication factor must be odd)"},
     {-1, "illegal replication factor -1 (replication factor must be positive)"},
     {11, "illegal replication factor 11 (max replication factor is 7)"}
@@ -4510,7 +4510,7 @@ TEST_F(ClientTest, TestCreateTable_TableNames) {
     // From http://stackoverflow.com/questions/1301402/example-invalid-utf8-string
     {string("foo\xf0\x28\x8c\xbc", 7), "invalid table name: invalid UTF8 sequence"},
     // Should pass validation but fail due to lack of tablet servers running.
-    {"你好", "Not enough live tablet servers"}
+    {"你好", "not enough live tablet servers"}
   };
 
   for (const auto& test_case : kCases) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/1a707b99/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index bf87e4f..1de0be1 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -588,6 +588,16 @@ string RequestorString(RpcContext* rpc) {
   }
 }
 
+// If 's' is not OK, fills in the RPC response with the error and provided code. Returns 's'.
+template<typename RespClass>
+Status SetupError(Status s, RespClass* resp, MasterErrorPB::Code code) {
+  if (PREDICT_FALSE(!s.ok())) {
+    StatusToPB(s, resp->mutable_error()->mutable_status());
+    resp->mutable_error()->set_code(code);
+  }
+  return s;
+}
+
 // If 's' indicates that the node is no longer the leader, setup
 // Service::UnavailableError as the error, set NOT_THE_LEADER as the
 // error code and return true.
@@ -600,24 +610,22 @@ void CheckIfNoLongerLeaderAndSetupError(const Status& s, RespClass* resp) {
   // that is no longer the leader, this suffices until we
   // distinguish this cause of write failure more explicitly.
   if (s.IsIllegalState() || s.IsAborted()) {
-    Status new_status = Status::ServiceUnavailable(
-        "operation requested can only be executed on a leader master, but this"
-        " master is no longer the leader", s.ToString());
-    SetupError(resp->mutable_error(), MasterErrorPB::NOT_THE_LEADER, new_status);
+    SetupError(Status::ServiceUnavailable(
+          "operation requested can only be executed on a leader master, but this"
+          " master is no longer the leader", s.ToString()),
+        resp, MasterErrorPB::NOT_THE_LEADER);
   }
 }
 
 template<class RespClass>
 Status CheckIfTableDeletedOrNotRunning(TableMetadataLock* lock, RespClass* resp) {
   if (lock->data().is_deleted()) {
-    Status s = Status::NotFound("The table was deleted", lock->data().pb.state_msg());
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(Status::NotFound("the table was deleted", lock->data().pb.state_msg()),
+                      resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   if (!lock->data().is_running()) {
-    Status s = Status::ServiceUnavailable("The table is not running");
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(Status::ServiceUnavailable("the table is not running"),
+                      resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   return Status::OK();
 }
@@ -1210,13 +1218,6 @@ void CatalogManager::Shutdown() {
   }
 }
 
-static void SetupError(MasterErrorPB* error,
-                       MasterErrorPB::Code code,
-                       const Status& s) {
-  StatusToPB(s, error->mutable_status());
-  error->set_code(code);
-}
-
 Status CatalogManager::CheckOnline() const {
   if (PREDICT_FALSE(!IsInitialized())) {
     return Status::ServiceUnavailable("CatalogManager is not running");
@@ -1303,11 +1304,6 @@ Status ValidateClientSchema(const boost::optional<string>& name,
 Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
                                    CreateTableResponsePB* resp,
                                    rpc::RpcContext* rpc) {
-  auto SetError = [&](MasterErrorPB::Code code, const Status& s) {
-    SetupError(resp->mutable_error(), code, s);
-    return s;
-  };
-
   leader_lock_.AssertAcquiredForReading();
   RETURN_NOT_OK(CheckOnline());
 
@@ -1323,22 +1319,19 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
   // on the protobuf here.
   for (int i = 0; i < req.schema().columns_size(); i++) {
     auto* col = req.mutable_schema()->mutable_columns(i);
-    Status s = ProcessColumnPBDefaults(col);
-    if (!s.ok()) {
-      return SetError(MasterErrorPB::INVALID_SCHEMA, s);
-    }
+    RETURN_NOT_OK(SetupError(ProcessColumnPBDefaults(col), resp, MasterErrorPB::INVALID_SCHEMA));
   }
 
   // a. Validate the user request.
   Schema client_schema;
   RETURN_NOT_OK(SchemaFromPB(req.schema(), &client_schema));
   const string& table_name = req.name();
-  Status s = ValidateClientSchema(table_name, client_schema);
-  if (s.ok() && client_schema.has_column_ids()) {
-    s = Status::InvalidArgument("User requests should not have Column IDs");
-  }
-  if (!s.ok()) {
-    return SetError(MasterErrorPB::INVALID_SCHEMA, s);
+
+  RETURN_NOT_OK(SetupError(ValidateClientSchema(table_name, client_schema),
+                           resp, MasterErrorPB::INVALID_SCHEMA));
+  if (client_schema.has_column_ids()) {
+    return SetupError(Status::InvalidArgument("user requests should not have Column IDs"),
+                      resp, MasterErrorPB::INVALID_SCHEMA);
   }
   Schema schema = client_schema.CopyWithColumnIds();
 
@@ -1346,10 +1339,9 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
   // the default partition schema (no hash bucket components and a range
   // partitioned on the primary key columns) will be used.
   PartitionSchema partition_schema;
-  s = PartitionSchema::FromPB(req.partition_schema(), schema, &partition_schema);
-  if (!s.ok()) {
-    return SetError(MasterErrorPB::INVALID_SCHEMA, s);
-  }
+  RETURN_NOT_OK(SetupError(
+        PartitionSchema::FromPB(req.partition_schema(), schema, &partition_schema),
+        resp, MasterErrorPB::INVALID_SCHEMA));
 
   // Decode split rows.
   vector<KuduPartialRow> split_rows;
@@ -1373,9 +1365,9 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
         if (i >= ops.size() ||
             (ops[i].type != RowOperationsPB::RANGE_UPPER_BOUND &&
              ops[i].type != RowOperationsPB::INCLUSIVE_RANGE_UPPER_BOUND)) {
-          return SetError(MasterErrorPB::UNKNOWN_ERROR,
-                          Status::InvalidArgument(
-                              "Missing upper range bound in create table request"));
+          return SetupError(
+              Status::InvalidArgument("missing upper range bound in create table request"),
+              resp, MasterErrorPB::UNKNOWN_ERROR);
         }
 
         if (op.type == RowOperationsPB::EXCLUSIVE_RANGE_LOWER_BOUND) {
@@ -1408,25 +1400,22 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
   // Reject create table with even replication factors, unless master flag
   // allow_unsafe_replication_factor is on.
   if (num_replicas % 2 == 0 && !FLAGS_allow_unsafe_replication_factor) {
-    s = Status::InvalidArgument(Substitute("illegal replication factor $0 (replication "
-                                           "factor must be odd)", num_replicas));
-    return SetError(MasterErrorPB::EVEN_REPLICATION_FACTOR, s);
+    return SetupError(Status::InvalidArgument(
+        Substitute("illegal replication factor $0 (replication factor must be odd)", num_replicas)),
+      resp, MasterErrorPB::EVEN_REPLICATION_FACTOR);
   }
 
   if (num_replicas > FLAGS_max_num_replicas) {
-    s = Status::InvalidArgument(Substitute("illegal replication factor $0 (max replication "
-                                           "factor is $1)",
-                                           num_replicas,
-                                           FLAGS_max_num_replicas));
-    return SetError(MasterErrorPB::REPLICATION_FACTOR_TOO_HIGH, s);
-
+    return SetupError(Status::InvalidArgument(
+          Substitute("illegal replication factor $0 (max replication factor is $1)",
+            num_replicas, FLAGS_max_num_replicas)),
+        resp, MasterErrorPB::REPLICATION_FACTOR_TOO_HIGH);
   }
   if (num_replicas <= 0) {
-    s = Status::InvalidArgument(Substitute("illegal replication factor $0 (replication factor "
-                                           "must be positive)",
-                                           num_replicas,
-                                           FLAGS_max_num_replicas));
-    return SetError(MasterErrorPB::ILLEGAL_REPLICATION_FACTOR, s);
+    return SetupError(Status::InvalidArgument(
+          Substitute("illegal replication factor $0 (replication factor must be positive)",
+            num_replicas, FLAGS_max_num_replicas)),
+        resp, MasterErrorPB::ILLEGAL_REPLICATION_FACTOR);
   }
 
   // Verify that the total number of tablets is reasonable, relative to the number
@@ -1436,21 +1425,20 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
   int num_live_tservers = ts_descs.size();
   int max_tablets = FLAGS_max_create_tablets_per_ts * num_live_tservers;
   if (num_replicas > 1 && max_tablets > 0 && partitions.size() > max_tablets) {
-    s = Status::InvalidArgument(Substitute("The requested number of tablets is over the "
-                                           "maximum permitted at creation time ($0). Additional "
-                                           "tablets may be added by adding range partitions to the "
-                                           "table post-creation.", max_tablets));
-    return SetError(MasterErrorPB::TOO_MANY_TABLETS, s);
+    return SetupError(Status::InvalidArgument(Substitute(
+            "the requested number of tablets is over the maximum permitted at creation time ($0), "
+            "additional tablets may be added by adding range partitions to the table post-creation",
+            max_tablets)),
+        resp, MasterErrorPB::TOO_MANY_TABLETS);
   }
 
   // Verify that the number of replicas isn't larger than the number of live tablet
   // servers.
-  if (FLAGS_catalog_manager_check_ts_count_for_create_table &&
-      num_replicas > num_live_tservers) {
-    s = Status::InvalidArgument(Substitute(
-        "Not enough live tablet servers to create a table with the requested replication "
-        "factor $0. $1 tablet servers are alive.", req.num_replicas(), num_live_tservers));
-    return SetError(MasterErrorPB::REPLICATION_FACTOR_TOO_HIGH, s);
+  if (FLAGS_catalog_manager_check_ts_count_for_create_table && num_replicas > num_live_tservers) {
+    return SetupError(Status::InvalidArgument(Substitute(
+            "not enough live tablet servers to create a table with the requested replication "
+            "factor $0; $1 tablet servers are alive", req.num_replicas(), num_live_tservers)),
+        resp, MasterErrorPB::REPLICATION_FACTOR_TOO_HIGH);
   }
 
   // Warn if the number of live tablet servers is not enough to re-replicate
@@ -1480,9 +1468,9 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
     // b. Verify that the table does not exist.
     table = FindPtrOrNull(table_names_map_, table_name);
     if (table != nullptr) {
-      s = Status::AlreadyPresent(Substitute("Table $0 already exists with id $1",
-                                 table_name, table->id()));
-      return SetError(MasterErrorPB::TABLE_ALREADY_PRESENT, s);
+      return SetupError(Status::AlreadyPresent(Substitute(
+              "table $0 already exists with id $1", table_name, table->id())),
+          resp, MasterErrorPB::TABLE_ALREADY_PRESENT);
     }
 
     // c. Reserve the table name if possible.
@@ -1492,9 +1480,9 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
       // 'AlreadyPresent', because a table name reservation can be rolled back
       // in the case of an error. Instead, we force the client to retry at a
       // later time.
-      s = Status::ServiceUnavailable(Substitute(
-          "New table name $0 is already reserved", table_name));
-      return SetError(MasterErrorPB::TABLE_ALREADY_PRESENT, s);
+      return SetupError(Status::ServiceUnavailable(Substitute(
+              "new table name $0 is already reserved", table_name)),
+          resp, MasterErrorPB::TABLE_ALREADY_PRESENT);
     }
   }
 
@@ -1536,13 +1524,12 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
   // It is critical that this step happen before writing the table to the sys catalog,
   // since this step validates that the table name is available in the HMS catalog.
   if (hms_catalog_) {
-    s = hms_catalog_->CreateTable(table->id(), req.name(), schema);
+    Status s = hms_catalog_->CreateTable(table->id(), req.name(), schema);
     if (!s.ok()) {
       s = s.CloneAndPrepend(Substitute("an error occurred while creating table $0 in the HMS",
                                        req.name()));
       LOG(WARNING) << s.ToString();
-      SetupError(resp->mutable_error(), MasterErrorPB::HIVE_METASTORE_ERROR, s);
-      return s;
+      return SetupError(std::move(s), resp, MasterErrorPB::HIVE_METASTORE_ERROR);
     }
   }
   // Delete the new HMS entry if we exit early.
@@ -1559,7 +1546,7 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
   SysCatalogTable::Actions actions;
   actions.table_to_add = table;
   actions.tablets_to_add = tablets;
-  s = sys_catalog_->Write(actions);
+  Status s = sys_catalog_->Write(actions);
   if (!s.ok()) {
     s = s.CloneAndPrepend("an error occurred while writing to the sys-catalog");
     LOG(WARNING) << s.ToString();
@@ -1619,9 +1606,9 @@ Status CatalogManager::IsCreateTableDone(const IsCreateTableDoneRequestPB* req,
   TableMetadataLock l;
   RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::READ, &table, &l));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(
+        Status::NotFound("the table does not exist", SecureShortDebugString(req->table())),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   RETURN_NOT_OK(CheckIfTableDeletedOrNotRunning(&l, resp));
 
@@ -1719,14 +1706,13 @@ Status CatalogManager::DeleteTable(const DeleteTableRequestPB* req,
   TableMetadataLock l;
   RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::WRITE, &table, &l));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(
+        Status::NotFound("the table does not exist", SecureShortDebugString(req->table())),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   if (l.data().is_deleted()) {
-    Status s = Status::NotFound("The table was deleted", l.data().pb.state_msg());
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(Status::NotFound("the table was deleted", l.data().pb.state_msg()),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
 
   // 2. Drop the HMS table entry.
@@ -1740,8 +1726,7 @@ Status CatalogManager::DeleteTable(const DeleteTableRequestPB* req,
       s.CloneAndPrepend(Substitute("an error occurred while dropping table $0 in the HMS",
                                    l.data().name()));
       LOG(WARNING) << s.ToString();
-      SetupError(resp->mutable_error(), MasterErrorPB::HIVE_METASTORE_ERROR, s);
-      return s;
+      return SetupError(std::move(s), resp, MasterErrorPB::HIVE_METASTORE_ERROR);
     }
   }
   // Re-create the HMS entry if we exit early.
@@ -2114,14 +2099,14 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
   TableMetadataLock l;
   RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::WRITE, &table, &l));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(
+        Status::NotFound("the table does not exist", SecureShortDebugString(req->table())),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   if (l.data().is_deleted()) {
-    Status s = Status::NotFound("The table was deleted", l.data().pb.state_msg());
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(
+        Status::NotFound("the table was deleted", l.data().pb.state_msg()),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
 
   string table_name = l.data().name();
@@ -2135,30 +2120,24 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
   // is essentialy a no-op. It's still important to execute because
   // ApplyAlterSchemaSteps populates 'new_schema', which is used below.
   TRACE("Apply alter schema");
-  Status s = ApplyAlterSchemaSteps(l.data().pb, alter_schema_steps, &new_schema, &next_col_id);
-  if (!s.ok()) {
-    SetupError(resp->mutable_error(), MasterErrorPB::INVALID_SCHEMA, s);
-    return s;
-  }
+  RETURN_NOT_OK(SetupError(
+        ApplyAlterSchemaSteps(l.data().pb, alter_schema_steps, &new_schema, &next_col_id),
+        resp, MasterErrorPB::INVALID_SCHEMA));
+
   DCHECK_NE(next_col_id, 0);
   DCHECK_EQ(new_schema.find_column_by_id(next_col_id),
             static_cast<int>(Schema::kColumnNotFound));
 
   // Just validate the schema, not the name (validated below).
-  s = ValidateClientSchema(boost::none, new_schema);
-  if (!s.ok()) {
-    SetupError(resp->mutable_error(), MasterErrorPB::INVALID_SCHEMA, s);
-    return s;
-  }
+  RETURN_NOT_OK(SetupError(
+        ValidateClientSchema(boost::none, new_schema),
+        resp, MasterErrorPB::INVALID_SCHEMA));
 
   // 4. Validate and try to acquire the new table name.
   if (req->has_new_table_name()) {
-    Status s = ValidateIdentifier(req->new_table_name());
-    if (!s.ok()) {
-      SetupError(resp->mutable_error(), MasterErrorPB::INVALID_SCHEMA,
-                 s.CloneAndPrepend("invalid table name"));
-      return s;
-    }
+    RETURN_NOT_OK(SetupError(
+          ValidateIdentifier(req->new_table_name()).CloneAndPrepend("invalid table name"),
+          resp, MasterErrorPB::INVALID_SCHEMA));
 
     std::lock_guard<LockType> catalog_lock(lock_);
     TRACE("Acquired catalog manager lock");
@@ -2166,10 +2145,10 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
     // Verify that the table does not exist.
     scoped_refptr<TableInfo> other_table = FindPtrOrNull(table_names_map_, req->new_table_name());
     if (other_table != nullptr) {
-      Status s = Status::AlreadyPresent(Substitute("Table $0 already exists with id $1",
-                                                   req->new_table_name(), table->id()));
-      SetupError(resp->mutable_error(), MasterErrorPB::TABLE_ALREADY_PRESENT, s);
-      return s;
+      return SetupError(
+          Status::AlreadyPresent(Substitute("table $0 already exists with id $1",
+              req->new_table_name(), table->id())),
+          resp, MasterErrorPB::TABLE_ALREADY_PRESENT);
     }
 
     // Reserve the new table name if possible.
@@ -2179,10 +2158,9 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
       // 'AlreadyPresent', because a table name reservation can be rolled back
       // in the case of an error. Instead, we force the client to retry at a
       // later time.
-      Status s = Status::ServiceUnavailable(Substitute(
-          "Table name $0 is already reserved", req->new_table_name()));
-      SetupError(resp->mutable_error(), MasterErrorPB::TABLE_ALREADY_PRESENT, s);
-      return s;
+      return SetupError(Status::ServiceUnavailable(Substitute(
+              "table name $0 is already reserved", req->new_table_name())),
+          resp, MasterErrorPB::TABLE_ALREADY_PRESENT);
     }
 
     l.mutable_data()->pb.set_name(req->new_table_name());
@@ -2202,13 +2180,12 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
   if (!alter_partitioning_steps.empty()) {
     TRACE("Apply alter partitioning");
     Schema client_schema;
-    RETURN_NOT_OK(SchemaFromPB(req->schema(), &client_schema));
-    Status s = ApplyAlterPartitioningSteps(l, table, client_schema, alter_partitioning_steps,
-                                           &tablets_to_add, &tablets_to_drop);
-    if (!s.ok()) {
-      SetupError(resp->mutable_error(), MasterErrorPB::UNKNOWN_ERROR, s);
-      return s;
-    }
+    RETURN_NOT_OK(SetupError(SchemaFromPB(req->schema(), &client_schema),
+          resp, MasterErrorPB::UNKNOWN_ERROR));
+    RETURN_NOT_OK(SetupError(
+          ApplyAlterPartitioningSteps(l, table, client_schema, alter_partitioning_steps,
+            &tablets_to_add, &tablets_to_drop),
+          resp, MasterErrorPB::UNKNOWN_ERROR));
   }
 
   // Set to true if columns are altered, added or dropped.
@@ -2260,8 +2237,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
       s = s.CloneAndPrepend(Substitute("an error occurred while altering table $0 in the HMS",
                                        table_name));
       LOG(WARNING) << s.ToString();
-      SetupError(resp->mutable_error(), MasterErrorPB::HIVE_METASTORE_ERROR, s);
-      return s;
+      return SetupError(std::move(s), resp, MasterErrorPB::HIVE_METASTORE_ERROR);
     }
   }
   // Roll-back the HMS alteration if we exit early.
@@ -2304,7 +2280,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
   }
   actions.tablets_to_update = tablets_to_drop;
 
-  s = sys_catalog_->Write(actions);
+  Status s = sys_catalog_->Write(actions);
   if (!s.ok()) {
     s = s.CloneAndPrepend("an error occurred while updating the sys-catalog");
     LOG(WARNING) << s.ToString();
@@ -2395,9 +2371,9 @@ Status CatalogManager::IsAlterTableDone(const IsAlterTableDoneRequestPB* req,
   TableMetadataLock l;
   RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::READ, &table, &l));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(Status::NotFound(
+          "the table does not exist", SecureShortDebugString(req->table())),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   RETURN_NOT_OK(CheckIfTableDeletedOrNotRunning(&l, resp));
 
@@ -2420,9 +2396,9 @@ Status CatalogManager::GetTableSchema(const GetTableSchemaRequestPB* req,
   TableMetadataLock l;
   RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::READ, &table, &l));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(Status::NotFound(
+          "the table does not exist", SecureShortDebugString(req->table())),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   RETURN_NOT_OK(CheckIfTableDeletedOrNotRunning(&l, resp));
 
@@ -4522,9 +4498,9 @@ Status CatalogManager::GetTableLocations(const GetTableLocationsRequestPB* req,
   TableMetadataLock l;
   RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::READ, &table, &l));
   if (table == nullptr) {
-    Status s = Status::NotFound("The table does not exist", SecureShortDebugString(req->table()));
-    SetupError(resp->mutable_error(), MasterErrorPB::TABLE_NOT_FOUND, s);
-    return s;
+    return SetupError(Status::NotFound(
+          "the table does not exist", SecureShortDebugString(req->table())),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   RETURN_NOT_OK(CheckIfTableDeletedOrNotRunning(&l, resp));