You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2022/09/07 20:06:23 UTC

[kudu] branch master updated: [tool] Add output format for 'table list' CLI tool.

This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new 2a4b48bc7 [tool] Add output format for 'table list' CLI tool.
2a4b48bc7 is described below

commit 2a4b48bc7cd8a236fdce841e15f1d48941be8af8
Author: kedeng <kd...@gmail.com>
AuthorDate: Thu Aug 11 14:24:20 2022 +0800

    [tool] Add output format for 'table list' CLI tool.
    
    The output format could be one of 'json', 'json_compact' or 'pretty'.
    The default format is 'pretty'. With this patch, we can get the table
    info more convenient for automatic processing in case of large-scale cluster.
    
    The command with 'list_table_output' looks like:
     `kudu table list <master_addresses> [-tables=<tables>] [-list_tablets] [-show_table_info] [-negotiation_timeout_ms=<ms>] [-timeout_ms=<ms>] [-list_table_output_format=<pretty|json|json_compact>]`
    
    The 'pretty' format output looks like:
    `
    TestTableListPartition num_tablets:4 num_replicas:1 live_row_count:0
      T 390c65375d13434393d7152690e80082 : HASH (key_hash0) PARTITION 0, RANGE (key_range) PARTITION 0 <= VALUES < 1
        L 7b12b3459e2943a981d22aa70b915637 tserver1:7050
      T 7cff0635ec7446f883b6b2ed9c4bf7be : HASH (key_hash0) PARTITION 0, RANGE (key_range) PARTITION 2 <= VALUES < 3
        L 7b12b3459e2943a981d22aa70b915637 tserver1:7050
      T 45c347f165a945bd9f54f60758bb8953 : HASH (key_hash0) PARTITION 1, RANGE (key_range) PARTITION 0 <= VALUES < 1
        L 7b12b3459e2943a981d22aa70b915637 tserver1:7050
      T 3eed6166dcfa4e1d94e4c1ceb3f4f6eb : HASH (key_hash0) PARTITION 1, RANGE (key_range) PARTITION 2 <= VALUES < 3
        L 7b12b3459e2943a981d22aa70b915637 tserver1:7050
    
    `
    
    The 'json' format output looks like:
    `
    {
        "tables": [
            {
                "name": "TestTableListPartition",
                "num_tablets": 4,
                "num_replicas": 1,
                "live_row_count": 0,
                "tablet_with_partition": [
                    {
                        "tablet_id": "9db571acf0ca42baac6a70664b787556",
                        "partition_info": "HASH (key_hash0) PARTITION 0, RANGE (key_range) PARTITION 0 <= VALUES < 1",
                        "replica_info": [
                            {
                                "role": "L",
                                "uuid": "f103342fc91245b18f098d34b4f422ac",
                                "host_port": "tserver1:7050"
                            }
                        ]
                    },
                    {
                        "tablet_id": "131aef39a8624ff5a6c47799e0556b52",
                        "partition_info": "HASH (key_hash0) PARTITION 0, RANGE (key_range) PARTITION 2 <= VALUES < 3",
                        "replica_info": [
                            {
                                "role": "L",
                                "uuid": "f103342fc91245b18f098d34b4f422ac",
                                "host_port": "tserver1:7050"
                            }
                        ]
                    },
                    {
                        "tablet_id": "dbaef222684040f5bec60e5cc36df7d6",
                        "partition_info": "HASH (key_hash0) PARTITION 1, RANGE (key_range) PARTITION 0 <= VALUES < 1",
                        "replica_info": [
                            {
                                "role": "L",
                                "uuid": "f103342fc91245b18f098d34b4f422ac",
                                "host_port": "tserver1:7050"
                            }
                        ]
                    },
                    {
                        "tablet_id": "f09ca0749c2641e988c02e8436e655d3",
                        "partition_info": "HASH (key_hash0) PARTITION 1, RANGE (key_range) PARTITION 2 <= VALUES < 3",
                        "replica_info": [
                            {
                                "role": "L",
                                "uuid": "f103342fc91245b18f098d34b4f422ac",
                                "host_port": "tserver1:7050"
                            }
                        ]
                    }
                ]
            }
        ]
    }
    `
    
    The 'json_compact' format output looks like:
    `
    {"tables":[{"name":"TestTableListPartition","num_tablets":4,"num_replicas":1,"live_row_count":0,"tablet_with_partition":[{"tablet_id":"5fc45322b2ec4cd0be050ea1814349e6","partition_info": "HASH (key_hash0) PARTITION 0, RANGE (key_range) PARTITION 0 <= VALUES < 1","replica_info":[{"role":"L","uuid":"e714a8155d5649a98a30af3ec3a6f0e5","host_port":"tserver1:7050"}]},{"tablet_id":"c4279b3929084c0697e2922d351ab369","partition_info": "HASH (key_hash0) PARTITION 0, RANGE (key_range) PARTITION  [...]
    `
    
    Change-Id: I3aaec73e18872fc93646e9c0ea675b578b0702f0
    Reviewed-on: http://gerrit.cloudera.org:8080/18834
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <al...@apache.org>
---
 src/kudu/tools/kudu-admin-test.cc   | 214 +++++++++++++++++++++++++++++-------
 src/kudu/tools/kudu-tool-test.cc    |  16 ++-
 src/kudu/tools/tool.proto           |  24 ++++
 src/kudu/tools/tool_action_table.cc | 106 +++++++++++++++---
 4 files changed, 302 insertions(+), 58 deletions(-)

diff --git a/src/kudu/tools/kudu-admin-test.cc b/src/kudu/tools/kudu-admin-test.cc
index 3dec5ebd3..215ea1be9 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -1661,7 +1661,7 @@ TEST_F(AdminCliTest, TestDeleteTable) {
   ASSERT_TRUE(tables.empty());
 }
 
-TEST_F(AdminCliTest, TestListTables) {
+TEST_F(AdminCliTest, TestListSoftDeletedTables) {
   FLAGS_num_tablet_servers = 1;
   FLAGS_num_replicas = 1;
 
@@ -1669,15 +1669,14 @@ TEST_F(AdminCliTest, TestListTables) {
 
   {
     string stdout;
-    ASSERT_OK(RunKuduTool({
+    string stderr;
+    Status s = RunKuduTool({
       "table",
       "list",
-      cluster_->master()->bound_rpc_addr().ToString()
-    }, &stdout));
-
-    vector<string> stdout_lines = Split(stdout, ",", strings::SkipEmpty());
-    ASSERT_EQ(1, stdout_lines.size());
-    ASSERT_EQ(Substitute("$0\n", kTableId), stdout_lines[0]);
+      cluster_->master()->bound_rpc_addr().ToString(),
+    }, &stdout, &stderr);
+    ASSERT_TRUE(s.ok()) << ToolRunInfo(s, stdout, stderr);
+    ASSERT_STR_CONTAINS(stdout, kTableId);
   }
 
   {
@@ -1690,7 +1689,8 @@ TEST_F(AdminCliTest, TestListTables) {
     }, &stdout));
 
     vector<string> stdout_lines = Split(stdout, ",", strings::SkipEmpty());
-    ASSERT_EQ(0, stdout_lines.size());
+    ASSERT_EQ(1, stdout_lines.size());
+    ASSERT_EQ("\n", stdout_lines[0]);
 
     ASSERT_OK(RunKuduTool({
       "table",
@@ -1701,21 +1701,68 @@ TEST_F(AdminCliTest, TestListTables) {
     }, &stdout));
 
     stdout.clear();
-    ASSERT_OK(RunKuduTool({
+    string stderr;
+    Status s = RunKuduTool({
       "table",
       "list",
       "-soft_deleted_only=true",
-      cluster_->master()->bound_rpc_addr().ToString()
-    }, &stdout));
+      cluster_->master()->bound_rpc_addr().ToString(),
+    }, &stdout, &stderr);
+    ASSERT_TRUE(s.ok()) << ToolRunInfo(s, stdout, stderr);
+    ASSERT_STR_CONTAINS(stdout, kTableId);
+  }
+}
 
-    stdout_lines.clear();
-    stdout_lines = Split(stdout, ",", strings::SkipEmpty());
+static vector<string> TableListFormat() {
+  return { "pretty", "json", "json_compact" };
+}
+
+class ListTableCliSimpleParamTest :
+    public AdminCliTest,
+    public ::testing::WithParamInterface<string> {
+};
+
+INSTANTIATE_TEST_SUITE_P(, ListTableCliSimpleParamTest,
+                         ::testing::ValuesIn(TableListFormat()));
+
+TEST_P(ListTableCliSimpleParamTest, TestListTables) {
+  FLAGS_num_tablet_servers = 1;
+  FLAGS_num_replicas = 1;
+
+  NO_FATALS(BuildAndStart());
+
+  string stdout;
+  string stderr;
+  Status s = RunKuduTool({
+    "table",
+    "list",
+    Substitute("--list_table_output_format=$0", GetParam()),
+    cluster_->master()->bound_rpc_addr().ToString(),
+  }, &stdout, &stderr);
+  ASSERT_TRUE(s.ok()) << ToolRunInfo(s, stdout, stderr);
+
+  if (GetParam() == "pretty") {
+    vector<string> stdout_lines = Split(stdout, ",", strings::SkipEmpty());
     ASSERT_EQ(1, stdout_lines.size());
-    ASSERT_EQ(Substitute("$0\n", kTableId), stdout_lines[0]);
+    ASSERT_STR_CONTAINS(stdout, Substitute("$0\n", kTableId));
+  } else if (GetParam() == "json") {
+    ASSERT_STR_CONTAINS(stdout, Substitute("\"name\": \"$0\"", kTableId));
+  } else if (GetParam() == "json_compact") {
+    ASSERT_STR_CONTAINS(stdout, Substitute("\"name\":\"$0\"", kTableId));
+  } else {
+    FAIL() << "unexpected table list format" << GetParam();
   }
 }
 
-TEST_F(AdminCliTest, TestListTablesDetail) {
+class ListTableCliDetailParamTest :
+    public AdminCliTest,
+    public ::testing::WithParamInterface<string> {
+};
+
+INSTANTIATE_TEST_SUITE_P(, ListTableCliDetailParamTest,
+                         ::testing::ValuesIn(TableListFormat()));
+
+TEST_P(ListTableCliDetailParamTest, TestListTablesDetail) {
   FLAGS_num_tablet_servers = 3;
   FLAGS_num_replicas = 3;
 
@@ -1739,25 +1786,53 @@ TEST_F(AdminCliTest, TestListTablesDetail) {
                        MonoDelta::FromSeconds(30), &tablet_ids);
 
   string stdout;
-  ASSERT_OK(RunKuduTool({
+  string stderr;
+  Status s = RunKuduTool({
     "table",
     "list",
     "--list_tablets",
-    cluster_->master()->bound_rpc_addr().ToString()
-  }, &stdout));
+    Substitute("--list_table_output_format=$0", GetParam()),
+    cluster_->master()->bound_rpc_addr().ToString(),
+  }, &stdout, &stderr);
+  ASSERT_TRUE(s.ok()) << ToolRunInfo(s, stdout, stderr);
 
-  vector<string> stdout_lines = Split(stdout, "\n", strings::SkipEmpty());
+  if (GetParam() == "pretty") {
+    vector<string> stdout_lines = Split(stdout, "\n", strings::SkipEmpty());
 
-  // Verify multiple tables along with their tablets and replica-uuids.
-  ASSERT_EQ(10, stdout_lines.size());
-  ASSERT_STR_CONTAINS(stdout, kTableId);
-  ASSERT_STR_CONTAINS(stdout, kAnotherTableId);
-  ASSERT_STR_CONTAINS(stdout, tablet_ids.front());
-  ASSERT_STR_CONTAINS(stdout, tablet_ids.back());
+    // Verify multiple tables along with their tablets and replica-uuids.
+    ASSERT_STR_CONTAINS(stdout, kTableId);
+    ASSERT_STR_CONTAINS(stdout, kAnotherTableId);
+    ASSERT_STR_CONTAINS(stdout, tablet_ids.front());
+    ASSERT_STR_CONTAINS(stdout, tablet_ids.back());
 
-  for (auto& ts : tservers) {
-    ASSERT_STR_CONTAINS(stdout, ts->uuid());
-    ASSERT_STR_CONTAINS(stdout, ts->uuid());
+    for (auto& ts : tservers) {
+      ASSERT_STR_CONTAINS(stdout, ts->uuid());
+      ASSERT_STR_CONTAINS(stdout, ts->uuid());
+    }
+  } else if (GetParam() == "json") {
+    // The 'json' format output should contain the table id for the table.
+    ASSERT_STR_CONTAINS(stdout, Substitute("\"name\": \"$0\"", kTableId));
+    ASSERT_STR_CONTAINS(stdout, Substitute("\"name\": \"$0\"", kAnotherTableId));
+    for (auto& tablet_id : tablet_ids) {
+      ASSERT_STR_CONTAINS(stdout, Substitute("\"tablet_id\": \"$0\"", tablet_id));
+    }
+
+    for (auto& ts : tservers) {
+      ASSERT_STR_CONTAINS(stdout, Substitute("\"uuid\": \"$0\"", ts->uuid()));
+    }
+  } else if (GetParam() == "json_compact") {
+    // The 'json_compact' format output should contain the table id for the table.
+    ASSERT_STR_CONTAINS(stdout, Substitute("\"name\":\"$0\"", kTableId));
+    ASSERT_STR_CONTAINS(stdout, Substitute("\"name\":\"$0\"", kAnotherTableId));
+    for (auto& tablet_id : tablet_ids) {
+      ASSERT_STR_CONTAINS(stdout, Substitute("\"tablet_id\":\"$0\"", tablet_id));
+    }
+
+    for (auto& ts : tservers) {
+      ASSERT_STR_CONTAINS(stdout, Substitute("\"uuid\":\"$0\"", ts->uuid()));
+    }
+  } else {
+    FAIL() << "unexpected table list format" << GetParam();
   }
 }
 
@@ -2181,15 +2256,32 @@ TEST_F(AdminCliTest, TestDescribeTableCustomHashSchema) {
                               "    PARTITION 100 <= VALUES < 200");
 }
 
-class ListTableCliParamTest : public AdminCliTest,
-                              public ::testing::WithParamInterface<bool> {
+class ListTableCliParamTest :
+    public AdminCliTest,
+    public ::testing::WithParamInterface<tuple<bool /* show_tablet_partition_info*/,
+                                               bool /* show_hash_partition_info*/,
+                                               string /* output format*/>> {
 };
 
+INSTANTIATE_TEST_SUITE_P(, ListTableCliParamTest,
+    ::testing::Combine(::testing::Bool(),
+                       ::testing::Bool(),
+                       ::testing::ValuesIn(TableListFormat())));
+
 // Basic test that the kudu tool works in the list tablets case.
 TEST_P(ListTableCliParamTest, ListTabletWithPartitionInfo) {
-  const auto show_hp = GetParam() ? PartitionSchema::HashPartitionInfo::SHOW
-                                  : PartitionSchema::HashPartitionInfo::HIDE;
+  const auto show_tp = std::get<0>(GetParam()) ? PartitionSchema::HashPartitionInfo::SHOW :
+      PartitionSchema::HashPartitionInfo::HIDE;
+  const auto show_hp = std::get<1>(GetParam()) ? PartitionSchema::HashPartitionInfo::SHOW :
+      PartitionSchema::HashPartitionInfo::HIDE;
   const auto kTimeout = MonoDelta::FromSeconds(30);
+
+  // This combination of parameters does not meet expectations.
+  if (show_tp == PartitionSchema::HashPartitionInfo::HIDE &&
+      show_hp == PartitionSchema::HashPartitionInfo::SHOW) {
+    return;
+  }
+
   FLAGS_num_tablet_servers = 1;
   FLAGS_num_replicas = 1;
 
@@ -2255,8 +2347,9 @@ TEST_P(ListTableCliParamTest, ListTabletWithPartitionInfo) {
     "table",
     "list",
     "--list_tablets",
-    "--show_tablet_partition_info",
-    Substitute("--show_hash_partition_info=$0", GetParam()),
+    Substitute("--show_tablet_partition_info=$0", std::get<0>(GetParam())),
+    Substitute("--show_hash_partition_info=$0", std::get<1>(GetParam())),
+    Substitute("--list_table_output_format=$0", std::get<2>(GetParam())),
     "--tables",
     kTableName,
     cluster_->master()->bound_rpc_addr().ToString(),
@@ -2278,15 +2371,54 @@ TEST_P(ListTableCliParamTest, ListTabletWithPartitionInfo) {
     for (const auto& pt : table.tablet_with_partition()) {
       Partition partition;
       Partition::FromPB(pt.partition(), &partition);
-      const auto partition_str = partition_schema.PartitionDebugString(
-          partition, schema_internal, show_hp);
-      const auto tablet_with_partition = pt.tablet_id() + " : " + partition_str;
-      ASSERT_STR_CONTAINS(stdout, tablet_with_partition);
+      string partition_str;
+      if (show_tp) {
+        partition_str = " : " + partition_schema.PartitionDebugString(partition,
+                                                                      schema_internal,
+                                                                      show_hp);
+      }
+      string tablet_with_partition = pt.tablet_id() + partition_str;
+      if (std::get<2>(GetParam()) == "pretty") {
+        ASSERT_STR_CONTAINS(stdout, tablet_with_partition);
+      }
+    }
+  }
+
+  if (std::get<2>(GetParam()) == "pretty") {
+    ASSERT_STR_CONTAINS(stdout, Substitute("$0", kTableName));
+
+    for (auto& tablet_id : delta_tablet_ids) {
+      ASSERT_STR_CONTAINS(stdout, tablet_id);
+    }
+
+    for (auto& ts : tservers) {
+      ASSERT_STR_CONTAINS(stdout, ts->uuid());
+      ASSERT_STR_CONTAINS(stdout, ts->uuid());
+    }
+  } else if (std::get<2>(GetParam()) == "json") {
+    ASSERT_STR_CONTAINS(stdout, Substitute("\"name\": \"$0\"", kTableName));
+
+    for (auto& tablet_id : delta_tablet_ids) {
+      ASSERT_STR_CONTAINS(stdout, Substitute("\"tablet_id\": \"$0\"", tablet_id));
+    }
+
+    for (auto& ts : tservers) {
+      ASSERT_STR_CONTAINS(stdout, Substitute("\"uuid\": \"$0\"", ts->uuid()));
     }
+  } else if (std::get<2>(GetParam()) == "json_compact") {
+    ASSERT_STR_CONTAINS(stdout, Substitute("\"name\":\"$0\"", kTableName));
+
+    for (auto& tablet_id : delta_tablet_ids) {
+      ASSERT_STR_CONTAINS(stdout, Substitute("\"tablet_id\":\"$0\"", tablet_id));
+    }
+
+    for (auto& ts : tservers) {
+      ASSERT_STR_CONTAINS(stdout, Substitute("\"uuid\":\"$0\"", ts->uuid()));
+    }
+  } else {
+    FAIL() << "unexpected table list format" << std::get<2>(GetParam());
   }
 }
-INSTANTIATE_TEST_SUITE_P(IsHashPartitionInfoShown, ListTableCliParamTest,
-                         ::testing::Bool());
 
 TEST_F(AdminCliTest, TestLocateRow) {
   FLAGS_num_tablet_servers = 1;
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index ab0417ad9..da9dcdb17 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -3280,6 +3280,7 @@ TEST_F(ToolTest, TestLoadgenKeepAutoTableAndData) {
   NO_FATALS(RunLoadgen(1, { "--keep_auto_table=true",
                             "--table_num_hash_partitions=1",
                             "--table_num_range_partitions=1" }));
+
   string auto_table_name;
   NO_FATALS(RunActionStdoutString(Substitute("table list $0",
       HostPort::ToCommaSeparatedString(cluster_->master_rpc_addrs())), &auto_table_name));
@@ -4732,12 +4733,14 @@ TEST_F(ToolTest, TestListTables) {
     if (kNumTables != num) {
       filter = Substitute("--tables=$0", JoinStrings(expected, ","));
     }
+
     vector<string> lines;
     NO_FATALS(RunActionStdoutLines(
         Substitute("table list $0 $1", master_addr, filter), &lines));
 
-    std::sort(lines.begin(), lines.end());
-    ASSERT_EQ(expected, lines);
+    for (auto& e : expected) {
+      ASSERT_STRINGS_ANY_MATCH(lines, e);
+    }
   };
 
   const auto ProcessTablets = [&] (const int num) {
@@ -4758,6 +4761,10 @@ TEST_F(ToolTest, TestListTables) {
       if (lines[i].empty()) {
         continue;
       }
+      if (MatchPattern(lines[i], "kudu.table_")) {
+        continue;
+      }
+      lines[i].erase(0, lines[i].find_first_not_of(' '));
       ASSERT_LE(i + 2, lines.size());
       output[lines[i]] = pair<string, string>(lines[i + 1], lines[i + 2]);
       i += 2;
@@ -4801,8 +4808,9 @@ TEST_F(ToolTest, TestListTables) {
     NO_FATALS(RunActionStdoutLines(
         Substitute("table list $0 $1 --show_table_info", master_addr, filter), &lines));
 
-    std::sort(lines.begin(), lines.end());
-    ASSERT_EQ(expected, lines);
+    for (auto& e : expected) {
+      ASSERT_STRINGS_ANY_MATCH(lines, e);
+    }
   };
 
   // List the tables and tablets.
diff --git a/src/kudu/tools/tool.proto b/src/kudu/tools/tool.proto
index f80e462ba..465c5bbf9 100644
--- a/src/kudu/tools/tool.proto
+++ b/src/kudu/tools/tool.proto
@@ -499,3 +499,27 @@ message CreateTablePB {
   // Table's comment.
   optional string comment = 8;
 }
+
+message TablesInfoPB {
+  message ReplicaInfoPB {
+    optional string role = 1;
+    optional string uuid = 2;
+    optional string host_port = 3;
+  }
+
+  message TabletWithPartitionPB {
+    optional string tablet_id = 1;
+    optional string partition_info = 2;
+    repeated ReplicaInfoPB replica_info = 3;
+  }
+
+  message TableInfoPB {
+    optional string name = 1;
+    optional int32 num_tablets = 2;
+    optional int32 num_replicas = 3;
+    optional uint64 live_row_count = 4;
+    repeated TabletWithPartitionPB tablet_with_partition = 5;
+  }
+
+  repeated TableInfoPB tables = 1;
+}
diff --git a/src/kudu/tools/tool_action_table.cc b/src/kudu/tools/tool_action_table.cc
index 85ff8f417..0c578d58a 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -85,12 +85,15 @@ using kudu::client::KuduTableCreator;
 using kudu::client::KuduTableStatistics;
 using kudu::client::KuduValue;
 using kudu::client::internal::ReplicaController;
+using kudu::iequals;
 using kudu::tools::PartitionPB;
 using std::cerr;
 using std::cout;
 using std::endl;
+using std::find_if;
 using std::make_unique;
 using std::map;
+using std::ostringstream;
 using std::pair;
 using std::set;
 using std::string;
@@ -146,6 +149,30 @@ DEFINE_bool(soft_deleted_only, false,
 DEFINE_string(new_table_name, "",
               "The new name for the recalled table. "
               "Leave empty to recall the table under its original name.");
+DEFINE_string(list_table_output_format, "pretty",
+              "One of 'json', 'json_compact' or 'pretty'. Pretty output flattens "
+              "the table list hierarchy.");
+
+static bool ValidateListTableOutput(const char* flag_name,
+                                    const string& flag_value) {
+  const vector<string> allowed_values = { "pretty", "json", "json_compact" };
+  if (find_if(allowed_values.begin(), allowed_values.end(),
+                   [&](const string& allowed_value) {
+                     return iequals(allowed_value, flag_value);
+                   }) != allowed_values.end()) {
+    return true;
+  }
+
+  LOG(ERROR) << Substitute("'$0' : unsupported value for -- $1 flag; "
+                           "should be one of $2.",
+                           flag_value, flag_name,
+                           JoinStrings(allowed_values, " "));
+
+  return false;
+}
+
+DEFINE_validator(list_table_output_format, ValidateListTableOutput);
+
 DEFINE_bool(modify_external_catalogs, true,
             "Whether to modify external catalogs, such as the Hive Metastore, "
             "when renaming or dropping a table.");
@@ -233,6 +260,38 @@ class TableLister {
     return pinfo;
   }
 
+  static string ToPrettyFormat(const TablesInfoPB& tables_info) {
+    string output;
+    for (const auto& table_info : tables_info.tables()) {
+      if (!output.empty()) {
+        output.append("\n");
+      }
+
+      output += table_info.name();
+      if (table_info.has_num_tablets()) {
+        output += Substitute(" num_tablets:$0 num_replicas:$1 live_row_count:$2",
+                             table_info.num_tablets(),
+                             table_info.num_replicas(),
+                             table_info.live_row_count());
+      }
+
+      for (const auto& tablet_info : table_info.tablet_with_partition()) {
+        output += string("\n") + string("  T ") + tablet_info.tablet_id();
+        if (tablet_info.has_partition_info()) {
+          output += tablet_info.partition_info();
+        }
+
+        for (const auto& replica_info : tablet_info.replica_info()) {
+          output += Substitute("\n    $0 $1 $2",
+                               replica_info.role(),
+                               replica_info.uuid(),
+                               replica_info.host_port());
+        }
+      }
+    }
+    return output;
+  }
+
   static Status ListTablets(const vector<string>& master_addresses) {
     client::sp::shared_ptr<KuduClient> client;
     RETURN_NOT_OK(CreateKuduClient(master_addresses,
@@ -246,15 +305,17 @@ class TableLister {
                                           FLAGS_show_tablet_partition_info,
                                           FLAGS_soft_deleted_only));
     vector<string> table_filters = Split(FLAGS_tables, ",", strings::SkipEmpty());
+    TablesInfoPB tables_info_pb;
     for (const auto& tinfo : tables_info) {
       const auto& tname = tinfo.table_name;
       if (!MatchesAnyPattern(table_filters, tname)) continue;
+
+      TablesInfoPB::TableInfoPB* table_info_pb = tables_info_pb.add_tables();
+      table_info_pb->set_name(tname);
       if (FLAGS_show_table_info) {
-        cout << tname << " " << "num_tablets:" << tinfo.num_tablets
-             << " num_replicas:" << tinfo.num_replicas
-             << " live_row_count:" << tinfo.live_row_count << endl;
-      } else {
-        cout << tname << endl;
+        table_info_pb->set_num_tablets(tinfo.num_tablets);
+        table_info_pb->set_num_replicas(tinfo.num_replicas);
+        table_info_pb->set_live_row_count(tinfo.live_row_count);
       }
 
       if (!FLAGS_list_tablets) {
@@ -272,19 +333,37 @@ class TableLister {
         string partition_info;
         string tablet_id = token->tablet().id();
         if (FLAGS_show_tablet_partition_info) {
-          partition_info = " : " + SearchPartitionInfo(tinfo, client_table, tablet_id);
+          if (iequals(FLAGS_list_table_output_format, "pretty")) {
+            partition_info = " : " + SearchPartitionInfo(tinfo, client_table, tablet_id);
+          } else {
+            partition_info = SearchPartitionInfo(tinfo, client_table, tablet_id);
+          }
         }
-        cout << "  T " << tablet_id << partition_info << endl;
+
+        TablesInfoPB::TabletWithPartitionPB* tpinfo = table_info_pb->add_tablet_with_partition();
+        tpinfo->set_tablet_id(tablet_id);
+        tpinfo->set_partition_info(partition_info);
+
         for (const auto* replica : token->tablet().replicas()) {
           const bool is_voter = ReplicaController::is_voter(*replica);
           const bool is_leader = replica->is_leader();
-          cout << Substitute("    $0 $1 $2:$3",
-              is_leader ? "L" : (is_voter ? "V" : "N"), replica->ts().uuid(),
-              replica->ts().hostname(), replica->ts().port()) << endl;
+          TablesInfoPB::ReplicaInfoPB* rinfo = tpinfo->add_replica_info();
+          rinfo->set_role(is_leader ? "L" : (is_voter ? "V" : "N"));
+          rinfo->set_uuid(replica->ts().uuid());
+          rinfo->set_host_port(replica->ts().hostname() + ":" +
+              std::to_string(replica->ts().port()));
         }
-        cout << endl;
       }
-      cout << endl;
+    }
+
+    if (iequals(FLAGS_list_table_output_format, "pretty")) {
+      cout << ToPrettyFormat(tables_info_pb) << endl;
+    } else {
+      DCHECK(iequals(FLAGS_list_table_output_format, "json") ||
+          iequals(FLAGS_list_table_output_format, "json_compact"));
+      auto mode = iequals(FLAGS_list_table_output_format, "json") ?
+          JsonWriter::Mode::PRETTY : JsonWriter::Mode::COMPACT;
+      cout << JsonWriter::ToJson(tables_info_pb, mode) << endl;
     }
 
     return Status::OK();
@@ -414,7 +493,7 @@ Status AddPrimitiveType(const ColumnSchema& col_schema, const string& type, Json
 Status PopulateAvroSchema(const string& table_name,
                           const string& cluster_id,
                           const KuduSchema& kudu_schema) {
-  std::ostringstream out;
+  ostringstream out;
   JsonWriter writer(&out, JsonWriter::Mode::PRETTY);
   // Start writing in Json format
   writer.StartObject();
@@ -1735,6 +1814,7 @@ unique_ptr<Mode> BuildTableMode() {
       .AddOptionalParameter("show_tablet_partition_info")
       .AddOptionalParameter("show_hash_partition_info")
       .AddOptionalParameter("show_table_info")
+      .AddOptionalParameter("list_table_output_format")
       .Build();
 
   unique_ptr<Action> locate_row =