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 19:17:25 UTC

kudu git commit: catalog-manager: return error from FindAndLockTable if table not found

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


catalog-manager: return error from FindAndLockTable if table not found

This cleans up some repeated boilerplate in the CatalogManager.

Change-Id: I4d8176d6886474a84b4c75c80f50763eabe70a6d
Reviewed-on: http://gerrit.cloudera.org:8080/10377
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/a65e58ec
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/a65e58ec
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/a65e58ec

Branch: refs/heads/master
Commit: a65e58ec0612e6c20757fa51fe4e37056d5a4815
Parents: 1a707b9
Author: Dan Burkert <da...@apache.org>
Authored: Thu May 10 12:40:27 2018 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Mon May 14 19:17:04 2018 +0000

----------------------------------------------------------------------
 src/kudu/master/catalog_manager.cc | 83 ++++++++++++---------------------
 src/kudu/master/catalog_manager.h  |  7 ++-
 2 files changed, 36 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a65e58ec/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 1de0be1..174c7fc 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -620,12 +620,15 @@ void CheckIfNoLongerLeaderAndSetupError(const Status& s, RespClass* resp) {
 template<class RespClass>
 Status CheckIfTableDeletedOrNotRunning(TableMetadataLock* lock, RespClass* resp) {
   if (lock->data().is_deleted()) {
-    return SetupError(Status::NotFound("the table was deleted", lock->data().pb.state_msg()),
-                      resp, MasterErrorPB::TABLE_NOT_FOUND);
+    return SetupError(Status::NotFound(
+          Substitute("table $0 was deleted", lock->data().name()),
+          lock->data().pb.state_msg()),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   if (!lock->data().is_running()) {
-    return SetupError(Status::ServiceUnavailable("the table is not running"),
-                      resp, MasterErrorPB::TABLE_NOT_FOUND);
+    return SetupError(Status::ServiceUnavailable(
+          Substitute("table $0 is not running", lock->data().name())),
+        resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
   return Status::OK();
 }
@@ -1601,15 +1604,9 @@ Status CatalogManager::IsCreateTableDone(const IsCreateTableDoneRequestPB* req,
   RETURN_NOT_OK(CheckOnline());
 
   // 1. Lookup the table and verify if it exists
-  TRACE("Looking up and locking table");
   scoped_refptr<TableInfo> table;
   TableMetadataLock l;
-  RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::READ, &table, &l));
-  if (table == nullptr) {
-    return SetupError(
-        Status::NotFound("the table does not exist", SecureShortDebugString(req->table())),
-        resp, MasterErrorPB::TABLE_NOT_FOUND);
-  }
+  RETURN_NOT_OK(FindAndLockTable(*req, resp, LockMode::READ, &table, &l));
   RETURN_NOT_OK(CheckIfTableDeletedOrNotRunning(&l, resp));
 
   // 2. Verify if the create is in-progress
@@ -1649,10 +1646,21 @@ scoped_refptr<TabletInfo> CatalogManager::CreateTabletInfo(const scoped_refptr<T
   return tablet;
 }
 
-Status CatalogManager::FindAndLockTable(const TableIdentifierPB& table_identifier,
+template<typename ReqClass, typename RespClass>
+Status CatalogManager::FindAndLockTable(const ReqClass& request,
+                                        RespClass* response,
                                         LockMode lock_mode,
                                         scoped_refptr<TableInfo>* table_info,
                                         TableMetadataLock* table_lock) {
+  TRACE("Looking up and locking table");
+  const TableIdentifierPB& table_identifier = request.table();
+
+  auto tnf_error = [&] {
+    return SetupError(Status::NotFound(
+          "the table does not exist", SecureShortDebugString(table_identifier)),
+        response, MasterErrorPB::TABLE_NOT_FOUND);
+  };
+
   scoped_refptr<TableInfo> table;
   {
     shared_lock<LockType> l(lock_);
@@ -1663,18 +1671,19 @@ Status CatalogManager::FindAndLockTable(const TableIdentifierPB& table_identifie
       // both match the same table.
       if (table_identifier.has_table_name() &&
           table.get() != FindPtrOrNull(table_names_map_, table_identifier.table_name()).get()) {
-        return Status::OK();
+        return tnf_error();
       }
     } else if (table_identifier.has_table_name()) {
       table = FindPtrOrNull(table_names_map_, table_identifier.table_name());
     } else {
-      return Status::InvalidArgument("Missing Table ID or Table Name");
+      return SetupError(Status::InvalidArgument("missing table ID or table name"),
+                        response, MasterErrorPB::UNKNOWN_ERROR);
     }
   }
 
   // If the table doesn't exist, don't attempt to lock it.
   if (!table) {
-    return Status::OK();
+    return tnf_error();
   }
 
   // Acquire the table lock.
@@ -1683,7 +1692,7 @@ Status CatalogManager::FindAndLockTable(const TableIdentifierPB& table_identifie
   if (table_identifier.has_table_name() && table_identifier.table_name() != lock.data().name()) {
     // We've encountered the table while it's in the process of being renamed;
     // pretend it doesn't yet exist.
-    return Status::OK();
+    return tnf_error();
   }
 
   *table_info = std::move(table);
@@ -1701,15 +1710,9 @@ Status CatalogManager::DeleteTable(const DeleteTableRequestPB* req,
                           RequestorString(rpc), SecureShortDebugString(*req));
 
   // 1. Look up the table, lock it, and mark it as removed.
-  TRACE("Looking up and locking table");
   scoped_refptr<TableInfo> table;
   TableMetadataLock l;
-  RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::WRITE, &table, &l));
-  if (table == nullptr) {
-    return SetupError(
-        Status::NotFound("the table does not exist", SecureShortDebugString(req->table())),
-        resp, MasterErrorPB::TABLE_NOT_FOUND);
-  }
+  RETURN_NOT_OK(FindAndLockTable(*req, resp, LockMode::WRITE, &table, &l));
   if (l.data().is_deleted()) {
     return SetupError(Status::NotFound("the table was deleted", l.data().pb.state_msg()),
         resp, MasterErrorPB::TABLE_NOT_FOUND);
@@ -2094,15 +2097,9 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
   }
 
   // 2. Lookup the table, verify if it exists, and lock it for modification.
-  TRACE("Looking up and locking table");
   scoped_refptr<TableInfo> table;
   TableMetadataLock l;
-  RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::WRITE, &table, &l));
-  if (table == nullptr) {
-    return SetupError(
-        Status::NotFound("the table does not exist", SecureShortDebugString(req->table())),
-        resp, MasterErrorPB::TABLE_NOT_FOUND);
-  }
+  RETURN_NOT_OK(FindAndLockTable(*req, resp, LockMode::WRITE, &table, &l));
   if (l.data().is_deleted()) {
     return SetupError(
         Status::NotFound("the table was deleted", l.data().pb.state_msg()),
@@ -2366,15 +2363,9 @@ Status CatalogManager::IsAlterTableDone(const IsAlterTableDoneRequestPB* req,
   RETURN_NOT_OK(CheckOnline());
 
   // 1. Lookup the table and verify if it exists
-  TRACE("Looking up and locking table");
   scoped_refptr<TableInfo> table;
   TableMetadataLock l;
-  RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::READ, &table, &l));
-  if (table == nullptr) {
-    return SetupError(Status::NotFound(
-          "the table does not exist", SecureShortDebugString(req->table())),
-        resp, MasterErrorPB::TABLE_NOT_FOUND);
-  }
+  RETURN_NOT_OK(FindAndLockTable(*req, resp, LockMode::READ, &table, &l));
   RETURN_NOT_OK(CheckIfTableDeletedOrNotRunning(&l, resp));
 
   // 2. Verify if the alter is in-progress
@@ -2391,15 +2382,9 @@ Status CatalogManager::GetTableSchema(const GetTableSchemaRequestPB* req,
   RETURN_NOT_OK(CheckOnline());
 
   // Lookup the table and verify if it exists
-  TRACE("Looking up and locking table");
   scoped_refptr<TableInfo> table;
   TableMetadataLock l;
-  RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::READ, &table, &l));
-  if (table == nullptr) {
-    return SetupError(Status::NotFound(
-          "the table does not exist", SecureShortDebugString(req->table())),
-        resp, MasterErrorPB::TABLE_NOT_FOUND);
-  }
+  RETURN_NOT_OK(FindAndLockTable(*req, resp, LockMode::READ, &table, &l));
   RETURN_NOT_OK(CheckIfTableDeletedOrNotRunning(&l, resp));
 
   if (l.data().pb.has_fully_applied_schema()) {
@@ -4489,19 +4474,13 @@ Status CatalogManager::GetTableLocations(const GetTableLocationsRequestPB* req,
     return Status::InvalidArgument("max_returned_locations must be greater than 0");
   }
 
-  // Lookup the table and verify if it exists
-  TRACE("Looking up and locking table");
   leader_lock_.AssertAcquiredForReading();
   RETURN_NOT_OK(CheckOnline());
 
+  // Lookup the table and verify if it exists
   scoped_refptr<TableInfo> table;
   TableMetadataLock l;
-  RETURN_NOT_OK(FindAndLockTable(req->table(), LockMode::READ, &table, &l));
-  if (table == nullptr) {
-    return SetupError(Status::NotFound(
-          "the table does not exist", SecureShortDebugString(req->table())),
-        resp, MasterErrorPB::TABLE_NOT_FOUND);
-  }
+  RETURN_NOT_OK(FindAndLockTable(*req, resp, LockMode::READ, &table, &l));
   RETURN_NOT_OK(CheckIfTableDeletedOrNotRunning(&l, resp));
 
   vector<scoped_refptr<TabletInfo>> tablets_in_range;

http://git-wip-us.apache.org/repos/asf/kudu/blob/a65e58ec/src/kudu/master/catalog_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index 60bff5f..957c3ea 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -768,8 +768,11 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
                                  TabletLocationsPB* locs_pb);
 
   // Looks up the table and locks it with the provided lock mode. If the table
-  // does not exist, the lock is not acquired and the table is not modified.
-  Status FindAndLockTable(const TableIdentifierPB& table_identifier,
+  // does not exist an error status is returned, and the appropriate error code
+  // is set in the response.
+  template<typename ReqClass, typename RespClass>
+  Status FindAndLockTable(const ReqClass& request,
+                          RespClass* response,
                           LockMode lock_mode,
                           scoped_refptr<TableInfo>* table_info,
                           TableMetadataLock* table_lock) WARN_UNUSED_RESULT;