From 31038f1ddc0aa1c3f2c7803bcc47b3418d200be7 Mon Sep 17 00:00:00 2001 From: "zhanghaobo@kanzhun.com" Date: Wed, 20 Mar 2024 10:40:19 +0800 Subject: [PATCH 1/2] HDFS-17433. metrics sumOfActorCommandQueueLength should only record valid commands. --- .../apache/hadoop/hdfs/server/datanode/BPServiceActor.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 462b9df3c9c81..07dc6f6349e1d 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -1487,8 +1487,10 @@ void enqueue(List cmds) throws InterruptedException { } void enqueue(DatanodeCommand[] cmds) throws InterruptedException { - queue.put(() -> processCommand(cmds)); - dn.getMetrics().incrActorCmdQueueLength(1); + if (cmds.length != 0) { + queue.put(() -> processCommand(cmds)); + dn.getMetrics().incrActorCmdQueueLength(1); + } } } From 2b7a5f58664d3c4467817b5f8b150e66ab71a6ba Mon Sep 17 00:00:00 2001 From: "zhanghaobo@kanzhun.com" Date: Wed, 20 Mar 2024 18:58:01 +0800 Subject: [PATCH 2/2] fix blanks eol. --- .../org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 07dc6f6349e1d..da30d057966c8 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -1489,7 +1489,7 @@ void enqueue(List cmds) throws InterruptedException { void enqueue(DatanodeCommand[] cmds) throws InterruptedException { if (cmds.length != 0) { queue.put(() -> processCommand(cmds)); - dn.getMetrics().incrActorCmdQueueLength(1); + dn.getMetrics().incrActorCmdQueueLength(1); } } }