You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2018/10/16 04:34:46 UTC

[3/4] kudu git commit: [sentry] move ParseTableName to table_util

[sentry] move ParseTableName to table_util

This commit moves HmsCatalog::ParseTableName to table_util, so that
other modules that need to parse a Kudu table name into a database and
table name can use it without including hms module, e.g sentry.

Change-Id: I7fff5aaff623ac65a949f6cabebd9eca997f5cdc
Reviewed-on: http://gerrit.cloudera.org:8080/11658
Reviewed-by: Dan Burkert <da...@apache.org>
Reviewed-by: Andrew Wong <aw...@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/f6eddf28
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/f6eddf28
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/f6eddf28

Branch: refs/heads/master
Commit: f6eddf2873214147a5f9cfd25a69d558253101ca
Parents: 3570c63
Author: Hao Hao <ha...@cloudera.com>
Authored: Wed Oct 10 17:33:19 2018 -0700
Committer: Hao Hao <ha...@cloudera.com>
Committed: Tue Oct 16 03:41:19 2018 +0000

----------------------------------------------------------------------
 src/kudu/common/CMakeLists.txt                 |  2 +
 src/kudu/common/table_util-test.cc             | 64 ++++++++++++++++++++
 src/kudu/common/table_util.cc                  | 67 +++++++++++++++++++++
 src/kudu/common/table_util.h                   | 38 ++++++++++++
 src/kudu/hms/hms_catalog-test.cc               | 34 -----------
 src/kudu/hms/hms_catalog.cc                    | 46 ++------------
 src/kudu/hms/hms_catalog.h                     | 10 ---
 src/kudu/integration-tests/master_hms-itest.cc |  4 +-
 8 files changed, 179 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/f6eddf28/src/kudu/common/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/common/CMakeLists.txt b/src/kudu/common/CMakeLists.txt
index 5d29a00..a789ccb 100644
--- a/src/kudu/common/CMakeLists.txt
+++ b/src/kudu/common/CMakeLists.txt
@@ -56,6 +56,7 @@ set(COMMON_SRCS
   row_operations.cc
   scan_spec.cc
   schema.cc
+  table_util.cc
   timestamp.cc
   types.cc
   wire_protocol.cc)
@@ -90,5 +91,6 @@ ADD_KUDU_TEST(row_changelist-test)
 ADD_KUDU_TEST(row_operations-test)
 ADD_KUDU_TEST(scan_spec-test)
 ADD_KUDU_TEST(schema-test)
+ADD_KUDU_TEST(table_util-test)
 ADD_KUDU_TEST(types-test)
 ADD_KUDU_TEST(wire_protocol-test)

http://git-wip-us.apache.org/repos/asf/kudu/blob/f6eddf28/src/kudu/common/table_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/table_util-test.cc b/src/kudu/common/table_util-test.cc
new file mode 100644
index 0000000..01344e3
--- /dev/null
+++ b/src/kudu/common/table_util-test.cc
@@ -0,0 +1,64 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/common/table_util.h"
+
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+namespace kudu {
+
+using std::string;
+
+TEST(TestTableUtil, TestParseHiveTableIdentifier) {
+  Slice db;
+  Slice tbl;
+  string table;
+
+  table = "foo.bar";
+  ASSERT_OK(ParseHiveTableIdentifier(table, &db, &tbl));
+  EXPECT_EQ("foo", db);
+  EXPECT_EQ("bar", tbl);
+
+  table = "99bottles.my_awesome/table/22";
+  ASSERT_OK(ParseHiveTableIdentifier(table, &db, &tbl));
+  EXPECT_EQ("99bottles", db);
+  EXPECT_EQ("my_awesome/table/22", tbl);
+
+  table = "_leading_underscore.trailing_underscore_";
+  ASSERT_OK(ParseHiveTableIdentifier(table, &db, &tbl));
+  EXPECT_EQ("_leading_underscore", db);
+  EXPECT_EQ("trailing_underscore_", tbl);
+
+  EXPECT_TRUE(ParseHiveTableIdentifier(".", &db, &tbl).IsInvalidArgument());
+  EXPECT_TRUE(ParseHiveTableIdentifier("no-table", &db, &tbl).IsInvalidArgument());
+  EXPECT_TRUE(ParseHiveTableIdentifier("lots.of.tables", &db, &tbl).IsInvalidArgument());
+  EXPECT_TRUE(ParseHiveTableIdentifier(".no_table", &db, &tbl).IsInvalidArgument());
+  EXPECT_TRUE(ParseHiveTableIdentifier("no_table.", &db, &tbl).IsInvalidArgument());
+  EXPECT_TRUE(ParseHiveTableIdentifier(".no_database", &db, &tbl).IsInvalidArgument());
+  EXPECT_TRUE(ParseHiveTableIdentifier("punctuation?.no", &db, &tbl).IsInvalidArgument());
+  EXPECT_TRUE(ParseHiveTableIdentifier("white space.no", &db, &tbl).IsInvalidArgument());
+  EXPECT_TRUE(ParseHiveTableIdentifier("unicode☃tables.no", &db, &tbl).IsInvalidArgument());
+  EXPECT_TRUE(ParseHiveTableIdentifier(string("\0.\0", 3), &db, &tbl).IsInvalidArgument());
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/f6eddf28/src/kudu/common/table_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/table_util.cc b/src/kudu/common/table_util.cc
new file mode 100644
index 0000000..04d0447
--- /dev/null
+++ b/src/kudu/common/table_util.cc
@@ -0,0 +1,67 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/common/table_util.h"
+
+#include <string>
+
+#include <boost/optional/optional.hpp>
+
+#include "kudu/gutil/strings/charset.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+using boost::optional;
+using std::string;
+
+namespace kudu {
+
+const char* const kInvalidTableError = "when the Hive Metastore integration "
+    "is enabled, Kudu table names must be a period ('.') separated database and table name "
+    "identifier pair, each containing only ASCII alphanumeric characters, '_', and '/'";
+
+Status ParseHiveTableIdentifier(const string& table_name,
+                                Slice* hms_database,
+                                Slice* hms_table) {
+  const char kSeparator = '.';
+  strings::CharSet charset("abcdefghijklmnopqrstuvwxyz"
+                           "ABCDEFGHIJKLMNOPQRSTUVWXYZ"
+                           "0123456789"
+                           "_/");
+
+  optional<int> separator_idx;
+  for (int idx = 0; idx < table_name.size(); idx++) {
+    char c = table_name[idx];
+    if (!charset.Test(c)) {
+      if (c == kSeparator && !separator_idx) {
+        separator_idx = idx;
+      } else {
+        return Status::InvalidArgument(kInvalidTableError, table_name);
+      }
+    }
+  }
+  if (!separator_idx || *separator_idx == 0 || *separator_idx == table_name.size() - 1) {
+    return Status::InvalidArgument(kInvalidTableError, table_name);
+  }
+
+  *hms_database = Slice(table_name.data(), *separator_idx);
+  *hms_table = Slice(table_name.data() + *separator_idx + 1,
+                     table_name.size() - *separator_idx - 1);
+  return Status::OK();
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/f6eddf28/src/kudu/common/table_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/table_util.h b/src/kudu/common/table_util.h
new file mode 100644
index 0000000..d68076b
--- /dev/null
+++ b/src/kudu/common/table_util.h
@@ -0,0 +1,38 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include <string>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Slice;
+
+extern const char* const kInvalidTableError;
+
+// Parses a Kudu table name of the form '<database>.<table>' into
+// a Hive database and table name. Returns an error if the Kudu
+// table name is not correctly formatted. The returned HMS database
+// and table slices must not outlive 'table_name'.
+Status ParseHiveTableIdentifier(const std::string& table_name,
+                                Slice* hms_database,
+                                Slice* hms_table) WARN_UNUSED_RESULT;
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/f6eddf28/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 83369fb..53bd462 100644
--- a/src/kudu/hms/hms_catalog-test.cc
+++ b/src/kudu/hms/hms_catalog-test.cc
@@ -39,7 +39,6 @@
 #include "kudu/security/test/mini_kdc.h"
 #include "kudu/thrift/client.h"
 #include "kudu/util/net/net_util.h"
-#include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
@@ -57,39 +56,6 @@ using strings::Substitute;
 namespace kudu {
 namespace hms {
 
-TEST(HmsCatalogStaticTest, TestParseTableName) {
-  Slice db;
-  Slice tbl;
-  string table;
-
-  table = "foo.bar";
-  ASSERT_OK(HmsCatalog::ParseTableName(table, &db, &tbl));
-  EXPECT_EQ("foo", db);
-  EXPECT_EQ("bar", tbl);
-
-  table = "99bottles.my_awesome/table/22";
-  ASSERT_OK(HmsCatalog::ParseTableName(table, &db, &tbl));
-  EXPECT_EQ("99bottles", db);
-  EXPECT_EQ("my_awesome/table/22", tbl);
-
-  table = "_leading_underscore.trailing_underscore_";
-  ASSERT_OK(HmsCatalog::ParseTableName(table, &db, &tbl));
-  EXPECT_EQ("_leading_underscore", db);
-  EXPECT_EQ("trailing_underscore_", tbl);
-
-  EXPECT_TRUE(HmsCatalog::ParseTableName(".", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName("no-table", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName("lots.of.tables", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName("no-table", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName("lots.of.tables", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName(".no_table", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName(".no_database", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName("punctuation?.no", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName("white space.no", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName("unicode☃tables.no", &db, &tbl).IsInvalidArgument());
-  EXPECT_TRUE(HmsCatalog::ParseTableName(string("\0.\0", 3), &db, &tbl).IsInvalidArgument());
-}
-
 TEST(HmsCatalogStaticTest, TestNormalizeTableName) {
   string table = "foo.bar";
   ASSERT_OK(HmsCatalog::NormalizeTableName(&table));

http://git-wip-us.apache.org/repos/asf/kudu/blob/f6eddf28/src/kudu/hms/hms_catalog.cc
----------------------------------------------------------------------
diff --git a/src/kudu/hms/hms_catalog.cc b/src/kudu/hms/hms_catalog.cc
index 7620b2e..589fe57 100644
--- a/src/kudu/hms/hms_catalog.cc
+++ b/src/kudu/hms/hms_catalog.cc
@@ -31,11 +31,11 @@
 
 #include "kudu/common/common.pb.h"
 #include "kudu/common/schema.h"
+#include "kudu/common/table_util.h"
 #include "kudu/common/types.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/ascii_ctype.h"
-#include "kudu/gutil/strings/charset.h"
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/hms/hive_metastore_types.h"
@@ -115,10 +115,6 @@ TAG_FLAG(hive_metastore_max_message_size, runtime);
 namespace kudu {
 namespace hms {
 
-const char* const HmsCatalog::kInvalidTableError = "when the Hive Metastore integration "
-    "is enabled, Kudu table names must be a period ('.') separated database and table name "
-    "identifier pair, each containing only ASCII alphanumeric characters, '_', and '/'";
-
 HmsCatalog::HmsCatalog(string master_addresses)
     : master_addresses_(std::move(master_addresses)) {
 }
@@ -171,7 +167,7 @@ Status HmsCatalog::DropLegacyTable(const string& name) {
 Status HmsCatalog::DropTable(const string& name, const hive::EnvironmentContext& env_ctx) {
   Slice hms_database;
   Slice hms_table;
-  RETURN_NOT_OK(ParseTableName(name, &hms_database, &hms_table));
+  RETURN_NOT_OK(ParseHiveTableIdentifier(name, &hms_database, &hms_table));
   return ha_client_.Execute([&] (HmsClient* client) {
     return client->DropTable(hms_database.ToString(), hms_table.ToString(), env_ctx);
   });
@@ -198,7 +194,7 @@ Status HmsCatalog::UpgradeLegacyImpalaTable(const string& id,
 Status HmsCatalog::DowngradeToLegacyImpalaTable(const string& name) {
   Slice hms_database;
   Slice hms_table;
-  RETURN_NOT_OK(ParseTableName(name, &hms_database, &hms_table));
+  RETURN_NOT_OK(ParseHiveTableIdentifier(name, &hms_database, &hms_table));
 
   return ha_client_.Execute([&] (HmsClient* client) {
     hive::Table table;
@@ -258,7 +254,7 @@ Status HmsCatalog::AlterTable(const string& id,
                               const Schema& schema) {
   Slice hms_database;
   Slice hms_table;
-  RETURN_NOT_OK(ParseTableName(name, &hms_database, &hms_table));
+  RETURN_NOT_OK(ParseHiveTableIdentifier(name, &hms_database, &hms_table));
 
   return ha_client_.Execute([&] (HmsClient* client) {
       // The HMS does not have a way to alter individual fields of a table
@@ -349,7 +345,7 @@ Status HmsCatalog::PopulateTable(const string& id,
                                  hive::Table* table) {
   Slice hms_database_name;
   Slice hms_table_name;
-  RETURN_NOT_OK(ParseTableName(name, &hms_database_name, &hms_table_name));
+  RETURN_NOT_OK(ParseHiveTableIdentifier(name, &hms_database_name, &hms_table_name));
   table->dbName = hms_database_name.ToString();
   table->tableName = hms_table_name.ToString();
   if (owner) {
@@ -387,7 +383,7 @@ Status HmsCatalog::NormalizeTableName(string* table_name) {
   CHECK_NOTNULL(table_name);
   Slice hms_database;
   Slice hms_table;
-  RETURN_NOT_OK(ParseTableName(*table_name, &hms_database, &hms_table));
+  RETURN_NOT_OK(ParseHiveTableIdentifier(*table_name, &hms_database, &hms_table));
 
   ToLowerCase(hms_database);
   ToLowerCase(hms_table);
@@ -395,36 +391,6 @@ Status HmsCatalog::NormalizeTableName(string* table_name) {
   return Status::OK();
 }
 
-Status HmsCatalog::ParseTableName(const string& table_name,
-                                  Slice* hms_database,
-                                  Slice* hms_table) {
-  const char kSeparator = '.';
-  strings::CharSet charset("abcdefghijklmnopqrstuvwxyz"
-                           "ABCDEFGHIJKLMNOPQRSTUVWXYZ"
-                           "0123456789"
-                           "_/");
-
-  optional<int> separator_idx;
-  for (int idx = 0; idx < table_name.size(); idx++) {
-    char c = table_name[idx];
-    if (!charset.Test(c)) {
-      if (c == kSeparator && !separator_idx) {
-        separator_idx = idx;
-      } else {
-        return Status::InvalidArgument(kInvalidTableError, table_name);
-      }
-    }
-  }
-  if (!separator_idx || *separator_idx == 0 || *separator_idx == table_name.size() - 1) {
-    return Status::InvalidArgument(kInvalidTableError, table_name);
-  }
-
-  *hms_database = Slice(table_name.data(), *separator_idx);
-  *hms_table = Slice(table_name.data() + *separator_idx + 1,
-                     table_name.size() - *separator_idx - 1);
-  return Status::OK();
-}
-
 Status HmsCatalog::ParseUris(const string& metastore_uris, vector<HostPort>* hostports) {
   hostports->clear();
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/f6eddf28/src/kudu/hms/hms_catalog.h
----------------------------------------------------------------------
diff --git a/src/kudu/hms/hms_catalog.h b/src/kudu/hms/hms_catalog.h
index 6139e28..d0b2a83 100644
--- a/src/kudu/hms/hms_catalog.h
+++ b/src/kudu/hms/hms_catalog.h
@@ -34,7 +34,6 @@ namespace kudu {
 
 class HostPort;
 class Schema;
-class Slice;
 
 namespace hms {
 
@@ -46,8 +45,6 @@ namespace hms {
 class HmsCatalog {
  public:
 
-  static const char* const kInvalidTableError;
-
   explicit HmsCatalog(std::string master_addresses);
   ~HmsCatalog();
 
@@ -172,13 +169,6 @@ class HmsCatalog {
   Status DropTable(const std::string& name,
                    const hive::EnvironmentContext& env_ctx) 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.
-  // The returned HMS database and table slices must not outlive 'table_name'.
-  static Status ParseTableName(const std::string& table_name,
-                               Slice* hms_database,
-                               Slice* hms_table) WARN_UNUSED_RESULT;
-
   // Parses a Hive Metastore URI string into a sequence of HostPorts.
   static Status ParseUris(const std::string& metastore_uris, std::vector<HostPort>* hostports);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/f6eddf28/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 cd148e7..6ea87a9 100644
--- a/src/kudu/integration-tests/master_hms-itest.cc
+++ b/src/kudu/integration-tests/master_hms-itest.cc
@@ -30,9 +30,9 @@
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
 #include "kudu/common/common.pb.h"
+#include "kudu/common/table_util.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/hms/hive_metastore_types.h"
-#include "kudu/hms/hms_catalog.h"
 #include "kudu/hms/hms_client.h"
 #include "kudu/hms/mini_hms.h"
 #include "kudu/integration-tests/external_mini_cluster-itest-base.h"
@@ -306,7 +306,7 @@ TEST_F(MasterHmsTest, TestRenameTable) {
   table_alterer.reset(client_->NewTableAlterer("db.a"));
   s = table_alterer->RenameTo("foo")->Alter();
   ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
-  ASSERT_STR_CONTAINS(s.ToString(), hms::HmsCatalog::kInvalidTableError);
+  ASSERT_STR_CONTAINS(s.ToString(), kInvalidTableError);
 
   // Attempt to rename the Kudu table to a non-existent database.
   table_alterer.reset(client_->NewTableAlterer("db.a"));