You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by la...@apache.org on 2020/08/15 14:59:28 UTC

[kudu] tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release created (now 4c2c075)

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

laiyingchun pushed a change to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git.


      at 4c2c075  (commit)
This tag includes the following new commits:

     new d5162ad  [metrics] make 'merged_entities_count_of_tablet' visible when only one entity merged
     new 4addeee  [www] convert human-readable int to int when compare numeric strings
     new 7ac9a69  [docs] Add docs for configuring maintenance priority for kudu tables
     new b65012f  [metrics] fix MeanGauge::snapshot()
     new b30742f  [build] do not remove thirdparty archives by default
     new cdb3e18  [master] reserve table for a period time after being deleted
     new d9b29d9  [collector] Add CPP implemented collector
     new e09b284  [script] Add script tools
     new d56a471  [script] fix minos_control_server.py: exit if execute command failed
     new 82b0fb4  [collector] fix metrics filter and merge url for version 1.11
     new 647df1b  [scripts] fix minos_control_server.py for minos2
     new 66681c6  [collector] collect server entity metrics
     new 0abd7ec  [collector] add service monitor
     new baae0c7  [script] add a tool for availability statistics
     new 8cea6f4  [collector] collect more for histogram metrics
     new b7ba14f  [collector] some improvements
     new 7caa5a7  [collector] modify some APIs
     new e6a5431  [version] Bump version to 1.12.0-mdh1.0.0-SNAPSHOT
     new b42d25b  [script] fix minos_control_server.py for newer versions
     new 6db4bae  [tserver] add 'runtime' tag for scanner_ttl_ms
     new 8c81544  [maintenance] use workload statistics to scale perf score of flushes/compactions
     new dac73ac  KUDU-3180: prioritize larger mem-stores in time-based flusing
     new 4c2c075  [version] Bump version to 1.12.0-mdh1.0.0

The 23 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[kudu] 18/23: [version] Bump version to 1.12.0-mdh1.0.0-SNAPSHOT

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit e6a5431ff8f32b85631f667c1977f00ca96014d7
Author: zhangyifan8 <zh...@xiaomi.com>
AuthorDate: Thu May 21 16:47:58 2020 +0800

    [version] Bump version to 1.12.0-mdh1.0.0-SNAPSHOT
---
 version.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/version.txt b/version.txt
index 8f34f8f..c19d3ed 100644
--- a/version.txt
+++ b/version.txt
@@ -1 +1 @@
-1.12.1-SNAPSHOT
+1.12.0-mdh1.0.0-SNAPSHOT


[kudu] 21/23: [maintenance] use workload statistics to scale perf score of flushes/compactions

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 8c815447d068d54383115e06ee596a38da99891d
Author: zhangyifan27 <ch...@163.com>
AuthorDate: Mon Jun 15 17:05:22 2020 +0800

    [maintenance] use workload statistics to scale perf score of flushes/compactions
    
    When we consider the performance improvement brought by maintenance
    operations, we could use workload statistics to find how 'hot' the
    tablet has been in the last few minutes and perform maintenance ops
    for 'hot' tablets in priority. This patch use recent read/write rate
    of a tablet as a workload score, calculate a final perf score based on
    a op's raw perf_improvement, the tablet's workload score and the table's
    priority, so maintenance ops for a 'hot' tablet are more likely to launch.
    
    In our use case, there is insert/update/delete traffic all the time,
    but some tables may have more read traffic at some time, so we want to
    dynamically adjust priorities of compaction/flush ops for different tables.
    
    We tested this on a 6-node cluster and set tservers with configs:
    -maintenance_manager_num_threads=1,
    -workload_score_upper_bound=10,
    and we run workloads with setting enable_workload_score_for_perf_improvement_ops
    false and true to see whether it can improve performance.
    
    We first insert 5,000,000,000 rows into table-C(256 tablets), and then
    insert 200,000,000 rows into table-A(8 tablets) and table-B(8 tablets)
    at the same time. Next we run different YCSB workloads on table-A and table-B,
    all the tablets have some uncompacted rowsets at this time, but there is
    no on-going workloads on table-C.
    
    workload for table_A: Update heavy workload, scan/update ratio is 50/50
      operationcount=10,000,000
      requestdistribution=zipfian
      maxscanlength=10
    
    workload for table_B: Scan mostly workload, scan/insert ratio is 80/20.
      operationcount=10,000,000
      requestdistribution=zipfian
      maxscanlength=10000
    
    result:
    measurement                                 Before change     After change
    [table-A:UPDATE]AverageLatency(us)          9.46              3.84
    [table-A:UPDATE]95thPercentileLatency(us)   12                7
    [table-A:UPDATE]99thPercentileLatency(us)   19                13
    [table-A:SCAN]AverageLatency(us)            2317              1419
    [table-A:SCAN]95thPercentileLatency(us)     4847              2939
    [table-A:SCAN]99thPercentileLatency(us)     10815             5703
    [table-B:INSERT]AverageLatency(us)          16.11             16.54
    [table-B:INSERT]95thPercentileLatency(us)   35                35
    [table-B:INSERT]99thPercentileLatency(us)   58                56
    [table-B:SCAN]AverageLatency(us)            6417              5545
    [table-B:SCAN]95thPercentileLatency(us)     12463             10063
    [table-B:SCAN]99thPercentileLatency(us)     18095             13511
    
    We run these workloads 5 times, we can see 10%-30% reduction in scan
    latency of table-B, and 38%-60% reduction in scan latency of table-A.
    
    This patch also add 'Workload score' to tserver /maintenance-manager page[1]
    so that we could adjust runtime flags based on current state.
    
    [1] http://ww1.sinaimg.cn/large/9b7ebaddly1gh83qh2bkfj21eb0d3goy.jpg
    
    Change-Id: Ie3afcc359002d1392164ba2fda885f8930ef8696
    Reviewed-on: http://gerrit.cloudera.org:8080/15995
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/tablet/tablet.cc                   | 82 +++++++++++++++++++++++++++--
 src/kudu/tablet/tablet.h                    | 16 ++++--
 src/kudu/tablet/tablet_mm_ops.cc            | 46 +++++++++++-----
 src/kudu/tablet/tablet_replica_mm_ops.cc    | 15 +++++-
 src/kudu/tserver/tablet_server-test-base.cc |  8 ++-
 src/kudu/tserver/tablet_server-test-base.h  |  4 +-
 src/kudu/tserver/tablet_server-test.cc      | 71 +++++++++++++++++++++++++
 src/kudu/tserver/tserver_path_handlers.cc   |  1 +
 src/kudu/util/maintenance_manager-test.cc   | 53 ++++++++++++++-----
 src/kudu/util/maintenance_manager.cc        | 30 +++++++++--
 src/kudu/util/maintenance_manager.h         | 24 ++++++++-
 src/kudu/util/maintenance_manager.proto     |  1 +
 www/maintenance-manager.mustache            |  2 +
 13 files changed, 309 insertions(+), 44 deletions(-)

diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index 7e1abf1..fe3e73b 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -80,7 +80,7 @@
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/locks.h"
 #include "kudu/util/logging.h"
-#include "kudu/util/maintenance_manager.h"
+#include "kudu/util/memory/arena.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/process_memory.h"
@@ -149,6 +149,34 @@ DEFINE_int32(max_encoded_key_size_bytes, 16 * 1024,
              "result in an error.");
 TAG_FLAG(max_encoded_key_size_bytes, unsafe);
 
+DEFINE_int32(workload_stats_rate_collection_min_interval_ms, 60 * 1000,
+             "The minimal interval in milliseconds at which we collect read/write rates.");
+TAG_FLAG(workload_stats_rate_collection_min_interval_ms, experimental);
+TAG_FLAG(workload_stats_rate_collection_min_interval_ms, runtime);
+
+DEFINE_int32(workload_stats_metric_collection_interval_ms, 5 * 60 * 1000,
+             "The interval in milliseconds at which we collect workload metrics.");
+TAG_FLAG(workload_stats_metric_collection_interval_ms, experimental);
+TAG_FLAG(workload_stats_metric_collection_interval_ms, runtime);
+
+DEFINE_double(workload_score_upper_bound, 1.0, "Upper bound for workload score.");
+TAG_FLAG(workload_score_upper_bound, experimental);
+TAG_FLAG(workload_score_upper_bound, runtime);
+
+DEFINE_int32(scans_started_per_sec_for_hot_tablets, 1,
+    "Minimum read rate for tablets to be considered 'hot' (scans/sec). If a tablet's "
+    "read rate exceeds this value, flush/compaction ops for it will be assigned the highest "
+    "possible workload score, which is defined by --workload_score_upper_bound.");
+TAG_FLAG(scans_started_per_sec_for_hot_tablets, experimental);
+TAG_FLAG(scans_started_per_sec_for_hot_tablets, runtime);
+
+DEFINE_int32(rows_writed_per_sec_for_hot_tablets, 1000,
+    "Minimum write rate for tablets to be considered 'hot' (rows/sec). If a tablet's "
+    "write rate exceeds this value, compaction ops for it will be assigned the highest "
+    "possible workload score, which is defined by --workload_score_upper_bound.");
+TAG_FLAG(rows_writed_per_sec_for_hot_tablets, experimental);
+TAG_FLAG(rows_writed_per_sec_for_hot_tablets, runtime);
+
 METRIC_DEFINE_entity(tablet);
 METRIC_DEFINE_gauge_size(tablet, memrowset_size, "MemRowSet Memory Usage",
                          kudu::MetricUnit::kBytes,
@@ -226,7 +254,12 @@ Tablet::Tablet(scoped_refptr<TabletMetadata> metadata,
     rowsets_flush_sem_(1),
     state_(kInitialized),
     last_write_time_(MonoTime::Now()),
-    last_read_time_(MonoTime::Now()) {
+    last_read_time_(MonoTime::Now()),
+    last_update_workload_stats_time_(MonoTime::Now()),
+    last_scans_started_(0),
+    last_rows_mutated_(0),
+    last_read_score_(0.0),
+    last_write_score_(0.0) {
       CHECK(schema()->has_column_ids());
   compaction_policy_.reset(CreateCompactionPolicy());
 
@@ -1823,9 +1856,6 @@ void Tablet::UpdateCompactionStats(MaintenanceOpStats* stats) {
     return;
   }
 
-  // TODO: use workload statistics here to find out how "hot" the tablet has
-  // been in the last 5 minutes, and somehow scale the compaction quality
-  // based on that, so we favor hot tablets.
   double quality = 0;
   unordered_set<const RowSet*> picked_set_ignored;
 
@@ -2027,6 +2057,48 @@ uint64_t Tablet::LastWriteElapsedSeconds() const {
   return static_cast<uint64_t>((MonoTime::Now() - last_write_time_).ToSeconds());
 }
 
+double Tablet::CollectAndUpdateWorkloadStats(MaintenanceOp::PerfImprovementOpType type) {
+  DCHECK(last_update_workload_stats_time_.Initialized());
+  double workload_score = 0;
+  MonoDelta elapse = MonoTime::Now() - last_update_workload_stats_time_;
+  if (metrics_) {
+    int64_t scans_started = metrics_->scans_started->value();
+    int64_t rows_mutated = metrics_->rows_inserted->value() +
+                           metrics_->rows_upserted->value() +
+                           metrics_->rows_updated->value() +
+                           metrics_->rows_deleted->value();
+    if (elapse.ToMilliseconds() > FLAGS_workload_stats_rate_collection_min_interval_ms) {
+      double last_read_rate =
+          static_cast<double>(scans_started - last_scans_started_) / elapse.ToSeconds();
+      last_read_score_ =
+          std::min(1.0, last_read_rate / FLAGS_scans_started_per_sec_for_hot_tablets) *
+          FLAGS_workload_score_upper_bound;
+      double last_write_rate =
+          static_cast<double>(rows_mutated - last_rows_mutated_) / elapse.ToSeconds();
+      last_write_score_ =
+          std::min(1.0, last_write_rate / FLAGS_rows_writed_per_sec_for_hot_tablets) *
+          FLAGS_workload_score_upper_bound;
+    }
+    if (elapse.ToMilliseconds() > FLAGS_workload_stats_metric_collection_interval_ms) {
+      last_update_workload_stats_time_ = MonoTime::Now();
+      last_scans_started_ = metrics_->scans_started->value();
+      last_rows_mutated_ = rows_mutated;
+    }
+  }
+  if (type == MaintenanceOp::FLUSH_OP) {
+    // Flush ops are already scored based on how hot the tablet is
+    // for writes, so we'll only adjust the workload score based on
+    // how hot the tablet is for reads.
+    workload_score = last_read_score_;
+  } else if (type == MaintenanceOp::COMPACT_OP) {
+    // Since compactions may improve both read and write performance, increase
+    // the workload score based on the read and write rate to the tablet.
+    workload_score = std::min(FLAGS_workload_score_upper_bound,
+                              last_read_score_ + last_write_score_);
+  }
+  return workload_score;
+}
+
 size_t Tablet::DeltaMemStoresSize() const {
   scoped_refptr<TabletComponents> comps;
   GetComponents(&comps);
diff --git a/src/kudu/tablet/tablet.h b/src/kudu/tablet/tablet.h
index 6177420..45caf98 100644
--- a/src/kudu/tablet/tablet.h
+++ b/src/kudu/tablet/tablet.h
@@ -44,6 +44,7 @@
 #include "kudu/tablet/tablet_metadata.h"
 #include "kudu/util/bloom_filter.h"
 #include "kudu/util/locks.h"
+#include "kudu/util/maintenance_manager.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/rw_semaphore.h"
@@ -55,9 +56,6 @@ class AlterTableTest;
 class ConstContiguousRow;
 class EncodedKey;
 class KeyRange;
-class MaintenanceManager;
-class MaintenanceOp;
-class MaintenanceOpStats;
 class MemTracker;
 class RowBlock;
 class ScanSpec;
@@ -485,6 +483,12 @@ class Tablet {
   // variable there.
   void UpdateLastReadTime() const;
 
+  // Collect and update recent workload statistics for the tablet.
+  // Return the current workload score of the tablet.
+  //
+  // This method is not thread safe and should only be called from a single thread at once.
+  double CollectAndUpdateWorkloadStats(MaintenanceOp::PerfImprovementOpType type);
+
  private:
   friend class kudu::AlterTableTest;
   friend class Iterator;
@@ -794,6 +798,12 @@ class Tablet {
   // ensures we do not attempt to collect metrics while calling the destructor.
   FunctionGaugeDetacher metric_detacher_;
 
+  MonoTime last_update_workload_stats_time_;
+  int64_t last_scans_started_;
+  int64_t last_rows_mutated_;
+  double last_read_score_;
+  double last_write_score_;
+
   DISALLOW_COPY_AND_ASSIGN(Tablet);
 };
 
diff --git a/src/kudu/tablet/tablet_mm_ops.cc b/src/kudu/tablet/tablet_mm_ops.cc
index c0f3caf..d5c83f5 100644
--- a/src/kudu/tablet/tablet_mm_ops.cc
+++ b/src/kudu/tablet/tablet_mm_ops.cc
@@ -83,6 +83,12 @@ DEFINE_bool(enable_deleted_rowset_gc, true,
     "considered ancient history (see --tablet_history_max_age_sec) are deleted.");
 TAG_FLAG(enable_deleted_rowset_gc, runtime);
 
+DEFINE_bool(enable_workload_score_for_perf_improvement_ops, false,
+            "Whether to enable prioritization of maintenance operations based on "
+            "whether there are on-going workloads, favoring ops of 'hot' tablets.");
+TAG_FLAG(enable_workload_score_for_perf_improvement_ops, experimental);
+TAG_FLAG(enable_workload_score_for_perf_improvement_ops, runtime);
+
 using std::string;
 using strings::Substitute;
 
@@ -128,6 +134,9 @@ void CompactRowSetsOp::UpdateStats(MaintenanceOpStats* stats) {
 
   std::lock_guard<simple_spinlock> l(lock_);
 
+  double workload_score = FLAGS_enable_workload_score_for_perf_improvement_ops ?
+                          tablet_->CollectAndUpdateWorkloadStats(MaintenanceOp::COMPACT_OP) : 0;
+
   // Any operation that changes the on-disk row layout invalidates the
   // cached stats.
   TabletMetrics* metrics = tablet_->metrics();
@@ -137,15 +146,16 @@ void CompactRowSetsOp::UpdateStats(MaintenanceOpStats* stats) {
     if (prev_stats_.valid() &&
         new_num_mrs_flushed == last_num_mrs_flushed_ &&
         new_num_rs_compacted == last_num_rs_compacted_) {
+      prev_stats_.set_workload_score(workload_score);
       *stats = prev_stats_;
       return;
-    } else {
-      last_num_mrs_flushed_ = new_num_mrs_flushed;
-      last_num_rs_compacted_ = new_num_rs_compacted;
     }
+    last_num_mrs_flushed_ = new_num_mrs_flushed;
+    last_num_rs_compacted_ = new_num_rs_compacted;
   }
 
   tablet_->UpdateCompactionStats(&prev_stats_);
+  prev_stats_.set_workload_score(workload_score);
   *stats = prev_stats_;
 }
 
@@ -199,6 +209,9 @@ void MinorDeltaCompactionOp::UpdateStats(MaintenanceOpStats* stats) {
 
   std::lock_guard<simple_spinlock> l(lock_);
 
+  double workload_score = FLAGS_enable_workload_score_for_perf_improvement_ops ?
+                          tablet_->CollectAndUpdateWorkloadStats(MaintenanceOp::COMPACT_OP) : 0;
+
   // Any operation that changes the number of REDO files invalidates the
   // cached stats.
   TabletMetrics* metrics = tablet_->metrics();
@@ -213,20 +226,21 @@ void MinorDeltaCompactionOp::UpdateStats(MaintenanceOpStats* stats) {
         new_num_dms_flushed == last_num_dms_flushed_ &&
         new_num_rs_compacted == last_num_rs_compacted_ &&
         new_num_rs_minor_delta_compacted == last_num_rs_minor_delta_compacted_) {
+      prev_stats_.set_workload_score(workload_score);
       *stats = prev_stats_;
       return;
-    } else {
-      last_num_mrs_flushed_ = new_num_mrs_flushed;
-      last_num_dms_flushed_ = new_num_dms_flushed;
-      last_num_rs_compacted_ = new_num_rs_compacted;
-      last_num_rs_minor_delta_compacted_ = new_num_rs_minor_delta_compacted;
     }
+    last_num_mrs_flushed_ = new_num_mrs_flushed;
+    last_num_dms_flushed_ = new_num_dms_flushed;
+    last_num_rs_compacted_ = new_num_rs_compacted;
+    last_num_rs_minor_delta_compacted_ = new_num_rs_minor_delta_compacted;
   }
 
   double perf_improv = tablet_->GetPerfImprovementForBestDeltaCompact(
       RowSet::MINOR_DELTA_COMPACTION, nullptr);
   prev_stats_.set_perf_improvement(perf_improv);
   prev_stats_.set_runnable(perf_improv > 0);
+  prev_stats_.set_workload_score(workload_score);
   *stats = prev_stats_;
 }
 
@@ -278,6 +292,9 @@ void MajorDeltaCompactionOp::UpdateStats(MaintenanceOpStats* stats) {
 
   std::lock_guard<simple_spinlock> l(lock_);
 
+  double workload_score = FLAGS_enable_workload_score_for_perf_improvement_ops ?
+                          tablet_->CollectAndUpdateWorkloadStats(MaintenanceOp::COMPACT_OP) : 0;
+
   // Any operation that changes the size of the on-disk data invalidates the
   // cached stats.
   TabletMetrics* metrics = tablet_->metrics();
@@ -295,21 +312,22 @@ void MajorDeltaCompactionOp::UpdateStats(MaintenanceOpStats* stats) {
         new_num_rs_compacted == last_num_rs_compacted_ &&
         new_num_rs_minor_delta_compacted == last_num_rs_minor_delta_compacted_ &&
         new_num_rs_major_delta_compacted == last_num_rs_major_delta_compacted_) {
+      prev_stats_.set_workload_score(workload_score);
       *stats = prev_stats_;
       return;
-    } else {
-      last_num_mrs_flushed_ = new_num_mrs_flushed;
-      last_num_dms_flushed_ = new_num_dms_flushed;
-      last_num_rs_compacted_ = new_num_rs_compacted;
-      last_num_rs_minor_delta_compacted_ = new_num_rs_minor_delta_compacted;
-      last_num_rs_major_delta_compacted_ = new_num_rs_major_delta_compacted;
     }
+    last_num_mrs_flushed_ = new_num_mrs_flushed;
+    last_num_dms_flushed_ = new_num_dms_flushed;
+    last_num_rs_compacted_ = new_num_rs_compacted;
+    last_num_rs_minor_delta_compacted_ = new_num_rs_minor_delta_compacted;
+    last_num_rs_major_delta_compacted_ = new_num_rs_major_delta_compacted;
   }
 
   double perf_improv = tablet_->GetPerfImprovementForBestDeltaCompact(
       RowSet::MAJOR_DELTA_COMPACTION, nullptr);
   prev_stats_.set_perf_improvement(perf_improv);
   prev_stats_.set_runnable(perf_improv > 0);
+  prev_stats_.set_workload_score(workload_score);
   *stats = prev_stats_;
 }
 
diff --git a/src/kudu/tablet/tablet_replica_mm_ops.cc b/src/kudu/tablet/tablet_replica_mm_ops.cc
index dafe746..61243a7 100644
--- a/src/kudu/tablet/tablet_replica_mm_ops.cc
+++ b/src/kudu/tablet/tablet_replica_mm_ops.cc
@@ -75,6 +75,7 @@ DEFINE_int32(flush_threshold_secs, 2 * 60,
 TAG_FLAG(flush_threshold_secs, experimental);
 TAG_FLAG(flush_threshold_secs, runtime);
 
+DECLARE_bool(enable_workload_score_for_perf_improvement_ops);
 
 METRIC_DEFINE_gauge_uint32(tablet, log_gc_running,
                            "Log GCs Running",
@@ -178,8 +179,12 @@ void FlushMRSOp::UpdateStats(MaintenanceOpStats* stats) {
   stats->set_logs_retained_bytes(
       tablet_replica_->tablet()->MemRowSetLogReplaySize(replay_size_map));
 
-  // TODO(todd): use workload statistics here to find out how "hot" the tablet has
-  // been in the last 5 minutes.
+  if (FLAGS_enable_workload_score_for_perf_improvement_ops) {
+    double workload_score =
+        tablet_replica_->tablet()->CollectAndUpdateWorkloadStats(MaintenanceOp::FLUSH_OP);
+    stats->set_workload_score(workload_score);
+  }
+
   FlushOpPerfImprovementPolicy::SetPerfImprovementForFlush(
       stats,
       time_since_flush_.elapsed().wall_millis());
@@ -248,6 +253,12 @@ void FlushDeltaMemStoresOp::UpdateStats(MaintenanceOpStats* stats) {
   stats->set_runnable(true);
   stats->set_logs_retained_bytes(retention_size);
 
+  if (FLAGS_enable_workload_score_for_perf_improvement_ops) {
+    double workload_score =
+        tablet_replica_->tablet()->CollectAndUpdateWorkloadStats(MaintenanceOp::FLUSH_OP);
+    stats->set_workload_score(workload_score);
+  }
+
   FlushOpPerfImprovementPolicy::SetPerfImprovementForFlush(
       stats,
       time_since_flush_.elapsed().wall_millis());
diff --git a/src/kudu/tserver/tablet_server-test-base.cc b/src/kudu/tserver/tablet_server-test-base.cc
index a172a1f..9cbbeeb 100644
--- a/src/kudu/tserver/tablet_server-test-base.cc
+++ b/src/kudu/tserver/tablet_server-test-base.cc
@@ -191,8 +191,12 @@ void TabletServerTestBase::ResetClientProxies() {
 
 // Inserts 'num_rows' test rows directly into the tablet (i.e not via RPC)
 void TabletServerTestBase::InsertTestRowsDirect(int32_t start_row,
-                                                int32_t num_rows) {
-  tablet::LocalTabletWriter writer(tablet_replica_->tablet(), &schema_);
+                                                int32_t num_rows,
+                                                const string& tablet_id) {
+  scoped_refptr<tablet::TabletReplica> tablet_replica;
+  ASSERT_TRUE(mini_server_->server()->tablet_manager()->LookupTablet(tablet_id,
+                                                                     &tablet_replica));
+  tablet::LocalTabletWriter writer(tablet_replica->tablet(), &schema_);
   KuduPartialRow row(&schema_);
   for (int32_t i = 0; i < num_rows; i++) {
     BuildTestRow(start_row + i, &row);
diff --git a/src/kudu/tserver/tablet_server-test-base.h b/src/kudu/tserver/tablet_server-test-base.h
index b10e3b5..8304202 100644
--- a/src/kudu/tserver/tablet_server-test-base.h
+++ b/src/kudu/tserver/tablet_server-test-base.h
@@ -71,7 +71,9 @@ class TabletServerTestBase : public KuduTest {
   void ResetClientProxies();
 
   // Inserts 'num_rows' test rows directly into the tablet (i.e not via RPC)
-  void InsertTestRowsDirect(int32_t start_row, int32_t num_rows);
+  void InsertTestRowsDirect(int32_t start_row,
+                            int32_t num_rows,
+                            const std::string& tablet_id = kTabletId);
 
   // Inserts 'num_rows' test rows remotely into the tablet (i.e via RPC)
   // Rows are grouped in batches of 'count'/'num_batches' size.
diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index a42fdf9..3e74107 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -84,6 +84,7 @@
 #include "kudu/tablet/metadata.pb.h"
 #include "kudu/tablet/tablet.h"
 #include "kudu/tablet/tablet_metadata.h"
+#include "kudu/tablet/tablet_metrics.h"
 #include "kudu/tablet/tablet_replica.h"
 #include "kudu/tserver/heartbeater.h"
 #include "kudu/tserver/mini_tablet_server.h"
@@ -165,15 +166,18 @@ DECLARE_bool(enable_flush_deltamemstores);
 DECLARE_bool(enable_flush_memrowset);
 DECLARE_bool(enable_maintenance_manager);
 DECLARE_bool(enable_rowset_compaction);
+DECLARE_bool(enable_workload_score_for_perf_improvement_ops);
 DECLARE_bool(fail_dns_resolution);
 DECLARE_bool(rowset_metadata_store_keys);
 DECLARE_double(cfile_inject_corruption);
 DECLARE_double(env_inject_eio);
 DECLARE_double(env_inject_full);
+DECLARE_double(workload_score_upper_bound);
 DECLARE_int32(flush_threshold_mb);
 DECLARE_int32(flush_threshold_secs);
 DECLARE_int32(fs_data_dirs_available_space_cache_seconds);
 DECLARE_int32(fs_target_data_dirs_per_tablet);
+DECLARE_int32(maintenance_manager_inject_latency_ms);
 DECLARE_int32(maintenance_manager_num_threads);
 DECLARE_int32(maintenance_manager_polling_interval_ms);
 DECLARE_int32(memory_pressure_percentage);
@@ -181,6 +185,8 @@ DECLARE_int32(metrics_retirement_age_ms);
 DECLARE_int32(scanner_batch_size_rows);
 DECLARE_int32(scanner_gc_check_interval_us);
 DECLARE_int32(scanner_ttl_ms);
+DECLARE_int32(workload_stats_rate_collection_min_interval_ms);
+DECLARE_int32(workload_stats_metric_collection_interval_ms);
 DECLARE_string(block_manager);
 DECLARE_string(env_inject_eio_globs);
 DECLARE_string(env_inject_full_globs);
@@ -4205,5 +4211,70 @@ TEST_F(TabletServerTest, TestScannerCheckMatchingUser) {
   }
 }
 
+TEST_F(TabletServerTest, TestStarvePerfImprovementOpsInColdTablet) {
+  SKIP_IF_SLOW_NOT_ALLOWED();
+  FLAGS_enable_maintenance_manager = true;
+  NO_FATALS(ShutdownAndRebuildTablet());
+
+  // Disable flushing and compactions to create overlapping rowsets.
+  FLAGS_enable_flush_memrowset = false;
+  FLAGS_enable_rowset_compaction = false;
+
+  // Create a cold tablet and insert sets of overlapping rows to it.
+  const char* kColdTablet = "cold_tablet";
+  ASSERT_OK(mini_server_->AddTestTablet(kTableId, kColdTablet, schema_));
+  ASSERT_OK(WaitForTabletRunning(kColdTablet));
+  scoped_refptr<TabletReplica> cold_replica;
+  ASSERT_TRUE(mini_server_->server()->tablet_manager()->LookupTablet(kColdTablet, &cold_replica));
+  NO_FATALS(InsertTestRowsDirect(0, 1, kColdTablet));
+  NO_FATALS(InsertTestRowsDirect(2, 1, kColdTablet));
+  ASSERT_OK(cold_replica->tablet()->Flush());
+  NO_FATALS(InsertTestRowsDirect(1, 1, kColdTablet));
+  ASSERT_OK(cold_replica->tablet()->Flush());
+
+  // Make MRS flush op starts out with the highest adjusted perf score.
+  FLAGS_enable_flush_memrowset = true;
+  FLAGS_enable_workload_score_for_perf_improvement_ops = true;
+  FLAGS_workload_stats_rate_collection_min_interval_ms = 10;
+  FLAGS_workload_score_upper_bound = 10;
+  // Make maintenance manager find best op more slowly than a new MRS could flush,
+  // so there are always something to do in the hot tablet.
+  FLAGS_flush_threshold_secs = 1;
+  FLAGS_maintenance_manager_inject_latency_ms = 1000;
+
+  // Make the default tablet 'hot'.
+  std::atomic<bool> keep_inserting_and_scanning(true);
+  thread insert_thread([&] {
+    int cur_row = 0;
+    while (keep_inserting_and_scanning) {
+      NO_FATALS(InsertTestRowsDirect(cur_row, 1000));
+      cur_row += 1000;
+      ScanResponsePB resp;
+      NO_FATALS(OpenScannerWithAllColumns(&resp));
+      ASSERT_TRUE(!resp.scanner_id().empty());
+    }
+  });
+  SCOPED_CLEANUP({
+    keep_inserting_and_scanning = false;
+    insert_thread.join();
+  });
+
+  // Wait a bit to allow workload stats collection to begin.
+  SleepFor(MonoDelta::FromSeconds(1));
+  FLAGS_enable_rowset_compaction = true;
+  // Wait a couple seconds to test if we can starve compaction in the cold tablet even
+  // if the compaction op has a higher raw perf improvement score than that of time-based
+  // flushes in the hot tablet.
+  SleepFor(MonoDelta::FromSeconds(5));
+  ASSERT_EQ(0, cold_replica->tablet()->metrics()->compact_rs_duration->TotalCount());
+
+  // Disable workload score, now we can launch compaction in the cold tablet.
+  FLAGS_enable_workload_score_for_perf_improvement_ops = false;
+  // Compaction in the cold tablet will eventually complete.
+  ASSERT_EVENTUALLY([&] {
+    ASSERT_EQ(1, cold_replica->tablet()->metrics()->compact_rs_duration->TotalCount());
+  });
+}
+
 } // namespace tserver
 } // namespace kudu
diff --git a/src/kudu/tserver/tserver_path_handlers.cc b/src/kudu/tserver/tserver_path_handlers.cc
index ffc0c1b..7fb2b81 100644
--- a/src/kudu/tserver/tserver_path_handlers.cc
+++ b/src/kudu/tserver/tserver_path_handlers.cc
@@ -640,6 +640,7 @@ void TabletServerPathHandlers::HandleMaintenanceManagerPage(const Webserver::Web
     registered_op["ram_anchored"] = HumanReadableNumBytes::ToString(op_pb.ram_anchored_bytes());
     registered_op["logs_retained"] = HumanReadableNumBytes::ToString(op_pb.logs_retained_bytes());
     registered_op["perf"] = op_pb.perf_improvement();
+    registered_op["workload_score"] = op_pb.workload_score();
   }
 }
 
diff --git a/src/kudu/util/maintenance_manager-test.cc b/src/kudu/util/maintenance_manager-test.cc
index c2d6301..bf4272c 100644
--- a/src/kudu/util/maintenance_manager-test.cc
+++ b/src/kudu/util/maintenance_manager-test.cc
@@ -66,10 +66,10 @@ DECLARE_bool(enable_maintenance_manager);
 DECLARE_int64(log_target_replay_size_mb);
 DECLARE_double(maintenance_op_multiplier);
 DECLARE_int32(max_priority_range);
-
 namespace kudu {
 
-static const int kHistorySize = 7;
+// Set this a bit bigger so that the manager could keep track of all possible completed ops.
+static const int kHistorySize = 10;
 static const char kFakeUuid[] = "12345";
 
 class MaintenanceManagerTest : public KuduTest {
@@ -124,7 +124,8 @@ class TestMaintenanceOp : public MaintenanceOp {
       remaining_runs_(1),
       prepared_runs_(0),
       sleep_time_(MonoDelta::FromSeconds(0)),
-      priority_(priority) {
+      priority_(priority),
+      workload_score_(0) {
   }
 
   ~TestMaintenanceOp() override = default;
@@ -160,6 +161,7 @@ class TestMaintenanceOp : public MaintenanceOp {
     stats->set_ram_anchored(ram_anchored_);
     stats->set_logs_retained_bytes(logs_retained_bytes_);
     stats->set_perf_improvement(perf_improvement_);
+    stats->set_workload_score(workload_score_);
   }
 
   void set_remaining_runs(int runs) {
@@ -187,6 +189,11 @@ class TestMaintenanceOp : public MaintenanceOp {
     perf_improvement_ = perf_improvement;
   }
 
+  void set_workload_score(uint64_t workload_score) {
+    std::lock_guard<Mutex> guard(lock_);
+    workload_score_ = workload_score;
+  }
+
   scoped_refptr<Histogram> DurationHistogram() const override {
     return maintenance_op_duration_;
   }
@@ -226,6 +233,8 @@ class TestMaintenanceOp : public MaintenanceOp {
 
   // Maintenance priority.
   int32_t priority_;
+
+  double workload_score_;
 };
 
 // Create an op and wait for it to start running.  Unregister it while it is
@@ -475,13 +484,14 @@ TEST_F(MaintenanceManagerTest, TestOpFactors) {
   TestMaintenanceOp op5("op5", MaintenanceOp::HIGH_IO_USAGE, FLAGS_max_priority_range + 1);
 
   ASSERT_DOUBLE_EQ(pow(FLAGS_maintenance_op_multiplier, -FLAGS_max_priority_range),
-                   manager_->PerfImprovement(1, op1.priority()));
+                   manager_->AdjustedPerfScore(1, 0, op1.priority()));
   ASSERT_DOUBLE_EQ(pow(FLAGS_maintenance_op_multiplier, -1),
-                   manager_->PerfImprovement(1, op2.priority()));
-  ASSERT_DOUBLE_EQ(1, manager_->PerfImprovement(1, op3.priority()));
-  ASSERT_DOUBLE_EQ(FLAGS_maintenance_op_multiplier, manager_->PerfImprovement(1, op4.priority()));
+                   manager_->AdjustedPerfScore(1, 0, op2.priority()));
+  ASSERT_DOUBLE_EQ(1, manager_->AdjustedPerfScore(1, 0, op3.priority()));
+  ASSERT_DOUBLE_EQ(FLAGS_maintenance_op_multiplier,
+                   manager_->AdjustedPerfScore(1, 0, op4.priority()));
   ASSERT_DOUBLE_EQ(pow(FLAGS_maintenance_op_multiplier, FLAGS_max_priority_range),
-                   manager_->PerfImprovement(1, op5.priority()));
+                   manager_->AdjustedPerfScore(1, 0, op5.priority()));
 }
 
 // Test priority OP launching.
@@ -513,8 +523,16 @@ TEST_F(MaintenanceManagerTest, TestPriorityOpLaunch) {
   // FLAGS_enable_maintenance_manager = false, which would cause the thread
   // to exit entirely instead of sleeping.
 
-  // Ops are listed here in perf improvement order, which is a function of the
-  // op's raw perf improvement as well as its priority.
+  // Ops are listed here in final perf score order, which is a function of the
+  // op's raw perf improvement, workload score and its priority.
+  // The 'op0' would never launch because it has a raw perf improvement 0, even if
+  // it has a high workload_score and a high priority.
+  TestMaintenanceOp op0("op0", MaintenanceOp::HIGH_IO_USAGE, FLAGS_max_priority_range + 1);
+  op0.set_perf_improvement(0);
+  op0.set_workload_score(10);
+  op0.set_remaining_runs(1);
+  op0.set_sleep_time(MonoDelta::FromMilliseconds(1));
+
   TestMaintenanceOp op1("op1", MaintenanceOp::HIGH_IO_USAGE, -FLAGS_max_priority_range - 1);
   op1.set_perf_improvement(10);
   op1.set_remaining_runs(1);
@@ -545,30 +563,40 @@ TEST_F(MaintenanceManagerTest, TestPriorityOpLaunch) {
   op6.set_remaining_runs(1);
   op6.set_sleep_time(MonoDelta::FromMilliseconds(1));
 
+  TestMaintenanceOp op7("op7", MaintenanceOp::HIGH_IO_USAGE, 0);
+  op7.set_perf_improvement(9);
+  op7.set_workload_score(10);
+  op7.set_remaining_runs(1);
+  op7.set_sleep_time(MonoDelta::FromMilliseconds(1));
+
   FLAGS_enable_maintenance_manager = false;
+  manager_->RegisterOp(&op0);
   manager_->RegisterOp(&op1);
   manager_->RegisterOp(&op2);
   manager_->RegisterOp(&op3);
   manager_->RegisterOp(&op4);
   manager_->RegisterOp(&op5);
   manager_->RegisterOp(&op6);
+  manager_->RegisterOp(&op7);
   FLAGS_enable_maintenance_manager = true;
 
   // From this point forward if an ASSERT fires, we'll hit a CHECK failure if
   // we don't unregister an op before it goes out of scope.
   SCOPED_CLEANUP({
+    manager_->UnregisterOp(&op0);
     manager_->UnregisterOp(&op1);
     manager_->UnregisterOp(&op2);
     manager_->UnregisterOp(&op3);
     manager_->UnregisterOp(&op4);
     manager_->UnregisterOp(&op5);
     manager_->UnregisterOp(&op6);
+    manager_->UnregisterOp(&op7);
   });
 
   ASSERT_EVENTUALLY([&]() {
     MaintenanceManagerStatusPB status_pb;
     manager_->GetMaintenanceManagerStatusDump(&status_pb);
-    ASSERT_EQ(status_pb.completed_operations_size(), 7);
+    ASSERT_EQ(8, status_pb.completed_operations_size());
   });
 
   // Wait for instances to complete by shutting down the maintenance manager.
@@ -578,7 +606,7 @@ TEST_F(MaintenanceManagerTest, TestPriorityOpLaunch) {
   // Check that running instances are removed from collection after completion.
   MaintenanceManagerStatusPB status_pb;
   manager_->GetMaintenanceManagerStatusDump(&status_pb);
-  ASSERT_EQ(status_pb.running_operations_size(), 0);
+  ASSERT_EQ(0, status_pb.running_operations_size());
 
   // Check that ops were executed in perf improvement order (from greatest to
   // least improvement). Note that completed ops are listed in _reverse_ execution order.
@@ -588,6 +616,7 @@ TEST_F(MaintenanceManagerTest, TestPriorityOpLaunch) {
                             "op4",
                             "op5",
                             "op6",
+                            "op7",
                             "early"});
   ASSERT_EQ(ordered_ops.size(), status_pb.completed_operations().size());
   for (const auto& instance : status_pb.completed_operations()) {
diff --git a/src/kudu/util/maintenance_manager.cc b/src/kudu/util/maintenance_manager.cc
index 6327048..033045f 100644
--- a/src/kudu/util/maintenance_manager.cc
+++ b/src/kudu/util/maintenance_manager.cc
@@ -33,6 +33,7 @@
 
 #include "kudu/gutil/dynamic_annotations.h"
 #include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -110,6 +111,11 @@ TAG_FLAG(max_priority_range, advanced);
 TAG_FLAG(max_priority_range, experimental);
 TAG_FLAG(max_priority_range, runtime);
 
+DEFINE_int32(maintenance_manager_inject_latency_ms, 0,
+             "Injects latency into maintenance thread. For use in tests only.");
+TAG_FLAG(maintenance_manager_inject_latency_ms, runtime);
+TAG_FLAG(maintenance_manager_inject_latency_ms, unsafe);
+
 namespace kudu {
 
 MaintenanceOpStats::MaintenanceOpStats() {
@@ -123,6 +129,7 @@ void MaintenanceOpStats::Clear() {
   logs_retained_bytes_ = 0;
   data_retained_bytes_ = 0;
   perf_improvement_ = 0;
+  workload_score_ = 0;
   last_modified_ = MonoTime();
 }
 
@@ -292,6 +299,12 @@ void MaintenanceManager::RunSchedulerThread() {
       return;
     }
 
+    if (PREDICT_FALSE(FLAGS_maintenance_manager_inject_latency_ms > 0)) {
+      LOG(WARNING) << "Injecting " << FLAGS_maintenance_manager_inject_latency_ms
+                   << "ms of latency into maintenance thread";
+      SleepFor(MonoDelta::FromMilliseconds(FLAGS_maintenance_manager_inject_latency_ms));
+    }
+
     // If we found no work to do, then we should sleep before trying again to schedule.
     // Otherwise, we can go right into trying to find the next op.
     prev_iter_found_no_work = !FindAndLaunchOp(&guard);
@@ -408,7 +421,8 @@ pair<MaintenanceOp*, string> MaintenanceManager::FindBestOp() {
                         op->name(), data_retained_bytes);
     }
 
-    const auto perf_improvement = PerfImprovement(stats.perf_improvement(), op->priority());
+    const auto perf_improvement =
+        AdjustedPerfScore(stats.perf_improvement(), stats.workload_score(), op->priority());
     if ((!best_perf_improvement_op) ||
         (perf_improvement > best_perf_improvement)) {
       best_perf_improvement_op = op;
@@ -472,14 +486,20 @@ pair<MaintenanceOp*, string> MaintenanceManager::FindBestOp() {
   return {nullptr, "no ops with positive improvement"};
 }
 
-double MaintenanceManager::PerfImprovement(double perf_improvement, int32_t priority) const {
+double MaintenanceManager::AdjustedPerfScore(double perf_improvement,
+                                             double workload_score,
+                                             int32_t priority) {
+  if (perf_improvement == 0) {
+    return 0;
+  }
+  double perf_score = perf_improvement + workload_score;
   if (priority == 0) {
-    return perf_improvement;
+    return perf_score;
   }
 
   priority = std::max(priority, -FLAGS_max_priority_range);
   priority = std::min(priority, FLAGS_max_priority_range);
-  return perf_improvement * std::pow(FLAGS_maintenance_op_multiplier, priority);
+  return perf_score * std::pow(FLAGS_maintenance_op_multiplier, priority);
 }
 
 void MaintenanceManager::LaunchOp(MaintenanceOp* op) {
@@ -542,11 +562,13 @@ void MaintenanceManager::GetMaintenanceManagerStatusDump(MaintenanceManagerStatu
       op_pb->set_ram_anchored_bytes(stats.ram_anchored());
       op_pb->set_logs_retained_bytes(stats.logs_retained_bytes());
       op_pb->set_perf_improvement(stats.perf_improvement());
+      op_pb->set_workload_score(stats.workload_score());
     } else {
       op_pb->set_runnable(false);
       op_pb->set_ram_anchored_bytes(0);
       op_pb->set_logs_retained_bytes(0);
       op_pb->set_perf_improvement(0.0);
+      op_pb->set_workload_score(0.0);
     }
   }
 
diff --git a/src/kudu/util/maintenance_manager.h b/src/kudu/util/maintenance_manager.h
index 6dad1b6..9bccc9e 100644
--- a/src/kudu/util/maintenance_manager.h
+++ b/src/kudu/util/maintenance_manager.h
@@ -106,6 +106,19 @@ class MaintenanceOpStats {
     perf_improvement_ = perf_improvement;
   }
 
+  double workload_score() const {
+    DCHECK(valid_);
+    return workload_score_;
+  }
+
+  void set_workload_score(double workload_score) {
+    if (workload_score == workload_score_) {
+      return;
+    }
+    UpdateLastModified();
+    workload_score_ = workload_score;
+  }
+
   const MonoTime& last_modified() const {
     DCHECK(valid_);
     return last_modified_;
@@ -146,6 +159,8 @@ class MaintenanceOpStats {
   // absolute scale (yet TBD).
   double perf_improvement_;
 
+  double workload_score_;
+
   // The last time that the stats were modified.
   MonoTime last_modified_;
 };
@@ -179,6 +194,11 @@ class MaintenanceOp {
     HIGH_IO_USAGE // Everything else.
   };
 
+  enum PerfImprovementOpType {
+    FLUSH_OP,
+    COMPACT_OP
+  };
+
   explicit MaintenanceOp(std::string name, IOUsage io_usage);
   virtual ~MaintenanceOp();
 
@@ -325,7 +345,9 @@ class MaintenanceManager : public std::enable_shared_from_this<MaintenanceManage
   // suitable for logging.
   std::pair<MaintenanceOp*, std::string> FindBestOp();
 
-  double PerfImprovement(double perf_improvement, int32_t priority) const;
+  // Adjust the perf score based on the raw perf score, the tablet's workload_score
+  // and the table's priority.
+  static double AdjustedPerfScore(double perf_improvement, double workload_score, int32_t priority);
 
   void LaunchOp(MaintenanceOp* op);
 
diff --git a/src/kudu/util/maintenance_manager.proto b/src/kudu/util/maintenance_manager.proto
index 77c8cf8..54f2280 100644
--- a/src/kudu/util/maintenance_manager.proto
+++ b/src/kudu/util/maintenance_manager.proto
@@ -29,6 +29,7 @@ message MaintenanceManagerStatusPB {
     required int64 ram_anchored_bytes = 4;
     required int64 logs_retained_bytes = 5;
     required double perf_improvement = 6;
+    required double workload_score = 7;
   }
 
   message OpInstancePB {
diff --git a/www/maintenance-manager.mustache b/www/maintenance-manager.mustache
index 81d5247..60151e2 100644
--- a/www/maintenance-manager.mustache
+++ b/www/maintenance-manager.mustache
@@ -66,6 +66,7 @@ under the License.
       <th data-sorter="bytesSorter" data-sortable="true">RAM anchored</th>
       <th data-sorter="bytesSorter" data-sortable="true">Logs retained</th>
       <th data-sortable="true">Perf</th>
+      <th data-sortable="true">Workload score</th>
     </tr>
   </thead>
   <tbody>
@@ -76,6 +77,7 @@ under the License.
       <td>{{ram_anchored}}</td>
       <td>{{logs_retained}}</td>
       <td>{{perf}}</td>
+      <td>{{workload_score}}</td>
     </tr>
    {{/registered_operations}}
   </tbody>


[kudu] 01/23: [metrics] make 'merged_entities_count_of_tablet' visible when only one entity merged

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit d5162ad09b989d899569381c3966aad648d6c237
Author: zhangyifan27 <ch...@163.com>
AuthorDate: Thu Apr 9 19:27:32 2020 +0800

    [metrics] make 'merged_entities_count_of_tablet' visible when only one entity merged
    
    Before this patch, the metric 'merged_entities_count_of_tablet' is hidden
    when tablet server aggregate metrics by the same attribute and there is
    only one entity merged, because it's modification epoch is not updated.
    We should expose the metric in this condition.
    
    Change-Id: Ia74cae25b5bde7d61db1186e44abdff487d3f23f
    Reviewed-on: http://gerrit.cloudera.org:8080/15707
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/tserver/tablet_server-test.cc | 13 +++++++++++++
 src/kudu/util/metrics.cc               |  3 +++
 2 files changed, 16 insertions(+)

diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index fab5ad4..a42fdf9 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -659,6 +659,19 @@ TEST_F(TabletServerTest, TestTombstonedTabletOnWebUI) {
   ASSERT_STR_NOT_CONTAINS(s, mini_server_->bound_rpc_addr().ToString());
 }
 
+// When tablet server merge metrics by the same attributes, the metric
+// 'merged_entities_count_of_tablet' should be visible
+TEST_F(TabletServerTest, TestMergedEntitiesCount) {
+  EasyCurl c;
+  faststring buf;
+  const string addr = mini_server_->bound_http_addr().ToString();
+  ASSERT_OK(c.FetchURL(Substitute("http://$0/metrics", addr), &buf));
+  ASSERT_STR_NOT_CONTAINS(buf.ToString(), "merged_entities_count_of_tablet");
+  ASSERT_OK(c.FetchURL(Substitute("http://$0/metrics?merge_rules=tablet|table|table_name", addr),
+                       &buf));
+  ASSERT_STR_CONTAINS(buf.ToString(), "merged_entities_count_of_tablet");
+}
+
 class TabletServerDiskSpaceTest : public TabletServerTestBase,
                                   public testing::WithParamInterface<string> {
  public:
diff --git a/src/kudu/util/metrics.cc b/src/kudu/util/metrics.cc
index f84edab..a48f870 100644
--- a/src/kudu/util/metrics.cc
+++ b/src/kudu/util/metrics.cc
@@ -426,6 +426,9 @@ Status MetricEntity::CollectTo(MergedEntityMetrics* collections,
     scoped_refptr<Metric> entry = FindPtrOrNull(entity_collection, prototype);
     if (!entry) {
       scoped_refptr<Metric> new_metric = metric->snapshot();
+      if (!new_metric->invalid_for_merge_) {
+        new_metric->UpdateModificationEpoch();
+      }
       InsertOrDie(&entity_collection, new_metric->prototype(), new_metric);
     } else {
       entry->MergeFrom(metric);


[kudu] 10/23: [collector] fix metrics filter and merge url for version 1.11

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 82b0fb4d3526fd0ff08125290d6325293e291fd9
Author: Yingchun Lai <40...@qq.com>
AuthorDate: Tue Dec 31 11:50:04 2019 +0800

    [collector] fix metrics filter and merge url for version 1.11
---
 src/kudu/collector/metrics_collector-test.cc | 21 ++++++++++-----------
 src/kudu/collector/metrics_collector.cc      | 19 +------------------
 src/kudu/collector/metrics_collector.h       |  1 -
 3 files changed, 11 insertions(+), 30 deletions(-)

diff --git a/src/kudu/collector/metrics_collector-test.cc b/src/kudu/collector/metrics_collector-test.cc
index 4a3e050..19c00cf 100644
--- a/src/kudu/collector/metrics_collector-test.cc
+++ b/src/kudu/collector/metrics_collector-test.cc
@@ -41,7 +41,6 @@ DECLARE_bool(collector_request_merged_metrics);
 DECLARE_string(collector_attributes);
 DECLARE_string(collector_cluster_level_metrics);
 DECLARE_string(collector_metrics);
-DECLARE_string(collector_table_names);
 DECLARE_string(collector_metrics_types_for_test);
 
 using std::map;
@@ -735,12 +734,11 @@ TEST(TestMetricsCollector, TestInitFilters) {
   ASSERT_EQ(collector->attributes_filter_, expect_attributes_filter);
 }
 
-#define CHECK_URL_PARAMETERS(metrics, request_merged, attributes, table_names, expect_url)        \
+#define CHECK_URL_PARAMETERS(metrics, request_merged, attributes, expect_url)                     \
 do {                                                                                              \
   FLAGS_collector_metrics = metrics;                                                              \
   FLAGS_collector_request_merged_metrics = request_merged;                                        \
   FLAGS_collector_attributes = attributes;                                                        \
-  FLAGS_collector_table_names = table_names;                                                      \
   auto collector = BuildCollector();                                                              \
   ASSERT_OK(collector->InitFilters());                                                            \
   ASSERT_OK(collector->InitMetricsUrlParameters());                                               \
@@ -748,17 +746,18 @@ do {
 } while (false)
 
 TEST(TestMetricsCollector, TestInitMetricsUrlParameters) {
-  CHECK_URL_PARAMETERS("", true, "", "",
-      "/metrics?compact=1&origin=false&merge=true");
-  CHECK_URL_PARAMETERS("m1,m2,m3", true, "", "",
-      "/metrics?compact=1&metrics=m1,m2,m3&origin=false&merge=true");
+  CHECK_URL_PARAMETERS("", true, "",
+      "/metrics?compact=1&merge_rules=tablet|table|table_name");
+  CHECK_URL_PARAMETERS("m1,m2,m3", true, "",
+      "/metrics?compact=1&metrics=m1,m2,m3&merge_rules=tablet|table|table_name");
   // TODO(yingchun): now FLAGS_collector_request_merged_metrics must be true
   //CHECK_URL_PARAMETERS("", false, "", "",
   //    "/metrics?compact=1");
-  CHECK_URL_PARAMETERS("", true, "attr1:a1,a2;attr2:a3", "",
-      "/metrics?compact=1&origin=false&merge=true&attributes=attr2,a3,attr1,a1,attr1,a2,");
-  CHECK_URL_PARAMETERS("", true, "", "t1,t2,t3",
-      "/metrics?compact=1&origin=false&merge=true&table_names=t1,t2,t3");
+  CHECK_URL_PARAMETERS("", true, "attr1:a1,a2;attr2:a3",
+      "/metrics?compact=1&merge_rules=tablet|table|table_name"
+      "&attributes=attr2,a3,attr1,a1,attr1,a2,");
+  CHECK_URL_PARAMETERS("", true, "",
+      "/metrics?compact=1&merge_rules=tablet|table|table_name");
 }
 
 TEST(TestMetricsCollector, TestInitClusterLevelMetrics) {
diff --git a/src/kudu/collector/metrics_collector.cc b/src/kudu/collector/metrics_collector.cc
index e5adfcb..688ea70 100644
--- a/src/kudu/collector/metrics_collector.cc
+++ b/src/kudu/collector/metrics_collector.cc
@@ -67,8 +67,6 @@ DEFINE_string(collector_metrics_types_for_test, "",
               "Only for test, used to initialize metric_types_by_entity_type_");
 DEFINE_bool(collector_request_merged_metrics, true,
             "Whether to request merged metrics and exclude unmerged metrics from server");
-DEFINE_string(collector_table_names, "",
-              "Table names to collect (comma-separated list of table names)");
 
 DECLARE_string(collector_cluster_name);
 DECLARE_int32(collector_interval_sec);
@@ -104,10 +102,8 @@ MetricsCollector::~MetricsCollector() {
 Status MetricsCollector::Init() {
   CHECK(!initialized_);
 
-  RETURN_NOT_OK(ValidateTableFilter(FLAGS_collector_attributes, FLAGS_collector_table_names));
   RETURN_NOT_OK(InitMetrics());
   RETURN_NOT_OK(InitFilters());
-  CHECK(attributes_filter_.empty());  // TODO(yingchun) disable now
   RETURN_NOT_OK(InitMetricsUrlParameters());
   RETURN_NOT_OK(InitClusterLevelMetrics());
 
@@ -178,15 +174,6 @@ Status MetricsCollector::UpdateThreadPool(int32_t thread_count) {
   return Status::OK();
 }
 
-Status MetricsCollector::ValidateTableFilter(const string& attribute_filter,
-                                             const string& /*table_filter*/) {
-  if (attribute_filter.empty()) {
-    return Status::OK();
-  }
-
-  return Status::InvalidArgument("attribute filter is not supported now");
-}
-
 Status MetricsCollector::InitMetrics() {
   string resp;
   if (PREDICT_TRUE(FLAGS_collector_metrics_types_for_test.empty())) {
@@ -266,7 +253,7 @@ Status MetricsCollector::InitMetricsUrlParameters() {
     metric_url_parameters_ += "&metrics=" + FLAGS_collector_metrics;
   }
   if (FLAGS_collector_request_merged_metrics) {
-    metric_url_parameters_ += "&origin=false&merge=true";
+    metric_url_parameters_ += "&merge_rules=tablet|table|table_name";
   } else {
     LOG(FATAL) << "Non-merge mode is not supported now, you should set "
                   "FLAGS_collector_request_merged_metrics to true if you "
@@ -282,10 +269,6 @@ Status MetricsCollector::InitMetricsUrlParameters() {
       metric_url_parameters_ += Substitute("$0,$1,", attribute_filter.first, value);
     }
   }
-  // TODO(yingchun) This is supported since internal version 1.8.0
-  if (!FLAGS_collector_table_names.empty()) {
-    metric_url_parameters_ += "&table_names=" + FLAGS_collector_table_names;
-  }
   return Status::OK();
 }
 
diff --git a/src/kudu/collector/metrics_collector.h b/src/kudu/collector/metrics_collector.h
index 1435e52..05673ef 100644
--- a/src/kudu/collector/metrics_collector.h
+++ b/src/kudu/collector/metrics_collector.h
@@ -92,7 +92,6 @@ class MetricsCollector : public RefCounted<MetricsCollector> {
 
   typedef std::unordered_map<std::string, std::string> MetricTypes;
 
-  Status ValidateTableFilter(const std::string& attribute_filter, const std::string& table_filter);
   Status InitMetrics();
   static Status ExtractMetricTypes(const JsonReader& r,
                                    const rapidjson::Value* entity,


[kudu] 04/23: [metrics] fix MeanGauge::snapshot()

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit b65012f50d55658180d8c7509a340292232399ec
Author: zhangyifan27 <ch...@163.com>
AuthorDate: Thu May 14 19:15:20 2020 +0800

    [metrics] fix MeanGauge::snapshot()
    
    MeanGauge::snapshot() will always return a metric whose
    total_sum/total_count is ‘0' because there is a bug in the method,
    that leads to wrong values when MeanGauge metrics merged.
    
    Change-Id: I15f5e0705759a4bdd513089885aaaea4f5332ce1
    Reviewed-on: http://gerrit.cloudera.org:8080/15916
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/util/metrics-test.cc | 12 ++++++++++++
 src/kudu/util/metrics.cc      |  1 +
 2 files changed, 13 insertions(+)

diff --git a/src/kudu/util/metrics-test.cc b/src/kudu/util/metrics-test.cc
index 136ba72..5664a4f 100644
--- a/src/kudu/util/metrics-test.cc
+++ b/src/kudu/util/metrics-test.cc
@@ -186,6 +186,18 @@ TEST_F(MetricsTest, SimpleMeanGaugeTest) {
   ASSERT_EQ(2.5, average_usage->value());
 }
 
+TEST_F(MetricsTest, SimpleMeanGaugeSnapshotTest) {
+  scoped_refptr<MeanGauge> average_usage =
+    METRIC_test_mean_gauge.InstantiateMeanGauge(entity_);
+  scoped_refptr<MeanGauge> old_metric =
+    down_cast<MeanGauge*>(average_usage->snapshot().get());
+  ASSERT_EQ(0, old_metric->value());
+  average_usage->set_value(10.0, 2.0);
+  scoped_refptr<MeanGauge> new_metric =
+    down_cast<MeanGauge*>(average_usage->snapshot().get());
+  ASSERT_EQ(5, new_metric->value());
+}
+
 TEST_F(MetricsTest, SimpleMeanGaugeMergeTest) {
   scoped_refptr<MeanGauge> average_usage =
     METRIC_test_mean_gauge.InstantiateMeanGauge(entity_);
diff --git a/src/kudu/util/metrics.cc b/src/kudu/util/metrics.cc
index a48f870..39bc450 100644
--- a/src/kudu/util/metrics.cc
+++ b/src/kudu/util/metrics.cc
@@ -800,6 +800,7 @@ void StringGauge::WriteValue(JsonWriter* writer) const {
 scoped_refptr<Metric> MeanGauge::snapshot() const {
   std::lock_guard<simple_spinlock> l(lock_);
   auto p = new MeanGauge(down_cast<const GaugePrototype<double>*>(prototype_));
+  p->set_value(total_sum_, total_count_);
   p->m_epoch_.store(m_epoch_);
   p->invalid_for_merge_ = invalid_for_merge_;
   p->retire_time_ = retire_time_;


[kudu] 15/23: [collector] collect more for histogram metrics

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 8cea6f4eb41967b9339b4d972b48effdfebc5631
Author: zhangyifan8 <zh...@xiaomi.com>
AuthorDate: Tue Mar 24 17:41:57 2020 +0800

    [collector] collect more for histogram metrics
    
    1. collector mean/percentile_75/percentile_95 for histogram metrics.
    2. fix merge rules for MeanGauge metrics.
---
 src/kudu/collector/metrics_collector-test.cc | 143 +++++++++++++++++++++++++++
 src/kudu/collector/metrics_collector.cc      |  24 ++++-
 src/kudu/collector/metrics_collector.h       |   4 +-
 src/kudu/scripts/falcon_screen.json          |  75 ++++++++++++++
 4 files changed, 241 insertions(+), 5 deletions(-)

diff --git a/src/kudu/collector/metrics_collector-test.cc b/src/kudu/collector/metrics_collector-test.cc
index 865f821..fa14062 100644
--- a/src/kudu/collector/metrics_collector-test.cc
+++ b/src/kudu/collector/metrics_collector-test.cc
@@ -392,6 +392,7 @@ TEST(TestMetricsCollector, TestParseMetrics) {
       {"server_metric", "COUNTER"},
       {"metric_counter1", "COUNTER"},
       {"metric_counter2", "COUNTER"},
+      {"average_metric", "MEANGAUGE"},
       {"server_metric_histogram", "HISTOGRAM"},
       {"metric_histogram1", "HISTOGRAM"},
       {"metric_histogram2", "HISTOGRAM"}
@@ -442,6 +443,12 @@ TEST(TestMetricsCollector, TestParseMetrics) {
       R"*(        "value": 20                           )*"
       R"*(      },                                      )*"
       R"*(      {                                       )*"
+      R"*(        "name": "average_metric",             )*"
+      R"*(        "value": 1,                           )*"
+      R"*(        "total_sum": 10,                      )*"
+      R"*(        "total_count": 10                     )*"
+      R"*(      },                                      )*"
+      R"*(      {                                       )*"
       R"*(        "name": "metric_histogram1",          )*"
       R"*(        "total_count": 17,                    )*"
       R"*(        "min": 6,                             )*"
@@ -469,6 +476,12 @@ TEST(TestMetricsCollector, TestParseMetrics) {
       R"*(        "value": 100                          )*"
       R"*(      },                                      )*"
       R"*(      {                                       )*"
+      R"*(        "name": "average_metric",             )*"
+      R"*(        "value": 0.5,                         )*"
+      R"*(        "total_sum": 20,                      )*"
+      R"*(        "total_count": 20                     )*"
+      R"*(      },                                      )*"
+      R"*(      {                                       )*"
       R"*(        "name": "metric_histogram1",          )*"
       R"*(        "total_count": 170,                   )*"
       R"*(        "min": 60,                            )*"
@@ -526,6 +539,30 @@ TEST(TestMetricsCollector, TestParseMetrics) {
         "table1",
         {
           {
+            "average_metric",
+            {
+              {10, 1}
+            }
+          },
+          {
+            "metric_histogram1_mean",
+            {
+              {17, 47.8235}
+            }
+          },
+          {
+            "metric_histogram1_percentile_75",
+            {
+              {17, 62}
+            }
+          },
+          {
+            "metric_histogram1_percentile_95",
+            {
+              {17, 72}
+            }
+          },
+          {
             "metric_histogram1_percentile_99",
             {
               {17, 73}
@@ -537,12 +574,54 @@ TEST(TestMetricsCollector, TestParseMetrics) {
         "table2",
         {
           {
+            "average_metric",
+            {
+              {20, 0.5}
+            }
+          },
+          {
+            "metric_histogram1_mean",
+            {
+              {170, 478.235}
+            }
+          },
+          {
+            "metric_histogram1_percentile_75",
+            {
+              {170, 620}
+            }
+          },
+          {
+            "metric_histogram1_percentile_95",
+            {
+              {170, 720}
+            }
+          },
+          {
             "metric_histogram1_percentile_99",
             {
               {170, 730}
             }
           },
           {
+            "metric_histogram2_mean",
+            {
+              {34, 47.8235}
+            }
+          },
+          {
+            "metric_histogram2_percentile_75",
+            {
+              {34, 62}
+            }
+          },
+          {
+            "metric_histogram2_percentile_95",
+            {
+              {34, 72}
+            }
+          },
+          {
             "metric_histogram2_percentile_99",
             {
               {34, 72}
@@ -558,6 +637,34 @@ TEST(TestMetricsCollector, TestParseMetrics) {
   }));
   ASSERT_EQ(host_hist_metrics, MetricsCollector::HistMetrics({
       {
+        "average_metric",
+        {
+          {10, 1},
+          {20, 0.5}
+        }
+      },
+      {
+        "metric_histogram1_mean",
+        {
+          {17, 47.8235},
+          {170, 478.235}
+        }
+      },
+      {
+        "metric_histogram1_percentile_75",
+        {
+          {17, 62},
+          {170, 620}
+        }
+      },
+      {
+        "metric_histogram1_percentile_95",
+        {
+          {17, 72},
+          {170, 720}
+        }
+      },
+      {
         "metric_histogram1_percentile_99",
         {
           {17, 73},
@@ -565,12 +672,48 @@ TEST(TestMetricsCollector, TestParseMetrics) {
         }
       },
       {
+        "metric_histogram2_mean",
+        {
+          {34, 47.8235}
+        }
+      },
+      {
+        "metric_histogram2_percentile_75",
+        {
+          {34, 62}
+        }
+      },
+      {
+        "metric_histogram2_percentile_95",
+        {
+          {34, 72}
+        }
+      },
+      {
         "metric_histogram2_percentile_99",
         {
           {34, 72}
         }
       },
       {
+        "server_metric_histogram_mean",
+        {
+          {60, 76.16666666666667}
+        }
+      },
+      {
+        "server_metric_histogram_percentile_75",
+        {
+          {60, 25}
+        }
+      },
+      {
+        "server_metric_histogram_percentile_95",
+        {
+          {60, 66}
+        }
+      },
+      {
         "server_metric_histogram_percentile_99",
         {
           {60, 79}
diff --git a/src/kudu/collector/metrics_collector.cc b/src/kudu/collector/metrics_collector.cc
index fc9f20a..b170422 100644
--- a/src/kudu/collector/metrics_collector.cc
+++ b/src/kudu/collector/metrics_collector.cc
@@ -39,6 +39,7 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/curl_util.h"
 #include "kudu/util/debug/trace_event.h"
@@ -85,7 +86,8 @@ using strings::Substitute;
 namespace kudu {
 namespace collector {
 
-const set<string> MetricsCollector::kRegisterPercentiles = {"percentile_99"};
+const set<string> MetricsCollector::kRegisterPercentiles =
+    {"mean", "percentile_75", "percentile_95", "percentile_99"};
 
 MetricsCollector::MetricsCollector(scoped_refptr<NodesChecker> nodes_checker,
                                    scoped_refptr<ReporterBase> reporter)
@@ -244,6 +246,10 @@ Status MetricsCollector::ExtractMetricTypes(const JsonReader& r,
   for (const Value* metric : metrics) {
     string name;
     RETURN_NOT_OK(r.ExtractString(metric, "name", &name));
+    if (HasPrefixString(name, "average_")) {
+      EmplaceOrDie(metric_types, std::make_pair(name, "MEANGAUGE"));
+      continue;
+    }
     string type;
     RETURN_NOT_OK(r.ExtractString(metric, "type", &type));
     string upper_type;
@@ -656,14 +662,26 @@ Status MetricsCollector::ParseEntityMetrics(const JsonReader& r,
         auto& found_metric = FindOrDie(*merged_kv_metrics, name);
         found_metric += value;
       }
+    } else if (*known_type == "MEANGAUGE") {
+      double total_count;
+      CHECK_OK(r.ExtractDouble(metric, "total_count", &total_count));
+      double value;
+      CHECK_OK(r.ExtractDouble(metric, "value", &value));
+      vector<SimpleHistogram> tmp({{static_cast<int64_t>(total_count), value}});
+      EmplaceOrDie(hist_metrics, std::make_pair(name, tmp));
+      if (merged_hist_metrics &&
+          !EmplaceIfNotPresent(merged_hist_metrics, std::make_pair(name, tmp))) {
+        auto& found_hist_metric = FindOrDie(*merged_hist_metrics, name);
+        found_hist_metric.emplace_back(tmp[0]);
+      }
     } else if (*known_type == "HISTOGRAM") {
       for (const auto& percentile : kRegisterPercentiles) {
         string hist_metric_name(name);
         hist_metric_name += "_" + percentile;
         int64_t total_count;
         CHECK_OK(r.ExtractInt64(metric, "total_count", &total_count));
-        int64_t percentile_value;
-        CHECK_OK(r.ExtractInt64(metric, percentile.c_str(), &percentile_value));
+        double percentile_value;
+        CHECK_OK(r.ExtractDouble(metric, percentile.c_str(), &percentile_value));
         vector<SimpleHistogram> tmp({{total_count, percentile_value}});
         EmplaceOrDie(hist_metrics, std::make_pair(hist_metric_name, tmp));
         if (merged_hist_metrics &&
diff --git a/src/kudu/collector/metrics_collector.h b/src/kudu/collector/metrics_collector.h
index 2401030..5c90002 100644
--- a/src/kudu/collector/metrics_collector.h
+++ b/src/kudu/collector/metrics_collector.h
@@ -87,8 +87,8 @@ class MetricsCollector : public RefCounted<MetricsCollector> {
     int64_t count;
     // 'percentile_xxx" value in histogram metric, percentile_xxx is specified
     // by kRegisterPercentiles.
-    int64_t value;
-    SimpleHistogram(int64_t c, int64_t v) : count(c), value(v) {
+    double value;
+    SimpleHistogram(int64_t c, double v) : count(c), value(v) {
     }
     inline bool operator==(const SimpleHistogram& rhs) const {
       return count == rhs.count && value == rhs.value;
diff --git a/src/kudu/scripts/falcon_screen.json b/src/kudu/scripts/falcon_screen.json
index e3ae54c..a7046b1 100644
--- a/src/kudu/scripts/falcon_screen.json
+++ b/src/kudu/scripts/falcon_screen.json
@@ -113,27 +113,51 @@
         "metric=all_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=alter_schema_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=average_diskrowset_height service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=bloom_lookups_per_op_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=bloom_lookups_per_op_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=bloom_lookups_per_op_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=bloom_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=bloom_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=bytes_flushed service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "table_c": [
+        "metric=commit_wait_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=commit_wait_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=commit_wait_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=commit_wait_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=compact_rs_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=compact_rs_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=compact_rs_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=compact_rs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=compact_rs_running service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "table_d": [
         "metric=delta_file_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_file_lookups_per_op_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_file_lookups_per_op_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_file_lookups_per_op_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=delta_file_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_major_compact_rs_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_major_compact_rs_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_major_compact_rs_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=delta_major_compact_rs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=delta_major_compact_rs_running service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_minor_compact_rs_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_minor_compact_rs_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_minor_compact_rs_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=delta_minor_compact_rs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=delta_minor_compact_rs_running service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "table_f": [
         "metric=failed_elections_since_stable_leader service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_dms_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_dms_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_dms_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=flush_dms_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=flush_dms_running service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_mrs_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_mrs_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_mrs_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=flush_mrs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=flush_mrs_running service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=follower_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4"
@@ -142,23 +166,47 @@
         "metric=in_progress_ops service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=insertions_failed_dup_key service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=key_file_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=key_file_lookups_per_op_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=key_file_lookups_per_op_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=key_file_lookups_per_op_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=key_file_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=kudu-table-health service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "table_l": [
         "metric=leader_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_append_latency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_append_latency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_append_latency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_append_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_bytes_logged service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_cache_num_ops service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_cache_size service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_entry_batches_per_group_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_entry_batches_per_group_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_entry_batches_per_group_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_entry_batches_per_group_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_gc_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_gc_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_gc_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_gc_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_gc_running service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_group_commit_latency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_group_commit_latency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_group_commit_latency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_group_commit_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_reader_bytes_read service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_reader_entries_read service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_reader_read_batch_latency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_reader_read_batch_latency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_reader_read_batch_latency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_reader_read_batch_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_roll_latency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_roll_latency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_roll_latency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_roll_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_sync_latency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_sync_latency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_sync_latency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_sync_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=lth service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
@@ -171,8 +219,17 @@
       "table_o": [
         "metric=on_disk_data_size service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=on_disk_size service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_queue_length_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_queue_length_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_queue_length_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=op_prepare_queue_length_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_queue_time_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_queue_time_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_queue_time_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=op_prepare_queue_time_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_run_time_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_run_time_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_run_time_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=op_prepare_run_time_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=ops_behind_leader service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
@@ -192,14 +249,26 @@
         "metric=scanner_rows_returned service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=scanner_rows_scanned service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=scans_started service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=snapshot_read_inflight_wait_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=snapshot_read_inflight_wait_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=snapshot_read_inflight_wait_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=snapshot_read_inflight_wait_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=state service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "table_u": [
         "metric=undo_delta_block_estimated_retained_bytes service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=undo_delta_block_gc_bytes_deleted service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_delete_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_delete_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_delete_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=undo_delta_block_gc_delete_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_init_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_init_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_init_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=undo_delta_block_gc_init_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_perform_duration_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_perform_duration_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_perform_duration_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=undo_delta_block_gc_perform_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=undo_delta_block_gc_running service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=upserts_as_updates service=kudu cluster=${cluster.name} level=${level} v=4"
@@ -208,7 +277,13 @@
         "metric=tablet_active_scanners service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=time_since_last_leader_heartbeat service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=transaction_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=write_op_duration_client_propagated_consistency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=write_op_duration_client_propagated_consistency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=write_op_duration_client_propagated_consistency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=write_op_duration_client_propagated_consistency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=write_op_duration_commit_wait_consistency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=write_op_duration_commit_wait_consistency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=write_op_duration_commit_wait_consistency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=write_op_duration_commit_wait_consistency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=write_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4"
       ],


[kudu] 14/23: [script] add a tool for availability statistics

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit baae0c73b36b6d7eb50b9416c8f7077290a67495
Author: 张一帆 <zh...@xiaomi.com>
AuthorDate: Mon Feb 24 11:04:24 2020 +0800

    [script] add a tool for availability statistics
---
 src/kudu/scripts/kudu_availability_stat.py | 95 ++++++++++++++++++++++++++++++
 src/kudu/scripts/kudurc                    |  6 ++
 2 files changed, 101 insertions(+)

diff --git a/src/kudu/scripts/kudu_availability_stat.py b/src/kudu/scripts/kudu_availability_stat.py
new file mode 100755
index 0000000..6094228
--- /dev/null
+++ b/src/kudu/scripts/kudu_availability_stat.py
@@ -0,0 +1,95 @@
+#! /usr/bin/env python
+# coding=utf-8
+
+# A tool for availability statistics of kudu clusters
+
+# Usage example:
+# ./kudu_availability_stat.py 2020-02-02 2020-03-01 # for all prc clusters
+# ./kudu_availability_stat.py 2020-02-02 2020-03-01 c3tst-master # for a specific cluster
+
+import commands
+import datetime
+import os
+import re
+import sys
+import time
+import kudu_utils
+import yaml
+
+g_clusters_info_dict = yaml.load(open(kudu_utils.g_script_path + '/kudurc', 'r').read(), Loader=yaml.FullLoader)
+g_clusters_info = g_clusters_info_dict['clusters_info']
+g_total_count = 0
+g_success_count = 0
+
+def calc_availability_for_cluster(cluster_name, start, end):
+    kudu_utils.LOG.info('Start to collect availability statistics for cluster %s' % cluster_name)
+    global g_total_count, g_success_count
+    monitor_table = 'system.monitor'
+    # Output: (int32 total_count=xxx, int32 success_count=xxx)
+    cmd = '%s/kudu table scan @%s %s -columns=total_count,success_count ' \
+          '-predicates=[\\"AND\\",[\\"\\>=\\",\\"key\\",%s],[\\"\\<\\",\\"key\\",%s]] | grep "total_count"' \
+          % (kudu_utils.g_script_path, cluster_name, monitor_table, start, end)
+    status, output = commands.getstatusoutput(cmd)
+    if status != 0:
+        kudu_utils.LOG.fatal('Unable to execute "kudu table scan": %s', output)
+        return 0.0
+    total_count = 0
+    success_count = 0
+    for line in output.splitlines():
+        match_obj = re.search(r'int32 total_count=([0-9]+), int32 success_count=([0-9]+)', line, re.M | re.I)
+        if match_obj:
+            total_count += int(match_obj.group(1))
+            success_count += int(match_obj.group(2))
+        else:
+            kudu_utils.LOG.error('Table %s value format error, line\n%s' % (monitor_table, line))
+    kudu_utils.LOG.info('total_count: %s, success_count: %s' % (total_count, success_count))
+    g_total_count += total_count
+    g_success_count += success_count
+    if total_count == 0:
+        availability = 0.0
+    else:
+        availability = float(success_count)/total_count
+    return availability
+
+
+def calc_availability_for_clusters(cluster_name_list, start_date, end_date):
+    start_ts = int(time.mktime(datetime.datetime.strptime(start_date, "%Y-%m-%d").timetuple()))
+    end_ts = int(time.mktime(datetime.datetime.strptime(end_date, "%Y-%m-%d").timetuple()))
+    print("%-30s%-12s" % ("cluster", "availability"))
+    for cluster_name in cluster_name_list:
+        availability = calc_availability_for_cluster(cluster_name, start_ts, end_ts)
+        print("%-30s%-12.6f" % (cluster_name, availability))
+    if g_total_count == 0:
+        total_availability = 0.0
+    else:
+        total_availability = float(g_success_count)/g_total_count
+    print("%-30s%-12.6f" % ("total", total_availability))
+
+
+def main(argv=None):
+    if argv is None:
+        argv = sys.argv
+
+    cluster_name_list = []
+    if len(argv) == 3:
+        # Calculate all clusters except public_share cluster
+        for cluster in g_clusters_info.iterkeys():
+            cluster_info = g_clusters_info[cluster]
+            if cluster_info['charge_type'] == 'public_share':
+                kudu_utils.LOG.warning('Ignore public_share cluster %s' % cluster)
+            else:
+                cluster_name_list.append(cluster)
+    elif len(argv) == 4:
+        # Calculate specified cluster
+        cluster_name_list.append(argv[3])
+    else:
+        kudu_utils.LOG.fatal('Usage: %s <start_date> <end_date> [cluster_name]' % argv[0])
+        return
+
+    start_date = argv[1]
+    end_date = argv[2]
+    calc_availability_for_clusters(cluster_name_list, start_date, end_date)
+
+
+if __name__ == "__main__":
+    main()
diff --git a/src/kudu/scripts/kudurc b/src/kudu/scripts/kudurc
index 6b28831..f017245 100644
--- a/src/kudu/scripts/kudurc
+++ b/src/kudu/scripts/kudurc
@@ -67,3 +67,9 @@ clusters_info:
     charge_type: public_share
     instance: HDD
     master_addresses: c3-hadoop-kudu-prc-ct01.bj:18000,c3-hadoop-kudu-prc-ct02.bj:18000,c3-hadoop-kudu-prc-ct03.bj:18000
+  c3tst-master:
+    olap_version: 2
+    region: chnbj-idc
+    charge_type: public_share
+    instance: HDD
+    master_addresses: c3-hadoop-srv-st2270.bj:16000,c3-hadoop-srv-st2271.bj:16000,c3-hadoop-kudu-prc-ct01.bj:16000


[kudu] 06/23: [master] reserve table for a period time after being deleted

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit cdb3e186c4ee8fd8e484cd30acb7a757734da420
Author: Yingchun Lai <40...@qq.com>
AuthorDate: Mon Sep 23 21:46:51 2019 +0800

    [master] reserve table for a period time after being deleted
    
    Change-Id: I2df75bdfb3288e6cfe9a14714db7eabe046ba577
---
 src/kudu/client/client-internal.cc             |  24 ++-
 src/kudu/client/client-internal.h              |   9 +-
 src/kudu/client/client-test.cc                 | 207 +++++++++++++++++++++++++
 src/kudu/client/client.cc                      |  24 ++-
 src/kudu/client/client.h                       |  35 ++++-
 src/kudu/client/master_proxy_rpc.cc            |   3 +
 src/kudu/client/table_alterer-internal.cc      |   1 +
 src/kudu/client/table_alterer-internal.h       |   2 +
 src/kudu/common/common.proto                   |   3 +
 src/kudu/common/wire_protocol.cc               |  56 +++++--
 src/kudu/common/wire_protocol.h                |  12 +-
 src/kudu/integration-tests/alter_table-test.cc |   2 +-
 src/kudu/master/catalog_manager.cc             | 109 +++++++++++--
 src/kudu/master/catalog_manager.h              |  31 +++-
 src/kudu/master/master-test.cc                 |  90 +++++++++++
 src/kudu/master/master.cc                      |  58 +++++++
 src/kudu/master/master.h                       |  13 ++
 src/kudu/master/master.proto                   |  21 +++
 src/kudu/master/master_service.cc              |  76 ++++++++-
 src/kudu/master/master_service.h               |   6 +
 src/kudu/server/server_base.cc                 |   4 +-
 src/kudu/server/server_base.h                  |   3 +-
 src/kudu/tools/kudu-admin-test.cc              |   4 +-
 src/kudu/tools/kudu-tool-test.cc               |  53 +++++++
 src/kudu/tools/tool_action_table.cc            |  37 ++++-
 25 files changed, 839 insertions(+), 44 deletions(-)

diff --git a/src/kudu/client/client-internal.cc b/src/kudu/client/client-internal.cc
index 9243836..06b303a 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -103,6 +103,8 @@ using master::IsCreateTableDoneRequestPB;
 using master::IsCreateTableDoneResponsePB;
 using master::ListTabletServersResponsePB;
 using master::ListTabletServersRequestPB;
+using master::RecallDeletedTableRequestPB;
+using master::RecallDeletedTableResponsePB;
 using master::MasterFeatures;
 using master::MasterServiceProxy;
 using master::TableIdentifierPB;
@@ -346,12 +348,16 @@ Status KuduClient::Data::WaitForCreateTableToFinish(
 Status KuduClient::Data::DeleteTable(KuduClient* client,
                                      const string& table_name,
                                      const MonoTime& deadline,
-                                     bool modify_external_catalogs) {
+                                     bool modify_external_catalogs,
+                                     bool force_on_trashed_table,
+                                     uint32_t reserve_seconds) {
   DeleteTableRequestPB req;
   DeleteTableResponsePB resp;
 
   req.mutable_table()->set_table_name(table_name);
   req.set_modify_external_catalogs(modify_external_catalogs);
+  req.set_force_on_trashed_table(force_on_trashed_table);
+  req.set_reserve_seconds(reserve_seconds);
   Synchronizer sync;
   AsyncLeaderMasterRpc<DeleteTableRequestPB, DeleteTableResponsePB> rpc(
       deadline, client, BackoffType::EXPONENTIAL, req, &resp,
@@ -360,6 +366,22 @@ Status KuduClient::Data::DeleteTable(KuduClient* client,
   return sync.Wait();
 }
 
+Status KuduClient::Data::RecallTable(KuduClient* client,
+                                     const std::string& table_name,
+                                     const MonoTime& deadline) {
+  RecallDeletedTableRequestPB req;
+  RecallDeletedTableResponsePB resp;
+
+  req.mutable_table()->set_table_name(table_name);
+  Synchronizer sync;
+  AsyncLeaderMasterRpc<RecallDeletedTableRequestPB, RecallDeletedTableResponsePB> rpc(
+      deadline, client, BackoffType::EXPONENTIAL, req, &resp,
+      &MasterServiceProxy::RecallDeletedTableAsync, "RecallDeletedTable", sync.AsStatusCallback(),
+      {});
+  rpc.SendRpc();
+  return sync.Wait();
+}
+
 Status KuduClient::Data::AlterTable(KuduClient* client,
                                     const AlterTableRequestPB& req,
                                     AlterTableResponsePB* resp,
diff --git a/src/kudu/client/client-internal.h b/src/kudu/client/client-internal.h
index bf5b3ce..774eb5d 100644
--- a/src/kudu/client/client-internal.h
+++ b/src/kudu/client/client-internal.h
@@ -115,7 +115,14 @@ class KuduClient::Data {
   static Status DeleteTable(KuduClient* client,
                             const std::string& table_name,
                             const MonoTime& deadline,
-                            bool modify_external_catalogs = true);
+                            bool modify_external_catalogs = true,
+                            bool force_on_trashed_table = false,
+                            uint32_t reserve_seconds = 0);
+
+
+  static Status RecallTable(KuduClient* client,
+                     const std::string& table_name,
+                     const MonoTime& deadline);
 
   static Status AlterTable(KuduClient* client,
                            const master::AlterTableRequestPB& req,
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index 6961ff3..5499c9b 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -132,6 +132,7 @@ DECLARE_bool(master_support_connect_to_master_rpc);
 DECLARE_bool(mock_table_metrics_for_testing);
 DECLARE_bool(rpc_trace_negotiation);
 DECLARE_bool(scanner_inject_service_unavailable_on_continue_scan);
+DECLARE_int32(check_outdated_table_interval_seconds);
 DECLARE_int32(flush_threshold_mb);
 DECLARE_int32(flush_threshold_secs);
 DECLARE_int32(heartbeat_interval_ms);
@@ -215,6 +216,7 @@ class ClientTest : public KuduTest {
     // Reduce the TS<->Master heartbeat interval
     FLAGS_heartbeat_interval_ms = 10;
     FLAGS_scanner_gc_check_interval_us = 50 * 1000; // 50 milliseconds.
+    FLAGS_check_outdated_table_interval_seconds = 1;
 
     SetLocationMappingCmd();
 
@@ -4250,6 +4252,19 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
     ASSERT_NE(boost::none, tablet_replica->tablet()->metadata()->extra_config());
     ASSERT_FALSE(tablet_replica->tablet()->metadata()->extra_config()->has_history_max_age_sec());
   }
+  // 4. Try to alter internal config.
+  {
+    map<string, string> extra_configs;
+    extra_configs[kTableConfigReserveSeconds] = "60";
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+    table_alterer->AlterExtraConfig(extra_configs);
+    Status s = table_alterer->Alter();
+    ASSERT_TRUE(s.IsInvalidArgument());
+    ASSERT_STR_CONTAINS(s.ToString(), "forbidden to change internal extra configuration by user");
+    ASSERT_EQ(11, tablet_replica->tablet()->metadata()->schema_version());
+    ASSERT_NE(boost::none, tablet_replica->tablet()->metadata()->extra_config());
+    ASSERT_FALSE(tablet_replica->tablet()->metadata()->extra_config()->has_reserve_seconds());
+  }
 
   // Test changing a table name.
   {
@@ -4320,6 +4335,198 @@ TEST_F(ClientTest, TestDeleteTable) {
   NO_FATALS(InsertTestRows(client_.get(), client_table_.get(), 10));
 }
 
+TEST_F(ClientTest, TestDeleteAndReserveTable) {
+  // Open the table before deleting it.
+  ASSERT_OK(client_->OpenTable(kTableName, &client_table_));
+
+  // Insert a few rows, and scan them back. This is to populate the MetaCache.
+  NO_FATALS(InsertTestRows(client_.get(), client_table_.get(), 10));
+  vector<string> rows;
+  ScanTableToStrings(client_table_.get(), &rows);
+  ASSERT_EQ(10, rows.size());
+
+  // Remove the table.
+  // NOTE that it returns when the operation is completed on the master side
+  string tablet_id = GetFirstTabletId(client_table_.get());
+  ASSERT_OK(client_->DeleteTable(kTableName, false, 60));
+  CatalogManager* catalog_manager = cluster_->mini_master()->master()->catalog_manager();
+  {
+    CatalogManager::ScopedLeaderSharedLock l(catalog_manager);
+    ASSERT_OK(l.first_failed_status());
+    bool exists;
+    ASSERT_OK(catalog_manager->TableNameExists(kTableName, &exists));
+    ASSERT_FALSE(exists);
+  }
+
+  // Exist tablet is still visible.
+  scoped_refptr<TabletReplica> tablet_replica;
+  ASSERT_TRUE(cluster_->mini_tablet_server(0)->server()->tablet_manager()->LookupTablet(
+                  tablet_id, &tablet_replica));
+
+  // Try to open the deleted table.
+  Status s = client_->OpenTable(kTableName, &client_table_);
+  ASSERT_TRUE(s.IsNotFound());
+  ASSERT_STR_CONTAINS(s.ToString(), "the table does not exist");
+
+  // Old table has been renamed.
+  vector<string> tables;
+  ASSERT_OK(client_->ListTables(&tables));
+  ASSERT_EQ(1, tables.size());
+  string trashed_table_name = tables[0];
+  string origin_table_name;
+  WallTime mark_delete_time;
+  ASSERT_TRUE(catalog_manager->GetOriginNameAndDeleteTimeOfTrashedTable(trashed_table_name,
+                                                                        &origin_table_name,
+                                                                        &mark_delete_time));
+  ASSERT_EQ(string(kTableName), origin_table_name);
+
+  // Alter trashed table is not allowed.
+  {
+    // Not allowed to rename.
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(trashed_table_name));
+    table_alterer->RenameTo(kTableName);
+    s = table_alterer->Alter();
+    ASSERT_TRUE(s.IsInvalidArgument());
+    ASSERT_STR_CONTAINS(s.ToString(), Substitute("trashed table $0 should not be altered",
+                                                 trashed_table_name));
+  }
+
+  {
+    // Not allowed to add column.
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(trashed_table_name));
+    table_alterer->AddColumn("new_column")->Type(KuduColumnSchema::INT32);
+    s = table_alterer->Alter();
+    ASSERT_TRUE(s.IsInvalidArgument());
+    ASSERT_STR_CONTAINS(s.ToString(), Substitute("trashed table $0 should not be altered",
+                                                 trashed_table_name));
+  }
+
+  {
+    // Not allowed to delete.
+    s = client_->DeleteTable(trashed_table_name);
+    ASSERT_TRUE(s.IsInvalidArgument());
+    ASSERT_STR_CONTAINS(s.ToString(), Substitute("trashed table $0 should not be deleted",
+                                                 trashed_table_name));
+  }
+
+  {
+    // Not allowed to set extra configs.
+    map<string, string> extra_configs;
+    extra_configs[kTableMaintenancePriority] = "3";
+    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(trashed_table_name));
+    table_alterer->AlterExtraConfig(extra_configs);
+    s = table_alterer->Alter();
+    ASSERT_TRUE(s.IsInvalidArgument());
+    ASSERT_STR_CONTAINS(s.ToString(), Substitute("trashed table $0 should not be altered",
+                                                 trashed_table_name));
+
+    // Alter trashed table is allowed on force.
+    table_alterer->force_on_trashed_table(true);
+    ASSERT_OK(table_alterer->Alter());
+  }
+
+  {
+    // Write and read are allowed for trashed table.
+    NO_FATALS(InsertTestRows(client_.get(), client_table_.get(), 20, 10));
+    ScanTableToStrings(client_table_.get(), &rows);
+    ASSERT_EQ(30, rows.size());
+  }
+
+  // Create a new table with the same name.
+  NO_FATALS(CreateTable(kTableName, 1, GenerateSplitRows(), {}, &client_table_));
+
+  // Two tables exist now.
+  tables.clear();
+  ASSERT_OK(client_->ListTables(&tables));
+  ASSERT_EQ(2, tables.size());
+  std::sort(tables.begin(), tables.end());
+  ASSERT_EQ(string(kTableName), tables[0]);
+  ASSERT_EQ(string(trashed_table_name), tables[1]);
+
+  // Should be able to insert successfully into the new table.
+  NO_FATALS(InsertTestRows(client_.get(), client_table_.get(), 10));
+  ScanTableToStrings(client_table_.get(), &rows);
+  ASSERT_EQ(10, rows.size());
+
+  // Force to delete the trashed table.
+  ASSERT_OK(client_->DeleteTable(trashed_table_name, true));
+
+  // Only one table left.
+  tables.clear();
+  ASSERT_OK(client_->ListTables(&tables));
+  ASSERT_EQ(1, tables.size());
+  ASSERT_EQ(kTableName, tables[0]);
+}
+
+TEST_F(ClientTest, TestDeleteAndRecallTable) {
+  // Open the table before deleting it.
+  ASSERT_OK(client_->OpenTable(kTableName, &client_table_));
+
+  // Insert a few rows, and scan them back. This is to populate the MetaCache.
+  NO_FATALS(InsertTestRows(client_.get(), client_table_.get(), 10));
+  vector<string> rows;
+  ScanTableToStrings(client_table_.get(), &rows);
+  ASSERT_EQ(10, rows.size());
+
+  // Remove the table
+  ASSERT_OK(client_->DeleteTable(kTableName, false, 60));
+  ASSERT_EVENTUALLY([&] () {
+    Status s = client_->OpenTable(kTableName, &client_table_);
+    ASSERT_TRUE(s.IsNotFound());
+    ASSERT_STR_CONTAINS(s.ToString(), "the table does not exist");
+  });
+
+  // Recall and reopen table.
+  vector<string> tables;
+  ASSERT_OK(client_->ListTables(&tables));
+  ASSERT_EQ(1, tables.size());
+  ASSERT_OK(client_->RecallTable(tables[0]));
+  ASSERT_OK(client_->OpenTable(kTableName, &client_table_));
+
+  // Check data from table.
+  ScanTableToStrings(client_table_.get(), &rows);
+  ASSERT_EQ(10, rows.size());
+}
+
+TEST_F(ClientTest, TestDeleteAndRecallAfterReserveTimeTable) {
+  // Open the table before deleting it.
+  ASSERT_OK(client_->OpenTable(kTableName, &client_table_));
+
+  // Insert a few rows, and scan them back. This is to populate the MetaCache.
+  NO_FATALS(InsertTestRows(client_.get(), client_table_.get(), 10));
+  vector<string> rows;
+  ScanTableToStrings(client_table_.get(), &rows);
+  ASSERT_EQ(10, rows.size());
+
+  // Remove the table
+  ASSERT_OK(client_->DeleteTable(kTableName, false, 2));
+  ASSERT_EVENTUALLY([&] () {
+    Status s = client_->OpenTable(kTableName, &client_table_);
+    ASSERT_TRUE(s.IsNotFound());
+    ASSERT_STR_CONTAINS(s.ToString(), "the table does not exist");
+  });
+
+  vector<string> tables;
+  ASSERT_OK(client_->ListTables(&tables));
+  ASSERT_EQ(1, tables.size());
+
+  // Wait util the table is removed completely.
+  ASSERT_EVENTUALLY([&] () {
+    Status s = client_->OpenTable(tables[0], &client_table_);
+    ASSERT_TRUE(s.IsNotFound());
+    ASSERT_STR_CONTAINS(s.ToString(), "the table does not exist");
+  });
+
+  // Try to recall the table.
+  Status s = client_->RecallTable(tables[0]);
+  ASSERT_TRUE(s.IsNotFound());
+  ASSERT_STR_CONTAINS(s.ToString(), "the table does not exist");
+
+  tables.clear();
+  ASSERT_OK(client_->ListTables(&tables));
+  ASSERT_TRUE(tables.empty());
+}
+
 TEST_F(ClientTest, TestGetTableSchema) {
   KuduSchema schema;
 
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index d4b607f..6104ca9 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -411,14 +411,24 @@ Status KuduClient::IsCreateTableInProgress(const string& table_name,
                                         create_in_progress);
 }
 
-Status KuduClient::DeleteTable(const string& table_name) {
-  return DeleteTableInCatalogs(table_name, true);
+Status KuduClient::DeleteTable(const string& table_name,
+                               bool force_on_trashed_table,
+                               uint32_t reserve_seconds) {
+  return DeleteTableInCatalogs(table_name, true, force_on_trashed_table, reserve_seconds);
 }
 
 Status KuduClient::DeleteTableInCatalogs(const string& table_name,
-                                         bool modify_external_catalogs) {
+                                         bool modify_external_catalogs,
+                                         bool force_on_trashed_table,
+                                         uint32_t reserve_seconds) {
   MonoTime deadline = MonoTime::Now() + default_admin_operation_timeout();
-  return data_->DeleteTable(this, table_name, deadline, modify_external_catalogs);
+  return data_->DeleteTable(this, table_name, deadline,
+      modify_external_catalogs, force_on_trashed_table, reserve_seconds);
+}
+
+Status KuduClient::RecallTable(const string& table_name) {
+  MonoTime deadline = MonoTime::Now() + default_admin_operation_timeout();
+  return data_->RecallTable(this, table_name, deadline);
 }
 
 KuduTableAlterer* KuduClient::NewTableAlterer(const string& table_name) {
@@ -1369,6 +1379,12 @@ KuduTableAlterer* KuduTableAlterer::modify_external_catalogs(
   return this;
 }
 
+KuduTableAlterer* KuduTableAlterer::force_on_trashed_table(
+    bool force_on_trashed_table) {
+  data_->force_on_trashed_table_ = force_on_trashed_table;
+  return this;
+}
+
 Status KuduTableAlterer::Alter() {
   AlterTableRequestPB req;
   AlterTableResponsePB resp;
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index f8d16d8..2e54947 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -349,8 +349,14 @@ class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
   ///
   /// @param [in] table_name
   ///   Name of the table to drop.
+  /// @param [in] force_on_trashed_table
+  ///   Whether to force to delete a trashed table.
+  /// @param [in] reserve_seconds
+  ///   Reserve seconds after being deleted.
   /// @return Operation status.
-  Status DeleteTable(const std::string& table_name);
+  Status DeleteTable(const std::string& table_name,
+                     bool force_on_trashed_table = false,
+                     uint32_t reserve_seconds = 0);
 
   /// @cond PRIVATE_API
 
@@ -363,9 +369,23 @@ class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
   /// @param [in] modify_external_catalogs
   ///   Whether to apply the deletion to external catalogs, such as the Hive Metastore,
   ///   which the Kudu master has been configured to integrate with.
+  /// @param [in] force_on_trashed_table
+  ///   Whether to force to delete a trashed table.
+  /// @param [in] reserve_seconds
+  ///   Reserve seconds after being deleted.
   /// @return Operation status.
   Status DeleteTableInCatalogs(const std::string& table_name,
-                               bool modify_external_catalogs) KUDU_NO_EXPORT;
+                               bool modify_external_catalogs,
+                               bool force_on_trashed_table = false,
+                               uint32_t reserve_seconds = 0) KUDU_NO_EXPORT;
+
+  /// Recall a deleted but still reserved table.
+  ///
+  /// @param [in] table_name
+  ///   Name of the table to recall.
+  /// @return Operation status.
+  Status RecallTable(const std::string& table_name);
+
   /// @endcond
 
   /// Create a KuduTableAlterer object.
@@ -1464,6 +1484,17 @@ class KUDU_EXPORT KuduTableAlterer {
   /// @return Raw pointer to this alterer object.
   KuduTableAlterer* modify_external_catalogs(bool modify_external_catalogs) KUDU_NO_EXPORT;
 
+  /// @cond PRIVATE_API
+
+  /// Force to alter a trashed table.
+  ///
+  /// Private API.
+  ///
+  /// @param [in] force_on_trashed_table
+  ///   Whether to alter on a trashed table.
+  /// @return Raw pointer to this alterer object.
+  KuduTableAlterer* force_on_trashed_table(bool force_on_trashed_table) KUDU_NO_EXPORT;
+
   /// @endcond
 
   /// @return Status of the ALTER TABLE operation. The return value
diff --git a/src/kudu/client/master_proxy_rpc.cc b/src/kudu/client/master_proxy_rpc.cc
index 815482d..e8ed376 100644
--- a/src/kudu/client/master_proxy_rpc.cc
+++ b/src/kudu/client/master_proxy_rpc.cc
@@ -74,6 +74,8 @@ using master::ListTabletServersRequestPB;
 using master::ListTabletServersResponsePB;
 using master::MasterServiceProxy;
 using master::MasterErrorPB;
+using master::RecallDeletedTableRequestPB;
+using master::RecallDeletedTableResponsePB;
 using master::ReplaceTabletRequestPB;
 using master::ReplaceTabletResponsePB;
 using rpc::BackoffType;
@@ -296,6 +298,7 @@ template class AsyncLeaderMasterRpc<GetTableStatisticsRequestPB, GetTableStatist
 template class AsyncLeaderMasterRpc<ListTablesRequestPB, ListTablesResponsePB>;
 template class AsyncLeaderMasterRpc<ListTabletServersRequestPB, ListTabletServersResponsePB>;
 template class AsyncLeaderMasterRpc<ListMastersRequestPB, ListMastersResponsePB>;
+template class AsyncLeaderMasterRpc<RecallDeletedTableRequestPB, RecallDeletedTableResponsePB>;
 template class AsyncLeaderMasterRpc<ReplaceTabletRequestPB, ReplaceTabletResponsePB>;
 
 } // namespace internal
diff --git a/src/kudu/client/table_alterer-internal.cc b/src/kudu/client/table_alterer-internal.cc
index 846150a..290d24e 100644
--- a/src/kudu/client/table_alterer-internal.cc
+++ b/src/kudu/client/table_alterer-internal.cc
@@ -63,6 +63,7 @@ Status KuduTableAlterer::Data::ToRequest(AlterTableRequestPB* req) {
 
   req->Clear();
   req->set_modify_external_catalogs(modify_external_catalogs_);
+  req->set_force_on_trashed_table(force_on_trashed_table_);
   req->mutable_table()->set_table_name(table_name_);
   if (rename_to_) {
     req->set_new_table_name(rename_to_.get());
diff --git a/src/kudu/client/table_alterer-internal.h b/src/kudu/client/table_alterer-internal.h
index 2cfa1de..1cb074a 100644
--- a/src/kudu/client/table_alterer-internal.h
+++ b/src/kudu/client/table_alterer-internal.h
@@ -87,6 +87,8 @@ class KuduTableAlterer::Data {
   // Metastore. The default value is true.
   bool modify_external_catalogs_ = true;
 
+  bool force_on_trashed_table_ = false;
+
  private:
   DISALLOW_COPY_AND_ASSIGN(Data);
 };
diff --git a/src/kudu/common/common.proto b/src/kudu/common/common.proto
index 559cd28..882e16c 100644
--- a/src/kudu/common/common.proto
+++ b/src/kudu/common/common.proto
@@ -446,4 +446,7 @@ message TableExtraConfigPB {
   // range [-FLAGS_max_priority_range, FLAGS_max_priority_range] when
   // calculate maintenance priority score.
   optional int32 maintenance_priority = 2;
+
+  // Reserve seconds after the table has been deleted.
+  optional uint32 reserve_seconds = 3;
 }
diff --git a/src/kudu/common/wire_protocol.cc b/src/kudu/common/wire_protocol.cc
index ef14d8b..821dbec 100644
--- a/src/kudu/common/wire_protocol.cc
+++ b/src/kudu/common/wire_protocol.cc
@@ -23,6 +23,7 @@
 #include <cstdint>
 #include <cstring>
 #include <ostream>
+#include <set>
 #include <string>
 #include <vector>
 
@@ -41,6 +42,7 @@
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/consensus/metadata.pb.h"
 #include "kudu/gutil/fixedarray.h"
+#include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/fastmem.h"
 #include "kudu/gutil/strings/numbers.h"
@@ -66,6 +68,7 @@ using google::protobuf::RepeatedPtrField;
 using kudu::pb_util::SecureDebugString;
 using kudu::pb_util::SecureShortDebugString;
 using std::map;
+using std::set;
 using std::string;
 using std::vector;
 using strings::Substitute;
@@ -646,8 +649,6 @@ Status ColumnPredicateFromPB(const Schema& schema,
   return Status::OK();
 }
 
-const char kTableHistoryMaxAgeSec[] = "kudu.table.history_max_age_sec";
-const char kTableMaintenancePriority[] = "kudu.table.maintenance_priority";
 Status ExtraConfigPBToMap(const TableExtraConfigPB& pb, map<string, string>* configs) {
   Map<string, string> tmp;
   RETURN_NOT_OK(ExtraConfigPBToPBMap(pb, &tmp));
@@ -664,28 +665,61 @@ Status ParseInt32Config(const string& name, const string& value, int32_t* result
   return Status::OK();
 }
 
-Status ExtraConfigPBFromPBMap(const Map<string, string>& configs, TableExtraConfigPB* pb) {
-  TableExtraConfigPB result;
-  for (const auto& config : configs) {
+Status ParseUint32Config(const string& name, const string& value, uint32_t* result) {
+  CHECK(result);
+  if (!safe_strtou32(value, result)) {
+    return Status::InvalidArgument(Substitute("unable to parse $0", name), value);
+  }
+  return Status::OK();
+}
+
+Status UpdateExtraConfigPB(const Map<string, string>& new_extra_configs,
+                           bool external_request,
+                           TableExtraConfigPB* pb) {
+  static const set<string> kSupportedConfigs({kTableHistoryMaxAgeSec,
+                                              kTableMaintenancePriority,
+                                              kTableConfigReserveSeconds});
+  static const set<string> kInternalConfigs({kTableConfigReserveSeconds});
+  for (const auto& config : new_extra_configs) {
     const string& name = config.first;
     const string& value = config.second;
+    if (!ContainsKey(kSupportedConfigs, name)) {
+      return Status::InvalidArgument(
+        Substitute("invalid extra configuration property: $0", name));
+    }
+    if (external_request && ContainsKey(kInternalConfigs, name)) {
+      return Status::InvalidArgument(
+        Substitute("forbidden to change internal extra configuration by user, property: $0", name));
+    }
+
     if (name == kTableHistoryMaxAgeSec) {
       if (!value.empty()) {
         int32_t history_max_age_sec;
         RETURN_NOT_OK(ParseInt32Config(name, value, &history_max_age_sec));
-        result.set_history_max_age_sec(history_max_age_sec);
+        pb->set_history_max_age_sec(history_max_age_sec);
+      } else {
+        pb->clear_history_max_age_sec();
       }
     } else if (name == kTableMaintenancePriority) {
       if (!value.empty()) {
         int32_t maintenance_priority;
         RETURN_NOT_OK(ParseInt32Config(name, value, &maintenance_priority));
-        result.set_maintenance_priority(maintenance_priority);
+        pb->set_maintenance_priority(maintenance_priority);
+      } else {
+        pb->clear_maintenance_priority();
+      }
+    } else if (name == kTableConfigReserveSeconds) {
+      if (!value.empty()) {
+        uint32_t reserve_seconds;
+        RETURN_NOT_OK(ParseUint32Config(name, value, &reserve_seconds));
+        pb->set_reserve_seconds(reserve_seconds);
+      } else {
+        pb->clear_reserve_seconds();
       }
     } else {
-      LOG(WARNING) << "Unknown extra configuration property: " << name;
+      LOG(FATAL) << "unparsed property: " << name;
     }
   }
-  *pb = std::move(result);
   return Status::OK();
 }
 
@@ -697,6 +731,10 @@ Status ExtraConfigPBToPBMap(const TableExtraConfigPB& pb, Map<string, string>* c
   if (pb.has_maintenance_priority()) {
     result[kTableMaintenancePriority] = std::to_string(pb.maintenance_priority());
   }
+  if (pb.has_reserve_seconds()) {
+    result[kTableConfigReserveSeconds] = std::to_string(pb.reserve_seconds());
+  }
+
   *configs = std::move(result);
   return Status::OK();
 }
diff --git a/src/kudu/common/wire_protocol.h b/src/kudu/common/wire_protocol.h
index 7ac76fd..d14d6a9 100644
--- a/src/kudu/common/wire_protocol.h
+++ b/src/kudu/common/wire_protocol.h
@@ -62,6 +62,10 @@ class ServerEntryPB;
 class ServerRegistrationPB;
 class TableExtraConfigPB;
 
+static const std::string kTableHistoryMaxAgeSec = "kudu.table.history_max_age_sec";
+static const std::string kTableMaintenancePriority = "kudu.table.maintenance_priority";
+static const std::string kTableConfigReserveSeconds = "kudu.table.reserve_seconds";
+
 // Convert the given C++ Status object into the equivalent Protobuf.
 void StatusToPB(const Status& status, AppStatusPB* pb);
 
@@ -147,9 +151,11 @@ Status ColumnPredicateFromPB(const Schema& schema,
 Status ExtraConfigPBToMap(const TableExtraConfigPB& pb,
                           std::map<std::string, std::string>* configs);
 
-// Convert the table's extra configuration protobuf::map to protobuf.
-Status ExtraConfigPBFromPBMap(const google::protobuf::Map<std::string, std::string>& configs,
-                              TableExtraConfigPB* pb);
+// Update or insert the table's extra configuration according to protobuf::map.
+Status UpdateExtraConfigPB(
+    const google::protobuf::Map<std::string, std::string>& new_extra_configs,
+    bool external_request,
+    TableExtraConfigPB* pb);
 
 // Parse int32_t type value from 'value', and store in 'result' when succeed.
 Status ParseInt32Config(const std::string& name, const std::string& value, int32_t* result);
diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc
index 714e923..99cbd2e 100644
--- a/src/kudu/integration-tests/alter_table-test.cc
+++ b/src/kudu/integration-tests/alter_table-test.cc
@@ -359,7 +359,7 @@ TEST_F(AlterTableTest, TestAddNotNullableColumnWithoutDefaults) {
         cluster_->mini_master()->master()->catalog_manager();
     master::CatalogManager::ScopedLeaderSharedLock l(catalog);
     ASSERT_OK(l.first_failed_status());
-    Status s = catalog->AlterTableRpc(req, &resp, /*rpc=*/nullptr);
+    Status s = catalog->AlterTableRpc(req, &resp, /*rpc=*/nullptr, true);
     ASSERT_TRUE(s.IsInvalidArgument());
     ASSERT_STR_CONTAINS(s.ToString(), "column `c2`: NOT NULL columns must have a default");
   }
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index af1b5f1..6aefb42 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -88,10 +88,11 @@
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/escaping.h"
 #include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.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/hms/hms_catalog.h"
 #include "kudu/master/authz_provider.h"
 #include "kudu/master/auto_rebalancer.h"
@@ -358,6 +359,7 @@ using kudu::consensus::RaftPeerPB;
 using kudu::consensus::StartTabletCopyRequestPB;
 using kudu::consensus::kMinimumTerm;
 using kudu::hms::HmsClientVerifyKuduSyncConfig;
+using kudu::master::TableIdentifierPB;
 using kudu::pb_util::SecureDebugString;
 using kudu::pb_util::SecureShortDebugString;
 using kudu::rpc::RpcContext;
@@ -1689,7 +1691,7 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
 
   // Verify the table's extra configuration properties.
   TableExtraConfigPB extra_config_pb;
-  RETURN_NOT_OK(ExtraConfigPBFromPBMap(req.extra_configs(), &extra_config_pb));
+  RETURN_NOT_OK(UpdateExtraConfigPB(req.extra_configs(), true, &extra_config_pb));
 
   scoped_refptr<TableInfo> table;
   {
@@ -2175,6 +2177,32 @@ Status CatalogManager::DeleteTable(const DeleteTableRequestPB& req,
   return Status::OK();
 }
 
+Status CatalogManager::RecallDeletedTableRpc(const RecallDeletedTableRequestPB& req,
+                                             RecallDeletedTableResponsePB* resp,
+                                             rpc::RpcContext* rpc) {
+  leader_lock_.AssertAcquiredForReading();
+  RETURN_NOT_OK(CheckOnline());
+
+  string origin_table_name;
+  WallTime mark_delete_time;
+  if (!GetOriginNameAndDeleteTimeOfTrashedTable(req.table().table_name(),
+                                                &origin_table_name, &mark_delete_time)) {
+    return SetupError(Status::InvalidArgument("not a trashed table"),
+                      resp, MasterErrorPB::TABLE_NOT_FOUND);
+  }
+
+  AlterTableRequestPB alter_req;
+  alter_req.mutable_table()->CopyFrom(req.table());
+  // Revert table name
+  alter_req.set_new_table_name(origin_table_name);
+  (*alter_req.mutable_new_extra_configs())[kTableMaintenancePriority] = "";
+  (*alter_req.mutable_new_extra_configs())[kTableConfigReserveSeconds] = "";
+
+  AlterTableResponsePB alter_resp;
+  RETURN_NOT_OK(AlterTableRpc(alter_req, &alter_resp, rpc, false));
+  return Status::OK();
+}
+
 Status CatalogManager::ApplyAlterSchemaSteps(const SysTablesEntryPB& current_pb,
                                              vector<AlterTableRequestPB::Step> steps,
                                              Schema* new_schema,
@@ -2434,7 +2462,8 @@ Status CatalogManager::ApplyAlterPartitioningSteps(
 
 Status CatalogManager::AlterTableRpc(const AlterTableRequestPB& req,
                                      AlterTableResponsePB* resp,
-                                     rpc::RpcContext* rpc) {
+                                     rpc::RpcContext* rpc,
+                                     bool external_request) {
   leader_lock_.AssertAcquiredForReading();
   RETURN_NOT_OK(CheckOnline());
 
@@ -2514,10 +2543,11 @@ Status CatalogManager::AlterTableRpc(const AlterTableRequestPB& req,
 
     return AlterTable(r, resp,
                       /*hms_notification_log_event_id=*/none,
-                      /*user=*/none);
+                      /*user=*/none,
+                      external_request);
   }
 
-  return AlterTable(req, resp, /*hms_notification_log_event_id=*/ none, user);
+  return AlterTable(req, resp, /*hms_notification_log_event_id=*/ none, user, external_request);
 }
 
 Status CatalogManager::RenameTableHms(const string& table_id,
@@ -2533,7 +2563,8 @@ Status CatalogManager::RenameTableHms(const string& table_id,
   // Use empty user to skip the authorization validation since the operation
   // originates from catalog manager. Moreover, this avoids duplicate effort,
   // because we already perform authorization before making any changes to the HMS.
-  RETURN_NOT_OK(AlterTable(req, &resp, notification_log_event_id, /*user=*/none));
+  RETURN_NOT_OK(AlterTable(req, &resp, notification_log_event_id, /*user=*/none,
+                           /*external_request=*/true));
 
   // Update the cached HMS notification log event ID.
   DCHECK_GT(notification_log_event_id, hms_notification_log_event_id_);
@@ -2545,7 +2576,8 @@ Status CatalogManager::RenameTableHms(const string& table_id,
 Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
                                   AlterTableResponsePB* resp,
                                   optional<int64_t> hms_notification_log_event_id,
-                                  optional<const string&> user) {
+                                  optional<const string&> user,
+                                  bool external_request) {
   leader_lock_.AssertAcquiredForReading();
   RETURN_NOT_OK(CheckOnline());
 
@@ -2684,14 +2716,12 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB& req,
   if (!req.new_extra_configs().empty()) {
     TRACE("Apply alter extra-config");
     Map<string, string> new_extra_configs;
-    RETURN_NOT_OK(ExtraConfigPBToPBMap(l.data().pb.extra_config(),
-                                       &new_extra_configs));
-    // Merge table's extra configuration properties.
     for (auto config : req.new_extra_configs()) {
       new_extra_configs[config.first] = config.second;
     }
-    RETURN_NOT_OK(ExtraConfigPBFromPBMap(new_extra_configs,
-                                         l.mutable_data()->pb.mutable_extra_config()));
+    RETURN_NOT_OK(UpdateExtraConfigPB(new_extra_configs,
+                                      external_request,
+                                      l.mutable_data()->pb.mutable_extra_config()));
   }
 
   // Set to true if columns are altered, added or dropped.
@@ -5303,6 +5333,60 @@ const char* CatalogManager::StateToString(State state) {
   __builtin_unreachable();
 }
 
+Status CatalogManager::IsOutdatedTable(const std::string& table_name,
+                                       bool* is_trashed_table,
+                                       bool* is_outdated_table) {
+  *is_trashed_table = false;
+  string origin_table_name;
+  WallTime mark_delete_time;
+  if (!GetOriginNameAndDeleteTimeOfTrashedTable(table_name,
+                                                &origin_table_name,
+                                                &mark_delete_time)) {
+    return Status::OK();
+  }
+
+  // TODO(yingchun): Check whether a table is 'trashed' or not by GetTableSchema is a little
+  // expensive.
+  GetTableSchemaRequestPB schema_req;
+  schema_req.mutable_table()->set_table_name(table_name);
+  GetTableSchemaResponsePB schema_resp;
+  RETURN_NOT_OK(GetTableSchema(&schema_req, &schema_resp, boost::none, nullptr));
+  auto found = FindOrNull(schema_resp.extra_configs(), kTableConfigReserveSeconds);
+  if (!found) {
+    return Status::OK();
+  }
+
+  uint32_t reserve_seconds = 0;
+  if (!safe_strtou32(*found, &reserve_seconds)) {
+    return Status::Corruption(Substitute("Table $0's config $1 is invalid",
+                              table_name, kTableConfigReserveSeconds));
+  }
+
+  *is_trashed_table = true;
+  if (is_outdated_table) {
+    *is_outdated_table = (WallTime_Now() - mark_delete_time > reserve_seconds);
+  }
+
+  return Status::OK();
+}
+
+bool CatalogManager::GetOriginNameAndDeleteTimeOfTrashedTable(const string& table_name,
+                                                              string* origin_table_name,
+                                                              WallTime* mark_delete_time) {
+  vector<string> sections = strings::Split(table_name, ":", strings::AllowEmpty());
+  if (sections.size() < 3 || sections[0] != Master::kTrashedTag || sections[2].empty()) {
+    return false;
+  }
+
+  if (!safe_strtod(sections[1], mark_delete_time)
+      || *mark_delete_time < 0 || *mark_delete_time >= WallTime_Now()) {
+    return false;
+  }
+
+  *origin_table_name = sections[2];
+  return true;
+}
+
 ////////////////////////////////////////////////////////////
 // CatalogManager::ScopedLeaderSharedLock
 ////////////////////////////////////////////////////////////
@@ -5409,6 +5493,7 @@ INITTED_AND_LEADER_OR_RESPOND(GetTableLocationsResponsePB);
 INITTED_AND_LEADER_OR_RESPOND(GetTableSchemaResponsePB);
 INITTED_AND_LEADER_OR_RESPOND(GetTableStatisticsResponsePB);
 INITTED_AND_LEADER_OR_RESPOND(GetTabletLocationsResponsePB);
+INITTED_AND_LEADER_OR_RESPOND(RecallDeletedTableResponsePB);
 INITTED_AND_LEADER_OR_RESPOND(ReplaceTabletResponsePB);
 
 #undef INITTED_OR_RESPOND
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index d6d5ff6..d927299 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -41,6 +41,7 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/walltime.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/tablet/metadata.pb.h"
 #include "kudu/tserver/tablet_replica_lookup.h"
@@ -72,6 +73,7 @@ struct ColumnId;
 
 // Working around FRIEND_TEST() ugliness.
 namespace client {
+class ClientTest_TestDeleteAndReserveTable_Test;
 class ServiceUnavailableRetryClientTest_CreateTable_Test;
 } // namespace client
 
@@ -610,13 +612,22 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
                         const std::string& table_id,
                         int64_t notification_log_event_id) WARN_UNUSED_RESULT;
 
+  // Recall a table in response to a RecallDeletedTableRequestPB RPC.
+  //
+  // The RPC context is provided for logging/tracing purposes,
+  // but this function does not itself respond to the RPC.
+  Status RecallDeletedTableRpc(const RecallDeletedTableRequestPB& req,
+                               RecallDeletedTableResponsePB* resp,
+                               rpc::RpcContext* rpc) WARN_UNUSED_RESULT;
+
   // Alter the specified table in response to an AlterTableRequest RPC.
   //
   // The RPC context is provided for logging/tracing purposes,
   // but this function does not itself respond to the RPC.
   Status AlterTableRpc(const AlterTableRequestPB& req,
                        AlterTableResponsePB* resp,
-                       rpc::RpcContext* rpc);
+                       rpc::RpcContext* rpc,
+                       bool external_request);
 
   // Rename the specified table in response to an 'ALTER TABLE RENAME' HMS
   // notification log listener event.
@@ -783,8 +794,14 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   // name is returned.
   static std::string NormalizeTableName(const std::string& table_name);
 
+  // Check whether the table is trashed and outdated.
+  Status IsOutdatedTable(const std::string& table_name,
+                         bool* is_trashed_table,
+                         bool* is_outdated_table = nullptr);
+
  private:
   // These tests call ElectedAsLeaderCb() directly.
+  FRIEND_TEST(kudu::client::ClientTest, TestDeleteAndReserveTable);
   FRIEND_TEST(MasterTest, TestShutdownDuringTableVisit);
   FRIEND_TEST(MasterTest, TestGetTableLocationsDuringRepeatedTableVisit);
   FRIEND_TEST(kudu::AuthzTokenTest, TestSingleMasterUnavailable);
@@ -795,6 +812,9 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   // This test exclusively acquires the leader_lock_ directly.
   FRIEND_TEST(kudu::client::ServiceUnavailableRetryClientTest, CreateTable);
 
+  // This test call GetOriginNameAndDeleteTimeOfTrashedTable directly.
+  FRIEND_TEST(MasterTest, TestGetOriginNameAndDeleteTimeOfTrashedTable);
+
   friend class AutoRebalancerTest;
   friend class TableLoader;
   friend class TabletLoader;
@@ -824,7 +844,8 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   Status AlterTable(const AlterTableRequestPB& req,
                     AlterTableResponsePB* resp,
                     boost::optional<int64_t> hms_notification_log_event_id,
-                    boost::optional<const std::string&> user) WARN_UNUSED_RESULT;
+                    boost::optional<const std::string&> user,
+                    bool external_request) WARN_UNUSED_RESULT;
 
   // Called by SysCatalog::SysCatalogStateChanged when this node
   // becomes the leader of a consensus configuration. Executes
@@ -1068,6 +1089,12 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   Status WaitForNotificationLogListenerCatchUp(RespClass* resp,
                                                rpc::RpcContext* rpc) WARN_UNUSED_RESULT;
 
+  // Get origin name and delete time of a trashed table.
+  // Returns false if failed.
+  static bool GetOriginNameAndDeleteTimeOfTrashedTable(const std::string& table_name,
+                                                       std::string* origin_table_name,
+                                                       WallTime* mark_delete_time);
+
   // TODO(unknown): the maps are a little wasteful of RAM, since the TableInfo/TabletInfo
   // objects have a copy of the string key. But STL doesn't make it
   // easy to make a "gettable set".
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index fc6c101..15bdedc 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -35,6 +35,7 @@
 
 #include <gflags/gflags_declare.h>
 #include <glog/logging.h>
+#include <google/protobuf/stubs/common.h>
 #include <gtest/gtest.h>
 #include <rapidjson/document.h>
 #include <rapidjson/rapidjson.h>
@@ -159,6 +160,7 @@ class MasterTest : public KuduTest {
                      const Schema& schema,
                      const vector<KuduPartialRow>& split_rows,
                      const vector<pair<KuduPartialRow, KuduPartialRow>>& bounds);
+  Status SetConfig(const string& table_name, const string& key, const string& value);
 
   shared_ptr<Messenger> client_messenger_;
   unique_ptr<MiniMaster> mini_master_;
@@ -556,6 +558,16 @@ Status MasterTest::CreateTable(const string& table_name,
   return Status::OK();
 }
 
+Status MasterTest::SetConfig(const string& table_name, const string& key, const string& value) {
+  AlterTableRequestPB req;
+  AlterTableResponsePB resp;
+  RpcController controller;
+  req.mutable_table()->set_table_name(table_name);
+  (*req.mutable_new_extra_configs())[key] = value;
+  RETURN_NOT_OK(master_->catalog_manager()->AlterTableRpc(req, &resp, nullptr, false));
+  return Status::OK();
+}
+
 void MasterTest::DoListTables(const ListTablesRequestPB& req, ListTablesResponsePB* resp) {
   RpcController controller;
   ASSERT_OK(proxy_->ListTables(req, resp, &controller));
@@ -2037,5 +2049,83 @@ TEST_P(AuthzTokenMasterTest, TestGenerateAuthzTokens) {
 
 INSTANTIATE_TEST_CASE_P(SupportsAuthzTokens, AuthzTokenMasterTest, ::testing::Bool());
 
+const vector<string> kInvalidTableNames
+    = { "abc",
+        Substitute("a$0:$1:abc", Master::kTrashedTag, WallTime_Now()),
+        Substitute("$0a:$1:abc", Master::kTrashedTag, WallTime_Now()),
+        Substitute("$0:$1", Master::kTrashedTag, WallTime_Now()),
+        Substitute("$0:a:abc", Master::kTrashedTag),
+        Substitute("$0:-123:abc", Master::kTrashedTag)
+      };
+
+TEST_F(MasterTest, TestGetOriginNameAndDeleteTimeOfTrashedTable) {
+  string origin_table_name;
+  WallTime mark_delete_time;
+  for (const auto& table_name : kInvalidTableNames) {
+    ASSERT_FALSE(CatalogManager::GetOriginNameAndDeleteTimeOfTrashedTable(
+      table_name, &origin_table_name, &mark_delete_time));
+  }
+
+  ASSERT_TRUE(CatalogManager::GetOriginNameAndDeleteTimeOfTrashedTable(
+      Substitute("$0:123:abc", Master::kTrashedTag),
+      &origin_table_name, &mark_delete_time));
+  ASSERT_EQ("abc", origin_table_name);
+  ASSERT_EQ(123, mark_delete_time);
+}
+
+TEST_F(MasterTest, TestIsOutdatedTable) {
+  const char* kTableName = "testtable";
+  const Schema kTableSchema({ ColumnSchema("key", INT32) }, 1);
+
+  bool is_trashed_table = false;
+  bool is_outdated_table = false;
+  // Invalid table names.
+  for (const auto& table_name : kInvalidTableNames) {
+    ASSERT_OK(master_->catalog_manager()
+        ->IsOutdatedTable(table_name, &is_trashed_table, &is_outdated_table));
+    ASSERT_FALSE(is_trashed_table);
+  }
+
+  // Create a new table.
+  const string kTrashedTableName
+      = Substitute("$0:$1:$2", Master::kTrashedTag, WallTime_Now(), kTableName);
+  ASSERT_OK(CreateTable(kTrashedTableName, kTableSchema));
+
+  // Default table is not outdated.
+  CatalogManager::ScopedLeaderSharedLock l(master_->catalog_manager());
+  ASSERT_OK(master_->catalog_manager()
+      ->IsOutdatedTable(kTrashedTableName, &is_trashed_table, &is_outdated_table));
+  ASSERT_FALSE(is_trashed_table);
+  ASSERT_FALSE(is_outdated_table);
+
+  // Set config: kTableConfigReserveSeconds
+  ASSERT_TRUE(SetConfig(kTrashedTableName, kTableConfigReserveSeconds, "-1").IsInvalidArgument());
+  ASSERT_OK(master_->catalog_manager()
+      ->IsOutdatedTable(kTrashedTableName, &is_trashed_table, &is_outdated_table));
+  ASSERT_FALSE(is_trashed_table);
+  ASSERT_FALSE(is_outdated_table);
+
+  ASSERT_TRUE(SetConfig(kTrashedTableName, kTableConfigReserveSeconds, "a").IsInvalidArgument());
+  ASSERT_OK(master_->catalog_manager()
+      ->IsOutdatedTable(kTrashedTableName, &is_trashed_table, &is_outdated_table));
+  ASSERT_FALSE(is_trashed_table);
+  ASSERT_FALSE(is_outdated_table);
+
+  // In reserve time, table is not outdated.
+  ASSERT_OK(SetConfig(kTrashedTableName, kTableConfigReserveSeconds, "100"));
+  ASSERT_OK(master_->catalog_manager()
+      ->IsOutdatedTable(kTrashedTableName, &is_trashed_table, &is_outdated_table));
+  ASSERT_TRUE(is_trashed_table);
+  ASSERT_FALSE(is_outdated_table);
+
+  // After reserve time, table is outdated.
+  ASSERT_OK(SetConfig(kTrashedTableName, kTableConfigReserveSeconds, "1"));
+  SleepFor(MonoDelta::FromSeconds(2));
+  ASSERT_OK(master_->catalog_manager()
+      ->IsOutdatedTable(kTrashedTableName, &is_trashed_table, &is_outdated_table));
+  ASSERT_TRUE(is_trashed_table);
+  ASSERT_TRUE(is_outdated_table);
+}
+
 } // namespace master
 } // namespace kudu
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index f55cc6a..32d6e8d 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -24,6 +24,7 @@
 #include <string>
 #include <vector>
 
+#include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 
@@ -53,12 +54,14 @@
 #include "kudu/server/webserver.h"
 #include "kudu/tserver/tablet_copy_service.h"
 #include "kudu/tserver/tablet_service.h"
+#include "kudu/util/countdown_latch.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/maintenance_manager.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
+#include "kudu/util/thread.h"
 #include "kudu/util/threadpool.h"
 #include "kudu/util/version_info.h"
 
@@ -84,6 +87,11 @@ DEFINE_int64(authz_token_validity_seconds, 60 * 5,
              "validity period expires.");
 TAG_FLAG(authz_token_validity_seconds, experimental);
 
+DEFINE_int32(check_outdated_table_interval_seconds, 60,
+             "Interval seconds to check whether there is any trashed table is "
+             "outdated, this kind of table will be deleted and can not be "
+             "recalled later.");
+
 DEFINE_string(location_mapping_cmd, "",
               "A Unix command which takes a single argument, the IP address or "
               "hostname of a tablet server or client, and returns the location "
@@ -109,6 +117,8 @@ using strings::Substitute;
 namespace kudu {
 namespace master {
 
+const char Master::kTrashedTag[] = "trashed";
+
 Master::Master(const MasterOptions& opts)
   : KuduServer("Master", opts, "kudu.master"),
     state_(kStopped),
@@ -190,6 +200,7 @@ Status Master::StartAsync() {
 
   // Start initializing the catalog manager.
   RETURN_NOT_OK(init_pool_->Submit([this]() { this->InitCatalogManagerTask(); }));
+  RETURN_NOT_OK(StartOutdatedReservedTablesDeleterThread());
   state_ = kRunning;
 
   return Status::OK();
@@ -299,6 +310,53 @@ void Master::CrashMasterOnCFileCorruption(const string& tablet_id) {
   LOG(FATAL) << Substitute("CFile corruption detected on system catalog $0", tablet_id);
 }
 
+Status Master::StartOutdatedReservedTablesDeleterThread() {
+  return Thread::Create("master", "outdated-reserved-tables-deleter",
+                        &Master::OutdatedReservedTablesDeleterThread,
+                        this, &outdated_reserved_tables_deleter_thread_);
+}
+
+void Master::OutdatedReservedTablesDeleterThread() {
+  // How often to attempt to delete outdated tables.
+  const MonoDelta kWait = MonoDelta::FromSeconds(FLAGS_check_outdated_table_interval_seconds);
+  while (!stop_background_threads_latch_.WaitUntil(MonoTime::Now() + kWait)) {
+    WARN_NOT_OK(DeleteOutdatedReservedTables(), "Unable to delete outdated reserved tables");
+  }
+}
+
+Status Master::DeleteOutdatedReservedTables() {
+  CatalogManager::ScopedLeaderSharedLock l(catalog_manager());
+  if (!l.first_failed_status().ok()) {
+    // Skip checking if this master is not leader.
+    return Status::OK();
+  }
+
+  ListTablesRequestPB list_req;
+  ListTablesResponsePB list_resp;
+  RETURN_NOT_OK(catalog_manager_->ListTables(&list_req, &list_resp, boost::none));
+  for (const auto& table : list_resp.tables()) {
+    bool is_trashed_table = false;
+    bool is_outdated_table = false;
+    Status s = catalog_manager_->IsOutdatedTable(table.name(),
+                                                 &is_trashed_table, &is_outdated_table);
+    if (!s.ok() || !is_trashed_table || !is_outdated_table) {
+      continue;
+    }
+
+    // Delete the table.
+    DeleteTableRequestPB del_req;
+    del_req.mutable_table()->set_table_id(table.id());
+    del_req.mutable_table()->set_table_name(table.name());
+    del_req.set_reserve_seconds(0);
+    DeleteTableResponsePB del_resp;
+    LOG(INFO) << "Start to delete trashed table " << table.name();
+    WARN_NOT_OK(catalog_manager_->DeleteTableRpc(del_req, &del_resp, nullptr),
+                Substitute("Failed to delete trashed table $0", table.name()));
+  }
+
+  return Status::OK();
+}
+
 namespace {
 
 // TODO(Alex Feinberg) this method should be moved to a separate class (along with
diff --git a/src/kudu/master/master.h b/src/kudu/master/master.h
index 3f77074..3a42c6f 100644
--- a/src/kudu/master/master.h
+++ b/src/kudu/master/master.h
@@ -22,9 +22,12 @@
 #include <string>
 #include <vector>
 
+#include <gtest/gtest_prod.h>
+
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
 #include "kudu/kserver/kserver.h"
 #include "kudu/master/master_options.h"
 #include "kudu/util/promise.h"
@@ -35,6 +38,7 @@ namespace kudu {
 class HostPortPB;
 class MaintenanceManager;
 class MonoDelta;
+class Thread;
 class ThreadPool;
 
 namespace master {
@@ -56,6 +60,7 @@ class Master : public kserver::KuduServer {
  public:
   static const uint16_t kDefaultPort = 7051;
   static const uint16_t kDefaultWebPort = 8051;
+  static const char kTrashedTag[];
 
   explicit Master(const MasterOptions& opts);
   ~Master();
@@ -123,6 +128,7 @@ class Master : public kserver::KuduServer {
 
  private:
   friend class MasterTest;
+  FRIEND_TEST(MasterTest, TestIsTableOutdated);
 
   void InitCatalogManagerTask();
   Status InitCatalogManager();
@@ -136,6 +142,11 @@ class Master : public kserver::KuduServer {
   // safe in a particular case.
   void ShutdownImpl();
 
+  // Start thread to delete outdated reserved tables.
+  Status StartOutdatedReservedTablesDeleterThread();
+  void OutdatedReservedTablesDeleterThread();
+  Status DeleteOutdatedReservedTables();
+
   enum MasterState {
     kStopped,
     kInitialized,
@@ -170,6 +181,8 @@ class Master : public kserver::KuduServer {
 
   std::unique_ptr<TSManager> ts_manager_;
 
+  scoped_refptr<Thread> outdated_reserved_tables_deleter_thread_;
+
   DISALLOW_COPY_AND_ASSIGN(Master);
 };
 
diff --git a/src/kudu/master/master.proto b/src/kudu/master/master.proto
index bda76e2..10d45a9 100644
--- a/src/kudu/master/master.proto
+++ b/src/kudu/master/master.proto
@@ -520,6 +520,12 @@ message DeleteTableRequestPB {
   // Whether to apply the deletion to external catalogs, such as the Hive Metastore,
   // which the Kudu master has been configured to integrate with.
   optional bool modify_external_catalogs = 2 [default = true];
+
+  // Reserve seconds after the table has been deleted.
+  optional uint32 reserve_seconds = 3;
+
+  // Force to delete a trashed table.
+  optional bool force_on_trashed_table = 4 [default = false];
 }
 
 message DeleteTableResponsePB {
@@ -527,6 +533,15 @@ message DeleteTableResponsePB {
   optional MasterErrorPB error = 1;
 }
 
+message RecallDeletedTableRequestPB {
+  required TableIdentifierPB table = 1;
+}
+
+message RecallDeletedTableResponsePB {
+  // The error, if an error occurred with this request.
+  optional MasterErrorPB error = 1;
+}
+
 message ListTablesRequestPB {
   // When used, only returns tables that satisfy a substring match on name_filter.
   optional string name_filter = 1;
@@ -671,6 +686,9 @@ message AlterTableRequestPB {
   optional bool modify_external_catalogs = 5 [default = true];
 
   map<string, string> new_extra_configs = 6;
+
+  // Force to alter a trashed table.
+  optional bool force_on_trashed_table = 7 [default = false];
 }
 
 message AlterTableResponsePB {
@@ -991,6 +1009,9 @@ service MasterService {
   rpc DeleteTable(DeleteTableRequestPB) returns (DeleteTableResponsePB) {
     option (kudu.rpc.authz_method) = "AuthorizeClient";
   }
+  rpc RecallDeletedTable(RecallDeletedTableRequestPB) returns (RecallDeletedTableResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
 
   rpc AlterTable(AlterTableRequestPB) returns (AlterTableResponsePB) {
     option (kudu.rpc.authz_method) = "AuthorizeClient";
diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc
index 50ed51d..73cffab 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -27,6 +27,7 @@
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
+#include <google/protobuf/stubs/common.h>
 
 #include "kudu/common/common.pb.h"
 #include "kudu/common/wire_protocol.h"
@@ -36,6 +37,7 @@
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
 #include "kudu/hms/hms_catalog.h"
 #include "kudu/master/authz_provider.h"
 #include "kudu/master/catalog_manager.h"
@@ -62,6 +64,7 @@
 
 DECLARE_bool(hive_metastore_sasl_enabled);
 DECLARE_bool(raft_prepare_replacement_before_eviction);
+DECLARE_int32(max_priority_range);
 DECLARE_string(hive_metastore_uris);
 
 DEFINE_int32(master_inject_latency_on_tablet_lookups_ms, 0,
@@ -446,7 +449,65 @@ void MasterServiceImpl::DeleteTable(const DeleteTableRequestPB* req,
     return;
   }
 
-  Status s = server_->catalog_manager()->DeleteTableRpc(*req, resp, rpc);
+  bool is_trashed_table = false;
+  Status s = server_->catalog_manager()
+      ->IsOutdatedTable(req->table().table_name(), &is_trashed_table);
+  if (s.ok() && is_trashed_table && !req->force_on_trashed_table()) {
+    s = Status::InvalidArgument(Substitute("trashed table $0 should not be deleted",
+                                           req->table().table_name()));
+  }
+
+  if (!s.ok()) {
+    CheckRespErrorOrSetUnknown(s, resp);
+    rpc->RespondSuccess();
+    return;
+  }
+
+  if ((is_trashed_table && req->force_on_trashed_table()) || req->reserve_seconds() == 0) {
+    Status s = server_->catalog_manager()->DeleteTableRpc(*req, resp, rpc);
+    CheckRespErrorOrSetUnknown(s, resp);
+    rpc->RespondSuccess();
+  } else {
+    DCHECK(!is_trashed_table);
+    AlterTableRequestPB alter_req;
+    alter_req.mutable_table()->CopyFrom(req->table());
+    alter_req.set_new_table_name(string(Master::kTrashedTag) + ":"
+                                 + std::to_string(WallTime_Now()) + ":"
+                                 + req->table().table_name());
+    alter_req.set_modify_external_catalogs(req->modify_external_catalogs());
+    (*alter_req.mutable_new_extra_configs())[kTableMaintenancePriority]
+        = std::to_string(-FLAGS_max_priority_range);
+    (*alter_req.mutable_new_extra_configs())[kTableConfigReserveSeconds]
+        = std::to_string(req->reserve_seconds());
+
+    AlterTableResponsePB alter_resp;
+    Status s = server_->catalog_manager()->AlterTableRpc(alter_req, &alter_resp, rpc, false);
+    CheckRespErrorOrSetUnknown(s, &alter_resp);
+    resp->set_allocated_error(alter_resp.release_error());
+    rpc->RespondSuccess();
+  }
+}
+
+void MasterServiceImpl::RecallDeletedTable(const RecallDeletedTableRequestPB* req,
+                                           RecallDeletedTableResponsePB* resp,
+                                           rpc::RpcContext* rpc) {
+  CatalogManager::ScopedLeaderSharedLock l(server_->catalog_manager());
+  if (!l.CheckIsInitializedAndIsLeaderOrRespond(resp, rpc)) {
+    return;
+  }
+
+  bool is_trashed_table = false;
+  Status s = server_->catalog_manager()
+      ->IsOutdatedTable(req->table().table_name(), &is_trashed_table);
+  if (s.ok()) {
+    if (is_trashed_table) {
+      s = server_->catalog_manager()->RecallDeletedTableRpc(*req, resp, rpc);
+    } else {
+      s = Status::InvalidArgument(Substitute("common table $0 should not be recalled",
+                                             req->table().table_name()));
+    }
+  }
+
   CheckRespErrorOrSetUnknown(s, resp);
   rpc->RespondSuccess();
 }
@@ -459,7 +520,18 @@ void MasterServiceImpl::AlterTable(const AlterTableRequestPB* req,
     return;
   }
 
-  Status s = server_->catalog_manager()->AlterTableRpc(*req, resp, rpc);
+  bool is_trashed_table = false;
+  Status s = server_->catalog_manager()
+      ->IsOutdatedTable(req->table().table_name(), &is_trashed_table);
+  if (s.ok()) {
+    if (!is_trashed_table || req->force_on_trashed_table()) {
+      s = server_->catalog_manager()->AlterTableRpc(*req, resp, rpc, true);
+    } else {
+      s = Status::InvalidArgument(Substitute("trashed table $0 should not be altered",
+                                             req->table().table_name()));
+    }
+  }
+
   CheckRespErrorOrSetUnknown(s, resp);
   rpc->RespondSuccess();
 }
diff --git a/src/kudu/master/master_service.h b/src/kudu/master/master_service.h
index 4098ce7..ae4670f 100644
--- a/src/kudu/master/master_service.h
+++ b/src/kudu/master/master_service.h
@@ -69,6 +69,8 @@ class ListTabletServersResponsePB;
 class Master;
 class PingRequestPB;
 class PingResponsePB;
+class RecallDeletedTableRequestPB;
+class RecallDeletedTableResponsePB;
 class ReplaceTabletRequestPB;
 class ReplaceTabletResponsePB;
 class ResetAuthzCacheRequestPB;
@@ -128,6 +130,10 @@ class MasterServiceImpl : public MasterServiceIf {
                    DeleteTableResponsePB* resp,
                    rpc::RpcContext* rpc) override;
 
+  void RecallDeletedTable(const RecallDeletedTableRequestPB* req,
+                          RecallDeletedTableResponsePB* resp,
+                          rpc::RpcContext* rpc) override;
+
   void AlterTable(const AlterTableRequestPB* req,
                   AlterTableResponsePB* resp,
                   rpc::RpcContext* rpc) override;
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 610d51d..8d63af2 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -400,12 +400,12 @@ ServerBase::ServerBase(string name, const ServerBaseOptions& options,
       result_tracker_(new rpc::ResultTracker(shared_ptr<MemTracker>(
           MemTracker::CreateTracker(-1, "result-tracker", mem_tracker_)))),
       is_first_run_(false),
+      stop_background_threads_latch_(1),
       dns_resolver_(new DnsResolver(
           FLAGS_dns_resolver_max_threads_num,
           FLAGS_dns_resolver_cache_capacity_mb * 1024 * 1024,
           MonoDelta::FromSeconds(FLAGS_dns_resolver_cache_ttl_sec))),
-      options_(options),
-      stop_background_threads_latch_(1) {
+      options_(options) {
   metric_entity_->NeverRetire(
       METRIC_merged_entities_count_of_server.InstantiateHidden(metric_entity_, 1));
 
diff --git a/src/kudu/server/server_base.h b/src/kudu/server/server_base.h
index 93aaf16..29f1603 100644
--- a/src/kudu/server/server_base.h
+++ b/src/kudu/server/server_base.h
@@ -203,6 +203,8 @@ class ServerBase {
   // The ACL of users who may act as part of the Kudu service.
   security::SimpleAcl service_acl_;
 
+  CountDownLatch stop_background_threads_latch_;
+
  private:
   Status InitAcls();
   void GenerateInstanceID();
@@ -240,7 +242,6 @@ class ServerBase {
 #ifdef TCMALLOC_ENABLED
   scoped_refptr<Thread> tcmalloc_memory_gc_thread_;
 #endif
-  CountDownLatch stop_background_threads_latch_;
 
   std::unique_ptr<ScopedGLogMetrics> glog_metrics_;
 
diff --git a/src/kudu/tools/kudu-admin-test.cc b/src/kudu/tools/kudu-admin-test.cc
index eae3b14..25cce5d 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -1637,7 +1637,9 @@ TEST_F(AdminCliTest, TestDeleteTable) {
     "table",
     "delete",
     master_address,
-    kTableId
+    kTableId,
+    "-force_on_trashed_table=false",
+    "-reserve_seconds=0"
   );
 
   vector<string> tables;
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 4bc44c5..84ebb4e 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -1128,6 +1128,7 @@ TEST_F(ToolTest, TestModeHelp) {
         "get_extra_configs.*Get the extra configuration properties for a table",
         "list.*List tables",
         "locate_row.*Locate which tablet a row belongs to",
+        "recall.*Recall a deleted but still reserved table",
         "rename_column.*Rename a column",
         "rename_table.*Rename a table",
         "scan.*Scan rows from a table",
@@ -3148,6 +3149,58 @@ TEST_F(ToolTest, TestRenameTable) {
   ASSERT_OK(client->OpenTable(kTableName, &table));
 }
 
+TEST_F(ToolTest, TestRecallTable) {
+  NO_FATALS(StartExternalMiniCluster());
+  shared_ptr<KuduClient> client;
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client));
+  string master_addr = cluster_->master()->bound_rpc_addr().ToString();
+
+  const string& kTableName = "kudu.table";
+
+  // Create the table.
+  TestWorkload workload(cluster_.get());
+  workload.set_table_name(kTableName);
+  workload.set_num_replicas(1);
+  workload.Setup();
+
+  // Delete the table.
+  string out;
+  NO_FATALS(RunActionStdoutNone(Substitute("table delete $0 $1",
+                                           master_addr, kTableName)));
+  shared_ptr<KuduTable> table;
+
+  // Try to open the table.
+  Status s = client->OpenTable(kTableName, &table);
+  ASSERT_TRUE(s.IsNotFound());
+  ASSERT_STR_CONTAINS(s.ToString(), Substitute("the table does not exist"));
+
+  // List trashed table.
+  vector<string> kudu_tables;
+  client->ListTables(&kudu_tables);
+  ASSERT_EQ(kudu_tables.size(), 1);
+
+  // Create another table.
+  workload.Setup();
+  ASSERT_OK(client->OpenTable(kTableName, &table));
+
+  // Try to recall the trashed table.
+  string stderr;
+  s = RunTool(Substitute("table recall $0 $1",
+                         master_addr, kudu_tables[0]),
+              nullptr, &stderr, {}, {});
+  ASSERT_TRUE(s.IsRuntimeError());
+  ASSERT_STR_CONTAINS(s.ToString(), "process exited with non-zero status");
+  SCOPED_TRACE(stderr);
+  ASSERT_STR_CONTAINS(stderr, Substitute("Already present: table $0 already exists", kTableName));
+
+  // Rename the new table and try to recall the trashed table.
+  NO_FATALS(RunActionStdoutNone(
+        Substitute("table rename_table $0 $1 $2",
+          master_addr, kTableName, kTableName + "new")));
+  NO_FATALS(RunActionStdoutNone(Substitute("table recall $0 $1",
+                                           master_addr, kudu_tables[0])));
+}
+
 TEST_F(ToolTest, TestRenameColumn) {
   NO_FATALS(StartExternalMiniCluster());
   const string& kTableName = "table";
diff --git a/src/kudu/tools/tool_action_table.cc b/src/kudu/tools/tool_action_table.cc
index 006cc71..8e98c5d 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <stdlib.h>
+
 #include <algorithm>
 #include <cstdint>
 #include <functional>
@@ -114,6 +116,10 @@ DEFINE_string(lower_bound_type, "INCLUSIVE_BOUND",
 DEFINE_string(upper_bound_type, "EXCLUSIVE_BOUND",
               "The type of the upper bound, either inclusive or exclusive. "
               "Defaults to exclusive. This flag is case-insensitive.");
+DEFINE_uint32(reserve_seconds, 604800,
+              "Reserve seconds after being deleted.");
+DEFINE_bool(force_on_trashed_table, false,
+            "Force to alter a trashed table");
 DECLARE_bool(show_values);
 DECLARE_string(tables);
 
@@ -192,7 +198,10 @@ Status DeleteTable(const RunnerContext& context) {
   const string& table_name = FindOrDie(context.required_args, kTableNameArg);
   client::sp::shared_ptr<KuduClient> client;
   RETURN_NOT_OK(CreateKuduClient(context, &client));
-  return client->DeleteTableInCatalogs(table_name, FLAGS_modify_external_catalogs);
+  return client->DeleteTableInCatalogs(table_name,
+                                       FLAGS_modify_external_catalogs,
+                                       FLAGS_force_on_trashed_table,
+                                       FLAGS_reserve_seconds);
 }
 
 Status DescribeTable(const RunnerContext& context) {
@@ -407,6 +416,14 @@ Status LocateRow(const RunnerContext& context) {
   return Status::OK();
 }
 
+Status RecallTable(const RunnerContext& context) {
+  const string& table_name = FindOrDie(context.required_args, kTableNameArg);
+
+  client::sp::shared_ptr<KuduClient> client;
+  RETURN_NOT_OK(CreateKuduClient(context, &client));
+  return client->RecallTable(table_name);
+}
+
 Status RenameTable(const RunnerContext& context) {
   const string& table_name = FindOrDie(context.required_args, kTableNameArg);
   const string& new_table_name = FindOrDie(context.required_args, kNewTableNameArg);
@@ -415,6 +432,7 @@ Status RenameTable(const RunnerContext& context) {
   RETURN_NOT_OK(CreateKuduClient(context, &client));
   unique_ptr<KuduTableAlterer> alterer(client->NewTableAlterer(table_name));
   return alterer->RenameTo(new_table_name)
+                ->force_on_trashed_table(FLAGS_force_on_trashed_table)
                 ->modify_external_catalogs(FLAGS_modify_external_catalogs)
                 ->Alter();
 }
@@ -477,8 +495,9 @@ Status SetExtraConfig(const RunnerContext& context) {
   client::sp::shared_ptr<KuduClient> client;
   RETURN_NOT_OK(CreateKuduClient(context, &client));
   unique_ptr<KuduTableAlterer> alterer(client->NewTableAlterer(table_name));
-  alterer->AlterExtraConfig({ { config_name, config_value} });
-  return alterer->Alter();
+  return alterer->AlterExtraConfig({ { config_name, config_value} })
+                ->force_on_trashed_table(FLAGS_force_on_trashed_table)
+                ->Alter();
 }
 
 Status GetExtraConfigs(const RunnerContext& context) {
@@ -1175,7 +1194,9 @@ unique_ptr<Mode> BuildTableMode() {
       .Description("Delete a table")
       .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to delete" })
+      .AddOptionalParameter("force_on_trashed_table")
       .AddOptionalParameter("modify_external_catalogs")
+      .AddOptionalParameter("reserve_seconds")
       .Build();
 
   unique_ptr<Action> describe_table =
@@ -1220,12 +1241,20 @@ unique_ptr<Mode> BuildTableMode() {
       .AddRequiredParameter({ kNewColumnNameArg, "New column name" })
       .Build();
 
+  unique_ptr<Action> recall =
+      ActionBuilder("recall", &RecallTable)
+      .Description("Recall a deleted but still reserved table")
+      .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
+      .AddRequiredParameter({ kTableNameArg, "Name of the table to recall" })
+      .Build();
+
   unique_ptr<Action> rename_table =
       ActionBuilder("rename_table", &RenameTable)
       .Description("Rename a table")
       .AddRequiredParameter({ kMasterAddressesArg, kMasterAddressesArgDesc })
       .AddRequiredParameter({ kTableNameArg, "Name of the table to rename" })
       .AddRequiredParameter({ kNewTableNameArg, "New table name" })
+      .AddOptionalParameter("force_on_trashed_table")
       .AddOptionalParameter("modify_external_catalogs")
       .Build();
 
@@ -1269,6 +1298,7 @@ unique_ptr<Mode> BuildTableMode() {
       .AddRequiredParameter({ kTableNameArg, "Name of the table to alter" })
       .AddRequiredParameter({ kConfigNameArg, "Name of the configuration" })
       .AddRequiredParameter({ kConfigValueArg, "New value for the configuration" })
+      .AddOptionalParameter("force_on_trashed_table")
       .Build();
 
   unique_ptr<Action> get_extra_configs =
@@ -1425,6 +1455,7 @@ unique_ptr<Mode> BuildTableMode() {
       .AddAction(std::move(list_tables))
       .AddAction(std::move(locate_row))
       .AddAction(std::move(rename_column))
+      .AddAction(std::move(recall))
       .AddAction(std::move(rename_table))
       .AddAction(std::move(scan_table))
       .AddAction(std::move(set_extra_config))


[kudu] 05/23: [build] do not remove thirdparty archives by default

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit b30742f261044c9b6c460c4fc2449a66d8a731c4
Author: laiyingchun <la...@xiaomi.com>
AuthorDate: Thu Nov 22 19:28:06 2018 +0800

    [build] do not remove thirdparty archives by default
---
 thirdparty/vars.sh | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh
index f30380e..de6b295 100644
--- a/thirdparty/vars.sh
+++ b/thirdparty/vars.sh
@@ -25,6 +25,8 @@ fi
 TP_SOURCE_DIR="$TP_DIR/src"
 TP_BUILD_DIR="$TP_DIR/build"
 
+NO_REMOVE_THIRDPARTY_ARCHIVES=1
+
 # This URL corresponds to the CloudFront Distribution for the S3
 # bucket cloudera-thirdparty-libs which is directly accessible at
 # http://cloudera-thirdparty-libs.s3.amazonaws.com/


[kudu] 07/23: [collector] Add CPP implemented collector

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit d9b29d901bcbabd1210064f84d73ece9931baff2
Author: Yingchun Lai <40...@qq.com>
AuthorDate: Wed Jun 19 16:20:06 2019 +0800

    [collector] Add CPP implemented collector
    
    Change-Id: I498347605a09e832d3398e76d9cefd3e52b1cfc6
---
 CMakeLists.txt                                |   1 +
 src/kudu/collector/CMakeLists.txt             |  63 ++
 src/kudu/collector/cluster_rebalancer-test.cc |  50 ++
 src/kudu/collector/cluster_rebalancer.cc      | 152 +++++
 src/kudu/collector/cluster_rebalancer.h       |  68 ++
 src/kudu/collector/collector-test.cc          |  44 ++
 src/kudu/collector/collector.cc               | 170 +++++
 src/kudu/collector/collector.h                |  74 +++
 src/kudu/collector/collector_main.cc          |  73 +++
 src/kudu/collector/collector_util-test.cc     |  33 +
 src/kudu/collector/collector_util.cc          |  46 ++
 src/kudu/collector/collector_util.h           |  32 +
 src/kudu/collector/falcon_reporter-test.cc    | 122 ++++
 src/kudu/collector/falcon_reporter.cc         | 255 ++++++++
 src/kudu/collector/falcon_reporter.h          | 108 ++++
 src/kudu/collector/local_reporter.cc          |  84 +++
 src/kudu/collector/local_reporter.h           |  58 ++
 src/kudu/collector/metrics_collector-test.cc  | 777 +++++++++++++++++++++++
 src/kudu/collector/metrics_collector.cc       | 852 ++++++++++++++++++++++++++
 src/kudu/collector/metrics_collector.h        | 205 +++++++
 src/kudu/collector/nodes_checker-test.cc      |  55 ++
 src/kudu/collector/nodes_checker.cc           | 358 +++++++++++
 src/kudu/collector/nodes_checker.h            |  90 +++
 src/kudu/collector/reporter_base.h            |  73 +++
 src/kudu/master/catalog_manager.cc            |   1 +
 src/kudu/tools/tool_action_table.cc           |   2 -
 src/kudu/util/jsonreader.h                    |   6 +
 27 files changed, 3850 insertions(+), 2 deletions(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index d2bb441..1797da9 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -1423,6 +1423,7 @@ add_subdirectory(src/kudu/cfile)
 add_subdirectory(src/kudu/client)
 add_subdirectory(src/kudu/clock)
 add_subdirectory(src/kudu/codegen)
+add_subdirectory(src/kudu/collector)
 add_subdirectory(src/kudu/common)
 add_subdirectory(src/kudu/consensus)
 add_subdirectory(src/kudu/experiments)
diff --git a/src/kudu/collector/CMakeLists.txt b/src/kudu/collector/CMakeLists.txt
new file mode 100644
index 0000000..5bbb1cb
--- /dev/null
+++ b/src/kudu/collector/CMakeLists.txt
@@ -0,0 +1,63 @@
+# 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.
+
+#########################################
+# collector
+#########################################
+
+set(COLLECTOR_SRCS
+  cluster_rebalancer.cc
+  collector.cc
+  collector_util.cc
+  falcon_reporter.cc
+  local_reporter.cc
+  metrics_collector.cc
+  nodes_checker.cc)
+
+add_library(collector ${COLLECTOR_SRCS})
+target_link_libraries(collector
+  kudu_curl_util
+  kudu_tools_test_util
+  log
+  security
+  server_process)
+
+#########################################
+# kudu-collector
+#########################################
+
+add_executable(kudu-collector collector_main.cc)
+target_link_libraries(kudu-collector
+  ${SANITIZER_OPTIONS_OVERRIDE}
+  ${KRB5_REALM_OVERRIDE}
+  collector
+  ${KUDU_BASE_LIBS})
+
+option(KUDU_COLLECTOR_INSTALL "Whether to install the Kudu Collector executable" ON)
+if(KUDU_COLLECTOR_INSTALL)
+  install(TARGETS kudu-collector RUNTIME DESTINATION ${CMAKE_INSTALL_SBINDIR})
+else()
+  message(STATUS "Skipping install rule for the Kudu Collector executable")
+endif()
+
+SET_KUDU_TEST_LINK_LIBS(collector)
+ADD_KUDU_TEST(collector-test)
+ADD_KUDU_TEST(collector_util-test)
+ADD_KUDU_TEST(cluster_rebalancer-test)
+ADD_KUDU_TEST(falcon_reporter-test)
+ADD_KUDU_TEST(metrics_collector-test)
+ADD_KUDU_TEST(nodes_checker-test)
diff --git a/src/kudu/collector/cluster_rebalancer-test.cc b/src/kudu/collector/cluster_rebalancer-test.cc
new file mode 100644
index 0000000..0a1e0a7
--- /dev/null
+++ b/src/kudu/collector/cluster_rebalancer-test.cc
@@ -0,0 +1,50 @@
+// 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/collector/cluster_rebalancer.h"
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+namespace kudu {
+namespace collector {
+
+TEST(TestClusterRebalancer, TestValidateHMTime) {
+  // 'time' in error format.
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime("12:34:56").IsInvalidArgument());
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime("1:23").IsInvalidArgument());
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime("12:3").IsInvalidArgument());
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime("12:").IsInvalidArgument());
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime(":3").IsInvalidArgument());
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime("12.34").IsInvalidArgument());
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime("-1:30").IsInvalidArgument());
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime("24:30").IsInvalidArgument());
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime("12:-1").IsInvalidArgument());
+  ASSERT_TRUE(ClusterRebalancer::ValidateHMTime("12:60").IsInvalidArgument());
+
+  // 'time' in valid format.
+  ASSERT_OK(ClusterRebalancer::ValidateHMTime("12:34"));
+  ASSERT_OK(ClusterRebalancer::ValidateHMTime("00:00"));
+  ASSERT_OK(ClusterRebalancer::ValidateHMTime("00:59"));
+  ASSERT_OK(ClusterRebalancer::ValidateHMTime("23:00"));
+  ASSERT_OK(ClusterRebalancer::ValidateHMTime("23:59"));
+}
+}  // namespace collector
+}  // namespace kudu
+
diff --git a/src/kudu/collector/cluster_rebalancer.cc b/src/kudu/collector/cluster_rebalancer.cc
new file mode 100644
index 0000000..0015544
--- /dev/null
+++ b/src/kudu/collector/cluster_rebalancer.cc
@@ -0,0 +1,152 @@
+// 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/collector/cluster_rebalancer.h"
+
+#include <stdio.h>
+#include <time.h>
+
+#include <ostream>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+
+#include "kudu/collector/collector_util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/tools/tool_test_util.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+DEFINE_bool(auto_rebalance, true, "Whether to rebalance cluster automatically");
+DEFINE_string(rebalance_time, "00:00",
+              "Time to perform cluster rebalance, format in HH:MM");
+
+DECLARE_string(collector_cluster_name);
+DECLARE_string(collector_master_addrs);
+DECLARE_int32(collector_interval_sec);
+DECLARE_int32(collector_timeout_sec);
+
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace collector {
+
+ClusterRebalancer::ClusterRebalancer()
+  : initialized_(false),
+    stop_background_threads_latch_(1) {
+}
+
+ClusterRebalancer::~ClusterRebalancer() {
+  Shutdown();
+}
+
+Status ClusterRebalancer::Init() {
+  CHECK(!initialized_);
+
+  RETURN_NOT_OK(ValidateHMTime(FLAGS_rebalance_time));
+
+  initialized_ = true;
+  return Status::OK();
+}
+
+Status ClusterRebalancer::Start() {
+  CHECK(initialized_);
+
+  if (!FLAGS_auto_rebalance) {
+    return Status::OK();
+  }
+
+  RETURN_NOT_OK(StartClusterRebalancerThread());
+
+  return Status::OK();
+}
+
+void ClusterRebalancer::Shutdown() {
+  if (initialized_) {
+    string name = ToString();
+    LOG(INFO) << name << " shutting down...";
+
+    stop_background_threads_latch_.CountDown();
+
+    if (cluster_rebalancer_thread_) {
+      cluster_rebalancer_thread_->Join();
+    }
+
+    LOG(INFO) << name << " shutdown complete.";
+  }
+}
+
+string ClusterRebalancer::ToString() const {
+  return "ClusterRebalancer";
+}
+
+Status ClusterRebalancer::StartClusterRebalancerThread() {
+  return Thread::Create("server", "cluster-rebalancer", &ClusterRebalancer::ClusterRebalancerThread,
+                        this, &cluster_rebalancer_thread_);
+}
+
+void ClusterRebalancer::ClusterRebalancerThread() {
+  const MonoDelta kWait = MonoDelta::FromSeconds(60);
+  while (!RunOnceMode() && !stop_background_threads_latch_.WaitFor(kWait)) {
+    string dst;
+    StringAppendStrftime(&dst, "%H:%M", time(nullptr), true);
+    if (dst == FLAGS_rebalance_time) {
+      WARN_NOT_OK(RebalanceCluster(), "Unable to rebalance cluster");
+    }
+  }
+}
+
+Status ClusterRebalancer::RebalanceCluster() {
+  vector<string> args = {
+    "cluster",
+    "rebalance",
+    FLAGS_collector_master_addrs
+  };
+  string tool_stdout;
+  string tool_stderr;
+  RETURN_NOT_OK_PREPEND(tools::RunKuduTool(args, &tool_stdout, &tool_stderr),
+                        Substitute("out: $0, err: $1", tool_stdout, tool_stderr));
+  LOG(INFO) << std::endl
+            << tool_stdout;
+  return Status::OK();
+}
+
+Status ClusterRebalancer::ValidateHMTime(const string& time) {
+  Status err = Status::InvalidArgument(
+      Substitute("Invalid time format '$0', should in format 'HH:MM'", time));
+  if (time.size() != 5) {
+    return err;
+  }
+
+  int hour, minute;
+  int count = sscanf(time.c_str(), "%d:%d", &hour, &minute);
+  if (count == 2 &&
+      0 <= hour && hour < 24 &&
+      0 <= minute && minute < 60) {
+    return Status::OK();
+  }
+
+  return err;
+}
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/cluster_rebalancer.h b/src/kudu/collector/cluster_rebalancer.h
new file mode 100644
index 0000000..ed08736
--- /dev/null
+++ b/src/kudu/collector/cluster_rebalancer.h
@@ -0,0 +1,68 @@
+// 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 <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+class Thread;
+}  // namespace kudu
+
+namespace kudu {
+
+namespace collector {
+
+class ClusterRebalancer : public RefCounted<ClusterRebalancer> {
+ public:
+  ClusterRebalancer();
+  ~ClusterRebalancer();
+
+  Status Init();
+  Status Start();
+  void Shutdown();
+
+  std::string ToString() const;
+
+ private:
+  friend class RefCounted<ClusterRebalancer>;
+
+  FRIEND_TEST(TestClusterRebalancer, TestValidateHMTime);
+
+  // Start thread to rebalance cluster.
+  Status StartClusterRebalancerThread();
+  void ClusterRebalancerThread();
+  static Status RebalanceCluster();
+
+  static Status ValidateHMTime(const std::string& time);
+
+  bool initialized_;
+
+  CountDownLatch stop_background_threads_latch_;
+
+  scoped_refptr<Thread> cluster_rebalancer_thread_;
+
+  DISALLOW_COPY_AND_ASSIGN(ClusterRebalancer);
+};
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/collector-test.cc b/src/kudu/collector/collector-test.cc
new file mode 100644
index 0000000..e8e1298
--- /dev/null
+++ b/src/kudu/collector/collector-test.cc
@@ -0,0 +1,44 @@
+// 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/collector/collector.h"
+
+#include <gtest/gtest.h>
+
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+namespace kudu {
+namespace collector {
+
+TEST(TestCollector, TestValidateIntervalAndTimeout) {
+  // 'interval' in error range.
+  ASSERT_TRUE(Collector::ValidateIntervalAndTimeout(9, 1).IsInvalidArgument());
+  ASSERT_TRUE(Collector::ValidateIntervalAndTimeout(61, 1).IsInvalidArgument());
+
+  // 'timeout' in error range.
+  ASSERT_TRUE(Collector::ValidateIntervalAndTimeout(10, 0).IsInvalidArgument());
+  ASSERT_TRUE(Collector::ValidateIntervalAndTimeout(10, 10).IsInvalidArgument());
+
+  // Both 'interval' and 'timeout' are in valid range.
+  ASSERT_OK(Collector::ValidateIntervalAndTimeout(10, 9));
+  ASSERT_OK(Collector::ValidateIntervalAndTimeout(60, 9));
+  ASSERT_OK(Collector::ValidateIntervalAndTimeout(60, 59));
+}
+}  // namespace collector
+}  // namespace kudu
+
diff --git a/src/kudu/collector/collector.cc b/src/kudu/collector/collector.cc
new file mode 100644
index 0000000..1c930e9
--- /dev/null
+++ b/src/kudu/collector/collector.cc
@@ -0,0 +1,170 @@
+// 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/collector/collector.h"
+
+#include <ostream>
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+
+#include "kudu/collector/cluster_rebalancer.h"
+#include "kudu/collector/falcon_reporter.h"
+#include "kudu/collector/local_reporter.h"
+#include "kudu/collector/metrics_collector.h"
+#include "kudu/collector/nodes_checker.h"
+#include "kudu/collector/reporter_base.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/init.h"
+#include "kudu/util/env.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+DEFINE_string(collector_cluster_name, "",
+              "Cluster name of this collector to operate");
+DEFINE_string(collector_master_addrs, "",
+              "Comma-separated list of Kudu master addresses where each address is of "
+              "form 'hostname:port");
+DEFINE_int32(collector_interval_sec, 60,
+             "Number of interval seconds to collect metrics");
+DEFINE_string(collector_report_method, "",
+              "Which monitor system the metrics reported to. Now supported system: falcon");
+DEFINE_int32(collector_timeout_sec, 10,
+             "Number of seconds to wait for a master, tserver, or CLI tool to return metrics");
+DEFINE_int32(collector_warn_threshold_ms, 1000,
+             "If a task takes more than this number of milliseconds, issue a warning with a "
+             "trace.");
+
+DECLARE_string(principal);
+DECLARE_string(keytab_file);
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+namespace collector {
+
+Collector::Collector()
+  : initialized_(false),
+    stop_background_threads_latch_(1) {
+}
+
+Collector::~Collector() {
+  Shutdown();
+}
+
+Status Collector::Init() {
+  CHECK(!initialized_);
+
+  RETURN_NOT_OK(ValidateIntervalAndTimeout(FLAGS_collector_interval_sec,
+                                           FLAGS_collector_timeout_sec));
+  RETURN_NOT_OK(security::InitKerberosForServer(FLAGS_principal, FLAGS_keytab_file));
+
+  if (FLAGS_collector_report_method == "falcon") {
+    reporter_.reset(new FalconReporter());
+  } else if (FLAGS_collector_report_method == "local") {
+    reporter_.reset(new LocalReporter());
+  } else {
+    LOG(FATAL) << Substitute("Unsupported FLAGS_collector_report_method $0",
+                             FLAGS_collector_report_method);
+  }
+  CHECK_OK(reporter_->Init());
+  nodes_checker_.reset(new NodesChecker(reporter_));
+  CHECK_OK(nodes_checker_->Init());
+  metrics_collector_.reset(new MetricsCollector(nodes_checker_, reporter_));
+  CHECK_OK(metrics_collector_->Init());
+  cluster_rebalancer_.reset(new ClusterRebalancer());
+  CHECK_OK(cluster_rebalancer_->Init());
+
+  initialized_ = true;
+  return Status::OK();
+}
+
+Status Collector::Start() {
+  CHECK(initialized_);
+
+  google::FlushLogFiles(google::INFO); // Flush the startup messages.
+
+  RETURN_NOT_OK(StartExcessLogFileDeleterThread());
+
+  reporter_->Start();
+  nodes_checker_->Start();
+  metrics_collector_->Start();
+  cluster_rebalancer_->Start();
+
+  return Status::OK();
+}
+
+void Collector::Shutdown() {
+  if (initialized_) {
+    string name = ToString();
+    LOG(INFO) << name << " shutting down...";
+
+    reporter_->Shutdown();
+    metrics_collector_->Shutdown();
+    nodes_checker_->Shutdown();
+    cluster_rebalancer_->Shutdown();
+
+    stop_background_threads_latch_.CountDown();
+
+    if (excess_log_deleter_thread_) {
+      excess_log_deleter_thread_->Join();
+    }
+
+    LOG(INFO) << name << " shutdown complete.";
+  }
+}
+
+string Collector::ToString() const {
+  return "Collector";
+}
+
+Status Collector::StartExcessLogFileDeleterThread() {
+  // Try synchronously deleting excess log files once at startup to make sure it
+  // works, then start a background thread to continue deleting them in the
+  // future.
+  if (!FLAGS_logtostderr) {
+    RETURN_NOT_OK_PREPEND(DeleteExcessLogFiles(Env::Default()),
+                          "Unable to delete excess log files");
+  }
+  return Thread::Create("server", "excess-log-deleter", &Collector::ExcessLogFileDeleterThread,
+                        this, &excess_log_deleter_thread_);
+}
+
+void Collector::ExcessLogFileDeleterThread() {
+  // How often to attempt to clean up excess glog files.
+  const MonoDelta kWait = MonoDelta::FromSeconds(60);
+  while (!stop_background_threads_latch_.WaitFor(kWait)) {
+    WARN_NOT_OK(DeleteExcessLogFiles(Env::Default()), "Unable to delete excess log files");
+  }
+}
+
+Status Collector::ValidateIntervalAndTimeout(int interval, int timeout) {
+  if (10 <= interval && interval <= 60 &&
+      0 < timeout && timeout < interval) {
+    return Status::OK();
+  }
+
+  return Status::InvalidArgument(
+      Substitute("Invalid interval '$0'(should in range [10, 60]), "
+                 "or invalid timeout '$1'(should in range (0, interval))", interval, timeout));
+}
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/collector.h b/src/kudu/collector/collector.h
new file mode 100644
index 0000000..8e4e236
--- /dev/null
+++ b/src/kudu/collector/collector.h
@@ -0,0 +1,74 @@
+// 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 <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+class Thread;
+}  // namespace kudu
+
+namespace kudu {
+
+namespace collector {
+
+class ClusterRebalancer;
+class MetricsCollector;
+class NodesChecker;
+class ReporterBase;
+
+class Collector {
+ public:
+  Collector();
+  ~Collector();
+
+  Status Init();
+  Status Start();
+  void Shutdown();
+
+  std::string ToString() const;
+
+ private:
+  FRIEND_TEST(TestCollector, TestValidateIntervalAndTimeout);
+
+  // Start thread to remove excess glog files.
+  Status StartExcessLogFileDeleterThread();
+  void ExcessLogFileDeleterThread();
+
+  static Status ValidateIntervalAndTimeout(int interval, int timeout);
+
+  bool initialized_;
+
+  scoped_refptr<ReporterBase> reporter_;
+  scoped_refptr<MetricsCollector> metrics_collector_;
+  scoped_refptr<NodesChecker> nodes_checker_;
+  scoped_refptr<ClusterRebalancer> cluster_rebalancer_;
+
+  CountDownLatch stop_background_threads_latch_;
+  scoped_refptr<Thread> excess_log_deleter_thread_;
+
+  DISALLOW_COPY_AND_ASSIGN(Collector);
+};
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/collector_main.cc b/src/kudu/collector/collector_main.cc
new file mode 100644
index 0000000..afc0768
--- /dev/null
+++ b/src/kudu/collector/collector_main.cc
@@ -0,0 +1,73 @@
+// 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 <iostream>
+#include <string>
+
+#include <glog/logging.h>
+
+#include "kudu/collector/collector.h"
+#include "kudu/collector/collector_util.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/init.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/version_info.h"
+
+namespace kudu {
+namespace collector {
+
+static int CollectorMain(int argc, char** argv) {
+  InitKuduOrDie();
+
+  GFlagsMap default_flags = GetFlagsMap();
+
+  ParseCommandLineFlags(&argc, &argv, true);
+  if (argc != 1) {
+    std::cerr << "usage: " << argv[0] << std::endl;
+    return 1;
+  }
+  std::string nondefault_flags = GetNonDefaultFlags(default_flags);
+  InitGoogleLoggingSafe(argv[0]);
+
+  LOG(INFO) << "Collector non-default flags:\n"
+            << nondefault_flags << '\n'
+            << "Collector version:\n"
+            << VersionInfo::GetAllVersionInfo();
+
+  Collector collector;
+  LOG(INFO) << "Initializing collector...";
+  CHECK_OK(collector.Init());
+
+  LOG(INFO) << "Starting collector...";
+  CHECK_OK(collector.Start());
+
+  LOG(INFO) << "Collector successfully started.";
+  while (!RunOnceMode()) {
+    SleepFor(MonoDelta::FromSeconds(60));
+  }
+
+  return 0;
+}
+
+} // namespace collector
+} // namespace kudu
+
+int main(int argc, char** argv) {
+  return kudu::collector::CollectorMain(argc, argv);
+}
diff --git a/src/kudu/collector/collector_util-test.cc b/src/kudu/collector/collector_util-test.cc
new file mode 100644
index 0000000..8b29c56
--- /dev/null
+++ b/src/kudu/collector/collector_util-test.cc
@@ -0,0 +1,33 @@
+// 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/collector/collector_util.h"
+
+#include <gtest/gtest.h>
+
+namespace kudu {
+namespace collector {
+
+TEST(TestCollectorUtil, TestExtractHostName) {
+  ASSERT_EQ(ExtractHostName("1.2.3.4:5555"), "1.2.3.4");
+  ASSERT_EQ(ExtractHostName("host-name.bj:5555"), "host-name.bj");
+  ASSERT_EQ(ExtractHostName("1.2.3.4"), "1.2.3.4");
+  ASSERT_EQ(ExtractHostName("host-name.bj"), "host-name.bj");
+}
+}  // namespace collector
+}  // namespace kudu
+
diff --git a/src/kudu/collector/collector_util.cc b/src/kudu/collector/collector_util.cc
new file mode 100644
index 0000000..aa79c40
--- /dev/null
+++ b/src/kudu/collector/collector_util.cc
@@ -0,0 +1,46 @@
+// 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.
+//
+// Utility functions for generating data for use by tools and tests.
+
+#include "kudu/collector/collector_util.h"
+
+#include <stddef.h>
+
+#include <gflags/gflags_declare.h>
+
+DECLARE_string(collector_report_method);
+
+using std::string;
+
+namespace kudu {
+namespace collector {
+
+string ExtractHostName(const string& url) {
+  size_t pos = url.find(':');
+  if (pos == string::npos) {
+    return url;
+  }
+  return url.substr(0, pos);
+}
+
+bool RunOnceMode() {
+  static bool run_once = (FLAGS_collector_report_method == "local");
+  return run_once;
+}
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/collector_util.h b/src/kudu/collector/collector_util.h
new file mode 100644
index 0000000..f9badc8
--- /dev/null
+++ b/src/kudu/collector/collector_util.h
@@ -0,0 +1,32 @@
+// 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.
+//
+// Utility functions for generating data for use by tools and tests.
+
+#pragma once
+
+#include <string>
+
+namespace kudu {
+namespace collector {
+
+std::string ExtractHostName(const std::string& url);
+
+bool RunOnceMode();
+
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/falcon_reporter-test.cc b/src/kudu/collector/falcon_reporter-test.cc
new file mode 100644
index 0000000..85810c7
--- /dev/null
+++ b/src/kudu/collector/falcon_reporter-test.cc
@@ -0,0 +1,122 @@
+// 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/collector/falcon_reporter.h"
+
+#include <list>
+#include <string>
+#include <utility>
+
+#include <gflags/gflags_declare.h>
+#include <gtest/gtest.h>
+
+#include "kudu/collector/reporter_base.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/test_macros.h"
+
+DECLARE_string(collector_cluster_name);
+DECLARE_int32(collector_falcon_metrics_version);
+DECLARE_int32(collector_interval_sec);
+
+using std::list;
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+namespace collector {
+
+TEST(TestFalconReporter, TestSerializeItems) {
+  FLAGS_collector_interval_sec = 30;
+  FLAGS_collector_cluster_name = "test";
+  FLAGS_collector_falcon_metrics_version = 8;
+  scoped_refptr<FalconReporter> reporter(new FalconReporter());
+  list<scoped_refptr<ItemBase>> falcon_items;
+  string data;
+  ASSERT_OK(FalconReporter::SerializeItems(falcon_items, &data));
+  ASSERT_EQ(data, "");
+
+  falcon_items.emplace_back(reporter->ConstructItem(
+    "tserver1",
+    "scan_count",
+    "host",
+    1234567890,
+    12345,
+    "COUNTER",
+    ""));
+  ASSERT_OK(FalconReporter::SerializeItems(falcon_items, &data));
+  ASSERT_EQ(data, Substitute(
+                  R"*([{"endpoint":"tserver1","metric":"scan_count","timestamp":1234567890,)*"
+                  R"*("step":$0,"value":12345,"counterType":"COUNTER",)*"
+                  R"*("tags":"service=kudu,cluster=$1,level=host,v=$2"}])*",
+                  FLAGS_collector_interval_sec,
+                  FLAGS_collector_cluster_name,
+                  FLAGS_collector_falcon_metrics_version));
+
+  falcon_items.emplace_back(reporter->ConstructItem(
+    "table1",
+    "disk_size",
+    "table",
+    1234567891,
+    67890,
+    "GAUGE",
+    ""));
+  ASSERT_OK(FalconReporter::SerializeItems(falcon_items, &data));
+  ASSERT_EQ(data, Substitute(
+                  R"*([{"endpoint":"tserver1","metric":"scan_count","timestamp":1234567890,)*"
+                  R"*("step":$0,"value":12345,"counterType":"COUNTER",)*"
+                  R"*("tags":"service=kudu,cluster=$1,level=host,v=$2"},)*"
+                  R"*({"endpoint":"table1","metric":"disk_size","timestamp":1234567891,)*"
+                  R"*("step":$0,"value":67890,"counterType":"GAUGE",)*"
+                  R"*("tags":"service=kudu,cluster=$1,level=table,v=$2"}])*",
+                  FLAGS_collector_interval_sec,
+                  FLAGS_collector_cluster_name,
+                  FLAGS_collector_falcon_metrics_version));
+}
+
+void GenerateItems(const scoped_refptr<FalconReporter>& reporter, int count) {
+  list<scoped_refptr<ItemBase>> items;
+  for (int i = 0; i < count; ++i) {
+    items.emplace_back(reporter->ConstructItem("endpoint", "metric", "level", 0, i, "GAUGE", ""));
+  }
+  reporter->PushItems(std::move(items));
+}
+
+TEST(TestFalconReporter, TestPushAndPopItems) {
+  scoped_refptr<FalconReporter> reporter(new FalconReporter());
+  ASSERT_FALSE(reporter->HasItems());
+  NO_FATALS(GenerateItems(reporter, 1));
+  ASSERT_TRUE(reporter->HasItems());
+  NO_FATALS(GenerateItems(reporter, 9));
+  ASSERT_TRUE(reporter->HasItems());
+
+  list<scoped_refptr<ItemBase>> falcon_items;
+  reporter->PopItems(&falcon_items);
+  ASSERT_FALSE(reporter->HasItems());
+  ASSERT_EQ(falcon_items.size(), 10);
+
+  NO_FATALS(GenerateItems(reporter, 5));
+  ASSERT_TRUE(reporter->HasItems());
+
+  list<scoped_refptr<ItemBase>> falcon_items2;
+  reporter->PopItems(&falcon_items2);
+  ASSERT_FALSE(reporter->HasItems());
+  ASSERT_EQ(falcon_items2.size(), 5);
+}
+}  // namespace collector
+}  // namespace kudu
+
diff --git a/src/kudu/collector/falcon_reporter.cc b/src/kudu/collector/falcon_reporter.cc
new file mode 100644
index 0000000..e492fd7
--- /dev/null
+++ b/src/kudu/collector/falcon_reporter.cc
@@ -0,0 +1,255 @@
+// 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/collector/falcon_reporter.h"
+
+#include <kudu/util/curl_util.h>
+#include <stddef.h>
+
+#include <algorithm>
+#include <functional>
+#include <iterator>
+#include <mutex>
+#include <ostream>
+#include <utility>
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/jsonwriter.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/threadpool.h"
+#include "kudu/util/trace.h"
+
+DEFINE_bool(collector_direct_push, false,
+            "Whether to push collected items to falcon agent directly, "
+            "otherwise items will be cached and then pushed to falcon "
+            "agent asynchronous");
+DEFINE_string(collector_falcon_agent, "http://127.0.0.1:1988/v1/push",
+              "The falcon agent URL to push metrics to");
+DEFINE_int32(collector_falcon_metrics_version, 4,
+             "Version of metrics pushed to falcon, it will be tagged in "
+             "'tag' section of an item");
+DEFINE_int32(collector_falcon_pusher_count, 4,
+             "Thread count to push collected items to falcon agent");
+DEFINE_int32(collector_report_batch_size, 1000,
+            "Count of items will be pushed to falcon agent by batch");
+DEFINE_int32(collector_push_timeout_ms, 20,
+             "Timeout for pushing items to falcon agent");
+
+DECLARE_string(collector_cluster_name);
+DECLARE_int32(collector_interval_sec);
+DECLARE_int32(collector_timeout_sec);
+DECLARE_int32(collector_warn_threshold_ms);
+
+using std::list;
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+namespace collector {
+
+FalconReporter::FalconReporter()
+  : initialized_(false),
+    stop_background_threads_latch_(1) {
+}
+
+FalconReporter::~FalconReporter() {
+  Shutdown();
+}
+
+Status FalconReporter::Init() {
+  CHECK(!initialized_);
+
+  // Simple test falcon agent.
+  EasyCurl curl;
+  faststring dst;
+  RETURN_NOT_OK(curl.PostToURL(FLAGS_collector_falcon_agent, "", &dst));
+
+  initialized_ = true;
+  return Status::OK();
+}
+
+Status FalconReporter::Start() {
+  CHECK(initialized_);
+
+  if (!FLAGS_collector_direct_push) {
+    RETURN_NOT_OK(StartFalconPusherThreadPool());
+  }
+
+  return Status::OK();
+}
+
+void FalconReporter::Shutdown() {
+  if (initialized_) {
+    string name = ToString();
+    LOG(INFO) << name << " shutting down...";
+
+    stop_background_threads_latch_.CountDown();
+
+    pusher_thread_pool_->Wait();
+
+    LOG(INFO) << name << " shutdown complete.";
+  }
+}
+
+string FalconReporter::ToString() const {
+  return "FalconReporter";
+}
+
+scoped_refptr<ItemBase> FalconReporter::ConstructItem(string endpoint,
+                                                      string metric,
+                                                      string level,
+                                                      uint64_t timestamp,
+                                                      int64_t value,
+                                                      string counter_type,
+                                                      string extra_tags) {
+  scoped_refptr<ItemBase> tmp(new FalconItem(std::move(endpoint),
+                        std::move(metric),
+                        Substitute("service=kudu,cluster=$0,level=$1,v=$2$3",
+                                   FLAGS_collector_cluster_name, level,
+                                   FLAGS_collector_falcon_metrics_version,
+                                   extra_tags.empty() ? "" : "," + extra_tags),
+                        timestamp,
+                        FLAGS_collector_interval_sec,
+                        value,
+                        std::move(counter_type)));
+  return tmp;
+}
+
+Status FalconReporter::PushItems(list<scoped_refptr<ItemBase>> items) {
+  if (FLAGS_collector_direct_push) {
+    RETURN_NOT_OK(PushToAgent(std::move(items)));
+  } else {
+    std::lock_guard<RWMutex> l(items_lock_);
+    buffer_items_.splice(buffer_items_.end(), std::move(items));
+  }
+  return Status::OK();
+}
+
+Status FalconReporter::StartFalconPusherThreadPool() {
+  RETURN_NOT_OK(ThreadPoolBuilder("falcon-pusher")
+      .set_min_threads(FLAGS_collector_falcon_pusher_count)
+      .set_max_threads(FLAGS_collector_falcon_pusher_count)
+      .set_idle_timeout(MonoDelta::FromMilliseconds(1))
+      .Build(&pusher_thread_pool_));
+  for (int i = 0; i < FLAGS_collector_falcon_pusher_count; ++i) {
+    RETURN_NOT_OK(pusher_thread_pool_->SubmitFunc(std::bind(&FalconReporter::FalconPusher,
+                                                            this)));
+  }
+
+  return Status::OK();
+}
+
+void FalconReporter::FalconPusher() {
+  while (HasItems() || !stop_background_threads_latch_.WaitFor(MonoDelta::FromSeconds(1))) {
+    ReportItems();
+  }
+}
+
+void FalconReporter::ReportItems() {
+  MonoTime start(MonoTime::Now());
+  scoped_refptr<Trace> trace(new Trace);
+  ADOPT_TRACE(trace.get());
+  TRACE_EVENT0("collector", "FalconReporter::ReportItems");
+  TRACE("init");
+
+  list<scoped_refptr<ItemBase>> falcon_items;
+  PopItems(&falcon_items);
+  WARN_NOT_OK(PushToAgent(std::move(falcon_items)), "PushToAgent failed");
+  int64_t elapsed_ms = (MonoTime::Now() - start).ToMilliseconds();
+  if (elapsed_ms > FLAGS_collector_warn_threshold_ms) {
+    if (Trace::CurrentTrace()) {
+      LOG(WARNING) << "Trace:" << std::endl
+                   << Trace::CurrentTrace()->DumpToString();
+    }
+  }
+}
+
+bool FalconReporter::HasItems() const {
+  std::lock_guard<RWMutex> l(items_lock_);
+  return !buffer_items_.empty();
+}
+
+void FalconReporter::PopItems(list<scoped_refptr<ItemBase>>* falcon_items) {
+  int items_left = 0;
+  CHECK(falcon_items);
+  {
+    std::lock_guard<RWMutex> l(items_lock_);
+    auto end_item = buffer_items_.begin();
+    std::advance(end_item, std::min(buffer_items_.size(),
+                                    static_cast<size_t>(FLAGS_collector_report_batch_size)));
+    falcon_items->splice(falcon_items->end(), buffer_items_, buffer_items_.begin(), end_item);
+    items_left = buffer_items_.size();
+  }
+  if (items_left > 1000000) {
+    LOG(INFO) << "Items left " << items_left << std::endl;
+  }
+  TRACE(Substitute("Pop items, count $0", falcon_items->size()));
+}
+
+Status FalconReporter::PushToAgent(list<scoped_refptr<ItemBase>> falcon_items) {
+  string data;
+  RETURN_NOT_OK(SerializeItems(std::move(falcon_items), &data));
+
+  EasyCurl curl;
+  faststring dst;
+  curl.set_timeout(MonoDelta::FromMilliseconds(FLAGS_collector_push_timeout_ms));
+  RETURN_NOT_OK(curl.PostToURL(FLAGS_collector_falcon_agent, data, &dst));
+  TRACE(Substitute("Pushed items to agent, size $0", data.size()));
+  return Status::OK();
+}
+
+Status FalconReporter::SerializeItems(list<scoped_refptr<ItemBase>> items, string* data) {
+  CHECK(data);
+  if (items.empty()) {
+    return Status::OK();
+  }
+  std::ostringstream str;
+  JsonWriter jw(&str, JsonWriter::COMPACT);
+  jw.StartArray();
+  for (const auto& item : items) {
+    scoped_refptr<FalconItem> falcon_item = dynamic_cast<FalconItem*>(item.get());
+    jw.StartObject();
+    jw.String("endpoint");
+    jw.String(falcon_item->endpoint);
+    jw.String("metric");
+    jw.String(falcon_item->metric);
+    jw.String("timestamp");
+    jw.Uint64(falcon_item->timestamp);
+    jw.String("step");
+    jw.Int(falcon_item->step);
+    jw.String("value");
+    jw.Int64(falcon_item->value);
+    jw.String("counterType");
+    jw.String(falcon_item->counter_type);
+    jw.String("tags");
+    jw.String(falcon_item->tags);
+    jw.EndObject();
+  }
+  jw.EndArray();
+  *data = str.str();
+  TRACE(Substitute("SerializeItems done, count $0, size $1", items.size(), data->size()));
+  return Status::OK();
+}
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/falcon_reporter.h b/src/kudu/collector/falcon_reporter.h
new file mode 100644
index 0000000..3b3cde5
--- /dev/null
+++ b/src/kudu/collector/falcon_reporter.h
@@ -0,0 +1,108 @@
+// 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 <cstdint>
+#include <list>
+#include <memory>
+#include <string>
+#include <utility>
+
+#include <gtest/gtest_prod.h>
+
+#include "kudu/collector/reporter_base.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/rw_mutex.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+class ThreadPool;
+}  // namespace kudu
+
+namespace kudu {
+
+namespace collector {
+
+// Open-Falcon is a distributed and high-performance monitoring system,
+// see more details http://open-falcon.com
+struct FalconItem : public ItemBase {
+  FalconItem(std::string ep, std::string m, std::string t,
+             uint64_t ts, int32_t s, int64_t v, std::string ct)
+  : endpoint(std::move(ep)),
+    metric(std::move(m)),
+    tags(std::move(t)),
+    timestamp(ts),
+    step(s),
+    value(v),
+    counter_type(std::move(ct)) {
+  }
+  ~FalconItem() override = default;
+
+  std::string endpoint;
+  std::string metric;
+  std::string tags;
+  uint64_t timestamp;
+  int32_t step;
+  int64_t value;
+  std::string counter_type;
+};
+
+class FalconReporter : public ReporterBase {
+ public:
+  FalconReporter();
+  ~FalconReporter() override;
+
+  Status Init() override;
+  Status Start() override;
+  void Shutdown() override;
+
+  std::string ToString() const override;
+
+  scoped_refptr<ItemBase> ConstructItem(std::string endpoint,
+                                        std::string metric,
+                                        std::string level,
+                                        uint64_t timestamp,
+                                        int64_t value,
+                                        std::string counter_type,
+                                        std::string extra_tags) override;
+
+  Status PushItems(std::list<scoped_refptr<ItemBase>> items) override;
+
+ private:
+  FRIEND_TEST(TestFalconReporter, TestSerializeItems);
+  FRIEND_TEST(TestFalconReporter, TestPushAndPopItems);
+
+  Status StartFalconPusherThreadPool();
+  void FalconPusher();
+
+  bool HasItems() const;
+  void ReportItems();
+  void PopItems(std::list<scoped_refptr<ItemBase>>* falcon_items);
+  static Status PushToAgent(std::list<scoped_refptr<ItemBase>> falcon_items);
+  static Status SerializeItems(std::list<scoped_refptr<ItemBase>> items, std::string* data);
+
+  bool initialized_;
+
+  CountDownLatch stop_background_threads_latch_;
+  std::unique_ptr<ThreadPool> pusher_thread_pool_;
+
+  mutable RWMutex items_lock_;
+  std::list<scoped_refptr<ItemBase>> buffer_items_;
+};
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/local_reporter.cc b/src/kudu/collector/local_reporter.cc
new file mode 100644
index 0000000..04110bb
--- /dev/null
+++ b/src/kudu/collector/local_reporter.cc
@@ -0,0 +1,84 @@
+// 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/collector/local_reporter.h"
+
+#include <iostream>
+
+#include <glog/logging.h>
+
+#include "kudu/util/status.h"
+
+using std::list;
+using std::string;
+
+namespace kudu {
+namespace collector {
+
+LocalReporter::LocalReporter()
+  : initialized_(false) {
+}
+
+LocalReporter::~LocalReporter() {
+  Shutdown();
+}
+
+Status LocalReporter::Init() {
+  CHECK(!initialized_);
+
+  initialized_ = true;
+  return Status::OK();
+}
+
+Status LocalReporter::Start() {
+  CHECK(initialized_);
+
+  return Status::OK();
+}
+
+void LocalReporter::Shutdown() {
+  if (initialized_) {
+    string name = ToString();
+    LOG(INFO) << name << " shutting down...";
+
+    LOG(INFO) << name << " shutdown complete.";
+  }
+}
+
+string LocalReporter::ToString() const {
+  return "LocalReporter";
+}
+
+scoped_refptr<ItemBase> LocalReporter::ConstructItem(string endpoint,
+                                                     string metric,
+                                                     string level,
+                                                     uint64_t /*timestamp*/,
+                                                     int64_t value,
+                                                     string /*counter_type*/,
+                                                     string extra_tags) {
+  MutexLock l(output_lock_);
+  std::cout << level << " " << metric << " " << endpoint << " "
+      << (extra_tags.empty() ? "" : extra_tags + " ") << value << std::endl;
+  return nullptr;
+}
+
+Status LocalReporter::PushItems(list<scoped_refptr<ItemBase>> /*items*/) {
+  return Status::OK();
+}
+
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/local_reporter.h b/src/kudu/collector/local_reporter.h
new file mode 100644
index 0000000..d796b3b
--- /dev/null
+++ b/src/kudu/collector/local_reporter.h
@@ -0,0 +1,58 @@
+// 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 <cstdint>
+#include <list>
+#include <string>
+
+#include "kudu/collector/reporter_base.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+namespace collector {
+
+class LocalReporter : public ReporterBase {
+ public:
+  LocalReporter();
+  ~LocalReporter() override;
+
+  Status Init() override;
+  Status Start() override;
+  void Shutdown() override;
+
+  std::string ToString() const override;
+
+  scoped_refptr<ItemBase> ConstructItem(std::string endpoint,
+                                        std::string metric,
+                                        std::string level,
+                                        uint64_t timestamp,
+                                        int64_t value,
+                                        std::string counter_type,
+                                        std::string extra_tags) override;
+
+  Status PushItems(std::list<scoped_refptr<ItemBase>> items) override;
+
+ private:
+  bool initialized_;
+  Mutex output_lock_;
+};
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/metrics_collector-test.cc b/src/kudu/collector/metrics_collector-test.cc
new file mode 100644
index 0000000..4a3e050
--- /dev/null
+++ b/src/kudu/collector/metrics_collector-test.cc
@@ -0,0 +1,777 @@
+// 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/collector/metrics_collector.h"
+
+#include <stdint.h>
+
+#include <map>
+#include <set>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <gtest/gtest.h>
+#include <rapidjson/document.h>
+
+#include "kudu/collector/local_reporter.h"
+#include "kudu/collector/nodes_checker.h"
+#include "kudu/collector/reporter_base.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/jsonreader.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+
+DECLARE_bool(collector_request_merged_metrics);
+DECLARE_string(collector_attributes);
+DECLARE_string(collector_cluster_level_metrics);
+DECLARE_string(collector_metrics);
+DECLARE_string(collector_table_names);
+DECLARE_string(collector_metrics_types_for_test);
+
+using std::map;
+using std::set;
+using std::string;
+using std::unordered_map;
+using std::vector;
+
+namespace kudu {
+namespace collector {
+
+scoped_refptr<MetricsCollector> BuildCollector() {
+  scoped_refptr<ReporterBase> reporter(new LocalReporter());
+  scoped_refptr<NodesChecker> nodes_checker(new NodesChecker(reporter));
+  return new MetricsCollector(nodes_checker, reporter);
+}
+
+TEST(TestMetricsCollector, TestConvertStateToInt) {
+  int64_t result = 1;
+  ASSERT_OK(MetricsCollector::ConvertStateToInt("", &result));
+  ASSERT_EQ(result, 0);
+  ASSERT_OK(MetricsCollector::ConvertStateToInt("STOPPED", &result));
+  ASSERT_EQ(result, 0);
+  ASSERT_OK(MetricsCollector::ConvertStateToInt("RUNNINGSTOPPED", &result));
+  ASSERT_EQ(result, 0);
+  ASSERT_OK(MetricsCollector::ConvertStateToInt("RUNNINGBOOTSTRAPPING", &result));
+  ASSERT_EQ(result, 0);
+  ASSERT_OK(MetricsCollector::ConvertStateToInt("RUNNING", &result));
+  ASSERT_EQ(result, 1);
+  ASSERT_OK(MetricsCollector::ConvertStateToInt("RUNNINGRUNNING", &result));
+  ASSERT_EQ(result, 1);
+}
+
+TEST(TestMetricsCollector, TestGetHistValue) {
+  {
+    vector<MetricsCollector::SimpleHistogram> hist_values({{10, 100}});
+    ASSERT_EQ(MetricsCollector::GetHistValue(hist_values), 100);
+  }
+  {
+    vector<MetricsCollector::SimpleHistogram> hist_values({{10, 100},
+                                                           {20, 200}});
+    ASSERT_EQ(MetricsCollector::GetHistValue(hist_values), 167);
+  }
+}
+
+TEST(TestMetricsCollector, TestMergeToTableLevelMetrics) {
+  // Merge empty metrics.
+  {
+    vector<MetricsCollector::TablesMetrics> hosts_tables_metrics;
+    vector<MetricsCollector::TablesHistMetrics> hosts_tables_hist_metrics;
+    MetricsCollector::TablesMetrics tables_metrics;
+    MetricsCollector::TablesHistMetrics tables_hist_metrics;
+    ASSERT_OK(MetricsCollector::MergeToTableLevelMetrics(
+      hosts_tables_metrics, hosts_tables_hist_metrics,
+      &tables_metrics, &tables_hist_metrics));
+    ASSERT_TRUE(tables_metrics.empty());
+    ASSERT_TRUE(tables_hist_metrics.empty());
+  }
+  // Merge multi metrics.
+  {
+    vector<MetricsCollector::TablesMetrics> hosts_tables_metrics{
+        {  // host-1
+          {
+            "table1",
+            {
+              {"metric1", 1},
+              {"metric2", 2}
+            }
+          },
+          {
+            "table2",
+            {
+              {"metric1", 100},
+              {"metric3", 200}
+            }
+          }
+        },
+        {  // host-2
+          {
+            "table1",
+            {
+              {"metric1", 100},
+              {"metric2", 200}
+            }
+          },
+          {
+            "table2",
+            {
+              {"metric1", 1},
+              {"metric2", 2}
+            }
+          },
+          {
+            "table3",
+            {
+              {"metric1", 1},
+              {"metric2", 2}
+            }
+          }
+        }
+    };
+    vector<MetricsCollector::TablesHistMetrics> hosts_tables_hist_metrics{
+        {  // host-1
+          {
+            "table1",
+            {
+              {
+                "metric3",
+                {
+                  {10, 100},
+                  {20, 200}
+                }
+              },
+              {
+                "metric4",
+                {
+                  {30, 300},
+                  {40, 400}
+                }
+              }
+            }
+          },
+          {
+            "table2",
+            {
+              {
+                "metric3",
+                {
+                  {10, 200},
+                  {20, 300}
+                }
+              },
+              {
+                "metric4",
+                {
+                  {40, 300},
+                  {50, 400}
+                }
+              }
+            }
+          }
+        },
+        {  // host-2
+          {
+            "table1",
+            {
+              {
+                "metric3",
+                {
+                  {10, 100},
+                  {20, 200}
+                }
+              },
+              {
+                "metric4",
+                {
+                  {30, 300},
+                  {40, 400}
+                }
+              }
+            }
+          },
+          {
+            "table2",
+            {
+              {
+                "metric3",
+                {
+                  {10, 200},
+                  {20, 300}
+                }
+              },
+              {
+                "metric4",
+                {
+                  {40, 300},
+                  {50, 400}
+                }
+              }
+            }
+          },
+          {
+            "table3",
+            {
+              {
+                "metric3",
+                {
+                  {10, 200},
+                  {20, 300}
+                }
+              },
+              {
+                "metric4",
+                {
+                  {40, 300},
+                  {50, 400}
+                }
+              }
+            }
+          }
+        }
+    };
+    MetricsCollector::TablesMetrics tables_metrics;
+    MetricsCollector::TablesHistMetrics tables_hist_metrics;
+    ASSERT_OK(MetricsCollector::MergeToTableLevelMetrics(
+        hosts_tables_metrics, hosts_tables_hist_metrics,
+        &tables_metrics, &tables_hist_metrics));
+    ASSERT_EQ(tables_metrics, MetricsCollector::TablesMetrics({
+        {
+          "table1",
+          {
+            {"metric1", 101},
+            {"metric2", 202}
+          }
+        },
+        {
+          "table2",
+          {
+            {"metric1", 101},
+            {"metric2", 2},
+            {"metric3", 200},
+          }
+        },
+        {
+          "table3",
+          {
+            {"metric1", 1},
+            {"metric2", 2}
+          }
+        }
+    }));
+    ASSERT_EQ(tables_hist_metrics, MetricsCollector::TablesHistMetrics({
+        {
+          "table1",
+          {
+            {
+              "metric3",
+              {
+                {10, 100},
+                {20, 200},
+                {10, 100},
+                {20, 200}
+              }
+            },
+            {
+              "metric4",
+              {
+                {30, 300},
+                {40, 400},
+                {30, 300},
+                {40, 400}
+              }
+            }
+          }
+        },
+        {
+          "table2",
+          {
+            {
+              "metric3",
+              {
+                {10, 200},
+                {20, 300},
+                {10, 200},
+                {20, 300}
+              }
+            },
+            {
+              "metric4",
+              {
+                {40, 300},
+                {50, 400},
+                {40, 300},
+                {50, 400}
+              }
+            }
+          }
+        },
+        {
+          "table3",
+          {
+            {
+              "metric3",
+              {
+                {10, 200},
+                {20, 300}
+              }
+            },
+            {
+              "metric4",
+              {
+                {40, 300},
+                {50, 400}
+              }
+            }
+          }
+        }
+    }));
+  }
+}
+
+TEST(TestMetricsCollector, TestMergeToClusterLevelMetrics) {
+  // Merge empty metrics.
+  {
+    MetricsCollector::TablesMetrics tables_metrics;
+    MetricsCollector::TablesHistMetrics tables_hist_metrics;
+    MetricsCollector::Metrics cluster_metrics;
+    ASSERT_OK(MetricsCollector::MergeToClusterLevelMetrics(tables_metrics, tables_hist_metrics,
+                                                           &cluster_metrics));
+    ASSERT_TRUE(cluster_metrics.empty());
+  }
+  // Merge multi metrics.
+  {
+    MetricsCollector::TablesMetrics tables_metrics(
+        {
+          {
+            "table1",
+            {
+              {"metric1", 100}
+            }
+          },
+          {
+            "table2",
+            {
+              {"metric1", 10},
+              {"metric2", 20}
+            }
+          },
+          {
+            "table3",
+            {
+              {"metric1", 1},
+              {"metric2", 2},
+              {"metric3", 3}
+            }
+          }
+        }
+    );
+    MetricsCollector::TablesHistMetrics tables_hist_metrics;  // TODO(yingchun) not used now.
+    MetricsCollector::Metrics cluster_metrics({{"metric2", 0}});
+    ASSERT_OK(MetricsCollector::MergeToClusterLevelMetrics(tables_metrics, tables_hist_metrics,
+                                                           &cluster_metrics));
+    ASSERT_EQ(cluster_metrics, MetricsCollector::Metrics({
+        {
+          {"metric2", 22}
+        }
+    }));
+  }
+}
+
+TEST(TestMetricsCollector, TestParseMetrics) {
+  // Check ParseServerMetrics and ParseTabletMetrics.
+  {
+    string data;
+    JsonReader r(data);
+    const rapidjson::Value entity;
+    ASSERT_TRUE(MetricsCollector::ParseServerMetrics(r, &entity).IsNotSupported());
+    ASSERT_TRUE(MetricsCollector::ParseTabletMetrics(r, &entity).IsNotSupported());
+  }
+  // Check ParseTableMetrics.
+  {
+    auto collector = BuildCollector();
+    collector->metric_types_by_entity_type_["tablet"] = {
+        {"test_metric", "COUNTER"},
+        {"metric_counter1", "COUNTER"},
+        {"metric_counter2", "COUNTER"},
+        {"metric_histogram1", "HISTOGRAM"},
+        {"metric_histogram2", "HISTOGRAM"}
+    };
+    string data(
+        R"*([                                             )*"
+        R"*(  {                                           )*"
+        R"*(    "type": "server",                         )*"
+        R"*(    "id": "server1",                          )*"
+        R"*(    "attributes": {                           )*"
+        R"*(      "attrA": "val1",                        )*"
+        R"*(      "attrB": "val2"                         )*"
+        R"*(    },                                        )*"
+        R"*(    "metrics": [                              )*"
+        R"*(      {                                       )*"
+        R"*(        "name": "test_metric",                )*"
+        R"*(        "value": 123                          )*"
+        R"*(      }                                       )*"
+        R"*(    ]                                         )*"
+        R"*(  },                                          )*"
+        R"*(  {                                           )*"
+        R"*(    "type": "tablet",                         )*"
+        R"*(    "id": "tablet1",                          )*"
+        R"*(    "attributes": {                           )*"
+        R"*(      "attr1": "val1",                        )*"
+        R"*(      "attr2": "val2"                         )*"
+        R"*(    },                                        )*"
+        R"*(    "metrics": [                              )*"
+        R"*(      {                                       )*"
+        R"*(        "name": "test_metric",                )*"
+        R"*(        "value": 321                          )*"
+        R"*(      }                                       )*"
+        R"*(    ]                                         )*"
+        R"*(  },                                          )*"
+        R"*(  {                                           )*"
+        R"*(    "type": "table",                          )*"
+        R"*(    "id": "table1",                           )*"
+        R"*(    "attributes": {                           )*"
+        R"*(      "attr1": "val2",                        )*"
+        R"*(      "attr2": "val3"                         )*"
+        R"*(    },                                        )*"
+        R"*(    "metrics": [                              )*"
+        R"*(      {                                       )*"
+        R"*(        "name": "metric_counter1",            )*"
+        R"*(        "value": 10                           )*"
+        R"*(      },                                      )*"
+        R"*(      {                                       )*"
+        R"*(        "name": "metric_counter2",            )*"
+        R"*(        "value": 20                           )*"
+        R"*(      },                                      )*"
+        R"*(      {                                       )*"
+        R"*(        "name": "metric_histogram1",          )*"
+        R"*(        "total_count": 17,                    )*"
+        R"*(        "min": 6,                             )*"
+        R"*(        "mean": 47.8235,                      )*"
+        R"*(        "percentile_75": 62,                  )*"
+        R"*(        "percentile_95": 72,                  )*"
+        R"*(        "percentile_99": 73,                  )*"
+        R"*(        "percentile_99_9": 73,                )*"
+        R"*(        "percentile_99_99": 73,               )*"
+        R"*(        "max": 73,                            )*"
+        R"*(        "total_sum": 813                      )*"
+        R"*(      }                                       )*"
+        R"*(    ]                                         )*"
+        R"*(  },                                          )*"
+        R"*(  {                                           )*"
+        R"*(    "type": "table",                          )*"
+        R"*(    "id": "table2",                           )*"
+        R"*(    "attributes": {                           )*"
+        R"*(      "attr1": "val3",                        )*"
+        R"*(      "attr2": "val2"                         )*"
+        R"*(    },                                        )*"
+        R"*(    "metrics": [                              )*"
+        R"*(      {                                       )*"
+        R"*(        "name": "metric_counter1",            )*"
+        R"*(        "value": 100                          )*"
+        R"*(      },                                      )*"
+        R"*(      {                                       )*"
+        R"*(        "name": "metric_histogram1",          )*"
+        R"*(        "total_count": 170,                   )*"
+        R"*(        "min": 60,                            )*"
+        R"*(        "mean": 478.235,                      )*"
+        R"*(        "percentile_75": 620,                 )*"
+        R"*(        "percentile_95": 720,                 )*"
+        R"*(        "percentile_99": 730,                 )*"
+        R"*(        "percentile_99_9": 735,               )*"
+        R"*(        "percentile_99_99": 735,              )*"
+        R"*(        "max": 735,                           )*"
+        R"*(        "total_sum": 8130                     )*"
+        R"*(      },                                      )*"
+        R"*(      {                                       )*"
+        R"*(        "name": "metric_histogram2",          )*"
+        R"*(        "total_count": 34,                    )*"
+        R"*(        "min": 6,                             )*"
+        R"*(        "mean": 47.8235,                      )*"
+        R"*(        "percentile_75": 62,                  )*"
+        R"*(        "percentile_95": 72,                  )*"
+        R"*(        "percentile_99": 72,                  )*"
+        R"*(        "percentile_99_9": 73,                )*"
+        R"*(        "percentile_99_99": 73,               )*"
+        R"*(        "max": 73,                            )*"
+        R"*(        "total_sum": 813                      )*"
+        R"*(      }                                       )*"
+        R"*(    ]                                         )*"
+        R"*(  }                                           )*"
+        R"*(]                                             )*");
+
+    // Attribute filter is empty.
+    {
+      MetricsCollector::TablesMetrics tables_metrics;
+      MetricsCollector::TablesHistMetrics tables_hist_metrics;
+      MetricsCollector::Metrics host_metrics;
+      MetricsCollector::HistMetrics host_hist_metrics;
+      ASSERT_OK(collector->ParseMetrics(data,
+                                       &tables_metrics, &host_metrics,
+                                       &tables_hist_metrics, &host_hist_metrics));
+      ASSERT_EQ(tables_metrics, MetricsCollector::TablesMetrics({
+          {
+            "table1",
+            {
+              {"metric_counter1", 10},
+              {"metric_counter2", 20},
+            }
+          },
+          {
+            "table2",
+            {
+              {"metric_counter1", 100}
+            }
+          }
+      }));
+      ASSERT_EQ(tables_hist_metrics, MetricsCollector::TablesHistMetrics({
+          {
+            "table1",
+            {
+              {
+                "metric_histogram1_percentile_99",
+                {
+                  {17, 73}
+                }
+              }
+            }
+          },
+          {
+            "table2",
+            {
+              {
+                "metric_histogram1_percentile_99",
+                {
+                  {170, 730}
+                }
+              },
+              {
+                "metric_histogram2_percentile_99",
+                {
+                  {34, 72}
+                }
+              }
+            }
+          }
+      }));
+      ASSERT_EQ(host_metrics, MetricsCollector::Metrics({
+          {"metric_counter1", 110},
+          {"metric_counter2", 20}
+      }));
+      ASSERT_EQ(host_hist_metrics, MetricsCollector::HistMetrics({
+          {
+            "metric_histogram1_percentile_99",
+            {
+              {17, 73},
+              {170, 730}
+            }
+          },
+          {
+            "metric_histogram2_percentile_99",
+            {
+              {34, 72}
+            }
+          }
+      }));
+    }
+
+    // Attribute filter is not empty.
+    {
+      collector->attributes_filter_ = {{"attr1", {"val1", "val2"}}};
+
+      MetricsCollector::TablesMetrics tables_metrics;
+      MetricsCollector::TablesHistMetrics tables_hist_metrics;
+      MetricsCollector::Metrics host_metrics;
+      MetricsCollector::HistMetrics host_hist_metrics;
+      ASSERT_OK(collector->ParseMetrics(data,
+                                       &tables_metrics, &host_metrics,
+                                       &tables_hist_metrics, &host_hist_metrics));
+      ASSERT_EQ(tables_metrics, MetricsCollector::TablesMetrics({
+          {
+            "table1",
+            {
+              {"metric_counter1", 10},
+              {"metric_counter2", 20},
+            }
+          }
+      }));
+      ASSERT_EQ(tables_hist_metrics, MetricsCollector::TablesHistMetrics({
+          {
+            "table1",
+            {
+              {
+                "metric_histogram1_percentile_99",
+                {
+                  {17, 73}
+                }
+              }
+            }
+          }
+      }));
+      ASSERT_EQ(host_metrics, MetricsCollector::Metrics({
+          {"metric_counter1", 10},
+          {"metric_counter2", 20}
+      }));
+      ASSERT_EQ(host_hist_metrics, MetricsCollector::HistMetrics({
+          {
+            "metric_histogram1_percentile_99",
+            {
+              {17, 73},
+            }
+          }
+      }));
+    }
+  }
+}
+
+TEST(TestMetricsCollector, TestInitMetrics) {
+  FLAGS_collector_metrics_types_for_test =
+      R"*([                                                       )*"
+      R"*(  {                                                     )*"
+      R"*(    "type": "tablet",                                   )*"
+      R"*(    "id": "table1",                                     )*"
+      R"*(    "metrics": [                                        )*"
+      R"*(      {                                                 )*"
+      R"*(        "name": "counter_metric1",                      )*"
+      R"*(        "type": "counter"                               )*"
+      R"*(      },                                                )*"
+      R"*(      {                                                 )*"
+      R"*(        "name": "histogram_metric1",                    )*"
+      R"*(        "type": "histogram"                             )*"
+      R"*(      },                                                )*"
+      R"*(      {                                                 )*"
+      R"*(        "name": "gauge_metric1",                        )*"
+      R"*(        "type": "gauge"                                 )*"
+      R"*(      }                                                 )*"
+      R"*(    ]                                                   )*"
+      R"*(  },                                                    )*"
+      R"*(  {                                                     )*"
+      R"*(    "type": "tablet",                                   )*"
+      R"*(    "id": "table2",                                     )*"
+      R"*(    "metrics": [                                        )*"
+      R"*(      {                                                 )*"
+      R"*(        "name": "counter_metric1",                      )*"
+      R"*(        "type": "counter"                               )*"
+      R"*(      },                                                )*"
+      R"*(      {                                                 )*"
+      R"*(        "name": "histogram_metric1",                    )*"
+      R"*(        "type": "histogram"                             )*"
+      R"*(      },                                                )*"
+      R"*(      {                                                 )*"
+      R"*(        "name": "gauge_metric1",                        )*"
+      R"*(        "type": "gauge"                                 )*"
+      R"*(      }                                                 )*"
+      R"*(    ]                                                   )*"
+      R"*(  },                                                    )*"
+      R"*(  {                                                     )*"
+      R"*(    "type": "server",                                   )*"
+      R"*(    "metrics": [                                        )*"
+      R"*(      {                                                 )*"
+      R"*(        "name": "counter_metric2",                      )*"
+      R"*(        "type": "counter"                               )*"
+      R"*(      },                                                )*"
+      R"*(      {                                                 )*"
+      R"*(        "name": "histogram_metric2",                    )*"
+      R"*(        "type": "histogram"                             )*"
+      R"*(      },                                                )*"
+      R"*(      {                                                 )*"
+      R"*(        "name": "gauge_metric2",                        )*"
+      R"*(        "type": "gauge"                                 )*"
+      R"*(      }                                                 )*"
+      R"*(    ]                                                   )*"
+      R"*(  }                                                     )*"
+      R"*(]                                                       )*";
+  auto collector = BuildCollector();
+  ASSERT_OK(collector->InitMetrics());
+  map<string, MetricsCollector::MetricTypes> expect_metric_types({
+      {
+        "tablet",
+        {
+          {"counter_metric1", "COUNTER"},
+          {"histogram_metric1", "HISTOGRAM"},
+          {"gauge_metric1", "GAUGE"},
+        }
+      },
+      {
+        "server",
+        {
+          {"counter_metric2", "COUNTER"},
+          {"histogram_metric2", "HISTOGRAM"},
+          {"gauge_metric2", "GAUGE"},
+        }
+      }
+  });
+  ASSERT_EQ(collector->metric_types_by_entity_type_, expect_metric_types);
+}
+
+TEST(TestMetricsCollector, TestInitFilters) {
+  FLAGS_collector_attributes = "attr1:val1,val2;attr2:val1";
+  auto collector = BuildCollector();
+  ASSERT_OK(collector->InitFilters());
+  unordered_map<string, set<string>> expect_attributes_filter({
+      {
+        "attr1",
+        {"val1", "val2"}
+      },
+      {
+        "attr2",
+        {"val1"}
+      }
+  });
+  ASSERT_EQ(collector->attributes_filter_, expect_attributes_filter);
+}
+
+#define CHECK_URL_PARAMETERS(metrics, request_merged, attributes, table_names, expect_url)        \
+do {                                                                                              \
+  FLAGS_collector_metrics = metrics;                                                              \
+  FLAGS_collector_request_merged_metrics = request_merged;                                        \
+  FLAGS_collector_attributes = attributes;                                                        \
+  FLAGS_collector_table_names = table_names;                                                      \
+  auto collector = BuildCollector();                                                              \
+  ASSERT_OK(collector->InitFilters());                                                            \
+  ASSERT_OK(collector->InitMetricsUrlParameters());                                               \
+  ASSERT_EQ(collector->metric_url_parameters_, expect_url);                                       \
+} while (false)
+
+TEST(TestMetricsCollector, TestInitMetricsUrlParameters) {
+  CHECK_URL_PARAMETERS("", true, "", "",
+      "/metrics?compact=1&origin=false&merge=true");
+  CHECK_URL_PARAMETERS("m1,m2,m3", true, "", "",
+      "/metrics?compact=1&metrics=m1,m2,m3&origin=false&merge=true");
+  // TODO(yingchun): now FLAGS_collector_request_merged_metrics must be true
+  //CHECK_URL_PARAMETERS("", false, "", "",
+  //    "/metrics?compact=1");
+  CHECK_URL_PARAMETERS("", true, "attr1:a1,a2;attr2:a3", "",
+      "/metrics?compact=1&origin=false&merge=true&attributes=attr2,a3,attr1,a1,attr1,a2,");
+  CHECK_URL_PARAMETERS("", true, "", "t1,t2,t3",
+      "/metrics?compact=1&origin=false&merge=true&table_names=t1,t2,t3");
+}
+
+TEST(TestMetricsCollector, TestInitClusterLevelMetrics) {
+  FLAGS_collector_cluster_level_metrics = "m1,m2,m3";
+  auto collector = BuildCollector();
+  ASSERT_OK(collector->InitClusterLevelMetrics());
+  MetricsCollector::Metrics cluster_metrics({
+      {"m1", 0},
+      {"m2", 0},
+      {"m3", 0},
+  });
+  ASSERT_EQ(collector->cluster_metrics_, cluster_metrics);
+}
+}  // namespace collector
+}  // namespace kudu
+
diff --git a/src/kudu/collector/metrics_collector.cc b/src/kudu/collector/metrics_collector.cc
new file mode 100644
index 0000000..e5adfcb
--- /dev/null
+++ b/src/kudu/collector/metrics_collector.cc
@@ -0,0 +1,852 @@
+// 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/collector/metrics_collector.h"
+
+#include <string.h>
+
+#include <cmath>
+#include <functional>
+#include <list>
+#include <ostream>
+#include <set>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <rapidjson/rapidjson.h>
+
+#include "kudu/collector/collector_util.h"
+#include "kudu/collector/nodes_checker.h"
+#include "kudu/collector/reporter_base.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/util/curl_util.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/jsonreader.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/string_case.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/threadpool.h"
+#include "kudu/util/trace.h"
+#include "kudu/util/zlib.h"
+
+DEFINE_string(collector_attributes, "",
+              "Entity attributes to collect (semicolon-separated list of entity attribute "
+              "name and values). e.g. attr_name1:attr_val1,attr_val2;attr_name2:attr_val3");
+DEFINE_string(collector_cluster_level_metrics, "on_disk_size,on_disk_data_size",
+              "Metric names which should be merged and pushed to cluster level view "
+              "(comma-separated list of metric names)");
+DEFINE_bool(collector_ignore_hosttable_level_metrics, false,
+            "Whether to ignore to report host-table level metrics.");
+DEFINE_string(collector_metrics, "",
+              "Metrics to collect (comma-separated list of metric names)");
+DEFINE_string(collector_metrics_types_for_test, "",
+              "Only for test, used to initialize metric_types_by_entity_type_");
+DEFINE_bool(collector_request_merged_metrics, true,
+            "Whether to request merged metrics and exclude unmerged metrics from server");
+DEFINE_string(collector_table_names, "",
+              "Table names to collect (comma-separated list of table names)");
+
+DECLARE_string(collector_cluster_name);
+DECLARE_int32(collector_interval_sec);
+DECLARE_int32(collector_timeout_sec);
+DECLARE_int32(collector_warn_threshold_ms);
+
+using rapidjson::Value;
+using std::list;
+using std::map;
+using std::set;
+using std::string;
+using std::vector;
+using std::unordered_map;
+using strings::Substitute;
+
+namespace kudu {
+namespace collector {
+
+const set<string> MetricsCollector::kRegisterPercentiles = {"percentile_99"};
+
+MetricsCollector::MetricsCollector(scoped_refptr<NodesChecker> nodes_checker,
+                                   scoped_refptr<ReporterBase> reporter)
+  : initialized_(false),
+    nodes_checker_(std::move(nodes_checker)),
+    reporter_(std::move(reporter)),
+    stop_background_threads_latch_(1) {
+}
+
+MetricsCollector::~MetricsCollector() {
+  Shutdown();
+}
+
+Status MetricsCollector::Init() {
+  CHECK(!initialized_);
+
+  RETURN_NOT_OK(ValidateTableFilter(FLAGS_collector_attributes, FLAGS_collector_table_names));
+  RETURN_NOT_OK(InitMetrics());
+  RETURN_NOT_OK(InitFilters());
+  CHECK(attributes_filter_.empty());  // TODO(yingchun) disable now
+  RETURN_NOT_OK(InitMetricsUrlParameters());
+  RETURN_NOT_OK(InitClusterLevelMetrics());
+
+  initialized_ = true;
+  return Status::OK();
+}
+
+Status MetricsCollector::Start() {
+  CHECK(initialized_);
+
+  RETURN_NOT_OK(StartMetricCollectorThread());
+
+  return Status::OK();
+}
+
+void MetricsCollector::Shutdown() {
+  if (initialized_) {
+    string name = ToString();
+    LOG(INFO) << name << " shutting down...";
+
+    stop_background_threads_latch_.CountDown();
+
+    if (metric_collector_thread_) {
+      metric_collector_thread_->Join();
+    }
+
+    LOG(INFO) << name << " shutdown complete.";
+  }
+}
+
+string MetricsCollector::ToString() const {
+  return "MetricsCollector";
+}
+
+Status MetricsCollector::StartMetricCollectorThread() {
+  return Thread::Create("server", "metric-collector", &MetricsCollector::MetricCollectorThread,
+                        this, &metric_collector_thread_);
+}
+
+void MetricsCollector::MetricCollectorThread() {
+  MonoTime collect_time;
+  do {
+    collect_time = MonoTime::Now();
+    WARN_NOT_OK(CollectAndReportMetrics(), "Unable to collect metrics");
+    collect_time += MonoDelta::FromSeconds(FLAGS_collector_interval_sec);
+  } while (!RunOnceMode() && !stop_background_threads_latch_.WaitUntil(collect_time));
+  LOG(INFO) << "MetricCollectorThread exit";
+}
+
+Status MetricsCollector::UpdateThreadPool(int32_t thread_count) {
+  if (host_metric_collector_thread_pool_ &&
+      host_metric_collector_thread_pool_->num_threads() == thread_count) {
+    return Status::OK();
+  }
+
+  if (host_metric_collector_thread_pool_) {
+    host_metric_collector_thread_pool_->Shutdown();
+  }
+  TRACE("Old thread pool shutdown");
+
+  RETURN_NOT_OK(ThreadPoolBuilder("host-metric-collector")
+      .set_min_threads(thread_count)
+      .set_max_threads(thread_count)
+      .set_idle_timeout(MonoDelta::FromMilliseconds(1))
+      .Build(&host_metric_collector_thread_pool_));
+  TRACE("New thread pool built");
+
+  return Status::OK();
+}
+
+Status MetricsCollector::ValidateTableFilter(const string& attribute_filter,
+                                             const string& /*table_filter*/) {
+  if (attribute_filter.empty()) {
+    return Status::OK();
+  }
+
+  return Status::InvalidArgument("attribute filter is not supported now");
+}
+
+Status MetricsCollector::InitMetrics() {
+  string resp;
+  if (PREDICT_TRUE(FLAGS_collector_metrics_types_for_test.empty())) {
+    RETURN_NOT_OK(GetMetrics(
+        nodes_checker_->GetFirstMaster() + "/metrics?include_schema=1", &resp));
+  } else {
+    resp = FLAGS_collector_metrics_types_for_test;
+  }
+  JsonReader r(resp);
+  RETURN_NOT_OK(r.Init());
+  vector<const Value*> entities;
+  RETURN_NOT_OK(r.ExtractObjectArray(r.root(), nullptr, &entities));
+
+  map<string, MetricTypes> metric_types_by_entity_type;
+  bool tablet_entity_inited = false;
+  bool server_entity_inited = false;
+  for (const Value* entity : entities) {
+    string entity_type;
+    CHECK_OK(r.ExtractString(entity, "type", &entity_type));
+    if (entity_type == "tablet") {
+      if (tablet_entity_inited) continue;
+      EmplaceOrDie(&metric_types_by_entity_type, std::make_pair("tablet", MetricTypes()));
+      auto& tablet_metric_types = FindOrDie(metric_types_by_entity_type, "tablet");
+      ExtractMetricTypes(r, entity, &tablet_metric_types);
+      tablet_entity_inited = true;
+    } else if (entity_type == "server") {
+      if (server_entity_inited) continue;
+      EmplaceOrDie(&metric_types_by_entity_type, std::make_pair("server", MetricTypes()));
+      auto& server_metric_types = FindOrDie(metric_types_by_entity_type, "server");
+      ExtractMetricTypes(r, entity, &server_metric_types);
+      server_entity_inited = true;
+    } else {
+      LOG(WARNING) << "unhandled entity type " << entity_type;
+    }
+  }
+  metric_types_by_entity_type_.swap(metric_types_by_entity_type);
+  return Status::OK();
+}
+
+Status MetricsCollector::ExtractMetricTypes(const JsonReader& r,
+                                            const Value* entity,
+                                            MetricTypes* metric_types) {
+  CHECK(metric_types);
+  vector<const Value*> metrics;
+  RETURN_NOT_OK(r.ExtractObjectArray(entity, "metrics", &metrics));
+  for (const Value* metric : metrics) {
+    string name;
+    RETURN_NOT_OK(r.ExtractString(metric, "name", &name));
+    string type;
+    RETURN_NOT_OK(r.ExtractString(metric, "type", &type));
+    string upper_type;
+    ToUpperCase(type, &upper_type);
+    EmplaceOrDie(metric_types, std::make_pair(name, upper_type));
+  }
+  return Status::OK();
+}
+
+Status MetricsCollector::InitFilters() {
+  unordered_map<string, set<string>> attributes_filter;
+  vector<string> attribute_values_by_name_list =
+      Split(FLAGS_collector_attributes, ";", strings::SkipEmpty());
+  for (const auto& attribute_values_by_name : attribute_values_by_name_list) {
+    vector<string> attribute_name_and_values =
+        Split(attribute_values_by_name, ":", strings::SkipEmpty());
+    CHECK_EQ(attribute_name_and_values.size(), 2);
+    set<string> values(Split(attribute_name_and_values[1], ",", strings::SkipEmpty()));
+    CHECK(!values.empty());
+    EmplaceOrDie(&attributes_filter, std::make_pair(attribute_name_and_values[0], values));
+  }
+  attributes_filter_.swap(attributes_filter);
+  return Status::OK();
+}
+
+Status MetricsCollector::InitMetricsUrlParameters() {
+  metric_url_parameters_ = "/metrics?compact=1";
+  if (!FLAGS_collector_metrics.empty()) {
+    metric_url_parameters_ += "&metrics=" + FLAGS_collector_metrics;
+  }
+  if (FLAGS_collector_request_merged_metrics) {
+    metric_url_parameters_ += "&origin=false&merge=true";
+  } else {
+    LOG(FATAL) << "Non-merge mode is not supported now, you should set "
+                  "FLAGS_collector_request_merged_metrics to true if you "
+                  "want collector work well";
+  }
+
+  // TODO(yingchun) This is supported since version 1.10
+  if (!attributes_filter_.empty()) {
+    metric_url_parameters_ += "&attributes=";
+  }
+  for (const auto& attribute_filter : attributes_filter_) {
+    for (const auto& value : attribute_filter.second) {
+      metric_url_parameters_ += Substitute("$0,$1,", attribute_filter.first, value);
+    }
+  }
+  // TODO(yingchun) This is supported since internal version 1.8.0
+  if (!FLAGS_collector_table_names.empty()) {
+    metric_url_parameters_ += "&table_names=" + FLAGS_collector_table_names;
+  }
+  return Status::OK();
+}
+
+Status MetricsCollector::InitClusterLevelMetrics() {
+  Metrics cluster_metrics;
+  vector<string> metric_names =
+      Split(FLAGS_collector_cluster_level_metrics, ",", strings::SkipEmpty());
+  for (const auto& metric_name : metric_names) {
+    cluster_metrics[metric_name] = 0;
+  }
+  cluster_metrics_.swap(cluster_metrics);
+  return Status::OK();
+}
+
+Status MetricsCollector::CollectAndReportMetrics() {
+  LOG(INFO) << "Start to CollectAndReportMetrics";
+  MonoTime start(MonoTime::Now());
+  scoped_refptr<Trace> trace(new Trace);
+  ADOPT_TRACE(trace.get());
+  TRACE_EVENT0("collector", "MetricsCollector::CollectAndReportMetrics");
+  TRACE("init");
+  vector<string> tserver_http_addrs = nodes_checker_->GetNodes();
+  TRACE("Nodes got");
+  if (tserver_http_addrs.empty()) {
+    return Status::OK();
+  }
+  RETURN_NOT_OK(UpdateThreadPool(static_cast<int32_t>(tserver_http_addrs.size())));
+  vector<TablesMetrics> hosts_metrics_by_table_name(tserver_http_addrs.size());
+  vector<TablesHistMetrics> hosts_hist_metrics_by_table_name(tserver_http_addrs.size());
+  for (int i = 0; i < tserver_http_addrs.size(); ++i) {
+    RETURN_NOT_OK(host_metric_collector_thread_pool_->SubmitFunc(
+      std::bind(&MetricsCollector::CollectAndReportHostLevelMetrics,
+                this,
+                tserver_http_addrs[i] + metric_url_parameters_,
+                &hosts_metrics_by_table_name[i],
+                &hosts_hist_metrics_by_table_name[i])));
+  }
+  TRACE("Thead pool jobs submitted");
+  host_metric_collector_thread_pool_->Wait();
+  TRACE("Thead pool jobs done");
+
+  // Merge to table level metrics.
+  TablesMetrics metrics_by_table_name;
+  TablesHistMetrics hist_metrics_by_table_name;
+  RETURN_NOT_OK(MergeToTableLevelMetrics(hosts_metrics_by_table_name,
+                                         hosts_hist_metrics_by_table_name,
+                                         &metrics_by_table_name,
+                                         &hist_metrics_by_table_name));
+
+  // Merge to cluster level metrics.
+  Metrics cluster_metrics(cluster_metrics_);
+  RETURN_NOT_OK(MergeToClusterLevelMetrics(metrics_by_table_name,
+                                           hist_metrics_by_table_name,
+                                           &cluster_metrics));
+
+  auto timestamp = static_cast<uint64_t>(WallTime_Now());
+
+  // Push table level metrics.
+  RETURN_NOT_OK(ReportTableLevelMetrics(timestamp,
+                                        metrics_by_table_name,
+                                        hist_metrics_by_table_name));
+
+  // Push cluster level metrics.
+  RETURN_NOT_OK(ReportClusterLevelMetrics(timestamp, cluster_metrics));
+
+  int64_t elapsed_ms = (MonoTime::Now() - start).ToMilliseconds();
+  if (elapsed_ms > FLAGS_collector_warn_threshold_ms) {
+    if (Trace::CurrentTrace()) {
+      LOG(WARNING) << "Trace:" << std::endl
+                   << Trace::CurrentTrace()->DumpToString();
+    }
+  }
+
+  return Status::OK();
+}
+
+Status MetricsCollector::MergeToTableLevelMetrics(
+  const vector<TablesMetrics>& hosts_metrics_by_table_name,
+  const vector<TablesHistMetrics>& hosts_hist_metrics_by_table_name,
+  TablesMetrics* metrics_by_table_name,
+  TablesHistMetrics* hist_metrics_by_table_name) {
+  CHECK(metrics_by_table_name);
+  CHECK(hist_metrics_by_table_name);
+
+  // GAUGE/COUNTER type metrics.
+  int metrics_count = 0;
+  for (const auto& host_metrics_by_table_name : hosts_metrics_by_table_name) {
+    for (const auto& table_metrics1 : host_metrics_by_table_name) {
+      const auto& table_name = table_metrics1.first;
+      const auto& metrics = table_metrics1.second;
+      metrics_count += metrics.size();
+      if (EmplaceIfNotPresent(metrics_by_table_name, std::make_pair(table_name, metrics))) {
+        continue;
+      }
+      // This table has been fetched by some other tserver.
+      auto& table_metrics = FindOrDie(*metrics_by_table_name, table_name);
+      for (const auto& metric_value : metrics) {
+        const auto& metric = metric_value.first;
+        const auto& value = metric_value.second;
+        if (EmplaceIfNotPresent(&table_metrics, std::make_pair(metric, value))) {
+          continue;
+        }
+        // This metric has been fetched by some other tserver.
+        auto& old_value = FindOrDie(table_metrics, metric);
+        old_value += value;
+      }
+    }
+  }
+  TRACE(Substitute("Table GAUGE/COUNTER type metrics merged, count $0", metrics_count));
+
+  // HISTOGRAM type metrics.
+  metrics_count = 0;
+  for (const auto& host_hist_metrics_by_table_name : hosts_hist_metrics_by_table_name) {
+    for (const auto& table_hist_metrics1 : host_hist_metrics_by_table_name) {
+      const auto& table_name = table_hist_metrics1.first;
+      const auto& metrics = table_hist_metrics1.second;
+      metrics_count += metrics.size();
+      if (EmplaceIfNotPresent(hist_metrics_by_table_name, std::make_pair(table_name, metrics))) {
+        continue;
+      }
+      // This table has been fetched by some other tserver.
+      auto& table_hist_metrics = FindOrDie(*hist_metrics_by_table_name, table_name);
+      for (const auto& metric_hist_values : metrics) {
+        const auto& metric = metric_hist_values.first;
+        const auto& hist_values = metric_hist_values.second;
+        if (EmplaceIfNotPresent(&table_hist_metrics, std::make_pair(metric, hist_values))) {
+          continue;
+        }
+        // This metric has been fetched by some other tserver.
+        auto& old_hist_values = FindOrDie(table_hist_metrics, metric);
+        for (auto& hist_value : hist_values) {
+          old_hist_values.emplace_back(hist_value);
+        }
+      }
+    }
+  }
+  TRACE(Substitute("Table HISTOGRAM type metrics merged, count $0", metrics_count));
+
+  return Status::OK();
+}
+
+Status MetricsCollector::MergeToClusterLevelMetrics(
+    const TablesMetrics& metrics_by_table_name,
+    const TablesHistMetrics& /*hist_metrics_by_table_name*/,
+    Metrics* cluster_metrics) {
+  CHECK(cluster_metrics);
+  if (!cluster_metrics->empty()) {
+    for (const auto& table_metrics : metrics_by_table_name) {
+      for (auto& cluster_metric : *cluster_metrics) {
+        auto *find = FindOrNull(table_metrics.second, cluster_metric.first);
+        if (find) {
+          cluster_metric.second += *find;
+        }
+      }
+    }
+  }
+  TRACE(Substitute("Cluster metrics merged, count $0", cluster_metrics->size()));
+
+  return Status::OK();
+}
+
+Status MetricsCollector::GetNumberMetricValue(const rapidjson::Value* metric,
+                                              const string& metric_name /*metric_name*/,
+                                              int64_t* result) const {
+  CHECK(result);
+  if (metric->IsUint64() || metric->IsInt64() || metric->IsUint() || metric->IsInt()) {
+    *result = metric->GetInt64();
+    return Status::OK();
+  }
+
+  if (metric->IsDouble()) {
+    double result_temp = metric->GetDouble();
+    // Multiply by 1000000 and convert to int64_t to avoid much data loss and keep compatibility
+    // with monitor system like Falcon.
+    *result = static_cast<int64_t>(result_temp * 1000000);
+    return Status::OK();
+  }
+
+  return Status::NotSupported(Substitute("unsupported metric $0", metric_name));
+}
+
+Status MetricsCollector::GetStringMetricValue(const Value* metric,
+                                              const string& metric_name,
+                                              int64_t* result) const {
+  CHECK(result);
+  string value(metric->GetString());
+  if (metric_name == "state") {
+    return ConvertStateToInt(value, result);
+  }
+  return Status::NotSupported(Substitute("unsupported metric $0", metric_name));
+}
+
+Status MetricsCollector::ConvertStateToInt(const string& value, int64_t* result) {
+  CHECK(result);
+  // TODO(yingchun) Here, table state is merged by several original tablet states, which is
+  // contacted by several sub-strings, like 'RUNNING', 'BOOTSTRAPPING', etc. It's tricky to
+  // fetch state now, we will improve in server side later.
+  const char* running = "RUNNING";
+  if (value.empty() || value.size() % strlen(running) != 0) {
+    *result = 0;
+    return Status::OK();
+  }
+  for (int i = 0; i < value.size(); i += strlen(running)) {
+    if (0 != strncmp(running, value.c_str() + i, strlen(running))) {
+      *result = 0;
+      return Status::OK();
+    }
+  }
+  *result = 1;
+  return Status::OK();
+}
+
+bool MetricsCollector::FilterByAttribute(const JsonReader& r,
+                                         const rapidjson::Value* entity) const {
+  if (attributes_filter_.empty()) {
+    return false;
+  }
+  const Value* attributes;
+  CHECK_OK(r.ExtractObject(entity, "attributes", &attributes));
+  for (const auto& name_values : attributes_filter_) {
+    string value;
+    Status s = r.ExtractString(attributes, name_values.first.c_str(), &value);
+    if (s.ok() && ContainsKey(name_values.second, value)) {
+      return false;
+    }
+  }
+  return true;
+}
+
+Status MetricsCollector::ParseServerMetrics(const JsonReader& /*r*/,
+                                            const rapidjson::Value* /*entity*/) {
+  return Status::NotSupported("server entity is not supported");
+}
+
+Status MetricsCollector::ParseTableMetrics(const JsonReader& r,
+                                           const rapidjson::Value* entity,
+                                           TablesMetrics* metrics_by_table_name,
+                                           Metrics* host_metrics,
+                                           TablesHistMetrics* hist_metrics_by_table_name,
+                                           HistMetrics* host_hist_metrics) const {
+  CHECK(metrics_by_table_name);
+  CHECK(host_metrics);
+  CHECK(hist_metrics_by_table_name);
+  CHECK(host_hist_metrics);
+
+  string table_name;
+  CHECK_OK(r.ExtractString(entity, "id", &table_name));
+  CHECK(!ContainsKey(*metrics_by_table_name, table_name));
+  CHECK(!ContainsKey(*hist_metrics_by_table_name, table_name));
+
+  EmplaceOrDie(metrics_by_table_name, std::make_pair(table_name, Metrics()));
+  auto& table_metrics = FindOrDie(*metrics_by_table_name, table_name);
+
+  EmplaceOrDie(hist_metrics_by_table_name, std::make_pair(table_name, HistMetrics()));
+  auto& table_hist_metrics = FindOrDie(*hist_metrics_by_table_name, table_name);
+
+  vector<const Value*> metrics;
+  CHECK_OK(r.ExtractObjectArray(entity, "metrics", &metrics));
+  for (const Value* metric : metrics) {
+    string name;
+    CHECK_OK(r.ExtractString(metric, "name", &name));
+    const auto* tablet_metric_types = FindOrNull(metric_types_by_entity_type_, "tablet");
+    CHECK(tablet_metric_types);
+    const auto* known_type = FindOrNull(*tablet_metric_types, name);
+    if (!known_type) {
+      LOG(ERROR) << Substitute("metric $0 has unknown type, ignore it", name);
+      continue;
+    }
+
+    if (*known_type == "GAUGE" || *known_type ==  "COUNTER") {
+      int64_t value = 0;
+      const Value* val;
+      RETURN_NOT_OK(r.ExtractField(metric, "value", &val));
+      rapidjson::Type type = val->GetType();
+      switch (type) {
+        case rapidjson::Type::kStringType:
+          CHECK_OK(GetStringMetricValue(val, name, &value));
+          break;
+        case rapidjson::Type::kNumberType:
+          CHECK_OK(GetNumberMetricValue(val, name, &value));
+          break;
+        default:
+          LOG(FATAL) << "Unknown type, metrics name: " << name;
+      }
+
+      EmplaceOrDie(&table_metrics, std::make_pair(name, value));
+      if (!EmplaceIfNotPresent(host_metrics, std::make_pair(name, value))) {
+        auto& host_metric = FindOrDie(*host_metrics, name);
+        host_metric += value;
+      }
+    } else if (*known_type == "HISTOGRAM") {
+      for (const auto& percentile : kRegisterPercentiles) {
+        string hist_metric_name(name);
+        hist_metric_name += "_" + percentile;
+        int64_t total_count;
+        CHECK_OK(r.ExtractInt64(metric, "total_count", &total_count));
+        int64_t percentile_value;
+        CHECK_OK(r.ExtractInt64(metric, percentile.c_str(), &percentile_value));
+        vector<SimpleHistogram> tmp({{total_count, percentile_value}});
+        EmplaceOrDie(&table_hist_metrics, std::make_pair(hist_metric_name, tmp));
+        if (!EmplaceIfNotPresent(host_hist_metrics, std::make_pair(hist_metric_name, tmp))) {
+          auto& host_hist_metric = FindOrDie(*host_hist_metrics, hist_metric_name);
+          host_hist_metric.emplace_back(tmp[0]);
+        }
+      }
+    } else {
+      LOG(FATAL) << "Unknown metric type: " << *known_type;
+    }
+  }
+
+  return Status::OK();
+}
+
+Status MetricsCollector::ParseTabletMetrics(const JsonReader& /*r*/,
+                                            const rapidjson::Value* /*entity*/) {
+  return Status::NotSupported("tablet entity is not supported");
+}
+
+Status MetricsCollector::CollectAndReportHostLevelMetrics(
+    const string& url,
+    TablesMetrics* metrics_by_table_name,
+    TablesHistMetrics* hist_metrics_by_table_name) {
+  MonoTime start(MonoTime::Now());
+  scoped_refptr<Trace> trace(new Trace);
+  ADOPT_TRACE(trace.get());
+  TRACE_EVENT1("collector", "MetricsCollector::CollectAndReportHostLevelMetrics",
+               "url", url);
+  TRACE("init");
+  CHECK(metrics_by_table_name);
+  CHECK(hist_metrics_by_table_name);
+
+  // Get metrics from server.
+  string resp;
+  RETURN_NOT_OK(GetMetrics(url, &resp));
+
+  // Merge metrics by table and metric type.
+  Metrics host_metrics;
+  HistMetrics host_hist_metrics;
+  RETURN_NOT_OK(ParseMetrics(resp, metrics_by_table_name, &host_metrics,
+                             hist_metrics_by_table_name, &host_hist_metrics));
+
+  string host_name = ExtractHostName(url);
+  auto timestamp = static_cast<uint64_t>(WallTime_Now());
+
+  // Host table level.
+  if (!FLAGS_collector_ignore_hosttable_level_metrics) {
+    RETURN_NOT_OK(ReportHostTableLevelMetrics(host_name, timestamp,
+                                              *metrics_by_table_name,
+                                              *hist_metrics_by_table_name));
+  }
+
+  // Host level.
+  RETURN_NOT_OK(ReportHostLevelMetrics(host_name, timestamp,
+                                       host_metrics,
+                                       host_hist_metrics));
+
+  int64_t elapsed_ms = (MonoTime::Now() - start).ToMilliseconds();
+  if (elapsed_ms > FLAGS_collector_warn_threshold_ms) {
+    if (Trace::CurrentTrace()) {
+      LOG(WARNING) << "Trace:" << std::endl
+                   << Trace::CurrentTrace()->DumpToString();
+    }
+  }
+  return Status::OK();
+}
+
+Status MetricsCollector::ParseMetrics(const string& data,
+                                      TablesMetrics* metrics_by_table_name,
+                                      Metrics* host_metrics,
+                                      TablesHistMetrics* hist_metrics_by_table_name,
+                                      HistMetrics* host_hist_metrics) {
+  JsonReader r(data);
+  RETURN_NOT_OK(r.Init());
+  vector<const Value*> entities;
+  RETURN_NOT_OK(r.ExtractObjectArray(r.root(), nullptr, &entities));
+
+  for (const Value* entity : entities) {
+    if (FilterByAttribute(r, entity)) {
+      continue;
+    }
+    string entity_type;
+    CHECK_OK(r.ExtractString(entity, "type", &entity_type));
+    if (entity_type == "server") {
+      CHECK(ParseServerMetrics(r, entity).IsNotSupported());
+    } else if (entity_type == "table") {
+      CHECK_OK(ParseTableMetrics(r, entity,
+                                 metrics_by_table_name, host_metrics,
+                                 hist_metrics_by_table_name, host_hist_metrics));
+    } else if (entity_type == "tablet") {
+      CHECK(ParseTabletMetrics(r, entity).IsNotSupported());
+    } else {
+      LOG(FATAL) << "Unknown entity_type: " << entity_type;
+    }
+  }
+  TRACE(Substitute("Metrics parsed, entity count $0", entities.size()));
+
+  return Status::OK();
+}
+
+void MetricsCollector::CollectMetrics(const string& endpoint,
+                                      const Metrics& metrics,
+                                      const std::string& level,
+                                      uint64_t timestamp,
+                                      const std::string& extra_tags,
+                                      list<scoped_refptr<ItemBase>>* items) {
+  for (const auto& metric : metrics) {
+    items->emplace_back(
+      reporter_->ConstructItem(endpoint,
+                               metric.first,
+                               level,
+                               timestamp,
+                               metric.second,
+                               FindOrDie(metric_types_by_entity_type_["tablet"], metric.first),
+                               extra_tags));
+  }
+}
+
+void MetricsCollector::CollectMetrics(const string& endpoint,
+                      const HistMetrics& metrics,
+                      const string& level,
+                      uint64_t timestamp,
+                      const string& extra_tags,
+                      list<scoped_refptr<ItemBase>>* items) {
+  for (const auto& metric : metrics) {
+    items->emplace_back(
+      reporter_->ConstructItem(endpoint,
+                               metric.first,
+                               level,
+                               timestamp,
+                               GetHistValue(metric.second),
+                               "GAUGE",
+                               extra_tags));
+  }
+}
+
+Status MetricsCollector::ReportHostTableLevelMetrics(
+    const string& host_name,
+    uint64_t timestamp,
+    const TablesMetrics& metrics_by_table_name,
+    const TablesHistMetrics& hist_metrics_by_table_name) {
+  list<scoped_refptr<ItemBase>> items;
+  // GAUGE/COUNTER type metrics.
+  int metrics_count = 0;
+  for (const auto& table_metrics : metrics_by_table_name) {
+    const auto extra_tag = Substitute("table=$0", table_metrics.first);
+    metrics_count += table_metrics.second.size();
+    CollectMetrics(host_name, table_metrics.second, "host_table", timestamp, extra_tag, &items);
+  }
+  TRACE(Substitute("Host-table GAUGE/COUNTER type metrics collected, count $0", metrics_count));
+
+  // HISTOGRAM type metrics.
+  int hist_metrics_count = 0;
+  for (const auto& table_hist_metrics : hist_metrics_by_table_name) {
+    const auto extra_tag = Substitute("table=$0", table_hist_metrics.first);
+    hist_metrics_count += table_hist_metrics.second.size();
+    CollectMetrics(host_name, table_hist_metrics.second,
+                   "host_table", timestamp, extra_tag,
+                   &items);
+  }
+  TRACE(Substitute("Host-table HISTOGRAM type metrics collected, count $0", hist_metrics_count));
+
+  reporter_->PushItems(std::move(items));
+  TRACE(Substitute("Host-table metrics reported, count $0", metrics_count + hist_metrics_count));
+
+  return Status::OK();
+}
+
+Status MetricsCollector::ReportHostLevelMetrics(
+    const string& host_name,
+    uint64_t timestamp,
+    const Metrics& host_metrics,
+    const HistMetrics& host_hist_metrics) {
+  list<scoped_refptr<ItemBase>> items;
+  // GAUGE/COUNTER type metrics.
+  CollectMetrics(host_name, host_metrics, "host", timestamp, "", &items);
+  TRACE(Substitute("Host GAUGE/COUNTER type metrics collected, count $0", host_metrics.size()));
+
+  // HISTOGRAM type metrics.
+  CollectMetrics(host_name, host_hist_metrics, "host", timestamp, "", &items);
+  TRACE(Substitute("Host HISTOGRAM type metrics collected, count $0", host_hist_metrics.size()));
+
+  reporter_->PushItems(std::move(items));
+  TRACE(Substitute("Host metrics reported, count $0",
+                   host_metrics.size() + host_hist_metrics.size()));
+
+  return Status::OK();
+}
+
+Status MetricsCollector::ReportTableLevelMetrics(
+    uint64_t timestamp,
+    const TablesMetrics& metrics_by_table_name,
+    const TablesHistMetrics& hist_metrics_by_table_name) {
+  list<scoped_refptr<ItemBase>> items;
+  // GAUGE/COUNTER type metrics.
+  int metrics_count = 0;
+  for (const auto& table_metrics : metrics_by_table_name) {
+    metrics_count += table_metrics.second.size();
+    CollectMetrics(table_metrics.first,
+                   table_metrics.second,
+                   "table", timestamp, "", &items);
+  }
+  TRACE(Substitute("Table GAUGE/COUNTER type metrics collected, count $0", metrics_count));
+
+  // HISTOGRAM type metrics.
+  int hist_metrics_count = 0;
+  for (const auto& table_hist_metrics : hist_metrics_by_table_name) {
+    hist_metrics_count += table_hist_metrics.second.size();
+    CollectMetrics(table_hist_metrics.first,
+                   table_hist_metrics.second,
+                   "table", timestamp, "", &items);
+  }
+  TRACE(Substitute("Table HISTOGRAM type metrics collected, count $0", hist_metrics_count));
+
+  reporter_->PushItems(std::move(items));
+  TRACE(Substitute("Table metrics reported, count $0", metrics_count + hist_metrics_count));
+
+  return Status::OK();
+}
+
+Status MetricsCollector::ReportClusterLevelMetrics(uint64_t timestamp,
+                                                   const Metrics& cluster_metrics) {
+  list<scoped_refptr<ItemBase>> items;
+  CollectMetrics(FLAGS_collector_cluster_name, cluster_metrics, "cluster", timestamp, "", &items);
+  TRACE(Substitute("Cluster metrics collected, count $0", cluster_metrics.size()));
+
+  reporter_->PushItems(std::move(items));
+  TRACE(Substitute("Cluster metrics reported, count $0", cluster_metrics.size()));
+
+  return Status::OK();
+}
+
+int64_t MetricsCollector::GetHistValue(const vector<SimpleHistogram>& hist_values) {
+  int64_t total_count = 0;
+  double total_value = 0.0;
+  for (const auto& hist_value : hist_values) {
+    total_count += hist_value.count;
+    total_value += hist_value.count * hist_value.value;
+  }
+  int64_t value = 0;
+  if (total_count != 0) {
+    value = std::llround(total_value / total_count);
+  }
+  return value;
+}
+
+Status MetricsCollector::GetMetrics(const string& url, string* resp) {
+  CHECK(resp);
+  EasyCurl curl;
+  faststring dst;
+  //curl.set_return_headers(true);
+  RETURN_NOT_OK(curl.FetchURL(url, &dst, {"Accept-Encoding: gzip"}));
+  std::ostringstream oss;
+  string dst_str = dst.ToString();
+  if (zlib::Uncompress(Slice(dst_str), &oss).ok()) {
+    *resp = oss.str();
+  } else {
+    *resp = dst_str;
+  }
+  TRACE(Substitute("Metrics got from server: $0", url));
+
+  return Status::OK();
+}
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/metrics_collector.h b/src/kudu/collector/metrics_collector.h
new file mode 100644
index 0000000..1435e52
--- /dev/null
+++ b/src/kudu/collector/metrics_collector.h
@@ -0,0 +1,205 @@
+// 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 <cstdint>
+#include <list>
+#include <map>
+#include <memory>
+#include <set>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include <gtest/gtest_prod.h>
+#include <rapidjson/document.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+class JsonReader;
+class Thread;
+class ThreadPool;
+
+namespace collector {
+struct ItemBase;
+}  // namespace collector
+}  // namespace kudu
+
+namespace kudu {
+
+namespace collector {
+
+class NodesChecker;
+class ReporterBase;
+
+class MetricsCollector : public RefCounted<MetricsCollector> {
+ public:
+  MetricsCollector(scoped_refptr<NodesChecker> nodes_checker,
+                   scoped_refptr<ReporterBase> reporter);
+  ~MetricsCollector();
+
+  Status Init();
+  Status Start();
+  void Shutdown();
+
+  std::string ToString() const;
+
+ private:
+  friend class RefCounted<MetricsCollector>;
+
+  FRIEND_TEST(TestMetricsCollector, TestConvertStateToInt);
+  FRIEND_TEST(TestMetricsCollector, TestGetHistValue);
+  FRIEND_TEST(TestMetricsCollector, TestMergeToTableLevelMetrics);
+  FRIEND_TEST(TestMetricsCollector, TestMergeToClusterLevelMetrics);
+  FRIEND_TEST(TestMetricsCollector, TestParseMetrics);
+  FRIEND_TEST(TestMetricsCollector, TestInitMetrics);
+  FRIEND_TEST(TestMetricsCollector, TestInitFilters);
+  FRIEND_TEST(TestMetricsCollector, TestInitMetricsUrlParameters);
+  FRIEND_TEST(TestMetricsCollector, TestInitClusterLevelMetrics);
+
+  typedef std::unordered_map<std::string, int64_t> Metrics;
+  typedef std::unordered_map<std::string, Metrics> TablesMetrics;
+  struct SimpleHistogram {
+    int64_t count;
+    int64_t value;
+    SimpleHistogram(int64_t c, int64_t v) : count(c), value(v) {
+    }
+    inline bool operator==(const SimpleHistogram& rhs) const {
+      return count == rhs.count && value == rhs.value;
+    }
+  };
+
+  typedef std::unordered_map<std::string, std::vector<SimpleHistogram>> HistMetrics;
+  typedef std::unordered_map<std::string, HistMetrics> TablesHistMetrics;
+
+  typedef std::unordered_map<std::string, std::string> MetricTypes;
+
+  Status ValidateTableFilter(const std::string& attribute_filter, const std::string& table_filter);
+  Status InitMetrics();
+  static Status ExtractMetricTypes(const JsonReader& r,
+                                   const rapidjson::Value* entity,
+                                   MetricTypes* metric_types);
+  Status InitFilters();
+  Status InitMetricsUrlParameters();
+  Status InitClusterLevelMetrics();
+
+  Status StartMetricCollectorThread();
+  void MetricCollectorThread();
+  Status CollectAndReportMetrics();
+
+  Status UpdateThreadPool(int32_t thread_count);
+
+  Status CollectAndReportHostLevelMetrics(const std::string& url,
+                                          TablesMetrics* metrics_by_table_name,
+                                          TablesHistMetrics* hist_metrics_by_table_name);
+
+  static Status MergeToTableLevelMetrics(
+      const std::vector<TablesMetrics>& hosts_metrics_by_table_name,
+      const std::vector<TablesHistMetrics>& hosts_hist_metrics_by_table_name,
+      TablesMetrics* metrics_by_table_name,
+      TablesHistMetrics* hist_metrics_by_table_name);
+  static Status MergeToClusterLevelMetrics(const TablesMetrics& metrics_by_table_name,
+                                           const TablesHistMetrics& hist_metrics_by_table_name,
+                                           Metrics* cluster_metrics);
+
+  // Report metrics to third-party monitor system.
+  void CollectMetrics(const std::string& endpoint,
+                      const Metrics& metrics,
+                      const std::string& level,
+                      uint64_t timestamp,
+                      const std::string& extra_tags,
+                      std::list<scoped_refptr<ItemBase>>* items);
+  void CollectMetrics(const std::string& endpoint,
+                      const HistMetrics& metrics,
+                      const std::string& level,
+                      uint64_t timestamp,
+                      const std::string& extra_tags,
+                      std::list<scoped_refptr<ItemBase>>* items);
+
+  Status ReportHostTableLevelMetrics(const std::string& host_name,
+                                     uint64_t timestamp,
+                                     const TablesMetrics& metrics_by_table_name,
+                                     const TablesHistMetrics& hist_metrics_by_table_name);
+  Status ReportHostLevelMetrics(const std::string& host_name,
+                                uint64_t timestamp,
+                                const Metrics& host_metrics,
+                                const HistMetrics& host_hist_metrics);
+  Status ReportTableLevelMetrics(uint64_t timestamp,
+                                 const TablesMetrics& metrics_by_table_name,
+                                 const TablesHistMetrics& hist_metrics_by_table_name);
+  Status ReportClusterLevelMetrics(uint64_t timestamp,
+                                   const Metrics& cluster_metrics);
+  static int64_t GetHistValue(const std::vector<SimpleHistogram>& hist_values);
+
+  // Get metrics from server by http method.
+  static Status GetMetrics(const std::string& url, std::string* resp);
+
+  // Parse metrics from http response, entities may be in different types.
+  Status ParseMetrics(const std::string& data,
+                      TablesMetrics* metrics_by_table_name,
+                      Metrics* host_metrics,
+                      TablesHistMetrics* hist_metrics_by_table_name,
+                      HistMetrics* host_hist_metrics);
+  static Status ParseServerMetrics(const JsonReader& r,
+                                   const rapidjson::Value* entity);
+  Status ParseTableMetrics(const JsonReader& r,
+                           const rapidjson::Value* entity,
+                           TablesMetrics* metrics_by_table_name,
+                           Metrics* host_metrics,
+                           TablesHistMetrics* hist_metrics_by_table_name,
+                           HistMetrics* host_hist_metrics) const;
+  static Status ParseTabletMetrics(const JsonReader& r,
+                                   const rapidjson::Value* entity);
+
+  // Return true when this entity could be filtered.
+  // When server side support attributes filter, this function has no effect.
+  bool FilterByAttribute(const JsonReader& r,
+                         const rapidjson::Value* entity) const;
+  Status GetNumberMetricValue(const rapidjson::Value* metric,
+                              const std::string& metric_name,
+                              int64_t* result) const;
+  Status GetStringMetricValue(const rapidjson::Value* metric,
+                              const std::string& metric_name,
+                              int64_t* result) const;
+  static Status ConvertStateToInt(const std::string& value, int64_t* result);
+
+  static const std::set<std::string> kRegisterPercentiles;
+
+  bool initialized_;
+
+  scoped_refptr<NodesChecker> nodes_checker_;
+  scoped_refptr<ReporterBase> reporter_;
+
+  std::map<std::string, MetricTypes> metric_types_by_entity_type_;
+  // Attribute filter, attributes not in this map will be filtered if it's not empty.
+  // attribute name ---> attribute values
+  std::unordered_map<std::string, std::set<std::string>> attributes_filter_;
+  std::string metric_url_parameters_;
+  Metrics cluster_metrics_;
+
+  CountDownLatch stop_background_threads_latch_;
+  scoped_refptr<Thread> metric_collector_thread_;
+  std::unique_ptr<ThreadPool> host_metric_collector_thread_pool_;
+
+  DISALLOW_COPY_AND_ASSIGN(MetricsCollector);
+};
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/nodes_checker-test.cc b/src/kudu/collector/nodes_checker-test.cc
new file mode 100644
index 0000000..2390a52
--- /dev/null
+++ b/src/kudu/collector/nodes_checker-test.cc
@@ -0,0 +1,55 @@
+// 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/collector/nodes_checker.h"
+
+#include <gtest/gtest.h>
+
+#include "kudu/rebalance/cluster_status.h"
+
+using kudu::cluster_summary::ServerHealth;
+using kudu::cluster_summary::HealthCheckResult;
+
+namespace kudu {
+namespace collector {
+
+TEST(TestNodesChecker, TestExtractServerHealthStatus) {
+  ASSERT_EQ(ServerHealth::HEALTHY,
+            NodesChecker::ExtractServerHealthStatus("HEALTHY"));
+  ASSERT_EQ(ServerHealth::UNAUTHORIZED,
+            NodesChecker::ExtractServerHealthStatus("UNAUTHORIZED"));
+  ASSERT_EQ(ServerHealth::UNAVAILABLE,
+            NodesChecker::ExtractServerHealthStatus("UNAVAILABLE"));
+  ASSERT_EQ(ServerHealth::WRONG_SERVER_UUID,
+            NodesChecker::ExtractServerHealthStatus("WRONG_SERVER_UUID"));
+}
+
+TEST(TestNodesChecker, TestExtractTableHealthStatus) {
+  ASSERT_EQ(HealthCheckResult::HEALTHY,
+            NodesChecker::ExtractTableHealthStatus("HEALTHY"));
+  ASSERT_EQ(HealthCheckResult::RECOVERING,
+            NodesChecker::ExtractTableHealthStatus("RECOVERING"));
+  ASSERT_EQ(HealthCheckResult::UNDER_REPLICATED,
+            NodesChecker::ExtractTableHealthStatus("UNDER_REPLICATED"));
+  ASSERT_EQ(HealthCheckResult::UNAVAILABLE,
+            NodesChecker::ExtractTableHealthStatus("UNAVAILABLE"));
+  ASSERT_EQ(HealthCheckResult::CONSENSUS_MISMATCH,
+            NodesChecker::ExtractTableHealthStatus("CONSENSUS_MISMATCH"));
+}
+}  // namespace collector
+}  // namespace kudu
+
diff --git a/src/kudu/collector/nodes_checker.cc b/src/kudu/collector/nodes_checker.cc
new file mode 100644
index 0000000..8b64c29
--- /dev/null
+++ b/src/kudu/collector/nodes_checker.cc
@@ -0,0 +1,358 @@
+// 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/collector/nodes_checker.h"
+
+#include <cstdint>
+#include <list>
+#include <mutex>
+#include <ostream>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <rapidjson/document.h>
+
+#include "kudu/collector/collector_util.h"
+#include "kudu/collector/reporter_base.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/tools/tool_test_util.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/jsonreader.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/trace.h"
+
+DECLARE_string(collector_cluster_name);
+DECLARE_string(collector_master_addrs);
+DECLARE_int32(collector_interval_sec);
+DECLARE_int32(collector_timeout_sec);
+DECLARE_int32(collector_warn_threshold_ms);
+
+using rapidjson::Value;
+using std::list;
+using std::string;
+using std::vector;
+using strings::Substitute;
+using kudu::cluster_summary::ServerHealth;
+using kudu::cluster_summary::HealthCheckResult;
+
+namespace kudu {
+namespace collector {
+
+const std::string NodesChecker::kMaster = "master";
+const std::string NodesChecker::kTserver = "tserver";
+
+NodesChecker::NodesChecker(scoped_refptr<ReporterBase> reporter)
+  : initialized_(false),
+    reporter_(std::move(reporter)),
+    stop_background_threads_latch_(1) {
+}
+
+NodesChecker::~NodesChecker() {
+  Shutdown();
+}
+
+Status NodesChecker::Init() {
+  CHECK(!initialized_);
+
+  RETURN_NOT_OK(UpdateNodes());
+  CHECK(!master_http_addrs_.empty());
+
+  initialized_ = true;
+  return Status::OK();
+}
+
+Status NodesChecker::Start() {
+  CHECK(initialized_);
+
+  RETURN_NOT_OK(StartNodesCheckerThread());
+
+  return Status::OK();
+}
+
+void NodesChecker::Shutdown() {
+  if (initialized_) {
+    string name = ToString();
+    LOG(INFO) << name << " shutting down...";
+
+    stop_background_threads_latch_.CountDown();
+
+    if (nodes_checker_thread_) {
+      nodes_checker_thread_->Join();
+    }
+
+    LOG(INFO) << name << " shutdown complete.";
+  }
+}
+
+string NodesChecker::ToString() const {
+  return "NodesChecker";
+}
+
+vector<string> NodesChecker::GetNodes() {
+  shared_lock<RWMutex> l(nodes_lock_);
+  return tserver_http_addrs_;
+}
+
+string NodesChecker::GetFirstMaster() {
+  shared_lock<RWMutex> l(nodes_lock_);
+  CHECK(!master_http_addrs_.empty());
+  return master_http_addrs_[0];
+}
+
+Status NodesChecker::StartNodesCheckerThread() {
+  return Thread::Create("collector", "nodes-checker", &NodesChecker::NodesCheckerThread,
+                        this, &nodes_checker_thread_);
+}
+
+void NodesChecker::NodesCheckerThread() {
+  MonoTime check_time;
+  do {
+    check_time = MonoTime::Now();
+    UpdateAndCheckNodes();
+    check_time += MonoDelta::FromSeconds(FLAGS_collector_interval_sec);
+  } while (!RunOnceMode() && !stop_background_threads_latch_.WaitUntil(check_time));
+  LOG(INFO) << "FalconPusherThread exit";
+}
+
+void NodesChecker::UpdateAndCheckNodes() {
+  LOG(INFO) << "Start to UpdateAndCheckNodes";
+  MonoTime start(MonoTime::Now());
+  scoped_refptr<Trace> trace(new Trace);
+  ADOPT_TRACE(trace.get());
+  TRACE_EVENT0("collector", "NodesChecker::UpdateAndCheckNodes");
+  WARN_NOT_OK(UpdateNodes(), "Unable to update nodes");
+  WARN_NOT_OK(CheckNodes(), "Unable to check nodes");
+  int64_t elapsed_ms = (MonoTime::Now() - start).ToMilliseconds();
+  if (elapsed_ms > FLAGS_collector_warn_threshold_ms) {
+    if (Trace::CurrentTrace()) {
+      LOG(WARNING) << "Trace:" << std::endl
+                   << Trace::CurrentTrace()->DumpToString();
+    }
+  }
+}
+
+Status NodesChecker::UpdateNodes() {
+  RETURN_NOT_OK(UpdateServers(kMaster));
+  RETURN_NOT_OK(UpdateServers(kTserver));
+  return Status::OK();
+}
+
+Status NodesChecker::UpdateServers(const std::string& role) {
+  DCHECK(role == kTserver || role == kMaster);
+  vector<string> args = {
+    role,
+    "list",
+    FLAGS_collector_master_addrs,
+    "-columns=http-addresses",
+    "-format=json",
+    Substitute("-timeout_ms=$0", FLAGS_collector_timeout_sec*1000)
+  };
+  string tool_stdout;
+  string tool_stderr;
+  RETURN_NOT_OK_PREPEND(tools::RunKuduTool(args, &tool_stdout, &tool_stderr),
+                        Substitute("out: $0, err: $1", tool_stdout, tool_stderr));
+  TRACE(Substitute("'$0 list' done", role));
+
+  JsonReader r(tool_stdout);
+  RETURN_NOT_OK(r.Init());
+  vector<const Value*> servers;
+  CHECK_OK(r.ExtractObjectArray(r.root(), nullptr, &servers));
+  vector<string> server_http_addrs;
+  for (const Value* server : servers) {
+    string http_address;
+    CHECK_OK(r.ExtractString(server, "http-addresses", &http_address));
+    server_http_addrs.emplace_back(http_address);
+  }
+  TRACE(Substitute("Result parsed, nodes count $0", server_http_addrs.size()));
+
+  if (role == kTserver) {
+    std::lock_guard<RWMutex> l(nodes_lock_);
+    tserver_http_addrs_.swap(server_http_addrs);
+  } else {
+    std::lock_guard<RWMutex> l(nodes_lock_);
+    master_http_addrs_.swap(server_http_addrs);
+  }
+  TRACE("Nodes updated");
+
+  return Status::OK();
+}
+
+Status NodesChecker::CheckNodes() const {
+  vector<string> args = {
+    "cluster",
+    "ksck",
+    FLAGS_collector_master_addrs,
+    "-consensus=false",
+    "-ksck_format=json_compact",
+    "-color=never",
+    "-sections=MASTER_SUMMARIES,TSERVER_SUMMARIES,TABLE_SUMMARIES,TOTAL_COUNT",
+    Substitute("-timeout_ms=$0", FLAGS_collector_timeout_sec*1000)
+  };
+  string tool_stdout;
+  string tool_stderr;
+  WARN_NOT_OK(tools::RunKuduTool(args, &tool_stdout, &tool_stderr),
+              Substitute("out: $0, err: $1", tool_stdout, tool_stderr));
+
+  TRACE("'cluster ksck' done");
+
+  RETURN_NOT_OK(ReportNodesMetrics(tool_stdout));
+  return Status::OK();
+}
+
+Status NodesChecker::ReportNodesMetrics(const string& data) const {
+  JsonReader r(data);
+  RETURN_NOT_OK(r.Init());
+  const Value* ksck;
+  CHECK_OK(r.ExtractObject(r.root(), nullptr, &ksck));
+  auto timestamp = static_cast<uint64_t>(WallTime_Now());
+
+  list<scoped_refptr<ItemBase>> items;
+  // Maters health info.
+  vector<const Value*> masters;
+  CHECK_OK(r.ExtractObjectArray(ksck, "master_summaries", &masters));
+  for (const Value* master : masters) {
+    string address;
+    CHECK_OK(r.ExtractString(master, "address", &address));
+    string health;
+    CHECK_OK(r.ExtractString(master, "health", &health));
+    items.emplace_back(reporter_->ConstructItem(
+      ExtractHostName(address),
+      "kudu-master-health",
+      "host",
+      timestamp,
+      static_cast<int64_t>(ExtractServerHealthStatus(health)),
+      "GAUGE",
+      ""));
+  }
+  TRACE(Substitute("Maters health info reported, count $0", masters.size()));
+
+  // Tservers health info.
+  vector<const Value*> tservers;
+  Status s = r.ExtractObjectArray(ksck, "tserver_summaries", &tservers);
+  CHECK(s.ok() || s.IsNotFound());
+  if (s.ok()) {
+    for (const Value* tserver : tservers) {
+      string address;
+      CHECK_OK(r.ExtractString(tserver, "address", &address));
+      string health;
+      CHECK_OK(r.ExtractString(tserver, "health", &health));
+      items.emplace_back(reporter_->ConstructItem(
+        ExtractHostName(address),
+        "kudu-tserver-health",
+        "host",
+        timestamp,
+        static_cast<int64_t>(ExtractServerHealthStatus(health)),
+        "GAUGE",
+        ""));
+    }
+    TRACE(Substitute("Tservers health info reported, count $0", tservers.size()));
+  }
+
+  // Tables health info.
+  uint32_t health_table_count = 0;
+  vector<const Value*> tables;
+  s = r.ExtractObjectArray(ksck, "table_summaries", &tables);
+  CHECK(s.ok() || s.IsNotFound());
+  if (s.ok()) {
+    for (const Value* table : tables) {
+      string name;
+      CHECK_OK(r.ExtractString(table, "name", &name));
+      string health;
+      CHECK_OK(r.ExtractString(table, "health", &health));
+      HealthCheckResult health_status = ExtractTableHealthStatus(health);
+      items.emplace_back(reporter_->ConstructItem(
+        name,
+        "kudu-table-health",
+        "table",
+        timestamp,
+        static_cast<int64_t>(health_status),
+        "GAUGE",
+        ""));
+      if (health_status == HealthCheckResult::HEALTHY) {
+        health_table_count += 1;
+      }
+    }
+    TRACE(Substitute("Tables health info reported, count $0", tables.size()));
+  }
+
+  // Healthy table ratio.
+  if (!tables.empty()) {
+    items.emplace_back(reporter_->ConstructItem(
+      FLAGS_collector_cluster_name,
+      "healthy_table_proportion",
+      "cluster",
+      timestamp,
+      100 * health_table_count / tables.size(),
+      "GAUGE",
+      ""));
+    TRACE("Healthy table ratio reported");
+  }
+
+  // Count summaries.
+  vector<const Value*> count_summaries;
+  CHECK_OK(r.ExtractObjectArray(ksck, "count_summaries", &count_summaries));
+  for (const Value* count_summarie : count_summaries) {
+    // TODO(yingchun) should auto iterate items
+    static const vector<string>
+        count_names({"masters", "tservers", "tables", "tablets", "replicas"});
+    for (const auto& name : count_names) {
+      int64_t count;
+      CHECK_OK(r.ExtractInt64(count_summarie, name.c_str(), &count));
+      items.emplace_back(reporter_->ConstructItem(
+        FLAGS_collector_cluster_name,
+        name + "_count",
+        "cluster",
+        timestamp,
+        count,
+        "GAUGE",
+        ""));
+    }
+  }
+  TRACE("Count summaries reported");
+
+  reporter_->PushItems(std::move(items));
+  TRACE("Pushed");
+
+  return Status::OK();
+}
+
+ServerHealth NodesChecker::ExtractServerHealthStatus(const string& health) {
+  if (health == "HEALTHY") return ServerHealth::HEALTHY;
+  if (health == "UNAUTHORIZED") return ServerHealth::UNAUTHORIZED;
+  if (health == "UNAVAILABLE") return ServerHealth::UNAVAILABLE;
+  if (health == "WRONG_SERVER_UUID") return ServerHealth::WRONG_SERVER_UUID;
+  CHECK(false) << "Unknown server health: " << health;
+  __builtin_unreachable();
+}
+
+HealthCheckResult NodesChecker::ExtractTableHealthStatus(const string& health) {
+  if (health == "HEALTHY") return HealthCheckResult::HEALTHY;
+  if (health == "RECOVERING") return HealthCheckResult::RECOVERING;
+  if (health == "UNDER_REPLICATED") return HealthCheckResult::UNDER_REPLICATED;
+  if (health == "UNAVAILABLE") return HealthCheckResult::UNAVAILABLE;
+  if (health == "CONSENSUS_MISMATCH") return HealthCheckResult::CONSENSUS_MISMATCH;
+  CHECK(false)  << "Unknown table health: " << health;
+  __builtin_unreachable();
+}
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/nodes_checker.h b/src/kudu/collector/nodes_checker.h
new file mode 100644
index 0000000..26aee89
--- /dev/null
+++ b/src/kudu/collector/nodes_checker.h
@@ -0,0 +1,90 @@
+// 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 <vector>
+
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rebalance/cluster_status.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/rw_mutex.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+class Thread;
+}  // namespace kudu
+
+namespace kudu {
+
+namespace collector {
+
+class ReporterBase;
+
+class NodesChecker : public RefCounted<NodesChecker> {
+ public:
+  explicit NodesChecker(scoped_refptr<ReporterBase> reporter);
+  ~NodesChecker();
+
+  Status Init();
+  Status Start();
+  void Shutdown();
+
+  std::string ToString() const;
+
+  std::vector<std::string> GetNodes();
+  std::string GetFirstMaster();
+
+ private:
+  friend class RefCounted<NodesChecker>;
+
+  FRIEND_TEST(TestNodesChecker, TestExtractServerHealthStatus);
+  FRIEND_TEST(TestNodesChecker, TestExtractTableHealthStatus);
+
+  Status StartNodesCheckerThread();
+  void NodesCheckerThread();
+
+  void UpdateAndCheckNodes();
+  Status UpdateNodes();
+  Status UpdateServers(const std::string& role);
+  Status CheckNodes() const;
+  Status ReportNodesMetrics(const std::string& data) const;
+
+  static cluster_summary::ServerHealth ExtractServerHealthStatus(const std::string& health);
+  static cluster_summary::HealthCheckResult ExtractTableHealthStatus(const std::string& health);
+
+  static const std::string kMaster;
+  static const std::string kTserver;
+
+  bool initialized_;
+
+  scoped_refptr<ReporterBase> reporter_;
+
+  CountDownLatch stop_background_threads_latch_;
+  scoped_refptr<Thread> nodes_checker_thread_;
+
+  mutable RWMutex nodes_lock_;
+  std::vector<std::string> tserver_http_addrs_;
+  std::vector<std::string> master_http_addrs_;
+
+  DISALLOW_COPY_AND_ASSIGN(NodesChecker);
+};
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/reporter_base.h b/src/kudu/collector/reporter_base.h
new file mode 100644
index 0000000..d03b80c
--- /dev/null
+++ b/src/kudu/collector/reporter_base.h
@@ -0,0 +1,73 @@
+// 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 <cstdint>
+#include <list>
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include <rapidjson/document.h>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/server/server_base.h"
+#include "kudu/tools/ksck_results.h"
+#include "kudu/util/jsonreader.h"
+#include "kudu/util/rw_mutex.h"
+#include "kudu/util/status.h"
+#include "kudu/util/threadpool.h"
+
+namespace kudu {
+
+namespace collector {
+
+struct ItemBase : public RefCounted<ItemBase> {
+  virtual ~ItemBase() = default;
+
+ private:
+  friend class RefCounted<ItemBase>;
+};
+
+class ReporterBase : public RefCounted<ReporterBase> {
+ public:
+  virtual ~ReporterBase() = default;
+
+  virtual Status Init() = 0;
+  virtual Status Start() = 0;
+  virtual void Shutdown() = 0;
+
+  virtual std::string ToString() const = 0;
+
+  // TODO(yingchun) This function is not generic enough for base class.
+  virtual scoped_refptr<ItemBase> ConstructItem(std::string endpoint,
+                                                std::string metric,
+                                                std::string level,
+                                                uint64_t timestamp,
+                                                int64_t value,
+                                                std::string counter_type,
+                                                std::string extra_tags) = 0;
+  virtual Status PushItems(std::list<scoped_refptr<ItemBase>> items) = 0;
+
+ protected:
+  friend class RefCounted<ReporterBase>;
+};
+} // namespace collector
+} // namespace kudu
+
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 6aefb42..21de53d 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -61,6 +61,7 @@
 
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <google/protobuf/stubs/common.h>
 
diff --git a/src/kudu/tools/tool_action_table.cc b/src/kudu/tools/tool_action_table.cc
index 8e98c5d..7834ea4 100644
--- a/src/kudu/tools/tool_action_table.cc
+++ b/src/kudu/tools/tool_action_table.cc
@@ -15,8 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <stdlib.h>
-
 #include <algorithm>
 #include <cstdint>
 #include <functional>
diff --git a/src/kudu/util/jsonreader.h b/src/kudu/util/jsonreader.h
index 125b762..dcc6603 100644
--- a/src/kudu/util/jsonreader.h
+++ b/src/kudu/util/jsonreader.h
@@ -28,6 +28,10 @@
 
 namespace kudu {
 
+namespace collector {
+class MetricsCollector;
+}  // namespace collector
+
 // Wraps the JSON parsing functionality of rapidjson::Document.
 //
 // Unlike JsonWriter, this class does not hide rapidjson internals from
@@ -93,6 +97,8 @@ class JsonReader {
   const rapidjson::Value* root() const { return &document_; }
 
  private:
+  friend class collector::MetricsCollector;
+
   Status ExtractField(const rapidjson::Value* object,
                       const char* field,
                       const rapidjson::Value** result) const;


[kudu] 12/23: [collector] collect server entity metrics

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 66681c688240449ccbe95e2af3519803070f0e84
Author: Yingchun Lai <40...@qq.com>
AuthorDate: Tue Dec 31 18:27:49 2019 +0800

    [collector] collect server entity metrics
    
    1. Cherry-pick a commit from community master branch to fix bugs when
       metrics do merge.
    2. Collector support to collect 'server' type entity metrics on
       tserver and master.
    3. Collector support to collect catalog table's metrics on master.
    4. Associate scripts to create or update falcon screens.
---
 src/kudu/collector/cluster_rebalancer.cc     |   4 +-
 src/kudu/collector/collector-test.cc         |  46 ++-
 src/kudu/collector/collector.cc              |  52 +--
 src/kudu/collector/collector.h               |   2 +-
 src/kudu/collector/collector_util.cc         |   2 +-
 src/kudu/collector/falcon_reporter-test.cc   |   2 +-
 src/kudu/collector/falcon_reporter.cc        |  26 +-
 src/kudu/collector/metrics_collector-test.cc | 452 +++++++++++----------------
 src/kudu/collector/metrics_collector.cc      | 279 +++++++++++------
 src/kudu/collector/metrics_collector.h       |  54 +++-
 src/kudu/collector/nodes_checker.cc          |  19 +-
 src/kudu/collector/nodes_checker.h           |   4 +-
 src/kudu/scripts/build_kudu.sh               |  33 +-
 src/kudu/scripts/falcon_screen.json          | 348 ++++++++++++++++++++-
 src/kudu/scripts/falcon_screen.py            |  43 ++-
 src/kudu/scripts/kudu_falcon_screen.sh       |  49 +--
 src/kudu/util/metrics.h                      |   1 +
 17 files changed, 918 insertions(+), 498 deletions(-)

diff --git a/src/kudu/collector/cluster_rebalancer.cc b/src/kudu/collector/cluster_rebalancer.cc
index 0015544..b46baa4 100644
--- a/src/kudu/collector/cluster_rebalancer.cc
+++ b/src/kudu/collector/cluster_rebalancer.cc
@@ -17,8 +17,8 @@
 
 #include "kudu/collector/cluster_rebalancer.h"
 
-#include <stdio.h>
-#include <time.h>
+#include <cstdio>
+#include <ctime>
 
 #include <ostream>
 #include <vector>
diff --git a/src/kudu/collector/collector-test.cc b/src/kudu/collector/collector-test.cc
index e8e1298..76e2a2f 100644
--- a/src/kudu/collector/collector-test.cc
+++ b/src/kudu/collector/collector-test.cc
@@ -17,27 +17,49 @@
 
 #include "kudu/collector/collector.h"
 
+#include <cstdint>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
 #include <gtest/gtest.h>
 
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 
+DECLARE_string(collector_cluster_name);
+DECLARE_string(collector_master_addrs);
+DECLARE_uint32(collector_interval_sec);
+DECLARE_uint32(collector_timeout_sec);
+
+using std::vector;
+
 namespace kudu {
 namespace collector {
 
 TEST(TestCollector, TestValidateIntervalAndTimeout) {
-  // 'interval' in error range.
-  ASSERT_TRUE(Collector::ValidateIntervalAndTimeout(9, 1).IsInvalidArgument());
-  ASSERT_TRUE(Collector::ValidateIntervalAndTimeout(61, 1).IsInvalidArgument());
-
-  // 'timeout' in error range.
-  ASSERT_TRUE(Collector::ValidateIntervalAndTimeout(10, 0).IsInvalidArgument());
-  ASSERT_TRUE(Collector::ValidateIntervalAndTimeout(10, 10).IsInvalidArgument());
-
-  // Both 'interval' and 'timeout' are in valid range.
-  ASSERT_OK(Collector::ValidateIntervalAndTimeout(10, 9));
-  ASSERT_OK(Collector::ValidateIntervalAndTimeout(60, 9));
-  ASSERT_OK(Collector::ValidateIntervalAndTimeout(60, 59));
+  FLAGS_collector_cluster_name = "test";
+  FLAGS_collector_master_addrs = "127.0.0.1:1234";
+  vector<std::pair<uint32_t, uint32_t>> invalid_arguments({{9, 1},
+                                                           {61, 1},
+                                                           {10, 0},
+                                                           {10, 10}});
+  for (const auto& arguments : invalid_arguments) {
+    FLAGS_collector_interval_sec = arguments.first;
+    FLAGS_collector_timeout_sec = arguments.second;
+    ASSERT_TRUE(Collector::ValidateFlags().IsInvalidArgument())
+        << FLAGS_collector_interval_sec << ", " << FLAGS_collector_timeout_sec;
+  }
+
+  vector<std::pair<uint32_t, uint32_t>> valid_arguments({{10, 9},
+                                                         {60, 9},
+                                                         {60, 59}});
+  for (const auto& arguments : valid_arguments) {
+    FLAGS_collector_interval_sec = arguments.first;
+    FLAGS_collector_timeout_sec = arguments.second;
+    ASSERT_OK(Collector::ValidateFlags());
+  }
 }
 }  // namespace collector
 }  // namespace kudu
diff --git a/src/kudu/collector/collector.cc b/src/kudu/collector/collector.cc
index 1c930e9..59f36df 100644
--- a/src/kudu/collector/collector.cc
+++ b/src/kudu/collector/collector.cc
@@ -42,15 +42,15 @@ DEFINE_string(collector_cluster_name, "",
 DEFINE_string(collector_master_addrs, "",
               "Comma-separated list of Kudu master addresses where each address is of "
               "form 'hostname:port");
-DEFINE_int32(collector_interval_sec, 60,
-             "Number of interval seconds to collect metrics");
-DEFINE_string(collector_report_method, "",
-              "Which monitor system the metrics reported to. Now supported system: falcon");
-DEFINE_int32(collector_timeout_sec, 10,
-             "Number of seconds to wait for a master, tserver, or CLI tool to return metrics");
-DEFINE_int32(collector_warn_threshold_ms, 1000,
-             "If a task takes more than this number of milliseconds, issue a warning with a "
-             "trace.");
+DEFINE_uint32(collector_interval_sec, 60,
+              "Number of interval seconds to collect metrics");
+DEFINE_string(collector_report_method, "falcon",
+              "Which monitor system the metrics reported to. Now supported system: local, falcon");
+DEFINE_uint32(collector_timeout_sec, 10,
+              "Number of seconds to wait for a master, tserver, or CLI tool to return metrics");
+DEFINE_uint32(collector_warn_threshold_ms, 1000,
+              "If a task takes more than this number of milliseconds, issue a warning with a "
+              "trace.");
 
 DECLARE_string(principal);
 DECLARE_string(keytab_file);
@@ -73,8 +73,7 @@ Collector::~Collector() {
 Status Collector::Init() {
   CHECK(!initialized_);
 
-  RETURN_NOT_OK(ValidateIntervalAndTimeout(FLAGS_collector_interval_sec,
-                                           FLAGS_collector_timeout_sec));
+  RETURN_NOT_OK(ValidateFlags());
   RETURN_NOT_OK(security::InitKerberosForServer(FLAGS_principal, FLAGS_keytab_file));
 
   if (FLAGS_collector_report_method == "falcon") {
@@ -82,9 +81,9 @@ Status Collector::Init() {
   } else if (FLAGS_collector_report_method == "local") {
     reporter_.reset(new LocalReporter());
   } else {
-    LOG(FATAL) << Substitute("Unsupported FLAGS_collector_report_method $0",
-                             FLAGS_collector_report_method);
+    __builtin_unreachable();
   }
+
   CHECK_OK(reporter_->Init());
   nodes_checker_.reset(new NodesChecker(reporter_));
   CHECK_OK(nodes_checker_->Init());
@@ -156,15 +155,28 @@ void Collector::ExcessLogFileDeleterThread() {
   }
 }
 
-Status Collector::ValidateIntervalAndTimeout(int interval, int timeout) {
-  if (10 <= interval && interval <= 60 &&
-      0 < timeout && timeout < interval) {
-    return Status::OK();
+Status Collector::ValidateFlags() {
+  if (FLAGS_collector_interval_sec < 10 ||
+      FLAGS_collector_interval_sec > 60 ||
+      FLAGS_collector_timeout_sec < 1 ||
+      FLAGS_collector_timeout_sec >= FLAGS_collector_interval_sec) {
+    return Status::InvalidArgument("--collector_interval_sec should in range [10, 60], and "
+                                   "--collector_timeout_sec should in range "
+                                   "(0, collector_interval_sec)");
+  }
+
+  if (FLAGS_collector_report_method != "local" &&
+      FLAGS_collector_report_method != "falcon") {
+    return Status::InvalidArgument("--collector_report_method only support 'local' and 'falcon'.");
   }
 
-  return Status::InvalidArgument(
-      Substitute("Invalid interval '$0'(should in range [10, 60]), "
-                 "or invalid timeout '$1'(should in range (0, interval))", interval, timeout));
+  if (FLAGS_collector_cluster_name.empty() ||
+      FLAGS_collector_master_addrs.empty()) {
+    return Status::InvalidArgument("--collector_cluster_name and --collector_master_addrs should "
+                                   "not be empty.");
+  }
+
+  return Status::OK();
 }
 } // namespace collector
 } // namespace kudu
diff --git a/src/kudu/collector/collector.h b/src/kudu/collector/collector.h
index 8e4e236..e135a3c 100644
--- a/src/kudu/collector/collector.h
+++ b/src/kudu/collector/collector.h
@@ -56,7 +56,7 @@ class Collector {
   Status StartExcessLogFileDeleterThread();
   void ExcessLogFileDeleterThread();
 
-  static Status ValidateIntervalAndTimeout(int interval, int timeout);
+  static Status ValidateFlags();
 
   bool initialized_;
 
diff --git a/src/kudu/collector/collector_util.cc b/src/kudu/collector/collector_util.cc
index aa79c40..ffbf0e1 100644
--- a/src/kudu/collector/collector_util.cc
+++ b/src/kudu/collector/collector_util.cc
@@ -19,7 +19,7 @@
 
 #include "kudu/collector/collector_util.h"
 
-#include <stddef.h>
+#include <cstddef>
 
 #include <gflags/gflags_declare.h>
 
diff --git a/src/kudu/collector/falcon_reporter-test.cc b/src/kudu/collector/falcon_reporter-test.cc
index 85810c7..3699b09 100644
--- a/src/kudu/collector/falcon_reporter-test.cc
+++ b/src/kudu/collector/falcon_reporter-test.cc
@@ -31,7 +31,7 @@
 
 DECLARE_string(collector_cluster_name);
 DECLARE_int32(collector_falcon_metrics_version);
-DECLARE_int32(collector_interval_sec);
+DECLARE_uint32(collector_interval_sec);
 
 using std::list;
 using std::string;
diff --git a/src/kudu/collector/falcon_reporter.cc b/src/kudu/collector/falcon_reporter.cc
index e492fd7..efe387b 100644
--- a/src/kudu/collector/falcon_reporter.cc
+++ b/src/kudu/collector/falcon_reporter.cc
@@ -17,10 +17,8 @@
 
 #include "kudu/collector/falcon_reporter.h"
 
-#include <kudu/util/curl_util.h>
-#include <stddef.h>
-
 #include <algorithm>
+#include <cstddef>
 #include <functional>
 #include <iterator>
 #include <mutex>
@@ -32,6 +30,7 @@
 #include <glog/logging.h>
 
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/curl_util.h"
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/faststring.h"
 #include "kudu/util/jsonwriter.h"
@@ -49,17 +48,17 @@ DEFINE_string(collector_falcon_agent, "http://127.0.0.1:1988/v1/push",
 DEFINE_int32(collector_falcon_metrics_version, 4,
              "Version of metrics pushed to falcon, it will be tagged in "
              "'tag' section of an item");
-DEFINE_int32(collector_falcon_pusher_count, 4,
-             "Thread count to push collected items to falcon agent");
-DEFINE_int32(collector_report_batch_size, 1000,
-            "Count of items will be pushed to falcon agent by batch");
-DEFINE_int32(collector_push_timeout_ms, 20,
-             "Timeout for pushing items to falcon agent");
+DEFINE_uint32(collector_falcon_pusher_count, 4,
+              "Thread count to push collected items to falcon agent");
+DEFINE_uint32(collector_report_batch_size, 1000,
+             "Count of items will be pushed to falcon agent by batch");
+DEFINE_uint32(collector_push_timeout_ms, 20,
+              "Timeout for pushing items to falcon agent");
 
 DECLARE_string(collector_cluster_name);
-DECLARE_int32(collector_interval_sec);
-DECLARE_int32(collector_timeout_sec);
-DECLARE_int32(collector_warn_threshold_ms);
+DECLARE_uint32(collector_interval_sec);
+DECLARE_uint32(collector_timeout_sec);
+DECLARE_uint32(collector_warn_threshold_ms);
 
 using std::list;
 using std::string;
@@ -214,7 +213,8 @@ Status FalconReporter::PushToAgent(list<scoped_refptr<ItemBase>> falcon_items) {
   EasyCurl curl;
   faststring dst;
   curl.set_timeout(MonoDelta::FromMilliseconds(FLAGS_collector_push_timeout_ms));
-  RETURN_NOT_OK(curl.PostToURL(FLAGS_collector_falcon_agent, data, &dst));
+  RETURN_NOT_OK_PREPEND(curl.PostToURL(FLAGS_collector_falcon_agent, data, &dst),
+      Substitute("Failed to pushed items to agent, size $0", data.size()));
   TRACE(Substitute("Pushed items to agent, size $0", data.size()));
   return Status::OK();
 }
diff --git a/src/kudu/collector/metrics_collector-test.cc b/src/kudu/collector/metrics_collector-test.cc
index 19c00cf..865f821 100644
--- a/src/kudu/collector/metrics_collector-test.cc
+++ b/src/kudu/collector/metrics_collector-test.cc
@@ -17,9 +17,8 @@
 
 #include "kudu/collector/metrics_collector.h"
 
-#include <stdint.h>
+#include <cstdint>
 
-#include <map>
 #include <set>
 #include <string>
 #include <unordered_map>
@@ -27,14 +26,11 @@
 
 #include <gflags/gflags_declare.h>
 #include <gtest/gtest.h>
-#include <rapidjson/document.h>
 
 #include "kudu/collector/local_reporter.h"
 #include "kudu/collector/nodes_checker.h"
 #include "kudu/collector/reporter_base.h"
 #include "kudu/gutil/ref_counted.h"
-#include "kudu/util/jsonreader.h"
-#include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 
 DECLARE_bool(collector_request_merged_metrics);
@@ -43,7 +39,6 @@ DECLARE_string(collector_cluster_level_metrics);
 DECLARE_string(collector_metrics);
 DECLARE_string(collector_metrics_types_for_test);
 
-using std::map;
 using std::set;
 using std::string;
 using std::unordered_map;
@@ -392,190 +387,158 @@ TEST(TestMetricsCollector, TestMergeToClusterLevelMetrics) {
 }
 
 TEST(TestMetricsCollector, TestParseMetrics) {
-  // Check ParseServerMetrics and ParseTabletMetrics.
-  {
-    string data;
-    JsonReader r(data);
-    const rapidjson::Value entity;
-    ASSERT_TRUE(MetricsCollector::ParseServerMetrics(r, &entity).IsNotSupported());
-    ASSERT_TRUE(MetricsCollector::ParseTabletMetrics(r, &entity).IsNotSupported());
-  }
-  // Check ParseTableMetrics.
-  {
-    auto collector = BuildCollector();
-    collector->metric_types_by_entity_type_["tablet"] = {
-        {"test_metric", "COUNTER"},
-        {"metric_counter1", "COUNTER"},
-        {"metric_counter2", "COUNTER"},
-        {"metric_histogram1", "HISTOGRAM"},
-        {"metric_histogram2", "HISTOGRAM"}
-    };
-    string data(
-        R"*([                                             )*"
-        R"*(  {                                           )*"
-        R"*(    "type": "server",                         )*"
-        R"*(    "id": "server1",                          )*"
-        R"*(    "attributes": {                           )*"
-        R"*(      "attrA": "val1",                        )*"
-        R"*(      "attrB": "val2"                         )*"
-        R"*(    },                                        )*"
-        R"*(    "metrics": [                              )*"
-        R"*(      {                                       )*"
-        R"*(        "name": "test_metric",                )*"
-        R"*(        "value": 123                          )*"
-        R"*(      }                                       )*"
-        R"*(    ]                                         )*"
-        R"*(  },                                          )*"
-        R"*(  {                                           )*"
-        R"*(    "type": "tablet",                         )*"
-        R"*(    "id": "tablet1",                          )*"
-        R"*(    "attributes": {                           )*"
-        R"*(      "attr1": "val1",                        )*"
-        R"*(      "attr2": "val2"                         )*"
-        R"*(    },                                        )*"
-        R"*(    "metrics": [                              )*"
-        R"*(      {                                       )*"
-        R"*(        "name": "test_metric",                )*"
-        R"*(        "value": 321                          )*"
-        R"*(      }                                       )*"
-        R"*(    ]                                         )*"
-        R"*(  },                                          )*"
-        R"*(  {                                           )*"
-        R"*(    "type": "table",                          )*"
-        R"*(    "id": "table1",                           )*"
-        R"*(    "attributes": {                           )*"
-        R"*(      "attr1": "val2",                        )*"
-        R"*(      "attr2": "val3"                         )*"
-        R"*(    },                                        )*"
-        R"*(    "metrics": [                              )*"
-        R"*(      {                                       )*"
-        R"*(        "name": "metric_counter1",            )*"
-        R"*(        "value": 10                           )*"
-        R"*(      },                                      )*"
-        R"*(      {                                       )*"
-        R"*(        "name": "metric_counter2",            )*"
-        R"*(        "value": 20                           )*"
-        R"*(      },                                      )*"
-        R"*(      {                                       )*"
-        R"*(        "name": "metric_histogram1",          )*"
-        R"*(        "total_count": 17,                    )*"
-        R"*(        "min": 6,                             )*"
-        R"*(        "mean": 47.8235,                      )*"
-        R"*(        "percentile_75": 62,                  )*"
-        R"*(        "percentile_95": 72,                  )*"
-        R"*(        "percentile_99": 73,                  )*"
-        R"*(        "percentile_99_9": 73,                )*"
-        R"*(        "percentile_99_99": 73,               )*"
-        R"*(        "max": 73,                            )*"
-        R"*(        "total_sum": 813                      )*"
-        R"*(      }                                       )*"
-        R"*(    ]                                         )*"
-        R"*(  },                                          )*"
-        R"*(  {                                           )*"
-        R"*(    "type": "table",                          )*"
-        R"*(    "id": "table2",                           )*"
-        R"*(    "attributes": {                           )*"
-        R"*(      "attr1": "val3",                        )*"
-        R"*(      "attr2": "val2"                         )*"
-        R"*(    },                                        )*"
-        R"*(    "metrics": [                              )*"
-        R"*(      {                                       )*"
-        R"*(        "name": "metric_counter1",            )*"
-        R"*(        "value": 100                          )*"
-        R"*(      },                                      )*"
-        R"*(      {                                       )*"
-        R"*(        "name": "metric_histogram1",          )*"
-        R"*(        "total_count": 170,                   )*"
-        R"*(        "min": 60,                            )*"
-        R"*(        "mean": 478.235,                      )*"
-        R"*(        "percentile_75": 620,                 )*"
-        R"*(        "percentile_95": 720,                 )*"
-        R"*(        "percentile_99": 730,                 )*"
-        R"*(        "percentile_99_9": 735,               )*"
-        R"*(        "percentile_99_99": 735,              )*"
-        R"*(        "max": 735,                           )*"
-        R"*(        "total_sum": 8130                     )*"
-        R"*(      },                                      )*"
-        R"*(      {                                       )*"
-        R"*(        "name": "metric_histogram2",          )*"
-        R"*(        "total_count": 34,                    )*"
-        R"*(        "min": 6,                             )*"
-        R"*(        "mean": 47.8235,                      )*"
-        R"*(        "percentile_75": 62,                  )*"
-        R"*(        "percentile_95": 72,                  )*"
-        R"*(        "percentile_99": 72,                  )*"
-        R"*(        "percentile_99_9": 73,                )*"
-        R"*(        "percentile_99_99": 73,               )*"
-        R"*(        "max": 73,                            )*"
-        R"*(        "total_sum": 813                      )*"
-        R"*(      }                                       )*"
-        R"*(    ]                                         )*"
-        R"*(  }                                           )*"
-        R"*(]                                             )*");
+  auto collector = BuildCollector();
+  collector->metric_types_ = {
+      {"server_metric", "COUNTER"},
+      {"metric_counter1", "COUNTER"},
+      {"metric_counter2", "COUNTER"},
+      {"server_metric_histogram", "HISTOGRAM"},
+      {"metric_histogram1", "HISTOGRAM"},
+      {"metric_histogram2", "HISTOGRAM"}
+  };
+  string data(
+      R"*([                                             )*"
+      R"*(  {                                           )*"
+      R"*(    "type": "server",                         )*"
+      R"*(    "id": "kudu.tabletserver",                )*"
+      R"*(    "attributes": {                           )*"
+      R"*(      "attrA": "val1",                        )*"
+      R"*(      "attrB": "val2"                         )*"
+      R"*(    },                                        )*"
+      R"*(    "metrics": [                              )*"
+      R"*(      {                                       )*"
+      R"*(        "name": "server_metric",              )*"
+      R"*(        "value": 123                          )*"
+      R"*(      },                                      )*"
+      R"*(      {                                       )*"
+      R"*(        "name": "server_metric_histogram",    )*"
+      R"*(        "total_count": 60,                    )*"
+      R"*(        "min": 4,                             )*"
+      R"*(        "mean": 76.16666666666667,            )*"
+      R"*(        "percentile_75": 25,                  )*"
+      R"*(        "percentile_95": 66,                  )*"
+      R"*(        "percentile_99": 79,                  )*"
+      R"*(        "percentile_99_9": 3486,              )*"
+      R"*(        "percentile_99_99": 3486,             )*"
+      R"*(        "max": 3486,                          )*"
+      R"*(        "total_sum": 4570                     )*"
+      R"*(      }                                       )*"
+      R"*(    ]                                         )*"
+      R"*(  },                                          )*"
+      R"*(  {                                           )*"
+      R"*(    "type": "table",                          )*"
+      R"*(    "id": "table1",                           )*"
+      R"*(    "attributes": {                           )*"
+      R"*(      "attr1": "val2",                        )*"
+      R"*(      "attr2": "val3"                         )*"
+      R"*(    },                                        )*"
+      R"*(    "metrics": [                              )*"
+      R"*(      {                                       )*"
+      R"*(        "name": "metric_counter1",            )*"
+      R"*(        "value": 10                           )*"
+      R"*(      },                                      )*"
+      R"*(      {                                       )*"
+      R"*(        "name": "metric_counter2",            )*"
+      R"*(        "value": 20                           )*"
+      R"*(      },                                      )*"
+      R"*(      {                                       )*"
+      R"*(        "name": "metric_histogram1",          )*"
+      R"*(        "total_count": 17,                    )*"
+      R"*(        "min": 6,                             )*"
+      R"*(        "mean": 47.8235,                      )*"
+      R"*(        "percentile_75": 62,                  )*"
+      R"*(        "percentile_95": 72,                  )*"
+      R"*(        "percentile_99": 73,                  )*"
+      R"*(        "percentile_99_9": 73,                )*"
+      R"*(        "percentile_99_99": 73,               )*"
+      R"*(        "max": 73,                            )*"
+      R"*(        "total_sum": 813                      )*"
+      R"*(      }                                       )*"
+      R"*(    ]                                         )*"
+      R"*(  },                                          )*"
+      R"*(  {                                           )*"
+      R"*(    "type": "table",                          )*"
+      R"*(    "id": "table2",                           )*"
+      R"*(    "attributes": {                           )*"
+      R"*(      "attr1": "val3",                        )*"
+      R"*(      "attr2": "val2"                         )*"
+      R"*(    },                                        )*"
+      R"*(    "metrics": [                              )*"
+      R"*(      {                                       )*"
+      R"*(        "name": "metric_counter1",            )*"
+      R"*(        "value": 100                          )*"
+      R"*(      },                                      )*"
+      R"*(      {                                       )*"
+      R"*(        "name": "metric_histogram1",          )*"
+      R"*(        "total_count": 170,                   )*"
+      R"*(        "min": 60,                            )*"
+      R"*(        "mean": 478.235,                      )*"
+      R"*(        "percentile_75": 620,                 )*"
+      R"*(        "percentile_95": 720,                 )*"
+      R"*(        "percentile_99": 730,                 )*"
+      R"*(        "percentile_99_9": 735,               )*"
+      R"*(        "percentile_99_99": 735,              )*"
+      R"*(        "max": 735,                           )*"
+      R"*(        "total_sum": 8130                     )*"
+      R"*(      },                                      )*"
+      R"*(      {                                       )*"
+      R"*(        "name": "metric_histogram2",          )*"
+      R"*(        "total_count": 34,                    )*"
+      R"*(        "min": 6,                             )*"
+      R"*(        "mean": 47.8235,                      )*"
+      R"*(        "percentile_75": 62,                  )*"
+      R"*(        "percentile_95": 72,                  )*"
+      R"*(        "percentile_99": 72,                  )*"
+      R"*(        "percentile_99_9": 73,                )*"
+      R"*(        "percentile_99_99": 73,               )*"
+      R"*(        "max": 73,                            )*"
+      R"*(        "total_sum": 813                      )*"
+      R"*(      }                                       )*"
+      R"*(    ]                                         )*"
+      R"*(  }                                           )*"
+      R"*(]                                             )*");
 
-    // Attribute filter is empty.
-    {
-      MetricsCollector::TablesMetrics tables_metrics;
-      MetricsCollector::TablesHistMetrics tables_hist_metrics;
-      MetricsCollector::Metrics host_metrics;
-      MetricsCollector::HistMetrics host_hist_metrics;
-      ASSERT_OK(collector->ParseMetrics(data,
-                                       &tables_metrics, &host_metrics,
-                                       &tables_hist_metrics, &host_hist_metrics));
-      ASSERT_EQ(tables_metrics, MetricsCollector::TablesMetrics({
-          {
-            "table1",
-            {
-              {"metric_counter1", 10},
-              {"metric_counter2", 20},
-            }
-          },
-          {
-            "table2",
-            {
-              {"metric_counter1", 100}
-            }
-          }
-      }));
-      ASSERT_EQ(tables_hist_metrics, MetricsCollector::TablesHistMetrics({
-          {
-            "table1",
-            {
-              {
-                "metric_histogram1_percentile_99",
-                {
-                  {17, 73}
-                }
-              }
-            }
-          },
+  MetricsCollector::TablesMetrics tables_metrics;
+  MetricsCollector::TablesHistMetrics tables_hist_metrics;
+  MetricsCollector::Metrics host_metrics;
+  MetricsCollector::HistMetrics host_hist_metrics;
+  ASSERT_OK(collector->ParseMetrics(MetricsCollector::NodeType::kTServer,
+                                    data,
+                                    &tables_metrics, &host_metrics,
+                                    &tables_hist_metrics, &host_hist_metrics));
+  ASSERT_EQ(tables_metrics, MetricsCollector::TablesMetrics({
+      {
+        "table1",
+        {
+          {"metric_counter1", 10},
+          {"metric_counter2", 20},
+        }
+      },
+      {
+        "table2",
+        {
+          {"metric_counter1", 100}
+        }
+      }
+  }));
+  ASSERT_EQ(tables_hist_metrics, MetricsCollector::TablesHistMetrics({
+      {
+        "table1",
+        {
           {
-            "table2",
+            "metric_histogram1_percentile_99",
             {
-              {
-                "metric_histogram1_percentile_99",
-                {
-                  {170, 730}
-                }
-              },
-              {
-                "metric_histogram2_percentile_99",
-                {
-                  {34, 72}
-                }
-              }
+              {17, 73}
             }
           }
-      }));
-      ASSERT_EQ(host_metrics, MetricsCollector::Metrics({
-          {"metric_counter1", 110},
-          {"metric_counter2", 20}
-      }));
-      ASSERT_EQ(host_hist_metrics, MetricsCollector::HistMetrics({
+        }
+      },
+      {
+        "table2",
+        {
           {
             "metric_histogram1_percentile_99",
             {
-              {17, 73},
               {170, 730}
             }
           },
@@ -585,63 +548,42 @@ TEST(TestMetricsCollector, TestParseMetrics) {
               {34, 72}
             }
           }
-      }));
-    }
-
-    // Attribute filter is not empty.
-    {
-      collector->attributes_filter_ = {{"attr1", {"val1", "val2"}}};
-
-      MetricsCollector::TablesMetrics tables_metrics;
-      MetricsCollector::TablesHistMetrics tables_hist_metrics;
-      MetricsCollector::Metrics host_metrics;
-      MetricsCollector::HistMetrics host_hist_metrics;
-      ASSERT_OK(collector->ParseMetrics(data,
-                                       &tables_metrics, &host_metrics,
-                                       &tables_hist_metrics, &host_hist_metrics));
-      ASSERT_EQ(tables_metrics, MetricsCollector::TablesMetrics({
-          {
-            "table1",
-            {
-              {"metric_counter1", 10},
-              {"metric_counter2", 20},
-            }
-          }
-      }));
-      ASSERT_EQ(tables_hist_metrics, MetricsCollector::TablesHistMetrics({
-          {
-            "table1",
-            {
-              {
-                "metric_histogram1_percentile_99",
-                {
-                  {17, 73}
-                }
-              }
-            }
-          }
-      }));
-      ASSERT_EQ(host_metrics, MetricsCollector::Metrics({
-          {"metric_counter1", 10},
-          {"metric_counter2", 20}
-      }));
-      ASSERT_EQ(host_hist_metrics, MetricsCollector::HistMetrics({
-          {
-            "metric_histogram1_percentile_99",
-            {
-              {17, 73},
-            }
-          }
-      }));
-    }
-  }
+        }
+      }
+  }));
+  ASSERT_EQ(host_metrics, MetricsCollector::Metrics({
+      {"metric_counter1", 110},
+      {"metric_counter2", 20},
+      {"server_metric", 123}
+  }));
+  ASSERT_EQ(host_hist_metrics, MetricsCollector::HistMetrics({
+      {
+        "metric_histogram1_percentile_99",
+        {
+          {17, 73},
+          {170, 730}
+        }
+      },
+      {
+        "metric_histogram2_percentile_99",
+        {
+          {34, 72}
+        }
+      },
+      {
+        "server_metric_histogram_percentile_99",
+        {
+          {60, 79}
+        }
+      }
+  }));
 }
 
 TEST(TestMetricsCollector, TestInitMetrics) {
   FLAGS_collector_metrics_types_for_test =
       R"*([                                                       )*"
       R"*(  {                                                     )*"
-      R"*(    "type": "tablet",                                   )*"
+      R"*(    "type": "table",                                   )*"
       R"*(    "id": "table1",                                     )*"
       R"*(    "metrics": [                                        )*"
       R"*(      {                                                 )*"
@@ -659,24 +601,6 @@ TEST(TestMetricsCollector, TestInitMetrics) {
       R"*(    ]                                                   )*"
       R"*(  },                                                    )*"
       R"*(  {                                                     )*"
-      R"*(    "type": "tablet",                                   )*"
-      R"*(    "id": "table2",                                     )*"
-      R"*(    "metrics": [                                        )*"
-      R"*(      {                                                 )*"
-      R"*(        "name": "counter_metric1",                      )*"
-      R"*(        "type": "counter"                               )*"
-      R"*(      },                                                )*"
-      R"*(      {                                                 )*"
-      R"*(        "name": "histogram_metric1",                    )*"
-      R"*(        "type": "histogram"                             )*"
-      R"*(      },                                                )*"
-      R"*(      {                                                 )*"
-      R"*(        "name": "gauge_metric1",                        )*"
-      R"*(        "type": "gauge"                                 )*"
-      R"*(      }                                                 )*"
-      R"*(    ]                                                   )*"
-      R"*(  },                                                    )*"
-      R"*(  {                                                     )*"
       R"*(    "type": "server",                                   )*"
       R"*(    "metrics": [                                        )*"
       R"*(      {                                                 )*"
@@ -696,25 +620,15 @@ TEST(TestMetricsCollector, TestInitMetrics) {
       R"*(]                                                       )*";
   auto collector = BuildCollector();
   ASSERT_OK(collector->InitMetrics());
-  map<string, MetricsCollector::MetricTypes> expect_metric_types({
-      {
-        "tablet",
-        {
-          {"counter_metric1", "COUNTER"},
-          {"histogram_metric1", "HISTOGRAM"},
-          {"gauge_metric1", "GAUGE"},
-        }
-      },
-      {
-        "server",
-        {
-          {"counter_metric2", "COUNTER"},
-          {"histogram_metric2", "HISTOGRAM"},
-          {"gauge_metric2", "GAUGE"},
-        }
-      }
+  MetricsCollector::MetricTypes expect_metric_types({
+      {"counter_metric1", "COUNTER"},
+      {"histogram_metric1", "HISTOGRAM"},
+      {"gauge_metric1", "GAUGE"},
+      {"counter_metric2", "COUNTER"},
+      {"histogram_metric2", "HISTOGRAM"},
+      {"gauge_metric2", "GAUGE"}
   });
-  ASSERT_EQ(collector->metric_types_by_entity_type_, expect_metric_types);
+  ASSERT_EQ(collector->metric_types_, expect_metric_types);
 }
 
 TEST(TestMetricsCollector, TestInitFilters) {
diff --git a/src/kudu/collector/metrics_collector.cc b/src/kudu/collector/metrics_collector.cc
index 688ea70..fc9f20a 100644
--- a/src/kudu/collector/metrics_collector.cc
+++ b/src/kudu/collector/metrics_collector.cc
@@ -17,9 +17,9 @@
 
 #include "kudu/collector/metrics_collector.h"
 
-#include <string.h>
-
+#include <algorithm>
 #include <cmath>
+#include <cstring>
 #include <functional>
 #include <list>
 #include <ostream>
@@ -59,27 +59,27 @@ DEFINE_string(collector_attributes, "",
 DEFINE_string(collector_cluster_level_metrics, "on_disk_size,on_disk_data_size",
               "Metric names which should be merged and pushed to cluster level view "
               "(comma-separated list of metric names)");
-DEFINE_bool(collector_ignore_hosttable_level_metrics, false,
-            "Whether to ignore to report host-table level metrics.");
+DEFINE_string(collector_hosttable_level_metrics, "merged_entities_count_of_tablet",
+              "Host-table level metrics need to report (comma-separated list of metric names).");
 DEFINE_string(collector_metrics, "",
               "Metrics to collect (comma-separated list of metric names)");
 DEFINE_string(collector_metrics_types_for_test, "",
-              "Only for test, used to initialize metric_types_by_entity_type_");
+              "Only for test, used to initialize metric_types_");
 DEFINE_bool(collector_request_merged_metrics, true,
             "Whether to request merged metrics and exclude unmerged metrics from server");
 
 DECLARE_string(collector_cluster_name);
-DECLARE_int32(collector_interval_sec);
-DECLARE_int32(collector_timeout_sec);
-DECLARE_int32(collector_warn_threshold_ms);
+DECLARE_uint32(collector_interval_sec);
+DECLARE_uint32(collector_timeout_sec);
+DECLARE_uint32(collector_warn_threshold_ms);
 
 using rapidjson::Value;
 using std::list;
-using std::map;
 using std::set;
 using std::string;
 using std::vector;
 using std::unordered_map;
+using std::unordered_set;
 using strings::Substitute;
 
 namespace kudu {
@@ -105,6 +105,7 @@ Status MetricsCollector::Init() {
   RETURN_NOT_OK(InitMetrics());
   RETURN_NOT_OK(InitFilters());
   RETURN_NOT_OK(InitMetricsUrlParameters());
+  RETURN_NOT_OK(InitHostTableLevelMetrics());
   RETURN_NOT_OK(InitClusterLevelMetrics());
 
   initialized_ = true;
@@ -147,7 +148,8 @@ void MetricsCollector::MetricCollectorThread() {
   MonoTime collect_time;
   do {
     collect_time = MonoTime::Now();
-    WARN_NOT_OK(CollectAndReportMetrics(), "Unable to collect metrics");
+    WARN_NOT_OK(CollectAndReportTServerMetrics(), "Unable to collect tserver metrics");
+    WARN_NOT_OK(CollectAndReportMasterMetrics(), "Unable to collect master metrics");
     collect_time += MonoDelta::FromSeconds(FLAGS_collector_interval_sec);
   } while (!RunOnceMode() && !stop_background_threads_latch_.WaitUntil(collect_time));
   LOG(INFO) << "MetricCollectorThread exit";
@@ -175,41 +177,61 @@ Status MetricsCollector::UpdateThreadPool(int32_t thread_count) {
 }
 
 Status MetricsCollector::InitMetrics() {
+  MetricTypes metric_types;
+  InitMetricsFromNode(NodeType::kMaster, &metric_types);
+
+  MetricTypes tserver_metric_types;
+  InitMetricsFromNode(NodeType::kTServer, &tserver_metric_types);
+
+  // TODO(yingchun): check values in debug mode.
+  for (const auto& metric_type : tserver_metric_types) {
+    const auto* type = FindOrNull(metric_types, metric_type.first);
+    if (type) {
+      CHECK_EQ(*type, metric_type.second);
+    } else {
+      EmplaceOrDie(&metric_types, std::make_pair(metric_type.first, metric_type.second));
+    }
+  }
+
+  metric_types_.swap(metric_types);
+  return Status::OK();
+}
+
+Status MetricsCollector::InitMetricsFromNode(NodeType node_type, MetricTypes* metric_types) const {
+  DCHECK(metric_types);
+
   string resp;
   if (PREDICT_TRUE(FLAGS_collector_metrics_types_for_test.empty())) {
+    auto node_addr = node_type == NodeType::kMaster ?
+        nodes_checker_->GetFirstMaster() : nodes_checker_->GetFirstTServer();
     RETURN_NOT_OK(GetMetrics(
-        nodes_checker_->GetFirstMaster() + "/metrics?include_schema=1", &resp));
+        node_addr + "/metrics?include_schema=1&merge_rules=tablet|table|table_name", &resp));
   } else {
     resp = FLAGS_collector_metrics_types_for_test;
   }
+
   JsonReader r(resp);
   RETURN_NOT_OK(r.Init());
   vector<const Value*> entities;
   RETURN_NOT_OK(r.ExtractObjectArray(r.root(), nullptr, &entities));
 
-  map<string, MetricTypes> metric_types_by_entity_type;
-  bool tablet_entity_inited = false;
+  bool table_entity_inited = false;
   bool server_entity_inited = false;
   for (const Value* entity : entities) {
     string entity_type;
     CHECK_OK(r.ExtractString(entity, "type", &entity_type));
-    if (entity_type == "tablet") {
-      if (tablet_entity_inited) continue;
-      EmplaceOrDie(&metric_types_by_entity_type, std::make_pair("tablet", MetricTypes()));
-      auto& tablet_metric_types = FindOrDie(metric_types_by_entity_type, "tablet");
-      ExtractMetricTypes(r, entity, &tablet_metric_types);
-      tablet_entity_inited = true;
+    if (entity_type == "table") {
+      if (table_entity_inited) continue;
+      ExtractMetricTypes(r, entity, metric_types);
+      table_entity_inited = true;
     } else if (entity_type == "server") {
       if (server_entity_inited) continue;
-      EmplaceOrDie(&metric_types_by_entity_type, std::make_pair("server", MetricTypes()));
-      auto& server_metric_types = FindOrDie(metric_types_by_entity_type, "server");
-      ExtractMetricTypes(r, entity, &server_metric_types);
+      ExtractMetricTypes(r, entity, metric_types);
       server_entity_inited = true;
     } else {
       LOG(WARNING) << "unhandled entity type " << entity_type;
     }
   }
-  metric_types_by_entity_type_.swap(metric_types_by_entity_type);
   return Status::OK();
 }
 
@@ -260,7 +282,6 @@ Status MetricsCollector::InitMetricsUrlParameters() {
                   "want collector work well";
   }
 
-  // TODO(yingchun) This is supported since version 1.10
   if (!attributes_filter_.empty()) {
     metric_url_parameters_ += "&attributes=";
   }
@@ -272,6 +293,13 @@ Status MetricsCollector::InitMetricsUrlParameters() {
   return Status::OK();
 }
 
+Status MetricsCollector::InitHostTableLevelMetrics() {
+  unordered_set<string> hosttable_metrics(
+      Split(FLAGS_collector_hosttable_level_metrics, ",", strings::SkipEmpty()));
+  hosttable_metrics_.swap(hosttable_metrics);
+  return Status::OK();
+}
+
 Status MetricsCollector::InitClusterLevelMetrics() {
   Metrics cluster_metrics;
   vector<string> metric_names =
@@ -283,14 +311,52 @@ Status MetricsCollector::InitClusterLevelMetrics() {
   return Status::OK();
 }
 
-Status MetricsCollector::CollectAndReportMetrics() {
-  LOG(INFO) << "Start to CollectAndReportMetrics";
+Status MetricsCollector::CollectAndReportMasterMetrics() {
+  LOG(INFO) << "Start to CollectAndReportMasterMetrics";
   MonoTime start(MonoTime::Now());
   scoped_refptr<Trace> trace(new Trace);
   ADOPT_TRACE(trace.get());
-  TRACE_EVENT0("collector", "MetricsCollector::CollectAndReportMetrics");
+  TRACE_EVENT0("collector", "MetricsCollector::CollectAndReportMasterMetrics");
   TRACE("init");
-  vector<string> tserver_http_addrs = nodes_checker_->GetNodes();
+  vector<string> master_http_addrs = nodes_checker_->GetMasters();
+  TRACE("Nodes got");
+  if (master_http_addrs.empty()) {
+    return Status::OK();
+  }
+  RETURN_NOT_OK(UpdateThreadPool(std::max(host_metric_collector_thread_pool_->num_threads(),
+                                          static_cast<int32_t>(master_http_addrs.size()))));
+  for (int i = 0; i < master_http_addrs.size(); ++i) {
+    RETURN_NOT_OK(host_metric_collector_thread_pool_->SubmitFunc(
+      std::bind(&MetricsCollector::CollectAndReportHostLevelMetrics,
+                this,
+                NodeType::kMaster,
+                master_http_addrs[i] + metric_url_parameters_,
+                nullptr,
+                nullptr)));
+  }
+  TRACE("Thead pool jobs submitted");
+  host_metric_collector_thread_pool_->Wait();
+  TRACE("Thead pool jobs done");
+
+  int64_t elapsed_ms = (MonoTime::Now() - start).ToMilliseconds();
+  if (elapsed_ms > FLAGS_collector_warn_threshold_ms) {
+    if (Trace::CurrentTrace()) {
+      LOG(WARNING) << "Trace:" << std::endl
+                   << Trace::CurrentTrace()->DumpToString();
+    }
+  }
+
+  return Status::OK();
+}
+
+Status MetricsCollector::CollectAndReportTServerMetrics() {
+  LOG(INFO) << "Start to CollectAndReportTServerMetrics";
+  MonoTime start(MonoTime::Now());
+  scoped_refptr<Trace> trace(new Trace);
+  ADOPT_TRACE(trace.get());
+  TRACE_EVENT0("collector", "MetricsCollector::CollectAndReportTServerMetrics");
+  TRACE("init");
+  vector<string> tserver_http_addrs = nodes_checker_->GetTServers();
   TRACE("Nodes got");
   if (tserver_http_addrs.empty()) {
     return Status::OK();
@@ -302,6 +368,7 @@ Status MetricsCollector::CollectAndReportMetrics() {
     RETURN_NOT_OK(host_metric_collector_thread_pool_->SubmitFunc(
       std::bind(&MetricsCollector::CollectAndReportHostLevelMetrics,
                 this,
+                NodeType::kTServer,
                 tserver_http_addrs[i] + metric_url_parameters_,
                 &hosts_metrics_by_table_name[i],
                 &hosts_hist_metrics_by_table_name[i])));
@@ -481,26 +548,21 @@ Status MetricsCollector::ConvertStateToInt(const string& value, int64_t* result)
   return Status::OK();
 }
 
-bool MetricsCollector::FilterByAttribute(const JsonReader& r,
-                                         const rapidjson::Value* entity) const {
-  if (attributes_filter_.empty()) {
-    return false;
-  }
-  const Value* attributes;
-  CHECK_OK(r.ExtractObject(entity, "attributes", &attributes));
-  for (const auto& name_values : attributes_filter_) {
-    string value;
-    Status s = r.ExtractString(attributes, name_values.first.c_str(), &value);
-    if (s.ok() && ContainsKey(name_values.second, value)) {
-      return false;
-    }
-  }
-  return true;
-}
+Status MetricsCollector::ParseServerMetrics(const JsonReader& r,
+                                            const rapidjson::Value* entity,
+                                            Metrics* host_metrics,
+                                            HistMetrics* host_hist_metrics) const {
+  CHECK(entity);
+  CHECK(host_metrics);
+  CHECK(host_hist_metrics);
+
+  string server_type;
+  CHECK_OK(r.ExtractString(entity, "id", &server_type));
+  CHECK(server_type == "kudu.tabletserver" || server_type == "kudu.master");
+
+  CHECK_OK(ParseEntityMetrics(r, entity, host_metrics, nullptr, host_hist_metrics, nullptr));
 
-Status MetricsCollector::ParseServerMetrics(const JsonReader& /*r*/,
-                                            const rapidjson::Value* /*entity*/) {
-  return Status::NotSupported("server entity is not supported");
+  return Status::OK();
 }
 
 Status MetricsCollector::ParseTableMetrics(const JsonReader& r,
@@ -509,6 +571,7 @@ Status MetricsCollector::ParseTableMetrics(const JsonReader& r,
                                            Metrics* host_metrics,
                                            TablesHistMetrics* hist_metrics_by_table_name,
                                            HistMetrics* host_hist_metrics) const {
+  CHECK(entity);
   CHECK(metrics_by_table_name);
   CHECK(host_metrics);
   CHECK(hist_metrics_by_table_name);
@@ -525,14 +588,47 @@ Status MetricsCollector::ParseTableMetrics(const JsonReader& r,
   EmplaceOrDie(hist_metrics_by_table_name, std::make_pair(table_name, HistMetrics()));
   auto& table_hist_metrics = FindOrDie(*hist_metrics_by_table_name, table_name);
 
+  CHECK_OK(ParseEntityMetrics(r, entity,
+      &table_metrics, host_metrics, &table_hist_metrics, host_hist_metrics));
+
+  return Status::OK();
+}
+
+Status MetricsCollector::ParseCatalogMetrics(const JsonReader& r,
+                                             const rapidjson::Value* entity,
+                                             Metrics* tablet_metrics,
+                                             HistMetrics* tablet_hist_metrics) const {
+  CHECK(entity);
+  CHECK(tablet_metrics);
+  CHECK(tablet_hist_metrics);
+
+  string tablet_id;
+  CHECK_OK(r.ExtractString(entity, "id", &tablet_id));
+  if (tablet_id != "sys.catalog") {  // Only used to parse 'sys.catalog'.
+    return Status::OK();
+  }
+
+  CHECK_OK(ParseEntityMetrics(r, entity, tablet_metrics, nullptr, tablet_hist_metrics, nullptr));
+
+  return Status::OK();
+}
+
+Status MetricsCollector::ParseEntityMetrics(const JsonReader& r,
+                                            const rapidjson::Value* entity,
+                                            Metrics* kv_metrics,
+                                            Metrics* merged_kv_metrics,
+                                            HistMetrics* hist_metrics,
+                                            HistMetrics* merged_hist_metrics) const {
+  CHECK(entity);
+  CHECK(kv_metrics);
+  CHECK(hist_metrics);
+
   vector<const Value*> metrics;
   CHECK_OK(r.ExtractObjectArray(entity, "metrics", &metrics));
   for (const Value* metric : metrics) {
     string name;
     CHECK_OK(r.ExtractString(metric, "name", &name));
-    const auto* tablet_metric_types = FindOrNull(metric_types_by_entity_type_, "tablet");
-    CHECK(tablet_metric_types);
-    const auto* known_type = FindOrNull(*tablet_metric_types, name);
+    const auto* known_type = FindOrNull(metric_types_, name);
     if (!known_type) {
       LOG(ERROR) << Substitute("metric $0 has unknown type, ignore it", name);
       continue;
@@ -554,10 +650,11 @@ Status MetricsCollector::ParseTableMetrics(const JsonReader& r,
           LOG(FATAL) << "Unknown type, metrics name: " << name;
       }
 
-      EmplaceOrDie(&table_metrics, std::make_pair(name, value));
-      if (!EmplaceIfNotPresent(host_metrics, std::make_pair(name, value))) {
-        auto& host_metric = FindOrDie(*host_metrics, name);
-        host_metric += value;
+      EmplaceOrDie(kv_metrics, std::make_pair(name, value));
+      if (merged_kv_metrics &&
+          !EmplaceIfNotPresent(merged_kv_metrics, std::make_pair(name, value))) {
+        auto& found_metric = FindOrDie(*merged_kv_metrics, name);
+        found_metric += value;
       }
     } else if (*known_type == "HISTOGRAM") {
       for (const auto& percentile : kRegisterPercentiles) {
@@ -568,10 +665,11 @@ Status MetricsCollector::ParseTableMetrics(const JsonReader& r,
         int64_t percentile_value;
         CHECK_OK(r.ExtractInt64(metric, percentile.c_str(), &percentile_value));
         vector<SimpleHistogram> tmp({{total_count, percentile_value}});
-        EmplaceOrDie(&table_hist_metrics, std::make_pair(hist_metric_name, tmp));
-        if (!EmplaceIfNotPresent(host_hist_metrics, std::make_pair(hist_metric_name, tmp))) {
-          auto& host_hist_metric = FindOrDie(*host_hist_metrics, hist_metric_name);
-          host_hist_metric.emplace_back(tmp[0]);
+        EmplaceOrDie(hist_metrics, std::make_pair(hist_metric_name, tmp));
+        if (merged_hist_metrics &&
+            !EmplaceIfNotPresent(merged_hist_metrics, std::make_pair(hist_metric_name, tmp))) {
+          auto& found_hist_metric = FindOrDie(*merged_hist_metrics, hist_metric_name);
+          found_hist_metric.emplace_back(tmp[0]);
         }
       }
     } else {
@@ -582,12 +680,8 @@ Status MetricsCollector::ParseTableMetrics(const JsonReader& r,
   return Status::OK();
 }
 
-Status MetricsCollector::ParseTabletMetrics(const JsonReader& /*r*/,
-                                            const rapidjson::Value* /*entity*/) {
-  return Status::NotSupported("tablet entity is not supported");
-}
-
 Status MetricsCollector::CollectAndReportHostLevelMetrics(
+    NodeType node_type,
     const string& url,
     TablesMetrics* metrics_by_table_name,
     TablesHistMetrics* hist_metrics_by_table_name) {
@@ -597,8 +691,6 @@ Status MetricsCollector::CollectAndReportHostLevelMetrics(
   TRACE_EVENT1("collector", "MetricsCollector::CollectAndReportHostLevelMetrics",
                "url", url);
   TRACE("init");
-  CHECK(metrics_by_table_name);
-  CHECK(hist_metrics_by_table_name);
 
   // Get metrics from server.
   string resp;
@@ -607,14 +699,14 @@ Status MetricsCollector::CollectAndReportHostLevelMetrics(
   // Merge metrics by table and metric type.
   Metrics host_metrics;
   HistMetrics host_hist_metrics;
-  RETURN_NOT_OK(ParseMetrics(resp, metrics_by_table_name, &host_metrics,
+  RETURN_NOT_OK(ParseMetrics(node_type, resp, metrics_by_table_name, &host_metrics,
                              hist_metrics_by_table_name, &host_hist_metrics));
 
   string host_name = ExtractHostName(url);
   auto timestamp = static_cast<uint64_t>(WallTime_Now());
 
   // Host table level.
-  if (!FLAGS_collector_ignore_hosttable_level_metrics) {
+  if (metrics_by_table_name && hist_metrics_by_table_name) {
     RETURN_NOT_OK(ReportHostTableLevelMetrics(host_name, timestamp,
                                               *metrics_by_table_name,
                                               *hist_metrics_by_table_name));
@@ -635,7 +727,8 @@ Status MetricsCollector::CollectAndReportHostLevelMetrics(
   return Status::OK();
 }
 
-Status MetricsCollector::ParseMetrics(const string& data,
+Status MetricsCollector::ParseMetrics(NodeType node_type,
+                                      const string& data,
                                       TablesMetrics* metrics_by_table_name,
                                       Metrics* host_metrics,
                                       TablesHistMetrics* hist_metrics_by_table_name,
@@ -646,19 +739,19 @@ Status MetricsCollector::ParseMetrics(const string& data,
   RETURN_NOT_OK(r.ExtractObjectArray(r.root(), nullptr, &entities));
 
   for (const Value* entity : entities) {
-    if (FilterByAttribute(r, entity)) {
-      continue;
-    }
     string entity_type;
     CHECK_OK(r.ExtractString(entity, "type", &entity_type));
     if (entity_type == "server") {
-      CHECK(ParseServerMetrics(r, entity).IsNotSupported());
+      CHECK_OK(ParseServerMetrics(r, entity, host_metrics, host_hist_metrics));
     } else if (entity_type == "table") {
-      CHECK_OK(ParseTableMetrics(r, entity,
-                                 metrics_by_table_name, host_metrics,
-                                 hist_metrics_by_table_name, host_hist_metrics));
-    } else if (entity_type == "tablet") {
-      CHECK(ParseTabletMetrics(r, entity).IsNotSupported());
+      if (NodeType::kMaster == node_type) {
+        CHECK_OK(ParseCatalogMetrics(r, entity, host_metrics, host_hist_metrics));
+      } else {
+        CHECK(NodeType::kTServer == node_type);
+        CHECK_OK(ParseTableMetrics(r, entity,
+                                   metrics_by_table_name, host_metrics,
+                                   hist_metrics_by_table_name, host_hist_metrics));
+      }
     } else {
       LOG(FATAL) << "Unknown entity_type: " << entity_type;
     }
@@ -681,17 +774,17 @@ void MetricsCollector::CollectMetrics(const string& endpoint,
                                level,
                                timestamp,
                                metric.second,
-                               FindOrDie(metric_types_by_entity_type_["tablet"], metric.first),
+                               metric_types_[metric.first],
                                extra_tags));
   }
 }
 
 void MetricsCollector::CollectMetrics(const string& endpoint,
-                      const HistMetrics& metrics,
-                      const string& level,
-                      uint64_t timestamp,
-                      const string& extra_tags,
-                      list<scoped_refptr<ItemBase>>* items) {
+                                      const HistMetrics& metrics,
+                                      const string& level,
+                                      uint64_t timestamp,
+                                      const string& extra_tags,
+                                      list<scoped_refptr<ItemBase>>* items) {
   for (const auto& metric : metrics) {
     items->emplace_back(
       reporter_->ConstructItem(endpoint,
@@ -714,8 +807,14 @@ Status MetricsCollector::ReportHostTableLevelMetrics(
   int metrics_count = 0;
   for (const auto& table_metrics : metrics_by_table_name) {
     const auto extra_tag = Substitute("table=$0", table_metrics.first);
-    metrics_count += table_metrics.second.size();
-    CollectMetrics(host_name, table_metrics.second, "host_table", timestamp, extra_tag, &items);
+    Metrics filtered_metrics;
+    for (const auto& metric : table_metrics.second) {
+      if (ContainsKey(hosttable_metrics_, metric.first)) {
+        filtered_metrics.insert(metric);
+      }
+    }
+    metrics_count += filtered_metrics.size();
+    CollectMetrics(host_name, filtered_metrics, "host_table", timestamp, extra_tag, &items);
   }
   TRACE(Substitute("Host-table GAUGE/COUNTER type metrics collected, count $0", metrics_count));
 
@@ -723,10 +822,14 @@ Status MetricsCollector::ReportHostTableLevelMetrics(
   int hist_metrics_count = 0;
   for (const auto& table_hist_metrics : hist_metrics_by_table_name) {
     const auto extra_tag = Substitute("table=$0", table_hist_metrics.first);
+    HistMetrics filtered_metrics;
+    for (const auto& metric : table_hist_metrics.second) {
+      if (ContainsKey(hosttable_metrics_, metric.first)) {
+        filtered_metrics.insert(metric);
+      }
+    }
     hist_metrics_count += table_hist_metrics.second.size();
-    CollectMetrics(host_name, table_hist_metrics.second,
-                   "host_table", timestamp, extra_tag,
-                   &items);
+    CollectMetrics(host_name, filtered_metrics, "host_table", timestamp, extra_tag, &items);
   }
   TRACE(Substitute("Host-table HISTOGRAM type metrics collected, count $0", hist_metrics_count));
 
diff --git a/src/kudu/collector/metrics_collector.h b/src/kudu/collector/metrics_collector.h
index 05673ef..2401030 100644
--- a/src/kudu/collector/metrics_collector.h
+++ b/src/kudu/collector/metrics_collector.h
@@ -18,11 +18,11 @@
 
 #include <cstdint>
 #include <list>
-#include <map>
 #include <memory>
 #include <set>
 #include <string>
 #include <unordered_map>
+#include <unordered_set>
 #include <vector>
 
 #include <gtest/gtest_prod.h>
@@ -70,15 +70,23 @@ class MetricsCollector : public RefCounted<MetricsCollector> {
   FRIEND_TEST(TestMetricsCollector, TestMergeToTableLevelMetrics);
   FRIEND_TEST(TestMetricsCollector, TestMergeToClusterLevelMetrics);
   FRIEND_TEST(TestMetricsCollector, TestParseMetrics);
+  FRIEND_TEST(TestMetricsCollector, TestParseTypesOfMetrics);
   FRIEND_TEST(TestMetricsCollector, TestInitMetrics);
   FRIEND_TEST(TestMetricsCollector, TestInitFilters);
   FRIEND_TEST(TestMetricsCollector, TestInitMetricsUrlParameters);
   FRIEND_TEST(TestMetricsCollector, TestInitClusterLevelMetrics);
 
+  // Metric name --> value, metric is in type of GAUGE or COUNTER.
   typedef std::unordered_map<std::string, int64_t> Metrics;
+  // Table name --> metric name-value pairs.
   typedef std::unordered_map<std::string, Metrics> TablesMetrics;
+
+  // Simple struct to collect histogram metrics.
   struct SimpleHistogram {
+    // 'total_count' value in histogram metric.
     int64_t count;
+    // 'percentile_xxx" value in histogram metric, percentile_xxx is specified
+    // by kRegisterPercentiles.
     int64_t value;
     SimpleHistogram(int64_t c, int64_t v) : count(c), value(v) {
     }
@@ -86,27 +94,37 @@ class MetricsCollector : public RefCounted<MetricsCollector> {
       return count == rhs.count && value == rhs.value;
     }
   };
-
+  // Metric name --> SimpleHistogram, metric is in type of HISTOGRAM.
   typedef std::unordered_map<std::string, std::vector<SimpleHistogram>> HistMetrics;
+  // Table name --> metric name-struct pairs.
   typedef std::unordered_map<std::string, HistMetrics> TablesHistMetrics;
 
+  // Metric name --> type, where type is in 'COUNTER', 'GAUGE' and 'HISTOGRAM'.
   typedef std::unordered_map<std::string, std::string> MetricTypes;
 
   Status InitMetrics();
+  enum class NodeType {
+    kMaster = 0,
+    kTServer = 1,
+  };
+  Status InitMetricsFromNode(NodeType node_type, MetricTypes* metric_types) const;
   static Status ExtractMetricTypes(const JsonReader& r,
                                    const rapidjson::Value* entity,
                                    MetricTypes* metric_types);
   Status InitFilters();
   Status InitMetricsUrlParameters();
+  Status InitHostTableLevelMetrics();
   Status InitClusterLevelMetrics();
 
   Status StartMetricCollectorThread();
   void MetricCollectorThread();
-  Status CollectAndReportMetrics();
+  Status CollectAndReportMasterMetrics();
+  Status CollectAndReportTServerMetrics();
 
   Status UpdateThreadPool(int32_t thread_count);
 
-  Status CollectAndReportHostLevelMetrics(const std::string& url,
+  Status CollectAndReportHostLevelMetrics(NodeType node_type,
+                                          const std::string& url,
                                           TablesMetrics* metrics_by_table_name,
                                           TablesHistMetrics* hist_metrics_by_table_name);
 
@@ -152,26 +170,33 @@ class MetricsCollector : public RefCounted<MetricsCollector> {
   static Status GetMetrics(const std::string& url, std::string* resp);
 
   // Parse metrics from http response, entities may be in different types.
-  Status ParseMetrics(const std::string& data,
+  Status ParseMetrics(NodeType node_type,
+                      const std::string& data,
                       TablesMetrics* metrics_by_table_name,
                       Metrics* host_metrics,
                       TablesHistMetrics* hist_metrics_by_table_name,
                       HistMetrics* host_hist_metrics);
-  static Status ParseServerMetrics(const JsonReader& r,
-                                   const rapidjson::Value* entity);
+  Status ParseServerMetrics(const JsonReader& r,
+                            const rapidjson::Value* entity,
+                            Metrics* host_metrics,
+                            HistMetrics* host_hist_metrics) const;
   Status ParseTableMetrics(const JsonReader& r,
                            const rapidjson::Value* entity,
                            TablesMetrics* metrics_by_table_name,
                            Metrics* host_metrics,
                            TablesHistMetrics* hist_metrics_by_table_name,
                            HistMetrics* host_hist_metrics) const;
-  static Status ParseTabletMetrics(const JsonReader& r,
-                                   const rapidjson::Value* entity);
+  Status ParseCatalogMetrics(const JsonReader& r,
+                             const rapidjson::Value* entity,
+                             Metrics* tablet_metrics,
+                             HistMetrics* tablet_hist_metrics) const;
+  Status ParseEntityMetrics(const JsonReader& r,
+                            const rapidjson::Value* entity,
+                            Metrics* kv_metrics,
+                            Metrics* merged_kv_metrics,
+                            HistMetrics* hist_metrics,
+                            HistMetrics* merged_hist_metrics) const;
 
-  // Return true when this entity could be filtered.
-  // When server side support attributes filter, this function has no effect.
-  bool FilterByAttribute(const JsonReader& r,
-                         const rapidjson::Value* entity) const;
   Status GetNumberMetricValue(const rapidjson::Value* metric,
                               const std::string& metric_name,
                               int64_t* result) const;
@@ -187,11 +212,12 @@ class MetricsCollector : public RefCounted<MetricsCollector> {
   scoped_refptr<NodesChecker> nodes_checker_;
   scoped_refptr<ReporterBase> reporter_;
 
-  std::map<std::string, MetricTypes> metric_types_by_entity_type_;
+  MetricTypes metric_types_;
   // Attribute filter, attributes not in this map will be filtered if it's not empty.
   // attribute name ---> attribute values
   std::unordered_map<std::string, std::set<std::string>> attributes_filter_;
   std::string metric_url_parameters_;
+  std::unordered_set<std::string> hosttable_metrics_;
   Metrics cluster_metrics_;
 
   CountDownLatch stop_background_threads_latch_;
diff --git a/src/kudu/collector/nodes_checker.cc b/src/kudu/collector/nodes_checker.cc
index 8b64c29..b2dacd8 100644
--- a/src/kudu/collector/nodes_checker.cc
+++ b/src/kudu/collector/nodes_checker.cc
@@ -43,9 +43,9 @@
 
 DECLARE_string(collector_cluster_name);
 DECLARE_string(collector_master_addrs);
-DECLARE_int32(collector_interval_sec);
-DECLARE_int32(collector_timeout_sec);
-DECLARE_int32(collector_warn_threshold_ms);
+DECLARE_uint32(collector_interval_sec);
+DECLARE_uint32(collector_timeout_sec);
+DECLARE_uint32(collector_warn_threshold_ms);
 
 using rapidjson::Value;
 using std::list;
@@ -108,7 +108,12 @@ string NodesChecker::ToString() const {
   return "NodesChecker";
 }
 
-vector<string> NodesChecker::GetNodes() {
+vector<string> NodesChecker::GetMasters() {
+  shared_lock<RWMutex> l(nodes_lock_);
+  return master_http_addrs_;
+}
+
+vector<string> NodesChecker::GetTServers() {
   shared_lock<RWMutex> l(nodes_lock_);
   return tserver_http_addrs_;
 }
@@ -119,6 +124,12 @@ string NodesChecker::GetFirstMaster() {
   return master_http_addrs_[0];
 }
 
+string NodesChecker::GetFirstTServer() {
+  shared_lock<RWMutex> l(nodes_lock_);
+  CHECK(!tserver_http_addrs_.empty());
+  return tserver_http_addrs_[0];
+}
+
 Status NodesChecker::StartNodesCheckerThread() {
   return Thread::Create("collector", "nodes-checker", &NodesChecker::NodesCheckerThread,
                         this, &nodes_checker_thread_);
diff --git a/src/kudu/collector/nodes_checker.h b/src/kudu/collector/nodes_checker.h
index 26aee89..8189271 100644
--- a/src/kudu/collector/nodes_checker.h
+++ b/src/kudu/collector/nodes_checker.h
@@ -49,8 +49,10 @@ class NodesChecker : public RefCounted<NodesChecker> {
 
   std::string ToString() const;
 
-  std::vector<std::string> GetNodes();
+  std::vector<std::string> GetMasters();
+  std::vector<std::string> GetTServers();
   std::string GetFirstMaster();
+  std::string GetFirstTServer();
 
  private:
   friend class RefCounted<NodesChecker>;
diff --git a/src/kudu/scripts/build_kudu.sh b/src/kudu/scripts/build_kudu.sh
index 1016697..58e92f4 100755
--- a/src/kudu/scripts/build_kudu.sh
+++ b/src/kudu/scripts/build_kudu.sh
@@ -42,6 +42,35 @@ function get_stdcpp_lib()
     fi
 }
 
+function get_system_lib()
+{
+    libname=`ldd ${BASE_DIR}/build/latest/bin/kudu 2>/dev/null | grep "lib${1}\.so"`
+    libname=`echo $libname | cut -f1 -d" "`
+    libs=(`ldconfig -p|grep $libname|awk '{print $NF}'`)
+
+    bit_mode=`getconf LONG_BIT`
+    for lib in ${libs[*]}; do
+        if [ "`check_bit $lib`" = "true" ]; then
+            echo "$lib"
+            return
+        fi
+    done;
+
+    # if get failed by ldconfig, then just extract lib from ldd result
+    libname=`ldd ${BASE_DIR}/build/latest/bin/kudu 2>/dev/null | grep "lib${1}\.so"`
+    libname=`echo $libname | cut -f3 -d" "`
+    if echo "$libname" | grep -q "lib${2}\.so"; then
+        echo "$libname"
+    fi
+}
+
+function get_system_libname()
+{
+    libname=`ldd ${BASE_DIR}/build/latest/bin/kudu 2>/dev/null | grep "lib${1}\.so"`
+    libname=`echo $libname | cut -f1 -d" "`
+    echo "$libname"
+}
+
 function check_bit()
 {
     bit_mode=`getconf LONG_BIT`
@@ -130,6 +159,7 @@ copy_file ${BASE_DIR}/build/latest/bin/kudu-master ${PACK_DIR}/kudu_master
 copy_file ${BASE_DIR}/build/latest/bin/kudu-tserver ${PACK_DIR}/kudu_tablet_server
 copy_file ${BASE_DIR}/build/latest/bin/kudu ${PACK_DIR}/
 copy_file `get_stdcpp_lib $custom_gcc` ${PACK_DIR}/
+copy_file `get_system_lib crypto` ${PACK_DIR}/`get_system_libname crypto`
 copy_file ${BASE_DIR}/src/kudu/scripts/batch_operate_on_tables.sh ${PACK_DIR}/
 copy_file ${BASE_DIR}/src/kudu/scripts/falcon_screen.json ${PACK_DIR}/
 copy_file ${BASE_DIR}/src/kudu/scripts/falcon_screen.py ${PACK_DIR}/
@@ -137,7 +167,8 @@ copy_file ${BASE_DIR}/src/kudu/scripts/kudu_falcon_screen.sh ${PACK_DIR}/
 copy_file ${BASE_DIR}/src/kudu/scripts/minos_control_server.py ${PACK_DIR}/
 copy_file ${BASE_DIR}/src/kudu/scripts/cal_bill_daily.py ${PACK_DIR}/
 copy_file ${BASE_DIR}/src/kudu/scripts/kudu_utils.py ${PACK_DIR}/
-copy_file ${BASE_DIR}/src/kudu/scripts/start_local_kudu.sh ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/start_kudu.sh ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/stop_kudu.sh ${PACK_DIR}/
 copy_file ${BASE_DIR}/src/kudu/scripts/kudurc ${PACK_DIR}/
 copy_file -r ${BASE_DIR}/www ${PACK_DIR}/
 cd ${BASE_DIR}/build
diff --git a/src/kudu/scripts/falcon_screen.json b/src/kudu/scripts/falcon_screen.json
index b15125c..68911d9 100644
--- a/src/kudu/scripts/falcon_screen.json
+++ b/src/kudu/scripts/falcon_screen.json
@@ -1,3 +1,4 @@
+
 {
   "comments": [
     {
@@ -16,9 +17,7 @@
   ],
   "version": "20180827",
   "counter_templates": {
-      "full": [
-          "metric=kudu-tserver-health service=kudu cluster=${cluster.name} level=${level} v=4",
-          "metric=kudu-table-health service=kudu cluster=${cluster.name} level=${level} v=4",
+      "tablet_metrics": [
           "metric=all_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=alter_schema_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=average_diskrowset_height service=kudu cluster=${cluster.name} level=${level} v=4",
@@ -45,6 +44,7 @@
           "metric=key_file_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=key_file_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=leader_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=live_row_count service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=log_append_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=log_bytes_logged service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=log_cache_num_ops service=kudu cluster=${cluster.name} level=${level} v=4",
@@ -69,7 +69,6 @@
           "metric=op_prepare_run_time_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=ops_behind_leader service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=raft_term service=kudu cluster=${cluster.name} level=${level} v=4",
-          "metric=replica_count service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=rows_deleted service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=rows_inserted service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=rows_updated service=kudu cluster=${cluster.name} level=${level} v=4",
@@ -97,6 +96,19 @@
           "metric=write_op_duration_commit_wait_consistency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=write_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
+      "master_health": [
+          "metric=kudu-master-health service=kudu cluster=${cluster.name} level=host v=4"
+      ],
+      "tserver_health": [
+          "metric=kudu-tserver-health service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=merged_entities_count_of_tablet service=kudu cluster=${cluster.name} level=host v=4"
+      ],
+      "table_health": [
+          "metric=kudu-table-health service=kudu cluster=${cluster.name} level=table v=4"
+      ],
+      "host_table_distribution": [
+          "metric=merged_entities_count_of_tablet service=kudu cluster=${cluster.name} level=host_table v=4"
+      ],
       "table_ab": [
         "metric=all_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=alter_schema_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
@@ -200,9 +212,6 @@
         "metric=write_op_duration_commit_wait_consistency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=write_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
-      "replica_count" : [
-          "metric=replica_count service=kudu cluster=${cluster.name} level=${level} v=4"
-      ],
       "cluster" : [
           "metric=kudu.success service=kudu level=${level}",
           "metric=kudu.writeLatency service=kudu level=${level}",
@@ -218,6 +227,227 @@
         "metric=on_disk_size service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=on_disk_data_size service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
+      "server_master": [
+          "metric=block_cache_evictions service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_hits_caching service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_hits service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_inserts service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_lookups service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_misses_caching service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_misses service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_usage service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_blocks_open_reading service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_blocks_open_writing service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_blocks_created service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_blocks_deleted service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_bytes_read service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_bytes_written service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_disk_sync service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_readable_blocks service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_writable_blocks service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=cluster_replica_skew service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=code_cache_hits service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=code_cache_queries service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=cpu_stime service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=cpu_utime service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=data_dirs_failed service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=data_dirs_full service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_evictions service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_hits_caching service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_hits service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_inserts service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_lookups service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_misses_caching service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_misses service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_usage service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=generic_current_allocated_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=generic_heap_size service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=glog_error_messages service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=glog_info_messages service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=glog_warning_messages service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_BulkChangeConfig_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_ChangeConfig_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_GetConsensusState_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_GetLastOpId_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_GetNodeInstance_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_LeaderStepDown_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_RequestConsensusVote_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_RunLeaderElection_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_StartTabletCopy_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_UnsafeChangeConfig_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_UpdateConsensus_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_AlterTable_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_ChangeTServerState_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_ConnectToMaster_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_CreateTable_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_DeleteTable_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_GetMasterRegistration_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_GetTableLocations_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_GetTableSchema_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_GetTableStatistics_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_GetTabletLocations_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_IsAlterTableDone_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_IsCreateTableDone_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_ListMasters_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_ListTables_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_ListTabletServers_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_Ping_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_ReplaceTablet_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_ResetAuthzCache_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_master_MasterService_TSHeartbeat_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_CheckLeaks_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_DumpMemTrackers_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_FlushCoverage_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_GetFlags_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_GetStatus_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_ServerClock_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_SetFlag_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_SetServerWallClockForTests_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletCopyService_BeginTabletCopySession_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletCopyService_CheckSessionActive_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletCopyService_EndTabletCopySession_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletCopyService_FetchData_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=hybrid_clock_error service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=hybrid_clock_timestamp service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=involuntary_context_switches service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_blocks_under_management service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_bytes_under_management service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_containers service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_dead_containers_deleted service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_full_containers service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_holes_punched service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=op_apply_queue_length_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=op_apply_queue_time_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=op_apply_run_time_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=reactor_active_latency_us_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=reactor_load_percent_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=rpc_connections_accepted service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=rpc_incoming_queue_time_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=rpcs_queue_overflow service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=rpcs_timed_out_in_queue service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=spinlock_contention_time service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablet_copy_bytes_sent service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablet_copy_open_source_sessions service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tcmalloc_current_total_thread_cache_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tcmalloc_max_total_thread_cache_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tcmalloc_pageheap_free_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tcmalloc_pageheap_unmapped_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=threads_running service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=threads_started service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=voluntary_context_switches service=kudu cluster=${cluster.name} level=host v=4"
+      ],
+      "server_tserver": [
+          "metric=active_scanners service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_evictions service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_hits_caching service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_hits service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_inserts service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_lookups service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_misses_caching service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_misses service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_cache_usage service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_blocks_open_reading service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_blocks_open_writing service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_blocks_created service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_blocks_deleted service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_bytes_read service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_bytes_written service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_disk_sync service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_readable_blocks service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=block_manager_total_writable_blocks service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=code_cache_hits service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=code_cache_queries service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=cpu_stime service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=cpu_utime service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=data_dirs_failed service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=data_dirs_full service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_evictions service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_hits_caching service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_hits service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_inserts service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_lookups service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_misses_caching service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_misses service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=file_cache_usage service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=generic_current_allocated_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=generic_heap_size service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=glog_error_messages service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=glog_info_messages service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=glog_warning_messages service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_BulkChangeConfig_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_ChangeConfig_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_GetConsensusState_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_GetLastOpId_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_GetNodeInstance_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_LeaderStepDown_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_RequestConsensusVote_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_RunLeaderElection_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_StartTabletCopy_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_UnsafeChangeConfig_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_consensus_ConsensusService_UpdateConsensus_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_CheckLeaks_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_DumpMemTrackers_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_FlushCoverage_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_GetFlags_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_GetStatus_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_ServerClock_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_SetFlag_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_server_GenericService_SetServerWallClockForTests_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletCopyService_BeginTabletCopySession_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletCopyService_CheckSessionActive_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletCopyService_EndTabletCopySession_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletCopyService_FetchData_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerAdminService_AlterSchema_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerAdminService_CreateTablet_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerAdminService_DeleteTablet_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerService_Checksum_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerService_ListTablets_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerService_Ping_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerService_ScannerKeepAlive_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerService_Scan_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerService_SplitKeyRange_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=handler_latency_kudu_tserver_TabletServerService_Write_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=hybrid_clock_error service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=hybrid_clock_timestamp service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=involuntary_context_switches service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_blocks_under_management service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_bytes_under_management service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_containers service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_dead_containers_deleted service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_full_containers service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=log_block_manager_holes_punched service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=op_apply_queue_length_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=op_apply_queue_time_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=op_apply_run_time_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=reactor_active_latency_us_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=reactor_load_percent_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=rpc_connections_accepted service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=rpc_incoming_queue_time_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=rpcs_queue_overflow service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=rpcs_timed_out_in_queue service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=scanner_duration_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=scanners_expired service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=spinlock_contention_time service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablet_copy_bytes_fetched service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablet_copy_bytes_sent service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablet_copy_open_client_sessions service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablet_copy_open_source_sessions service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablets_num_bootstrapping service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablets_num_failed service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablets_num_initialized service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablets_num_not_initialized service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablets_num_running service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablets_num_shutdown service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablets_num_stopped service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tablets_num_stopping service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tcmalloc_current_total_thread_cache_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tcmalloc_max_total_thread_cache_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tcmalloc_pageheap_free_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=tcmalloc_pageheap_unmapped_bytes service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=threads_running service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=threads_started service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=voluntary_context_switches service=kudu cluster=${cluster.name} level=host v=4"
+      ],
       "sys" : [
           "cpu.busy",
           "load.15min",
@@ -535,6 +765,60 @@
       ]
     },
     {
+      "screen": "${cluster.name} [nodes_healthy]",
+      "graphs": [
+        {
+          "title": "tserver健康状态",
+          "endpoints": ["${for.each.tserver}"],
+          "counters": {
+              "level": "host",
+              "template": "tserver_health"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        },
+        {
+          "title": "master健康状态",
+          "endpoints": ["${for.each.master}"],
+          "counters": {
+              "level": "host",
+              "template": "master_health"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        },
+        {
+          "title": "table健康状态",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_health"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [host_table_distribution]",
+      "graphs": [
+        {
+          "title": "tserver各表tablet分布",
+          "endpoints": ["${for.each.tserver} ${for.each.table}"],
+          "counters": {
+              "level": "host",
+              "template": "host_table_distribution"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
       "screen": "${cluster.name} [tserver]",
       "graphs": [
         {
@@ -542,7 +826,55 @@
           "endpoints": ["${for.each.tserver}"],
           "counters": {
               "level": "host",
-              "template": "full"
+              "template": "tablet_metrics"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [server_tserver]",
+      "graphs": [
+        {
+          "title": "tserver Kudu服务指标",
+          "endpoints": ["${for.each.tserver}"],
+          "counters": {
+              "level": "host",
+              "template": "server_tserver"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [server_master]",
+      "graphs": [
+        {
+          "title": "master Kudu服务指标",
+          "endpoints": ["${for.each.master}"],
+          "counters": {
+              "level": "host",
+              "template": "server_master"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [master_catalog]",
+      "graphs": [
+        {
+          "title": "master catalog表指标",
+          "endpoints": ["${for.each.master}"],
+          "counters": {
+              "level": "host",
+              "template": "tablet_metrics"
           },
           "graph_type": "h",
           "method": "",
diff --git a/src/kudu/scripts/falcon_screen.py b/src/kudu/scripts/falcon_screen.py
index 26f330a..1695ca5 100755
--- a/src/kudu/scripts/falcon_screen.py
+++ b/src/kudu/scripts/falcon_screen.py
@@ -25,6 +25,11 @@ KUDU_CLUSTER_ID = 37613
 KUDU_TABLES_ID = 37638
 KUDU_TSERVER_ID = 37639
 KUDU_SYS_ID = 37640
+KUDU_SERVER_TSERVER = 40185
+KUDU_SERVER_MASTER = 40186
+KUDU_MASTER_CATALOG = 40207
+KUDU_NODES_HEALTHY = 40229
+KUDU_TSERVER_TABLET_DISTRIBUTION = 40258
 screenIdList = {
     KUDU_CLUSTER_ID: "[cluster]",
     KUDU_TABLES_ID: [
@@ -41,7 +46,13 @@ screenIdList = {
         "[metrics_u]",
         "[metrics_tw]"],
     KUDU_TSERVER_ID: "[tserver]",
-    KUDU_SYS_ID: "[server-sys]"}
+    KUDU_SYS_ID: "[server-sys]",
+    KUDU_SERVER_TSERVER: "[server_tserver]",
+    KUDU_SERVER_MASTER: "[server_master]",
+    KUDU_MASTER_CATALOG: "[master_catalog]",
+    KUDU_NODES_HEALTHY: "[nodes_healthy]",
+    KUDU_TSERVER_TABLET_DISTRIBUTION: "[host_table_distribution]"
+}
 # kuduScreenId = 351
 sessionId = ""
 metaPort = ""
@@ -185,7 +196,6 @@ def parse_lines(file_name):
 # return: screenConfigs
 def prepare_screen_config(
         clusterName,
-        templateName,
         screenTemplateFile,
         tableListFile,
         masterListFile,
@@ -276,7 +286,7 @@ def prepare_screen_config(
                     "ERROR: bad json: [details][%s][graphs][%s]: [counters] should be provided as non-empty list/dict" %
                     (screen, title))
                 sys.exit(1)
-            for counter in templateJson[counters["template"] if counters.has_key("template") else templateName]:
+            for counter in templateJson[counters["template"]]:
                 newCounters.append(
                     counter.replace(
                         "${cluster.name}",
@@ -550,31 +560,30 @@ if __name__ == '__main__':
             sys.argv[0])
         sys.exit(1)
 
-    if len(sys.argv) != 7:
+    if len(sys.argv) != 6:
         print(
-            "USAGE: python %s <cluster_name> <template_name> <screen_template_file> <master_list_file> <tserver_list_file> <table_list_file>" %
+            "USAGE: python %s <cluster_name> <screen_template_file> <master_list_file> <tserver_list_file> <table_list_file>" %
             sys.argv[0])
         sys.exit(1)
 
     clusterName = sys.argv[1]
-    templateName = sys.argv[2]
-    screenTemplateFile = sys.argv[3]
-    masterListFile = sys.argv[4]
-    tserverListFile = sys.argv[5]
-    tableListFile = sys.argv[6]
+    screenTemplateFile = sys.argv[2]
+    masterListFile = sys.argv[3]
+    tserverListFile = sys.argv[4]
+    tableListFile = sys.argv[5]
+
+    screenConfigs = prepare_screen_config(
+        clusterName,
+        screenTemplateFile,
+        tableListFile,
+        masterListFile,
+        tserverListFile)
 
     login()
 
     for scrid, scrNames in screenIdList.items():
         oldKuduScreens = get_kudu_screens(scrid)
         oldScreenName2Id = {}
-        screenConfigs = prepare_screen_config(
-            clusterName,
-            templateName,
-            screenTemplateFile,
-            tableListFile,
-            masterListFile,
-            tserverListFile)
         for oldScreen in oldKuduScreens:
             oldScreenName2Id[oldScreen['name']] = oldScreen['id']
         if scrid == KUDU_TABLES_ID:
diff --git a/src/kudu/scripts/kudu_falcon_screen.sh b/src/kudu/scripts/kudu_falcon_screen.sh
index 046352b..7dd2f90 100755
--- a/src/kudu/scripts/kudu_falcon_screen.sh
+++ b/src/kudu/scripts/kudu_falcon_screen.sh
@@ -3,11 +3,6 @@
 PID=$$
 BASE_DIR="$( cd "$( dirname "$0" )" && pwd )"
 KUDU=${KUDU_HOME}/kudu
-COLLECTOR=${KUDU_HOME}/kudu_collector
-if [[ ! -f ${KUDU} || ! -f ${COLLECTOR} ]]; then
-  echo "ERROR: ${KUDU} or ${COLLECTOR} not found"
-  exit 1
-fi
 KUDURC=${KUDU_CONFIG}/kudurc
 if [[ ! -f ${KUDURC} ]]; then
   echo "ERROR: ${KUDURC} not found"
@@ -17,10 +12,8 @@ fi
 function usage() {
 cat << EOF
 This tool is for update falcon screen for specified kudu cluster.
-USAGE: $0 <cluster_name> [table_count] [metrics_template]
+USAGE: $0 <cluster_name>
        cluster_name       Cluster name operated on, should be configurated in $KUDU_CONFIG/kudurc
-       table_count        An indicator of how many tables will be monitored, actual monitored table count is in range [table_count, 3*table_count]
-       metrics_template   Which metric template will be used, 'simple' or 'full'
 EOF
 }
 
@@ -31,28 +24,10 @@ then
 fi
 
 CLUSTER=$1
-TABLE_COUNT=9999
-if [[ $# -ge 2 ]]
-then
-  TABLE_COUNT=$2
-fi
-
-TEMPLATE_NAME='full'
-if [[ $# -ge 3 ]]
-then
-  TEMPLATE_NAME=$3
-fi
-if [[ "${TEMPLATE_NAME}"x != "simple"x && "${TEMPLATE_NAME}"x != "full"x ]]
-then
-  usage
-  exit 1
-fi
 
 echo "UID: ${UID}"
 echo "PID: ${PID}"
 echo "cluster: ${CLUSTER}"
-echo "top n table: ${TABLE_COUNT}"
-echo "metric template: ${TEMPLATE_NAME}"
 echo "Start time: `date`"
 ALL_START_TIME=$((`date +%s`))
 echo
@@ -83,29 +58,11 @@ if [[ ${TSERVER_COUNT} -eq 0 ]]; then
     exit 1
 fi
 
-function parse_yaml() {
-  python -c "import yaml;print(yaml.load(open('$1').read(), Loader=yaml.FullLoader)['clusters_info']['$2']['master_addresses'])"
-}
-MASTERS=$(parse_yaml ${KUDURC} ${CLUSTER})
-
 # get table list
-${COLLECTOR} -collector_master_addrs=${MASTERS} -collector_cluster_name=${CLUSTER} -collector_report_method=local -collector_metrics=bytes_flushed,on_disk_size,scanner_bytes_returned -log_dir=./log > /tmp/${UID}.${PID}.kudu.metric_table_value
-if [[ $? -ne 0 ]]; then
-    echo "ERROR: ${COLLECTOR} execute failed"
-    exit 1
-fi
-
-cat /tmp/${UID}.${PID}.kudu.metric_table_value | egrep "^table bytes_flushed " | sort -rnk4 | head -n ${TABLE_COUNT} | awk '{print $3}' > /tmp/${UID}.${PID}.kudu.top.bytes_flushed
-cat /tmp/${UID}.${PID}.kudu.metric_table_value | egrep "^table on_disk_size " | sort -rnk4 | head -n ${TABLE_COUNT} | awk '{print $3}' > /tmp/${UID}.${PID}.kudu.top.on_disk_size
-cat /tmp/${UID}.${PID}.kudu.metric_table_value | egrep "^table scanner_bytes_returned " | sort -rnk4 | head -n ${TABLE_COUNT} | awk '{print $3}' > /tmp/${UID}.${PID}.kudu.top.scanner_bytes_returned
-cat /tmp/${UID}.${PID}.kudu.top.* | sort -n | uniq > /tmp/${UID}.${PID}.kudu.table.list
+${KUDU} table list @${CLUSTER} | sort -n &>/tmp/${UID}.${PID}.kudu.table.list
 echo "total `wc -l /tmp/${UID}.${PID}.kudu.table.list | awk '{print $1}'` tables to monitor"
-echo -e "\033[32m Please set the following one line to the kudu collector's \`collector_attributes\` argument manually\033[0m"
-echo -n "table_name:"
-awk BEGIN{RS=EOF}'{gsub(/\n/,",");print}' /tmp/${UID}.${PID}.kudu.table.list
-echo ""
 
-python ${BASE_DIR}/falcon_screen.py ${CLUSTER} ${TEMPLATE_NAME} ${BASE_DIR}/falcon_screen.json /tmp/${UID}.${PID}.kudu.master.list /tmp/${UID}.${PID}.kudu.tserver.list /tmp/${UID}.${PID}.kudu.table.list
+python ${BASE_DIR}/falcon_screen.py ${CLUSTER} ${BASE_DIR}/falcon_screen.json /tmp/${UID}.${PID}.kudu.master.list /tmp/${UID}.${PID}.kudu.tserver.list /tmp/${UID}.${PID}.kudu.table.list
 if [[ $? -ne 0 ]]; then
     echo "ERROR: falcon screen operate failed"
     exit 1
diff --git a/src/kudu/util/metrics.h b/src/kudu/util/metrics.h
index ab31113..ea010b2 100644
--- a/src/kudu/util/metrics.h
+++ b/src/kudu/util/metrics.h
@@ -1239,6 +1239,7 @@ class FunctionGauge : public Gauge {
   // This should be used during destruction. If you want a settable
   // Gauge, use a normal Gauge instead of a FunctionGauge.
   void DetachToConstant(T v) {
+    UpdateModificationEpoch();
     std::lock_guard<simple_spinlock> l(lock_);
     function_ = [v]() { return v; };
   }


[kudu] 02/23: [www] convert human-readable int to int when compare numeric strings

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 4addeeec38456f8a2b681783397011d3dd854f5d
Author: zhangyifan27 <ch...@163.com>
AuthorDate: Wed Apr 22 19:25:42 2020 +0800

    [www] convert human-readable int to int when compare numeric strings
    
    The /tables page show `tablet_count` as a HumanReadableInt, when sort
    tables by tablet_count, human-readable int should be converted to int.
    
    Change-Id: Ic7e832beb0dfdf5c2162bf0e9faeb3d0d0d737cd
    Reviewed-on: http://gerrit.cloudera.org:8080/15784
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 www/kudu.js | 36 +++++++++++++++++++++++++++++++++---
 1 file changed, 33 insertions(+), 3 deletions(-)

diff --git a/www/kudu.js b/www/kudu.js
index 9f02320..27d5ebc 100644
--- a/www/kudu.js
+++ b/www/kudu.js
@@ -45,6 +45,36 @@ function toNumBytes(humanReadableBytes) {
   return val;
 }
 
+// Converts a human-readable numeric strings like '1.23k' or '985.32M' to int.
+// Returns -1 if there's some failure.
+function toInt(humanReadableNum) {
+  len = humanReadableNum.length;
+  if (len < 1) {
+    return -1;
+  }
+  end = humanReadableNum[len - 1];
+  val = parseFloat(humanReadableNum);
+  if (isNaN(val)) {
+    return -1;
+  }
+  if (end == 'k') {
+    val *= 1e3;
+  } else if (end == 'M') {
+    val *= 1e6;
+  } else if (end == 'B') {
+    val *= 1e9;
+  } else if (end == 'T') {
+    val *= 1e12; // Number bigger than 1E15 use scientific notation.
+  } else if (isNaN(end)) {
+    // Not a number.
+    return -1;
+  }
+  if (val < 0) {
+    return parseInt(val - 0.5);
+  }
+  return parseInt(val + 0.5);
+}
+
 // A comparison function for human-readable byte strings.
 function bytesSorter(left, right) {
   if (right.length == 0 && left.length == 0) {
@@ -80,10 +110,10 @@ function floatsSorter(left, right) {
   return 0;
 }
 
-// Converts numeric strings to numbers and then compares them.
+// Converts human-readable numeric strings to numbers and then compares them.
 function numericStringsSorter(left, right) {
-  left_num = parseInt(left, 10);
-  right_num = parseInt(right, 10);
+  left_num = toInt(left);
+  right_num = toInt(right);
   if (left_num < right_num) {
     return -1;
   }


[kudu] 23/23: [version] Bump version to 1.12.0-mdh1.0.0

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 4c2c075886bfc44862412c7d0fe6bd91e6f0b14c
Author: zhangyifan8 <zh...@xiaomi.com>
AuthorDate: Fri Aug 14 15:08:40 2020 +0800

    [version] Bump version to 1.12.0-mdh1.0.0
---
 version.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/version.txt b/version.txt
index c19d3ed..64f4ad6 100644
--- a/version.txt
+++ b/version.txt
@@ -1 +1 @@
-1.12.0-mdh1.0.0-SNAPSHOT
+1.12.0-mdh1.0.0


[kudu] 17/23: [collector] modify some APIs

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 7caa5a7dbebdd3d9386063365f2814ba4f276ed7
Author: zhangyifan8 <zh...@xiaomi.com>
AuthorDate: Thu May 21 11:52:57 2020 +0800

    [collector] modify some APIs
    
    Modify some APIs for collector according to the following patches:
    1. thread: simplify Thread::Create API https://gerrit.cloudera.org/c/15448/
    2. threadpool: simplify Submit API  threadpool: simplify Submit API
       https://gerrit.cloudera.org/c/15401/
    3. [master/tserver] non-zero code from main() instead of crashing
       https://gerrit.cloudera.org/c/14908/
    4. [tools] Support running the master and tablet server via the kudu binary
       https://gerrit.cloudera.org/c/12517/
---
 src/kudu/collector/cluster_rebalancer.cc |  5 +++--
 src/kudu/collector/collector.cc          |  5 +++--
 src/kudu/collector/collector_main.cc     | 10 ++++------
 src/kudu/collector/falcon_reporter.cc    |  2 +-
 src/kudu/collector/metrics_collector.cc  | 15 ++++++++++-----
 src/kudu/collector/nodes_checker.cc      |  5 +++--
 src/kudu/collector/service_monitor.cc    |  5 +++--
 src/kudu/master/master.cc                |  4 ++--
 8 files changed, 29 insertions(+), 22 deletions(-)

diff --git a/src/kudu/collector/cluster_rebalancer.cc b/src/kudu/collector/cluster_rebalancer.cc
index b46baa4..c33d1a7 100644
--- a/src/kudu/collector/cluster_rebalancer.cc
+++ b/src/kudu/collector/cluster_rebalancer.cc
@@ -101,8 +101,9 @@ string ClusterRebalancer::ToString() const {
 }
 
 Status ClusterRebalancer::StartClusterRebalancerThread() {
-  return Thread::Create("server", "cluster-rebalancer", &ClusterRebalancer::ClusterRebalancerThread,
-                        this, &cluster_rebalancer_thread_);
+  return Thread::Create("server", "cluster-rebalancer",
+                        [this]() { this->ClusterRebalancerThread(); },
+                        &cluster_rebalancer_thread_);
 }
 
 void ClusterRebalancer::ClusterRebalancerThread() {
diff --git a/src/kudu/collector/collector.cc b/src/kudu/collector/collector.cc
index 3501300..9eb5f3d 100644
--- a/src/kudu/collector/collector.cc
+++ b/src/kudu/collector/collector.cc
@@ -148,8 +148,9 @@ Status Collector::StartExcessLogFileDeleterThread() {
     RETURN_NOT_OK_PREPEND(DeleteExcessLogFiles(Env::Default()),
                           "Unable to delete excess log files");
   }
-  return Thread::Create("server", "excess-log-deleter", &Collector::ExcessLogFileDeleterThread,
-                        this, &excess_log_deleter_thread_);
+  return Thread::Create("server", "excess-log-deleter",
+                        [this]() { this->ExcessLogFileDeleterThread(); },
+                        &excess_log_deleter_thread_);
 }
 
 void Collector::ExcessLogFileDeleterThread() {
diff --git a/src/kudu/collector/collector_main.cc b/src/kudu/collector/collector_main.cc
index afc0768..614c288 100644
--- a/src/kudu/collector/collector_main.cc
+++ b/src/kudu/collector/collector_main.cc
@@ -33,16 +33,14 @@ namespace kudu {
 namespace collector {
 
 static int CollectorMain(int argc, char** argv) {
-  InitKuduOrDie();
-
-  GFlagsMap default_flags = GetFlagsMap();
+  RETURN_MAIN_NOT_OK(InitKudu(), "InitKudu() failed", 1);
 
   ParseCommandLineFlags(&argc, &argv, true);
   if (argc != 1) {
     std::cerr << "usage: " << argv[0] << std::endl;
     return 1;
   }
-  std::string nondefault_flags = GetNonDefaultFlags(default_flags);
+  std::string nondefault_flags = GetNonDefaultFlags();
   InitGoogleLoggingSafe(argv[0]);
 
   LOG(INFO) << "Collector non-default flags:\n"
@@ -52,10 +50,10 @@ static int CollectorMain(int argc, char** argv) {
 
   Collector collector;
   LOG(INFO) << "Initializing collector...";
-  CHECK_OK(collector.Init());
+  RETURN_MAIN_NOT_OK(collector.Init(), "Init() failed", 3);
 
   LOG(INFO) << "Starting collector...";
-  CHECK_OK(collector.Start());
+  RETURN_MAIN_NOT_OK(collector.Start(), "Start() failed", 4);
 
   LOG(INFO) << "Collector successfully started.";
   while (!RunOnceMode()) {
diff --git a/src/kudu/collector/falcon_reporter.cc b/src/kudu/collector/falcon_reporter.cc
index efe387b..35d8af4 100644
--- a/src/kudu/collector/falcon_reporter.cc
+++ b/src/kudu/collector/falcon_reporter.cc
@@ -152,7 +152,7 @@ Status FalconReporter::StartFalconPusherThreadPool() {
       .set_idle_timeout(MonoDelta::FromMilliseconds(1))
       .Build(&pusher_thread_pool_));
   for (int i = 0; i < FLAGS_collector_falcon_pusher_count; ++i) {
-    RETURN_NOT_OK(pusher_thread_pool_->SubmitFunc(std::bind(&FalconReporter::FalconPusher,
+    RETURN_NOT_OK(pusher_thread_pool_->Submit(std::bind(&FalconReporter::FalconPusher,
                                                             this)));
   }
 
diff --git a/src/kudu/collector/metrics_collector.cc b/src/kudu/collector/metrics_collector.cc
index 6d09a0d..a79a609 100644
--- a/src/kudu/collector/metrics_collector.cc
+++ b/src/kudu/collector/metrics_collector.cc
@@ -142,8 +142,9 @@ string MetricsCollector::ToString() const {
 }
 
 Status MetricsCollector::StartMetricCollectorThread() {
-  return Thread::Create("server", "metric-collector", &MetricsCollector::MetricCollectorThread,
-                        this, &metric_collector_thread_);
+  return Thread::Create("server", "metric-collector",
+                        [this]() { this->MetricCollectorThread(); },
+                        &metric_collector_thread_);
 }
 
 void MetricsCollector::MetricCollectorThread() {
@@ -334,7 +335,7 @@ Status MetricsCollector::CollectAndReportMasterMetrics() {
   RETURN_NOT_OK(UpdateThreadPool(std::max(host_metric_collector_thread_pool_->num_threads(),
                                           static_cast<int32_t>(master_http_addrs.size()))));
   for (int i = 0; i < master_http_addrs.size(); ++i) {
-    RETURN_NOT_OK(host_metric_collector_thread_pool_->SubmitFunc(
+    RETURN_NOT_OK(host_metric_collector_thread_pool_->Submit(
       std::bind(&MetricsCollector::CollectAndReportHostLevelMetrics,
                 this,
                 NodeType::kMaster,
@@ -373,7 +374,7 @@ Status MetricsCollector::CollectAndReportTServerMetrics() {
   vector<TablesMetrics> hosts_metrics_by_table_name(tserver_http_addrs.size());
   vector<TablesHistMetrics> hosts_hist_metrics_by_table_name(tserver_http_addrs.size());
   for (int i = 0; i < tserver_http_addrs.size(); ++i) {
-    RETURN_NOT_OK(host_metric_collector_thread_pool_->SubmitFunc(
+    RETURN_NOT_OK(host_metric_collector_thread_pool_->Submit(
       std::bind(&MetricsCollector::CollectAndReportHostLevelMetrics,
                 this,
                 NodeType::kTServer,
@@ -654,8 +655,12 @@ Status MetricsCollector::ParseEntityMetrics(const JsonReader& r,
         case rapidjson::Type::kNumberType:
           CHECK_OK(GetNumberMetricValue(val, name, &value));
           break;
+        case rapidjson::Type::kFalseType:
+        case rapidjson::Type::kTrueType:
+          // Do not process true/false type.
+          break;
         default:
-          LOG(FATAL) << "Unknown type, metrics name: " << name;
+          LOG(WARNING) << "Unknown type, metrics name: " << name;
       }
 
       EmplaceOrDie(kv_metrics, std::make_pair(name, value));
diff --git a/src/kudu/collector/nodes_checker.cc b/src/kudu/collector/nodes_checker.cc
index 036d72a..15a6d22 100644
--- a/src/kudu/collector/nodes_checker.cc
+++ b/src/kudu/collector/nodes_checker.cc
@@ -131,8 +131,9 @@ string NodesChecker::GetFirstTServer() {
 }
 
 Status NodesChecker::StartNodesCheckerThread() {
-  return Thread::Create("collector", "nodes-checker", &NodesChecker::NodesCheckerThread,
-                        this, &nodes_checker_thread_);
+  return Thread::Create("collector", "nodes-checker",
+                        [this]() { this->NodesCheckerThread(); },
+                        &nodes_checker_thread_);
 }
 
 void NodesChecker::NodesCheckerThread() {
diff --git a/src/kudu/collector/service_monitor.cc b/src/kudu/collector/service_monitor.cc
index e97fb79..70ea57a 100644
--- a/src/kudu/collector/service_monitor.cc
+++ b/src/kudu/collector/service_monitor.cc
@@ -342,8 +342,9 @@ Status ServiceMonitor::CallLeaderStepDown(const string& tablet_id, const string&
 }
 
 Status ServiceMonitor::StartServiceMonitorThread() {
-  return Thread::Create("collector", "nodes-checker", &ServiceMonitor::ServiceMonitorThread,
-                        this, &service_monitor_thread_);
+  return Thread::Create("collector", "nodes-checker",
+                        [this]() { this->ServiceMonitorThread(); },
+                        &service_monitor_thread_);
 }
 
 void ServiceMonitor::ServiceMonitorThread() {
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index 32d6e8d..0fc7718 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -312,8 +312,8 @@ void Master::CrashMasterOnCFileCorruption(const string& tablet_id) {
 
 Status Master::StartOutdatedReservedTablesDeleterThread() {
   return Thread::Create("master", "outdated-reserved-tables-deleter",
-                        &Master::OutdatedReservedTablesDeleterThread,
-                        this, &outdated_reserved_tables_deleter_thread_);
+                        [this]() { this->OutdatedReservedTablesDeleterThread(); },
+                        &outdated_reserved_tables_deleter_thread_);
 }
 
 void Master::OutdatedReservedTablesDeleterThread() {


[kudu] 19/23: [script] fix minos_control_server.py for newer versions

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit b42d25b4dc5c41d3a379f0a4324910612472660c
Author: zhangyifan8 <zh...@xiaomi.com>
AuthorDate: Fri Jun 5 18:12:01 2020 +0800

    [script] fix minos_control_server.py for newer versions
    
    According to https://kudu.apache.org/docs/administration.html#rolling_restart,
    fix minos_control_server.py. And also add some new metrics introduced in 1.12.0.
---
 src/kudu/scripts/falcon_screen.json      | 30 +++++++++++++-
 src/kudu/scripts/falcon_screen.py        |  1 +
 src/kudu/scripts/minos_control_server.py | 69 ++++++++++++++++++++++++++++----
 3 files changed, 90 insertions(+), 10 deletions(-)

diff --git a/src/kudu/scripts/falcon_screen.json b/src/kudu/scripts/falcon_screen.json
index 26b7acd..8f467e5 100644
--- a/src/kudu/scripts/falcon_screen.json
+++ b/src/kudu/scripts/falcon_screen.json
@@ -44,6 +44,8 @@
           "metric=key_file_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=key_file_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=leader_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=last_read_elapsed_time service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=last_write_elapsed_time service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=live_row_count service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=log_append_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=log_bytes_logged service=kudu cluster=${cluster.name} level=${level} v=4",
@@ -84,6 +86,7 @@
           "metric=state service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=tablet_active_scanners service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=time_since_last_leader_heartbeat service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=transaction_memory_limit_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=transaction_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=undo_delta_block_estimated_retained_bytes service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=undo_delta_block_gc_bytes_deleted service=kudu cluster=${cluster.name} level=${level} v=4",
@@ -173,7 +176,12 @@
         "metric=kudu-table-health service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "table_l": [
+        "metric=last_read_elapsed_time service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=last_write_elapsed_time service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=leader_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=live_row_count service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_log": [
         "metric=log_append_latency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_append_latency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_append_latency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
@@ -207,8 +215,7 @@
         "metric=log_sync_latency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_sync_latency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_sync_latency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
-        "metric=log_sync_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
-        "metric=live_row_count service=kudu cluster=${cluster.name} level=${level} v=4"
+        "metric=log_sync_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "table_mn": [
         "metric=majority_done_ops service=kudu cluster=${cluster.name} level=${level} v=4",
@@ -276,6 +283,7 @@
       "table_tw": [
         "metric=tablet_active_scanners service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=time_since_last_leader_heartbeat service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=transaction_memory_limit_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=transaction_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=write_op_duration_client_propagated_consistency_mean service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=write_op_duration_client_propagated_consistency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
@@ -401,6 +409,7 @@
           "metric=rpcs_queue_overflow service=kudu cluster=${cluster.name} level=host v=4",
           "metric=rpcs_timed_out_in_queue service=kudu cluster=${cluster.name} level=host v=4",
           "metric=spinlock_contention_time service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=sys_catalog_oversized_write_requests service=kudu cluster=${cluster.name} level=host v=4",
           "metric=tablet_copy_bytes_sent service=kudu cluster=${cluster.name} level=host v=4",
           "metric=tablet_copy_open_source_sessions service=kudu cluster=${cluster.name} level=host v=4",
           "metric=tcmalloc_current_total_thread_cache_bytes service=kudu cluster=${cluster.name} level=host v=4",
@@ -491,6 +500,7 @@
           "metric=log_block_manager_dead_containers_deleted service=kudu cluster=${cluster.name} level=host v=4",
           "metric=log_block_manager_full_containers service=kudu cluster=${cluster.name} level=host v=4",
           "metric=log_block_manager_holes_punched service=kudu cluster=${cluster.name} level=host v=4",
+          "metric=num_raft_leaders service=kudu cluster=${cluster.name} level=host v=4",
           "metric=op_apply_queue_length_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
           "metric=op_apply_queue_time_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
           "metric=op_apply_run_time_percentile_99 service=kudu cluster=${cluster.name} level=host v=4",
@@ -744,6 +754,22 @@
       ]
     },
     {
+      "screen": "${cluster.name} [metrics_log]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_log"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
       "screen": "${cluster.name} [metrics_mn]",
       "graphs": [
         {
diff --git a/src/kudu/scripts/falcon_screen.py b/src/kudu/scripts/falcon_screen.py
index 1695ca5..000d05f 100755
--- a/src/kudu/scripts/falcon_screen.py
+++ b/src/kudu/scripts/falcon_screen.py
@@ -39,6 +39,7 @@ screenIdList = {
         "[metrics_f]",
         "[metrics_ghijk]",
         "[metrics_l]",
+        "[metrics_log]",
         "[metrics_mn]",
         "[metrics_o]",
         "[metrics_r]",
diff --git a/src/kudu/scripts/minos_control_server.py b/src/kudu/scripts/minos_control_server.py
index 875802c..e35bf33 100755
--- a/src/kudu/scripts/minos_control_server.py
+++ b/src/kudu/scripts/minos_control_server.py
@@ -53,7 +53,7 @@ def get_host(host_port):
 
 def is_cluster_health():
     status, output = commands.getstatusoutput('${KUDU_HOME}/kudu cluster ksck @%s -consensus=false'
-                                              ' -ksck_format=json_compact -color=never'
+                                              ' -ksck_format=json_compact -quiescing_info=false -color=never'
                                               ' -sections=MASTER_SUMMARIES,TSERVER_SUMMARIES,TABLE_SUMMARIES'
                                               ' 2>/dev/null'
                                               % cluster)
@@ -140,6 +140,22 @@ def get_tablet_server_info(hostname, tservers_info):
     return rpc_address, uuid
 
 
+def get_cluster_version():
+    version_info = dict()
+    version = ''
+    min_version = '9.9.9'
+    status, output = commands.getstatusoutput('${KUDU_HOME}/kudu cluster ksck @%s -sections=VERSION_SUMMARIES '
+                                              '-quiescing_info=false -ksck_format=json_compact'
+                                              % cluster)
+    exit_if_failed(status, output)
+    version_info = json.loads(output)
+    for item in version_info['version_summaries']:
+        version = item['version']
+        if version < min_version:
+            min_version = version
+    return min_version
+
+
 def set_flag(rpc_address, seconds):
     cmd = ('${KUDU_HOME}/kudu tserver set_flag %s follower_unavailable_considered_failed_sec %s'
            % (rpc_address, seconds))
@@ -147,13 +163,42 @@ def set_flag(rpc_address, seconds):
     exit_if_failed(status, output)
 
 
+def maintain_tserver(op_type, ts_uuid):
+    cmd = ('${KUDU_HOME}/kudu tserver state %s @%s %s'
+           % (op_type, cluster, ts_uuid))
+    status, output = commands.getstatusoutput(cmd)
+    exit_if_failed(status, output)
+
+
+def wait_tserver_quiesce(rpc_address):
+    print(time_header() + 'Start to quiesce tserver ' + rpc_address)
+    cmd = ('${KUDU_HOME}/kudu tserver quiesce start %s -error_if_not_fully_quiesced' % (rpc_address))
+    is_quiesced = False
+    while not is_quiesced:
+        status, output = commands.getstatusoutput(cmd)
+        if status == 0:
+            print(time_header() + 'Tablet server is fully quiesced.')
+            is_quiesced = True
+        else:
+            print(time_header() + output)
+            time.sleep(1)
+
+
 def rebalance_cluster(blacklist_tserver_uuid):
     ignored_tservers_uuid = set()
     for node in known_unhealth_nodes:
         rpc_address, uuid = get_tablet_server_info(node, tservers_info)
         ignored_tservers_uuid.add(uuid)
-    cmd = ('${KUDU_HOME}/kudu cluster rebalance @%s -blacklist_tservers=%s -ignored_tservers=%s'
-           % (cluster, blacklist_tserver_uuid, str(','.join(ignored_tservers_uuid))))
+    if blacklist_tserver_uuid == '':
+        cmd = ('${KUDU_HOME}/kudu cluster rebalance @%s -ignored_tservers=%s -quiescing_info=false'
+               % (cluster, str(','.join(ignored_tservers_uuid))))
+    elif version < '1.11':
+        cmd = ('${KUDU_HOME}/kudu cluster rebalance @%s -blacklist_tservers=%s -ignored_tservers=%s -quiescing_info=false'
+               % (cluster, blacklist_tserver_uuid, str(','.join(ignored_tservers_uuid))))
+    else:
+        ignored_tservers_uuid.add(blacklist_tserver_uuid)
+        cmd = ('${KUDU_HOME}/kudu cluster rebalance @%s -ignored_tservers=%s -move_replicas_from_ignored_tservers'
+               % (cluster, str(','.join(ignored_tservers_uuid))))
     p = subprocess.Popen(cmd, stdout = subprocess.PIPE, shell=True)
     for line in iter(p.stdout.readline, b''):
         print line
@@ -184,8 +229,10 @@ check_parameter('You will rebalance cluster after operation: %s? (y/n)', rebalan
 
 tservers_info = get_tservers_info()
 wait_cluster_health()
+version = get_cluster_version()
+print('The cluster version(before rolling_update) is ' + version)
 
-if 'tablet_server' in job and operate in ['restart', 'rolling_update']:
+if version < '1.11' and 'tablet_server' in job and operate in ['restart', 'rolling_update']:
     for tserver in tservers_info:
         set_flag(tserver['rpc-addresses'], 7200)
 
@@ -195,13 +242,16 @@ for task in tasks:
         exit()
 
     if 'tablet_server' in job:
-        cmd = ('%s/deploy show kudu %s --job %s --task %d'
-          % (minos_client_path, cluster, job, task))
+        cmd = ('%s/deploy show kudu %s --job %s --task %d' % (minos_client_path, cluster, job, task))
         status, output = commands.getstatusoutput(cmd)
         exit_if_failed(status, output)
         print(output)
         hostname = parse_node_from_minos_output(output, job)
         rpc_address, uuid = get_tablet_server_info(hostname, tservers_info)
+        if version > '1.11':
+            maintain_tserver("enter_maintenance", uuid)
+            if version > '1.12':
+                wait_tserver_quiesce(rpc_address)
         if operate == 'stop':
             # migrate replicas on tserver
             rebalance_cluster(uuid)
@@ -218,12 +268,15 @@ for task in tasks:
     wait_cluster_health()
 
     if 'tablet_server' in job and operate in ['restart', 'rolling_update']:
-        set_flag(rpc_address, 7200)
+        if version < '1.11':
+            set_flag(rpc_address, 7200)
+        else:
+            maintain_tserver("exit_maintenance", uuid)
 
     print(time_header() + '==========================')
     time.sleep(10)
 
-if 'tablet_server' in job and operate in ['restart', 'rolling_update']:
+if version < '1.11' and 'tablet_server' in job and operate in ['restart', 'rolling_update']:
     for tserver in tservers_info:
         set_flag(tserver['rpc-addresses'], default_follower_unavailable_considered_failed_sec)
 


[kudu] 13/23: [collector] add service monitor

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 0abd7ecf9ff6e46c86f42e7abed1b8a935cc55c2
Author: 张一帆 <zh...@xiaomi.com>
AuthorDate: Tue Feb 18 10:52:55 2020 +0800

    [collector] add service monitor
---
 src/kudu/collector/CMakeLists.txt     |   4 +-
 src/kudu/collector/collector.cc       |   5 +
 src/kudu/collector/collector.h        |   2 +
 src/kudu/collector/nodes_checker.cc   |   2 +-
 src/kudu/collector/service_monitor.cc | 492 ++++++++++++++++++++++++++++++++++
 src/kudu/collector/service_monitor.h  |  92 +++++++
 src/kudu/scripts/falcon_screen.json   |   6 +-
 7 files changed, 598 insertions(+), 5 deletions(-)

diff --git a/src/kudu/collector/CMakeLists.txt b/src/kudu/collector/CMakeLists.txt
index 5bbb1cb..7cdf30b 100644
--- a/src/kudu/collector/CMakeLists.txt
+++ b/src/kudu/collector/CMakeLists.txt
@@ -26,10 +26,12 @@ set(COLLECTOR_SRCS
   falcon_reporter.cc
   local_reporter.cc
   metrics_collector.cc
-  nodes_checker.cc)
+  nodes_checker.cc
+  service_monitor.cc)
 
 add_library(collector ${COLLECTOR_SRCS})
 target_link_libraries(collector
+  kudu_client
   kudu_curl_util
   kudu_tools_test_util
   log
diff --git a/src/kudu/collector/collector.cc b/src/kudu/collector/collector.cc
index 59f36df..3501300 100644
--- a/src/kudu/collector/collector.cc
+++ b/src/kudu/collector/collector.cc
@@ -29,6 +29,7 @@
 #include "kudu/collector/metrics_collector.h"
 #include "kudu/collector/nodes_checker.h"
 #include "kudu/collector/reporter_base.h"
+#include "kudu/collector/service_monitor.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/security/init.h"
 #include "kudu/util/env.h"
@@ -91,6 +92,8 @@ Status Collector::Init() {
   CHECK_OK(metrics_collector_->Init());
   cluster_rebalancer_.reset(new ClusterRebalancer());
   CHECK_OK(cluster_rebalancer_->Init());
+  service_monitor_.reset(new ServiceMonitor(reporter_));
+  CHECK_OK(service_monitor_->Init());
 
   initialized_ = true;
   return Status::OK();
@@ -107,6 +110,7 @@ Status Collector::Start() {
   nodes_checker_->Start();
   metrics_collector_->Start();
   cluster_rebalancer_->Start();
+  service_monitor_->Start();
 
   return Status::OK();
 }
@@ -120,6 +124,7 @@ void Collector::Shutdown() {
     metrics_collector_->Shutdown();
     nodes_checker_->Shutdown();
     cluster_rebalancer_->Shutdown();
+    service_monitor_->Shutdown();
 
     stop_background_threads_latch_.CountDown();
 
diff --git a/src/kudu/collector/collector.h b/src/kudu/collector/collector.h
index e135a3c..12abaf2 100644
--- a/src/kudu/collector/collector.h
+++ b/src/kudu/collector/collector.h
@@ -37,6 +37,7 @@ class ClusterRebalancer;
 class MetricsCollector;
 class NodesChecker;
 class ReporterBase;
+class ServiceMonitor;
 
 class Collector {
  public:
@@ -64,6 +65,7 @@ class Collector {
   scoped_refptr<MetricsCollector> metrics_collector_;
   scoped_refptr<NodesChecker> nodes_checker_;
   scoped_refptr<ClusterRebalancer> cluster_rebalancer_;
+  scoped_refptr<ServiceMonitor> service_monitor_;
 
   CountDownLatch stop_background_threads_latch_;
   scoped_refptr<Thread> excess_log_deleter_thread_;
diff --git a/src/kudu/collector/nodes_checker.cc b/src/kudu/collector/nodes_checker.cc
index b2dacd8..036d72a 100644
--- a/src/kudu/collector/nodes_checker.cc
+++ b/src/kudu/collector/nodes_checker.cc
@@ -142,7 +142,7 @@ void NodesChecker::NodesCheckerThread() {
     UpdateAndCheckNodes();
     check_time += MonoDelta::FromSeconds(FLAGS_collector_interval_sec);
   } while (!RunOnceMode() && !stop_background_threads_latch_.WaitUntil(check_time));
-  LOG(INFO) << "FalconPusherThread exit";
+  LOG(INFO) << "NodesCheckerThread exit";
 }
 
 void NodesChecker::UpdateAndCheckNodes() {
diff --git a/src/kudu/collector/service_monitor.cc b/src/kudu/collector/service_monitor.cc
new file mode 100644
index 0000000..a92d309
--- /dev/null
+++ b/src/kudu/collector/service_monitor.cc
@@ -0,0 +1,492 @@
+// 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/collector/service_monitor.h"
+
+#include <cstdint>
+#include <list>
+#include <memory>
+#include <ostream>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+
+#include "kudu/client/client.h"
+#include "kudu/client/scan_batch.h"
+#include "kudu/client/scan_predicate.h"
+#include "kudu/client/value.h"
+#include "kudu/client/write_op.h"
+#include "kudu/collector/collector_util.h"
+#include "kudu/collector/reporter_base.h"
+#include "kudu/common/partial_row.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/tools/tool_test_util.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+#include "kudu/util/trace.h"
+
+DEFINE_string(collector_monitor_table_name, "system.monitor",
+              "Table name of monitor table.");
+DEFINE_uint32(collector_check_monitor_table_interval_sec, 3600,
+              "Number of interval seconds to check monitor table.");
+DEFINE_uint32(collector_monitor_avg_record_count_per_tablet, 100,
+              "Average record count for each tablet.");
+DEFINE_uint32(collector_monitor_avg_tablets_count_on_each_node, 10,
+              "Number of tablets of monitor table on each tablet server.");
+DEFINE_uint32(collector_monitor_timeout_threshold_sec, 30,
+              "If operations for checkintg service and record the result "
+              "take more than this number of seconds, "
+              "issue a warning with a trace.");
+DEFINE_uint32(collector_monitor_upsert_timeout_ms, 100,
+              "Timeout for one upsert operation");
+
+DECLARE_string(collector_cluster_name);
+DECLARE_string(collector_master_addrs);
+DECLARE_uint32(collector_interval_sec);
+DECLARE_uint32(collector_warn_threshold_ms);
+
+using kudu::client::KuduClientBuilder;
+using kudu::client::KuduColumnSchema;
+using kudu::client::KuduInsert;
+using kudu::client::KuduPredicate;
+using kudu::client::KuduScanBatch;
+using kudu::client::KuduScanner;
+using kudu::client::KuduScanToken;
+using kudu::client::KuduScanTokenBuilder;
+using kudu::client::KuduSchema;
+using kudu::client::KuduSchemaBuilder;
+using kudu::client::KuduSession;
+using kudu::client::KuduTable;
+using kudu::client::KuduTableCreator;
+using kudu::client::KuduTablet;
+using kudu::client::KuduTabletServer;
+using kudu::client::KuduUpsert;
+using kudu::client::KuduValue;
+using kudu::client::sp::shared_ptr;
+using kudu::KuduPartialRow;
+
+using std::list;
+using std::string;
+using std::vector;
+using std::unique_ptr;
+using std::unordered_map;
+using strings::Substitute;
+
+namespace kudu {
+namespace collector {
+
+ServiceMonitor::ServiceMonitor(scoped_refptr<ReporterBase> reporter)
+  : initialized_(false),
+    reporter_(std::move(reporter)),
+    stop_background_threads_latch_(1) {
+}
+
+ServiceMonitor::~ServiceMonitor() {
+  Shutdown();
+}
+
+Status ServiceMonitor::Init() {
+  CHECK(!initialized_);
+
+  RETURN_NOT_OK(InitCilent());
+  CHECK(client_);
+
+  last_check_table_time_ = MonoTime::Now();
+  RETURN_NOT_OK(CheckMonitorTable());
+
+  initialized_ = true;
+  return Status::OK();
+}
+
+Status ServiceMonitor::Start() {
+  CHECK(initialized_);
+
+  RETURN_NOT_OK(StartServiceMonitorThread());
+
+  return Status::OK();
+}
+
+void ServiceMonitor::Shutdown() {
+  if (initialized_) {
+    string name = ToString();
+    LOG(INFO) << name << " shutting down...";
+
+    stop_background_threads_latch_.CountDown();
+
+    if (service_monitor_thread_) {
+      service_monitor_thread_->Join();
+    }
+
+    LOG(INFO) << name << " shutdown complete.";
+  }
+}
+
+string ServiceMonitor::ToString() const {
+  return "ServiceMonitor";
+}
+
+Status ServiceMonitor::InitCilent() {
+  CHECK(client_.get() == nullptr);
+  const vector<string>& master_addresses =
+      Split(FLAGS_collector_master_addrs, ",", strings::SkipEmpty());
+  return KuduClientBuilder()
+      .master_server_addrs(master_addresses)
+      .Build(&client_);
+}
+
+KuduSchema ServiceMonitor::CreateTableSchema() {
+  KuduSchema schema;
+  KuduSchemaBuilder b;
+  b.AddColumn("key")->Type(KuduColumnSchema::INT64)->NotNull()->PrimaryKey();
+  b.AddColumn("value")->Type(KuduColumnSchema::INT64);
+  b.AddColumn("total_count")->Type(KuduColumnSchema::INT32);
+  b.AddColumn("success_count")->Type(KuduColumnSchema::INT32);
+  CHECK_OK(b.Build(&schema));
+  return schema;
+}
+
+Status ServiceMonitor::CreateMonitorTable(const string& table_name) {
+  vector<KuduTabletServer*> servers;
+  ElementDeleter deleter(&servers);
+  RETURN_NOT_OK(client_->ListTabletServers(&servers));
+  int num_tablets = servers.size() * FLAGS_collector_monitor_avg_tablets_count_on_each_node;
+
+  KuduSchema schema(CreateTableSchema());
+  vector<string> hash_keys = {"key"};
+  unique_ptr<KuduTableCreator> table_creator(client_->NewTableCreator());
+  table_creator->table_name(table_name)
+      .schema(&schema)
+      .add_hash_partitions(hash_keys, num_tablets)
+      .num_replicas(3);
+  RETURN_NOT_OK(table_creator->Create());
+  LOG(INFO) << Substitute("Created table $0.", table_name);
+  return Status::OK();
+}
+
+Status ServiceMonitor::CheckMonitorTable() {
+  string table_name = FLAGS_collector_monitor_table_name;
+  LOG(INFO) << Substitute("Checking monitor table $0.", table_name);
+  bool exist = false;
+  RETURN_NOT_OK(client_->TableExists(table_name, &exist));
+  if (!exist) {
+    RETURN_NOT_OK(CreateMonitorTable(table_name));
+  }
+
+  // Check monitor table's schema.
+  KuduSchema schema;
+  RETURN_NOT_OK(client_->GetTableSchema(table_name, &schema));
+  if (!schema.Equals(CreateTableSchema())) {
+    LOG(FATAL) << Substitute("$0 table $0 has an incorrect schema.", table_name);
+  }
+
+  // Check if monitor table's tablet count matches the cluster's node count.
+  shared_ptr<KuduTable> table;
+  CHECK_OK(client_->OpenTable(table_name, &table));
+  vector<KuduScanToken*> tokens;
+  ElementDeleter token_deleter(&tokens);
+  KuduScanTokenBuilder builder(table.get());
+  RETURN_NOT_OK(builder.Build(&tokens));
+  int replica_count = tokens.size() * table->num_replicas();
+  vector<KuduTabletServer*> servers;
+  ElementDeleter deleter(&servers);
+  RETURN_NOT_OK(client_->ListTabletServers(&servers));
+  if (replica_count < servers.size()) {
+    LOG(FATAL) <<
+        Substitute("$0 table's replica count doesn't match cluster's node count.", table_name);
+  }
+
+  // Check if all tablet servers at least has one leader replica running on it.
+  unordered_map<string, vector<string>> ts_tablets;
+  unordered_map<string, int> ts_leader_replica_count;
+  for (const auto* token : tokens) {
+    const auto& tablet =  token->tablet();
+    for (const auto* replica : tablet.replicas()) {
+      string ts = replica->ts().uuid();
+      if (replica->is_leader()) {
+        EmplaceIfNotPresent(&ts_leader_replica_count, ts, 0);
+        auto& leader_count = FindOrDie(ts_leader_replica_count, ts);
+        leader_count++;
+      }
+      EmplaceIfNotPresent(&ts_tablets, ts, vector<string>());
+      auto& tablets = FindOrDie(ts_tablets, ts);
+      tablets.emplace_back(tablet.id());
+    }
+  }
+  for (const auto* server : servers) {
+    const string& ts_uuid = server->uuid();
+    if (ContainsKey(ts_leader_replica_count, ts_uuid)) {
+      auto& leader_replica_count = FindOrDie(ts_leader_replica_count, ts_uuid);
+      LOG(INFO) << Substitute("TS $0 has $1 leader replicas on it",
+                              ts_uuid, leader_replica_count);
+      continue;
+    }
+    if (!ContainsKey(ts_tablets, ts_uuid)) {
+      LOG(WARNING) << Substitute("TS $0 has no replica running on it", ts_uuid);
+      RETURN_NOT_OK(RebalanceMonitorTable());
+      RETURN_NOT_OK(CheckMonitorTable());
+      return Status::OK();
+    }
+
+    LOG(WARNING) << Substitute("TS $0 has no leader replica running on it", ts_uuid);
+    const auto& tablets = FindOrDie(ts_tablets, ts_uuid);
+    string leader_step_down_tablet =
+        FindLeaderStepDownTablet(ts_leader_replica_count,
+                                 tablets,
+                                 FLAGS_collector_monitor_avg_tablets_count_on_each_node);
+    if (!leader_step_down_tablet.empty()) {
+      RETURN_NOT_OK(CallLeaderStepDown(leader_step_down_tablet, ts_uuid));
+      continue;
+    }
+    leader_step_down_tablet = FindLeaderStepDownTablet(ts_leader_replica_count,tablets, 1);
+    if (!leader_step_down_tablet.empty()) {
+      RETURN_NOT_OK(CallLeaderStepDown(leader_step_down_tablet, ts_uuid));
+      continue;
+    }
+    LOG(FATAL) << Substitute(
+        "Unable to call leader_step_down for replicas on ts $0, "
+        "set a larger number for 'collector_monitor_avg_tablets_count_on_each_node' ", ts_uuid);
+  }
+
+  return Status::OK();
+}
+
+Status ServiceMonitor::RebalanceMonitorTable() {
+  vector<string> args = {
+    "cluster",
+    "rebalance",
+    FLAGS_collector_master_addrs,
+    "--tables=" + FLAGS_collector_monitor_table_name
+  };
+  string tool_stdout;
+  string tool_stderr;
+  RETURN_NOT_OK_PREPEND(tools::RunKuduTool(args, &tool_stdout, &tool_stderr),
+                        Substitute("out: $0, err: $1", tool_stdout, tool_stderr));
+  LOG(INFO) << std::endl
+            << tool_stdout;
+  return Status::OK();
+}
+
+string ServiceMonitor::FindLeaderStepDownTablet(
+    const unordered_map<string, int>& ts_leader_replica_count,
+    const vector<string>& tablets,
+    int least_num_of_leader_replicas) {
+  string leader_step_down_tablet;
+  for (const auto& tablet : tablets) {
+    string leader_host_uuid;
+    Status s = GetLeaderHost(tablet, &leader_host_uuid);
+    if (!s.ok()) {
+      LOG(WARNING) << s.ToString();
+      continue;
+    }
+    CHECK(!leader_host_uuid.empty());
+    auto& leader_replica_count = FindOrDie(ts_leader_replica_count, leader_host_uuid);
+    if (leader_replica_count > least_num_of_leader_replicas) {
+      leader_step_down_tablet = tablet;
+      break;
+    }
+  }
+  return leader_step_down_tablet;
+}
+
+Status ServiceMonitor::GetLeaderHost(const string& tablet_id, string* leader_host) {
+  KuduTablet* tablet_raw = nullptr;
+  RETURN_NOT_OK(client_->GetTablet(tablet_id, &tablet_raw));
+  unique_ptr<KuduTablet> tablet(tablet_raw);
+  for (const auto* r : tablet->replicas()) {
+    if (r->is_leader()) {
+      *leader_host = r->ts().uuid();
+      return Status::OK();
+    }
+  }
+  return Status::NotFound(Substitute("No leader replica found for tablet $0", tablet_id));
+}
+
+Status ServiceMonitor::CallLeaderStepDown(const string& tablet_id, const string& ts_uuid) {
+  vector<string> args = {
+    "tablet",
+    "leader_step_down",
+    FLAGS_collector_master_addrs,
+    tablet_id,
+    "--new_leader_uuid=" + ts_uuid
+  };
+  string tool_stdout;
+  string tool_stderr;
+  RETURN_NOT_OK_PREPEND(tools::RunKuduTool(args, &tool_stdout, &tool_stderr),
+                        Substitute("out: $0, err: $1", tool_stdout, tool_stderr));
+  LOG(INFO) << std::endl
+            << tool_stdout;
+  return Status::OK();
+}
+
+Status ServiceMonitor::StartServiceMonitorThread() {
+  return Thread::Create("collector", "nodes-checker", &ServiceMonitor::ServiceMonitorThread,
+                        this, &service_monitor_thread_);
+}
+
+void ServiceMonitor::ServiceMonitorThread() {
+  MonoTime check_time;
+  do {
+    check_time = MonoTime::Now();
+    CheckService();
+    check_time += MonoDelta::FromSeconds(FLAGS_collector_interval_sec);
+  } while (!RunOnceMode() && !stop_background_threads_latch_.WaitUntil(check_time));
+  LOG(INFO) << "ServiceMonitorThread exit";
+}
+
+void ServiceMonitor::CheckService() {
+  int32_t elapsed_seconds = (MonoTime::Now() - last_check_table_time_).ToSeconds();
+  if (elapsed_seconds >= FLAGS_collector_check_monitor_table_interval_sec) {
+    last_check_table_time_ = MonoTime::Now();
+    WARN_NOT_OK(CheckMonitorTable(), "Unable to check monitor table");
+  }
+
+  LOG(INFO) << "Start to CheckService";
+  MonoTime start(MonoTime::Now());
+  scoped_refptr<Trace> trace(new Trace);
+  ADOPT_TRACE(trace.get());
+  TRACE_EVENT0("collector", "ServiceMonitor::CheckService");
+  TRACE("init");
+  bool exist = false;
+  CHECK_OK(client_->TableExists(FLAGS_collector_monitor_table_name, &exist));
+  if (!exist) {
+    WARN_NOT_OK(CheckMonitorTable(), "Unable to check monitor table");
+  }
+  shared_ptr<KuduTable> table;
+  CHECK_OK(client_->OpenTable(FLAGS_collector_monitor_table_name, &table));
+
+  WARN_NOT_OK(UpsertAndScanRows(table), "Unable to upsert and scan some rows");
+
+  int64_t elapsed_sec = (MonoTime::Now() - start).ToSeconds();
+  if (elapsed_sec > FLAGS_collector_monitor_timeout_threshold_sec) {
+    if (Trace::CurrentTrace()) {
+      LOG(WARNING) << "Trace:" << std::endl
+                   << Trace::CurrentTrace()->DumpToString();
+    }
+  }
+}
+
+Status ServiceMonitor::UpsertAndScanRows(const shared_ptr<KuduTable>& table) {
+  shared_ptr<KuduSession> session = table->client()->NewSession();
+  RETURN_NOT_OK(session->SetFlushMode(KuduSession::AUTO_FLUSH_SYNC));
+  session->SetTimeoutMillis(FLAGS_collector_monitor_upsert_timeout_ms);
+  vector<KuduScanToken*> tokens;
+  ElementDeleter deleter(&tokens);
+  KuduScanTokenBuilder builder(table.get());
+  RETURN_NOT_OK(builder.Build(&tokens));
+  int record_count = tokens.size() * FLAGS_collector_monitor_avg_record_count_per_tablet;
+  int64_t timestamp = static_cast<uint64_t>(WallTime_Now());
+
+  // Check if we can upsert some rows.
+  int write_success = 0;
+  MonoTime start(MonoTime::Now());
+  for (int i = 0; i < record_count; i++) {
+    KuduUpsert* upsert = table->NewUpsert();
+    KuduPartialRow* row = upsert->mutable_row();
+    RETURN_NOT_OK(row->SetInt64("key", i));
+    RETURN_NOT_OK(row->SetInt64("value", timestamp));
+    Status s = session->Apply(upsert);
+    if (s.ok()) {
+      write_success++;
+    } else {
+      LOG(WARNING) << s.ToString() <<  Substitute(": unable to upsert row (id=$0).", i);
+    }
+  }
+  int64_t write_latency_ms = (MonoTime::Now() - start).ToMilliseconds();
+  TRACE("Upsert some rows");
+  if (write_success != record_count) {
+    LOG(WARNING) << Substitute("Expect to upsert $0 rows, actually upsert $1 rows.",
+                                record_count, write_success);
+  }
+
+  // Check if rows upserted
+  KuduScanner scanner(table.get());
+  RETURN_NOT_OK(scanner.SetFaultTolerant());
+  // Add a predicate: WHERE key >= 0
+  KuduPredicate* p = table->NewComparisonPredicate(
+      "key", KuduPredicate::GREATER_EQUAL, KuduValue::FromInt(0));
+  RETURN_NOT_OK(scanner.AddConjunctPredicate(p));
+  // Add a predicate: WHERE key < record_count
+  p = table->NewComparisonPredicate(
+      "key", KuduPredicate::LESS, KuduValue::FromInt(record_count));
+  RETURN_NOT_OK(scanner.AddConjunctPredicate(p));
+  RETURN_NOT_OK(scanner.Open());
+
+  int read_success = 0;
+  start = MonoTime::Now();
+  KuduScanBatch batch;
+  while (scanner.HasMoreRows()) {
+    RETURN_NOT_OK(scanner.NextBatch(&batch));
+    for (KuduScanBatch::const_iterator it = batch.begin(); it != batch.end(); ++it) {
+      KuduScanBatch::RowPtr row(*it);
+      int64_t val;
+      RETURN_NOT_OK(row.GetInt64("value", &val));
+      if (val == timestamp) {
+        read_success++;
+      }
+    }
+  }
+  int64_t scan_latency_ms = (MonoTime::Now() - start).ToMilliseconds();
+  TRACE("Scan some rows");
+  if (read_success != write_success) {
+    LOG(WARNING) << Substitute("Expect to get $0 rows, actually get $1 rows.",
+                                write_success, read_success);
+  }
+
+  double total_count = record_count* 2;
+  double success_count = write_success + read_success;
+  double kudu_success = success_count/total_count*100;
+
+  KuduInsert* insert = table->NewInsert();
+  KuduPartialRow* row = insert->mutable_row();
+  RETURN_NOT_OK(row->SetInt64("key", timestamp));
+  RETURN_NOT_OK(row->SetInt32("total_count", total_count));
+  RETURN_NOT_OK(row->SetInt32("success_count", success_count));
+  RETURN_NOT_OK(session->Apply(insert));
+  RETURN_NOT_OK(session->Close());
+
+  unordered_map<string, int64_t> report_metrics;
+  report_metrics.emplace("kudu.scanLatency", scan_latency_ms);
+  report_metrics.emplace("kudu.writeLatency", write_latency_ms);
+  report_metrics.emplace("kudu.success", kudu_success);
+  list<scoped_refptr<ItemBase>> items;
+  for (const auto& elem : report_metrics) {
+    items.emplace_back(reporter_->ConstructItem(
+      FLAGS_collector_cluster_name,
+      elem.first,
+      "cluster",
+      timestamp,
+      elem.second,
+      "GAUGE",
+      ""));
+  }
+  reporter_->PushItems(std::move(items));
+  TRACE("Pushed results");
+
+  return Status::OK();
+}
+
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/collector/service_monitor.h b/src/kudu/collector/service_monitor.h
new file mode 100644
index 0000000..ab1515a
--- /dev/null
+++ b/src/kudu/collector/service_monitor.h
@@ -0,0 +1,92 @@
+// 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 <unordered_map>
+#include <vector>
+
+#include "kudu/client/schema.h"
+#include "kudu/client/shared_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Thread;
+
+namespace client {
+class KuduClient;
+class KuduTable;
+} // namespace client
+
+namespace collector {
+
+class ReporterBase;
+
+class ServiceMonitor : public RefCounted<ServiceMonitor> {
+ public:
+  explicit ServiceMonitor(scoped_refptr<ReporterBase> reporter);
+  ~ServiceMonitor();
+
+  Status Init();
+  Status Start();
+  void Shutdown();
+
+  std::string ToString() const;
+
+ private:
+  friend class RefCounted<ServiceMonitor>;
+
+  Status StartServiceMonitorThread();
+  void ServiceMonitorThread();
+  void CheckService();
+
+  client::KuduSchema CreateTableSchema();
+
+  Status CallLeaderStepDown(const std::string& tablet_id, const std::string& ts_uuid);
+  Status CheckMonitorTable();
+  Status CreateMonitorTable(const std::string& table_name);
+  Status InitCilent();
+  Status RebalanceMonitorTable();
+  Status UpsertAndScanRows(const client::sp::shared_ptr<client::KuduTable>& table);
+
+  // Find a tablet from 'tablets', whose leader replica host has n leader replicas,
+  // n > least_num_of_leader_replicas.
+  std::string FindLeaderStepDownTablet(
+      const std::unordered_map<std::string, int>& ts_leader_replica_count,
+      const std::vector<std::string>& tablets,
+      int least_num_of_leader_replicas);
+
+  // Get leader host uuid for a given tablet id.
+  Status GetLeaderHost(const std::string& tablet_id, std::string* leader_host);
+
+  bool initialized_;
+
+  client::sp::shared_ptr<client::KuduClient> client_;
+  scoped_refptr<ReporterBase> reporter_;
+  CountDownLatch stop_background_threads_latch_;
+  scoped_refptr<Thread> service_monitor_thread_;
+  MonoTime last_check_table_time_;
+
+  DISALLOW_COPY_AND_ASSIGN(ServiceMonitor);
+};
+} // namespace collector
+} // namespace kudu
diff --git a/src/kudu/scripts/falcon_screen.json b/src/kudu/scripts/falcon_screen.json
index 68911d9..e3ae54c 100644
--- a/src/kudu/scripts/falcon_screen.json
+++ b/src/kudu/scripts/falcon_screen.json
@@ -213,9 +213,9 @@
         "metric=write_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "cluster" : [
-          "metric=kudu.success service=kudu level=${level}",
-          "metric=kudu.writeLatency service=kudu level=${level}",
-          "metric=kudu.scanLatency service=kudu level=${level}",
+          "metric=kudu.success service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=kudu.writeLatency service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=kudu.scanLatency service=kudu cluster=${cluster.name} level=${level} v=4",
           "metric=healthy_table_proportion service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "cluster_stat" : [


[kudu] 03/23: [docs] Add docs for configuring maintenance priority for kudu tables

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 7ac9a698cde33c2af39a13f9dcfb3ceb50c72e31
Author: zhangyifan27 <ch...@163.com>
AuthorDate: Thu May 14 15:57:06 2020 +0800

    [docs] Add docs for configuring maintenance priority for kudu tables
    
    Change-Id: I56c96e1e86600a503fa817d62733aef3c0489915
    Reviewed-on: http://gerrit.cloudera.org:8080/15915
    Tested-by: Kudu Jenkins
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 docs/configuration.adoc | 1 +
 1 file changed, 1 insertion(+)

diff --git a/docs/configuration.adoc b/docs/configuration.adoc
index e9bb291..3667528 100644
--- a/docs/configuration.adoc
+++ b/docs/configuration.adoc
@@ -149,6 +149,7 @@ line tool.
 | Configuration      | Valid Options     | Default     | Description
 
 | kudu.table.history_max_age_sec | integer | | Number of seconds to retain history for tablets in this table.
+| kudu.table.maintenance_priority | integer | 0 | Priority level of a table for maintenance.
 |===
 
 == Next Steps


[kudu] 08/23: [script] Add script tools

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit e09b284b8fc41701ef017e3aacdfab5543332fe1
Author: laiyingchun <la...@xiaomi.com>
AuthorDate: Wed Jan 9 19:27:16 2019 +0800

    [script] Add script tools
---
 kudu                                        |   1 +
 src/kudu/scripts/batch_operate_on_tables.sh |  68 ++++
 src/kudu/scripts/build_env.sh               |  24 ++
 src/kudu/scripts/build_kudu.sh              | 162 ++++++++
 src/kudu/scripts/cal_bill_daily.py          | 280 +++++++++++++
 src/kudu/scripts/falcon_screen.json         | 603 ++++++++++++++++++++++++++++
 src/kudu/scripts/falcon_screen.py           | 603 ++++++++++++++++++++++++++++
 src/kudu/scripts/kudu_falcon_screen.sh      | 119 ++++++
 src/kudu/scripts/kudu_utils.py              | 106 +++++
 src/kudu/scripts/kudurc                     |  69 ++++
 src/kudu/scripts/minos_control_server.py    | 225 +++++++++++
 11 files changed, 2260 insertions(+)

diff --git a/kudu b/kudu
new file mode 120000
index 0000000..f2638cd
--- /dev/null
+++ b/kudu
@@ -0,0 +1 @@
+build/release/bin/kudu
\ No newline at end of file
diff --git a/src/kudu/scripts/batch_operate_on_tables.sh b/src/kudu/scripts/batch_operate_on_tables.sh
new file mode 100755
index 0000000..09a0e3e
--- /dev/null
+++ b/src/kudu/scripts/batch_operate_on_tables.sh
@@ -0,0 +1,68 @@
+#!/bin/bash
+
+if [ $# -lt 3 ]
+then
+  echo "This tool is for batch operation on batch of tables in a cluster"
+  echo "USAGE: $0 file operate cluster [dst-cluster]"
+  echo "        file: A file contains several table names in a cluster, one table name per line."
+  echo "              Or 'auto' means all tables in this cluster"
+  echo "     operate: Now support 'copy', 'delete', 'describe' and 'scan'"
+  echo "     cluster: CLuster name or master RPC addresses"
+  echo " dst-cluster: Master addresses of destination cluster, needed only when 'operate' is 'copy'"
+  exit -1
+fi
+
+FILE=$1
+OPERATE=$2
+CLUSTER=$3
+DST_CLUSTER=$4
+#FLAGS="-show_attributes"
+#FLAGS="-create_table=false -write_type=upsert"
+BIN_PATH=${KUDU_HOME}/kudu
+PID=$$
+
+echo "UID: ${UID}"
+echo "PID: ${PID}"
+echo "tables:"
+if [ "${FILE}" == "auto" ]
+then
+    echo "All tables in the cluster"
+else
+    cat ${FILE}
+fi
+echo "operate: ${OPERATE}"
+echo "cluster: ${CLUSTER}"
+echo "dst cluster: ${DST_CLUSTER}"
+echo "flags: ${FLAGS}"
+
+echo ""
+echo "All params above have been checked? (yes)"
+read INPUT
+if [ ! -n "${INPUT}" ] || [ "${INPUT}" != "yes" ]
+then
+    exit $?
+fi
+
+if [ -n "${DST_CLUSTER}" ]
+then
+    DST_CLUSTER=@${DST_CLUSTER}
+fi
+
+if [ "${FILE}" == "auto" ]
+then
+    TABLE_LIST=/tmp/$UID.${PID}.table.list
+    ${BIN_PATH} table list @${CLUSTER} | sort -n >${TABLE_LIST}
+else
+    TABLE_LIST=${FILE}
+fi
+
+if [ ! -f "${TABLE_LIST}" ]
+then
+    echo "file ${TABLE_LIST} is not exist!"
+    exit $?
+fi
+
+while read TABLE
+do
+    ${BIN_PATH} table ${OPERATE} @${CLUSTER} ${TABLE} ${DST_CLUSTER} ${FLAGS}
+done < ${TABLE_LIST}
diff --git a/src/kudu/scripts/build_env.sh b/src/kudu/scripts/build_env.sh
new file mode 100755
index 0000000..dde2d19
--- /dev/null
+++ b/src/kudu/scripts/build_env.sh
@@ -0,0 +1,24 @@
+#!/bin/bash
+
+OS=`lsb_release -d | awk '{print $2}'`
+echo "Setup build env for kudu on $OS"
+
+if [[ "$OS" == "CentOS" ]]; then
+  sudo yum -y install autoconf automake cyrus-sasl-devel cyrus-sasl-gssapi \
+    cyrus-sasl-plain flex gcc gcc-c++ gdb git java-1.8.0-openjdk-devel \
+    krb5-server krb5-workstation libtool make openssl-devel patch pkgconfig \
+    redhat-lsb-core rsync unzip vim-common which
+  DTLS_RPM=rhscl-devtoolset-3-epel-6-x86_64-1-2.noarch.rpm
+  DTLS_RPM_URL=https://www.softwarecollections.org/repos/rhscl/devtoolset-3/epel-6-x86_64/noarch/${DTLS_RPM}
+  wget ${DTLS_RPM_URL} -O ${DTLS_RPM}
+  sudo yum install -y scl-utils ${DTLS_RPM}
+  sudo yum install -y devtoolset-3-toolchain
+elif [[ "$OS" == "Ubuntu" ]]; then
+  sudo apt-get -y install autoconf automake curl flex g++ gcc gdb git \
+    krb5-admin-server krb5-kdc krb5-user libkrb5-dev libsasl2-dev libsasl2-modules \
+    libsasl2-modules-gssapi-mit libssl-dev libtool lsb-release make ntp \
+    openjdk-8-jdk openssl patch pkg-config python rsync unzip vim-common
+else
+  echo "Unsupported OS: $OS in $0"
+  exit
+fi
diff --git a/src/kudu/scripts/build_kudu.sh b/src/kudu/scripts/build_kudu.sh
new file mode 100755
index 0000000..1016697
--- /dev/null
+++ b/src/kudu/scripts/build_kudu.sh
@@ -0,0 +1,162 @@
+#!/bin/bash
+
+BASE_DIR="$( cd "$( dirname "$0"  )" && cd ../../.. && pwd )"
+
+function usage()
+{
+    echo "Options:"
+    echo "  -h"
+    echo "  -g|--custom-gcc"
+    exit 0
+}
+
+#USAGE: copy_file src [src...] dest
+function copy_file() {
+  if [[ $# -lt 2 ]]; then
+    echo "ERROR: invalid copy file command: cp $*"
+    exit 1
+  fi  
+  cp -v $*
+  if [[ $? -ne 0 ]]; then
+    echo "ERROR: copy file failed: cp $*"
+    exit 1
+  fi  
+}
+
+function get_stdcpp_lib()
+{
+    libname=`ldd ${BASE_DIR}/build/latest/bin/kudu 2>/dev/null | grep libstdc++`
+    libname=`echo $libname | cut -f1 -d" "`
+    if [ $1 = "true" ]; then
+        gcc_path=`which gcc`
+        echo `dirname $gcc_path`/../lib64/$libname
+    else
+        libs=(`ldconfig -p|grep $libname|awk '{print $NF}'`)
+
+        for lib in ${libs[*]}; do
+            if [ "`check_bit $lib`" = "true" ]; then
+                echo "$lib"
+                return
+            fi
+        done;
+    fi
+}
+
+function check_bit()
+{
+    bit_mode=`getconf LONG_BIT`
+    lib=$1
+    check_bit=""
+    is_softlink=`file $lib | grep "symbolic link"`
+
+    if [ -z "$is_softlink" ]; then
+        check_bit=`file $lib |grep "$bit_mode-bit"`
+    else
+        real_lib_name=`ls -l $lib |awk '{print $NF}'`
+        lib_path=${lib%/*}
+        real_lib=${lib_path}"/"${real_lib_name}
+        check_bit=`file $real_lib |grep "$bit_mode-bit"`
+    fi
+    if [ -n "$check_bit" ]; then
+        echo "true"
+    fi
+}
+
+custom_gcc="false"
+while [[ $# > 0 ]]; do
+    option_key="$1"
+    case $option_key in
+        -g|--custom-gcc)
+            custom_gcc="true"
+            ;;
+        -h|--help)
+            usage
+            ;;
+    esac
+    shift
+done
+
+KUDU_VERSION=`cat ${BASE_DIR}/version.txt`
+OS=`lsb_release -d | awk '{print $2}'`
+echo "Start to build kudu $KUDU_VERSION on $OS"
+
+if [[ "$OS" == "CentOS" ]]; then
+  ${BASE_DIR}/build-support/enable_devtoolset.sh
+  ${BASE_DIR}/thirdparty/build-if-necessary.sh
+elif [[ "$OS" == "Ubuntu" ]]; then
+  ${BASE_DIR}/thirdparty/build-if-necessary.sh
+else
+  echo "ERROR: unsupported OS: $OS in $0"
+  exit 1
+fi
+
+rm -rf ${BASE_DIR}/build/release
+mkdir -p ${BASE_DIR}/build/release
+cd ${BASE_DIR}/build/release
+../../thirdparty/installed/common/bin/cmake -DCMAKE_BUILD_TYPE=release ../..
+make -j `cat /proc/cpuinfo | egrep "^processor\s:" | wc -l`
+if [[ $? -ne 0 ]]; then
+  echo "ERROR: build Kudu failed"
+  exit 1
+fi
+echo "Build Kudu succeed"
+
+VERSION_DEFINES=${BASE_DIR}/build/release/src/kudu/generated/version_defines.h
+if [[ ! -f ${VERSION_DEFINES} ]]; then
+  echo "ERROR: $VERSION_DEFINES not found"
+  exit 1
+fi
+
+CLEAN_REPO=`grep "^#define KUDU_BUILD_CLEAN_REPO " ${VERSION_DEFINES} | awk '{print $NF}' | tr 'A-Z' 'a-z'`
+if [[ "$CLEAN_REPO"x != "true"x ]]; then
+  echo "ERROR: repository is not clean"
+  exit 1
+fi
+
+VERSION=`grep "^#define KUDU_VERSION_STRING " ${VERSION_DEFINES} | cut -d "\"" -f 2`
+COMMIT_ID=`grep "^#define KUDU_GIT_HASH " ${VERSION_DEFINES} | cut -d "\"" -f 2`
+BUILD_TYPE=`grep "^#define KUDU_BUILD_TYPE " ${VERSION_DEFINES} | cut -d "\"" -f 2`
+PACK_VERSION=`echo ${VERSION}-${COMMIT_ID:0:7}-${OS}-${BUILD_TYPE} | tr 'A-Z' 'a-z'`
+PACK_NAME=kudu-${PACK_VERSION}
+
+echo "Starting package $PACK_NAME"
+PACK_DIR=${BASE_DIR}/build/${PACK_NAME}
+PACKAGE=${PACK_NAME}.tar.gz
+rm -rf ${PACK_DIR} ${BASE_DIR}/build/${PACKAGE}
+mkdir -p ${PACK_DIR}
+echo "Coping files to $PACK_DIR"
+copy_file ${BASE_DIR}/build/latest/bin/kudu-collector ${PACK_DIR}/kudu_collector
+copy_file ${BASE_DIR}/build/latest/bin/kudu-master ${PACK_DIR}/kudu_master
+copy_file ${BASE_DIR}/build/latest/bin/kudu-tserver ${PACK_DIR}/kudu_tablet_server
+copy_file ${BASE_DIR}/build/latest/bin/kudu ${PACK_DIR}/
+copy_file `get_stdcpp_lib $custom_gcc` ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/batch_operate_on_tables.sh ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/falcon_screen.json ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/falcon_screen.py ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/kudu_falcon_screen.sh ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/minos_control_server.py ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/cal_bill_daily.py ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/kudu_utils.py ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/start_local_kudu.sh ${PACK_DIR}/
+copy_file ${BASE_DIR}/src/kudu/scripts/kudurc ${PACK_DIR}/
+copy_file -r ${BASE_DIR}/www ${PACK_DIR}/
+cd ${BASE_DIR}/build
+tar -czf ${PACKAGE} ${PACK_NAME}
+echo "Packaged $PACKAGE succeed"
+
+PACK_TEMPLATE=""
+if [[ -n "$MINOS_CONFIG_FILE" ]]; then
+  PACK_TEMPLATE=`dirname $MINOS_CONFIG_FILE`/xiaomi-config/package/kudu.yaml
+fi
+
+if [[ -f ${PACK_TEMPLATE} ]]; then
+  echo "Modifying $PACK_TEMPLATE ..."
+  sed -i "/^version:/c version: \"$PACK_VERSION\"" ${PACK_TEMPLATE}
+  sed -i "/^build:/c build: \"\.\/run.sh pack\"" ${PACK_TEMPLATE}
+  sed -i "/^source:/c source: \"$BASE_DIR/build\"" ${PACK_TEMPLATE}
+else
+  echo "ERROR: modify kudu.yaml failed"
+  exit 1
+fi
+
+echo "Done"
diff --git a/src/kudu/scripts/cal_bill_daily.py b/src/kudu/scripts/cal_bill_daily.py
new file mode 100755
index 0000000..a92b72b
--- /dev/null
+++ b/src/kudu/scripts/cal_bill_daily.py
@@ -0,0 +1,280 @@
+#! /usr/bin/env python
+# coding=utf-8
+
+import commands
+import datetime
+from git import Repo
+import heapq
+import logging
+from logging.handlers import RotatingFileHandler
+import json
+import os
+import re
+import sys
+import time
+import kudu_utils
+import yaml
+
+
+g_ignore_db_set = ('system', 'lcsbinlog', 'default', 'zhangxu_test_kudu')
+g_month_path, g_month_data_path = kudu_utils.prepare_pricing_month_path()
+g_clusters_info_dict = yaml.load(open(kudu_utils.g_script_path + '/kudurc', 'r').read(), Loader=yaml.FullLoader)
+g_clusters_info = g_clusters_info_dict['clusters_info']
+g_commit_filenames = list()
+g_git_repo_dir = ''
+
+
+def printtsr(level, table, size, reason):
+    kudu_utils.LOG.log(level, 'table: ' + table + (', size: %fG' % (size/(1 << 30)) + ', reason: ') + reason)
+
+
+class TopKHeap(object):
+    def __init__(self, k):
+        self.k = k
+        self.data = []
+
+    def push(self, elem):
+        if len(self.data) < self.k:
+            heapq.heappush(self.data, elem)
+        else:
+            top_k_small = self.data[0]
+            if elem['size'] > top_k_small['size']:
+                heapq.heapreplace(self.data, elem)
+
+    def top_k(self):
+        return {x['table']: x['size'] for x in reversed([heapq.heappop(self.data) for _ in xrange(len(self.data))])}
+
+
+def add_org_size(dbtable, org, size, org_size_desc):
+    if len(org) == 0:
+        printtsr(logging.WARNING, dbtable, size, 'Org name is empty')
+        return False
+
+    if org not in org_size_desc.keys():
+        org_size_desc[org] = {}
+        org_size_desc[org]['size'] = 0
+        org_size_desc[org]['desc'] = TopKHeap(10)
+    org_size_desc[org]['size'] += size
+    org_size_desc[org]['desc'].push({'size': size, 'table': dbtable})
+    return True
+
+
+def get_org_size_desc_from_olap(cluster_name, dbtable_size_dict, known_db_org_dict):
+    db_org_dict = {}
+    meta_table = 'system.kudu_table_owners'
+    cmd = '%s/kudu table scan @%s %s -show_values=true' \
+          ' -columns=name,db,org 2>&1 | grep "(string name=\\\""'\
+          % (kudu_utils.script_path(), cluster_name, meta_table)
+    status, output = commands.getstatusoutput(cmd)
+    if status != 0:
+        kudu_utils.LOG.error('Scan table %s error, command %s, status %d, output \n%s' % (meta_table, cmd, status, output))
+    else:
+        for line in output.splitlines():
+            match_obj = re.search(r'string name="(.*)", string db="(.*)", string org="(.*)"', line, re.M | re.I)
+            if match_obj:
+                db = match_obj.group(2)
+                org = match_obj.group(3)
+                db_org_dict[db] = org
+            else:
+                kudu_utils.LOG.error('Table %s value format error, line\n%s' % (meta_table, line))
+
+    total_ignored_size = 0.0
+    org_size_desc = {}
+    for dbtable, size in dbtable_size_dict.iteritems():
+        db_table_list = dbtable.split('.')
+        if len(db_table_list) != 2:
+            total_ignored_size += size
+            printtsr(logging.WARNING, dbtable, size, 'Lack db')
+            continue
+
+        db, table = db_table_list[0], db_table_list[1]
+        if db in g_ignore_db_set:
+            total_ignored_size += size
+            printtsr(logging.INFO, dbtable, size, 'Ignored table')
+            continue
+
+        if db in known_db_org_dict.keys():
+            # 'org' from config file
+            org = known_db_org_dict[db]
+        elif db in db_org_dict.keys():
+            # 'org' from system table
+            org = db_org_dict[db]
+        else:
+            total_ignored_size += size
+            printtsr(logging.WARNING, db, size, 'Lack org ID')
+            continue
+
+        if not add_org_size(dbtable, org, size, org_size_desc):
+            total_ignored_size += size
+            continue
+
+    printtsr(logging.WARNING, 'TOTAL', total_ignored_size, 'Total ignored size')
+    return org_size_desc
+
+
+def get_cluster_stat_filename(date, cluster_name):
+    return g_month_data_path + date + '_' + cluster_name
+
+
+def get_service_usage_filename(date):
+    return g_month_data_path + date + '_kudu_total'
+
+
+def collect_origin_usage_for_cluster(cluster_name, cluster_info):
+    kudu_utils.LOG.info('Start to collect usage info for cluster %s' % cluster_name)
+    # Output: db.table size
+    cmd = '%s/kudu_collector -collector_master_addrs=%s ' \
+          '-collector_report_method=local -collector_metrics=on_disk_size -log_dir=./log | ' \
+          'egrep "^table on_disk_size " | sort | awk \'{print $3, $4}\'' \
+          % (kudu_utils.g_script_path, cluster_info['master_addresses'])
+    status, output = commands.getstatusoutput(cmd)
+    if status != 0:
+        kudu_utils.LOG.fatal('Table stat error')
+        return
+
+    dbtable_size_dict = {}
+    for dbtable_size_str in output.splitlines():
+        dbtable_size_list = dbtable_size_str.split(' ')
+        assert(len(dbtable_size_list) == 2)
+        dbtable_size_dict[dbtable_size_list[0]] = float(dbtable_size_list[1])
+    known_db_org_dict = {}
+    if 'special_db_org' in cluster_info.keys():
+        known_db_org_dict = cluster_info['special_db_org']
+    org_size_desc = get_org_size_desc_from_olap(cluster_name, dbtable_size_dict, known_db_org_dict)
+
+    results = []
+    date = time.strftime('%Y-%m-%d', time.localtime())
+    period = int(time.mktime(datetime.datetime.strptime(date, "%Y-%m-%d").timetuple()))
+    for org, size_desc in org_size_desc.iteritems():
+        result = dict()
+        result['period'] = period
+        result['service_name'] = 'kudu'
+        result['region_name'] = cluster_info['region']
+        result['charge_type_name'] = cluster_info['charge_type']
+        result['instance_name'] = cluster_info['instance']
+        result['cluster'] = cluster_name
+        result['account_type'] = 'org' if org.find('CL') != -1 else 'kerberos'
+        result['account'] = org
+        result['usage'] = size_desc['size']
+        result['charge_object'] = size_desc['desc'].top_k()
+        results.append(result)
+    origin_usage_filename = get_cluster_stat_filename(date, cluster_name)
+    with open(origin_usage_filename, 'w') as origin_usage_file:
+        json.dump(results, origin_usage_file)
+        origin_usage_file.close()
+
+        g_commit_filenames.append(origin_usage_filename)
+
+
+def get_cluster_info(cluster_name):
+    if cluster_name not in g_clusters_info.keys():
+        kudu_utils.LOG.fatal('Cluster %s not found' % cluster_name)
+        return None
+
+    cluster_info = g_clusters_info[cluster_name]
+    if cluster_info['charge_type'] == 'public_share':
+        kudu_utils.LOG.warning('Ignore public_share cluster %s' % cluster_name)
+        return None
+
+    return cluster_info
+
+
+def collect_origin_usage_for_clusters(cluster_name_list):
+    for cluster_name in cluster_name_list:
+        cluster_info = get_cluster_info(cluster_name)
+        if not cluster_info:
+            continue
+        collect_origin_usage_for_cluster(cluster_name, cluster_info)
+
+
+def calc_usage_result(origin_usage_filename, service_usage_file):
+    kudu_utils.LOG.info('Start to process daily statistics file %s' % origin_usage_filename)
+    if not os.path.exists(origin_usage_filename):
+        kudu_utils.LOG.error('File not exist')
+        return
+    with open(origin_usage_filename, 'r') as origin_usage_file:
+        users_usage = json.load(origin_usage_file)
+        for user_usage in users_usage:
+            service_usage_file.write('%s, %s, %s, %s, %s, %s, %s, %s, \'{"storage_bytes":%d}\', \'%s\'\n'
+                                    % (user_usage['period'],
+                                       user_usage['service_name'],
+                                       user_usage['region_name'],
+                                       user_usage['charge_type_name'],
+                                       user_usage['instance_name'],
+                                       user_usage['cluster'],
+                                       user_usage['account_type'],
+                                       user_usage['account'],
+                                       user_usage['usage'],
+                                       json.dumps(user_usage['charge_object'])))
+        origin_usage_file.close()
+    kudu_utils.LOG.info('Write to file finished')
+
+
+def calc_usage_result_for_cluster(service_usage_file, cluster_name, date):
+    origin_usage_filename = get_cluster_stat_filename(date, cluster_name)
+    calc_usage_result(origin_usage_filename, service_usage_file)
+
+
+def calc_usage_result_for_clusters(cluster_name_list, date_list):
+    for date in date_list:
+        service_usage_filename = get_service_usage_filename(date)
+        with open(service_usage_filename, 'w') as service_usage_file:
+            # Write header
+            service_usage_file.write('period, service_name, region_name, charge_type_name, instance_name, '
+                              'cluster, account_type, account, usage, charge_object\n')
+            for cluster_name in cluster_name_list:
+                cluster_info = get_cluster_info(cluster_name)
+                if not cluster_info:
+                    continue
+                calc_usage_result_for_cluster(service_usage_file, cluster_name, date)
+            service_usage_file.close()
+        kudu_utils.upload_usage_data('append', service_usage_filename)
+        g_commit_filenames.append(service_usage_filename)
+
+
+def push_file_to_repo(filenames):
+    repo = Repo(g_git_repo_dir)
+    assert not repo.bare
+
+    remote = repo.remote()
+    remote.pull()
+
+    index = repo.index
+    index.add(filenames)
+    index.commit('Kudu add statistics files')
+
+    remote.push()
+
+    kudu_utils.LOG.info('Pushed files %s to repo' % str(filenames))
+
+
+def main(argv=None):
+    if not os.path.exists(g_git_repo_dir + '/.git'):
+        kudu_utils.LOG.fatal('You must set `g_git_repo_dir` to a valid directory contains `.git`')
+        return
+
+    if argv is None:
+        argv = sys.argv
+
+    cluster_name_list = []
+    if len(argv) == 1:
+        # Calculate all clusters
+        cluster_name_list = list(g_clusters_info.iterkeys())
+    elif len(argv) == 2:
+        # Calculate specified cluster
+        cluster_name_list.append(argv[1])
+    else:
+        kudu_utils.LOG.fatal('Usage: $0 [cluster_name]')
+        return
+
+    collect_origin_usage_for_clusters(cluster_name_list)
+
+    # date_list = kudu_utils.get_date_list('2019-06-01', kudu_utils.get_date())
+    date_list = [kudu_utils.get_date()]
+    calc_usage_result_for_clusters(cluster_name_list, date_list)
+
+    push_file_to_repo(g_commit_filenames)
+
+
+if __name__ == "__main__":
+    main()
diff --git a/src/kudu/scripts/falcon_screen.json b/src/kudu/scripts/falcon_screen.json
new file mode 100644
index 0000000..b15125c
--- /dev/null
+++ b/src/kudu/scripts/falcon_screen.json
@@ -0,0 +1,603 @@
+{
+  "comments": [
+    {
+      "screen": "screen名称",
+      "graphs": [
+        {
+          "title": "graph名称",
+          "endpoints": ["机器名或者tag标识,tag之间用空格分隔"],
+          "counters": ["counters名称,多个用逗号分隔。可以使用模糊匹配,支持metric(metric和tags(可选)空格隔开),精确匹配"],
+          "graph_type": "展示类型,endpoint视角为h,counters视角为k,组合视角为a",
+          "method": "绘图是否进行求和,求和填写sum,不求和填写空字符串",
+          "timespan": "展示的时间跨度,单位为秒"
+        }
+      ]
+    }
+  ],
+  "version": "20180827",
+  "counter_templates": {
+      "full": [
+          "metric=kudu-tserver-health service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=kudu-table-health service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=all_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=alter_schema_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=average_diskrowset_height service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=bloom_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=bloom_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=bytes_flushed service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=commit_wait_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=compact_rs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=compact_rs_running service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=delta_file_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=delta_file_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=delta_major_compact_rs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=delta_major_compact_rs_running service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=delta_minor_compact_rs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=delta_minor_compact_rs_running service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=failed_elections_since_stable_leader service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=flush_dms_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=flush_dms_running service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=flush_mrs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=flush_mrs_running service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=follower_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=in_progress_ops service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=insertions_failed_dup_key service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=key_file_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=key_file_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=leader_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_append_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_bytes_logged service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_cache_num_ops service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_cache_size service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_entry_batches_per_group_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_gc_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_gc_running service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_group_commit_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_reader_bytes_read service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_reader_entries_read service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_reader_read_batch_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_roll_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=log_sync_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=majority_done_ops service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=memrowset_size service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=mrs_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=num_rowsets_on_disk service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=on_disk_data_size service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=on_disk_size service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=op_prepare_queue_length_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=op_prepare_queue_time_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=op_prepare_run_time_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=ops_behind_leader service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=raft_term service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=replica_count service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=rows_deleted service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=rows_inserted service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=rows_updated service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=rows_upserted service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=scanner_bytes_returned service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=scanner_bytes_scanned_from_disk service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=scanner_cells_returned service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=scanner_cells_scanned_from_disk service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=scanner_rows_returned service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=scanner_rows_scanned service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=scans_started service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=snapshot_read_inflight_wait_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=state service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=tablet_active_scanners service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=time_since_last_leader_heartbeat service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=transaction_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=undo_delta_block_estimated_retained_bytes service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=undo_delta_block_gc_bytes_deleted service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=undo_delta_block_gc_delete_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=undo_delta_block_gc_init_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=undo_delta_block_gc_perform_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=undo_delta_block_gc_running service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=upserts_as_updates service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=write_op_duration_client_propagated_consistency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=write_op_duration_commit_wait_consistency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+          "metric=write_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_ab": [
+        "metric=all_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=alter_schema_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=average_diskrowset_height service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=bloom_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=bloom_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=bytes_flushed service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_c": [
+        "metric=commit_wait_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=compact_rs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=compact_rs_running service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_d": [
+        "metric=delta_file_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_file_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_major_compact_rs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_major_compact_rs_running service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_minor_compact_rs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=delta_minor_compact_rs_running service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_f": [
+        "metric=failed_elections_since_stable_leader service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_dms_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_dms_running service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_mrs_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=flush_mrs_running service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=follower_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_ghijk": [
+        "metric=in_progress_ops service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=insertions_failed_dup_key service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=key_file_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=key_file_lookups_per_op_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=kudu-table-health service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_l": [
+        "metric=leader_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_append_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_bytes_logged service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_cache_num_ops service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_cache_size service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_entry_batches_per_group_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_gc_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_gc_running service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_group_commit_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_reader_bytes_read service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_reader_entries_read service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_reader_read_batch_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_roll_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=log_sync_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=lth service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_mn": [
+        "metric=majority_done_ops service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=memrowset_size service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=mrs_lookups service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=num_rowsets_on_disk service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_o": [
+        "metric=on_disk_data_size service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=on_disk_size service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_queue_length_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_queue_time_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=op_prepare_run_time_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=ops_behind_leader service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_r": [
+        "metric=raft_term service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=replica_count service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=rows_deleted service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=rows_inserted service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=rows_updated service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=rows_upserted service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_s": [
+        "metric=scanner_bytes_returned service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=scanner_bytes_scanned_from_disk service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=scanner_cells_returned service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=scanner_cells_scanned_from_disk service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=scanner_rows_returned service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=scanner_rows_scanned service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=scans_started service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=snapshot_read_inflight_wait_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=state service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_u": [
+        "metric=undo_delta_block_estimated_retained_bytes service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_bytes_deleted service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_delete_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_init_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_perform_duration_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=undo_delta_block_gc_running service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=upserts_as_updates service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "table_tw": [
+        "metric=tablet_active_scanners service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=time_since_last_leader_heartbeat service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=transaction_memory_pressure_rejections service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=write_op_duration_client_propagated_consistency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=write_op_duration_commit_wait_consistency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=write_transactions_inflight service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "replica_count" : [
+          "metric=replica_count service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "cluster" : [
+          "metric=kudu.success service=kudu level=${level}",
+          "metric=kudu.writeLatency service=kudu level=${level}",
+          "metric=kudu.scanLatency service=kudu level=${level}",
+          "metric=healthy_table_proportion service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "cluster_stat" : [
+        "metric=masters_count service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=tservers_count service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=tables_count service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=tablets_count service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=replicas_count service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=on_disk_size service=kudu cluster=${cluster.name} level=${level} v=4",
+        "metric=on_disk_data_size service=kudu cluster=${cluster.name} level=${level} v=4"
+      ],
+      "sys" : [
+          "cpu.busy",
+          "load.15min",
+          "load.1min",
+          "load.5min",
+          "mem.memused",
+          "mem.memused.percent",
+          "net.if.in.bytes/iface=eth0",
+          "net.if.out.bytes/iface=eth0",
+          "net.if.total.dropped/iface=eth0",
+          "sys.ntp.offset/procname=chronyd"
+      ],
+      "disk_usage_percent" : [
+          "df.bytes.used.percent/fstype=ext4,mount=/home",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd1",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd2",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd3",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd4",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd5",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd6",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd7",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd8",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd9",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd10",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd11",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/ssd12",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd1",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd2",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd3",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd4",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd5",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd6",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd7",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd8",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd9",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd10",
+          "df.bytes.used.percent/fstype=ext4,mount=/home/work/hdd11"
+      ],
+      "disk_usage_size" : [
+          "df.bytes.used/fstype=ext4,mount=/home",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd1",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd2",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd3",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd4",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd5",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd6",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd7",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd8",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd9",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd10",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd11",
+          "df.bytes.used/fstype=ext4,mount=/home/work/ssd12",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd1",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd2",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd3",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd4",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd5",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd6",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd7",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd8",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd9",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd10",
+          "df.bytes.used/fstype=ext4,mount=/home/work/hdd11"
+      ],
+      "disk_io_util" : [
+          "disk.io.util/device=sdb",
+          "disk.io.util/device=sdc",
+          "disk.io.util/device=sdd",
+          "disk.io.util/device=sde",
+          "disk.io.util/device=sdf",
+          "disk.io.util/device=sdg",
+          "disk.io.util/device=sdh",
+          "disk.io.util/device=sdi",
+          "disk.io.util/device=sdj",
+          "disk.io.util/device=sdk",
+          "disk.io.util/device=sdl",
+          "disk.io.util/device=sdm",
+          "disk.io.util/device=nvme0n1",
+          "disk.io.util/device=nvme1n1",
+          "disk.io.util/device=nvme2n1",
+          "disk.io.util/device=nvme3n1",
+          "disk.io.util/device=nvme4n1",
+          "disk.io.util/device=nvme5n1",
+          "disk.io.util/device=xvda",
+          "disk.io.util/device=xvdb",
+          "disk.io.util/device=xvdc",
+          "disk.io.util/device=xvdd",
+          "disk.io.util/device=xvde",
+          "disk.io.util/device=xvdf",
+          "disk.io.util/device=vda",
+          "disk.io.util/device=vdb",
+          "disk.io.util/device=vdc",
+          "disk.io.util/device=vdd",
+          "disk.io.util/device=vde"
+      ]
+  },
+  "details": [
+    {
+      "screen": "${cluster.name} [cluster]",
+      "graphs": [
+        {
+          "title": "集群可用度",
+          "endpoints": ["${cluster.name}"],
+          "counters": {
+            "level": "cluster",
+            "template": "cluster"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        },
+        {
+          "title": "集群统计信息",
+          "endpoints": ["${cluster.name}"],
+          "counters": {
+            "level": "cluster",
+            "template": "cluster_stat"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_ab]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_ab"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_c]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_c"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_d]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_d"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_f]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_f"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_ghijk]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_ghijk"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_l]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_l"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_mn]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_mn"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_o]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_o"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_r]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_r"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_s]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_s"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_u]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_u"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [metrics_tw]",
+      "graphs": [
+        {
+          "title": "单表metrics",
+          "endpoints": ["${for.each.table}"],
+          "counters": {
+              "level": "table",
+              "template": "table_tw"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [tserver]",
+      "graphs": [
+        {
+          "title": "单节点metrics",
+          "endpoints": ["${for.each.tserver}"],
+          "counters": {
+              "level": "host",
+              "template": "full"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    },
+    {
+      "screen": "${cluster.name} [server-sys]",
+      "graphs": [
+        {
+          "title": "单节点sys指标",
+          "endpoints": ["${for.each.tserver}", "${for.each.master}"],
+          "counters": {
+              "level": "host",
+              "template": "sys"
+          },
+          "graph_type": "h",
+          "method": "",
+          "timespan": 86400
+        },
+        {
+          "title": "磁盘用量(百分比)",
+          "endpoints": ["${for.each.tserver}", "${for.each.master}"],
+          "counters": {
+              "level": "host",
+              "template": "disk_usage_percent"
+          },
+          "graph_type": "a",
+          "method": "",
+          "timespan": 86400
+        },
+        {
+          "title": "磁盘用量(占用空间)",
+          "endpoints": ["${for.each.tserver}", "${for.each.master}"],
+          "counters": {
+              "level": "host",
+              "template": "disk_usage_size"
+          },
+          "graph_type": "a",
+          "method": "",
+          "timespan": 86400
+        },
+        {
+          "title": "磁盘IO util",
+          "endpoints": ["${for.each.tserver}", "${for.each.master}"],
+          "counters": {
+              "level": "host",
+              "template": "disk_io_util"
+          },
+          "graph_type": "a",
+          "method": "",
+          "timespan": 86400
+        }
+      ]
+    }
+  ]
+}
diff --git a/src/kudu/scripts/falcon_screen.py b/src/kudu/scripts/falcon_screen.py
new file mode 100755
index 0000000..26f330a
--- /dev/null
+++ b/src/kudu/scripts/falcon_screen.py
@@ -0,0 +1,603 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+
+import requests
+import json
+import re
+import sys
+
+#
+# RESTful API doc: http://wiki.n.miui.com/pages/viewpage.action?pageId=66037692
+# falcon ctrl api: http://dev.falcon.srv/doc/
+#
+
+# account info
+serviceAccount = ""
+serviceSeedMd5 = ""
+
+###############################################################################
+
+# global variables
+falconServiceUrl = "http://falcon.srv"
+# falconServiceUrl = "http://dev.falcon.srv"
+kuduScreenId = 25748
+KUDU_CLUSTER_ID = 37613
+KUDU_TABLES_ID = 37638
+KUDU_TSERVER_ID = 37639
+KUDU_SYS_ID = 37640
+screenIdList = {
+    KUDU_CLUSTER_ID: "[cluster]",
+    KUDU_TABLES_ID: [
+        "[metrics_ab]",
+        "[metrics_c]",
+        "[metrics_d]",
+        "[metrics_f]",
+        "[metrics_ghijk]",
+        "[metrics_l]",
+        "[metrics_mn]",
+        "[metrics_o]",
+        "[metrics_r]",
+        "[metrics_s]",
+        "[metrics_u]",
+        "[metrics_tw]"],
+    KUDU_TSERVER_ID: "[tserver]",
+    KUDU_SYS_ID: "[server-sys]"}
+# kuduScreenId = 351
+sessionId = ""
+metaPort = ""
+replicaPort = ""
+collectorPort = ""
+
+
+# return:
+def get_session_id():
+    url = falconServiceUrl + "/v1.0/auth/info"
+    headers = {
+        "Accept": "text/plain"
+    }
+
+    r = requests.get(url, headers=headers)
+    if r.status_code != 200:
+        print(
+            "ERROR: get_session_id failed, status_code = %s, result:\n%s" %
+            (r.status_code, r.text))
+        sys.exit(1)
+
+    c = r.headers['Set-Cookie']
+    m = re.search('falconSessionId=([^;]+);', c)
+    if m:
+        global sessionId
+        sessionId = m.group(1)
+        print("INFO: sessionId =", sessionId)
+    else:
+        print("ERROR: get_session_id failed, cookie not set")
+        sys.exit(1)
+
+
+# return:
+def auth_by_misso():
+    url = falconServiceUrl + "/v1.0/auth/callback/misso"
+    headers = {
+        "Cookie": "falconSessionId=" +
+        sessionId,
+        "Authorization": serviceAccount +
+        ";" +
+        serviceSeedMd5 +
+        ";" +
+        serviceSeedMd5}
+
+    r = requests.get(url, headers=headers)
+    if r.status_code != 200:
+        print(
+            "ERROR: auth_by_misso failed, status_code = %s, result:\n%s" %
+            (r.status_code, r.text))
+        sys.exit(1)
+
+
+# return:
+def check_auth_info():
+    url = falconServiceUrl + "/v1.0/auth/info"
+    headers = {
+        "Cookie": "falconSessionId=" + sessionId
+    }
+
+    r = requests.get(url, headers=headers)
+    if r.status_code != 200:
+        print(
+            "ERROR: check_auth_info failed, status_code = %s, result:\n%s" %
+            (r.status_code, r.text))
+        sys.exit(1)
+
+    j = json.loads(r.text)
+    if "user" not in j or j["user"] is None or "name" not in j["user"] or j["user"]["name"] != serviceAccount:
+        print("ERROR: check_auth_info failed, bad json result:\n%s" % r.text)
+        sys.exit(1)
+
+
+def login():
+    get_session_id()
+    auth_by_misso()
+    check_auth_info()
+    print("INFO: login succeed")
+
+
+# return:
+def logout():
+    url = falconServiceUrl + "/v1.0/auth/logout"
+    headers = {
+        "Cookie": "falconSessionId=" + sessionId
+    }
+
+    r = requests.get(url, headers=headers)
+    if r.status_code != 200:
+        print(
+            "ERROR: logout failed, status_code = %s, result:\n%s" %
+            (r.status_code, r.text))
+        sys.exit(1)
+
+    print("INFO: logout succeed")
+
+
+# return: screenId
+def create_screen(screenName, scrid):
+    url = falconServiceUrl + "/v1.0/dashboard/screen"
+    headers = {
+        "Cookie": "falconSessionId=" + sessionId
+    }
+    req = {
+        "pid": scrid,
+        "name": screenName
+    }
+
+    r = requests.post(url, headers=headers, data=json.dumps(req))
+    if r.status_code != 200:
+        print(
+            "ERROR: create_screen failed, screenName = %s, status_code = %s, result:\n%s" %
+            (screenName, r.status_code, r.text))
+        sys.exit(1)
+
+    j = json.loads(r.text)
+    if "id" not in j:
+        print(
+            "ERROR: create_screen failed, screenName = %s, bad json result\n%s" %
+            (screenName, r.text))
+        sys.exit(1)
+
+    screenId = j["id"]
+    print(
+        "INFO: create_screen succeed, screenName = %s, screenId = %s" %
+        (screenName, screenId))
+    return screenId
+
+
+def parse_lines(file_name):
+    lines = []
+    for line in open(file_name):
+        line.strip()
+        if len(line) > 0:
+            if line in lines:
+                print("ERROR: bad file: duplicate line '%s'" % line)
+                sys.exit(1)
+            lines.append(line)
+    return lines
+
+
+# return: screenConfigs
+def prepare_screen_config(
+        clusterName,
+        templateName,
+        screenTemplateFile,
+        tableListFile,
+        masterListFile,
+        tserverListFile):
+    # tableList
+    tableList = parse_lines(tableListFile)
+    if len(tableList) == 0:
+        print("WARN: empty table list file, will not create table level falcon screen")
+
+    # masterList
+    masterList = parse_lines(masterListFile)
+    if len(masterList) == 0:
+        print("ERROR: bad master list file: should be non-empty list")
+        sys.exit(1)
+
+    # tserverList
+    tserverList = parse_lines(tserverListFile)
+    if len(tserverList) == 0:
+        print("ERROR: bad tserver list file: should be non-empty list")
+        sys.exit(1)
+
+    # template json
+    jsonData = json.loads(open(screenTemplateFile).read())
+    templateJson = jsonData['counter_templates']
+    screensJson = jsonData['details']
+    if not isinstance(screensJson, list) or len(screensJson) == 0:
+        print(
+            "ERROR: bad screen template json: [details] should be provided as non-empty list")
+        sys.exit(1)
+
+    screenConfigs = {}
+    for screenJson in screensJson:
+        # screen name
+        screen = screenJson["screen"]
+        if not isinstance(screen, (str, unicode)) or len(screen) == 0:
+            print(
+                "ERROR: bad json: [details][screen]: should be provided as non-empty str")
+            sys.exit(1)
+        screen = screen.replace("${cluster.name}", clusterName)
+        if screen in screenConfigs:
+            print("ERROR: duplicate screen '%s'" % screen)
+            sys.exit(1)
+
+        # graphs in screen
+        graphConfigs = []
+        position = 1
+        for graphJson in screenJson['graphs']:
+            # title
+            title = graphJson["title"]
+            if not isinstance(title, (str, unicode)) or len(title) == 0:
+                print(
+                    "ERROR: bad json: [details][%s][graphs][%s]: [title] should be provided as non-empty str" %
+                    (screen, title))
+                sys.exit(1)
+            if title in graphConfigs:
+                print("ERROR: duplicate title '%s'" % title)
+                sys.exit(1)
+
+            # endpoints
+            endpoints = graphJson["endpoints"]
+            newEndpoints = []
+            for endpoint in endpoints:
+                if len(endpoint) != 0:
+                    if endpoint.find("${cluster.name}") != -1:
+                        newEndpoints.append(
+                            endpoint.replace(
+                                "${cluster.name}",
+                                clusterName))
+                    elif endpoint.find("${for.each.master}") != -1:
+                        newEndpoints += masterList
+                    elif endpoint.find("${for.each.tserver}") != -1:
+                        newEndpoints += tserverList
+                    elif endpoint.find("${for.each.table}") != -1:
+                        newEndpoints += tableList
+                    else:
+                        newEndpoints.append(endpoint)
+            newEndpoints = list(set(newEndpoints))
+            if len(newEndpoints) == 0:
+                print(
+                    "WARN: bad json: [details][%s][graphs][%s]: [endpoints] should be provided as non-empty list" %
+                    (screen, title))
+
+            # counters
+            newCounters = []
+            counters = graphJson["counters"]
+            if not isinstance(counters, dict) or len(counters) == 0:
+                print(
+                    "ERROR: bad json: [details][%s][graphs][%s]: [counters] should be provided as non-empty list/dict" %
+                    (screen, title))
+                sys.exit(1)
+            for counter in templateJson[counters["template"] if counters.has_key("template") else templateName]:
+                newCounters.append(
+                    counter.replace(
+                        "${cluster.name}",
+                        clusterName). replace(
+                        "${level}",
+                        counters["level"]))
+            if len(newCounters) == 0:
+                print(
+                    "ERROR: bad json: [details][%s][graphs][%s]: [counters] should be provided as non-empty list" %
+                    (screen, title))
+                sys.exit(1)
+
+            # graphType
+            graphType = graphJson["graph_type"]
+            if not isinstance(graphType, (str, unicode)):
+                print(
+                    "ERROR: bad json: [details][%s][graphs][%s]: [graph_type] should be provided as non-empty list" %
+                    (screen, title))
+                sys.exit(1)
+            if graphType != "h" and graphType != "k" and graphType != "a":
+                print(
+                    "ERROR: bad json: [details][%s][graphs][%s]: [graph_type] should be 'h' or 'k' or 'a'" %
+                    (screen, title))
+                sys.exit(1)
+
+            # method
+            method = graphJson["method"]
+            if not isinstance(method, (str, unicode)):
+                print(
+                    "ERROR: bad json: [details][%s][graphs][%s]: [method] should be provided as str" %
+                    (screen, title))
+                sys.exit(1)
+            if method != "" and method != "sum":
+                print(
+                    "ERROR: bad json: [details][%s][graphs][%s]: [method] should be '' or 'sum'" %
+                    (screen, title))
+                sys.exit(1)
+
+            # timespan
+            timespan = graphJson["timespan"]
+            if not isinstance(timespan, int) or timespan <= 0:
+                print(
+                    "ERROR: bad json: [details][%s][graphs][%s]: [timespan] should be provided as positive int" %
+                    (screen, title))
+                sys.exit(1)
+
+            graphConfig = {}
+            graphConfig["counters"] = newCounters
+            graphConfig["endpoints"] = newEndpoints
+            graphConfig["falcon_tags"] = ""
+            graphConfig["graph_type"] = graphType
+            graphConfig["method"] = method
+            graphConfig["position"] = position
+            graphConfig["timespan"] = timespan
+            graphConfig["title"] = title
+            graphConfigs.append(graphConfig)
+
+            position += 1
+        screenConfigs[screen] = graphConfigs
+
+    return screenConfigs
+
+
+# return: graphId
+def create_graph(graphConfig):
+    url = falconServiceUrl + "/v1.0/dashboard/graph"
+    headers = {
+        "Cookie": "falconSessionId=" + sessionId
+    }
+
+    r = requests.post(url, headers=headers, data=json.dumps(graphConfig))
+    if r.status_code != 200:
+        print(
+            "ERROR: create_graph failed, graphTitle = \"%s\", status_code = %s, result:\n%s" %
+            (graphConfig["title"], r.status_code, r.text))
+        sys.exit(1)
+
+    j = json.loads(r.text)
+    if "id" not in j:
+        print(
+            "ERROR: create_graph failed, graphTitle = \"%s\", bad json result\n%s" %
+            (graphConfig["title"], r.text))
+        sys.exit(1)
+
+    graphId = j["id"]
+    print("INFO: create_graph succeed, graphTitle = \"%s\", graphId = %s"
+          % (graphConfig["title"], graphId))
+
+    # udpate graph position immediately
+    graphConfig["id"] = graphId
+    update_graph(graphConfig, "position")
+
+    return graphId
+
+
+# return: screen[]
+def get_kudu_screens(scrid):
+    url = falconServiceUrl + "/v1.0/dashboard/screen/pid/" + str(scrid)
+    headers = {
+        "Cookie": "falconSessionId=" + sessionId
+    }
+
+    r = requests.get(url, headers=headers)
+    if r.status_code != 200:
+        print(
+            "ERROR: get_kudu_screens failed, status_code = %s, result:\n%s" %
+            (r.status_code, r.text))
+        sys.exit(1)
+
+    j = json.loads(r.text)
+
+    print("INFO: get_kudu_screens succeed, screenCount = %s" % len(j))
+    return j
+
+
+# return: graph[]
+def get_screen_graphs(screenName, screenId):
+    url = falconServiceUrl + "/v1.0/dashboard/graph/screen/" + str(screenId)
+    headers = {
+        "Cookie": "falconSessionId=" + sessionId
+    }
+
+    r = requests.get(url, headers=headers)
+    if r.status_code != 200:
+        print(
+            "ERROR: get_screen_graphs failed, screenName = %s, screenId = %s, status_code = %s, result:\n%s" %
+            (screenName, screenId, r.status_code, r.text))
+        sys.exit(1)
+
+    j = json.loads(r.text)
+
+    print(
+        "INFO: get_screen_graphs succeed, screenName = %s, screenId = %s, graphCount = %s" %
+        (screenName, screenId, len(j)))
+    return j
+
+
+# return:
+def delete_graph(graphTitle, graphId):
+    url = falconServiceUrl + "/v1.0/dashboard/graph/" + str(graphId)
+    headers = {
+        "Cookie": "falconSessionId=" + sessionId
+    }
+
+    r = requests.delete(url, headers=headers)
+    if r.status_code != 200 or r.text.find("delete success!") == -1:
+        print(
+            "ERROR: delete_graph failed, graphTitle = \"%s\", graphId = %s, status_code = %s, result:\n%s" %
+            (graphTitle, graphId, r.status_code, r.text))
+        sys.exit(1)
+
+    print(
+        "INFO: delete_graph succeed, graphTitle = \"%s\", graphId = %s" %
+        (graphTitle, graphId))
+
+
+# return:
+def update_graph(graphConfig, updateReason):
+    url = falconServiceUrl + "/v1.0/dashboard/graph/" + str(graphConfig["id"])
+    headers = {
+        "Cookie": "falconSessionId=" + sessionId
+    }
+
+    r = requests.put(url, headers=headers, data=json.dumps(graphConfig))
+    if r.status_code != 200:
+        print(
+            "ERROR: update_graph failed, graphTitle = \"%s\", graphId = %s, status_code = %s, result:\n%s" %
+            (graphConfig["title"], graphConfig["id"], r.status_code, r.text))
+        sys.exit(1)
+
+    j = json.loads(r.text)
+    if "id" not in j:
+        print(
+            "ERROR: update_graph failed, graphTitle = \"%s\", graphId = %s, bad json result\n%s" %
+            (graphConfig["title"], graphConfig["id"], r.text))
+        sys.exit(1)
+
+    print(
+        "INFO: update_graph succeed, graphTitle = \"%s\", graphId = %s, updateReason = \"%s changed\"" %
+        (graphConfig["title"], graphConfig["id"], updateReason))
+
+
+# return: bool, reason
+def is_equal(graph1, graph2):
+    if graph1["title"] != graph2["title"]:
+        return False, "title"
+    if graph1["graph_type"] != graph2["graph_type"]:
+        return False, "graph_type"
+    if graph1["method"] != graph2["method"]:
+        return False, "method"
+    if graph1["position"] != graph2["position"]:
+        return False, "position"
+    if graph1["timespan"] != graph2["timespan"]:
+        return False, "timespan"
+    endpoints1 = graph1["endpoints"]
+    endpoints2 = graph2["endpoints"]
+    if len(endpoints1) != len(endpoints2):
+        return False, "endpoints"
+    for endpoint in endpoints1:
+        if endpoint not in endpoints2:
+            return False, "endpoints"
+    counters1 = graph1["counters"]
+    counters2 = graph2["counters"]
+    if len(counters1) != len(counters2):
+        return False, "counters"
+    for counter in counters1:
+        if counter not in counters2:
+            return False, "counters"
+    return True, ""
+
+
+def create_screen_and_graphs(screenName, scrid, graphConfigs):
+
+    # create screen
+    screenId = create_screen(screenName, scrid)
+    for graphConfig in graphConfigs:
+        graphConfig["screen_id"] = screenId
+        create_graph(graphConfig)
+    print("INFO: %s graphs created for %s" % (len(graphConfigs), screenName))
+
+
+def update_screen_and_graphs(screenName, screenId, graphConfigs):
+    oldGraphConfigs = get_screen_graphs(screenName, screenId)
+    if oldGraphConfigs is None:
+        print(
+            "ERROR: screen '%s' not exit, please create it first" %
+            clusterName)
+        sys.exit(1)
+
+    # list -> dict
+    oldGraphConfigsDict = {}
+    newGraphConfigsDict = {}
+    for graph in oldGraphConfigs:
+        oldGraphConfigsDict[graph["title"]] = graph
+    for graph in graphConfigs:
+        newGraphConfigsDict[graph["title"]] = graph
+
+    deleteConfigList = []
+    createConfigList = []
+    updateConfigList = []
+    for graph in oldGraphConfigs:
+        if not graph["title"] in newGraphConfigsDict:
+            deleteConfigList.append((graph["title"], graph["graph_id"]))
+    for graph in graphConfigs:
+        if not graph["title"] in oldGraphConfigsDict:
+            graph["screen_id"] = screenId
+            createConfigList.append(graph)
+        else:
+            oldGraph = oldGraphConfigsDict[graph["title"]]
+            equal, reason = is_equal(graph, oldGraph)
+            if not equal:
+                graph["id"] = oldGraph["graph_id"]
+                graph["screen_id"] = screenId
+                updateConfigList.append((graph, reason))
+
+    for graphTitle, graphId in deleteConfigList:
+        delete_graph(graphTitle, graphId)
+    for graph in createConfigList:
+        create_graph(graph)
+    for graph, reason in updateConfigList:
+        update_graph(graph, reason)
+
+    print("INFO: %d graphs deleted, %d graphs created, %d graphs updated" %
+          (len(deleteConfigList), len(createConfigList), len(updateConfigList)))
+
+
+if __name__ == '__main__':
+    if serviceAccount == "" or serviceSeedMd5 == "":
+        print(
+            "ERROR: please set 'serviceAccount' and 'serviceSeedMd5' in %s" %
+            sys.argv[0])
+        sys.exit(1)
+
+    if len(sys.argv) != 7:
+        print(
+            "USAGE: python %s <cluster_name> <template_name> <screen_template_file> <master_list_file> <tserver_list_file> <table_list_file>" %
+            sys.argv[0])
+        sys.exit(1)
+
+    clusterName = sys.argv[1]
+    templateName = sys.argv[2]
+    screenTemplateFile = sys.argv[3]
+    masterListFile = sys.argv[4]
+    tserverListFile = sys.argv[5]
+    tableListFile = sys.argv[6]
+
+    login()
+
+    for scrid, scrNames in screenIdList.items():
+        oldKuduScreens = get_kudu_screens(scrid)
+        oldScreenName2Id = {}
+        screenConfigs = prepare_screen_config(
+            clusterName,
+            templateName,
+            screenTemplateFile,
+            tableListFile,
+            masterListFile,
+            tserverListFile)
+        for oldScreen in oldKuduScreens:
+            oldScreenName2Id[oldScreen['name']] = oldScreen['id']
+        if scrid == KUDU_TABLES_ID:
+            for scrName in scrNames:
+                screenName = clusterName + " " + scrName
+                graphConfigs = screenConfigs[screenName]
+                if screenName not in oldScreenName2Id:
+                    # create screen
+                    create_screen_and_graphs(screenName, scrid, graphConfigs)
+                else:
+                    # update screen
+                    screenId = oldScreenName2Id[screenName]
+                    update_screen_and_graphs(
+                        screenName, screenId, graphConfigs)
+        else:
+            screenName = clusterName + " " + scrNames
+            graphConfigs = screenConfigs[screenName]
+            if screenName not in oldScreenName2Id:
+                # create screen
+                create_screen_and_graphs(screenName, scrid, graphConfigs)
+            else:
+                # update screen
+                screenId = oldScreenName2Id[screenName]
+                update_screen_and_graphs(screenName, screenId, graphConfigs)
+
+    logout()
diff --git a/src/kudu/scripts/kudu_falcon_screen.sh b/src/kudu/scripts/kudu_falcon_screen.sh
new file mode 100755
index 0000000..046352b
--- /dev/null
+++ b/src/kudu/scripts/kudu_falcon_screen.sh
@@ -0,0 +1,119 @@
+#!/bin/bash
+
+PID=$$
+BASE_DIR="$( cd "$( dirname "$0" )" && pwd )"
+KUDU=${KUDU_HOME}/kudu
+COLLECTOR=${KUDU_HOME}/kudu_collector
+if [[ ! -f ${KUDU} || ! -f ${COLLECTOR} ]]; then
+  echo "ERROR: ${KUDU} or ${COLLECTOR} not found"
+  exit 1
+fi
+KUDURC=${KUDU_CONFIG}/kudurc
+if [[ ! -f ${KUDURC} ]]; then
+  echo "ERROR: ${KUDURC} not found"
+  exit 1
+fi
+
+function usage() {
+cat << EOF
+This tool is for update falcon screen for specified kudu cluster.
+USAGE: $0 <cluster_name> [table_count] [metrics_template]
+       cluster_name       Cluster name operated on, should be configurated in $KUDU_CONFIG/kudurc
+       table_count        An indicator of how many tables will be monitored, actual monitored table count is in range [table_count, 3*table_count]
+       metrics_template   Which metric template will be used, 'simple' or 'full'
+EOF
+}
+
+if [[ $# -lt 1 ]]
+then
+  usage
+  exit 1
+fi
+
+CLUSTER=$1
+TABLE_COUNT=9999
+if [[ $# -ge 2 ]]
+then
+  TABLE_COUNT=$2
+fi
+
+TEMPLATE_NAME='full'
+if [[ $# -ge 3 ]]
+then
+  TEMPLATE_NAME=$3
+fi
+if [[ "${TEMPLATE_NAME}"x != "simple"x && "${TEMPLATE_NAME}"x != "full"x ]]
+then
+  usage
+  exit 1
+fi
+
+echo "UID: ${UID}"
+echo "PID: ${PID}"
+echo "cluster: ${CLUSTER}"
+echo "top n table: ${TABLE_COUNT}"
+echo "metric template: ${TEMPLATE_NAME}"
+echo "Start time: `date`"
+ALL_START_TIME=$((`date +%s`))
+echo
+
+# get master list
+${KUDU} master list @${CLUSTER} -format=space | awk -F' |:' '{print $2}' | sort -n &>/tmp/${UID}.${PID}.kudu.master.list
+if [[ $? -ne 0 ]]; then
+    echo "`kudu master list @${CLUSTER} -format=space` failed"
+    exit $?
+fi
+
+MASTER_COUNT=`cat /tmp/${UID}.${PID}.kudu.master.list | wc -l`
+if [[ ${MASTER_COUNT} -eq 0 ]]; then
+    echo "ERROR: master list is empty, please check the cluster ${CLUSTER}"
+    exit -1
+fi
+
+# get tserver list
+${KUDU} tserver list @${CLUSTER} -format=space | awk -F' |:' '{print $2}' | sort -n &>/tmp/${UID}.${PID}.kudu.tserver.list
+if [[ $? -ne 0 ]]; then
+    echo "`kudu tserver list @${CLUSTER} -format=space` failed"
+    exit $?
+fi
+
+TSERVER_COUNT=`cat /tmp/${UID}.${PID}.kudu.tserver.list | wc -l`
+if [[ ${TSERVER_COUNT} -eq 0 ]]; then
+    echo "ERROR: tserver list is empty, please check the cluster ${CLUSTER}"
+    exit 1
+fi
+
+function parse_yaml() {
+  python -c "import yaml;print(yaml.load(open('$1').read(), Loader=yaml.FullLoader)['clusters_info']['$2']['master_addresses'])"
+}
+MASTERS=$(parse_yaml ${KUDURC} ${CLUSTER})
+
+# get table list
+${COLLECTOR} -collector_master_addrs=${MASTERS} -collector_cluster_name=${CLUSTER} -collector_report_method=local -collector_metrics=bytes_flushed,on_disk_size,scanner_bytes_returned -log_dir=./log > /tmp/${UID}.${PID}.kudu.metric_table_value
+if [[ $? -ne 0 ]]; then
+    echo "ERROR: ${COLLECTOR} execute failed"
+    exit 1
+fi
+
+cat /tmp/${UID}.${PID}.kudu.metric_table_value | egrep "^table bytes_flushed " | sort -rnk4 | head -n ${TABLE_COUNT} | awk '{print $3}' > /tmp/${UID}.${PID}.kudu.top.bytes_flushed
+cat /tmp/${UID}.${PID}.kudu.metric_table_value | egrep "^table on_disk_size " | sort -rnk4 | head -n ${TABLE_COUNT} | awk '{print $3}' > /tmp/${UID}.${PID}.kudu.top.on_disk_size
+cat /tmp/${UID}.${PID}.kudu.metric_table_value | egrep "^table scanner_bytes_returned " | sort -rnk4 | head -n ${TABLE_COUNT} | awk '{print $3}' > /tmp/${UID}.${PID}.kudu.top.scanner_bytes_returned
+cat /tmp/${UID}.${PID}.kudu.top.* | sort -n | uniq > /tmp/${UID}.${PID}.kudu.table.list
+echo "total `wc -l /tmp/${UID}.${PID}.kudu.table.list | awk '{print $1}'` tables to monitor"
+echo -e "\033[32m Please set the following one line to the kudu collector's \`collector_attributes\` argument manually\033[0m"
+echo -n "table_name:"
+awk BEGIN{RS=EOF}'{gsub(/\n/,",");print}' /tmp/${UID}.${PID}.kudu.table.list
+echo ""
+
+python ${BASE_DIR}/falcon_screen.py ${CLUSTER} ${TEMPLATE_NAME} ${BASE_DIR}/falcon_screen.json /tmp/${UID}.${PID}.kudu.master.list /tmp/${UID}.${PID}.kudu.tserver.list /tmp/${UID}.${PID}.kudu.table.list
+if [[ $? -ne 0 ]]; then
+    echo "ERROR: falcon screen operate failed"
+    exit 1
+fi
+
+echo
+echo "Finish time: `date`"
+ALL_FINISH_TIME=$((`date +%s`))
+echo "Falcon screen operate done, elapsed time is $((ALL_FINISH_TIME - ALL_START_TIME)) seconds."
+
+rm -f /tmp/${UID}.${PID}.kudu.* &>/dev/null
diff --git a/src/kudu/scripts/kudu_utils.py b/src/kudu/scripts/kudu_utils.py
new file mode 100755
index 0000000..8dc5dfa
--- /dev/null
+++ b/src/kudu/scripts/kudu_utils.py
@@ -0,0 +1,106 @@
+#! /usr/bin/env python
+# coding=utf-8
+
+import datetime
+import dateutil.relativedelta
+import logging
+from logging.handlers import RotatingFileHandler
+import os
+import requests
+
+LOG = logging.getLogger()
+g_time = datetime.datetime.now()
+
+
+def init_log():
+    if not os.path.exists('log/'):
+        os.makedirs('log')
+    handler = RotatingFileHandler('log/kudu.log',
+                                  mode='a',
+                                  maxBytes=100*1024*1024,
+                                  backupCount=10)
+    handler.setFormatter(
+        logging.Formatter(
+            fmt='%(asctime)s [%(thread)d] [%(levelname)s] %(filename)s:%(lineno)d %(message)s',
+            datefmt='%Y-%m-%d %H:%M:%S'))
+    LOG.addHandler(handler)
+    LOG.setLevel(logging.INFO)
+
+
+def make_dir(path):
+    try:
+        os.mkdir(path)
+    except OSError, e:
+        if e.errno != os.errno.EEXIST:
+            raise
+        pass
+
+
+def script_path():
+    return os.path.split(os.path.realpath(__file__))[0]
+
+
+def get_year(last_month):
+    time = g_time
+    if last_month:
+        time += dateutil.relativedelta.relativedelta(months=-1)
+    return time.strftime('%Y')
+
+
+def get_month(last_month):
+    time = g_time
+    if last_month:
+        time += dateutil.relativedelta.relativedelta(months=-1)
+    return time.strftime('%m')
+
+
+def prepare_pricing_month_path(last_month=False):
+    month_base_path = script_path() + '/year=' + get_year(last_month)
+    make_dir(month_base_path)
+    month_base_path += '/month=' + get_month(last_month)
+    make_dir(month_base_path)
+    data_path = month_base_path + '/data'
+    make_dir(data_path)
+    return month_base_path + '/', data_path + '/'
+
+
+def get_year_month(last_month):
+    return get_year(last_month) + '-' + get_month(last_month)
+
+
+def get_date():
+    time = g_time
+    return time.strftime('%Y-%m-%d')
+
+
+def get_date_list(start, end, step=1, format="%Y-%m-%d"):
+    strptime, strftime = datetime.datetime.strptime, datetime.datetime.strftime
+    days = (strptime(end, format) - strptime(start, format)).days
+    return [strftime(strptime(start, format) + datetime.timedelta(i), format) for i in xrange(0, days, step)]
+
+
+# method:
+#   append: 追加写入账单
+#   reload: 清空数据并重新写入账单
+def upload_usage_data(method, filename):
+    LOG.info('Start to report %s by %s mode' % (filename, method))
+    report_url = "http://production-cost.api.xiaomi.net/api/v1/data/upload"
+    token = "c2534683e5504ab4850c49873a36de61"
+    url = "%s?sk=%s&method=%s" % (report_url, token, method)
+    with open(filename, "rb") as f:
+        resp = requests.post(url, files={"file": f})
+        if resp.status_code == 200:
+            # 成功
+            details = resp.json()
+            if details['code'] == 0:
+                LOG.info('Succeed to report %s by %s mode' % (filename, method))
+            else:
+                LOG.fatal('Failed to report %s by %s mode, details: %s' % (filename, method, str(details).decode("unicode-escape")))
+        else:
+            # 失败
+            LOG.fatal('Report failed, code %d' % resp.status_code)
+
+
+g_script_path = script_path()
+os.environ['KUDU_CONFIG'] = g_script_path
+init_log()
diff --git a/src/kudu/scripts/kudurc b/src/kudu/scripts/kudurc
new file mode 100644
index 0000000..6b28831
--- /dev/null
+++ b/src/kudu/scripts/kudurc
@@ -0,0 +1,69 @@
+clusters_info:
+  c3prc-hadoop:
+    olap_version: 2
+    region: chnbj-idc
+    charge_type: share
+    instance: SSD
+    special_db_org:
+      mifi: CL3894
+      b2c: CL5281
+    master_addresses: c3-hadoop-kudu-prc-ct01.bj:18600,c3-hadoop-kudu-prc-ct02.bj:18600,c3-hadoop-kudu-prc-ct03.bj:18600
+  zjyprc-hadoop:
+    olap_version: 2
+    region: chnbj-idc
+    charge_type: share
+    instance: SSD
+    master_addresses: zjy-hadoop-prc-ct01.bj:14000,zjy-hadoop-prc-ct02.bj:14000,zjy-hadoop-prc-ct03.bj:14000
+  zjyprc-analysis:
+    olap_version: 2
+    region: chnbj-idc
+    charge_type: exclusive
+    instance: SSD
+    special_db_org:
+      kudu_demo: CL18605
+      ga_test: CL18605
+    master_addresses: zjy-hadoop-prc-ct01.bj:15000,zjy-hadoop-prc-ct02.bj:15000,zjy-hadoop-prc-ct03.bj:15000
+  azmbcommonprc-hadoop:
+    olap_version: 2
+    region: indmb-aws
+    charge_type: share
+    instance: SSD
+    master_addresses: mb1-hadoop-kudu-prc-ct01.awsind:14000,mb2-hadoop-kudu-prc-ct02.awsind:14000,mb3-hadoop-kudu-prc-ct03.awsind:14000
+  ksmosprc-xiaomi:
+    olap_version: 2
+    region: rusmos-ks
+    charge_type: share
+    instance: SSD
+    master_addresses: mos1-hadoop-kudu-prc-ct01.ksru:14000,mos1-hadoop-kudu-prc-ct02.ksru:14000,mos1-hadoop-kudu-prc-ct03.ksru:14000
+  alsgprc-xiaomi:
+    olap_version: 2
+    region: sg-ali
+    charge_type: share
+    instance: SSD
+    special_db_org:
+      b2c: CL5281
+    master_addresses: sgp1-hadoop-kudu-prc-ct01.alisgp:15000,sgp2-hadoop-kudu-prc-ct02.alisgp:15000,sgp2-hadoop-kudu-prc-ct03.alisgp:15000
+  tjwqstaging-hdd:
+    olap_version: 2
+    region: chnwq-ks
+    charge_type: public_share
+    instance: SSD
+    master_addresses: tj1-hadoop-kudu-tst-ct01.kscn:18600,tj1-hadoop-kudu-tst-ct02.kscn:18600,tj1-hadoop-kudu-tst-ct03.kscn:18600
+  tjwqtst-dev:
+    olap_version: 2
+    region: chnwq-ks
+    charge_type: public_share
+    instance: SSD
+    master_addresses: tj1-hadoop-kudu-tst-ct01.kscn:15000,tj1-hadoop-kudu-tst-ct02.kscn:15000,tj1-hadoop-kudu-tst-ct03.kscn:15000
+  c3tst-test:
+    olap_version: 2
+    region: chnbj-idc
+    charge_type: public_share
+    instance: HDD
+    master_addresses: c3-hadoop-kudu-prc-ct01.bj:15000,c3-hadoop-kudu-prc-ct02.bj:15000,c3-hadoop-kudu-prc-ct03.bj:15000
+  c3tst-dev:
+    olap_version: 2
+    region: chnbj-idc
+    charge_type: public_share
+    instance: HDD
+    master_addresses: c3-hadoop-kudu-prc-ct01.bj:18000,c3-hadoop-kudu-prc-ct02.bj:18000,c3-hadoop-kudu-prc-ct03.bj:18000
diff --git a/src/kudu/scripts/minos_control_server.py b/src/kudu/scripts/minos_control_server.py
new file mode 100755
index 0000000..1d74878
--- /dev/null
+++ b/src/kudu/scripts/minos_control_server.py
@@ -0,0 +1,225 @@
+#! /usr/bin/env python
+# coding=utf-8
+
+# A tool for restarting servers, typically to restart tservers in kudu cluster
+
+import sys
+import commands
+import time
+import json
+import re
+import os
+import subprocess
+
+cluster = ''            # cluster name in minos config
+job = 'tablet_server'   # job name in minos config
+operate = 'stop'        # minos operate type, currently support: restart, stop, rolling_update
+tasks = range(0, 5)     # an int element list, e.g. '[n]' for a single node, or 'range(m, n)' for several nodes
+flags = ''              # minos flags, e.g. '--update_config' for updating config
+known_unhealth_nodes = set()
+#known_unhealth_nodes.add()    # it's ok to add some known unhealth nodes, e.g. some already stoped servers
+default_follower_unavailable_considered_failed_sec = 300    # default value of follower_unavailable_considered_failed_sec
+rebalance_cluster_after_operation = True    # whether to rebalance cluster after operation
+
+def get_minos_type(cluster_name):
+    minos_type = 'null'
+    minos_clinet_path = None
+
+    minos_config_file = os.getenv('MINOS_CONFIG_FILE')
+    minos_clinet_dir = os.getenv('MINOS_CLIENT_DIR')
+    if minos_config_file is not None and minos_clinet_dir is not None:
+        minos_config_dir = os.path.dirname(minos_config_file)
+        minos_config = '%s/xiaomi-config/conf/kudu/kudu-%s.cfg' % (minos_config_dir, cluster_name)
+        if os.path.exists(minos_config) and os.path.exists(minos_clinet_dir + '/deploy'):
+            return 'minos1.0', minos_clinet_dir
+    
+    minos2_config_file = os.getenv('MINOS2_CONFIG_FILE')
+    minos2_clinet_dir = os.getenv('MINOS2_CLIENT_DIR')
+    if minos2_config_file is not None and minos2_clinet_dir is not None:
+        minos2_config_dir = os.path.dirname(minos2_config_file)
+        minos2_config = '%s/xiaomi-config/conf/kudu/kudu-%s.yaml' % (minos2_config_dir, cluster_name)
+        if os.path.exists(minos2_config) and os.path.exists(minos2_clinet_dir + '/deploy'):
+            return 'minos2.0', minos2_clinet_dir
+
+    return minos_type, minos_clinet_path
+
+def get_host(host_port):
+    return host_port.split(':')[0]
+
+def is_cluster_health():
+    status, output = commands.getstatusoutput('${KUDU_HOME}/kudu cluster ksck @%s -consensus=false'
+                                              ' -ksck_format=json_compact -color=never'
+                                              ' -sections=MASTER_SUMMARIES,TSERVER_SUMMARIES,TABLE_SUMMARIES'
+                                              ' 2>/dev/null'
+                                              % cluster)
+    unhealth_nodes = set()
+    if status == 0 or status == 256:
+        ksck_info = json.loads(output)
+        for master in ksck_info['master_summaries']:
+            if master['health'] != 'HEALTHY':
+                unhealth_nodes.add(get_host(master['address']))
+        for tserver in ksck_info['tserver_summaries']:
+            if tserver['health'] != 'HEALTHY':
+                unhealth_nodes.add(get_host(tserver['address']))
+        if 'table_summaries' in ksck_info:
+            for table in ksck_info['table_summaries']:
+                if table['health'] != 'HEALTHY':
+                    unhealth_nodes.add(table['name'])
+    else:
+        unhealth_nodes.add('mockone')
+
+    return unhealth_nodes
+
+
+def check_parameter(message, parameter, allow_empty = False):
+    print(message % parameter)
+    answer = sys.stdin.readline().strip('\n').lower()
+    if answer != 'y' and answer != '':
+        exit()
+    if (not allow_empty and
+        (not parameter or
+         (isinstance(parameter, list) and len(parameter) == 0) or
+         (isinstance(parameter, str) and parameter.strip() == ''))):
+        print(time_header() + 'You should provide a valid parameter')
+        exit()
+
+
+def wait_cluster_health():
+    print(time_header() + 'Wait cluster to be health ...')
+    nodes = is_cluster_health()
+    health = (len(nodes) == 0)
+    while not health:
+        health = True
+        for node in nodes:
+            if node not in known_unhealth_nodes:
+                health = False
+                print(time_header() + 'Unhealthy node: ' + node)
+                time.sleep(5)
+                nodes = is_cluster_health()
+                break
+
+
+def parse_node_from_minos_output(output, job):
+    host = ''
+    regex = re.compile('[a-zA-Z\s]*[tT]ask [0-9]+ of (%s) on ([0-9a-z-.]+)\(0\).+' % job)
+    match = regex.search(output)
+    if match is not None:
+        host = match.group(2)
+    else:
+        print(time_header() + 'Fail to parse node from minos output')
+        exit()
+    return host
+
+
+def time_header():
+    return time.strftime("%Y-%m-%d %H:%M:%S ", time.localtime())
+
+
+def get_tservers_info():
+    tservers_info = dict()
+    status, output = commands.getstatusoutput('${KUDU_HOME}/kudu tserver list @%s -format=json'
+                                              % cluster)
+    if status == 0 or status == 256:
+        tservers_info = json.loads(output)
+    return tservers_info
+
+
+def get_tablet_server_info(hostname, tservers_info):
+    rpc_address = ''
+    uuid = ''
+    for tserver in tservers_info:
+        if hostname in tserver['rpc-addresses']:
+            rpc_address = tserver['rpc-addresses']
+            uuid = tserver['uuid']
+            break
+    return rpc_address, uuid
+
+
+def set_flag(rpc_address, seconds):
+    cmd = ('${KUDU_HOME}/kudu tserver set_flag %s follower_unavailable_considered_failed_sec %s'
+           % (rpc_address, seconds))
+    status, output = commands.getstatusoutput(cmd)
+
+
+def rebalance_cluster(blacklist_tserver_uuid):
+    ignored_tservers_uuid = set()
+    for node in known_unhealth_nodes:
+        rpc_address, uuid = get_tablet_server_info(node, tservers_info)
+        ignored_tservers_uuid.add(uuid)
+    cmd = ('${KUDU_HOME}/kudu cluster rebalance @%s -blacklist_tservers=%s -ignored_tservers=%s'
+           % (cluster, blacklist_tserver_uuid, str(','.join(ignored_tservers_uuid))))
+    p = subprocess.Popen(cmd, stdout = subprocess.PIPE, shell=True)
+    for line in iter(p.stdout.readline, b''):
+        print line
+    p.stdout.close()
+    p.wait()
+
+
+check_parameter('You will operate on cluster: %s? (y/n)', cluster)
+minos_type, minos_client_path = get_minos_type(cluster)
+if minos_type == 'null' or minos_client_path is None:
+    print("You should set these environment variables:\n* MINOS_CONFIG_FILE\n* MINOS_CLIENT_DIR\n" +
+          "* MINOS2_CONFIG_FILE\n* MINOS2_CLIENT_DIR\nand check cluster name")
+    exit()
+check_parameter('The minos type is: %s? (y/n)', minos_type)
+check_parameter('The minos client path is: %s? (y/n)', minos_client_path)
+check_parameter('You will operate on job: %s? (y/n)', job)
+check_parameter('You will operate on tasks: %s? (y/n)', tasks)
+check_parameter('The operate is: %s? (y/n)', operate)
+if operate == 'rolling_update' and flags.find('--update_package') == -1:
+    flags += ' --update_package'
+    if minos_type == 'minos2.0' and flags.find('--confirm_install') == -1:
+        flags += ' --confirm_install'
+check_parameter('The extra flags are: %s? (y/n)', flags, True)
+check_parameter('The known unhealth nodes are: %s? (y/n)', ','.join(known_unhealth_nodes), True)
+check_parameter('The default value of follower_unavailable_considered_failed_sec is: %s? (y/n)',
+                default_follower_unavailable_considered_failed_sec, True)
+check_parameter('You will rebalance cluster after operation: %s? (y/n)', rebalance_cluster_after_operation, True)
+
+tservers_info = get_tservers_info()
+wait_cluster_health()
+
+if 'tablet_server' in job and operate in ['restart', 'rolling_update']:
+    for tserver in tservers_info:
+        set_flag(tserver['rpc-addresses'], 7200)
+
+for task in tasks:
+    if not isinstance(task, int):
+        print(time_header() + '%s is not a valid integer task id' % str(task))
+        exit()
+
+    if 'tablet_server' in job:
+        cmd = ('%s/deploy show kudu %s --job %s --task %d'
+          % (minos_client_path, cluster, job, task))
+        status, output = commands.getstatusoutput(cmd)
+        print(output)
+        hostname = parse_node_from_minos_output(output, job)
+        rpc_address, uuid = get_tablet_server_info(hostname, tservers_info)
+        if operate == 'stop':
+            # migrate replicas on tserver
+            rebalance_cluster(uuid)
+
+    print(time_header() + 'Start to operate on task %d' % task)
+    cmd = ('%s/deploy %s kudu %s --job %s --task %d --skip_confirm %s'
+          % (minos_client_path, operate, cluster, job, task, flags))
+    status, output = commands.getstatusoutput(cmd)
+    print(output)
+    if operate == 'stop':
+        known_unhealth_nodes.add(parse_node_from_minos_output(output, job))
+
+    wait_cluster_health()
+
+    if 'tablet_server' in job and operate in ['restart', 'rolling_update']:
+        set_flag(rpc_address, 7200)
+
+    print(time_header() + '==========================')
+    time.sleep(10)
+
+if 'tablet_server' in job and operate in ['restart', 'rolling_update']:
+    for tserver in tservers_info:
+        set_flag(tserver['rpc-addresses'], default_follower_unavailable_considered_failed_sec)
+
+if rebalance_cluster_after_operation:
+    rebalance_cluster('')
+
+print(time_header() + 'Complete sucessfully')


[kudu] 11/23: [scripts] fix minos_control_server.py for minos2

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 647df1bfaffe5e4b441146ca1ab7f9e94ed97fe9
Author: 张一帆 <zh...@xiaomi.com>
AuthorDate: Mon Jan 13 16:18:29 2020 +0800

    [scripts] fix minos_control_server.py for minos2
---
 src/kudu/scripts/minos_control_server.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/kudu/scripts/minos_control_server.py b/src/kudu/scripts/minos_control_server.py
index 8db2bdc..875802c 100755
--- a/src/kudu/scripts/minos_control_server.py
+++ b/src/kudu/scripts/minos_control_server.py
@@ -58,7 +58,7 @@ def is_cluster_health():
                                               ' 2>/dev/null'
                                               % cluster)
     unhealth_nodes = set()
-    if status == 0:
+    if status == 0 or status == 256:
         ksck_info = json.loads(output)
         for master in ksck_info['master_summaries']:
             if master['health'] != 'HEALTHY':
@@ -106,7 +106,7 @@ def wait_cluster_health():
 
 def parse_node_from_minos_output(output, job):
     host = ''
-    regex = re.compile('[a-zA-Z\s]*[tT]ask [0-9]+ of (%s) on ([0-9a-z-.]+)\(0\).+' % job)
+    regex = re.compile('[a-zA-Z\s]*[tT]ask [0-9]+ of (%s) on ([0-9a-z-.]+)(:[0-9]+)*\(0\).+' % job)
     match = regex.search(output)
     if match is not None:
         host = match.group(2)


[kudu] 16/23: [collector] some improvements

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit b7ba14f8a2b2e740c6ad65a64541c52bc4dd1790
Author: zhangyifan8 <zh...@xiaomi.com>
AuthorDate: Thu Apr 2 12:07:30 2020 +0800

    [collector] some improvements
    
    -service monitor should not return if errors occurred when insert/upsert rows.
    -add two new metrics when init metric type.
---
 src/kudu/collector/metrics_collector.cc | 2 ++
 src/kudu/collector/service_monitor.cc   | 8 +++++---
 src/kudu/scripts/falcon_screen.json     | 2 +-
 3 files changed, 8 insertions(+), 4 deletions(-)

diff --git a/src/kudu/collector/metrics_collector.cc b/src/kudu/collector/metrics_collector.cc
index b170422..6d09a0d 100644
--- a/src/kudu/collector/metrics_collector.cc
+++ b/src/kudu/collector/metrics_collector.cc
@@ -194,6 +194,8 @@ Status MetricsCollector::InitMetrics() {
       EmplaceOrDie(&metric_types, std::make_pair(metric_type.first, metric_type.second));
     }
   }
+  EmplaceIfNotPresent(&metric_types, std::make_pair("merged_entities_count_of_tablet", "GAUGE"));
+  EmplaceIfNotPresent(&metric_types, std::make_pair("live_row_count", "GAUGE"));
 
   metric_types_.swap(metric_types);
   return Status::OK();
diff --git a/src/kudu/collector/service_monitor.cc b/src/kudu/collector/service_monitor.cc
index a92d309..e97fb79 100644
--- a/src/kudu/collector/service_monitor.cc
+++ b/src/kudu/collector/service_monitor.cc
@@ -61,7 +61,7 @@ DEFINE_uint32(collector_monitor_timeout_threshold_sec, 30,
               "take more than this number of seconds, "
               "issue a warning with a trace.");
 DEFINE_uint32(collector_monitor_upsert_timeout_ms, 100,
-              "Timeout for one upsert operation");
+              "Timeout for one insert/upsert operation");
 
 DECLARE_string(collector_cluster_name);
 DECLARE_string(collector_master_addrs);
@@ -411,7 +411,7 @@ Status ServiceMonitor::UpsertAndScanRows(const shared_ptr<KuduTable>& table) {
     if (s.ok()) {
       write_success++;
     } else {
-      LOG(WARNING) << s.ToString() <<  Substitute(": unable to upsert row (id=$0).", i);
+      LOG(WARNING) << s.ToString() <<  Substitute(": unable to upsert row (key=$0).", i);
     }
   }
   int64_t write_latency_ms = (MonoTime::Now() - start).ToMilliseconds();
@@ -464,7 +464,9 @@ Status ServiceMonitor::UpsertAndScanRows(const shared_ptr<KuduTable>& table) {
   RETURN_NOT_OK(row->SetInt64("key", timestamp));
   RETURN_NOT_OK(row->SetInt32("total_count", total_count));
   RETURN_NOT_OK(row->SetInt32("success_count", success_count));
-  RETURN_NOT_OK(session->Apply(insert));
+  WARN_NOT_OK(session->Apply(insert),
+              Substitute("unable to insert row (key=$0, total_count=$1, success_count=$2)",
+                         timestamp, total_count, success_count));
   RETURN_NOT_OK(session->Close());
 
   unordered_map<string, int64_t> report_metrics;
diff --git a/src/kudu/scripts/falcon_screen.json b/src/kudu/scripts/falcon_screen.json
index a7046b1..26b7acd 100644
--- a/src/kudu/scripts/falcon_screen.json
+++ b/src/kudu/scripts/falcon_screen.json
@@ -208,7 +208,7 @@
         "metric=log_sync_latency_percentile_75 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_sync_latency_percentile_95 service=kudu cluster=${cluster.name} level=${level} v=4",
         "metric=log_sync_latency_percentile_99 service=kudu cluster=${cluster.name} level=${level} v=4",
-        "metric=lth service=kudu cluster=${cluster.name} level=${level} v=4"
+        "metric=live_row_count service=kudu cluster=${cluster.name} level=${level} v=4"
       ],
       "table_mn": [
         "metric=majority_done_ops service=kudu cluster=${cluster.name} level=${level} v=4",


[kudu] 09/23: [script] fix minos_control_server.py: exit if execute command failed

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit d56a471a3f8101482c2a6a0d2e9bf44a62e8668a
Author: 张一帆 <zh...@xiaomi.com>
AuthorDate: Fri Dec 27 18:17:19 2019 +0800

    [script] fix minos_control_server.py: exit if execute command failed
---
 src/kudu/scripts/minos_control_server.py | 14 +++++++++++---
 1 file changed, 11 insertions(+), 3 deletions(-)

diff --git a/src/kudu/scripts/minos_control_server.py b/src/kudu/scripts/minos_control_server.py
index 1d74878..8db2bdc 100755
--- a/src/kudu/scripts/minos_control_server.py
+++ b/src/kudu/scripts/minos_control_server.py
@@ -21,6 +21,11 @@ known_unhealth_nodes = set()
 default_follower_unavailable_considered_failed_sec = 300    # default value of follower_unavailable_considered_failed_sec
 rebalance_cluster_after_operation = True    # whether to rebalance cluster after operation
 
+def exit_if_failed(status, output):
+    if status != 0:
+        print(output)
+        exit();
+
 def get_minos_type(cluster_name):
     minos_type = 'null'
     minos_clinet_path = None
@@ -53,7 +58,7 @@ def is_cluster_health():
                                               ' 2>/dev/null'
                                               % cluster)
     unhealth_nodes = set()
-    if status == 0 or status == 256:
+    if status == 0:
         ksck_info = json.loads(output)
         for master in ksck_info['master_summaries']:
             if master['health'] != 'HEALTHY':
@@ -119,8 +124,8 @@ def get_tservers_info():
     tservers_info = dict()
     status, output = commands.getstatusoutput('${KUDU_HOME}/kudu tserver list @%s -format=json'
                                               % cluster)
-    if status == 0 or status == 256:
-        tservers_info = json.loads(output)
+    exit_if_failed(status, output)
+    tservers_info = json.loads(output)
     return tservers_info
 
 
@@ -139,6 +144,7 @@ def set_flag(rpc_address, seconds):
     cmd = ('${KUDU_HOME}/kudu tserver set_flag %s follower_unavailable_considered_failed_sec %s'
            % (rpc_address, seconds))
     status, output = commands.getstatusoutput(cmd)
+    exit_if_failed(status, output)
 
 
 def rebalance_cluster(blacklist_tserver_uuid):
@@ -192,6 +198,7 @@ for task in tasks:
         cmd = ('%s/deploy show kudu %s --job %s --task %d'
           % (minos_client_path, cluster, job, task))
         status, output = commands.getstatusoutput(cmd)
+        exit_if_failed(status, output)
         print(output)
         hostname = parse_node_from_minos_output(output, job)
         rpc_address, uuid = get_tablet_server_info(hostname, tservers_info)
@@ -203,6 +210,7 @@ for task in tasks:
     cmd = ('%s/deploy %s kudu %s --job %s --task %d --skip_confirm %s'
           % (minos_client_path, operate, cluster, job, task, flags))
     status, output = commands.getstatusoutput(cmd)
+    exit_if_failed(status, output)
     print(output)
     if operate == 'stop':
         known_unhealth_nodes.add(parse_node_from_minos_output(output, job))


[kudu] 22/23: KUDU-3180: prioritize larger mem-stores in time-based flusing

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit dac73ac23d088058c11262358c7a5a3e65150dec
Author: zhangyifan27 <ch...@163.com>
AuthorDate: Mon Aug 10 19:39:21 2020 +0800

    KUDU-3180: prioritize larger mem-stores in time-based flusing
    
    Current time-based flush policy will always pick a mem-store
    that haven't been flushed in a long time instead of a mem-store
    anchoring more memory, this may lead to:
    - more memory used by mem-stores.
    - more small rowsets on disk so we need to do more compaction.
    
    This patch improve current flush policy by considering both
    mem-stores' size and time since last flush. When a mem-store
    become large or old enough, it will be more likely to flush,
    then we can avoid anchoring large (but below the threshold)
    mem-stores or WALs for too long.
    
    Change-Id: I0a826643709a4990e40b0a49f89f4ea34f14163b
    Reviewed-on: http://gerrit.cloudera.org:8080/16319
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/tablet/tablet_replica-test.cc   | 22 ++++++++++++-----
 src/kudu/tablet/tablet_replica_mm_ops.cc | 42 ++++++++++++++++----------------
 2 files changed, 37 insertions(+), 27 deletions(-)

diff --git a/src/kudu/tablet/tablet_replica-test.cc b/src/kudu/tablet/tablet_replica-test.cc
index b4d1750..339f80a 100644
--- a/src/kudu/tablet/tablet_replica-test.cc
+++ b/src/kudu/tablet/tablet_replica-test.cc
@@ -673,16 +673,18 @@ TEST_F(TabletReplicaTest, TestFlushOpsPerfImprovements) {
 
   MaintenanceOpStats stats;
 
-  // Just on the threshold and not enough time has passed for a time-based flush.
+  // Just on the threshold and not enough time has passed for a time-based flush,
+  // we'll expect improvement equal to '1'.
   stats.set_ram_anchored(64 * 1024 * 1024);
   FlushOpPerfImprovementPolicy::SetPerfImprovementForFlush(&stats, 1);
-  ASSERT_EQ(0.0, stats.perf_improvement());
+  ASSERT_EQ(1.0, stats.perf_improvement());
   stats.Clear();
 
-  // Just on the threshold and enough time has passed, we'll have a low improvement.
-  stats.set_ram_anchored(64 * 1024 * 1024);
+  // Below the threshold and enough time has passed, we'll have a low improvement.
+  stats.set_ram_anchored(2 * 1024 * 1024);
   FlushOpPerfImprovementPolicy::SetPerfImprovementForFlush(&stats, 3 * 60 * 1000);
-  ASSERT_GT(stats.perf_improvement(), 0.01);
+  ASSERT_LT(0.01, stats.perf_improvement());
+  ASSERT_GT(0.1, stats.perf_improvement());
   stats.Clear();
 
   // Over the threshold, we expect improvement equal to the excess MB.
@@ -692,11 +694,19 @@ TEST_F(TabletReplicaTest, TestFlushOpsPerfImprovements) {
   stats.Clear();
 
   // Below the threshold but have been there a long time, closing in to 1.0.
-  stats.set_ram_anchored(30 * 1024 * 1024);
+  stats.set_ram_anchored(1);
   FlushOpPerfImprovementPolicy::SetPerfImprovementForFlush(&stats, 60 * 50 * 1000);
   ASSERT_LT(0.7, stats.perf_improvement());
   ASSERT_GT(1.0, stats.perf_improvement());
   stats.Clear();
+
+  // Approaching threshold, enough time has passed but haven't been there a long time,
+  // closing in to 1.0.
+  stats.set_ram_anchored(63 * 1024 * 1024);
+  FlushOpPerfImprovementPolicy::SetPerfImprovementForFlush(&stats, 3 * 60 * 1000);
+  ASSERT_LT(0.9, stats.perf_improvement());
+  ASSERT_GT(1.0, stats.perf_improvement());
+  stats.Clear();
 }
 
 // Test that the schema of a tablet will be rolled forward upon replaying an
diff --git a/src/kudu/tablet/tablet_replica_mm_ops.cc b/src/kudu/tablet/tablet_replica_mm_ops.cc
index 61243a7..ea53453 100644
--- a/src/kudu/tablet/tablet_replica_mm_ops.cc
+++ b/src/kudu/tablet/tablet_replica_mm_ops.cc
@@ -17,11 +17,11 @@
 
 #include "kudu/tablet/tablet_replica_mm_ops.h"
 
+#include <algorithm>
 #include <map>
 #include <mutex>
 #include <ostream>
 #include <string>
-#include <utility>
 
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
@@ -30,7 +30,6 @@
 #include "kudu/common/common.pb.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
-#include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/tablet_metadata.h"
 #include "kudu/tablet/tablet_metrics.h"
 #include "kudu/util/flag_tags.h"
@@ -63,18 +62,25 @@ TAG_FLAG(enable_log_gc, runtime);
 TAG_FLAG(enable_log_gc, unsafe);
 
 DEFINE_int32(flush_threshold_mb, 1024,
-             "Size at which MemRowSet flushes are triggered. "
+             "Size at which MRS/DMS flushes are triggered. "
              "A MRS can still flush below this threshold if it hasn't flushed in a while, "
              "or if the server-wide memory limit has been reached.");
 TAG_FLAG(flush_threshold_mb, experimental);
 TAG_FLAG(flush_threshold_mb, runtime);
 
 DEFINE_int32(flush_threshold_secs, 2 * 60,
-             "Number of seconds after which a non-empty MemRowSet will become flushable "
+             "Number of seconds after which a non-empty MRS/DMS will become flushable "
              "even if it is not large.");
 TAG_FLAG(flush_threshold_secs, experimental);
 TAG_FLAG(flush_threshold_secs, runtime);
 
+DEFINE_int32(flush_upper_bound_ms, 60 * 60 * 1000,
+             "Number of milliseconds after which the time-based performance improvement "
+             "score of a non-empty MRS/DMS flush op will reach its maximum value. "
+             "The score may further increase as the MRS/DMS grows in size.");
+TAG_FLAG(flush_upper_bound_ms, experimental);
+TAG_FLAG(flush_upper_bound_ms, runtime);
+
 DECLARE_bool(enable_workload_score_for_perf_improvement_ops);
 
 METRIC_DEFINE_gauge_uint32(tablet, log_gc_running,
@@ -93,10 +99,6 @@ namespace kudu {
 namespace tablet {
 
 using std::map;
-using strings::Substitute;
-
-// Upper bound for how long it takes to reach "full perf improvement" in time-based flushing.
-const double kFlushUpperBoundMs = 60 * 60 * 1000;
 
 //
 // FlushOpPerfImprovementPolicy.
@@ -106,27 +108,25 @@ void FlushOpPerfImprovementPolicy::SetPerfImprovementForFlush(MaintenanceOpStats
                                                               double elapsed_ms) {
   double anchored_mb = static_cast<double>(stats->ram_anchored()) / (1024 * 1024);
   const double threshold_mb = FLAGS_flush_threshold_mb;
-  if (anchored_mb > threshold_mb) {
+  const double upper_bound_ms = FLAGS_flush_upper_bound_ms;
+  if (anchored_mb >= threshold_mb) {
     // If we're over the user-specified flush threshold, then consider the perf
-    // improvement to be 1 for every extra MB.  This produces perf_improvement results
-    // which are much higher than most compactions would produce, and means that, when
-    // there is an MRS over threshold, a flush will almost always be selected instead of
-    // a compaction.  That's not necessarily a good thing, but in the absence of better
+    // improvement to be 1 for every extra MB (at least 1). This produces perf_improvement
+    // results which are much higher than most compactions would produce, and means that,
+    // when there is an MRS over threshold, a flush will almost always be selected instead of
+    // a compaction. That's not necessarily a good thing, but in the absence of better
     // heuristics, it will do for now.
     double extra_mb = anchored_mb - threshold_mb;
     DCHECK_GE(extra_mb, 0);
-    stats->set_perf_improvement(extra_mb);
+    stats->set_perf_improvement(std::max(1.0, extra_mb));
   } else if (elapsed_ms > FLAGS_flush_threshold_secs * 1000) {
     // Even if we aren't over the threshold, consider flushing if we haven't flushed
     // in a long time. But, don't give it a large perf_improvement score. We should
     // only do this if we really don't have much else to do, and if we've already waited a bit.
-    // The following will give an improvement that's between 0.0 and 1.0, gradually growing
-    // as 'elapsed_ms' approaches 'kFlushUpperBoundMs'.
-    double perf = elapsed_ms / kFlushUpperBoundMs;
-    if (perf > 1.0) {
-      perf = 1.0;
-    }
-    stats->set_perf_improvement(perf);
+    // The following will give an improvement that's between 0.0 and 1.0, gradually growing as
+    // 'elapsed_ms' approaches 'upper_bound_ms' or 'anchored_mb' approaches 'threshold_mb'.
+    double perf = std::max(elapsed_ms / upper_bound_ms, anchored_mb / threshold_mb);
+    stats->set_perf_improvement(std::min(1.0, perf));
   }
 }
 


[kudu] 20/23: [tserver] add 'runtime' tag for scanner_ttl_ms

Posted by la...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to tag kudu-1.12.0-mdh1.0.0-4c2c075-centos-release
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 6db4bae65d2fca4030db0cdf4ad20220bbd3c413
Author: zhangyifan27 <ch...@163.com>
AuthorDate: Wed Jun 3 16:51:59 2020 +0800

    [tserver] add 'runtime' tag for scanner_ttl_ms
    
    The --scanner_ttl_ms flag has runtime-settable behavior,
    this patch simply adds 'runtime' tag so we don't need
    to use '--force' when using 'kudu tserver set_flag'.
    
    Change-Id: I91c2a9192f7f80edc3927fe226ddcd1d7be0d78f
    Reviewed-on: http://gerrit.cloudera.org:8080/16028
    Tested-by: Kudu Jenkins
    Reviewed-by: Yingchun Lai <40...@qq.com>
    Reviewed-by: Todd Lipcon <to...@apache.org>
---
 src/kudu/tserver/scanners.cc | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/src/kudu/tserver/scanners.cc b/src/kudu/tserver/scanners.cc
index 8d90567..c713894 100644
--- a/src/kudu/tserver/scanners.cc
+++ b/src/kudu/tserver/scanners.cc
@@ -50,6 +50,8 @@ DEFINE_int32(scanner_ttl_ms, 60000,
              "Number of milliseconds of inactivity allowed for a scanner"
              "before it may be expired");
 TAG_FLAG(scanner_ttl_ms, advanced);
+TAG_FLAG(scanner_ttl_ms, runtime);
+
 DEFINE_int32(scanner_gc_check_interval_us, 5 * 1000L *1000L, // 5 seconds
              "Number of microseconds in the interval at which we remove expired scanners");
 TAG_FLAG(scanner_gc_check_interval_us, hidden);