You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@uniffle.apache.org by ro...@apache.org on 2022/08/17 10:32:41 UTC

[incubator-uniffle] branch master updated: [MINOR] Introduce more grpc server metrics (#150)

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

roryqi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-uniffle.git


The following commit(s) were added to refs/heads/master by this push:
     new 425c02cb [MINOR] Introduce more grpc server metrics (#150)
425c02cb is described below

commit 425c02cb1ae482f555f162d7029d08fdb5a9bb55
Author: Junfan Zhang <ju...@outlook.com>
AuthorDate: Wed Aug 17 18:32:36 2022 +0800

    [MINOR] Introduce more grpc server metrics (#150)
    
    ### What changes were proposed in this pull request?
    Introduce more grpc server metrics
    1. grpc_server_executor_active_threads
    2. grpc_server_executor_blocking_queue_size
    3. grpc_server_connection_size
    
    ### Why are the changes needed?
    Now if the executors size is not enough for handling requests, there is no way to find this problem. And the connection number is need to use the linux command of `nestat` to find.
    
    So this PR is to introduce these metrics.
    
    ### Does this PR introduce _any_ user-facing change?
    No
    
    ### How was this patch tested?
    Unit Tests
---
 .../apache/uniffle/common/metrics/GRPCMetrics.java | 32 ++++++++
 .../org/apache/uniffle/common/rpc/GrpcServer.java  | 44 ++++++++++-
 .../rpc/MonitoringServerTransportFilter.java       | 46 +++++++++++
 .../org/apache/uniffle/common/GrpcServerTest.java  | 89 ++++++++++++++++++++++
 .../coordinator/CoordinatorMetricsTest.java        |  2 +-
 .../uniffle/test/CoordinatorGrpcServerTest.java    | 88 +++++++++++++++++++++
 .../apache/uniffle/test/CoordinatorGrpcTest.java   | 20 +++--
 .../uniffle/server/ShuffleServerMetricsTest.java   |  2 +-
 8 files changed, 312 insertions(+), 11 deletions(-)

diff --git a/common/src/main/java/org/apache/uniffle/common/metrics/GRPCMetrics.java b/common/src/main/java/org/apache/uniffle/common/metrics/GRPCMetrics.java
index 451e56ab..0e3fc7fd 100644
--- a/common/src/main/java/org/apache/uniffle/common/metrics/GRPCMetrics.java
+++ b/common/src/main/java/org/apache/uniffle/common/metrics/GRPCMetrics.java
@@ -25,6 +25,13 @@ import io.prometheus.client.Counter;
 import io.prometheus.client.Gauge;
 
 public abstract class GRPCMetrics {
+  // Grpc server internal executor metrics
+  public static final String GRPC_SERVER_EXECUTOR_ACTIVE_THREADS_KEY = "grpcServerExecutorActiveThreads";
+  private static final String GRPC_SERVER_EXECUTOR_ACTIVE_THREADS = "grpc_server_executor_active_threads";
+  public static final String GRPC_SERVER_EXECUTOR_BLOCKING_QUEUE_SIZE_KEY = "grpcServerExecutorBlockingQueueSize";
+  private static final String GRPC_SERVER_EXECUTOR_BLOCKING_QUEUE_SIZE = "grpc_server_executor_blocking_queue_size";
+  public static final String GRCP_SERVER_CONNECTION_NUMBER_KEY = "grpcServerConnectionNumber";
+  private static final String GRCP_SERVER_CONNECTION_NUMBER = "grpc_server_connection_number";
 
   private boolean isRegister = false;
   protected Map<String, Counter> counterMap = Maps.newConcurrentMap();
@@ -38,11 +45,36 @@ public abstract class GRPCMetrics {
   public void register(CollectorRegistry collectorRegistry) {
     if (!isRegister) {
       metricsManager = new MetricsManager(collectorRegistry);
+      registerGeneralMetrics();
       registerMetrics();
       isRegister = true;
     }
   }
 
+  private void registerGeneralMetrics() {
+    gaugeMap.putIfAbsent(
+        GRPC_SERVER_EXECUTOR_ACTIVE_THREADS_KEY,
+        metricsManager.addGauge(GRPC_SERVER_EXECUTOR_ACTIVE_THREADS)
+    );
+    gaugeMap.putIfAbsent(
+        GRPC_SERVER_EXECUTOR_BLOCKING_QUEUE_SIZE_KEY,
+        metricsManager.addGauge(GRPC_SERVER_EXECUTOR_BLOCKING_QUEUE_SIZE)
+    );
+    gaugeMap.putIfAbsent(
+        GRCP_SERVER_CONNECTION_NUMBER_KEY,
+        metricsManager.addGauge(GRCP_SERVER_CONNECTION_NUMBER)
+    );
+  }
+
+  public void setGauge(String tag, double value) {
+    if (isRegister) {
+      Gauge gauge = gaugeMap.get(tag);
+      if (gauge != null) {
+        gauge.set(value);
+      }
+    }
+  }
+
   public void incCounter(String methodName) {
     if (isRegister) {
       Gauge gauge = gaugeMap.get(methodName);
diff --git a/common/src/main/java/org/apache/uniffle/common/rpc/GrpcServer.java b/common/src/main/java/org/apache/uniffle/common/rpc/GrpcServer.java
index 5f14782e..2283a662 100644
--- a/common/src/main/java/org/apache/uniffle/common/rpc/GrpcServer.java
+++ b/common/src/main/java/org/apache/uniffle/common/rpc/GrpcServer.java
@@ -18,9 +18,12 @@
 package org.apache.uniffle.common.rpc;
 
 import java.io.IOException;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.collect.Queues;
 import io.grpc.BindableService;
@@ -47,13 +50,14 @@ public class GrpcServer implements ServerInterface {
     this.port = conf.getInteger(RssBaseConf.RPC_SERVER_PORT);
     long maxInboundMessageSize = conf.getLong(RssBaseConf.RPC_MESSAGE_MAX_SIZE);
     int rpcExecutorSize = conf.getInteger(RssBaseConf.RPC_EXECUTOR_SIZE);
-    pool = new ThreadPoolExecutor(
+    pool = new GrpcThreadPoolExecutor(
         rpcExecutorSize,
         rpcExecutorSize * 2,
         10,
         TimeUnit.MINUTES,
         Queues.newLinkedBlockingQueue(Integer.MAX_VALUE),
-        ThreadUtils.getThreadFactory("Grpc-%d")
+        ThreadUtils.getThreadFactory("Grpc-%d"),
+        grpcMetrics
     );
 
     boolean isMetricsEnabled = conf.getBoolean(RssBaseConf.RPC_METRICS_ENABLED);
@@ -64,6 +68,7 @@ public class GrpcServer implements ServerInterface {
           .forPort(port)
           .addService(ServerInterceptors.intercept(service, monitoringInterceptor))
           .executor(pool)
+          .addTransportFilter(new MonitoringServerTransportFilter(grpcMetrics))
           .maxInboundMessageSize((int)maxInboundMessageSize)
           .build();
     } else {
@@ -76,6 +81,41 @@ public class GrpcServer implements ServerInterface {
     }
   }
 
+  public static class GrpcThreadPoolExecutor extends ThreadPoolExecutor {
+    private final GRPCMetrics grpcMetrics;
+    private final AtomicLong activeThreadSize = new AtomicLong(0L);
+
+    public GrpcThreadPoolExecutor(
+        int corePoolSize,
+        int maximumPoolSize,
+        long keepAliveTime,
+        TimeUnit unit,
+        BlockingQueue<Runnable> workQueue,
+        ThreadFactory threadFactory,
+        GRPCMetrics grpcMetrics) {
+      super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory);
+      this.grpcMetrics = grpcMetrics;
+    }
+
+    @Override
+    protected void beforeExecute(Thread t, Runnable r) {
+      grpcMetrics.setGauge(GRPCMetrics.GRPC_SERVER_EXECUTOR_ACTIVE_THREADS_KEY,
+          activeThreadSize.incrementAndGet());
+      grpcMetrics.setGauge(GRPCMetrics.GRPC_SERVER_EXECUTOR_BLOCKING_QUEUE_SIZE_KEY,
+          getQueue().size());
+      super.beforeExecute(t, r);
+    }
+
+    @Override
+    protected void afterExecute(Runnable r, Throwable t) {
+      grpcMetrics.setGauge(GRPCMetrics.GRPC_SERVER_EXECUTOR_ACTIVE_THREADS_KEY,
+          activeThreadSize.decrementAndGet());
+      grpcMetrics.setGauge(GRPCMetrics.GRPC_SERVER_EXECUTOR_BLOCKING_QUEUE_SIZE_KEY,
+          getQueue().size());
+      super.afterExecute(r, t);
+    }
+  }
+
   public void start() throws IOException {
     try {
       server.start();
diff --git a/common/src/main/java/org/apache/uniffle/common/rpc/MonitoringServerTransportFilter.java b/common/src/main/java/org/apache/uniffle/common/rpc/MonitoringServerTransportFilter.java
new file mode 100644
index 00000000..62ad537a
--- /dev/null
+++ b/common/src/main/java/org/apache/uniffle/common/rpc/MonitoringServerTransportFilter.java
@@ -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.
+ */
+
+package org.apache.uniffle.common.rpc;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import io.grpc.Attributes;
+import io.grpc.ServerTransportFilter;
+
+import org.apache.uniffle.common.metrics.GRPCMetrics;
+
+import static org.apache.uniffle.common.metrics.GRPCMetrics.GRCP_SERVER_CONNECTION_NUMBER_KEY;
+
+public class MonitoringServerTransportFilter extends ServerTransportFilter {
+  private final AtomicLong connectionSize = new AtomicLong(0);
+  private final GRPCMetrics grpcMetrics;
+
+  public MonitoringServerTransportFilter(GRPCMetrics grpcMetrics) {
+    this.grpcMetrics = grpcMetrics;
+  }
+
+  public Attributes transportReady(Attributes transportAttrs) {
+    grpcMetrics.setGauge(GRCP_SERVER_CONNECTION_NUMBER_KEY, connectionSize.incrementAndGet());
+    return super.transportReady(transportAttrs);
+  }
+
+  public void transportTerminated(Attributes transportAttrs) {
+    grpcMetrics.setGauge(GRCP_SERVER_CONNECTION_NUMBER_KEY, connectionSize.decrementAndGet());
+    super.transportTerminated(transportAttrs);
+  }
+}
diff --git a/common/src/test/java/org/apache/uniffle/common/GrpcServerTest.java b/common/src/test/java/org/apache/uniffle/common/GrpcServerTest.java
new file mode 100644
index 00000000..67ac320c
--- /dev/null
+++ b/common/src/test/java/org/apache/uniffle/common/GrpcServerTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.
+ */
+
+package org.apache.uniffle.common;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.Queues;
+import io.prometheus.client.CollectorRegistry;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.metrics.GRPCMetrics;
+import org.apache.uniffle.common.rpc.GrpcServer;
+import org.apache.uniffle.common.util.ThreadUtils;
+
+import static org.apache.uniffle.common.metrics.GRPCMetrics.GRPC_SERVER_EXECUTOR_ACTIVE_THREADS_KEY;
+import static org.apache.uniffle.common.metrics.GRPCMetrics.GRPC_SERVER_EXECUTOR_BLOCKING_QUEUE_SIZE_KEY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class GrpcServerTest {
+  private static final Logger LOGGER = LoggerFactory.getLogger(GrpcServerTest.class);
+
+  static class MockedGRPCMetrics extends GRPCMetrics {
+    @Override
+    public void registerMetrics() {
+      // ignore
+    }
+  }
+
+  @Test
+  public void testGrpcExecutorPool() throws Exception {
+    GRPCMetrics grpcMetrics = new MockedGRPCMetrics();
+    grpcMetrics.register(new CollectorRegistry(true));
+    GrpcServer.GrpcThreadPoolExecutor executor = new GrpcServer.GrpcThreadPoolExecutor(
+        2,
+        2,
+        100,
+        TimeUnit.MINUTES,
+        Queues.newLinkedBlockingQueue(Integer.MAX_VALUE),
+        ThreadUtils.getThreadFactory("Grpc-%d"),
+        grpcMetrics
+    );
+
+    CountDownLatch countDownLatch = new CountDownLatch(3);
+    for (int i = 0; i < 3; i++) {
+      final int index = i;
+      executor.submit(() -> {
+        try {
+          Thread.sleep(1000 * 2);
+        } catch (InterruptedException interruptedException) {
+          interruptedException.printStackTrace();
+        }
+        LOGGER.info("Finished task: {}", index);
+        countDownLatch.countDown();
+      });
+    }
+
+    Thread.sleep(1000L);
+    double activeThreads = grpcMetrics.getGaugeMap().get(GRPC_SERVER_EXECUTOR_ACTIVE_THREADS_KEY).get();
+    assertEquals(2, activeThreads);
+    double queueSize = grpcMetrics.getGaugeMap().get(GRPC_SERVER_EXECUTOR_BLOCKING_QUEUE_SIZE_KEY).get();
+    assertEquals(1, queueSize);
+
+    countDownLatch.await();
+    activeThreads = grpcMetrics.getGaugeMap().get(GRPC_SERVER_EXECUTOR_ACTIVE_THREADS_KEY).get();
+    assertEquals(0, activeThreads);
+    queueSize = grpcMetrics.getGaugeMap().get(GRPC_SERVER_EXECUTOR_BLOCKING_QUEUE_SIZE_KEY).get();
+    assertEquals(0, queueSize);
+
+    executor.shutdown();
+  }
+}
diff --git a/coordinator/src/test/java/org/apache/uniffle/coordinator/CoordinatorMetricsTest.java b/coordinator/src/test/java/org/apache/uniffle/coordinator/CoordinatorMetricsTest.java
index 872d713c..1b4a2345 100644
--- a/coordinator/src/test/java/org/apache/uniffle/coordinator/CoordinatorMetricsTest.java
+++ b/coordinator/src/test/java/org/apache/uniffle/coordinator/CoordinatorMetricsTest.java
@@ -104,7 +104,7 @@ public class CoordinatorMetricsTest {
     ObjectMapper mapper = new ObjectMapper();
     JsonNode actualObj = mapper.readTree(content);
     assertEquals(2, actualObj.size());
-    assertEquals(6, actualObj.get("metrics").size());
+    assertEquals(9, actualObj.get("metrics").size());
   }
 
   private static void writeRemoteStorageConf(File cfgFile, String value) throws Exception {
diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcServerTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcServerTest.java
new file mode 100644
index 00000000..0c1e7c07
--- /dev/null
+++ b/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcServerTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.
+ */
+
+package org.apache.uniffle.test;
+
+import io.grpc.stub.StreamObserver;
+import io.prometheus.client.CollectorRegistry;
+import org.junit.jupiter.api.Test;
+
+import org.apache.uniffle.client.impl.grpc.CoordinatorGrpcClient;
+import org.apache.uniffle.client.request.RssAppHeartBeatRequest;
+import org.apache.uniffle.common.config.RssBaseConf;
+import org.apache.uniffle.common.metrics.GRPCMetrics;
+import org.apache.uniffle.common.rpc.GrpcServer;
+import org.apache.uniffle.coordinator.CoordinatorGrpcMetrics;
+import org.apache.uniffle.proto.CoordinatorServerGrpc;
+import org.apache.uniffle.proto.RssProtos;
+
+import static org.apache.uniffle.common.metrics.GRPCMetrics.GRCP_SERVER_CONNECTION_NUMBER_KEY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * This class is to test the GRPC server's related metrics like {@code GRPC_SERVER_EXECUTOR_ACTIVE_THREADS_TAG}
+ */
+public class CoordinatorGrpcServerTest {
+
+  static class MockedCoordinatorGrpcService extends CoordinatorServerGrpc.CoordinatorServerImplBase {
+    @Override
+    public void appHeartbeat(
+        RssProtos.AppHeartBeatRequest request,
+        StreamObserver<RssProtos.AppHeartBeatResponse> responseObserver) {
+      RssProtos.AppHeartBeatResponse response = RssProtos.AppHeartBeatResponse
+          .newBuilder()
+          .setRetMsg("")
+          .setStatus(RssProtos.StatusCode.SUCCESS)
+          .build();
+      responseObserver.onNext(response);
+      responseObserver.onCompleted();
+    }
+  }
+
+  @Test
+  public void testGrpcConnectionSize() throws Exception {
+    RssBaseConf baseConf = new RssBaseConf();
+    baseConf.set(RssBaseConf.RPC_SERVER_PORT, 20001);
+    baseConf.set(RssBaseConf.RPC_EXECUTOR_SIZE, 2);
+
+    GRPCMetrics grpcMetrics = new CoordinatorGrpcMetrics();
+    grpcMetrics.register(new CollectorRegistry(true));
+    GrpcServer grpcServer = new GrpcServer(baseConf, new MockedCoordinatorGrpcService(), grpcMetrics);
+    grpcServer.start();
+
+    // case1: test the single one connection metric
+    double connSize = grpcMetrics.getGaugeMap().get(GRCP_SERVER_CONNECTION_NUMBER_KEY).get();
+    assertEquals(0, connSize);
+
+    CoordinatorGrpcClient coordinatorGrpcClient = new CoordinatorGrpcClient("localhost", 20001);
+    coordinatorGrpcClient.sendAppHeartBeat(new RssAppHeartBeatRequest("testGrpcConnectionSize", 10000));
+
+    connSize = grpcMetrics.getGaugeMap().get(GRCP_SERVER_CONNECTION_NUMBER_KEY).get();
+    assertEquals(1, connSize);
+
+    // case2: test the multiple connections
+    CoordinatorGrpcClient client1 = new CoordinatorGrpcClient("localhost", 20001);
+    CoordinatorGrpcClient client2 = new CoordinatorGrpcClient("localhost", 20001);
+    client1.sendAppHeartBeat(new RssAppHeartBeatRequest("testGrpcConnectionSize", 10000));
+    client2.sendAppHeartBeat(new RssAppHeartBeatRequest("testGrpcConnectionSize", 10000));
+
+    connSize = grpcMetrics.getGaugeMap().get(GRCP_SERVER_CONNECTION_NUMBER_KEY).get();
+    assertEquals(3, connSize);
+
+    grpcServer.stop();
+  }
+}
diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcTest.java
index 826d2404..8fd896a3 100644
--- a/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcTest.java
+++ b/integration-test/common/src/test/java/org/apache/uniffle/test/CoordinatorGrpcTest.java
@@ -17,8 +17,16 @@
 
 package org.apache.uniffle.test;
 
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
 import org.apache.uniffle.client.request.RssAppHeartBeatRequest;
 import org.apache.uniffle.client.request.RssGetShuffleAssignmentsRequest;
 import org.apache.uniffle.client.response.ResponseStatusCode;
@@ -39,14 +47,8 @@ import org.apache.uniffle.proto.RssProtos.PartitionRangeAssignment;
 import org.apache.uniffle.proto.RssProtos.ShuffleServerId;
 import org.apache.uniffle.server.ShuffleServer;
 import org.apache.uniffle.server.ShuffleServerConf;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
+import static org.apache.uniffle.common.metrics.GRPCMetrics.GRCP_SERVER_CONNECTION_NUMBER_KEY;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -264,6 +266,10 @@ public class CoordinatorGrpcTest extends CoordinatorTestBase {
     assertEquals(0,
         coordinators.get(0).getGrpcMetrics().getGaugeMap()
             .get(CoordinatorGrpcMetrics.GET_SHUFFLE_ASSIGNMENTS_METHOD).get(), 0.5);
+
+    double connectionSize = coordinators.get(0)
+        .getGrpcMetrics().getGaugeMap().get(GRCP_SERVER_CONNECTION_NUMBER_KEY).get();
+    assertTrue(connectionSize > 0);
   }
 
   private GetShuffleAssignmentsResponse generateShuffleAssignmentsResponse() {
diff --git a/server/src/test/java/org/apache/uniffle/server/ShuffleServerMetricsTest.java b/server/src/test/java/org/apache/uniffle/server/ShuffleServerMetricsTest.java
index 1aee302a..a8d57bf8 100644
--- a/server/src/test/java/org/apache/uniffle/server/ShuffleServerMetricsTest.java
+++ b/server/src/test/java/org/apache/uniffle/server/ShuffleServerMetricsTest.java
@@ -146,7 +146,7 @@ public class ShuffleServerMetricsTest {
     ObjectMapper mapper = new ObjectMapper();
     JsonNode actualObj = mapper.readTree(content);
     assertEquals(2, actualObj.size());
-    assertEquals(24, actualObj.get("metrics").size());
+    assertEquals(27, actualObj.get("metrics").size());
   }
 
   @Test