You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/09/20 06:50:48 UTC

[GitHub] [iotdb] yyt86 opened a new pull request #4003: TsFileResourceManager

yyt86 opened a new pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003


   ##  Motivation
   The index files are stored as TsFileResource with DeviceTimeIndex in memory, which costs large memory and may have a risk of OOM.
   ## Description
   The idea about managing the TsFileResource is to degrade some DeviceTimeIndex to FileTimeIndex, so the index files that are memory resident can be more.  
   
   ## Implementation
   Create a new class named TsFileResourceManager to manage all tsFileResource in memory. Set a threshold for total memory cost of timeIndex. Once the total memory cost of timeIndex exceeds the threshold, trigger degrade process until the cost is lower than the threshold. The detailed implementation is shown in [confluence](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=184617885). And unit testing and integration testing are built for the class.
   
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] yyt86 commented on a change in pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
yyt86 commented on a change in pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#discussion_r716275692



##########
File path: server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
##########
@@ -296,6 +296,7 @@ private void redoLogs(
       }
 
       restorableTsFileIOWriter.endFile();
+      tsFileResource.close();

Review comment:
       We found the although the tsFileResource was set to be closed, the memory size of timeIndex for the recovered tsFileResource was larger than others and the close() method of TimeIndex was not called.




-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] JackieTien97 commented on a change in pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#discussion_r716121222



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -847,8 +848,34 @@ public void setTimeIndex(ITimeIndex timeIndex) {
     this.timeIndex = timeIndex;
   }
 
-  // change tsFile name
+  public byte getTimeIndexType() {
+    return timeIndexType;

Review comment:
       In my opinion, we don't need the `timeIndexType` field any more, because we already have  the `timeIndex` field. From  the `timeIndex` we can infer its `TimeIndexLevel`.

##########
File path: server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
##########
@@ -296,6 +296,7 @@ private void redoLogs(
       }
 
       restorableTsFileIOWriter.endFile();
+      tsFileResource.close();

Review comment:
       Why we need this line? We already set the closed flag of tsFileResource when necessary outside this `redologs` function.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -847,8 +848,34 @@ public void setTimeIndex(ITimeIndex timeIndex) {
     this.timeIndex = timeIndex;
   }
 
-  // change tsFile name
+  public byte getTimeIndexType() {
+    return timeIndexType;
+  }
+
+  public int compareIndexDegradePriority(TsFileResource tsFileResource) {
+    return timeIndex.compareDegradePriority(tsFileResource.timeIndex);
+  }
 
+  /** the DeviceTimeIndex degrade to FileTimeIndex and release memory */
+  public long degradeTimeIndex() {
+    TimeIndexLevel timeIndexLevel = TimeIndexLevel.valueOf(timeIndexType);
+    // if current timeIndex is FileTimeIndex, no need to degrade
+    if (timeIndexLevel == TimeIndexLevel.FILE_TIME_INDEX) return 0;
+    long previousMemory = calculateRamSize();
+    // get the minimum startTime
+    long startTime = timeIndex.getMinStartTime();
+    long endTime = Long.MIN_VALUE;
+    // get the maximum endTime
+    for (String devicesId : timeIndex.getDevices(file.getPath())) {
+      endTime = Math.max(endTime, timeIndex.getEndTime(devicesId));
+    }

Review comment:
       put this block into a function in ITimeIndex, like `getMinStartTime()`.

##########
File path: server/src/main/java/org/apache/iotdb/db/rescon/TsFileResourceManager.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.iotdb.db.rescon;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel;
+import org.apache.iotdb.db.utils.TestOnly;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.util.PriorityQueue;
+
+public class TsFileResourceManager {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileResourceManager.class);
+
+  private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
+
+  /** threshold total memory for all TimeIndex */
+  private double TIME_INDEX_MEMORY_THRESHOLD =
+      CONFIG.getAllocateMemoryForRead() * CONFIG.getTimeIndexMemoryProportion();
+
+  /** store the sealed TsFileResource, sorted by priority of TimeIndex */
+  private PriorityQueue<TsFileResource> sealedTsFileResources =
+      new PriorityQueue<>(TsFileResource::compareIndexDegradePriority);
+
+  /** total used memory for TimeIndex */
+  private long totalTimeIndexMemCost;
+
+  @TestOnly
+  public void setTimeIndexMemoryThreshold(double timeIndexMemoryThreshold) {
+    TIME_INDEX_MEMORY_THRESHOLD = timeIndexMemoryThreshold;
+  }
+
+  @TestOnly
+  public long getPriorityQueueSize() {
+    return sealedTsFileResources.size();
+  }
+
+  /**
+   * add the closed TsFileResource into priorityQueue and increase memory cost of timeIndex, once
+   * memory cost is larger than threshold, degradation is triggered.
+   */
+  public synchronized void registerSealedTsFileResource(TsFileResource tsFileResource) {
+    sealedTsFileResources.add(tsFileResource);
+    totalTimeIndexMemCost += tsFileResource.calculateRamSize();
+    chooseTsFileResourceToDegrade();
+  }
+
+  /** delete the TsFileResource in PriorityQueue when the source file is deleted */
+  public synchronized void removeTsFileResource(TsFileResource tsFileResource) {
+    sealedTsFileResources.remove(tsFileResource);
+    totalTimeIndexMemCost -= tsFileResource.calculateRamSize();
+  }
+
+  /** once degradation is triggered, the total memory for timeIndex should reduce */
+  public synchronized void releaseTimeIndexMemCost(long memCost) {
+    totalTimeIndexMemCost -= memCost;
+  }
+
+  /**
+   * choose the top TsFileResource in priorityQueue to degrade until the memory is smaller than
+   * threshold.
+   */
+  private void chooseTsFileResourceToDegrade() {
+    while (BigDecimal.valueOf(totalTimeIndexMemCost)
+            .compareTo(BigDecimal.valueOf(TIME_INDEX_MEMORY_THRESHOLD))
+        > 0) {
+      TsFileResource tsFileResource = sealedTsFileResources.poll();
+      if (TimeIndexLevel.valueOf(tsFileResource.getTimeIndexType())
+          == TimeIndexLevel.FILE_TIME_INDEX) {

Review comment:
       ```suggestion
         if (tsFileResource == null || TimeIndexLevel.valueOf(tsFileResource.getTimeIndexType())
             == TimeIndexLevel.FILE_TIME_INDEX) {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java
##########
@@ -700,6 +713,7 @@ private boolean merge(
                 i,
                 toMergeTsFiles.size());
             writeLock();
+            tsFileResourceManager.registerSealedTsFileResource(newResource);

Review comment:
       move it after the old tsfileResources being removed.

##########
File path: server/src/main/java/org/apache/iotdb/db/rescon/TsFileResourceManager.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.iotdb.db.rescon;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel;
+import org.apache.iotdb.db.utils.TestOnly;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.util.PriorityQueue;
+
+public class TsFileResourceManager {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileResourceManager.class);
+
+  private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
+
+  /** threshold total memory for all TimeIndex */
+  private double TIME_INDEX_MEMORY_THRESHOLD =
+      CONFIG.getAllocateMemoryForRead() * CONFIG.getTimeIndexMemoryProportion();
+
+  /** store the sealed TsFileResource, sorted by priority of TimeIndex */
+  private PriorityQueue<TsFileResource> sealedTsFileResources =

Review comment:
       ```suggestion
     private final PriorityQueue<TsFileResource> sealedTsFileResources =
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/rescon/TsFileResourceManager.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.iotdb.db.rescon;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel;
+import org.apache.iotdb.db.utils.TestOnly;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.util.PriorityQueue;
+
+public class TsFileResourceManager {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileResourceManager.class);
+
+  private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
+
+  /** threshold total memory for all TimeIndex */
+  private double TIME_INDEX_MEMORY_THRESHOLD =
+      CONFIG.getAllocateMemoryForRead() * CONFIG.getTimeIndexMemoryProportion();
+
+  /** store the sealed TsFileResource, sorted by priority of TimeIndex */
+  private PriorityQueue<TsFileResource> sealedTsFileResources =
+      new PriorityQueue<>(TsFileResource::compareIndexDegradePriority);
+
+  /** total used memory for TimeIndex */
+  private long totalTimeIndexMemCost;
+
+  @TestOnly
+  public void setTimeIndexMemoryThreshold(double timeIndexMemoryThreshold) {
+    TIME_INDEX_MEMORY_THRESHOLD = timeIndexMemoryThreshold;
+  }
+
+  @TestOnly
+  public long getPriorityQueueSize() {
+    return sealedTsFileResources.size();
+  }
+
+  /**
+   * add the closed TsFileResource into priorityQueue and increase memory cost of timeIndex, once
+   * memory cost is larger than threshold, degradation is triggered.
+   */
+  public synchronized void registerSealedTsFileResource(TsFileResource tsFileResource) {
+    sealedTsFileResources.add(tsFileResource);
+    totalTimeIndexMemCost += tsFileResource.calculateRamSize();
+    chooseTsFileResourceToDegrade();
+  }
+
+  /** delete the TsFileResource in PriorityQueue when the source file is deleted */
+  public synchronized void removeTsFileResource(TsFileResource tsFileResource) {
+    sealedTsFileResources.remove(tsFileResource);
+    totalTimeIndexMemCost -= tsFileResource.calculateRamSize();
+  }
+
+  /** once degradation is triggered, the total memory for timeIndex should reduce */
+  public synchronized void releaseTimeIndexMemCost(long memCost) {

Review comment:
       ```suggestion
     private void releaseTimeIndexMemCost(long memCost) {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java
##########
@@ -505,6 +517,7 @@ public void recover() {
                 modifications);
             // complete compaction and delete source file
             writeLock();
+            tsFileResourceManager.registerSealedTsFileResource(targetResource);

Review comment:
       Move it after the old tsfileResources being removed.

##########
File path: server/src/main/java/org/apache/iotdb/db/rescon/TsFileResourceManager.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.iotdb.db.rescon;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel;
+import org.apache.iotdb.db.utils.TestOnly;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.util.PriorityQueue;
+
+public class TsFileResourceManager {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileResourceManager.class);
+
+  private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
+
+  /** threshold total memory for all TimeIndex */
+  private double TIME_INDEX_MEMORY_THRESHOLD =
+      CONFIG.getAllocateMemoryForRead() * CONFIG.getTimeIndexMemoryProportion();
+
+  /** store the sealed TsFileResource, sorted by priority of TimeIndex */
+  private PriorityQueue<TsFileResource> sealedTsFileResources =
+      new PriorityQueue<>(TsFileResource::compareIndexDegradePriority);
+
+  /** total used memory for TimeIndex */
+  private long totalTimeIndexMemCost;
+
+  @TestOnly
+  public void setTimeIndexMemoryThreshold(double timeIndexMemoryThreshold) {
+    TIME_INDEX_MEMORY_THRESHOLD = timeIndexMemoryThreshold;
+  }
+
+  @TestOnly
+  public long getPriorityQueueSize() {
+    return sealedTsFileResources.size();
+  }
+
+  /**
+   * add the closed TsFileResource into priorityQueue and increase memory cost of timeIndex, once
+   * memory cost is larger than threshold, degradation is triggered.
+   */
+  public synchronized void registerSealedTsFileResource(TsFileResource tsFileResource) {
+    sealedTsFileResources.add(tsFileResource);
+    totalTimeIndexMemCost += tsFileResource.calculateRamSize();
+    chooseTsFileResourceToDegrade();
+  }
+
+  /** delete the TsFileResource in PriorityQueue when the source file is deleted */
+  public synchronized void removeTsFileResource(TsFileResource tsFileResource) {
+    sealedTsFileResources.remove(tsFileResource);

Review comment:
       The time complexity for`remove` function in `PriorityQueue` is O(n) which will be inefficient when there are many TsfileResources, maybe you can use another data structure to replace it, like `TreeSet` in java. The time complexity for`remove` function in `TreeSet` is O(log n)

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java
##########
@@ -306,4 +311,25 @@ public long getEndTime(String deviceId) {
   public boolean checkDeviceIdExist(String deviceId) {
     return deviceToIndex.containsKey(deviceId);
   }
+
+  @Override
+  public long getMinStartTime() {
+    long minStartTime = Long.MAX_VALUE;
+    for (int i = 0; i < deviceToIndex.size(); i++) {
+      minStartTime = Math.min(minStartTime, startTimes[i]);
+    }
+    return minStartTime;
+  }
+
+  @Override
+  public int compareDegradePriority(ITimeIndex timeIndex) {
+    if (timeIndex instanceof DeviceTimeIndex) {
+      return Long.compare(getMinStartTime(), timeIndex.getMinStartTime());
+    } else if (timeIndex instanceof FileTimeIndex) {
+      return -1;
+    } else {
+      logger.error("Wrong timeIndex type {}", timeIndex.getClass().getName());
+      throw new RuntimeException("Wrong timeIndex type " + timeIndex.getClass().getName());
+    }
+  }

Review comment:
       I think it's better to add a `minStartTime` field in DeviceTimeIndex, and update this field whenever new time is added. Because the `compareDegradePriority` may be called many times while adding or polling element in `PriorityQueue`, and each time it is called, it will iterate the `startTimes` array which is inefficient.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/compaction/level/LevelCompactionTsFileManagement.java
##########
@@ -473,6 +484,7 @@ public void recover() {
           } else {
             writer.close();
           }
+          tsFileResourceManager.registerSealedTsFileResource(targetTsFileResource);
           // complete compaction and delete source file
           deleteAllSubLevelFiles(isSeq, timePartition);

Review comment:
       ```suggestion
             // complete compaction and delete source file
             deleteAllSubLevelFiles(isSeq, timePartition);
             
             tsFileResourceManager.registerSealedTsFileResource(targetTsFileResource);
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/rescon/TsFileResourceManager.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.iotdb.db.rescon;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel;
+import org.apache.iotdb.db.utils.TestOnly;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.util.PriorityQueue;
+
+public class TsFileResourceManager {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileResourceManager.class);
+
+  private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
+
+  /** threshold total memory for all TimeIndex */
+  private double TIME_INDEX_MEMORY_THRESHOLD =
+      CONFIG.getAllocateMemoryForRead() * CONFIG.getTimeIndexMemoryProportion();
+
+  /** store the sealed TsFileResource, sorted by priority of TimeIndex */
+  private PriorityQueue<TsFileResource> sealedTsFileResources =
+      new PriorityQueue<>(TsFileResource::compareIndexDegradePriority);
+
+  /** total used memory for TimeIndex */
+  private long totalTimeIndexMemCost;
+
+  @TestOnly
+  public void setTimeIndexMemoryThreshold(double timeIndexMemoryThreshold) {
+    TIME_INDEX_MEMORY_THRESHOLD = timeIndexMemoryThreshold;
+  }
+
+  @TestOnly
+  public long getPriorityQueueSize() {
+    return sealedTsFileResources.size();
+  }
+
+  /**
+   * add the closed TsFileResource into priorityQueue and increase memory cost of timeIndex, once
+   * memory cost is larger than threshold, degradation is triggered.
+   */
+  public synchronized void registerSealedTsFileResource(TsFileResource tsFileResource) {
+    sealedTsFileResources.add(tsFileResource);
+    totalTimeIndexMemCost += tsFileResource.calculateRamSize();
+    chooseTsFileResourceToDegrade();
+  }
+
+  /** delete the TsFileResource in PriorityQueue when the source file is deleted */
+  public synchronized void removeTsFileResource(TsFileResource tsFileResource) {
+    sealedTsFileResources.remove(tsFileResource);
+    totalTimeIndexMemCost -= tsFileResource.calculateRamSize();
+  }
+
+  /** once degradation is triggered, the total memory for timeIndex should reduce */
+  public synchronized void releaseTimeIndexMemCost(long memCost) {
+    totalTimeIndexMemCost -= memCost;
+  }
+
+  /**
+   * choose the top TsFileResource in priorityQueue to degrade until the memory is smaller than
+   * threshold.
+   */
+  private void chooseTsFileResourceToDegrade() {
+    while (BigDecimal.valueOf(totalTimeIndexMemCost)
+            .compareTo(BigDecimal.valueOf(TIME_INDEX_MEMORY_THRESHOLD))
+        > 0) {

Review comment:
       ```suggestion
       while (totalTimeIndexMemCost > TIME_INDEX_MEMORY_THRESHOLD) {
   ```




-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] yyt86 commented on a change in pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
yyt86 commented on a change in pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#discussion_r716354127



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -847,8 +848,34 @@ public void setTimeIndex(ITimeIndex timeIndex) {
     this.timeIndex = timeIndex;
   }
 
-  // change tsFile name
+  public byte getTimeIndexType() {
+    return timeIndexType;

Review comment:
       But there is no method that can get <timeIndex> in <TsFileResource>. So I defined  a method to get the <timeIndexType>.




-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576






-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576


   
   [![Coverage Status](https://coveralls.io/builds/43077773/badge)](https://coveralls.io/builds/43077773)
   
   Coverage decreased (-0.09%) to 67.463% when pulling **b9bab773f6bb40485a0540bb73b0df5a0b7233f3 on yyt86:timeIndex_management** into **9dd88f08fcb9205212a0687192d960462b6efdc4 on apache:master**.
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576






-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576


   
   [![Coverage Status](https://coveralls.io/builds/43077794/badge)](https://coveralls.io/builds/43077794)
   
   Coverage decreased (-0.1%) to 67.457% when pulling **b9bab773f6bb40485a0540bb73b0df5a0b7233f3 on yyt86:timeIndex_management** into **9dd88f08fcb9205212a0687192d960462b6efdc4 on apache:master**.
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576


   
   [![Coverage Status](https://coveralls.io/builds/43108373/badge)](https://coveralls.io/builds/43108373)
   
   Coverage decreased (-0.1%) to 67.454% when pulling **da4082a9b848cd5c3e282249027fdf5abaa7ea12 on yyt86:timeIndex_management** into **9dd88f08fcb9205212a0687192d960462b6efdc4 on apache:master**.
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] JackieTien97 merged pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
JackieTien97 merged pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003


   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576


   
   [![Coverage Status](https://coveralls.io/builds/43135126/badge)](https://coveralls.io/builds/43135126)
   
   Coverage decreased (-0.1%) to 67.452% when pulling **8c6d7cfd4ecc5e155bb274429b86d95fa85b0c3c on yyt86:timeIndex_management** into **9dd88f08fcb9205212a0687192d960462b6efdc4 on apache:master**.
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576


   
   [![Coverage Status](https://coveralls.io/builds/43041236/badge)](https://coveralls.io/builds/43041236)
   
   Coverage decreased (-0.005%) to 67.551% when pulling **f57f3a4f1fe299bf7c81688c780fd527ef26155c on yyt86:timeIndex_management** into **9dd88f08fcb9205212a0687192d960462b6efdc4 on apache:master**.
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576


   
   [![Coverage Status](https://coveralls.io/builds/43074504/badge)](https://coveralls.io/builds/43074504)
   
   Coverage decreased (-0.1%) to 67.459% when pulling **853ffd57d98798d0e064e8b262b7bd915001f241 on yyt86:timeIndex_management** into **9dd88f08fcb9205212a0687192d960462b6efdc4 on apache:master**.
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576


   
   [![Coverage Status](https://coveralls.io/builds/43140258/badge)](https://coveralls.io/builds/43140258)
   
   Coverage increased (+0.2%) to 67.753% when pulling **fd69fec84f4d220339f36b8dd33c28ff49eee0c9 on yyt86:timeIndex_management** into **9dd88f08fcb9205212a0687192d960462b6efdc4 on apache:master**.
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576


   
   [![Coverage Status](https://coveralls.io/builds/43108383/badge)](https://coveralls.io/builds/43108383)
   
   Coverage decreased (-0.09%) to 67.471% when pulling **da4082a9b848cd5c3e282249027fdf5abaa7ea12 on yyt86:timeIndex_management** into **9dd88f08fcb9205212a0687192d960462b6efdc4 on apache:master**.
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] yyt86 commented on a change in pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
yyt86 commented on a change in pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#discussion_r716354127



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -847,8 +848,34 @@ public void setTimeIndex(ITimeIndex timeIndex) {
     this.timeIndex = timeIndex;
   }
 
-  // change tsFile name
+  public byte getTimeIndexType() {
+    return timeIndexType;

Review comment:
       But there is no method that can get timeIndex in TsFileResource. So I defined a method to get the timeIndexType.




-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls commented on pull request #4003: [IOTDB-1372]Enhance management of TsFileResource

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #4003:
URL: https://github.com/apache/iotdb/pull/4003#issuecomment-926322576


   
   [![Coverage Status](https://coveralls.io/builds/43040796/badge)](https://coveralls.io/builds/43040796)
   
   Coverage decreased (-0.01%) to 67.545% when pulling **5a0aec1090448214267b7de51e49729e70cd85eb on yyt86:timeIndex_management** into **9dd88f08fcb9205212a0687192d960462b6efdc4 on apache:master**.
   


-- 
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: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org