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

kudu git commit: KUDU-2191 (11/n): remove fallback logic from HmsCatalog

Repository: kudu
Updated Branches:
  refs/heads/master 5d113ec3c -> 61cd9510f


KUDU-2191 (11/n): remove fallback logic from HmsCatalog

HmsCatalog was originally written with fallback logic that would
liberally correct HMS entries when possible. In particular, altering a
non-existent table entry would result in the HMS Catalog creating a new
table entry. These liberal corrections allowed the HMS catalog to
'upgrade' legacy Kudu tables to have HMS entries automatically upon any
kind of alteration, but in certain race conditions[1] it could lead to a
single Kudu table having multiple HMS table entries with different
names. This will be problematic down the line for other integrations
like Sentry, which rely on HMS metadata being canonical.

This commit removes this logic, which simplifies HmsCatalog, and removes
the need for a bunch of the tests which covered this fallback logic. As
a result, the metadata upgrade tool is going to need a special path
which allows it to alter Kudu tables without modifying the HMS. That
will be included in a follow up commit.

[1]: consider the following operation sequence:

      - in Hive: ALTER TABLE a RENAME TO b
      - in Kudu: ALTER TABLE a RENAME TO c

  When handling the Kudu alter table, the 'a' table exists in the Kudu
  catalog, but not in the HMS. As a result the HmsCatalog creates a new
  table entry with name 'c'. At this point the HMS contains two table
  entries ('b' and 'c'), both representing the same Kudu table. This
  will cause problems for Sentry, for dropping the table, etc.

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


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

Branch: refs/heads/master
Commit: 61cd9510ff07229280499ec0155fb6ca6b46d510
Parents: 5d113ec
Author: Dan Burkert <da...@apache.org>
Authored: Mon Apr 23 14:14:46 2018 -0700
Committer: Hao Hao <ha...@cloudera.com>
Committed: Mon May 14 22:35:01 2018 +0000

----------------------------------------------------------------------
 src/kudu/hms/hms_catalog-test.cc               |  77 +++----------
 src/kudu/hms/hms_catalog.cc                    | 120 ++------------------
 src/kudu/hms/hms_catalog.h                     |  26 ++---
 src/kudu/integration-tests/master_hms-itest.cc |  51 ++++-----
 4 files changed, 58 insertions(+), 216 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/61cd9510/src/kudu/hms/hms_catalog-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/hms/hms_catalog-test.cc b/src/kudu/hms/hms_catalog-test.cc
index cbfe528..5a25d00 100644
--- a/src/kudu/hms/hms_catalog-test.cc
+++ b/src/kudu/hms/hms_catalog-test.cc
@@ -256,14 +256,8 @@ TEST_P(HmsCatalogTestParameterized, TestTableLifecycle) {
   ASSERT_OK(hms_catalog_->CreateTable(kTableId, kTableName, schema));
   NO_FATALS(CheckTable(kHmsDatabase, kHmsTableName, kTableId, schema));
 
-  // Create the table again. This should succeed since the table ID matches. The
-  // HMS catalog will automatically short-circuit creating the table.
-  // TODO(dan): once we have HMS catalog stats, assert that the op short circuits.
-  ASSERT_OK(hms_catalog_->CreateTable(kTableId, kTableName, schema));
-  NO_FATALS(CheckTable(kHmsDatabase, kHmsTableName, kTableId, schema));
-
-  // Create the table again, but with a different table ID.
-  Status s = hms_catalog_->CreateTable("new-table-id", kTableName, schema);
+  // Create the table again, and check that the expected failure occurs.
+  Status s = hms_catalog_->CreateTable(kTableId, kTableName, schema);
   ASSERT_TRUE(s.IsAlreadyPresent()) << s.ToString();
   NO_FATALS(CheckTable(kHmsDatabase, kHmsTableName, kTableId, schema));
 
@@ -281,35 +275,6 @@ TEST_P(HmsCatalogTestParameterized, TestTableLifecycle) {
   NO_FATALS(CheckTableDoesNotExist(kHmsDatabase, kHmsAlteredTableName));
 }
 
-// Checks that 'legacy' Kudu tables can be altered and dropped by the
-// HmsCatalog. Altering a legacy table to be HMS compliant should result in a
-// valid HMS table entry being created. Dropping a legacy table should do
-// nothing, but return success.
-TEST_F(HmsCatalogTest, TestLegacyTables) {
-  const string kTableId = "table-id";
-  const string kHmsDatabase = "default";
-
-  Schema schema = AllTypesSchema();
-  hive::Table table;
-
-  // Alter a table containing a non Hive-compatible character, and ensure an
-  // entry is created with the new (valid) name.
-  NO_FATALS(CheckTableDoesNotExist(kHmsDatabase, "a"));
-  ASSERT_OK(hms_catalog_->AlterTable(kTableId, "default.☃", "default.a", schema));
-  NO_FATALS(CheckTable(kHmsDatabase, "a", kTableId, schema));
-
-  // Alter a table without a database and ensure an entry is created with the new (valid) name.
-  NO_FATALS(CheckTableDoesNotExist(kHmsDatabase, "b"));
-  ASSERT_OK(hms_catalog_->AlterTable(kTableId, "no_database", "default.b", schema));
-  NO_FATALS(CheckTable(kHmsDatabase, "b", kTableId, schema));
-
-  // Drop a table containing a Hive incompatible character, and ensure it doesn't fail.
-  ASSERT_OK(hms_catalog_->DropTable(kTableId, "foo.☃"));
-
-  // Drop a table without a database, and ensure it doesn't fail.
-  ASSERT_OK(hms_catalog_->DropTable(kTableId, "no_database"));
-}
-
 // Checks that Kudu tables will not replace or modify existing HMS entries that
 // belong to external tables from other systems.
 TEST_F(HmsCatalogTest, TestExternalTable) {
@@ -347,39 +312,23 @@ TEST_F(HmsCatalogTest, TestExternalTable) {
   NO_FATALS(CheckExternalTable());
   NO_FATALS(CheckTable("default", "a", kTableId, schema));
 
-  // Try and rename a Kudu table from the external table name to a new name.
-  // This depends on the Kudu table not actually existing in the HMS catalog.
-  ASSERT_OK(hms_catalog_->AlterTable(kTableId, "default.ext", "default.b", schema));
-  NO_FATALS(CheckExternalTable());
-  // The 'renamed' table is really just created with the new name.
-  NO_FATALS(CheckTable("default", "b", kTableId, schema));
-
-  // Try the previous alter operation again. This should succeed, since the
-  // destination table ID matches, so the HMS catalog knows its the same table.
-  // TODO(dan): once we have HMS catalog stats, assert that the op short circuits.
-  ASSERT_OK(hms_catalog_->AlterTable(kTableId, "default.ext", "default.b", schema));
-  NO_FATALS(CheckExternalTable());
-  // The 'renamed' table is really just created with the new name.
-  NO_FATALS(CheckTable("default", "b", kTableId, schema));
-
-  // Try the previous alter operation again, but with a different table ID.
-  s = hms_catalog_->AlterTable("new-table-id", "default.ext", "default.b", schema);
-  ASSERT_TRUE(s.IsAlreadyPresent()) << s.ToString();
-
-  // Try and alter a Kudu table with the same name as the external table.
-  // This depends on the Kudu table not actually existing in the HMS catalog.
-  s = hms_catalog_->AlterTable(kTableId, "default.ext", "default.ext", schema);
-  EXPECT_TRUE(s.IsAlreadyPresent()) << s.ToString();
+  // Try and rename the external table. This shouldn't succeed because the Table
+  // ID doesn't match.
+  s = hms_catalog_->AlterTable(kTableId, "default.ext", "default.b", schema);
+  EXPECT_TRUE(s.IsNotFound()) << s.ToString();
   NO_FATALS(CheckExternalTable());
+  NO_FATALS(CheckTable("default", "a", kTableId, schema));
+  NO_FATALS(CheckTableDoesNotExist("default", "b"));
 
-  // Try and drop the external table as if it were a Kudu table.  This should
-  // return an OK status, but not actually modify the external table.
-  ASSERT_OK(hms_catalog_->DropTable(kTableId, "default.ext"));
+  // Try and drop the external table as if it were a Kudu table.
+  s = hms_catalog_->DropTable(kTableId, "default.ext");
+  EXPECT_TRUE(s.IsRemoteError()) << s.ToString();
   NO_FATALS(CheckExternalTable());
 
   // Drop a Kudu table with no corresponding HMS entry.
   NO_FATALS(CheckTableDoesNotExist("default", "bogus_table_name"));
-  ASSERT_OK(hms_catalog_->DropTable(kTableId, "default.bogus_table_name"));
+  s = hms_catalog_->DropTable(kTableId, "default.bogus_table_name");
+  ASSERT_TRUE(s.IsNotFound()) << s.ToString();
   NO_FATALS(CheckTableDoesNotExist("default", "bogus_table_name"));
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/61cd9510/src/kudu/hms/hms_catalog.cc
----------------------------------------------------------------------
diff --git a/src/kudu/hms/hms_catalog.cc b/src/kudu/hms/hms_catalog.cc
index 9b68ecb..b32348c 100644
--- a/src/kudu/hms/hms_catalog.cc
+++ b/src/kudu/hms/hms_catalog.cc
@@ -135,44 +135,22 @@ Status HmsCatalog::CreateTable(const string& id,
                                const string& name,
                                const Schema& schema) {
   return Execute([&] (HmsClient* client) {
-      return CreateOrUpdateTable(client, id, name, schema, master_addresses_);
+      hive::Table table;
+      RETURN_NOT_OK(PopulateTable(id, name, schema, master_addresses_, &table));
+      return client->CreateTable(table, EnvironmentContext());
   });
 }
 
 Status HmsCatalog::DropTable(const string& id, const string& name) {
   string hms_database;
   string hms_table;
-
-  if (!ParseTableName(name, &hms_database, &hms_table).ok()) {
-    // This is a legacy table; it can't be in the HMS, so return success.
-    VLOG(1) << "Ignoring missing HMS table entry while dropping table "
-            << name;
-    return Status::OK();
-  }
+  RETURN_NOT_OK(ParseTableName(name, &hms_database, &hms_table));
 
   hive::EnvironmentContext env_ctx = EnvironmentContext();
   env_ctx.properties.insert(make_pair(HmsClient::kKuduTableIdKey, id));
 
   return Execute([&] (HmsClient* client) {
-    Status s = client->DropTable(hms_database, hms_table, env_ctx);
-    if (s.IsNotFound()) {
-      VLOG(1) << "Ignoring missing HMS table entry while dropping table "
-              << name << "(" << id << ")";
-      return Status::OK();
-    }
-    if (s.IsRemoteError() &&
-        s.message().ToString().find("Kudu table ID does not match the non-Kudu HMS entry")
-        != string::npos) {
-      // TODO(dan): the string match above is extraordinarily hacky, is there a
-      // better way to match this failure scenario?  Unfortunately errors
-      // occuring in the Kudu Metastore Plugin only manifest as a
-      // 'MetaException', so they can't have a more specific error code.
-      VLOG(1) << "Ignoring HMS table entry which does not belong to the Kudu table being dropped "
-              << name << "(" << id << ")";
-      return Status::OK();
-    }
-
-    return s;
+    return client->DropTable(hms_database, hms_table, env_ctx);
   });
 }
 
@@ -190,54 +168,26 @@ Status HmsCatalog::AlterTable(const string& id,
       // and the full set of columns. This ensures entries are fully 'repaired'
       // during an alter operation.
       //
-      // This can go wrong in a myriad of ways, including:
+      // This can go wrong in a number of ways, including:
       //
       // - The original table name isn't a valid Hive database/table pair
       // - The new table name isn't a valid Hive database/table pair
-      // - The original table entry does not exist in the HMS
-      // - The original table entry doesn't match the Kudu table being altered
-      // - The alteration has already been applied to the HMS
-      //
-      // Where possible, we try to repair the HMS state to whatever it should be
-      // in these situations, unless we detect that such a repair would alter an
-      // unrelated table entry.
-
-      // If this is not a rename, then attempt to update the existing entry, or
-      // create it if it's missing.
-      if (name == new_name) {
-        return CreateOrUpdateTable(client, id, name, schema, master_addresses_);
-      }
+      // - The original table does not exist in the HMS
+      // - The original table doesn't match the Kudu table being altered
 
       string hms_database;
       string hms_table;
-      Status s = ParseTableName(name, &hms_database, &hms_table);
-      if (!s.ok()) {
-        // Parsing the original table name has failed, so it can not be present in
-        // the HMS. Instead of altering the table, create it in the HMS as a new table.
-        VLOG(1) << "Failed to parse the name of the table being renamed as an "
-                   "HMS database/table pair, will attempt to create an HMS table entry "
-                   "with the new name: "
-                << s.ToString();
-        return CreateOrUpdateTable(client, id, new_name, schema, master_addresses_);
-      }
+      RETURN_NOT_OK(ParseTableName(name, &hms_database, &hms_table));
 
       hive::Table table;
-      s = client->GetTable(hms_database, hms_table, &table);
-
-      if (s.IsNotFound()) {
-        // The table doesn't already exist in the HMS, so create it.
-        VLOG(1) << "The table being renamed does not have an existing HMS entry, "
-                   "will attempt to create an HMS table entry with the new name.";
-        return CreateOrUpdateTable(client, id, new_name, schema, master_addresses_);
-      }
+      RETURN_NOT_OK(client->GetTable(hms_database, hms_table, &table));
 
       // Check that the HMS entry belongs to the table being altered.
       if (table.parameters[HmsClient::kStorageHandlerKey] != HmsClient::kKuduStorageHandler ||
           table.parameters[HmsClient::kKuduTableIdKey] != id) {
         // The original table isn't a Kudu table, or isn't the same Kudu table.
-        VLOG(1) << "The HMS entry for the table being renamed belongs to another table, "
-                    "will attempt to create an HMS entry with the new name.";
-        return CreateOrUpdateTable(client, id, new_name, schema, master_addresses_);
+        return Status::NotFound("the HMS entry for the table being "
+                                "altered belongs to another table");
       }
 
       // Overwrite fields in the table that have changed, including the new name.
@@ -448,52 +398,6 @@ Status HmsCatalog::PopulateTable(const string& id,
   return Status::OK();
 }
 
-Status HmsCatalog::CreateOrUpdateTable(hms::HmsClient* client,
-                                       const string& id,
-                                       const string& name,
-                                       const Schema& schema,
-                                       const string& master_addresses) {
-  string hms_database;
-  string hms_table;
-  RETURN_NOT_OK(ParseTableName(name, &hms_database, &hms_table));
-
-  hive::Table existing_table;
-  Status s = client->GetTable(hms_database, hms_table, &existing_table);
-
-  if (s.IsNotFound()) {
-      VLOG(1) << "Table " << name << " (" << id << ") does not have an existing HMS entry, "
-                  "will attempt to create a new HMS table entry.";
-      hive::Table table;
-      RETURN_NOT_OK(PopulateTable(id, name, schema, master_addresses, &table));
-
-      return client->CreateTable(table, EnvironmentContext());
-  }
-
-  // All other errors are fatal.
-  RETURN_NOT_OK(s);
-
-  // The table already exists, so we update it.
-
-  // Check if the existing HMS entry belongs to the table being altered.
-  if (existing_table.parameters[HmsClient::kStorageHandlerKey] != HmsClient::kKuduStorageHandler ||
-      existing_table.parameters[HmsClient::kKuduTableIdKey] != id) {
-
-    // Otherwise we fail, since we must not update an entry belonging to a different table.
-    return Status::AlreadyPresent(Substitute("table $0 already exists in the HMS", name));
-  }
-
-  // Create a copy of the table object, and set the Kudu fields in it. If
-  // the original table object and the new table object match exactly then
-  // we don't need to alter the table in the HMS.
-  hive::Table table(existing_table);
-  RETURN_NOT_OK(PopulateTable(id, name, schema, master_addresses, &table));
-  if (existing_table == table) {
-    VLOG(1) << "Short-circuiting alter or update table for " << name << " (" << id << ")";
-    return Status::OK();
-  }
-  return client->AlterTable(hms_database, hms_table, table, EnvironmentContext());
-}
-
 Status HmsCatalog::ParseTableName(const string& table,
                                   string* hms_database,
                                   string* hms_table) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/61cd9510/src/kudu/hms/hms_catalog.h
----------------------------------------------------------------------
diff --git a/src/kudu/hms/hms_catalog.h b/src/kudu/hms/hms_catalog.h
index a0374f1..330c29d 100644
--- a/src/kudu/hms/hms_catalog.h
+++ b/src/kudu/hms/hms_catalog.h
@@ -65,18 +65,19 @@ class HmsCatalog {
                      const std::string& name,
                      const Schema& schema) WARN_UNUSED_RESULT;
 
-  // Drops a table entry from the HMS, if it exists.
+  // Drops a table entry from the HMS.
   //
-  // This method will fail if the HMS is unreachable, or if the table entry in
-  // the HMS doesn't match the specified Kudu table ID.
+  // This method will fail if the HMS is unreachable, if the table does not
+  // exist in the HMS, or if the table entry in the HMS doesn't match the
+  // specified Kudu table ID.
   Status DropTable(const std::string& id,
                    const std::string& name) WARN_UNUSED_RESULT;
 
-  // Alters a table entry in the HMS, if it exists. If the table entry does not
-  // exist it will be created instead.
+  // Alters a table entry in the HMS.
   //
-  // This method will fail if the HMS is unreachable, or if the table entry in
-  // the HMS doesn't match the specified Kudu table ID.
+  // This method will fail if the HMS is unreachable, if the table doesn't exist
+  // in the HMS, or if the table entry in the HMS doesn't match the specified
+  // Kudu table ID.
   Status AlterTable(const std::string& id,
                     const std::string& name,
                     const std::string& new_name,
@@ -115,17 +116,6 @@ class HmsCatalog {
                               const std::string& master_addresses,
                               hive::Table* table) WARN_UNUSED_RESULT;
 
-  // Creates a table entry in the HMS, or updates that existing entry if the
-  // table already has an entry.
-  //
-  // Instead of only attempting to create or alter a table entry, this method should be
-  // used to ensure the HMS is kept synchronized in as many edge cases as possible.
-  static Status CreateOrUpdateTable(hms::HmsClient* client,
-                                    const std::string& id,
-                                    const std::string& name,
-                                    const Schema& schema,
-                                    const std::string& master_addresses) WARN_UNUSED_RESULT;
-
   // Parses a Kudu table name into a Hive database and table name.
   // Returns an error if the Kudu table name is not correctly formatted.
   static Status ParseTableName(const std::string& table,

http://git-wip-us.apache.org/repos/asf/kudu/blob/61cd9510/src/kudu/integration-tests/master_hms-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_hms-itest.cc b/src/kudu/integration-tests/master_hms-itest.cc
index d65f669..f30f89f 100644
--- a/src/kudu/integration-tests/master_hms-itest.cc
+++ b/src/kudu/integration-tests/master_hms-itest.cc
@@ -167,6 +167,12 @@ class MasterHmsTest : public ExternalMiniClusterITestBase {
     ASSERT_TRUE(s.IsNotFound()) << s.ToString();
   }
 
+  static hive::EnvironmentContext MasterEnvCtx() {
+    hive::EnvironmentContext env_ctx;
+    env_ctx.__set_properties({ std::make_pair(hms::HmsClient::kKuduMasterEventKey, "true") });
+    return env_ctx;
+  }
+
  protected:
 
   unique_ptr<HmsClient> hms_client_;
@@ -256,19 +262,9 @@ TEST_F(MasterHmsTest, TestRenameTable) {
   ASSERT_TRUE(s.IsIllegalState()) << s.ToString();
   ASSERT_STR_CONTAINS(s.ToString(), "☃ is not a valid object name");
 
-  // Drop the HMS table entry and rename the table. This tests that the
-  // HmsCatalog will create a new entry when necessary.
-  shared_ptr<KuduTable> table;
-  ASSERT_OK(client_->OpenTable("db.a", &table));
-  ASSERT_OK(hms_client_->DropTable("db", "a"));
-  table_alterer.reset(client_->NewTableAlterer("db.a"));
-  ASSERT_OK(table_alterer->RenameTo("db.c")->Alter());
-  NO_FATALS(CheckTable("db", "c"));
-  NO_FATALS(CheckTableDoesNotExist("db", "a"));
-
   // Shutdown the HMS and try to rename the table.
   ASSERT_OK(StopHms());
-  table_alterer.reset(client_->NewTableAlterer("db.c")->RenameTo("db.a"));
+  table_alterer.reset(client_->NewTableAlterer("db.a")->RenameTo("db.c"));
   s = table_alterer->Alter();
   ASSERT_TRUE(s.IsNetworkError()) << s.ToString();
 
@@ -278,25 +274,25 @@ TEST_F(MasterHmsTest, TestRenameTable) {
     // HmsCatalog throttles reconnections, so it's necessary to wait out the backoff.
     ASSERT_OK(table_alterer->Alter());
   });
-  NO_FATALS(CheckTable("db", "a"));
-  NO_FATALS(CheckTableDoesNotExist("db", "c"));
+  NO_FATALS(CheckTable("db", "c"));
+  NO_FATALS(CheckTableDoesNotExist("db", "a"));
 
   // Drop the HMS table entry, then create a non-Kudu table entry in it's place,
   // and attempt to rename the table.
-  ASSERT_OK(hms_client_->DropTable("db", "a"));
+  ASSERT_OK(hms_client_->DropTable("db", "c"));
   hive::Table external_table_2;
   external_table_2.dbName = "db";
-  external_table_2.tableName = "a";
+  external_table_2.tableName = "c";
   ASSERT_OK(hms_client_->CreateTable(external_table_2));
-  table_alterer.reset(client_->NewTableAlterer("db.a"));
-  ASSERT_OK(table_alterer->RenameTo("db.c")->Alter());
-  NO_FATALS(CheckTable("db", "c"));
+  table_alterer.reset(client_->NewTableAlterer("db.c"));
+  s = table_alterer->RenameTo("db.a")->Alter();
+  ASSERT_TRUE(s.IsNotFound()) << s.ToString();
 
   // Check that all three tables still exist.
   vector<string> tables;
   ASSERT_OK(hms_client_->GetAllTables("db", &tables));
   std::sort(tables.begin(), tables.end());
-  ASSERT_EQ(tables, vector<string>({ "a", "b", "c" })) << tables;
+  ASSERT_EQ(tables, vector<string>({ "b", "c" })) << tables;
 }
 
 TEST_F(MasterHmsTest, TestAlterTable) {
@@ -315,9 +311,7 @@ TEST_F(MasterHmsTest, TestAlterTable) {
   ASSERT_OK(hms_client_->GetTable(hms_database_name, hms_table_name, &hms_table));
   hms_table.sd.cols.clear();
   // The KuduMetastorePlugin requires column alteration events to come from a Kudu Master.
-  hive::EnvironmentContext env_ctx;
-  env_ctx.__set_properties({ std::make_pair(hms::HmsClient::kKuduMasterEventKey, "true") });
-  ASSERT_OK(hms_client_->AlterTable(hms_database_name, hms_table_name, hms_table, env_ctx));
+  ASSERT_OK(hms_client_->AlterTable(hms_database_name, hms_table_name, hms_table, MasterEnvCtx()));
   hive::Table altered_table;
   ASSERT_OK(hms_client_->GetTable(hms_database_name, hms_table_name, &altered_table));
   ASSERT_TRUE(altered_table.sd.cols.empty());
@@ -351,8 +345,8 @@ TEST_F(MasterHmsTest, TestAlterTable) {
 
   table_alterer.reset(client_->NewTableAlterer(table_name));
   s = table_alterer->DropColumn("int32_val")->Alter();
-  EXPECT_TRUE(s.IsAlreadyPresent()) << s.ToString();
-  ASSERT_STR_CONTAINS(s.ToString(), "already exists in the HMS");
+  EXPECT_TRUE(s.IsNotFound()) << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "belongs to another table");
 }
 
 TEST_F(MasterHmsTest, TestDeleteTable) {
@@ -371,16 +365,21 @@ TEST_F(MasterHmsTest, TestDeleteTable) {
   NO_FATALS(CheckTableDoesNotExist(hms_database_name, hms_table_name));
 
   // Create the Kudu table, remove the HMS entry, and ensure the Kudu table can
-  // still be dropped.
+  // not be dropped.
   ASSERT_OK(CreateKuduTable(hms_database_name, hms_table_name));
   NO_FATALS(CheckTable(hms_database_name, hms_table_name));
+  ASSERT_OK(hms_client_->GetTable(hms_database_name, hms_table_name, &hms_table));
   shared_ptr<KuduTable> table;
   ASSERT_OK(client_->OpenTable(table_name, &table));
   ASSERT_OK(hms_client_->DropTable(hms_database_name, hms_table_name));
   Status s = hms_client_->GetTable(hms_database_name, hms_table_name, &hms_table);
   ASSERT_TRUE(s.IsNotFound()) << s.ToString();
+  s = client_->DeleteTable(table_name);
+  ASSERT_TRUE(s.IsNotFound()) << s.ToString();
+
+  // Re-create the HMS catalog entry and try again.
+  ASSERT_OK(hms_client_->CreateTable(hms_table, MasterEnvCtx()));
   ASSERT_OK(client_->DeleteTable(table_name));
-  NO_FATALS(CheckTableDoesNotExist(hms_database_name, hms_table_name));
 
   // Ensure that dropping a table while the HMS is unreachable fails.
   ASSERT_OK(CreateKuduTable(hms_database_name, hms_table_name));