You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2020/03/05 08:07:18 UTC

[kudu] 02/02: subprocess: report metrics even when calls fail

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

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

commit 0702fc5e099478ceba9e3f6f6b25f2ffae2afc82
Author: Andrew Wong <aw...@cloudera.com>
AuthorDate: Wed Mar 4 16:50:55 2020 -0800

    subprocess: report metrics even when calls fail
    
    This moves metric reporting for metrics returned by the subprocess into
    the SubprocessServer so they are still reported in the event that the
    call fails.
    
    To do this, I moved the SubprocessMetrics into the SubprocessServer,
    which means that these metrics are now required by the server. As such,
    I moved the declarations of Echo metrics out of subprocess_proxy-test so
    they could be reused by other tests.
    
    This should prove more useful, particularly given the fact that failed
    calls warrant a deeper look at these metrics.
    
    Change-Id: I87cc82f82a0a767383284799ec6159d680526ea9
    Reviewed-on: http://gerrit.cloudera.org:8080/15366
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/ranger/ranger_client-test.cc         |  5 +-
 src/kudu/subprocess/CMakeLists.txt            |  8 +++
 src/kudu/subprocess/echo_subprocess.cc        | 68 +++++++++++++++++++
 src/kudu/subprocess/echo_subprocess.h         | 39 +++++++++++
 src/kudu/subprocess/server.cc                 | 15 ++++-
 src/kudu/subprocess/server.h                  | 14 +++-
 src/kudu/subprocess/subprocess_proxy-test.cc  | 96 ++++++++++++---------------
 src/kudu/subprocess/subprocess_proxy.h        | 33 +--------
 src/kudu/subprocess/subprocess_server-test.cc | 11 ++-
 9 files changed, 198 insertions(+), 91 deletions(-)

diff --git a/src/kudu/ranger/ranger_client-test.cc b/src/kudu/ranger/ranger_client-test.cc
index 4213593..b2cae2c 100644
--- a/src/kudu/ranger/ranger_client-test.cc
+++ b/src/kudu/ranger/ranger_client-test.cc
@@ -42,6 +42,7 @@ namespace kudu {
 namespace ranger {
 
 using boost::hash_combine;
+using kudu::subprocess::SubprocessMetrics;
 using kudu::subprocess::SubprocessRequestPB;
 using kudu::subprocess::SubprocessResponsePB;
 using kudu::subprocess::SubprocessServer;
@@ -91,8 +92,8 @@ class MockSubprocessServer : public SubprocessServer {
 
   ~MockSubprocessServer() override {}
 
-  MockSubprocessServer() :
-    SubprocessServer({"mock"}) {}
+  MockSubprocessServer()
+      : SubprocessServer({"mock"}, SubprocessMetrics()) {}
 
   Status Execute(SubprocessRequestPB* req,
                  SubprocessResponsePB* resp) override {
diff --git a/src/kudu/subprocess/CMakeLists.txt b/src/kudu/subprocess/CMakeLists.txt
index a7514a4..ceab0ca 100644
--- a/src/kudu/subprocess/CMakeLists.txt
+++ b/src/kudu/subprocess/CMakeLists.txt
@@ -59,6 +59,13 @@ target_link_libraries(kudu_subprocess
 )
 add_dependencies(kudu_subprocess subprocess_jar)
 
+add_library(echo_subprocess
+  echo_subprocess.cc
+)
+target_link_libraries(echo_subprocess
+  kudu_subprocess
+)
+
 #######################################
 # Unit tests
 #######################################
@@ -66,6 +73,7 @@ add_dependencies(kudu_subprocess subprocess_jar)
 if (NOT NO_TESTS)
   SET_KUDU_TEST_LINK_LIBS(
     kudu_subprocess
+    echo_subprocess
   )
 endif()
 
diff --git a/src/kudu/subprocess/echo_subprocess.cc b/src/kudu/subprocess/echo_subprocess.cc
new file mode 100644
index 0000000..9faac0e
--- /dev/null
+++ b/src/kudu/subprocess/echo_subprocess.cc
@@ -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.
+
+#include "kudu/subprocess/echo_subprocess.h"
+
+#include "kudu/util/metrics.h"
+
+METRIC_DEFINE_histogram(server, echo_subprocess_inbound_queue_length,
+    "Echo subprocess inbound queue length",
+    kudu::MetricUnit::kMessages,
+    "Number of request messages in the Echo subprocess' inbound request queue",
+    kudu::MetricLevel::kInfo,
+    1000, 1);
+METRIC_DEFINE_histogram(server, echo_subprocess_outbound_queue_length,
+    "Echo subprocess outbound queue length",
+    kudu::MetricUnit::kMessages,
+    "Number of request messages in the Echo subprocess' outbound response queue",
+    kudu::MetricLevel::kInfo,
+    1000, 1);
+METRIC_DEFINE_histogram(server, echo_subprocess_inbound_queue_time_ms,
+    "Echo subprocess inbound queue time (ms)",
+    kudu::MetricUnit::kMilliseconds,
+    "Duration of time in ms spent in the Echo subprocess' inbound request queue",
+    kudu::MetricLevel::kInfo,
+    60000LU, 1);
+METRIC_DEFINE_histogram(server, echo_subprocess_outbound_queue_time_ms,
+    "Echo subprocess outbound queue time (ms)",
+    kudu::MetricUnit::kMilliseconds,
+    "Duration of time in ms spent in the Echo subprocess' outbound response queue",
+    kudu::MetricLevel::kInfo,
+    60000LU, 1);
+METRIC_DEFINE_histogram(server, echo_subprocess_execution_time_ms,
+    "Echo subprocess execution time (ms)",
+    kudu::MetricUnit::kMilliseconds,
+    "Duration of time in ms spent executing the Echo subprocess request, excluding "
+    "time spent spent in the subprocess queues",
+    kudu::MetricLevel::kInfo,
+    60000LU, 1);
+
+namespace kudu {
+namespace subprocess {
+
+#define HISTINIT(member, x) member = METRIC_##x.Instantiate(entity)
+EchoSubprocessMetrics::EchoSubprocessMetrics(const scoped_refptr<MetricEntity>& entity) {
+  HISTINIT(inbound_queue_length, echo_subprocess_inbound_queue_length);
+  HISTINIT(outbound_queue_length, echo_subprocess_outbound_queue_length);
+  HISTINIT(inbound_queue_time_ms, echo_subprocess_inbound_queue_time_ms);
+  HISTINIT(outbound_queue_time_ms, echo_subprocess_outbound_queue_time_ms);
+  HISTINIT(execution_time_ms, echo_subprocess_execution_time_ms);
+}
+#undef HISTINIT
+
+} // namespace subprocess
+} // namespace kudu
diff --git a/src/kudu/subprocess/echo_subprocess.h b/src/kudu/subprocess/echo_subprocess.h
new file mode 100644
index 0000000..c78170b
--- /dev/null
+++ b/src/kudu/subprocess/echo_subprocess.h
@@ -0,0 +1,39 @@
+// 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 "kudu/gutil/ref_counted.h"
+#include "kudu/subprocess/server.h"
+#include "kudu/subprocess/subprocess_proxy.h"
+
+namespace kudu {
+
+class MetricEntity;
+
+namespace subprocess {
+
+class EchoRequestPB;
+class EchoResponsePB;
+
+struct EchoSubprocessMetrics : public SubprocessMetrics {
+  explicit EchoSubprocessMetrics(const scoped_refptr<MetricEntity>& entity);
+};
+
+typedef SubprocessProxy<EchoRequestPB, EchoResponsePB, EchoSubprocessMetrics> EchoSubprocess;
+
+} // namespace subprocess
+} // namespace kudu
diff --git a/src/kudu/subprocess/server.cc b/src/kudu/subprocess/server.cc
index 6d81d27..f23c1af 100644
--- a/src/kudu/subprocess/server.cc
+++ b/src/kudu/subprocess/server.cc
@@ -83,13 +83,14 @@ using strings::Substitute;
 namespace kudu {
 namespace subprocess {
 
-SubprocessServer::SubprocessServer(vector<string> subprocess_argv)
+SubprocessServer::SubprocessServer(vector<string> subprocess_argv, SubprocessMetrics metrics)
     : call_timeout_(MonoDelta::FromSeconds(FLAGS_subprocess_timeout_secs)),
       next_id_(1),
       closing_(1),
       process_(make_shared<Subprocess>(std::move(subprocess_argv))),
       outbound_call_queue_(FLAGS_subprocess_request_queue_size_bytes),
-      inbound_response_queue_(FLAGS_subprocess_response_queue_size_bytes) {
+      inbound_response_queue_(FLAGS_subprocess_response_queue_size_bytes),
+      metrics_(std::move(metrics)) {
   process_->ShareParentStdin(false);
   process_->ShareParentStdout(false);
 }
@@ -221,6 +222,16 @@ void SubprocessServer::ResponderThread() {
         LOG(FATAL) << Substitute("Received invalid response: $0",
                                  pb_util::SecureDebugString(resp));
       }
+      // Regardless of whether this call succeeded or not, parse the returned
+      // metrics.
+      if (PREDICT_TRUE(resp.has_metrics())) {
+        const auto& pb = resp.metrics();
+        metrics_.inbound_queue_length->Increment(pb.inbound_queue_length());
+        metrics_.outbound_queue_length->Increment(pb.outbound_queue_length());
+        metrics_.inbound_queue_time_ms->Increment(pb.inbound_queue_time_ms());
+        metrics_.outbound_queue_time_ms->Increment(pb.outbound_queue_time_ms());
+        metrics_.execution_time_ms->Increment(pb.execution_time_ms());
+      }
     }
     vector<pair<shared_ptr<SubprocessCall>, SubprocessResponsePB>> calls_and_resps;
     calls_and_resps.reserve(resps.size());
diff --git a/src/kudu/subprocess/server.h b/src/kudu/subprocess/server.h
index c175e22..d4e0ebe 100644
--- a/src/kudu/subprocess/server.h
+++ b/src/kudu/subprocess/server.h
@@ -39,6 +39,7 @@
 #include "kudu/util/blocking_queue.h"
 #include "kudu/util/countdown_latch.h"
 #include "kudu/util/locks.h"
+#include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/mutex.h"
 #include "kudu/util/status.h"
@@ -53,6 +54,14 @@ namespace subprocess {
 
 typedef int64_t CallId;
 
+struct SubprocessMetrics {
+  scoped_refptr<Histogram> inbound_queue_length;
+  scoped_refptr<Histogram> outbound_queue_length;
+  scoped_refptr<Histogram> inbound_queue_time_ms;
+  scoped_refptr<Histogram> outbound_queue_time_ms;
+  scoped_refptr<Histogram> execution_time_ms;
+};
+
 // Encapsulates the pending state of a request that is in the process of being
 // sent to a subprocess. These calls are added to an in-flight map before
 // calling SendRequest(). See the method comments for some discussion about
@@ -188,7 +197,7 @@ typedef BlockingQueue<SubprocessResponsePB, ResponseLogicalSize> ResponseQueue;
 // Public methods are virtual so a mock server can be used in tests.
 class SubprocessServer {
  public:
-  explicit SubprocessServer(std::vector<std::string> subprocess_argv);
+  SubprocessServer(std::vector<std::string> subprocess_argv, SubprocessMetrics metrics);
   virtual ~SubprocessServer();
 
   // Initialize the server, starting the subprocess and worker threads.
@@ -268,6 +277,9 @@ class SubprocessServer {
   // Inbound queue of responses sent by the subprocess.
   ResponseQueue inbound_response_queue_;
 
+  // Metrics for this subprocess.
+  SubprocessMetrics metrics_;
+
   // Calls that are currently in-flight (the requests are being sent over the
   // pipe or waiting for a response), ordered by ID. This ordering allows for
   // lookup by ID, and gives us a rough way to get the calls with earliest
diff --git a/src/kudu/subprocess/subprocess_proxy-test.cc b/src/kudu/subprocess/subprocess_proxy-test.cc
index 958807e..897dd18 100644
--- a/src/kudu/subprocess/subprocess_proxy-test.cc
+++ b/src/kudu/subprocess/subprocess_proxy-test.cc
@@ -23,11 +23,13 @@
 #include <utility>
 #include <vector>
 
+#include <gflags/gflags_declare.h>
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/casts.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/subprocess/echo_subprocess.h"
 #include "kudu/subprocess/subprocess.pb.h"
 #include "kudu/util/env.h"
 #include "kudu/util/metrics.h"
@@ -36,65 +38,23 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-using std::make_shared;
-using std::shared_ptr;
+DECLARE_int32(subprocess_timeout_secs);
+
+METRIC_DECLARE_histogram(echo_subprocess_inbound_queue_length);
+METRIC_DECLARE_histogram(echo_subprocess_outbound_queue_length);
+METRIC_DECLARE_histogram(echo_subprocess_inbound_queue_time_ms);
+METRIC_DECLARE_histogram(echo_subprocess_outbound_queue_time_ms);
+METRIC_DECLARE_histogram(echo_subprocess_execution_time_ms);
+
+using std::unique_ptr;
 using std::string;
 using std::vector;
 using strings::Substitute;
 
-METRIC_DEFINE_histogram(server, echo_subprocess_inbound_queue_length,
-    "Echo subprocess inbound queue length",
-    kudu::MetricUnit::kMessages,
-    "Number of request messages in the Echo subprocess' inbound request queue",
-    kudu::MetricLevel::kInfo,
-    1000, 1);
-METRIC_DEFINE_histogram(server, echo_subprocess_outbound_queue_length,
-    "Echo subprocess outbound queue length",
-    kudu::MetricUnit::kMessages,
-    "Number of request messages in the Echo subprocess' outbound response queue",
-    kudu::MetricLevel::kInfo,
-    1000, 1);
-METRIC_DEFINE_histogram(server, echo_subprocess_inbound_queue_time_ms,
-    "Echo subprocess inbound queue time (ms)",
-    kudu::MetricUnit::kMilliseconds,
-    "Duration of time in ms spent in the Echo subprocess' inbound request queue",
-    kudu::MetricLevel::kInfo,
-    60000LU, 1);
-METRIC_DEFINE_histogram(server, echo_subprocess_outbound_queue_time_ms,
-    "Echo subprocess outbound queue time (ms)",
-    kudu::MetricUnit::kMilliseconds,
-    "Duration of time in ms spent in the Echo subprocess' outbound response queue",
-    kudu::MetricLevel::kInfo,
-    60000LU, 1);
-METRIC_DEFINE_histogram(server, echo_subprocess_execution_time_ms,
-    "Echo subprocess execution time (ms)",
-    kudu::MetricUnit::kMilliseconds,
-    "Duration of time in ms spent executing the Echo subprocess request, excluding "
-    "time spent spent in the subprocess queues",
-    kudu::MetricLevel::kInfo,
-    60000LU, 1);
-
 
 namespace kudu {
 namespace subprocess {
 
-
-#define GINIT(member, x) member = METRIC_##x.Instantiate(entity, 0)
-#define HISTINIT(member, x) member = METRIC_##x.Instantiate(entity)
-struct EchoSubprocessMetrics : public SubprocessMetrics {
-  explicit EchoSubprocessMetrics(const scoped_refptr<MetricEntity>& entity) {
-    HISTINIT(inbound_queue_length, echo_subprocess_inbound_queue_length);
-    HISTINIT(outbound_queue_length, echo_subprocess_outbound_queue_length);
-    HISTINIT(inbound_queue_time_ms, echo_subprocess_inbound_queue_time_ms);
-    HISTINIT(outbound_queue_time_ms, echo_subprocess_outbound_queue_time_ms);
-    HISTINIT(execution_time_ms, echo_subprocess_execution_time_ms);
-  }
-};
-#undef HISTINIT
-#undef MINIT
-
-typedef SubprocessProxy<EchoRequestPB, EchoResponsePB, EchoSubprocessMetrics> EchoSubprocess;
-
 class EchoSubprocessTest : public KuduTest {
  public:
   EchoSubprocessTest()
@@ -117,17 +77,17 @@ class EchoSubprocessTest : public KuduTest {
       "-cp", Substitute("$0/kudu-subprocess.jar", bin_dir),
       "org.apache.kudu.subprocess.echo.EchoSubprocessMain"
     };
-    echo_subprocess_ = make_shared<EchoSubprocess>(std::move(argv), metric_entity_);
+    echo_subprocess_.reset(new EchoSubprocess(std::move(argv), metric_entity_));
     return echo_subprocess_->Start();
   }
 
  protected:
   MetricRegistry metric_registry_;
   scoped_refptr<MetricEntity> metric_entity_;
-  shared_ptr<EchoSubprocess> echo_subprocess_;
+  unique_ptr<EchoSubprocess> echo_subprocess_;
 };
 
-TEST_F(EchoSubprocessTest, TestBasicMetrics) {
+TEST_F(EchoSubprocessTest, TestBasicSubprocessMetrics) {
   const string kMessage = "don't catch you slippin' now";
   const int64_t kSleepMs = 1000;
   EchoRequestPB req;
@@ -164,5 +124,33 @@ TEST_F(EchoSubprocessTest, TestBasicMetrics) {
   ASSERT_LT(kSleepMs, exec_hist->MaxValueForTests());
 }
 
+// Test that we'll still report metrics when we recieve them from the
+// subprocess, even if the call itself failed.
+TEST_F(EchoSubprocessTest, TestSubprocessMetricsOnError) {
+  // Set things up so we'll time out.
+  FLAGS_subprocess_timeout_secs = 1;
+  const int64_t kSleepMs = 2000;
+  ASSERT_OK(ResetEchoSubprocess());
+
+  EchoRequestPB req;
+  req.set_data("garbage!");
+  req.set_sleep_ms(kSleepMs);
+  EchoResponsePB resp;
+  Status s = echo_subprocess_->Execute(req, &resp);
+  ASSERT_TRUE(s.IsTimedOut()) << s.ToString();
+
+  // Immediately following our call, we won't have any metrics from the subprocess.
+  Histogram* exec_hist = down_cast<Histogram*>(metric_entity_->FindOrNull(
+      METRIC_echo_subprocess_execution_time_ms).get());
+  ASSERT_EQ(0, exec_hist->TotalCount());
+
+  // Eventually the subprocess will return our call, and we'll see some
+  // metrics.
+  ASSERT_EVENTUALLY([&] {
+    ASSERT_EQ(1, exec_hist->TotalCount());
+    ASSERT_LT(kSleepMs, exec_hist->MaxValueForTests());
+  });
+}
+
 } // namespace subprocess
 } // namespace kudu
diff --git a/src/kudu/subprocess/subprocess_proxy.h b/src/kudu/subprocess/subprocess_proxy.h
index 33e0f9a..d8cf6c7 100644
--- a/src/kudu/subprocess/subprocess_proxy.h
+++ b/src/kudu/subprocess/subprocess_proxy.h
@@ -14,6 +14,7 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+#pragma once
 
 #include <vector>
 #include <string>
@@ -32,15 +33,6 @@
 namespace kudu {
 namespace subprocess {
 
-// TODO(awong): add server metrics.
-struct SubprocessMetrics {
-  scoped_refptr<Histogram> inbound_queue_length;
-  scoped_refptr<Histogram> outbound_queue_length;
-  scoped_refptr<Histogram> inbound_queue_time_ms;
-  scoped_refptr<Histogram> outbound_queue_time_ms;
-  scoped_refptr<Histogram> execution_time_ms;
-};
-
 // Template that wraps a SubprocessServer, exposing only the underlying ReqPB
 // and RespPB as an interface. The given MetricsPB will be initialized,
 // allowing for metrics specific to each specialized SubprocessServer.
@@ -48,7 +40,7 @@ template<class ReqPB, class RespPB, class MetricsPB>
 class SubprocessProxy {
  public:
   SubprocessProxy(std::vector<std::string> argv, const scoped_refptr<MetricEntity>& entity)
-      : server_(new SubprocessServer(std::move(argv))), metrics_(entity) {}
+      : server_(new SubprocessServer(std::move(argv), MetricsPB(entity))) {}
 
   // Starts the underlying subprocess.
   Status Start() {
@@ -68,11 +60,6 @@ class SubprocessProxy {
                                         pb_util::SecureDebugString(sresp));
       return Status::Corruption("unable to unpack response");
     }
-    // The subprocess metrics should still be valid regardless of whether there
-    // was an error, so parse them first.
-    if (sresp.has_metrics()) {
-      ParseMetricsPB(sresp.metrics());
-    }
     if (sresp.has_error()) {
       return StatusFromPB(sresp.error());
     }
@@ -84,23 +71,7 @@ class SubprocessProxy {
     server_ = std::move(server);
   }
  private:
-  // Parses the given metrics protobuf and updates 'metrics_' based on its
-  // contents.
-  void ParseMetricsPB(const SubprocessMetricsPB& pb) {
-    DCHECK(pb.has_inbound_queue_length());
-    DCHECK(pb.has_outbound_queue_length());
-    DCHECK(pb.has_inbound_queue_time_ms());
-    DCHECK(pb.has_outbound_queue_time_ms());
-    DCHECK(pb.has_execution_time_ms());
-    metrics_.inbound_queue_length->Increment(pb.inbound_queue_length());
-    metrics_.outbound_queue_length->Increment(pb.outbound_queue_length());
-    metrics_.inbound_queue_time_ms->Increment(pb.inbound_queue_time_ms());
-    metrics_.outbound_queue_time_ms->Increment(pb.outbound_queue_time_ms());
-    metrics_.execution_time_ms->Increment(pb.execution_time_ms());
-  }
-
   std::unique_ptr<SubprocessServer> server_;
-  MetricsPB metrics_;
 };
 
 } // namespace subprocess
diff --git a/src/kudu/subprocess/subprocess_server-test.cc b/src/kudu/subprocess/subprocess_server-test.cc
index 2de5ec7..d14293f 100644
--- a/src/kudu/subprocess/subprocess_server-test.cc
+++ b/src/kudu/subprocess/subprocess_server-test.cc
@@ -26,10 +26,13 @@
 #include <google/protobuf/any.pb.h>
 #include <gtest/gtest.h>
 
+#include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/subprocess/echo_subprocess.h"
 #include "kudu/subprocess/server.h"
 #include "kudu/subprocess/subprocess.pb.h"
 #include "kudu/util/env.h"
+#include "kudu/util/metrics.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/status.h"
@@ -75,6 +78,9 @@ const char* kHello = "hello world";
 
 class SubprocessServerTest : public KuduTest {
  public:
+  SubprocessServerTest()
+      : metric_entity_(METRIC_ENTITY_server.Instantiate(&metric_registry_,
+                                                        "subprocess_server-test")) {}
   void SetUp() override {
     KuduTest::SetUp();
     ASSERT_OK(ResetSubprocessServer());
@@ -103,11 +109,14 @@ class SubprocessServerTest : public KuduTest {
       argv.emplace_back("p");
       argv.emplace_back(std::to_string(java_parser_threads));
     }
-    server_ = make_shared<SubprocessServer>(std::move(argv));
+    server_ = make_shared<SubprocessServer>(std::move(argv),
+                                            EchoSubprocessMetrics(metric_entity_));
     return server_->Init();
   }
 
  protected:
+  MetricRegistry metric_registry_;
+  scoped_refptr<MetricEntity> metric_entity_;
   shared_ptr<SubprocessServer> server_;
 };