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 14:12:23 UTC

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

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



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -63,16 +64,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;
 
   /**
-   * device -> start time
+   * start times array. 
    */
-  protected Map<String, Long> startTimeMap;
+  private long[] startTimes;
 
   /**
-   * device -> end time. It is null if it's an unsealed sequence tsfile
+   * end times array. The values in this array are -1 if it's an unsealed sequence tsfile

Review comment:
       ```suggestion
      * end times array. The values in this array are Long.MIN_VALUE if it's an unsealed sequence tsfile
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -221,18 +237,22 @@ private void generateTimeSeriesMetadata() throws IOException {
     }
   }
 
+  private void initTimes(long[] times) {
+    Arrays.fill(times, -1);

Review comment:
       You can fill startTime by MAX_VALUE and fill endTimes by MIN_VALUE

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -339,12 +362,106 @@ public long getFileSize() {
     return file.length();
   }
 
-  public Map<String, Long> getStartTimeMap() {
-    return startTimeMap;
+  public long getStartTime(String deviceId) {
+    if (!deviceToIndex.containsKey(deviceId)) {
+      return -1;

Review comment:
       ```suggestion
         return Long.MAX_VALUE;
   ```
   is this ok?

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -283,16 +306,16 @@ public void deserialize() throws IOException {
   }
 
   public void updateStartTime(String device, long time) {
-    long startTime = startTimeMap.getOrDefault(device, Long.MAX_VALUE);
+    long startTime = getOrDefaultStartTime(device, Long.MAX_VALUE);

Review comment:
       You can fill startTime by MAX_VALUE and fill endTimes by MIN_VALUE

##########
File path: server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
##########
@@ -282,9 +282,8 @@ public void testWriteAndClose() throws IOException, WriteProcessException{
   private void closeTsFileProcessor(TsFileProcessor unsealedTsFileProcessor) throws TsFileProcessorException {
     TsFileResource resource = unsealedTsFileProcessor.getTsFileResource();
     synchronized (resource) {
-      for (Entry<String, Long> startTime : resource.getStartTimeMap().entrySet()) {
-        String deviceId = startTime.getKey();
-        resource.getEndTimeMap().put(deviceId, resource.getStartTimeMap().get(deviceId));
+      for (Entry<String, Integer> entry : resource.getDeviceToIndexMap().entrySet()) {
+        resource.putEndTime(entry.getKey(), resource.getStartTime(entry.getValue()));

Review comment:
       I don't understand why here set start time to end time before.....

##########
File path: server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
##########
@@ -155,8 +155,8 @@ private void replayBatchInsert(InsertTabletPlan insertTabletPlan)
   private void replayInsert(InsertPlan insertPlan) {
     if (currentTsFileResource != null) {
       // the last chunk group may contain the same data with the logs, ignore such logs in seq file
-      Long lastEndTime = currentTsFileResource.getEndTimeMap().get(insertPlan.getDeviceId());
-      if (lastEndTime != null && lastEndTime >= insertPlan.getTime() &&
+      long lastEndTime = currentTsFileResource.getEndTime(insertPlan.getDeviceId());
+      if (lastEndTime >= 0 && lastEndTime >= insertPlan.getTime() &&

Review comment:
       the same 

##########
File path: server/src/main/java/org/apache/iotdb/db/query/executor/fill/LastPointReader.java
##########
@@ -183,11 +182,8 @@ private boolean shouldUpdate(long time, long version, long newTime, long newVers
     PriorityQueue<TsFileResource> unseqTsFilesSet =
         new PriorityQueue<>(
             (o1, o2) -> {
-              Map<String, Long> startTimeMap = o1.getEndTimeMap();
-              Long minTimeOfO1 = startTimeMap.get(seriesPath.getDevice());
-              Map<String, Long> startTimeMap2 = o2.getEndTimeMap();
-              Long minTimeOfO2 = startTimeMap2.get(seriesPath.getDevice());
-
+              Long minTimeOfO1 = o1.getEndTime(seriesPath.getDevice());
+              Long minTimeOfO2 = o2.getEndTime(seriesPath.getDevice());

Review comment:
       ```suggestion
                 Long maxTimeOfO1 = o1.getEndTime(seriesPath.getDevice());
                 Long maxTimeOfO2 = o2.getEndTime(seriesPath.getDevice());
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeMultiChunkTask.java
##########
@@ -152,9 +152,9 @@ private void pathsMergeOneFile(int seqFileIdx, IPointReader[] unseqReaders)
       throws IOException {
     TsFileResource currTsFile = resource.getSeqFiles().get(seqFileIdx);
     String deviceId = currMergingPaths.get(0).getDevice();
-    Long currDeviceMinTime = currTsFile.getStartTimeMap().get(deviceId);
+    long currDeviceMinTime = currTsFile.getStartTime(deviceId);
     //COMMENTS: is this correct? how about if there are other devices (in the currMergingPaths) that have unseq data?
-    if (currDeviceMinTime == null) {
+    if (currDeviceMinTime < 0) {

Review comment:
       == MAX_VALUE

##########
File path: server/src/main/java/org/apache/iotdb/db/writelog/recover/LogReplayer.java
##########
@@ -127,8 +127,8 @@ private void replayBatchInsert(InsertTabletPlan insertTabletPlan)
       throws WriteProcessException, QueryProcessException {
     if (currentTsFileResource != null) {
       // the last chunk group may contain the same data with the logs, ignore such logs in seq file
-      Long lastEndTime = currentTsFileResource.getEndTimeMap().get(insertTabletPlan.getDeviceId());
-      if (lastEndTime != null && lastEndTime >= insertTabletPlan.getMinTime() &&
+      long lastEndTime = currentTsFileResource.getEndTime(insertTabletPlan.getDeviceId());
+      if (lastEndTime >= 0 && lastEndTime >= insertTabletPlan.getMinTime() &&

Review comment:
       ```suggestion
         if (lastEndTime != Long.MIN_VALUE && lastEndTime >= insertTabletPlan.getMinTime() &&
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
##########
@@ -339,12 +362,106 @@ 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 long getStartTime(int index) {
+    return startTimes[index];
+  }
+
+  public long getEndTime(String deviceId) {
+    if (!deviceToIndex.containsKey(deviceId)) {
+      return -1;

Review comment:
       ```suggestion
         return Long.MIN_TIME;
   ```




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