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 2020/05/27 04:14:11 UTC

[GitHub] [incubator-iotdb] HTHou commented on a change in pull request #1269: [IOTDB-707] Optimize TsFileResource memory usage

HTHou commented on a change in pull request #1269:
URL: https://github.com/apache/incubator-iotdb/pull/1269#discussion_r430305568



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -63,16 +63,22 @@
   public static final String RESOURCE_SUFFIX = ".resource";
   static final String TEMP_SUFFIX = ".temp";
   private static final String CLOSING_SUFFIX = ".closing";
+  private static final int INIT_ARRAY_SIZE = 64;

Review comment:
       I'm not sure if this init array size is good or not. Please take a look. 

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -339,12 +363,91 @@ public long getFileSize() {
     return file.length();
   }
 
-  public Map<String, Long> getStartTimeMap() {
-    return startTimeMap;
+  public long getStartTime(String deviceId) {
+    if (!deviceToIndex.containsKey(deviceId)) {
+      return -1;
+    }
+    return startTimes[deviceToIndex.get(deviceId)];
   }
 
-  public Map<String, Long> getEndTimeMap() {
-    return endTimeMap;
+  public long getEndTime(String deviceId) {
+    if (!deviceToIndex.containsKey(deviceId)) {
+      return -1;
+    }
+    return endTimes[deviceToIndex.get(deviceId)];
+  }
+
+  public long getOrDefaultStartTime(String deviceId, long defaultTime) {
+    return getStartTime(deviceId) >= 0 ? startTimes[deviceToIndex.get(deviceId)] : defaultTime;

Review comment:
       Fixed~

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -339,12 +363,91 @@ public long getFileSize() {
     return file.length();
   }
 
-  public Map<String, Long> getStartTimeMap() {
-    return startTimeMap;
+  public long getStartTime(String deviceId) {
+    if (!deviceToIndex.containsKey(deviceId)) {
+      return -1;
+    }
+    return startTimes[deviceToIndex.get(deviceId)];
   }
 
-  public Map<String, Long> getEndTimeMap() {
-    return endTimeMap;
+  public long getEndTime(String deviceId) {
+    if (!deviceToIndex.containsKey(deviceId)) {
+      return -1;
+    }
+    return endTimes[deviceToIndex.get(deviceId)];
+  }
+
+  public long getOrDefaultStartTime(String deviceId, long defaultTime) {
+    return getStartTime(deviceId) >= 0 ? startTimes[deviceToIndex.get(deviceId)] : defaultTime;
+  }
+
+  public long getOrDefaultEndTime(String deviceId, long defaultTime) {
+    return getEndTime(deviceId) >= 0 ? endTimes[deviceToIndex.get(deviceId)] : defaultTime;

Review comment:
       Fixed~

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -318,12 +318,18 @@ private void recover() throws StorageGroupProcessorException {
 
     for (TsFileResource resource : sequenceFileTreeSet) {
       long timePartitionId = resource.getTimePartition();
+      Map<String, Long> endTimeMap = new HashMap<>();
+      for (Entry<String, Integer> entry : resource.getDeviceToIndexMap().entrySet()) {
+        String deviceId = entry.getKey();
+        long endTime = resource.getEndTime(deviceId);

Review comment:
       Fixed

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/merge/selector/MaxFileMergeFileSelector.java
##########
@@ -213,18 +213,18 @@ private boolean checkClosed(TsFileResource unseqFile) {
   private void selectOverlappedSeqFiles(TsFileResource unseqFile) {
 
     int tmpSelectedNum = 0;
-    for (Entry<String, Long> deviceStartTimeEntry : unseqFile.getStartTimeMap().entrySet()) {
+    for (Entry<String, Integer> deviceStartTimeEntry : unseqFile.getDeviceToIndexMap().entrySet()) {
       String deviceId = deviceStartTimeEntry.getKey();
-      Long unseqStartTime = deviceStartTimeEntry.getValue();
-      Long unseqEndTime = unseqFile.getEndTimeMap().get(deviceId);
+      long unseqStartTime = unseqFile.getStartTime(deviceId);
+      long unseqEndTime = unseqFile.getEndTime(deviceId);

Review comment:
       Fixed

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -344,16 +350,16 @@ private void updateLastestFlushedTime() throws IOException {
     VersionController versionController = new SimpleFileVersionController(storageGroupSysDir.getPath());
     long currentVersion = versionController.currVersion();
     for (TsFileResource resource : upgradeSeqFileList) {
-      for (Entry<String, Long> entry : resource.getEndTimeMap().entrySet()) {
+      for (Entry<String, Integer> entry : resource.getDeviceToIndexMap().entrySet()) {
         String deviceId = entry.getKey();
-        long endTime = entry.getValue();
+        long endTime = resource.getEndTime(deviceId);

Review comment:
       Fixed~

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -344,16 +350,16 @@ private void updateLastestFlushedTime() throws IOException {
     VersionController versionController = new SimpleFileVersionController(storageGroupSysDir.getPath());
     long currentVersion = versionController.currVersion();
     for (TsFileResource resource : upgradeSeqFileList) {
-      for (Entry<String, Long> entry : resource.getEndTimeMap().entrySet()) {
+      for (Entry<String, Integer> entry : resource.getDeviceToIndexMap().entrySet()) {
         String deviceId = entry.getKey();
-        long endTime = entry.getValue();
+        long endTime = resource.getEndTime(deviceId);
         long endTimePartitionId = StorageEngine.getTimePartition(endTime);
         latestTimeForEachDevice.computeIfAbsent(endTimePartitionId, l -> new HashMap<>())
                 .put(deviceId, endTime);
-        globalLatestFlushedTimeForEachDevice.putAll(resource.getEndTimeMap());
+        globalLatestFlushedTimeForEachDevice.put(deviceId, endTime);
 
         // set all the covered partition's LatestFlushedTime to Long.MAX_VALUE
-        long partitionId = StorageEngine.getTimePartition(resource.startTimeMap.get(deviceId));
+        long partitionId = StorageEngine.getTimePartition(resource.getStartTime(deviceId));

Review comment:
       Fixed~

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -63,16 +63,22 @@
   public static final String RESOURCE_SUFFIX = ".resource";
   static final String TEMP_SUFFIX = ".temp";
   private static final String CLOSING_SUFFIX = ".closing";
+  private static final int INIT_ARRAY_SIZE = 64;

Review comment:
       Here is an advice by @qiaojialin. When closing a resource, remove the redundant length of `startTimes` and `endTimes`. And when enlarging the arrays, use ` * 1.5` instead. 
   By the way, the maximum number of devices in this TsFileResource shouldn't exceed the number of devices in the storage group. I'm looking for a way to find out this number.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -63,16 +63,22 @@
   public static final String RESOURCE_SUFFIX = ".resource";
   static final String TEMP_SUFFIX = ".temp";
   private static final String CLOSING_SUFFIX = ".closing";
+  private static final int INIT_ARRAY_SIZE = 64;

Review comment:
       Here is an advice from @qiaojialin. When closing a resource, remove the redundant length of `startTimes` and `endTimes`. And when enlarging the arrays, use ` * 1.5` instead. 
   By the way, the maximum number of devices in this TsFileResource shouldn't exceed the number of devices in the storage group. I'm looking for a way to find out this number.




----------------------------------------------------------------
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.

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