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

spricoder pushed a commit to branch feature/disk-metric
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/feature/disk-metric by this 
push:
     new ca4aae48945 Add database and region
ca4aae48945 is described below

commit ca4aae489459fadbb61abb0e21bfa5a11bfbeeb2
Author: spricoder <[email protected]>
AuthorDate: Thu Sep 21 14:50:26 2023 +0800

    Add database and region
---
 .../iotdb/db/service/metrics/FileMetrics.java      | 121 +++++++++++----------
 1 file changed, 62 insertions(+), 59 deletions(-)

diff --git 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/FileMetrics.java
 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/FileMetrics.java
index 8b7d627ea37..f36431e4ec4 100644
--- 
a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/FileMetrics.java
+++ 
b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/FileMetrics.java
@@ -71,10 +71,6 @@ public class FileMetrics implements IMetricSet {
   private final Map<String, Map<String, Long>> unseqFileSizeGaugeMap = new 
ConcurrentHashMap<>();
   private final Map<String, Map<String, Integer>> seqFileNumGaugeMap = new 
ConcurrentHashMap<>();
   private final Map<String, Map<String, Integer>> unseqFileNumGaugeMap = new 
ConcurrentHashMap<>();
-  private final AtomicLong seqFileSize = new AtomicLong(0);
-  private final AtomicLong unseqFileSize = new AtomicLong(0);
-  private final AtomicInteger seqFileNum = new AtomicInteger(0);
-  private final AtomicInteger unseqFileNum = new AtomicInteger(0);
 
   private final AtomicInteger modFileNum = new AtomicInteger(0);
 
@@ -345,35 +341,37 @@ public class FileMetrics implements IMetricSet {
             if (v == null) {
               v = new ConcurrentHashMap<>();
             } else if (v.containsKey(regionId)) {
-              count += countDelta;
+              count = v.get(regionId);
             }
-            v.put(regionId, count);
+            v.put(regionId, count + countDelta);
             return v;
           });
-      // update sequence file size metric
-      metricService
-          .getOrCreateGauge(
-              Metric.FILE_SIZE.toString(),
-              MetricLevel.CORE,
-              Tag.NAME.toString(),
-              "seq",
-              Tag.DATABASE.toString(),
-              database,
-              Tag.REGION.toString(),
-              regionId)
-          .set(seqFileSizeGaugeMap.get(database).get(regionId));
-      // update sequence file number metric
-      metricService
-          .getOrCreateGauge(
-              Metric.FILE_COUNT.toString(),
-              MetricLevel.CORE,
-              Tag.NAME.toString(),
-              "seq",
-              Tag.DATABASE.toString(),
-              database,
-              Tag.REGION.toString(),
-              regionId)
-          .set(seqFileNumGaugeMap.get(database).get(regionId));
+      if (metricService != null) {
+        // update sequence file size metric
+        metricService
+            .getOrCreateGauge(
+                Metric.FILE_SIZE.toString(),
+                MetricLevel.CORE,
+                Tag.NAME.toString(),
+                "seq",
+                Tag.DATABASE.toString(),
+                database,
+                Tag.REGION.toString(),
+                regionId)
+            .set(seqFileSizeGaugeMap.get(database).get(regionId));
+        // update sequence file number metric
+        metricService
+            .getOrCreateGauge(
+                Metric.FILE_COUNT.toString(),
+                MetricLevel.CORE,
+                Tag.NAME.toString(),
+                "seq",
+                Tag.DATABASE.toString(),
+                database,
+                Tag.REGION.toString(),
+                regionId)
+            .set(seqFileNumGaugeMap.get(database).get(regionId));
+      }
     } else {
       // update unsequence file size
       unseqFileSizeGaugeMap.compute(
@@ -401,30 +399,32 @@ public class FileMetrics implements IMetricSet {
             v.put(regionId, count + countDelta);
             return v;
           });
-      // update unsequence file size metric
-      metricService
-          .getOrCreateGauge(
-              Metric.FILE_SIZE.toString(),
-              MetricLevel.CORE,
-              Tag.NAME.toString(),
-              "unseq",
-              Tag.DATABASE.toString(),
-              database,
-              Tag.REGION.toString(),
-              regionId)
-          .set(unseqFileSizeGaugeMap.get(database).get(regionId));
-      // update unsequence file number metric
-      metricService
-          .getOrCreateGauge(
-              Metric.FILE_COUNT.toString(),
-              MetricLevel.CORE,
-              Tag.NAME.toString(),
-              "unseq",
-              Tag.DATABASE.toString(),
-              database,
-              Tag.REGION.toString(),
-              regionId)
-          .set(unseqFileNumGaugeMap.get(database).get(regionId));
+      if (metricService != null) {
+        // update unsequence file size metric
+        metricService
+            .getOrCreateGauge(
+                Metric.FILE_SIZE.toString(),
+                MetricLevel.CORE,
+                Tag.NAME.toString(),
+                "unseq",
+                Tag.DATABASE.toString(),
+                database,
+                Tag.REGION.toString(),
+                regionId)
+            .set(unseqFileSizeGaugeMap.get(database).get(regionId));
+        // update unsequence file number metric
+        metricService
+            .getOrCreateGauge(
+                Metric.FILE_COUNT.toString(),
+                MetricLevel.CORE,
+                Tag.NAME.toString(),
+                "unseq",
+                Tag.DATABASE.toString(),
+                database,
+                Tag.REGION.toString(),
+                regionId)
+            .set(unseqFileNumGaugeMap.get(database).get(regionId));
+      }
     }
   }
 
@@ -540,12 +540,15 @@ public class FileMetrics implements IMetricSet {
     }
   }
 
-  public long getFileSize(boolean seq) {
-    return seq ? seqFileSize.get() : unseqFileSize.get();
-  }
-
   public long getFileNum(boolean seq) {
-    return seq ? seqFileNum.get() : unseqFileNum.get();
+    long fileNum = 0;
+    for (Map.Entry<String, Map<String, Integer>> entry :
+        (seq ? seqFileNumGaugeMap : unseqFileNumGaugeMap).entrySet()) {
+      for (Map.Entry<String, Integer> regionEntry : 
entry.getValue().entrySet()) {
+        fileNum += regionEntry.getValue();
+      }
+    }
+    return fileNum;
   }
 
   public int getModFileNum() {

Reply via email to