[#1598] fix(server) Fix inaccurate used_direct_memory_size metric (#1599)

### What changes were proposed in this pull request?

Fix the inaccurate `used_direct_memory_size` metric.
Also `used_direct_memory_size_by_netty` and `used_direct_memory_size_by_grpc_netty` metrics are added to provide more detailed indicators for locating and analyzing in production.

### Why are the changes needed?

Fix https://github.com/apache/incubator-uniffle/issues/1598.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Tested in our env.
diff --git a/server/src/main/java/org/apache/uniffle/server/NettyDirectMemoryTracker.java b/server/src/main/java/org/apache/uniffle/server/NettyDirectMemoryTracker.java
index 96206cc..e9eb170 100644
--- a/server/src/main/java/org/apache/uniffle/server/NettyDirectMemoryTracker.java
+++ b/server/src/main/java/org/apache/uniffle/server/NettyDirectMemoryTracker.java
@@ -53,11 +53,20 @@
     service.scheduleAtFixedRate(
         () -> {
           try {
-            long usedDirectMemory = PlatformDependent.usedDirectMemory();
+            long usedDirectMemoryByNetty = PlatformDependent.usedDirectMemory();
+            long usedDirectMemoryByGrpcNetty =
+                io.grpc.netty.shaded.io.netty.util.internal.PlatformDependent.usedDirectMemory();
             if (LOG.isDebugEnabled()) {
-              LOG.debug("Current usedDirectMemory:{}", usedDirectMemory);
+              LOG.debug(
+                  "Current usedDirectMemoryByNetty:{}, usedDirectMemoryByGrpcNetty:{}",
+                  usedDirectMemoryByNetty,
+                  usedDirectMemoryByGrpcNetty);
             }
-            ShuffleServerMetrics.gaugeUsedDirectMemorySize.set(usedDirectMemory);
+            ShuffleServerMetrics.gaugeUsedDirectMemorySizeByNetty.set(usedDirectMemoryByNetty);
+            ShuffleServerMetrics.gaugeUsedDirectMemorySizeByGrpcNetty.set(
+                usedDirectMemoryByGrpcNetty);
+            ShuffleServerMetrics.gaugeUsedDirectMemorySize.set(
+                usedDirectMemoryByNetty + usedDirectMemoryByGrpcNetty);
           } catch (Throwable t) {
             LOG.error("Failed to report direct memory.", t);
           }
diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerMetrics.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerMetrics.java
index 274cde0..f1f3736 100644
--- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerMetrics.java
+++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerMetrics.java
@@ -79,8 +79,9 @@
   private static final String USED_BUFFER_SIZE = "used_buffer_size";
   private static final String READ_USED_BUFFER_SIZE = "read_used_buffer_size";
   private static final String USED_DIRECT_MEMORY_SIZE = "used_direct_memory_size";
-  private static final String ALLOCATED_DIRECT_MEMORY_SIZE = "allocated_direct_memory_size";
-  private static final String PINNED_DIRECT_MEMORY_SIZE = "pinned_direct_memory_size";
+  private static final String USED_DIRECT_MEMORY_SIZE_BY_NETTY = "used_direct_memory_size_by_netty";
+  private static final String USED_DIRECT_MEMORY_SIZE_BY_GRPC_NETTY =
+      "used_direct_memory_size_by_grpc_netty";
   private static final String TOTAL_FAILED_WRITTEN_EVENT_NUM = "total_failed_written_event_num";
   private static final String TOTAL_DROPPED_EVENT_NUM = "total_dropped_event_num";
   private static final String TOTAL_HADOOP_WRITE_DATA = "total_hadoop_write_data";
@@ -186,6 +187,8 @@
   public static Gauge.Child gaugeUsedBufferSize;
   public static Gauge.Child gaugeReadBufferUsedSize;
   public static Gauge.Child gaugeUsedDirectMemorySize;
+  public static Gauge.Child gaugeUsedDirectMemorySizeByNetty;
+  public static Gauge.Child gaugeUsedDirectMemorySizeByGrpcNetty;
   public static Gauge.Child gaugeWriteHandler;
   public static Gauge.Child gaugeEventQueueSize;
   public static Gauge.Child gaugeHadoopFlushThreadPoolQueueSize;
@@ -382,6 +385,10 @@
     gaugeUsedBufferSize = metricsManager.addLabeledGauge(USED_BUFFER_SIZE);
     gaugeReadBufferUsedSize = metricsManager.addLabeledGauge(READ_USED_BUFFER_SIZE);
     gaugeUsedDirectMemorySize = metricsManager.addLabeledGauge(USED_DIRECT_MEMORY_SIZE);
+    gaugeUsedDirectMemorySizeByNetty =
+        metricsManager.addLabeledGauge(USED_DIRECT_MEMORY_SIZE_BY_NETTY);
+    gaugeUsedDirectMemorySizeByGrpcNetty =
+        metricsManager.addLabeledGauge(USED_DIRECT_MEMORY_SIZE_BY_GRPC_NETTY);
     gaugeWriteHandler = metricsManager.addLabeledGauge(TOTAL_WRITE_HANDLER);
     gaugeEventQueueSize = metricsManager.addLabeledGauge(EVENT_QUEUE_SIZE);
     gaugeHadoopFlushThreadPoolQueueSize =