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:44 UTC

[1/4] kudu git commit: Remove column ids from KuduSchema::ToString

Repository: kudu
Updated Branches:
  refs/heads/master 5894af6ff -> 1197eb01a


Remove column ids from KuduSchema::ToString

708d0f736c5a9d60ca02041bdb3b753e11478b6d added a KuduSchema::ToString
method, which calls into Schema::ToString. This is fine except
Schema::ToString includes the column ids when they are set, and that
information is implementation and ought not be shown to the user.

I think the original testing missed this because it never called
KuduSchema::ToString on a KuduSchema built from a Schema that had column
ids. A new test fixes this oversight.

Change-Id: Ib5999c485b63b6bdd4f4778b329f6768680b5193
Reviewed-on: http://gerrit.cloudera.org:8080/11668
Tested-by: Will Berkeley <wd...@gmail.com>
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/df5fc241
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/df5fc241
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/df5fc241

Branch: refs/heads/master
Commit: df5fc241e5e459a0978afe5035eca1a24f66d017
Parents: 5894af6
Author: Will Berkeley <wd...@gmail.org>
Authored: Fri Oct 12 10:11:24 2018 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Fri Oct 12 22:05:03 2018 +0000

----------------------------------------------------------------------
 src/kudu/client/client-unittest.cc | 26 +++++++++++++++++++++++++-
 src/kudu/client/schema.cc          |  3 ++-
 src/kudu/common/schema-test.cc     | 13 ++++++++++++-
 src/kudu/common/schema.cc          |  4 ++--
 src/kudu/common/schema.h           |  9 ++++++++-
 5 files changed, 49 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/df5fc241/src/kudu/client/client-unittest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-unittest.cc b/src/kudu/client/client-unittest.cc
index 3ac2cf9..d2c4866 100644
--- a/src/kudu/client/client-unittest.cc
+++ b/src/kudu/client/client-unittest.cc
@@ -17,6 +17,8 @@
 //
 // Tests for the client which are true unit tests and don't require a cluster, etc.
 
+#include "kudu/client/client.h"
+
 #include <cstddef>
 #include <string>
 #include <vector>
@@ -25,18 +27,22 @@
 #include <boost/function.hpp>
 #include <gtest/gtest.h>
 
-#include "kudu/client/client.h"
 #include "kudu/client/client-internal.h"
+#include "kudu/client/client-test-util.h"
 #include "kudu/client/error_collector.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/value.h"
+#include "kudu/common/common.pb.h"
+#include "kudu/common/schema.h"
 #include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 
 using std::string;
 using std::vector;
+using strings::Substitute;
 using kudu::client::internal::ErrorCollector;
 
 namespace kudu {
@@ -272,5 +278,23 @@ TEST(ClientUnitTest, TestKuduSchemaToString) {
   EXPECT_EQ(schema_str_2, s2.ToString());
 }
 
+TEST(ClientUnitTest, TestKuduSchemaToStringWithColumnIds) {
+  // Build a KuduSchema from a Schema, so that the KuduSchema's internal Schema
+  // has column ids.
+  SchemaBuilder builder;
+  builder.AddKeyColumn("key", DataType::INT32);
+  const auto schema = builder.Build();
+  const auto kudu_schema = KuduSchemaFromSchema(schema);
+
+  // The string version of the KuduSchema should not have column ids, even
+  // though the default string version of the underlying Schema should.
+  EXPECT_EQ(
+      Substitute("Schema [\n\tprimary key (key),\n\t$0:key[int32 NOT NULL]\n]",
+                 schema.column_id(0)),
+      schema.ToString());
+  EXPECT_EQ("Schema [\n\tprimary key (key),\n\tkey[int32 NOT NULL]\n]",
+            kudu_schema.ToString());
+}
+
 } // namespace client
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/df5fc241/src/kudu/client/schema.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/schema.cc b/src/kudu/client/schema.cc
index f250fd0..308e49d 100644
--- a/src/kudu/client/schema.cc
+++ b/src/kudu/client/schema.cc
@@ -738,7 +738,8 @@ void KuduSchema::GetPrimaryKeyColumnIndexes(vector<int>* indexes) const {
 }
 
 string KuduSchema::ToString() const {
-  return schema_ ? schema_->ToString() : "Schema []";
+  return schema_ ? schema_->ToString(Schema::ToStringMode::WITHOUT_COLUMN_IDS)
+                 : "Schema []";
 }
 
 } // namespace client

http://git-wip-us.apache.org/repos/asf/kudu/blob/df5fc241/src/kudu/common/schema-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/schema-test.cc b/src/kudu/common/schema-test.cc
index 87ba59a..3fcbdbb 100644
--- a/src/kudu/common/schema-test.cc
+++ b/src/kudu/common/schema-test.cc
@@ -103,6 +103,18 @@ TEST_F(TestSchema, TestSchema) {
   EXPECT_EQ("uint32 NULLABLE", schema.column(1).TypeToString());
 }
 
+TEST_F(TestSchema, TestSchemaToStringMode) {
+  SchemaBuilder builder;
+  builder.AddKeyColumn("key", DataType::INT32);
+  const auto schema = builder.Build();
+  EXPECT_EQ(
+      Substitute("Schema [\n\tprimary key (key),\n\t$0:key[int32 NOT NULL]\n]",
+                 schema.column_id(0)),
+      schema.ToString());
+  EXPECT_EQ("Schema [\n\tprimary key (key),\n\tkey[int32 NOT NULL]\n]",
+            schema.ToString(Schema::ToStringMode::WITHOUT_COLUMN_IDS));
+}
+
 TEST_F(TestSchema, TestCopyAndMove) {
   auto check_schema = [](const Schema& schema) {
     ASSERT_EQ(sizeof(Slice) + sizeof(uint32_t) + sizeof(int32_t),
@@ -403,7 +415,6 @@ TEST_F(TestSchema, TestProjectRename) {
   ASSERT_EQ(row_projector.projection_defaults()[0], 2);      // non_present schema2
 }
 
-
 // Test that the schema can be used to compare and stringify rows.
 TEST_F(TestSchema, TestRowOperations) {
   Schema schema({ ColumnSchema("col1", STRING),

http://git-wip-us.apache.org/repos/asf/kudu/blob/df5fc241/src/kudu/common/schema.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/schema.cc b/src/kudu/common/schema.cc
index b15652e..dbd9cc7 100644
--- a/src/kudu/common/schema.cc
+++ b/src/kudu/common/schema.cc
@@ -388,7 +388,7 @@ Status Schema::GetMappedReadProjection(const Schema& projection,
   return Status::OK();
 }
 
-string Schema::ToString() const {
+string Schema::ToString(ToStringMode mode) const {
   if (cols_.empty()) return "Schema []";
 
   vector<string> pk_strs;
@@ -397,7 +397,7 @@ string Schema::ToString() const {
   }
 
   vector<string> col_strs;
-  if (has_column_ids()) {
+  if (has_column_ids() && mode != ToStringMode::WITHOUT_COLUMN_IDS) {
     for (int i = 0; i < cols_.size(); ++i) {
       col_strs.push_back(Substitute("$0:$1", col_ids_[i], cols_[i].ToString()));
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/df5fc241/src/kudu/common/schema.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/schema.h b/src/kudu/common/schema.h
index 449f3cd..64283f0 100644
--- a/src/kudu/common/schema.h
+++ b/src/kudu/common/schema.h
@@ -723,9 +723,16 @@ class Schema {
     return Slice(*dst);
   }
 
+  // Enum to configure how a Schema is stringified.
+  enum class ToStringMode {
+    // Include column ids if this instance has them.
+    WITH_COLUMN_IDS,
+    // Do not include column ids.
+    WITHOUT_COLUMN_IDS,
+  };
   // Stringify this Schema. This is not particularly efficient,
   // so should only be used when necessary for output.
-  std::string ToString() const;
+  std::string ToString(ToStringMode mode = ToStringMode::WITH_COLUMN_IDS) const;
 
   // Return true if the schemas have exactly the same set of columns
   // and respective types.


[4/4] kudu git commit: dist_test: collect the correct tmpdir

Posted by aw...@apache.org.
dist_test: collect the correct tmpdir

Over the course of many refactors, we've come to not properly collect
the test tmpdir on failed executions of run_dist_test.py.

I tested this to ensure we are now getting the right tmpdir for C++
tests: http://dist-test.cloudera.org/job?job_id=awong.1539136867.37741

It seems like Java tests will wipe the tmpdir before returning control
to run_dist_test.py, so just stdout and sterr are collected (as was the
case before), though this includes logs from the minicluster:
http://dist-test.cloudera.org/job?job_id=awong.1539137951.43284#

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


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

Branch: refs/heads/master
Commit: 1197eb01a21519c71a19f15d5133485a6aa82ec3
Parents: f6eddf2
Author: Andrew Wong <aw...@cloudera.com>
Authored: Tue Oct 9 19:02:45 2018 -0700
Committer: Andrew Wong <aw...@cloudera.com>
Committed: Tue Oct 16 03:55:40 2018 +0000

----------------------------------------------------------------------
 build-support/run_dist_test.py | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1197eb01/build-support/run_dist_test.py
----------------------------------------------------------------------
diff --git a/build-support/run_dist_test.py b/build-support/run_dist_test.py
index 1edcce2..7982f30 100755
--- a/build-support/run_dist_test.py
+++ b/build-support/run_dist_test.py
@@ -167,6 +167,10 @@ def main():
   stderr = None
   if options.test_language == 'cpp':
     cmd = [os.path.join(ROOT, "build-support/run-test.sh")] + args
+    # Get the grandparent directory of the test executable, which takes the
+    # form "../release/bin/foo-test", so we can get the build directory.
+    relative_build_dir = os.path.dirname(os.path.dirname(args[0]))
+    test_logdir = os.path.abspath(os.path.join(os.getcwd(), relative_build_dir, "test-logs"))
   elif options.test_language == 'java':
     test_logdir = os.path.abspath(os.path.join(ROOT, "build/java/test-logs"))
     if not os.path.exists(test_logdir):
@@ -183,7 +187,7 @@ def main():
   rc = subprocess.call(cmd, env=env, stdout=stdout, stderr=stderr)
 
   if rc != 0 and options.collect_tmpdir:
-    os.system("tar czf %s %s" % (os.path.join(test_dir, "..", "test-logs", "test_tmpdir.tgz"), test_tmpdir))
+    os.system("tar czf %s %s" % (os.path.join(test_logdir, "test_tmpdir.tgz"), test_tmpdir))
   sys.exit(rc)
 
 


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

Posted by aw...@apache.org.
[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"));


[2/4] kudu git commit: [sentry] SentryAction

Posted by aw...@apache.org.
[sentry] SentryAction

This commit adds SentryAction class to represent a Sentry action in
HiveSQL authorizable model. This class provides validation on whether
an action can imply another, which allows a higher-level authorization
provider to determine if an operation on a object should be allowed.

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

Branch: refs/heads/master
Commit: 3570c638681d8f13e890c9c48fef61a566cbbffd
Parents: df5fc24
Author: Hao Hao <ha...@cloudera.com>
Authored: Wed Oct 10 12:05:29 2018 -0700
Committer: Hao Hao <ha...@cloudera.com>
Committed: Tue Oct 16 03:41:04 2018 +0000

----------------------------------------------------------------------
 src/kudu/sentry/CMakeLists.txt        |  2 +
 src/kudu/sentry/sentry_action-test.cc | 89 +++++++++++++++++++++++++++
 src/kudu/sentry/sentry_action.cc      | 96 ++++++++++++++++++++++++++++++
 src/kudu/sentry/sentry_action.h       | 83 ++++++++++++++++++++++++++
 4 files changed, 270 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/3570c638/src/kudu/sentry/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/sentry/CMakeLists.txt b/src/kudu/sentry/CMakeLists.txt
index 588bf9f..3222793 100644
--- a/src/kudu/sentry/CMakeLists.txt
+++ b/src/kudu/sentry/CMakeLists.txt
@@ -32,6 +32,7 @@ add_dependencies(sentry_thrift ${SENTRY_THRIFT_TGTS})
 ##############################
 
 set(SENTRY_SRCS
+  sentry_action.cc
   sentry_client.cc)
 set(SENTRY_DEPS
   kudu_common
@@ -75,5 +76,6 @@ if (NOT NO_TESTS)
     mini_kdc
     mini_sentry)
 
+  ADD_KUDU_TEST(sentry_action-test)
   ADD_KUDU_TEST(sentry_client-test)
 endif()

http://git-wip-us.apache.org/repos/asf/kudu/blob/3570c638/src/kudu/sentry/sentry_action-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/sentry/sentry_action-test.cc b/src/kudu/sentry/sentry_action-test.cc
new file mode 100644
index 0000000..d646cc4
--- /dev/null
+++ b/src/kudu/sentry/sentry_action-test.cc
@@ -0,0 +1,89 @@
+// 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/sentry/sentry_action.h"
+
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+
+namespace sentry {
+
+TEST(SentryActionTest, TestImplyAction) {
+  SentryAction all(SentryAction::Action::ALL);
+  SentryAction metadata(SentryAction::Action::METADATA);
+  SentryAction select(SentryAction::Action::SELECT);
+  SentryAction insert(SentryAction::Action::INSERT);
+  SentryAction update(SentryAction::Action::UPDATE);
+  SentryAction del(SentryAction::Action::DELETE);
+  SentryAction alter(SentryAction::Action::ALTER);
+  SentryAction create(SentryAction::Action::CREATE);
+  SentryAction drop(SentryAction::Action::DROP);
+  SentryAction owner(SentryAction::Action::OWNER);
+
+  // Different action cannot imply each other.
+  ASSERT_FALSE(insert.Imply(select));
+  ASSERT_FALSE(select.Imply(insert));
+
+  vector<SentryAction> actions({ all, select, insert, update,
+                                 del, alter, create, drop, owner });
+
+  // Any action subsumes METADATA, not vice versa.
+  for (const auto& action : actions) {
+    ASSERT_TRUE(action.Imply(metadata));
+    ASSERT_FALSE(metadata.Imply(action));
+  }
+
+  actions.insert(actions.end(), metadata);
+  for (const auto& action : actions) {
+    // Action ALL implies all other actions.
+    ASSERT_TRUE(all.Imply(action));
+
+    // Action OWNER equals to ALL, which implies all other actions.
+    ASSERT_TRUE(owner.Imply(action));
+
+    // Any action implies itself.
+    ASSERT_TRUE(action.Imply(action));
+  }
+}
+
+TEST(SentryActionTest, TestFromString) {
+  // Action '*' equals to ALL.
+  SentryAction wildcard_action;
+  ASSERT_OK(wildcard_action.FromString(SentryAction::kWildCard));
+  SentryAction wildcard(wildcard_action);
+  SentryAction all(SentryAction::Action::ALL);
+  ASSERT_TRUE(all.Imply(wildcard));
+  ASSERT_TRUE(wildcard.Imply(all));
+
+  // Unsupported action, such as '+', throws invalid argument error.
+  SentryAction invalid_action;
+  Status s = invalid_action.FromString("+");
+  ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+}
+
+} // namespace sentry
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/3570c638/src/kudu/sentry/sentry_action.cc
----------------------------------------------------------------------
diff --git a/src/kudu/sentry/sentry_action.cc b/src/kudu/sentry/sentry_action.cc
new file mode 100644
index 0000000..2fe0a2a
--- /dev/null
+++ b/src/kudu/sentry/sentry_action.cc
@@ -0,0 +1,96 @@
+// 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/sentry/sentry_action.h"
+
+#include <string>
+
+#include <boost/algorithm/string/predicate.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+namespace sentry {
+
+const char* const SentryAction::kWildCard = "*";
+
+SentryAction::SentryAction()
+  : action_(Action::UNINITIALIZED) {
+}
+
+SentryAction::SentryAction(Action action)
+  : action_(action) {
+}
+
+Status SentryAction::FromString(const string& action) {
+  // Consider action '*' equals to ALL to be compatible with the existing
+  // Java Sentry client.
+  //
+  // See org.apache.sentry.api.service.thrift.SentryPolicyServiceClientDefaultImpl.
+  if (boost::iequals(action, "ALL") || action == kWildCard) {
+    action_ = Action::ALL;
+  } else if (boost::iequals(action, "METADATA")) {
+    action_ = Action::METADATA;
+  } else if (boost::iequals(action, "SELECT")) {
+    action_ = Action::SELECT;
+  } else if (boost::iequals(action, "INSERT")) {
+    action_ = Action::INSERT;
+  } else if (boost::iequals(action, "UPDATE")) {
+    action_ = Action::UPDATE;
+  } else if (boost::iequals(action, "DELETE")) {
+    action_ = Action::DELETE;
+  } else if (boost::iequals(action, "ALTER")) {
+    action_ = Action::ALTER;
+  } else if (boost::iequals(action, "CREATE")) {
+    action_ = Action::CREATE;
+  } else if (boost::iequals(action, "DROP")) {
+    action_ = Action::DROP;
+  } else if (boost::iequals(action, "OWNER")) {
+    action_ = Action::OWNER;
+  } else {
+    return Status::InvalidArgument(Substitute("unknown SentryAction: $0",
+                                              action));
+  }
+
+  return Status::OK();
+}
+
+bool SentryAction::Imply(const SentryAction& other) const {
+  // Any action must be initialized.
+  CHECK(action() != Action::UNINITIALIZED);
+  CHECK(other.action() != Action::UNINITIALIZED);
+
+  // Action ALL and OWNER subsume every other action.
+  if (action() == Action::ALL ||
+      action() == Action::OWNER) {
+    return true;
+  }
+
+  // Any action subsumes Action METADATA
+  if (other.action() == Action::METADATA) {
+    return true;
+  }
+
+  return action() == other.action();
+}
+
+} // namespace sentry
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/3570c638/src/kudu/sentry/sentry_action.h
----------------------------------------------------------------------
diff --git a/src/kudu/sentry/sentry_action.h b/src/kudu/sentry/sentry_action.h
new file mode 100644
index 0000000..957ac56
--- /dev/null
+++ b/src/kudu/sentry/sentry_action.h
@@ -0,0 +1,83 @@
+// 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/util/status.h"
+
+namespace kudu {
+namespace sentry {
+
+// A replication of Sentry Action, which is the operation taken
+// on an authorizable/object. In this case, HiveSQL model is chosen
+// to define the actions. One action can imply another following rules
+// defined in Imply().
+//
+// This class is not thread-safe.
+class SentryAction {
+ public:
+  static const char* const kWildCard;
+
+  // Actions that are supported. All actions are independent,
+  // except that ALL subsumes every other action, and every
+  // action subsumes METADATA. OWNER is a special action that
+  // behaves like the ALL.
+  // Note that 'UNINITIALIZED' is not an actual operation but
+  // only to represent an action in uninitialized state.
+  //
+  // See org.apache.sentry.core.model.db.HiveActionFactory.
+  enum class Action {
+    UNINITIALIZED,
+    ALL,
+    METADATA,
+    SELECT,
+    INSERT,
+    UPDATE,
+    DELETE,
+    ALTER,
+    CREATE,
+    DROP,
+    OWNER,
+  };
+
+  SentryAction();
+
+  explicit SentryAction(Action action);
+
+  Action action() const {
+    return action_;
+  }
+
+  // Create an Action from string.
+  Status FromString(const std::string& action);
+
+  // Check if an action implies the other. In general,
+  //   1. an action only implies itself.
+  //   2. with the exceptions that ALL, OWNER imply all other actions,
+  //      and any action implies METADATA.
+  //
+  // See org.apache.sentry.policy.common.CommonPrivilege.impliesAction.
+  bool Imply(const SentryAction& other) const;
+
+ private:
+  Action action_;
+};
+
+} // namespace sentry
+} // namespace kudu