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/07/24 21:14:22 UTC

kudu git commit: Allow non-HMS compatible tables to be renamed after turning on HMS integration

Repository: kudu
Updated Branches:
  refs/heads/master ccdb6b557 -> 7fe04438e


Allow non-HMS compatible tables to be renamed after turning on HMS integration

This is a bug fix followup to cee17c03bc30037bf2a, which introduced
automatic downcasing of table names during new table creation and table
renaming.

Change-Id: If554efbda06e33b5fdb40565e700bad8a306c143
Reviewed-on: http://gerrit.cloudera.org:8080/11016
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Reviewed-by: Hao Hao <ha...@cloudera.com>
Tested-by: Dan Burkert <da...@apache.org>


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

Branch: refs/heads/master
Commit: 7fe04438ecd8e100509ba224826e753ffc851d49
Parents: ccdb6b5
Author: Dan Burkert <da...@apache.org>
Authored: Tue Jul 17 17:33:00 2018 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Tue Jul 24 21:14:07 2018 +0000

----------------------------------------------------------------------
 src/kudu/client/client.h                       |  2 ++
 src/kudu/integration-tests/master_hms-itest.cc | 29 +++++++++++++++++++
 src/kudu/master/catalog_manager.cc             | 31 ++++++++++++---------
 3 files changed, 49 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/7fe04438/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 9ad7afc..13e25da 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -52,6 +52,7 @@ namespace kudu {
 class ClientStressTest_TestUniqueClientIds_Test;
 class KuduPartialRow;
 class MasterHmsTest_TestAlterTable_Test;
+class MasterHmsUpgradeTest_TestRenameExistingTables_Test;
 class MonoDelta;
 class PartitionSchema;
 class SecurityUnknownTskTest;
@@ -1198,6 +1199,7 @@ class KUDU_EXPORT KuduTableAlterer {
       const std::string& new_name);
 
   FRIEND_TEST(kudu::MasterHmsTest, TestAlterTable);
+  FRIEND_TEST(kudu::MasterHmsUpgradeTest, TestRenameExistingTables);
 
   KuduTableAlterer(KuduClient* client,
                    const std::string& name);

http://git-wip-us.apache.org/repos/asf/kudu/blob/7fe04438/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 4706a9d..7ba3c8c 100644
--- a/src/kudu/integration-tests/master_hms-itest.cc
+++ b/src/kudu/integration-tests/master_hms-itest.cc
@@ -579,4 +579,33 @@ TEST_F(MasterHmsUpgradeTest, TestConflictingNormalizedNames) {
   std::sort(tables.begin(), tables.end());
   ASSERT_EQ(tables, vector<string>({ "default.MyTable", "default.mytable-renamed" }));
 }
+
+// Checks that existing tables with HMS-incompatible names can be renamed post
+// upgrade using a Kudu-catalog only alter.
+TEST_F(MasterHmsUpgradeTest, TestRenameExistingTables) {
+  ASSERT_OK(CreateKuduTable("default", "UPPERCASE"));
+  ASSERT_OK(CreateKuduTable("default", "illegal-chars⁉"));
+
+  // Shutdown the masters and turn on the HMS integration
+  cluster_->ShutdownNodes(cluster::ClusterNodes::MASTERS_ONLY);
+  cluster_->EnableMetastoreIntegration();
+  ASSERT_OK(cluster_->Restart());
+
+  vector<string> tables;
+  ASSERT_OK(client_->ListTables(&tables));
+  std::sort(tables.begin(), tables.end());
+  ASSERT_EQ(tables, vector<string>({ "default.UPPERCASE", "default.illegal-chars⁉" }));
+
+  // Rename the tables using a Kudu catalog only rename.
+  unique_ptr<KuduTableAlterer> alterer(client_->NewTableAlterer("default.UPPERCASE"));
+  ASSERT_OK(alterer->RenameTo("default.uppercase")->alter_external_catalogs(false)->Alter());
+
+  alterer.reset(client_->NewTableAlterer("default.illegal-chars⁉"));
+  ASSERT_OK(alterer->RenameTo("default.illegal_chars")->alter_external_catalogs(false)->Alter());
+
+  tables.clear();
+  client_->ListTables(&tables);
+  std::sort(tables.begin(), tables.end());
+  ASSERT_EQ(tables, vector<string>({ "default.illegal_chars", "default.uppercase" }));
+}
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/7fe04438/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index b10a4cc..d6b4654 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -2261,7 +2261,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
         resp, MasterErrorPB::TABLE_NOT_FOUND);
   }
 
-  string table_name = l.data().name();
+  string normalized_table_name = NormalizeTableName(l.data().name());
   *resp->mutable_table_id() = table->id();
 
   // 3. Calculate and validate new schema for the on-disk state, not persisted yet.
@@ -2288,22 +2288,27 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
   // 4. Validate and try to acquire the new table name.
   string normalized_new_table_name = NormalizeTableName(req.new_table_name());
   if (req.has_new_table_name()) {
+
+    // Validate the new table name.
     RETURN_NOT_OK(SetupError(
-          ValidateIdentifier(normalized_new_table_name).CloneAndPrepend("invalid table name"),
+          ValidateIdentifier(req.new_table_name()).CloneAndPrepend("invalid table name"),
           resp, MasterErrorPB::INVALID_SCHEMA));
 
-    // Validate the new table name.
     std::lock_guard<LockType> catalog_lock(lock_);
     TRACE("Acquired catalog manager lock");
 
-    // Verify that the table does not exist.
+    // Verify that a table does not already exist with the new name. This
+    // also disallows no-op renames (ALTER TABLE a RENAME TO a).
+    //
+    // Special case: if this is a rename of a table from a non-normalized to
+    // normalized name (ALTER TABLE A RENAME to a), then allow it.
     scoped_refptr<TableInfo> other_table = FindPtrOrNull(normalized_table_names_map_,
                                                          normalized_new_table_name);
-
-    if (other_table != nullptr) {
+    if (other_table &&
+        !(table.get() == other_table.get() && l.data().name() != normalized_new_table_name)) {
       return SetupError(
           Status::AlreadyPresent(Substitute("table $0 already exists with id $1",
-              normalized_new_table_name, table->id())),
+              normalized_new_table_name, other_table->id())),
           resp, MasterErrorPB::TABLE_ALREADY_PRESENT);
     }
 
@@ -2323,7 +2328,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
   }
 
   // Ensure that we drop our reservation upon return.
-  auto cleanup = MakeScopedCleanup([&] () {
+  SCOPED_CLEANUP({
     if (req.has_new_table_name()) {
       std::lock_guard<LockType> l(lock_);
       CHECK_EQ(1, reserved_normalized_table_names_.erase(normalized_new_table_name));
@@ -2423,7 +2428,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
     // and tablets indices.
     std::lock_guard<LockType> lock(lock_);
     if (req.has_new_table_name()) {
-      if (normalized_table_names_map_.erase(table_name) != 1) {
+      if (normalized_table_names_map_.erase(normalized_table_name) != 1) {
         LOG(FATAL) << "Could not remove table " << table->ToString()
                    << " from map in response to AlterTable request: "
                    << SecureShortDebugString(req);
@@ -2473,10 +2478,10 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
     // table rename, since we split out the rename portion into its own
     // 'transaction' which is serialized through the HMS.
     DCHECK(!req.has_new_table_name());
-    WARN_NOT_OK(hms_catalog_->AlterTable(table->id(), table_name, table_name, new_schema),
-                Substitute(
-                  "failed to alter HiveMetastore schema for table $0, "
-                  "HMS schema information will be stale", table->ToString()));
+    WARN_NOT_OK(hms_catalog_->AlterTable(
+          table->id(), normalized_table_name, normalized_table_name, new_schema),
+        Substitute("failed to alter HiveMetastore schema for table $0, "
+                   "HMS schema information will be stale", table->ToString()));
   }
 
   if (!tablets_to_add.empty() || has_metadata_changes) {