You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2016/12/01 23:52:17 UTC

[1/3] kudu git commit: KUDU-981 (part 1): validate identifiers as UTF8 with no null bytes

Repository: kudu
Updated Branches:
  refs/heads/master f165ef7d6 -> 06569f218


KUDU-981 (part 1): validate identifiers as UTF8 with no null bytes

Ensures that table names and identifiers are valid UTF8 with no embedded
null bytes which could cause problems.

KUDU-981 proposes more stringent restrictions such as limiting to only
ASCII. We should collect more info from users before making such a
limitation since it may affect Chinese users, etc. This more lenient
validation is meant more at catching bugs and identifiers that may cause
downstream software to crash.

Change-Id: Ic3b05e2882c9c2ce9b47c16450d9d54a04d3e38b
Reviewed-on: http://gerrit.cloudera.org:8080/5296
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: b28c6dd683bfeddde0876ddb6b228395e0763798
Parents: f165ef7
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Nov 30 17:10:54 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Dec 1 22:48:56 2016 +0000

----------------------------------------------------------------------
 src/kudu/client/client-test.cc     | 41 +++++++++++++++++++++------------
 src/kudu/master/catalog_manager.cc | 17 ++++++++++++++
 2 files changed, 43 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/b28c6dd6/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index caadd24..7827912 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -3856,21 +3856,32 @@ TEST_F(ClientTest, TestCreateTableWithTooManyColumns) {
                       "permitted maximum 300");
 }
 
-TEST_F(ClientTest, TestCreateTableWithTooLongTableName) {
-  const string kLongName(1000, 'x');
-  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
-  KuduSchema schema;
-  KuduSchemaBuilder schema_builder;
-  schema_builder.AddColumn("key")->Type(KuduColumnSchema::INT32)->NotNull()->PrimaryKey();
-  ASSERT_OK(schema_builder.Build(&schema));
-  Status s = table_creator->table_name(kLongName)
-      .schema(&schema)
-      .set_range_partition_columns({ "key" })
-      .Create();
-  ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
-  ASSERT_STR_MATCHES(s.ToString(),
-                     "invalid table name: identifier 'xxx*' "
-                     "longer than maximum permitted length 256");
+TEST_F(ClientTest, TestCreateTable_TableNames) {
+  const vector<pair<string, string>> kCases = {
+    {string(1000, 'x'), "longer than maximum permitted length"},
+    {string("foo\0bar", 7), "invalid table name: identifier must not contain null bytes"},
+    // From http://stackoverflow.com/questions/1301402/example-invalid-utf8-string
+    {string("foo\xf0\x28\x8c\xbc", 7), "invalid table name: invalid UTF8 sequence"},
+    // Should pass validation but fail due to lack of tablet servers running.
+    {"\u4f60\u597d", "Not enough live tablet servers"}
+  };
+
+  for (const auto& test_case : kCases) {
+    const auto& bad_name = test_case.first;
+    const auto& substr = test_case.second;
+    SCOPED_TRACE(bad_name);
+
+    unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+    KuduSchema schema;
+    KuduSchemaBuilder schema_builder;
+    schema_builder.AddColumn("key")->Type(KuduColumnSchema::INT32)->NotNull()->PrimaryKey();
+    ASSERT_OK(schema_builder.Build(&schema));
+    Status s = table_creator->table_name(bad_name)
+        .schema(&schema)
+        .set_range_partition_columns({ "key" })
+        .Create();
+    ASSERT_STR_CONTAINS(s.ToString(), substr);
+  }
 }
 
 TEST_F(ClientTest, TestCreateTableWithTooLongColumnName) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/b28c6dd6/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 33fcc92..0f3dff3 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -70,6 +70,7 @@
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/sysinfo.h"
+#include "kudu/gutil/utf/utf.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/master/master.h"
 #include "kudu/master/master.pb.h"
@@ -835,6 +836,22 @@ Status ValidateIdentifier(const string& id) {
         id, FLAGS_max_identifier_length));
   }
 
+  // Identifiers should be valid UTF8.
+  const char* p = id.data();
+  int rem = id.size();
+  while (rem > 0) {
+    Rune rune = Runeerror;
+    int rune_len = charntorune(&rune, p, rem);
+    if (rune == Runeerror) {
+      return Status::InvalidArgument("invalid UTF8 sequence");
+    }
+    if (rune == 0) {
+      return Status::InvalidArgument("identifier must not contain null bytes");
+    }
+    rem -= rune_len;
+    p += rune_len;
+  }
+
   return Status::OK();
 }
 


[2/3] kudu git commit: tool: Replace Kudu replica with tablet replica in help text

Posted by ad...@apache.org.
tool: Replace Kudu replica with tablet replica in help text

The help text is clearer when we use the term "tablet replica" instead
of "Kudu replica" or just standalone "replica", especially for new
users.

This does not change the names of the modes or actions.

Change-Id: Iec236fb145cd4b9c7a5027898f9beb9262f5893a
Reviewed-on: http://gerrit.cloudera.org:8080/5310
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/648d69e7
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/648d69e7
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/648d69e7

Branch: refs/heads/master
Commit: 648d69e7e16a54b7ab38c3242732a7fd03e07e74
Parents: b28c6dd
Author: Mike Percy <mp...@apache.org>
Authored: Thu Dec 1 19:54:08 2016 +0000
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Thu Dec 1 23:50:26 2016 +0000

----------------------------------------------------------------------
 src/kudu/tools/kudu-tool-test.cc             | 31 +++++++++++------------
 src/kudu/tools/tool_action_local_replica.cc  | 14 +++++-----
 src/kudu/tools/tool_action_remote_replica.cc | 12 ++++-----
 3 files changed, 28 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/648d69e7/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 176b5a2..5aa34ee 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -258,10 +258,10 @@ TEST_F(ToolTest, TestTopLevelHelp) {
   const vector<string> kTopLevelRegexes = {
       "cluster.*Kudu cluster",
       "fs.*Kudu filesystem",
-      "local_replica.*Kudu replicas",
+      "local_replica.*tablet replicas",
       "master.*Kudu Master",
       "pbc.*protobuf container",
-      "remote_replica.*replicas on a Kudu Tablet Server",
+      "remote_replica.*tablet replicas on a Kudu Tablet Server",
       "table.*Kudu tables",
       "tablet.*Kudu tablets",
       "test.*tests",
@@ -295,11 +295,11 @@ TEST_F(ToolTest, TestModeHelp) {
   }
   {
     const vector<string> kLocalReplicaModeRegexes = {
-        "cmeta.*Operate on a local Kudu replica's consensus",
+        "cmeta.*Operate on a local tablet replica's consensus",
         "dump.*Dump a Kudu filesystem",
-        "copy_from_remote.*Copy a replica",
-        "delete.*Delete Kudu replica from the local filesystem",
-        "list.*Show list of Kudu replicas"
+        "copy_from_remote.*Copy a tablet replica",
+        "delete.*Delete tablet replica from the local filesystem",
+        "list.*Show list of tablet replicas"
     };
     NO_FATALS(RunTestHelp("local_replica", kLocalReplicaModeRegexes));
   }
@@ -311,14 +311,13 @@ TEST_F(ToolTest, TestModeHelp) {
         "wals.*Dump all WAL"
     };
     NO_FATALS(RunTestHelp("local_replica dump", kLocalReplicaDumpModeRegexes));
-
   }
   {
-    const vector<string> kCmetaModeRegexes = {
-        "print_replica_uuids.*Print all replica UUIDs",
-        "rewrite_raft_config.*Rewrite a replica"
+    const vector<string> kLocalReplicaCMetaRegexes = {
+        "print_replica_uuids.*Print all tablet replica peer UUIDs",
+        "rewrite_raft_config.*Rewrite a tablet replica"
     };
-    NO_FATALS(RunTestHelp("local_replica cmeta", kCmetaModeRegexes));
+    NO_FATALS(RunTestHelp("local_replica cmeta", kLocalReplicaCMetaRegexes));
   }
   {
     const vector<string> kClusterModeRegexes = {
@@ -342,11 +341,11 @@ TEST_F(ToolTest, TestModeHelp) {
   }
   {
     const vector<string> kRemoteReplicaModeRegexes = {
-        "check.*Check if all replicas",
-        "copy.*Copy a replica from one Kudu Tablet Server to another",
-        "delete.*Delete a replica",
-        "dump.*Dump the data of a replica",
-        "list.*List all replicas"
+        "check.*Check if all tablet replicas",
+        "copy.*Copy a tablet replica from one Kudu Tablet Server to another",
+        "delete.*Delete a tablet replica",
+        "dump.*Dump the data of a tablet replica",
+        "list.*List all tablet replicas"
     };
     NO_FATALS(RunTestHelp("remote_replica", kRemoteReplicaModeRegexes));
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/648d69e7/src/kudu/tools/tool_action_local_replica.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_local_replica.cc b/src/kudu/tools/tool_action_local_replica.cc
index 21f0e23..47e60aa 100644
--- a/src/kudu/tools/tool_action_local_replica.cc
+++ b/src/kudu/tools/tool_action_local_replica.cc
@@ -696,7 +696,7 @@ unique_ptr<Mode> BuildDumpMode() {
 unique_ptr<Mode> BuildLocalReplicaMode() {
   unique_ptr<Action> print_replica_uuids =
       ActionBuilder("print_replica_uuids", &PrintReplicaUuids)
-      .Description("Print all replica UUIDs found in a "
+      .Description("Print all tablet replica peer UUIDs found in a "
         "tablet's Raft configuration")
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddOptionalParameter("fs_wal_dir")
@@ -705,7 +705,7 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
 
   unique_ptr<Action> rewrite_raft_config =
       ActionBuilder("rewrite_raft_config", &RewriteRaftConfig)
-      .Description("Rewrite a replica's Raft configuration")
+      .Description("Rewrite a tablet replica's Raft configuration")
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredVariadicParameter({
         "peers", "List of peers where each peer is of "
@@ -716,7 +716,7 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
 
   unique_ptr<Mode> cmeta =
       ModeBuilder("cmeta")
-      .Description("Operate on a local Kudu replica's consensus "
+      .Description("Operate on a local tablet replica's consensus "
         "metadata file")
       .AddAction(std::move(print_replica_uuids))
       .AddAction(std::move(rewrite_raft_config))
@@ -724,7 +724,7 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
 
   unique_ptr<Action> copy_from_remote =
       ActionBuilder("copy_from_remote", &CopyFromRemote)
-      .Description("Copy a replica from a remote server")
+      .Description("Copy a tablet replica from a remote server")
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ "source", "Source RPC address of "
         "form hostname:port" })
@@ -734,7 +734,7 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
 
   unique_ptr<Action> list =
       ActionBuilder("list", &ListLocalReplicas)
-      .Description("Show list of Kudu replicas in the local filesystem")
+      .Description("Show list of tablet replicas in the local filesystem")
       .AddOptionalParameter("fs_wal_dir")
       .AddOptionalParameter("fs_data_dirs")
       .AddOptionalParameter("list_detail")
@@ -742,7 +742,7 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
 
   unique_ptr<Action> delete_local_replica =
       ActionBuilder("delete", &DeleteLocalReplica)
-      .Description("Delete Kudu replica from the local filesystem")
+      .Description("Delete tablet replica from the local filesystem")
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddOptionalParameter("fs_wal_dir")
       .AddOptionalParameter("fs_data_dirs")
@@ -750,7 +750,7 @@ unique_ptr<Mode> BuildLocalReplicaMode() {
       .Build();
 
   return ModeBuilder("local_replica")
-      .Description("Operate on local Kudu replicas via the local filesystem")
+      .Description("Operate on local tablet replicas via the local filesystem")
       .AddMode(std::move(cmeta))
       .AddAction(std::move(copy_from_remote))
       .AddAction(std::move(delete_local_replica))

http://git-wip-us.apache.org/repos/asf/kudu/blob/648d69e7/src/kudu/tools/tool_action_remote_replica.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_remote_replica.cc b/src/kudu/tools/tool_action_remote_replica.cc
index 625afb0..ec8c6ea 100644
--- a/src/kudu/tools/tool_action_remote_replica.cc
+++ b/src/kudu/tools/tool_action_remote_replica.cc
@@ -339,13 +339,13 @@ Status CopyReplica(const RunnerContext& context) {
 unique_ptr<Mode> BuildRemoteReplicaMode() {
   unique_ptr<Action> check_replicas =
       ActionBuilder("check", &CheckReplicas)
-      .Description("Check if all replicas on a Kudu Tablet Server are running")
+      .Description("Check if all tablet replicas on a Kudu Tablet Server are running")
       .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .Build();
 
   unique_ptr<Action> copy_replica =
       ActionBuilder("copy", &CopyReplica)
-      .Description("Copy a replica from one Kudu Tablet Server to another")
+      .Description("Copy a tablet replica from one Kudu Tablet Server to another")
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ kSrcAddressArg, kTServerAddressDesc })
       .AddRequiredParameter({ kDstAddressArg, kTServerAddressDesc })
@@ -353,7 +353,7 @@ unique_ptr<Mode> BuildRemoteReplicaMode() {
 
   unique_ptr<Action> delete_replica =
       ActionBuilder("delete", &DeleteReplica)
-      .Description("Delete a replica from a Kudu Tablet Server")
+      .Description("Delete a tablet replica from a Kudu Tablet Server")
       .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .AddRequiredParameter({ kReasonArg, "Reason for deleting the replica" })
@@ -361,19 +361,19 @@ unique_ptr<Mode> BuildRemoteReplicaMode() {
 
   unique_ptr<Action> dump_replica =
       ActionBuilder("dump", &DumpReplica)
-      .Description("Dump the data of a replica on a Kudu Tablet Server")
+      .Description("Dump the data of a tablet replica on a Kudu Tablet Server")
       .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .AddRequiredParameter({ kTabletIdArg, kTabletIdArgDesc })
       .Build();
 
   unique_ptr<Action> list =
       ActionBuilder("list", &ListReplicas)
-      .Description("List all replicas on a Kudu Tablet Server")
+      .Description("List all tablet replicas on a Kudu Tablet Server")
       .AddRequiredParameter({ kTServerAddressArg, kTServerAddressDesc })
       .Build();
 
   return ModeBuilder("remote_replica")
-      .Description("Operate on replicas on a Kudu Tablet Server")
+      .Description("Operate on remote tablet replicas on a Kudu Tablet Server")
       .AddAction(std::move(check_replicas))
       .AddAction(std::move(copy_replica))
       .AddAction(std::move(delete_replica))


[3/3] kudu git commit: tool: Allow hyphens to be used instead of underscores

Posted by ad...@apache.org.
tool: Allow hyphens to be used instead of underscores

This works for both modes and actions.

Change-Id: Ie8bf958ee8841e9624687fb5ae50cc908e3e41c3
Reviewed-on: http://gerrit.cloudera.org:8080/5311
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/06569f21
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/06569f21
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/06569f21

Branch: refs/heads/master
Commit: 06569f218675893948e61b427d7ffe9155c7fffa
Parents: 648d69e
Author: Mike Percy <mp...@apache.org>
Authored: Thu Dec 1 19:45:14 2016 +0000
Committer: Mike Percy <mp...@apache.org>
Committed: Thu Dec 1 23:51:13 2016 +0000

----------------------------------------------------------------------
 src/kudu/tools/kudu-tool-test.cc | 13 +++++++++++++
 src/kudu/tools/tool_main.cc      | 15 +++++++++++++--
 2 files changed, 26 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/06569f21/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 5aa34ee..7d2e62a 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -185,6 +185,7 @@ class ToolTest : public KuduTest {
     ASSERT_OK(s);
   }
 
+  // Run tool with specified arguments, expecting help output.
   void RunTestHelp(const string& arg_str,
                    const vector<string>& regexes,
                    const Status& expected_status = Status::OK()) const {
@@ -318,6 +319,18 @@ TEST_F(ToolTest, TestModeHelp) {
         "rewrite_raft_config.*Rewrite a tablet replica"
     };
     NO_FATALS(RunTestHelp("local_replica cmeta", kLocalReplicaCMetaRegexes));
+    // Try with a hyphen instead of an underscore.
+    NO_FATALS(RunTestHelp("local-replica cmeta", kLocalReplicaCMetaRegexes));
+  }
+  {
+    const vector<string> kLocalReplicaCopyFromRemoteRegexes = {
+        "Copy a tablet replica from a remote server"
+    };
+    NO_FATALS(RunTestHelp("local_replica copy_from_remote --help",
+                          kLocalReplicaCopyFromRemoteRegexes));
+    // Try with hyphens instead of underscores.
+    NO_FATALS(RunTestHelp("local-replica copy-from-remote --help",
+                          kLocalReplicaCopyFromRemoteRegexes));
   }
   {
     const vector<string> kClusterModeRegexes = {

http://git-wip-us.apache.org/repos/asf/kudu/blob/06569f21/src/kudu/tools/tool_main.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_main.cc b/src/kudu/tools/tool_main.cc
index 8ea1dac..59d7f11 100644
--- a/src/kudu/tools/tool_main.cc
+++ b/src/kudu/tools/tool_main.cc
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <algorithm>
 #include <deque>
 #include <iostream>
 #include <memory>
@@ -108,6 +109,12 @@ int DispatchCommand(const vector<Mode*>& chain,
   }
 }
 
+// Replace hyphens with underscores in a string and return a copy.
+string HyphensToUnderscores(string str) {
+  std::replace(str.begin(), str.end(), '-', '_');
+  return str;
+}
+
 int RunTool(int argc, char** argv, bool show_help) {
   unique_ptr<Mode> root = ModeBuilder(argv[0])
     .Description("doesn't matter") // root mode description isn't printed
@@ -135,7 +142,9 @@ int RunTool(int argc, char** argv, bool show_help) {
 
     // Match argument with a mode.
     for (const auto& m : cur->modes()) {
-      if (m->name() == argv[i]) {
+      if (m->name() == argv[i] ||
+          // Allow hyphens in addition to underscores in mode names.
+          m->name() == HyphensToUnderscores(argv[i])) {
         next_mode = m.get();
         break;
       }
@@ -143,7 +152,9 @@ int RunTool(int argc, char** argv, bool show_help) {
 
     // Match argument with an action.
     for (const auto& a : cur->actions()) {
-      if (a->name() == argv[i]) {
+      if (a->name() == argv[i] ||
+          // Allow hyphens in addition to underscores in action names.
+          a->name() == HyphensToUnderscores(argv[i])) {
         next_action = a.get();
         break;
       }