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/09/11 22:45:40 UTC

kudu git commit: HMS integration: set table owner field in HMS table metadata

Repository: kudu
Updated Branches:
  refs/heads/master ed60c11a3 -> 97dbffb95


HMS integration: set table owner field in HMS table metadata

Other systems use table ownership for purposes like assigning
privileges. This patch sets the owner field in the HMS for Kudu tables
to the client's user name. A follow-up patch will add additional APIs to
the client CreateTable builders which will allow clients to override the
owner, for situations in which the client is actually proxying through
the table creation on behalf of a different user.

The HMS fix tool will create HMS table metadata for Kudu tables which
are missing it. This replacement table metadata will omit the table
owner, since it can't be reconstructed just from the Kudu table
metadata. This is the conservative approach, if we want to extend the
tool to allow passing in the owner as a flag we can do that in the
future.

Change-Id: I1a25aa0bb52bdd28df28a078fe91f55db9e29482
Reviewed-on: http://gerrit.cloudera.org:8080/11398
Tested-by: Kudu Jenkins
Reviewed-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/97dbffb9
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/97dbffb9
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/97dbffb9

Branch: refs/heads/master
Commit: 97dbffb95228b29b35ecba4be6708a33522ec152
Parents: ed60c11
Author: Dan Burkert <da...@apache.org>
Authored: Thu Sep 6 16:08:41 2018 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Tue Sep 11 22:45:28 2018 +0000

----------------------------------------------------------------------
 src/kudu/hms/hms_catalog-test.cc               | 46 ++++++++-----
 src/kudu/hms/hms_catalog.cc                    | 18 +++--
 src/kudu/hms/hms_catalog.h                     |  9 +++
 src/kudu/integration-tests/master_hms-itest.cc | 46 +++++++++++--
 src/kudu/master/catalog_manager.cc             |  3 +-
 src/kudu/tools/kudu-tool-test.cc               | 74 ++++++++++++++-------
 src/kudu/tools/tool_action_hms.cc              |  4 +-
 7 files changed, 145 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/97dbffb9/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 4574402..cf9977b 100644
--- a/src/kudu/hms/hms_catalog-test.cc
+++ b/src/kudu/hms/hms_catalog-test.cc
@@ -24,6 +24,7 @@
 #include <utility>
 #include <vector>
 
+#include <boost/optional/optional.hpp>
 #include <gflags/gflags_declare.h>
 #include <gtest/gtest.h>
 
@@ -236,10 +237,16 @@ class HmsCatalogTest : public KuduTest {
   void CheckTable(const string& database_name,
                   const string& table_name,
                   const string& table_id,
+                  const boost::optional<const string&>& owner,
                   const Schema& schema) {
     hive::Table table;
     ASSERT_OK(hms_client_->GetTable(database_name, table_name, &table));
 
+    if (owner) {
+      EXPECT_EQ(table.owner, *owner);
+    } else {
+      EXPECT_TRUE(table.owner.empty());
+    }
     EXPECT_EQ(table.parameters[HmsClient::kKuduTableIdKey], table_id);
     EXPECT_EQ(table.parameters[HmsClient::kKuduMasterAddrsKey], kMasterAddrs);
     EXPECT_EQ(table.parameters[HmsClient::kStorageHandlerKey], HmsClient::kKuduStorageHandler);
@@ -313,17 +320,18 @@ TEST_P(HmsCatalogTestParameterized, TestTableLifecycle) {
   const string kTableName = Substitute("$0.$1", kHmsDatabase, kHmsTableName);
   const string kHmsAlteredTableName = "altered_table_name";
   const string kAlteredTableName = Substitute("$0.$1", kHmsDatabase, kHmsAlteredTableName);
+  const string kOwner = "alice";
 
   Schema schema = AllTypesSchema();
 
   // Create the table.
-  ASSERT_OK(hms_catalog_->CreateTable(kTableId, kTableName, schema));
-  NO_FATALS(CheckTable(kHmsDatabase, kHmsTableName, kTableId, schema));
+  ASSERT_OK(hms_catalog_->CreateTable(kTableId, kTableName, kOwner, schema));
+  NO_FATALS(CheckTable(kHmsDatabase, kHmsTableName, kTableId, kOwner, schema));
 
   // Create the table again, and check that the expected failure occurs.
-  Status s = hms_catalog_->CreateTable(kTableId, kTableName, schema);
+  Status s = hms_catalog_->CreateTable(kTableId, kTableName, kOwner, schema);
   ASSERT_TRUE(s.IsAlreadyPresent()) << s.ToString();
-  NO_FATALS(CheckTable(kHmsDatabase, kHmsTableName, kTableId, schema));
+  NO_FATALS(CheckTable(kHmsDatabase, kHmsTableName, kTableId, kOwner, schema));
 
   // Alter the table.
   SchemaBuilder b(schema);
@@ -331,7 +339,7 @@ TEST_P(HmsCatalogTestParameterized, TestTableLifecycle) {
   Schema altered_schema = b.Build();
   ASSERT_OK(hms_catalog_->AlterTable(kTableId, kTableName, kAlteredTableName, altered_schema));
   NO_FATALS(CheckTableDoesNotExist(kHmsDatabase, kHmsTableName));
-  NO_FATALS(CheckTable(kHmsDatabase, kHmsAlteredTableName, kTableId, altered_schema));
+  NO_FATALS(CheckTable(kHmsDatabase, kHmsAlteredTableName, kTableId, kOwner, altered_schema));
 
   // Drop the table.
   ASSERT_OK(hms_catalog_->DropTable(kTableId, kAlteredTableName));
@@ -362,11 +370,11 @@ TEST_F(HmsCatalogTest, TestExternalTable) {
 
   // Create the Kudu table (default.a).
   Schema schema = AllTypesSchema();
-  ASSERT_OK(hms_catalog_->CreateTable(kTableId, "default.a", schema));
-  NO_FATALS(CheckTable("default", "a", kTableId, schema));
+  ASSERT_OK(hms_catalog_->CreateTable(kTableId, "default.a", boost::none, schema));
+  NO_FATALS(CheckTable("default", "a", kTableId, boost::none, schema));
 
   // Try and create a Kudu table with the same name as the external table.
-  Status s = hms_catalog_->CreateTable(kTableId, "default.ext", schema);
+  Status s = hms_catalog_->CreateTable(kTableId, "default.ext", boost::none, schema);
   EXPECT_TRUE(s.IsAlreadyPresent()) << s.ToString();
   NO_FATALS(CheckExternalTable());
 
@@ -374,14 +382,14 @@ TEST_F(HmsCatalogTest, TestExternalTable) {
   s = hms_catalog_->AlterTable(kTableId, "default.a", "default.ext", schema);
   EXPECT_TRUE(s.IsIllegalState()) << s.ToString();
   NO_FATALS(CheckExternalTable());
-  NO_FATALS(CheckTable("default", "a", kTableId, schema));
+  NO_FATALS(CheckTable("default", "a", kTableId, boost::none, schema));
 
   // 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(CheckTable("default", "a", kTableId, boost::none, schema));
   NO_FATALS(CheckTableDoesNotExist("default", "b"));
 
   // Try and drop the external table as if it were a Kudu table.
@@ -402,6 +410,7 @@ TEST_F(HmsCatalogTest, TestGetKuduTables) {
   const string kExternalTableName = "external_table";
   const string kTableName = "external_table";
   const string kNonKuduTableName = "non_kudu_table";
+  const string kOwner = "alice";
 
   // Create a legacy Impala managed table, a legacy Impala external table, a
   // Kudu table, and a non Kudu table.
@@ -414,7 +423,7 @@ TEST_F(HmsCatalogTest, TestGetKuduTables) {
   ASSERT_OK(CreateLegacyTable("db", "external_table", HmsClient::kExternalTable));
   ASSERT_OK(hms_client_->GetTable("db", "external_table", &table));
 
-  ASSERT_OK(hms_catalog_->CreateTable("fake-id", "db.table", Schema()));
+  ASSERT_OK(hms_catalog_->CreateTable("fake-id", "db.table", kOwner, Schema()));
 
   hive::Table non_kudu_table;
   non_kudu_table.dbName = "db";
@@ -438,9 +447,10 @@ TEST_F(HmsCatalogTest, TestReconnect) {
 
   const string kTableId = "table-id";
   const string kHmsDatabase = "default";
+  const string kOwner = "alice";
   Schema schema = AllTypesSchema();
-  ASSERT_OK(hms_catalog_->CreateTable(kTableId, "default.a", schema));
-  NO_FATALS(CheckTable(kHmsDatabase, "a", kTableId, schema));
+  ASSERT_OK(hms_catalog_->CreateTable(kTableId, "default.a", kOwner, schema));
+  NO_FATALS(CheckTable(kHmsDatabase, "a", kTableId, kOwner, schema));
 
   // Shutdown the HMS and try a few operations.
   ASSERT_OK(StopHms());
@@ -449,7 +459,7 @@ TEST_F(HmsCatalogTest, TestReconnect) {
   // while the HMS is unavailable results in a non-linear number of reconnect
   // attempts.
 
-  Status s = hms_catalog_->CreateTable(kTableId, "default.b", schema);
+  Status s = hms_catalog_->CreateTable(kTableId, "default.b", kOwner, schema);
   EXPECT_TRUE(s.IsNetworkError()) << s.ToString();
 
   s = hms_catalog_->AlterTable(kTableId, "default.a", "default.c", schema);
@@ -459,14 +469,14 @@ TEST_F(HmsCatalogTest, TestReconnect) {
   ASSERT_OK(StartHms());
   ASSERT_EVENTUALLY([&] {
     // HmsCatalog throttles reconnections, so it's necessary to wait out the backoff.
-    ASSERT_OK(hms_catalog_->CreateTable(kTableId, "default.d", schema));
+    ASSERT_OK(hms_catalog_->CreateTable(kTableId, "default.d", kOwner, schema));
   });
 
-  NO_FATALS(CheckTable(kHmsDatabase, "a", kTableId, schema));
-  NO_FATALS(CheckTable(kHmsDatabase, "d", kTableId, schema));
+  NO_FATALS(CheckTable(kHmsDatabase, "a", kTableId, kOwner, schema));
+  NO_FATALS(CheckTable(kHmsDatabase, "d", kTableId, kOwner, schema));
 
   EXPECT_OK(hms_catalog_->AlterTable(kTableId, "default.a", "default.c", schema));
-  NO_FATALS(CheckTable(kHmsDatabase, "c", kTableId, schema));
+  NO_FATALS(CheckTable(kHmsDatabase, "c", kTableId, kOwner, schema));
   NO_FATALS(CheckTableDoesNotExist(kHmsDatabase, "a"));
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/97dbffb9/src/kudu/hms/hms_catalog.cc
----------------------------------------------------------------------
diff --git a/src/kudu/hms/hms_catalog.cc b/src/kudu/hms/hms_catalog.cc
index 1b57ccb..f62f9ba 100644
--- a/src/kudu/hms/hms_catalog.cc
+++ b/src/kudu/hms/hms_catalog.cc
@@ -48,6 +48,7 @@
 #include "kudu/util/slice.h"
 #include "kudu/util/threadpool.h"
 
+using boost::none;
 using boost::optional;
 using std::move;
 using std::string;
@@ -144,10 +145,11 @@ void HmsCatalog::Stop() {
 
 Status HmsCatalog::CreateTable(const string& id,
                                const string& name,
+                               optional<const string&> owner,
                                const Schema& schema) {
   return Execute([&] (HmsClient* client) {
       hive::Table table;
-      RETURN_NOT_OK(PopulateTable(id, name, schema, master_addresses_, &table));
+      RETURN_NOT_OK(PopulateTable(id, name, owner, schema, master_addresses_, &table));
       return client->CreateTable(table, EnvironmentContext());
   });
 }
@@ -172,8 +174,8 @@ Status HmsCatalog::DropTable(const string& name, const hive::EnvironmentContext&
 }
 
 Status HmsCatalog::UpgradeLegacyImpalaTable(const string& id,
-                                            const std::string& db_name,
-                                            const std::string& tb_name,
+                                            const string& db_name,
+                                            const string& tb_name,
                                             const Schema& schema) {
   return Execute([&] (HmsClient* client) {
     hive::Table table;
@@ -184,12 +186,12 @@ Status HmsCatalog::UpgradeLegacyImpalaTable(const string& id,
     }
 
     RETURN_NOT_OK(PopulateTable(id, Substitute("$0.$1", db_name, tb_name),
-                                schema, master_addresses_, &table));
+                                none, schema, master_addresses_, &table));
     return client->AlterTable(db_name, tb_name, table, EnvironmentContext());
   });
 }
 
-Status HmsCatalog::DowngradeToLegacyImpalaTable(const std::string& name) {
+Status HmsCatalog::DowngradeToLegacyImpalaTable(const string& name) {
   return Execute([&] (HmsClient* client) {
     Slice hms_database;
     Slice hms_table;
@@ -283,7 +285,7 @@ Status HmsCatalog::AlterTable(const string& id,
       }
 
       // Overwrite fields in the table that have changed, including the new name.
-      RETURN_NOT_OK(PopulateTable(id, new_name, schema, master_addresses_, &table));
+      RETURN_NOT_OK(PopulateTable(id, new_name, none, schema, master_addresses_, &table));
       return client->AlterTable(hms_database.ToString(), hms_table.ToString(),
                                 table, EnvironmentContext());
   });
@@ -482,6 +484,7 @@ void ToLowerCase(Slice s) {
 
 Status HmsCatalog::PopulateTable(const string& id,
                                  const string& name,
+                                 const optional<const string&>& owner,
                                  const Schema& schema,
                                  const string& master_addresses,
                                  hive::Table* table) {
@@ -490,6 +493,9 @@ Status HmsCatalog::PopulateTable(const string& id,
   RETURN_NOT_OK(ParseTableName(name, &hms_database_name, &hms_table_name));
   table->dbName = hms_database_name.ToString();
   table->tableName = hms_table_name.ToString();
+  if (owner) {
+    table->owner = *owner;
+  }
 
   // Add the Kudu-specific parameters. This intentionally avoids overwriting
   // other parameters.

http://git-wip-us.apache.org/repos/asf/kudu/blob/97dbffb9/src/kudu/hms/hms_catalog.h
----------------------------------------------------------------------
diff --git a/src/kudu/hms/hms_catalog.h b/src/kudu/hms/hms_catalog.h
index bb807a1..de95ccf 100644
--- a/src/kudu/hms/hms_catalog.h
+++ b/src/kudu/hms/hms_catalog.h
@@ -21,6 +21,7 @@
 #include <string>
 #include <vector>
 
+#include <boost/optional/optional.hpp>
 #include <gtest/gtest_prod.h>
 
 #include "kudu/gutil/gscoped_ptr.h"
@@ -60,9 +61,14 @@ class HmsCatalog {
 
   // Creates a new table entry in the HMS.
   //
+  // If 'owner' is omitted the table will be created without an owner. This is
+  // useful in circumstances where the owner is not known, for example when
+  // creating an HMS table entry for an existing Kudu table.
+  //
   // Fails the HMS is unreachable, or a table with the same name is already present.
   Status CreateTable(const std::string& id,
                      const std::string& name,
+                     boost::optional<const std::string&> owner,
                      const Schema& schema) WARN_UNUSED_RESULT;
 
   // Drops a table entry from the HMS.
@@ -133,8 +139,11 @@ class HmsCatalog {
   static bool IsEnabled();
 
   // Sets the Kudu-specific fields in the table without overwriting unrelated fields.
+  //
+  // The table owner will not be overwritten if an owner is not provided.
   static Status PopulateTable(const std::string& id,
                               const std::string& name,
+                              const boost::optional<const std::string&>& owner,
                               const Schema& schema,
                               const std::string& master_addresses,
                               hive::Table* table) WARN_UNUSED_RESULT;

http://git-wip-us.apache.org/repos/asf/kudu/blob/97dbffb9/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 2fb632f..9b25dfb 100644
--- a/src/kudu/integration-tests/master_hms-itest.cc
+++ b/src/kudu/integration-tests/master_hms-itest.cc
@@ -38,11 +38,13 @@
 #include "kudu/integration-tests/external_mini_cluster-itest-base.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
 #include "kudu/mini-cluster/mini_cluster.h"
+#include "kudu/security/test/mini_kdc.h"
 #include "kudu/util/decimal_util.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
+#include "kudu/util/user.h"
 
 namespace kudu {
 
@@ -65,10 +67,6 @@ using strings::Substitute;
 class MasterHmsTest : public ExternalMiniClusterITestBase {
  public:
 
-  virtual HmsMode GetHmsMode() {
-    return HmsMode::ENABLE_METASTORE_INTEGRATION;
-  }
-
   void SetUp() override {
     ExternalMiniClusterITestBase::SetUp();
 
@@ -76,11 +74,14 @@ class MasterHmsTest : public ExternalMiniClusterITestBase {
     opts.hms_mode = GetHmsMode();
     opts.num_masters = 1;
     opts.num_tablet_servers = 1;
+    opts.enable_kerberos = EnableKerberos();
     // Tune down the notification log poll period in order to speed up catalog convergence.
     opts.extra_master_flags.emplace_back("--hive_metastore_notification_log_poll_period_seconds=1");
     StartClusterWithOpts(std::move(opts));
 
-    hms_client_.reset(new HmsClient(cluster_->hms()->address(), HmsClientOptions()));
+    HmsClientOptions hms_opts;
+    hms_opts.enable_kerberos = EnableKerberos();
+    hms_client_.reset(new HmsClient(cluster_->hms()->address(), hms_opts));
     ASSERT_OK(hms_client_->Start());
   }
 
@@ -180,6 +181,10 @@ class MasterHmsTest : public ExternalMiniClusterITestBase {
     hive::Table hms_table;
     ASSERT_OK(hms_client_->GetTable(database_name, table_name, &hms_table));
 
+    string username;
+    ASSERT_OK(GetLoggedInUser(&username));
+
+    ASSERT_EQ(hms_table.owner, username);
     ASSERT_EQ(schema.num_columns(), hms_table.sd.cols.size());
     for (int idx = 0; idx < schema.num_columns(); idx++) {
       ASSERT_EQ(schema.Column(idx).name(), hms_table.sd.cols[idx].name);
@@ -213,6 +218,16 @@ class MasterHmsTest : public ExternalMiniClusterITestBase {
  protected:
 
   unique_ptr<HmsClient> hms_client_;
+
+ private:
+
+  virtual HmsMode GetHmsMode() {
+    return HmsMode::ENABLE_METASTORE_INTEGRATION;
+  }
+
+  virtual bool EnableKerberos() {
+    return false;
+  }
 };
 
 TEST_F(MasterHmsTest, TestCreateTable) {
@@ -619,4 +634,25 @@ TEST_F(MasterHmsUpgradeTest, TestRenameExistingTables) {
   std::sort(tables.begin(), tables.end());
   ASSERT_EQ(tables, vector<string>({ "default.illegal_chars", "default.uppercase" }));
 }
+
+class MasterHmsKerberizedTest : public MasterHmsTest {
+ public:
+  bool EnableKerberos() override {
+    return true;
+  }
+};
+
+// Checks that table ownership in a Kerberized cluster is set to the user
+// short-name (instead of the full Kerberos principal).
+TEST_F(MasterHmsKerberizedTest, TestTableOwnership) {
+  // Log in as the test user and reset the client to pick up the change in user.
+  ASSERT_OK(cluster_->kdc()->Kinit("test-user"));
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client_));
+
+  // Create a table as the user and ensure that the ownership is set correctly.
+  ASSERT_OK(CreateKuduTable("default", "my_table"));
+  hive::Table table;
+  ASSERT_OK(hms_client_->GetTable("default", "my_table", &table));
+  ASSERT_EQ("test-user", table.owner);
+}
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/97dbffb9/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 27e9722..6aa2186 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -1565,7 +1565,8 @@ 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_) {
-    Status s = hms_catalog_->CreateTable(table->id(), normalized_table_name, schema);
+    Status s = hms_catalog_->CreateTable(table->id(), normalized_table_name,
+                                         rpc->remote_user().username(), schema);
     if (!s.ok()) {
       s = s.CloneAndPrepend(Substitute("an error occurred while creating table $0 in the HMS",
                                        normalized_table_name));

http://git-wip-us.apache.org/repos/asf/kudu/blob/97dbffb9/src/kudu/tools/kudu-tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 059753b..66b8648 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -133,15 +133,16 @@ DECLARE_string(hive_metastore_uris);
 METRIC_DECLARE_counter(bloom_lookups);
 METRIC_DECLARE_entity(tablet);
 
+using boost::optional;
 using kudu::cfile::CFileWriter;
 using kudu::cfile::StringDataGenerator;
 using kudu::cfile::WriterOptions;
 using kudu::client::KuduClient;
 using kudu::client::KuduClientBuilder;
-using kudu::client::KuduSchema;
-using kudu::client::KuduSchemaBuilder;
 using kudu::client::KuduScanToken;
 using kudu::client::KuduScanTokenBuilder;
+using kudu::client::KuduSchema;
+using kudu::client::KuduSchemaBuilder;
 using kudu::client::KuduTable;
 using kudu::client::sp::shared_ptr;
 using kudu::cluster::ExternalMiniCluster;
@@ -178,8 +179,8 @@ using kudu::tserver::MiniTabletServer;
 using kudu::tserver::WriteRequestPB;
 using std::back_inserter;
 using std::copy;
-using std::map;
 using std::make_pair;
+using std::map;
 using std::ostringstream;
 using std::pair;
 using std::string;
@@ -1910,7 +1911,7 @@ TEST_F(ToolTest, TestLocalReplicaTombstoneDelete) {
   // so that we can compare the size of the data on disk before and
   // after the deletion of local_replica to verify that the size-on-disk
   // is reduced after the tool operation.
-  boost::optional<OpId> last_logged_opid;
+  optional<OpId> last_logged_opid;
   string tablet_id;
   {
     vector<scoped_refptr<TabletReplica>> tablet_replicas;
@@ -1951,7 +1952,7 @@ TEST_F(ToolTest, TestLocalReplicaTombstoneDelete) {
     ASSERT_EQ(tablet_id, tablet_replicas[0]->tablet_id());
     ASSERT_EQ(TabletDataState::TABLET_DATA_TOMBSTONED,
               tablet_replicas[0]->tablet_metadata()->tablet_data_state());
-    boost::optional<OpId> tombstoned_opid =
+    optional<OpId> tombstoned_opid =
         tablet_replicas[0]->tablet_metadata()->tombstone_last_logged_opid();
     ASSERT_NE(boost::none, tombstoned_opid);
     ASSERT_NE(boost::none, last_logged_opid);
@@ -2313,11 +2314,15 @@ Status CreateLegacyHmsTable(HmsClient* client,
                             const string& hms_table_name,
                             const string& kudu_table_name,
                             const string& kudu_master_addrs,
-                            const string& table_type) {
+                            const string& table_type,
+                            const optional<const string&>& owner) {
   hive::Table table;
   table.dbName = hms_database_name;
   table.tableName = hms_table_name;
   table.tableType = table_type;
+  if (owner) {
+    table.owner = *owner;
+  }
 
   table.__set_parameters({
       make_pair(HmsClient::kStorageHandlerKey, HmsClient::kLegacyKuduStorageHandler),
@@ -2443,6 +2448,7 @@ TEST_P(ToolTestKerberosParameterized, TestHmsDowngrade) {
 // Test HMS inconsistencies that can be automatically fixed.
 // Kerberos is enabled in order to test the tools work in secure clusters.
 TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
+  string kUsername = "alice";
   ExternalMiniClusterOptions opts;
   opts.hms_mode = HmsMode::ENABLE_HIVE_METASTORE;
   opts.enable_kerberos = EnableKerberos();
@@ -2471,7 +2477,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(CreateKuduTable(kudu_client, "default.control"));
   ASSERT_OK(kudu_client->OpenTable("default.control", &control));
   ASSERT_OK(hms_catalog.CreateTable(
-        control->id(), control->name(),
+        control->id(), control->name(), kUsername,
         client::SchemaFromKuduSchema(control->schema())));
 
   // Test case: Upper-case names are handled specially in a few places.
@@ -2479,7 +2485,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(CreateKuduTable(kudu_client, "default.UPPERCASE"));
   ASSERT_OK(kudu_client->OpenTable("default.UPPERCASE", &test_uppercase));
   ASSERT_OK(hms_catalog.CreateTable(
-        test_uppercase->id(), test_uppercase->name(),
+        test_uppercase->id(), test_uppercase->name(), kUsername,
         client::SchemaFromKuduSchema(test_uppercase->schema())));
 
   // Test case: inconsistent schema.
@@ -2487,7 +2493,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(CreateKuduTable(kudu_client, "default.inconsistent_schema"));
   ASSERT_OK(kudu_client->OpenTable("default.inconsistent_schema", &inconsistent_schema));
   ASSERT_OK(hms_catalog.CreateTable(
-        inconsistent_schema->id(), inconsistent_schema->name(),
+        inconsistent_schema->id(), inconsistent_schema->name(), kUsername,
         SchemaBuilder().Build()));
 
   // Test case: inconsistent name.
@@ -2495,7 +2501,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(CreateKuduTable(kudu_client, "default.inconsistent_name"));
   ASSERT_OK(kudu_client->OpenTable("default.inconsistent_name", &inconsistent_name));
   ASSERT_OK(hms_catalog.CreateTable(
-        inconsistent_name->id(), "default.inconsistent_name_hms",
+        inconsistent_name->id(), "default.inconsistent_name_hms", kUsername,
         client::SchemaFromKuduSchema(inconsistent_name->schema())));
 
   // Test case: inconsistent master addresses.
@@ -2506,19 +2512,19 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   HmsCatalog invalid_hms_catalog("invalid-master-addrs");
   ASSERT_OK(invalid_hms_catalog.Start());
   ASSERT_OK(invalid_hms_catalog.CreateTable(
-        inconsistent_master_addrs->id(), inconsistent_master_addrs->name(),
+        inconsistent_master_addrs->id(), inconsistent_master_addrs->name(), kUsername,
         client::SchemaFromKuduSchema(inconsistent_master_addrs->schema())));
 
   // Test cases: orphan tables in the HMS.
   ASSERT_OK(hms_catalog.CreateTable(
-        "orphan-hms-table-id", "default.orphan_hms_table",
+        "orphan-hms-table-id", "default.orphan_hms_table", kUsername,
         SchemaBuilder().Build()));
   ASSERT_OK(CreateLegacyHmsTable(&hms_client, "default", "orphan_hms_table_legacy_external",
         "default.orphan_hms_table_legacy_external",
-        master_addr, HmsClient::kExternalTable));
+        master_addr, HmsClient::kExternalTable, kUsername));
   ASSERT_OK(CreateLegacyHmsTable(&hms_client, "default", "orphan_hms_table_legacy_managed",
         "impala::default.orphan_hms_table_legacy_managed",
-        master_addr, HmsClient::kExternalTable));
+        master_addr, HmsClient::kManagedTable, kUsername));
 
   // Test case: orphan table in Kudu.
   ASSERT_OK(CreateKuduTable(kudu_client, "default.kudu_orphan"));
@@ -2528,23 +2534,30 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   ASSERT_OK(CreateKuduTable(kudu_client, "default.legacy_external"));
   ASSERT_OK(kudu_client->OpenTable("default.legacy_external", &legacy_external));
   ASSERT_OK(CreateLegacyHmsTable(&hms_client, "default", "legacy_external",
-        "default.legacy_external",
-        master_addr, HmsClient::kExternalTable));
+        "default.legacy_external", master_addr, HmsClient::kExternalTable, kUsername));
 
   // Test case: legacy managed table.
   shared_ptr<KuduTable> legacy_managed;
   ASSERT_OK(CreateKuduTable(kudu_client, "impala::default.legacy_managed"));
   ASSERT_OK(kudu_client->OpenTable("impala::default.legacy_managed", &legacy_managed));
   ASSERT_OK(CreateLegacyHmsTable(&hms_client, "default", "legacy_managed",
-        "impala::default.legacy_managed", master_addr, HmsClient::kManagedTable));
+        "impala::default.legacy_managed", master_addr, HmsClient::kManagedTable, kUsername));
+
+  // Test case: legacy managed table with no owner.
+  shared_ptr<KuduTable> legacy_no_owner;
+  ASSERT_OK(CreateKuduTable(kudu_client, "impala::default.legacy_no_owner"));
+  ASSERT_OK(kudu_client->OpenTable("impala::default.legacy_no_owner", &legacy_no_owner));
+  ASSERT_OK(CreateLegacyHmsTable(&hms_client, "default", "legacy_no_owner",
+        "impala::default.legacy_no_owner", master_addr, HmsClient::kManagedTable, boost::none));
 
   // Test case: legacy external table with a Hive-incompatible name (no database).
   shared_ptr<KuduTable> legacy_external_hive_incompatible_name;
   ASSERT_OK(CreateKuduTable(kudu_client, "legacy_external_hive_incompatible_name"));
   ASSERT_OK(kudu_client->OpenTable("legacy_external_hive_incompatible_name",
-  &legacy_external_hive_incompatible_name));
+        &legacy_external_hive_incompatible_name));
   ASSERT_OK(CreateLegacyHmsTable(&hms_client, "default", "legacy_external_hive_incompatible_name",
-        "legacy_external_hive_incompatible_name", master_addr, HmsClient::kExternalTable));
+        "legacy_external_hive_incompatible_name", master_addr,
+        HmsClient::kExternalTable, kUsername));
 
   // Test case: Kudu table in non-default database.
   hive::Database db;
@@ -2572,6 +2585,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
     "default.kudu_orphan",
     "default.legacy_external",
     "default.legacy_managed",
+    "default.legacy_no_owner",
     "legacy_external_hive_incompatible_name",
     "my_db.table",
   };
@@ -2640,6 +2654,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
   make_consistent({
     "default.legacy_external",
     "default.legacy_managed",
+    "default.legacy_no_owner",
     "legacy_external_hive_incompatible_name",
   });
   NO_FATALS(check());
@@ -2685,13 +2700,26 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndAutomaticFixHmsMetadata) {
     "default.legacy_external",
     "default.legacy_external_hive_incompatible_name",
     "default.legacy_managed",
+    "default.legacy_no_owner",
     "default.uppercase",
     "my_db.table",
   }), kudu_tables);
+
+  // Check that table ownership is preserved in upgraded legacy tables.
+  for (auto p : vector<pair<string, string>>({
+        make_pair("legacy_external", kUsername),
+        make_pair("legacy_managed", kUsername),
+        make_pair("legacy_no_owner", ""),
+  })) {
+    hive::Table table;
+    ASSERT_OK(hms_client.GetTable("default", p.first, &table));
+    ASSERT_EQ(p.second, table.owner);
+  }
 }
 
 // Test HMS inconsistencies that must be manually fixed.
 TEST_P(ToolTestKerberosParameterized, TestCheckAndManualFixHmsMetadata) {
+  string kUsername = "alice";
   ExternalMiniClusterOptions opts;
   opts.hms_mode = HmsMode::ENABLE_HIVE_METASTORE;
   opts.enable_kerberos = EnableKerberos();
@@ -2720,14 +2748,14 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndManualFixHmsMetadata) {
   ASSERT_OK(CreateKuduTable(kudu_client, "default.duplicate_hms_tables"));
   ASSERT_OK(kudu_client->OpenTable("default.duplicate_hms_tables", &duplicate_hms_tables));
   ASSERT_OK(hms_catalog.CreateTable(
-        duplicate_hms_tables->id(), "default.duplicate_hms_tables",
+        duplicate_hms_tables->id(), "default.duplicate_hms_tables", kUsername,
         client::SchemaFromKuduSchema(duplicate_hms_tables->schema())));
   ASSERT_OK(hms_catalog.CreateTable(
-        duplicate_hms_tables->id(), "default.duplicate_hms_tables_2",
+        duplicate_hms_tables->id(), "default.duplicate_hms_tables_2", kUsername,
         client::SchemaFromKuduSchema(duplicate_hms_tables->schema())));
   ASSERT_OK(CreateLegacyHmsTable(&hms_client, "default", "duplicate_hms_tables_3",
         "default.duplicate_hms_tables",
-        master_addr, HmsClient::kExternalTable));
+        master_addr, HmsClient::kExternalTable, kUsername));
 
   // Test case: Kudu tables Hive-incompatible names.
   ASSERT_OK(CreateKuduTable(kudu_client, "default.hive-incompatible-name"));
@@ -2739,7 +2767,7 @@ TEST_P(ToolTestKerberosParameterized, TestCheckAndManualFixHmsMetadata) {
   // Test case: a legacy table with a Hive name which conflicts with another table in Kudu.
   ASSERT_OK(CreateLegacyHmsTable(&hms_client, "default", "conflicting_legacy_table",
         "impala::default.conflicting_legacy_table",
-        master_addr, HmsClient::kManagedTable));
+        master_addr, HmsClient::kManagedTable, kUsername));
   ASSERT_OK(CreateKuduTable(kudu_client, "impala::default.conflicting_legacy_table"));
   ASSERT_OK(CreateKuduTable(kudu_client, "default.conflicting_legacy_table"));
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/97dbffb9/src/kudu/tools/tool_action_hms.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_hms.cc b/src/kudu/tools/tool_action_hms.cc
index ccf2557..ded8871 100644
--- a/src/kudu/tools/tool_action_hms.cc
+++ b/src/kudu/tools/tool_action_hms.cc
@@ -165,7 +165,7 @@ bool IsSynced(const string& master_addresses,
               const hive::Table& hms_table) {
   Schema schema(client::SchemaFromKuduSchema(kudu_table.schema()));
   hive::Table hms_table_copy(hms_table);
-  Status s = HmsCatalog::PopulateTable(kudu_table.id(), kudu_table.name(),
+  Status s = HmsCatalog::PopulateTable(kudu_table.id(), kudu_table.name(), boost::none,
                                        schema, master_addresses, &hms_table_copy);
   return s.ok() && hms_table_copy == hms_table;
 }
@@ -513,7 +513,7 @@ Status FixHmsMetadata(const RunnerContext& context) {
       if (FLAGS_dryrun) {
         LOG(INFO) << "[dryrun] Creating HMS table for Kudu table " << TableIdent(*kudu_table);
       } else {
-        Status s = hms_catalog->CreateTable(table_id, table_name, schema);
+        Status s = hms_catalog->CreateTable(table_id, table_name, boost::none, schema);
         if (s.IsAlreadyPresent()) {
           LOG(ERROR) << "Failed to create HMS table for Kudu table "
                      << TableIdent(*kudu_table)