OneSizeFitsQuorum commented on code in PR #11228:
URL: https://github.com/apache/iotdb/pull/11228#discussion_r1349510261


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/FileMetrics.java:
##########
@@ -344,138 +340,240 @@ public void addFile(long size, boolean seq, String 
name) {
     }
   }
 
-  private void updateGlobalCountAndSize(long sizeDelta, int countDelta, 
boolean seq) {
-    if (seq) {
-      seqFileSize.getAndAdd(sizeDelta);
-      seqFileNum.getAndAdd(countDelta);
+  private void updateGlobalCountAndSize(
+      String database, String regionId, long sizeDelta, int countDelta, 
boolean seq) {
+    long fileSize =
+        updateFileMapAndGet(
+            (seq ? seqFileSizeMap : unseqFileSizeMap), database, regionId, 
sizeDelta, Long::sum);
+    long fileNum =
+        updateFileMapAndGet(
+            (seq ? seqFileNumMap : unseqFileNumMap), database, regionId, 
countDelta, Integer::sum);
+    if (metricService != null) {
+      updateGlobalGauge(
+          database,
+          regionId,
+          fileSize,
+          (seq ? seqFileSizeGaugeMap : unseqFileSizeGaugeMap),
+          (seq ? SEQUENCE : UNSEQUENCE),
+          Metric.FILE_SIZE.toString());
+      updateGlobalGauge(
+          database,
+          regionId,
+          fileNum,
+          (seq ? seqFileNumGaugeMap : unseqFileNumGaugeMap),
+          (seq ? SEQUENCE : UNSEQUENCE),
+          Metric.FILE_COUNT.toString());
+      checkIfThereRemainingData();
     } else {
-      unseqFileSize.getAndAdd(sizeDelta);
-      unseqFileNum.getAndAdd(countDelta);
+      // the metric service has not been set yet
+      hasRemainData.set(true);
     }
   }
 
-  private void updateLevelCountAndSize(long sizeDelta, int countDelta, boolean 
seq, int level) {
-    int count = 0;
-    long totalSize = 0;
-    if (seq) {
-      count =
-          seqLevelTsFileCountMap.compute(level, (k, v) -> v == null ? 
countDelta : v + countDelta);
-      totalSize =
-          seqLevelTsFileSizeMap.compute(level, (k, v) -> v == null ? sizeDelta 
: v + sizeDelta);
-    } else {
-      count =
-          unseqLevelTsFileCountMap.compute(
-              level, (k, v) -> v == null ? countDelta : v + countDelta);
-      totalSize =
-          unseqLevelTsFileSizeMap.compute(level, (k, v) -> v == null ? 
sizeDelta : v + sizeDelta);
-    }
-    updateLevelFileInfoInMetricService(totalSize, count, seq, level);
+  private <T> T updateFileMapAndGet(
+      Map<String, Map<String, T>> map,
+      String database,
+      String regionId,
+      T value,
+      BinaryOperator<T> mergeFunction) {
+    Map<String, T> innerMap = map.computeIfAbsent(database, k -> new 
ConcurrentHashMap<>());
+    return innerMap.merge(regionId, value, mergeFunction);
+  }
+
+  private void updateGlobalGauge(
+      String database,
+      String regionId,
+      long value,
+      Map<String, Map<String, Gauge>> gaugeMap,
+      String orderStr,
+      String gaugeName) {
+    gaugeMap.compute(
+        database,
+        (k, v) -> {
+          if (v == null) {
+            v = new HashMap<>();
+          }
+          if (!v.containsKey(regionId)) {
+            v.put(
+                regionId,
+                metricService.getOrCreateGauge(
+                    gaugeName,
+                    MetricLevel.CORE,
+                    Tag.NAME.toString(),
+                    orderStr,
+                    Tag.DATABASE.toString(),
+                    database,
+                    Tag.REGION.toString(),
+                    regionId));
+          }
+          v.get(regionId).set(value);
+          return v;
+        });
   }
 
-  private void updateLevelFileInfoInMetricService(
-      long totalSize, int count, boolean seq, int level) {
+  private void updateLevelCountAndSize(long sizeDelta, int countDelta, boolean 
seq, int level) {
+    int count =
+        (seq ? seqLevelTsFileCountMap : unseqLevelTsFileCountMap)
+            .compute(level, (k, v) -> v == null ? countDelta : v + countDelta);
+    long totalSize =
+        (seq ? seqLevelTsFileSizeMap : unseqLevelTsFileSizeMap)
+            .compute(level, (k, v) -> v == null ? sizeDelta : v + sizeDelta);
     if (metricService != null) {
-      updateCountGauge(
+      updateLevelGauge(
           level,
           count,
           seq ? seqLevelCountGaugeMap : unseqLevelCountGaugeMap,
-          seq ? SEQUENCE : UNSEQUENCE);
-      updateSizeGauge(
+          seq ? SEQUENCE : UNSEQUENCE,
+          FILE_LEVEL_COUNT);
+      updateLevelGauge(
           level,
           totalSize,
           seq ? seqLevelSizeGaugeMap : unseqLevelSizeGaugeMap,
-          seq ? SEQUENCE : UNSEQUENCE);
+          seq ? SEQUENCE : UNSEQUENCE,
+          FILE_LEVEL_SIZE);
       checkIfThereRemainingData();
     } else {
       // the metric service has not been set yet
       hasRemainData.set(true);
     }
   }
 
-  private void checkIfThereRemainingData() {
-    if (hasRemainData.get()) {
-      synchronized (this) {
-        if (hasRemainData.get()) {
-          hasRemainData.set(false);
-          updateRemainData();
-        }
-      }
-    }
-  }
-
-  private void updateCountGauge(
-      int level, int count, Map<Integer, Gauge> countGaugeMap, String 
orderStr) {
-    countGaugeMap
+  private void updateLevelGauge(
+      int level, long value, Map<Integer, Gauge> gaugeMap, String orderStr, 
String gaugeName) {
+    gaugeMap
         .computeIfAbsent(
             level,
             l ->
                 metricService.getOrCreateGauge(
-                    FILE_LEVEL_COUNT,
+                    gaugeName,
                     MetricLevel.CORE,
-                    Tag.TYPE.toString(),
+                    Tag.NAME.toString(),
                     orderStr,
                     LEVEL,
                     String.valueOf(level)))
-        .set(count);
-  }
-
-  private void updateSizeGauge(
-      int level, long size, Map<Integer, Gauge> sizeGaugeMap, String orderStr) 
{
-    sizeGaugeMap
-        .computeIfAbsent(
-            level,
-            l ->
-                metricService.getOrCreateGauge(
-                    FILE_LEVEL_SIZE,
-                    MetricLevel.CORE,
-                    Tag.TYPE.toString(),
-                    orderStr,
-                    LEVEL,
-                    String.valueOf(level)))
-        .set(size);
-  }
-
-  public void deleteFile(long[] sizeList, boolean seq, List<String> names) {
-    long totalSize = 0;
-    for (long size : sizeList) {
-      totalSize += size;
-    }
-    updateGlobalCountAndSize(-totalSize, -sizeList.length, seq);
-    for (int i = 0, length = names.size(); i < length; ++i) {
-      int level = -1;
-      String name = names.get(i);
-      long size = sizeList[i];
+        .set(value);
+  }
+
+  public void deleteRegion(String database, String regionId) {
+    Arrays.asList(seqFileNumMap, unseqFileNumMap)
+        .forEach(map -> deleteRegionFromMap(map, database, regionId));
+    Arrays.asList(seqFileSizeMap, unseqFileSizeMap)
+        .forEach(map -> deleteRegionFromMap(map, database, regionId));
+    Arrays.asList(
+            seqFileNumGaugeMap, unseqFileNumGaugeMap, seqFileSizeGaugeMap, 
unseqFileSizeGaugeMap)
+        .forEach(map -> deleteRegionFromMap(map, database, regionId));
+  }
+
+  public <T> void deleteRegionFromMap(
+      Map<String, Map<String, T>> map, String database, String regionId) {
+    Map<String, T> innerMap =
+        map.computeIfPresent(
+            database,
+            (k, v) -> {
+              v.remove(regionId);
+              return v.isEmpty() ? null : v;
+            });
+  }
+
+  public void deleteFile(boolean seq, List<TsFileResource> tsFileResourceList) 
{
+    for (TsFileResource tsFileResource : tsFileResourceList) {
+      String name = tsFileResource.getTsFile().getName();
+      long size = tsFileResource.getTsFileSize();
+      updateGlobalCountAndSize(
+          tsFileResource.getDatabaseName(), tsFileResource.getDataRegionId(), 
-size, -1, seq);
       try {
         TsFileNameGenerator.TsFileName tsFileName = 
TsFileNameGenerator.getTsFileName(name);
-        level = tsFileName.getInnerCompactionCnt();
+        int level = tsFileName.getInnerCompactionCnt();
         updateLevelCountAndSize(-size, -1, seq, level);
       } catch (IOException e) {
         log.warn("Unexpected error occurred when getting tsfile name", e);
       }
     }
   }
 
+  private void checkIfThereRemainingData() {
+    if (hasRemainData.get()) {
+      synchronized (this) {
+        if (hasRemainData.get()) {
+          hasRemainData.set(false);
+          updateRemainData();
+        }
+      }
+    }
+  }
+
   private void updateRemainData() {
+    for (Map.Entry<String, Map<String, Integer>> entry : 
seqFileNumMap.entrySet()) {
+      for (Map.Entry<String, Integer> innerEntry : 
entry.getValue().entrySet()) {
+        updateGlobalGauge(
+            entry.getKey(),
+            innerEntry.getKey(),
+            innerEntry.getValue(),
+            seqFileNumGaugeMap,
+            SEQUENCE,
+            Metric.FILE_COUNT.toString());
+      }
+    }
+    for (Map.Entry<String, Map<String, Integer>> entry : 
unseqFileNumMap.entrySet()) {
+      for (Map.Entry<String, Integer> innerEntry : 
entry.getValue().entrySet()) {
+        updateGlobalGauge(
+            entry.getKey(),
+            innerEntry.getKey(),
+            innerEntry.getValue(),
+            unseqFileNumGaugeMap,
+            UNSEQUENCE,
+            Metric.FILE_COUNT.toString());
+      }
+    }
+    for (Map.Entry<String, Map<String, Long>> entry : 
seqFileSizeMap.entrySet()) {
+      for (Map.Entry<String, Long> innerEntry : entry.getValue().entrySet()) {
+        updateGlobalGauge(
+            entry.getKey(),
+            innerEntry.getKey(),
+            innerEntry.getValue(),
+            seqFileSizeGaugeMap,
+            SEQUENCE,
+            Metric.FILE_SIZE.toString());
+      }
+    }
+    for (Map.Entry<String, Map<String, Long>> entry : 
unseqFileSizeMap.entrySet()) {
+      for (Map.Entry<String, Long> innerEntry : entry.getValue().entrySet()) {
+        updateGlobalGauge(
+            entry.getKey(),
+            innerEntry.getKey(),
+            innerEntry.getValue(),
+            unseqFileSizeGaugeMap,
+            UNSEQUENCE,
+            Metric.FILE_SIZE.toString());
+      }
+    }
     for (Map.Entry<Integer, Integer> entry : 
seqLevelTsFileCountMap.entrySet()) {
-      updateCountGauge(entry.getKey(), entry.getValue(), 
seqLevelCountGaugeMap, SEQUENCE);
+      updateLevelGauge(
+          entry.getKey(), entry.getValue(), seqLevelCountGaugeMap, SEQUENCE, 
FILE_LEVEL_COUNT);
     }
     for (Map.Entry<Integer, Long> entry : seqLevelTsFileSizeMap.entrySet()) {
-      updateSizeGauge(entry.getKey(), entry.getValue(), seqLevelSizeGaugeMap, 
SEQUENCE);
+      updateLevelGauge(
+          entry.getKey(), entry.getValue(), seqLevelSizeGaugeMap, SEQUENCE, 
FILE_LEVEL_SIZE);
     }
     for (Map.Entry<Integer, Integer> entry : 
unseqLevelTsFileCountMap.entrySet()) {
-      updateCountGauge(entry.getKey(), entry.getValue(), 
unseqLevelCountGaugeMap, UNSEQUENCE);
+      updateLevelGauge(
+          entry.getKey(), entry.getValue(), unseqLevelCountGaugeMap, 
UNSEQUENCE, FILE_LEVEL_COUNT);
     }
     for (Map.Entry<Integer, Long> entry : unseqLevelTsFileSizeMap.entrySet()) {
-      updateSizeGauge(entry.getKey(), entry.getValue(), 
unseqLevelSizeGaugeMap, UNSEQUENCE);
+      updateLevelGauge(
+          entry.getKey(), entry.getValue(), unseqLevelSizeGaugeMap, 
UNSEQUENCE, FILE_LEVEL_SIZE);
     }
   }
 
-  public long getFileSize(boolean seq) {
-    return seq ? seqFileSize.get() : unseqFileSize.get();
-  }
-
   public long getFileNum(boolean seq) {

Review Comment:
   is this a testonly function now?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to