You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2021/07/20 08:15:11 UTC

[iotdb] branch autoai updated: fix recover TsFileResource time not accurate

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

qiaojialin pushed a commit to branch autoai
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/autoai by this push:
     new f276488  fix recover TsFileResource time not accurate
f276488 is described below

commit f2764888cbfa9e3b87b75cb5b4cd035ae686be92
Author: qiaojialin <64...@qq.com>
AuthorDate: Tue Jul 20 16:13:50 2021 +0800

    fix recover TsFileResource time not accurate
---
 .../writelog/recover/TsFileRecoverPerformer.java   | 25 ++++++++---
 .../iotdb/db/integration/IoTDBRestartIT.java       | 48 ++++++++++++++++++++++
 2 files changed, 67 insertions(+), 6 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java b/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
index 72ea41c..da53108 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
@@ -41,6 +41,8 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -188,13 +190,24 @@ public class TsFileRecoverPerformer {
     for (Map.Entry<String, List<ChunkMetadata>> entry : deviceChunkMetaDataMap.entrySet()) {
       String deviceId = entry.getKey();
       List<ChunkMetadata> chunkMetadataList = entry.getValue();
-      TSDataType dataType = entry.getValue().get(entry.getValue().size() - 1).getDataType();
-      for (ChunkMetadata chunkMetaData : chunkMetadataList) {
-        if (!chunkMetaData.getDataType().equals(dataType)) {
-          continue;
+
+      Map<String, List<ChunkMetadata>> measurementToChunkMetadatas = new HashMap<>();
+      for (ChunkMetadata chunkMetadata : chunkMetadataList) {
+        List<ChunkMetadata> list =
+            measurementToChunkMetadatas.computeIfAbsent(
+                chunkMetadata.getMeasurementUid(), n -> new ArrayList<>());
+        list.add(chunkMetadata);
+      }
+
+      for (List<ChunkMetadata> metadataList : measurementToChunkMetadatas.values()) {
+        TSDataType dataType = metadataList.get(metadataList.size() - 1).getDataType();
+        for (ChunkMetadata chunkMetaData : chunkMetadataList) {
+          if (!chunkMetaData.getDataType().equals(dataType)) {
+            continue;
+          }
+          tsFileResource.updateStartTime(deviceId, chunkMetaData.getStartTime());
+          tsFileResource.updateEndTime(deviceId, chunkMetaData.getEndTime());
         }
-        tsFileResource.updateStartTime(deviceId, chunkMetaData.getStartTime());
-        tsFileResource.updateEndTime(deviceId, chunkMetaData.getEndTime());
       }
     }
     tsFileResource.updatePlanIndexes(restorableTsFileIOWriter.getMinPlanIndex());
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
index d01b9d9..1395c59 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBRestartIT.java
@@ -18,6 +18,8 @@
  */
 package org.apache.iotdb.db.integration;
 
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.compaction.CompactionMergeTaskPoolManager;
 import org.apache.iotdb.db.exception.StorageEngineException;
@@ -353,6 +355,52 @@ public class IoTDBRestartIT {
   }
 
   @Test
+  public void testRecoverWALDeleteSchemaCheckResourceTime() throws Exception {
+    EnvironmentUtils.envSetUp();
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+    int avgSeriesPointNumberThreshold = config.getAvgSeriesPointNumberThreshold();
+    config.setAvgSeriesPointNumberThreshold(2);
+    long tsfileSize = config.getTsFileSizeThreshold();
+    config.setTsFileSizeThreshold(10000000);
+
+    try (Connection connection =
+            DriverManager.getConnection(
+                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute("create timeseries root.turbine1.d1.s1 with datatype=INT64");
+      statement.execute("insert into root.turbine1.d1(timestamp,s1) values(1,1)");
+      statement.execute("insert into root.turbine1.d1(timestamp,s1) values(2,1)");
+      statement.execute("create timeseries root.turbine1.d1.s2 with datatype=BOOLEAN");
+      statement.execute("insert into root.turbine1.d1(timestamp,s2) values(3,true)");
+      statement.execute("insert into root.turbine1.d1(timestamp,s2) values(4,true)");
+    }
+
+    Thread.sleep(1000);
+    EnvironmentUtils.restartDaemon();
+
+    try (Connection connection =
+            DriverManager.getConnection(
+                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      long[] result = new long[] {1L, 2L};
+      statement.execute("select s1 from root.turbine1.d1 where time < 3");
+      ResultSet resultSet = statement.getResultSet();
+      int cnt = 0;
+      while (resultSet.next()) {
+        assertEquals(resultSet.getLong(1), result[cnt]);
+        cnt++;
+      }
+      assertEquals(2, cnt);
+    }
+
+    config.setAvgSeriesPointNumberThreshold(avgSeriesPointNumberThreshold);
+    config.setTsFileSizeThreshold(tsfileSize);
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
   public void testRestartCompaction()
       throws SQLException, ClassNotFoundException, IOException, StorageEngineException {
     EnvironmentUtils.envSetUp();