You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2017/10/23 23:43:21 UTC

[2/2] kudu git commit: tool: remove dependency on libcurl

tool: remove dependency on libcurl

This dependency was introduced in commit 761ce10, when I added support for
spawning mini clusters through the CLI tool. We could treat it as a valid
new dependency and doc it, but it only exists because of one ExternalDaemon
test function. So instead, let's move that function from ExternalDaemon to
the group of free functions in cluster_itest_util.

I verified that with this patch the CLI no longer depends on kudu_curl_util
or on libcurl, and that the build still passes with NO_TESTS=1.

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


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

Branch: refs/heads/master
Commit: 09ac5bfd6288825345ce6a3ce51c2dccd28dccaa
Parents: ae03324
Author: Adar Dembo <ad...@cloudera.com>
Authored: Mon Oct 23 15:38:20 2017 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Mon Oct 23 23:43:02 2017 +0000

----------------------------------------------------------------------
 .../integration-tests/client-stress-test.cc     |  7 +-
 .../integration-tests/cluster_itest_util.cc     | 70 +++++++++++++++-
 src/kudu/integration-tests/cluster_itest_util.h | 19 +++++
 .../integration-tests/create-table-itest.cc     |  4 +-
 .../integration-tests/delete_table-itest.cc     | 19 +++--
 src/kudu/integration-tests/dense_node-itest.cc  |  9 +-
 .../integration-tests/disk_failure-itest.cc     |  6 +-
 .../integration-tests/disk_reservation-itest.cc | 13 +--
 .../integration-tests/master_failover-itest.cc  |  8 +-
 .../raft_consensus-itest-base.cc                |  6 +-
 .../integration-tests/raft_consensus-itest.cc   |  4 +-
 .../raft_consensus_nonvoter-itest.cc            |  5 +-
 src/kudu/integration-tests/tablet_copy-itest.cc | 87 +++++++++++---------
 .../tombstoned_voting-itest.cc                  | 12 +--
 src/kudu/mini-cluster/CMakeLists.txt            |  1 -
 .../mini-cluster/external_mini_cluster-test.cc  | 22 -----
 src/kudu/mini-cluster/external_mini_cluster.cc  | 66 ---------------
 src/kudu/mini-cluster/external_mini_cluster.h   | 17 ----
 src/kudu/util/CMakeLists.txt                    | 18 ++--
 19 files changed, 205 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/client-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/client-stress-test.cc b/src/kudu/integration-tests/client-stress-test.cc
index f420ca3..5146e13 100644
--- a/src/kudu/integration-tests/client-stress-test.cc
+++ b/src/kudu/integration-tests/client-stress-test.cc
@@ -38,6 +38,7 @@
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/integration-tests/cluster_itest_util.h"
 #include "kudu/integration-tests/test_workload.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
 #include "kudu/util/countdown_latch.h"
@@ -269,7 +270,8 @@ TEST_F(ClientStressTest_LowMemory, TestMemoryThrottling) {
     // or metric is truly missing, we'll eventually timeout and fail.
     for (int i = 0; i < cluster_->num_tablet_servers(); i++) {
       int64_t value;
-      Status s = cluster_->tablet_server(i)->GetInt64Metric(
+      Status s = itest::GetInt64Metric(
+          cluster_->tablet_server(i)->bound_http_hostport(),
           &METRIC_ENTITY_tablet,
           nullptr,
           &METRIC_leader_memory_pressure_rejections,
@@ -279,7 +281,8 @@ TEST_F(ClientStressTest_LowMemory, TestMemoryThrottling) {
         ASSERT_OK(s);
         total_num_rejections += value;
       }
-      s = cluster_->tablet_server(i)->GetInt64Metric(
+      s = itest::GetInt64Metric(
+          cluster_->tablet_server(i)->bound_http_hostport(),
           &METRIC_ENTITY_tablet,
           nullptr,
           &METRIC_follower_memory_pressure_rejections,

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/cluster_itest_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/cluster_itest_util.cc b/src/kudu/integration-tests/cluster_itest_util.cc
index 6da465b..e956804 100644
--- a/src/kudu/integration-tests/cluster_itest_util.cc
+++ b/src/kudu/integration-tests/cluster_itest_util.cc
@@ -23,6 +23,7 @@
 #include <boost/optional/optional.hpp>
 #include <glog/logging.h>
 #include <glog/stl_logging.h>
+#include <rapidjson/document.h>
 
 #include "kudu/client/schema.h"
 #include "kudu/common/common.pb.h"
@@ -48,6 +49,10 @@
 #include "kudu/tserver/tserver_admin.pb.h"
 #include "kudu/tserver/tserver_admin.proxy.h"
 #include "kudu/tserver/tserver_service.proxy.h"
+#include "kudu/util/curl_util.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/jsonreader.h"
+#include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/net/sockaddr.h"
@@ -81,8 +86,9 @@ using master::ListTabletServersResponsePB;
 using master::ListTabletServersResponsePB_Entry;
 using master::MasterServiceProxy;
 using master::TabletLocationsPB;
-using kudu::pb_util::SecureDebugString;
-using kudu::pb_util::SecureShortDebugString;
+using pb_util::SecureDebugString;
+using pb_util::SecureShortDebugString;
+using rapidjson::Value;
 using rpc::Messenger;
 using rpc::RpcController;
 using std::min;
@@ -1032,5 +1038,65 @@ Status BeginTabletCopySession(const TServerDetails* ts,
   return Status::OK();
 }
 
+
+Status GetInt64Metric(const HostPort& http_hp,
+                      const MetricEntityPrototype* entity_proto,
+                      const char* entity_id,
+                      const MetricPrototype* metric_proto,
+                      const char* value_field,
+                      int64_t* value) {
+  // Fetch metrics whose name matches the given prototype.
+  string url = Substitute(
+      "http://$0/jsonmetricz?metrics=$1",
+      http_hp.ToString(), metric_proto->name());
+  EasyCurl curl;
+  faststring dst;
+  RETURN_NOT_OK(curl.FetchURL(url, &dst));
+
+  // Parse the results, beginning with the top-level entity array.
+  JsonReader r(dst.ToString());
+  RETURN_NOT_OK(r.Init());
+  vector<const Value*> entities;
+  RETURN_NOT_OK(r.ExtractObjectArray(r.root(), nullptr, &entities));
+  for (const Value* entity : entities) {
+    // Find the desired entity.
+    string type;
+    RETURN_NOT_OK(r.ExtractString(entity, "type", &type));
+    if (type != entity_proto->name()) {
+      continue;
+    }
+    if (entity_id) {
+      string id;
+      RETURN_NOT_OK(r.ExtractString(entity, "id", &id));
+      if (id != entity_id) {
+        continue;
+      }
+    }
+
+    // Find the desired metric within the entity.
+    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));
+      if (name != metric_proto->name()) {
+        continue;
+      }
+      RETURN_NOT_OK(r.ExtractInt64(metric, value_field, value));
+      return Status::OK();
+    }
+  }
+  string msg;
+  if (entity_id) {
+    msg = Substitute("Could not find metric $0.$1 for entity $2",
+                     entity_proto->name(), metric_proto->name(),
+                     entity_id);
+  } else {
+    msg = Substitute("Could not find metric $0.$1",
+                     entity_proto->name(), metric_proto->name());
+  }
+  return Status::NotFound(msg);
+}
+
 } // namespace itest
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/cluster_itest_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/cluster_itest_util.h b/src/kudu/integration-tests/cluster_itest_util.h
index c5b7ff8..63c28c7 100644
--- a/src/kudu/integration-tests/cluster_itest_util.h
+++ b/src/kudu/integration-tests/cluster_itest_util.h
@@ -50,6 +50,8 @@
 
 namespace kudu {
 class HostPort;
+class MetricEntityPrototype;
+class MetricPrototype;
 class MonoDelta;
 class Status;
 
@@ -386,6 +388,23 @@ Status BeginTabletCopySession(const TServerDetails* ts,
                               const MonoDelta& timeout,
                               tserver::TabletCopyErrorPB::Code* error_code = nullptr);
 
+// Retrieve the value of a given metric from the server whose webserver is
+// bound to 'http_hp'. The metric must be of int64_t type.
+//
+// 'value_field' represents the particular field of the metric to be read.
+// For example, for a counter or gauge, this should be 'value'. For a
+// histogram, it might be 'total_count' or 'mean'.
+//
+// 'entity_id' may be NULL, in which case the first entity of the same type
+// as 'entity_proto' will be matched.
+Status GetInt64Metric(const HostPort& http_hp,
+                      const MetricEntityPrototype* entity_proto,
+                      const char* entity_id,
+                      const MetricPrototype* metric_proto,
+                      const char* value_field,
+                      int64_t* value);
+
+
 } // namespace itest
 } // namespace kudu
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/create-table-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/create-table-itest.cc b/src/kudu/integration-tests/create-table-itest.cc
index 257463d..b223156 100644
--- a/src/kudu/integration-tests/create-table-itest.cc
+++ b/src/kudu/integration-tests/create-table-itest.cc
@@ -41,6 +41,7 @@
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/mathlimits.h"
 #include "kudu/gutil/ref_counted.h"
+#include "kudu/integration-tests/cluster_itest_util.h"
 #include "kudu/integration-tests/external_mini_cluster-itest-base.h"
 #include "kudu/integration-tests/external_mini_cluster_fs_inspector.h"
 #include "kudu/master/master.pb.h"
@@ -104,7 +105,8 @@ TEST_F(CreateTableITest, TestCreateWhenMajorityOfReplicasFailCreation) {
   int64_t num_create_attempts = 0;
   while (num_create_attempts < 3) {
     SleepFor(MonoDelta::FromMilliseconds(100));
-    ASSERT_OK(cluster_->tablet_server(0)->GetInt64Metric(
+    ASSERT_OK(itest::GetInt64Metric(
+        cluster_->tablet_server(0)->bound_http_hostport(),
         &METRIC_ENTITY_server,
         "kudu.tabletserver",
         &METRIC_handler_latency_kudu_tserver_TabletServerAdminService_CreateTablet,

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/delete_table-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/delete_table-itest.cc b/src/kudu/integration-tests/delete_table-itest.cc
index 605430a..2620b4f 100644
--- a/src/kudu/integration-tests/delete_table-itest.cc
+++ b/src/kudu/integration-tests/delete_table-itest.cc
@@ -93,6 +93,7 @@ using kudu::consensus::RaftPeerPB;
 using kudu::itest::AddServer;
 using kudu::itest::DeleteTablet;
 using kudu::itest::DeleteTabletWithRetries;
+using kudu::itest::GetInt64Metric;
 using kudu::itest::RemoveServer;
 using kudu::itest::TServerDetails;
 using kudu::pb_util::SecureDebugString;
@@ -1111,13 +1112,15 @@ TEST_F(DeleteTableITest, TestUnknownTabletsAreNotDeleted) {
   int64_t num_delete_attempts;
   ASSERT_EVENTUALLY([&]() {
     int64_t num_heartbeats;
-    ASSERT_OK(cluster_->master()->GetInt64Metric(
+    ASSERT_OK(GetInt64Metric(
+        cluster_->master()->bound_http_hostport(),
         &METRIC_ENTITY_server, "kudu.master",
         &METRIC_handler_latency_kudu_master_MasterService_TSHeartbeat, "total_count",
         &num_heartbeats));
     ASSERT_GE(num_heartbeats, 1);
   });
-  ASSERT_OK(cluster_->tablet_server(0)->GetInt64Metric(
+  ASSERT_OK(GetInt64Metric(
+      cluster_->tablet_server(0)->bound_http_hostport(),
       &METRIC_ENTITY_server, "kudu.tabletserver",
       &METRIC_handler_latency_kudu_tserver_TabletServerAdminService_DeleteTablet,
       "total_count", &num_delete_attempts));
@@ -1201,7 +1204,8 @@ TEST_F(DeleteTableITest, TestNoDeleteTombstonedTablets) {
     inspect_->WaitForNoDataOnTS(to_remove_index, kTimeout);
 
     int64_t num_delete_attempts;
-    ASSERT_OK(cluster_->tablet_server(to_remove_index)->GetInt64Metric(
+    ASSERT_OK(GetInt64Metric(
+        cluster_->tablet_server(to_remove_index)->bound_http_hostport(),
         &METRIC_ENTITY_server, "kudu.tabletserver",
         &METRIC_handler_latency_kudu_tserver_TabletServerAdminService_DeleteTablet,
         "total_count", &num_delete_attempts));
@@ -1228,7 +1232,8 @@ TEST_F(DeleteTableITest, TestNoDeleteTombstonedTablets) {
     SCOPED_TRACE(max_expected_deletes);
 
     int64_t prev_heartbeats;
-    ASSERT_OK(cluster_->master()->GetInt64Metric(
+    ASSERT_OK(GetInt64Metric(
+        cluster_->master()->bound_http_hostport(),
         &METRIC_ENTITY_server, "kudu.master",
         &METRIC_handler_latency_kudu_master_MasterService_TSHeartbeat,
         "total_count",
@@ -1239,7 +1244,8 @@ TEST_F(DeleteTableITest, TestNoDeleteTombstonedTablets) {
     SCOPED_LOG_TIMING(INFO, "waiting for heartbeats");
     ASSERT_EVENTUALLY([&] {
       int64_t num_delete_attempts;
-      ASSERT_OK(cluster_->tablet_server(to_remove_index)->GetInt64Metric(
+      ASSERT_OK(GetInt64Metric(
+          cluster_->tablet_server(to_remove_index)->bound_http_hostport(),
           &METRIC_ENTITY_server, "kudu.tabletserver",
           &METRIC_handler_latency_kudu_tserver_TabletServerAdminService_DeleteTablet,
           "total_count",
@@ -1247,7 +1253,8 @@ TEST_F(DeleteTableITest, TestNoDeleteTombstonedTablets) {
       ASSERT_LE(num_delete_attempts, max_expected_deletes);
 
       int64_t num_heartbeats;
-      ASSERT_OK(cluster_->master()->GetInt64Metric(
+      ASSERT_OK(GetInt64Metric(
+          cluster_->master()->bound_http_hostport(),
           &METRIC_ENTITY_server, "kudu.master",
           &METRIC_handler_latency_kudu_master_MasterService_TSHeartbeat,
           "total_count",

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/dense_node-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/dense_node-itest.cc b/src/kudu/integration-tests/dense_node-itest.cc
index 02f278c..e177d8d 100644
--- a/src/kudu/integration-tests/dense_node-itest.cc
+++ b/src/kudu/integration-tests/dense_node-itest.cc
@@ -31,6 +31,7 @@
 
 #include "kudu/client/schema.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/integration-tests/cluster_itest_util.h"
 #include "kudu/integration-tests/external_mini_cluster-itest-base.h"
 #include "kudu/integration-tests/test_workload.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
@@ -181,11 +182,9 @@ TEST_F(DenseNodeTest, RunTest) {
                          &METRIC_log_block_manager_full_containers,
                          &METRIC_threads_running }) {
     int64_t value;
-    ASSERT_OK(cluster_->tablet_server(0)->GetInt64Metric(&METRIC_ENTITY_server,
-                                 "kudu.tabletserver",
-                                 m,
-                                 "value",
-                                 &value));
+    ASSERT_OK(itest::GetInt64Metric(
+        cluster_->tablet_server(0)->bound_http_hostport(), &METRIC_ENTITY_server,
+        "kudu.tabletserver", m, "value", &value));
     metrics.emplace_back(m->name(), value);
   }
   cluster_->Shutdown();

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/disk_failure-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/disk_failure-itest.cc b/src/kudu/integration-tests/disk_failure-itest.cc
index 4bb888d..3657846 100644
--- a/src/kudu/integration-tests/disk_failure-itest.cc
+++ b/src/kudu/integration-tests/disk_failure-itest.cc
@@ -22,15 +22,17 @@
 
 #include <gtest/gtest.h>
 
+#include "kudu/fs/block_manager.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/integration-tests/cluster_itest_util.h"
 #include "kudu/integration-tests/cluster_verifier.h"
 #include "kudu/integration-tests/external_mini_cluster-itest-base.h"
 #include "kudu/integration-tests/test_workload.h"
-#include "kudu/fs/block_manager.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/path_util.h"
+#include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
@@ -56,7 +58,7 @@ class DiskFailureITest : public ExternalMiniClusterITestBase,
                            int64_t target_failed_disks = 1) const {
     ASSERT_EVENTUALLY([&] {
       int64_t failed_on_ts;
-      ASSERT_OK(ext_tserver->GetInt64Metric(
+      ASSERT_OK(itest::GetInt64Metric(ext_tserver->bound_http_hostport(),
           &METRIC_ENTITY_server, nullptr, &METRIC_data_dirs_failed, "value", &failed_on_ts));
       ASSERT_EQ(target_failed_disks, failed_on_ts);
     });

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/disk_reservation-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/disk_reservation-itest.cc b/src/kudu/integration-tests/disk_reservation-itest.cc
index 5b5d26b..5032762 100644
--- a/src/kudu/integration-tests/disk_reservation-itest.cc
+++ b/src/kudu/integration-tests/disk_reservation-itest.cc
@@ -26,6 +26,7 @@
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/integration-tests/cluster_itest_util.h"
 #include "kudu/integration-tests/external_mini_cluster-itest-base.h"
 #include "kudu/integration-tests/test_workload.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
@@ -48,12 +49,12 @@ using cluster::ExternalTabletServer;
 
 namespace {
 Status GetTsCounterValue(ExternalTabletServer* ets, MetricPrototype* metric, int64_t* value) {
-  return ets->GetInt64Metric(
-             &METRIC_ENTITY_server,
-             "kudu.tabletserver",
-             metric,
-             "value",
-             value);
+  return itest::GetInt64Metric(ets->bound_http_hostport(),
+                               &METRIC_ENTITY_server,
+                               "kudu.tabletserver",
+                               metric,
+                               "value",
+                               value);
 }
 } // namespace
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/master_failover-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_failover-itest.cc b/src/kudu/integration-tests/master_failover-itest.cc
index 01d9c32..a0015c4 100644
--- a/src/kudu/integration-tests/master_failover-itest.cc
+++ b/src/kudu/integration-tests/master_failover-itest.cc
@@ -33,6 +33,7 @@
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/integration-tests/cluster_itest_util.h"
 #include "kudu/master/sys_catalog.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
 #include "kudu/util/metrics.h"
@@ -60,6 +61,7 @@ using cluster::ExternalMaster;
 using cluster::ExternalMiniCluster;
 using cluster::ExternalMiniClusterOptions;
 using cluster::ScopedResumeExternalDaemon;
+using itest::GetInt64Metric;
 using sp::shared_ptr;
 using std::set;
 using std::string;
@@ -331,7 +333,8 @@ TEST_F(MasterFailoverTest, TestMasterUUIDResolution) {
   // their UUIDs.
   for (int i = 0; i < cluster_->num_masters(); i++) {
     int64_t num_get_node_instances;
-    ASSERT_OK(cluster_->master(i)->GetInt64Metric(
+    ASSERT_OK(GetInt64Metric(
+        cluster_->master(i)->bound_http_hostport(),
         &METRIC_ENTITY_server, "kudu.master",
         &METRIC_handler_latency_kudu_consensus_ConsensusService_GetNodeInstance,
         "total_count", &num_get_node_instances));
@@ -351,7 +354,8 @@ TEST_F(MasterFailoverTest, TestMasterUUIDResolution) {
   for (int i = 0; i < cluster_->num_masters(); i++) {
     ExternalMaster* master = cluster_->master(i);
     int64_t num_get_node_instances;
-    ASSERT_OK(master->GetInt64Metric(
+    ASSERT_OK(GetInt64Metric(
+        master->bound_http_hostport(),
         &METRIC_ENTITY_server, "kudu.master",
         &METRIC_handler_latency_kudu_consensus_ConsensusService_GetNodeInstance,
         "total_count", &num_get_node_instances));

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/raft_consensus-itest-base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest-base.cc b/src/kudu/integration-tests/raft_consensus-itest-base.cc
index 08b2871..1c7a0e1 100644
--- a/src/kudu/integration-tests/raft_consensus-itest-base.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest-base.cc
@@ -75,6 +75,7 @@ using kudu::client::KuduTable;
 using kudu::client::sp::shared_ptr;
 using kudu::cluster::ExternalTabletServer;
 using kudu::consensus::OpId;
+using kudu::itest::GetInt64Metric;
 using kudu::itest::TServerDetails;
 using kudu::pb_util::SecureDebugString;
 using kudu::rpc::RpcController;
@@ -270,8 +271,9 @@ void RaftConsensusITestBase::CauseFollowerToFallBehindLogGC(
 
 Status RaftConsensusITestBase::GetTermMetricValue(ExternalTabletServer* ts,
                                                   int64_t *term) {
-  return ts->GetInt64Metric(&METRIC_ENTITY_tablet, nullptr, &METRIC_raft_term,
-                            "value", term);
+  return GetInt64Metric(ts->bound_http_hostport(),
+                        &METRIC_ENTITY_tablet, nullptr, &METRIC_raft_term,
+                        "value", term);
 }
 
 }  // namespace tserver

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/raft_consensus-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc b/src/kudu/integration-tests/raft_consensus-itest.cc
index 1400906..e41a331 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -105,6 +105,7 @@ using kudu::consensus::RaftPeerPB;
 using kudu::consensus::ReplicateMsg;
 using kudu::itest::AddServer;
 using kudu::itest::DONT_WAIT_FOR_LEADER;
+using kudu::itest::GetInt64Metric;
 using kudu::itest::LeaderStepDown;
 using kudu::itest::RemoveServer;
 using kudu::itest::StartElection;
@@ -1964,7 +1965,8 @@ TEST_F(RaftConsensusITest, TestMemoryRemainsConstantDespiteTwoDeadFollowers) {
   MonoTime deadline = MonoTime::Now() + kMaxWaitTime;
   while (true) {
     int64_t num_rejections = 0;
-    ASSERT_OK(cluster_->tablet_server(leader_ts_idx)->GetInt64Metric(
+    ASSERT_OK(GetInt64Metric(
+        cluster_->tablet_server(leader_ts_idx)->bound_http_hostport(),
         &METRIC_ENTITY_tablet,
         nullptr,
         &METRIC_transaction_memory_pressure_rejections,

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc b/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc
index 52109ac..b4287d7 100644
--- a/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc
@@ -54,6 +54,7 @@ using kudu::cluster::ExternalDaemon;
 using kudu::cluster::ExternalTabletServer;
 using kudu::consensus::RaftPeerPB;
 using kudu::itest::AddServer;
+using kudu::itest::GetInt64Metric;
 using kudu::itest::LeaderStepDown;
 using kudu::itest::RemoveServer;
 using kudu::itest::StartElection;
@@ -100,14 +101,14 @@ class RaftConsensusNonVoterITest : public RaftConsensusITestBase {
 
 Status RaftConsensusNonVoterITest::GetTabletCopySourceSessionsCount(
     const ExternalDaemon& server, int64_t* count) {
-  return server.GetInt64Metric(
+  return GetInt64Metric(server.bound_http_hostport(),
       &METRIC_ENTITY_server, "kudu.tabletserver",
       &METRIC_tablet_copy_open_source_sessions, "value", count);
 }
 
 Status RaftConsensusNonVoterITest::GetTabletCopyTargetSessionsCount(
     const ExternalDaemon& server, int64_t* count) {
-  return server.GetInt64Metric(
+  return GetInt64Metric(server.bound_http_hostport(),
       &METRIC_ENTITY_server, "kudu.tabletserver",
       &METRIC_tablet_copy_open_client_sessions, "value", count);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/tablet_copy-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/tablet_copy-itest.cc b/src/kudu/integration-tests/tablet_copy-itest.cc
index 97ff48e..44f62d2 100644
--- a/src/kudu/integration-tests/tablet_copy-itest.cc
+++ b/src/kudu/integration-tests/tablet_copy-itest.cc
@@ -103,6 +103,7 @@ using kudu::consensus::MakeOpId;
 using kudu::consensus::RaftPeerPB;
 using kudu::itest::AddServer;
 using kudu::itest::DeleteTablet;
+using kudu::itest::GetInt64Metric;
 using kudu::itest::FindTabletLeader;
 using kudu::itest::StartElection;
 using kudu::itest::StartTabletCopy;
@@ -716,52 +717,57 @@ namespace {
 // LogBlockManager (not used on OS X).
 int64_t CountBlocksUnderManagement(ExternalTabletServer* ets) {
   int64_t ret;
-  CHECK_OK(ets->GetInt64Metric(
-               &METRIC_ENTITY_server,
-               "kudu.tabletserver",
-               &METRIC_log_block_manager_blocks_under_management,
-               "value",
-               &ret));
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
+      &METRIC_ENTITY_server,
+      "kudu.tabletserver",
+      &METRIC_log_block_manager_blocks_under_management,
+      "value",
+      &ret));
   return ret;
 }
 #endif // #ifndef __APPLE__
 
 int64_t CountUpdateConsensusCalls(ExternalTabletServer* ets) {
   int64_t ret;
-  CHECK_OK(ets->GetInt64Metric(
-               &METRIC_ENTITY_server,
-               "kudu.tabletserver",
-               &METRIC_handler_latency_kudu_consensus_ConsensusService_UpdateConsensus,
-               "total_count",
-               &ret));
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
+      &METRIC_ENTITY_server,
+      "kudu.tabletserver",
+      &METRIC_handler_latency_kudu_consensus_ConsensusService_UpdateConsensus,
+      "total_count",
+      &ret));
   return ret;
 }
 int64_t CountLogMessages(ExternalTabletServer* ets) {
   int64_t total = 0;
 
   int64_t count;
-  CHECK_OK(ets->GetInt64Metric(
-               &METRIC_ENTITY_server,
-               "kudu.tabletserver",
-               &METRIC_glog_info_messages,
-               "value",
-               &count));
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
+      &METRIC_ENTITY_server,
+      "kudu.tabletserver",
+      &METRIC_glog_info_messages,
+      "value",
+      &count));
   total += count;
 
-  CHECK_OK(ets->GetInt64Metric(
-               &METRIC_ENTITY_server,
-               "kudu.tabletserver",
-               &METRIC_glog_warning_messages,
-               "value",
-               &count));
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
+      &METRIC_ENTITY_server,
+      "kudu.tabletserver",
+      &METRIC_glog_warning_messages,
+      "value",
+      &count));
   total += count;
 
-  CHECK_OK(ets->GetInt64Metric(
-               &METRIC_ENTITY_server,
-               "kudu.tabletserver",
-               &METRIC_glog_error_messages,
-               "value",
-               &count));
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
+      &METRIC_ENTITY_server,
+      "kudu.tabletserver",
+      &METRIC_glog_error_messages,
+      "value",
+      &count));
   total += count;
 
   return total;
@@ -1478,7 +1484,8 @@ TEST_P(BadTabletCopyITest, TestBadCopy) {
 namespace {
 int64_t TabletCopyBytesSent(ExternalTabletServer* ets) {
   int64_t ret;
-  CHECK_OK(ets->GetInt64Metric(
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
       &METRIC_ENTITY_server,
       "kudu.tabletserver",
       &METRIC_tablet_copy_bytes_sent,
@@ -1489,7 +1496,8 @@ int64_t TabletCopyBytesSent(ExternalTabletServer* ets) {
 
 int64_t TabletCopyBytesFetched(ExternalTabletServer* ets) {
   int64_t ret;
-  CHECK_OK(ets->GetInt64Metric(
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
       &METRIC_ENTITY_server,
       "kudu.tabletserver",
       &METRIC_tablet_copy_bytes_fetched,
@@ -1500,7 +1508,8 @@ int64_t TabletCopyBytesFetched(ExternalTabletServer* ets) {
 
 int64_t TabletCopyOpenSourceSessions(ExternalTabletServer* ets) {
   int64_t ret;
-  CHECK_OK(ets->GetInt64Metric(
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
       &METRIC_ENTITY_server,
       "kudu.tabletserver",
       &METRIC_tablet_copy_open_source_sessions,
@@ -1511,7 +1520,8 @@ int64_t TabletCopyOpenSourceSessions(ExternalTabletServer* ets) {
 
 int64_t TabletCopyOpenClientSessions(ExternalTabletServer* ets) {
   int64_t ret;
-  CHECK_OK(ets->GetInt64Metric(
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
       &METRIC_ENTITY_server,
       "kudu.tabletserver",
       &METRIC_tablet_copy_open_client_sessions,
@@ -1600,7 +1610,8 @@ TEST_F(TabletCopyITest, TestTabletCopyMetrics) {
 namespace {
 int64_t CountRunningTablets(ExternalTabletServer *ets) {
   int64_t ret;
-  CHECK_OK(ets->GetInt64Metric(
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
       &METRIC_ENTITY_server,
       nullptr,
       &METRIC_tablets_num_running,
@@ -1611,7 +1622,8 @@ int64_t CountRunningTablets(ExternalTabletServer *ets) {
 
 int64_t CountBootstrappingTablets(ExternalTabletServer *ets) {
   int64_t ret;
-  CHECK_OK(ets->GetInt64Metric(
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
       &METRIC_ENTITY_server,
       nullptr,
       &METRIC_tablets_num_bootstrapping,
@@ -1622,7 +1634,8 @@ int64_t CountBootstrappingTablets(ExternalTabletServer *ets) {
 
 int64_t CountRowsInserted(ExternalTabletServer *ets) {
   int64_t ret;
-  CHECK_OK(ets->GetInt64Metric(
+  CHECK_OK(GetInt64Metric(
+      ets->bound_http_hostport(),
       &METRIC_ENTITY_tablet,
       nullptr,
       &METRIC_rows_inserted,

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/integration-tests/tombstoned_voting-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/tombstoned_voting-itest.cc b/src/kudu/integration-tests/tombstoned_voting-itest.cc
index 575e493..69c48e4 100644
--- a/src/kudu/integration-tests/tombstoned_voting-itest.cc
+++ b/src/kudu/integration-tests/tombstoned_voting-itest.cc
@@ -45,6 +45,7 @@ METRIC_DECLARE_histogram(handler_latency_kudu_master_MasterService_TSHeartbeat);
 using kudu::cluster::ExternalMaster;
 using kudu::cluster::ExternalTabletServer;
 using kudu::consensus::MakeOpId;
+using kudu::itest::GetInt64Metric;
 using kudu::itest::TServerDetails;
 using kudu::tablet::TABLET_DATA_COPYING;
 using kudu::tablet::TABLET_DATA_TOMBSTONED;
@@ -58,11 +59,12 @@ class TombstonedVotingITest : public ExternalMiniClusterITestBase {
 };
 
 Status CountMasterHeartbeatsRecvd(ExternalMaster* m, int64_t* count) {
-  return m->GetInt64Metric(&METRIC_ENTITY_server,
-                           "kudu.master",
-                           &METRIC_handler_latency_kudu_master_MasterService_TSHeartbeat,
-                           "total_count",
-                           count);
+  return GetInt64Metric(m->bound_http_hostport(),
+                        &METRIC_ENTITY_server,
+                        "kudu.master",
+                        &METRIC_handler_latency_kudu_master_MasterService_TSHeartbeat,
+                        "total_count",
+                        count);
 }
 
 // Test that a replica that crashes during a first-time tablet copy after

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/mini-cluster/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/CMakeLists.txt b/src/kudu/mini-cluster/CMakeLists.txt
index f1cc253..57f945d 100644
--- a/src/kudu/mini-cluster/CMakeLists.txt
+++ b/src/kudu/mini-cluster/CMakeLists.txt
@@ -29,7 +29,6 @@ target_link_libraries(mini_cluster
   gutil
   kudu_client
   kudu_common
-  kudu_curl_util
   kudu_test_util
   kudu_util
   krpc

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/mini-cluster/external_mini_cluster-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/external_mini_cluster-test.cc b/src/kudu/mini-cluster/external_mini_cluster-test.cc
index 584591a..d257ba0 100644
--- a/src/kudu/mini-cluster/external_mini_cluster-test.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster-test.cc
@@ -15,7 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <cstdint>
 #include <string>
 #include <utility>
 #include <vector>
@@ -28,15 +27,12 @@
 #include "kudu/mini-cluster/external_mini_cluster.h"
 #include "kudu/mini-cluster/mini_cluster.h"
 #include "kudu/security/test/mini_kdc.h"
-#include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-METRIC_DECLARE_gauge_uint64(threads_running);
-
 namespace kudu {
 
 namespace cluster {
@@ -119,15 +115,6 @@ TEST_P(ExternalMiniClusterTest, TestBasicOperation) {
 
     HostPort master_http = master->bound_http_hostport();
     EXPECT_TRUE(HasPrefixString(master_http.ToString(), expected_prefix)) << master_http.ToString();
-
-    // Retrieve a thread metric, which should always be present on any master.
-    int64_t value;
-    ASSERT_OK(master->GetInt64Metric(&METRIC_ENTITY_server,
-                                     "kudu.master",
-                                     &METRIC_threads_running,
-                                     "value",
-                                     &value));
-    EXPECT_GT(value, 0);
   }
 
   // Verify each of the tablet servers.
@@ -143,15 +130,6 @@ TEST_P(ExternalMiniClusterTest, TestBasicOperation) {
 
     HostPort ts_http = ts->bound_http_hostport();
     EXPECT_TRUE(HasPrefixString(ts_http.ToString(), expected_prefix)) << ts_http.ToString();
-
-    // Retrieve a thread metric, which should always be present on any TS.
-    int64_t value;
-    ASSERT_OK(ts->GetInt64Metric(&METRIC_ENTITY_server,
-                                 "kudu.tabletserver",
-                                 &METRIC_threads_running,
-                                 "value",
-                                 &value));
-    EXPECT_GT(value, 0);
   }
 
   // Ensure that all of the tablet servers can register with the masters.

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/mini-cluster/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index 7609376..25ce80f 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -30,7 +30,6 @@
 #include <gflags/gflags.h>
 #include <gflags/gflags_declare.h>
 #include <gtest/gtest.h>
-#include <rapidjson/document.h>
 
 #include "kudu/client/client.h"
 #include "kudu/client/master_rpc.h"
@@ -52,13 +51,9 @@
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/tserver/tserver_service.proxy.h"
 #include "kudu/util/async_util.h"
-#include "kudu/util/curl_util.h"
 #include "kudu/util/env.h"
 #include "kudu/util/env_util.h"
-#include "kudu/util/faststring.h"
 #include "kudu/util/fault_injection.h"
-#include "kudu/util/jsonreader.h"
-#include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/path_util.h"
@@ -79,7 +74,6 @@ using kudu::server::ServerStatusPB;
 using kudu::tserver::ListTabletsRequestPB;
 using kudu::tserver::ListTabletsResponsePB;
 using kudu::tserver::TabletServerServiceProxy;
-using rapidjson::Value;
 using std::pair;
 using std::string;
 using std::unique_ptr;
@@ -1056,66 +1050,6 @@ const string& ExternalDaemon::uuid() const {
   return status_->node_instance().permanent_uuid();
 }
 
-Status ExternalDaemon::GetInt64Metric(const MetricEntityPrototype* entity_proto,
-                                      const char* entity_id,
-                                      const MetricPrototype* metric_proto,
-                                      const char* value_field,
-                                      int64_t* value) const {
-  CHECK(bound_http_hostport().Initialized());
-  // Fetch metrics whose name matches the given prototype.
-  string url = Substitute(
-      "http://$0/jsonmetricz?metrics=$1",
-      bound_http_hostport().ToString(),
-      metric_proto->name());
-  EasyCurl curl;
-  faststring dst;
-  RETURN_NOT_OK(curl.FetchURL(url, &dst));
-
-  // Parse the results, beginning with the top-level entity array.
-  JsonReader r(dst.ToString());
-  RETURN_NOT_OK(r.Init());
-  vector<const Value*> entities;
-  RETURN_NOT_OK(r.ExtractObjectArray(r.root(), NULL, &entities));
-  for (const Value* entity : entities) {
-    // Find the desired entity.
-    string type;
-    RETURN_NOT_OK(r.ExtractString(entity, "type", &type));
-    if (type != entity_proto->name()) {
-      continue;
-    }
-    if (entity_id) {
-      string id;
-      RETURN_NOT_OK(r.ExtractString(entity, "id", &id));
-      if (id != entity_id) {
-        continue;
-      }
-    }
-
-    // Find the desired metric within the entity.
-    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));
-      if (name != metric_proto->name()) {
-        continue;
-      }
-      RETURN_NOT_OK(r.ExtractInt64(metric, value_field, value));
-      return Status::OK();
-    }
-  }
-  string msg;
-  if (entity_id) {
-    msg = Substitute("Could not find metric $0.$1 for entity $2",
-                     entity_proto->name(), metric_proto->name(),
-                     entity_id);
-  } else {
-    msg = Substitute("Could not find metric $0.$1",
-                     entity_proto->name(), metric_proto->name());
-  }
-  return Status::NotFound(msg);
-}
-
 //------------------------------------------------------------
 // ScopedResumeExternalDaemon
 //------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/mini-cluster/external_mini_cluster.h
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/external_mini_cluster.h b/src/kudu/mini-cluster/external_mini_cluster.h
index 8d5e9c3..789467f 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.h
+++ b/src/kudu/mini-cluster/external_mini_cluster.h
@@ -43,8 +43,6 @@
 
 namespace kudu {
 
-class MetricEntityPrototype;
-class MetricPrototype;
 class NodeInstancePB;
 class Sockaddr;
 class Subprocess;
@@ -456,21 +454,6 @@ class ExternalDaemon : public RefCountedThreadSafe<ExternalDaemon> {
   // Modifying these flags will only take effect on the next restart.
   std::vector<std::string>* mutable_flags() { return &opts_.extra_flags; }
 
-  // Retrieve the value of a given metric from this server. The metric must
-  // be of int64_t type.
-  //
-  // 'value_field' represents the particular field of the metric to be read.
-  // For example, for a counter or gauge, this should be 'value'. For a
-  // histogram, it might be 'total_count' or 'mean'.
-  //
-  // 'entity_id' may be NULL, in which case the first entity of the same type
-  // as 'entity_proto' will be matched.
-  Status GetInt64Metric(const MetricEntityPrototype* entity_proto,
-                        const char* entity_id,
-                        const MetricPrototype* metric_proto,
-                        const char* value_field,
-                        int64_t* value) const;
-
  protected:
   friend class RefCountedThreadSafe<ExternalDaemon>;
   virtual ~ExternalDaemon();

http://git-wip-us.apache.org/repos/asf/kudu/blob/09ac5bfd/src/kudu/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index ec71364..a6e0497 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -285,15 +285,15 @@ endif()
 #######################################
 # kudu_curl_util
 #######################################
-
-# Used by mini-cluster, so must be built even when NO_TESTS=0.
-add_library(kudu_curl_util
-  curl_util.cc)
-target_link_libraries(kudu_curl_util
-  security
-  ${CURL_LIBRARIES}
-  glog
-  gutil)
+if(NOT NO_TESTS)
+  add_library(kudu_curl_util
+    curl_util.cc)
+  target_link_libraries(kudu_curl_util
+    security
+    ${CURL_LIBRARIES}
+    glog
+    gutil)
+endif()
 
 #######################################
 # kudu_test_main