You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ha...@apache.org on 2023/03/06 02:05:39 UTC

[iotdb] branch fix_time_partition created (now 969eb03b16)

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

haonan pushed a change to branch fix_time_partition
in repository https://gitbox.apache.org/repos/asf/iotdb.git


      at 969eb03b16 [IOTDB-5620] Fix cannot flush when there is a lot of time partitions in each DataRegion

This branch includes the following new commits:

     new d7b0e2f7f6 fix
     new 969eb03b16 [IOTDB-5620] Fix cannot flush when there is a lot of time partitions in each DataRegion

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[iotdb] 01/02: fix

Posted by ha...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d7b0e2f7f629e25053ace84f881a1f3c40d47533
Author: HTHou <hh...@outlook.com>
AuthorDate: Fri Mar 3 22:52:09 2023 +0800

    fix
---
 .../main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java | 6 ++++++
 .../org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java     | 7 +++++++
 .../java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java    | 5 +++++
 .../src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java    | 2 ++
 server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java     | 2 +-
 server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java | 5 +++--
 .../apache/iotdb/db/engine/storagegroup/TimePartitionManager.java  | 5 +++++
 7 files changed, 29 insertions(+), 3 deletions(-)

diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
index c1bcbde4ec..fe9f3a6767 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppCommonConfig.java
@@ -334,4 +334,10 @@ public class MppCommonConfig extends MppBaseConfig implements CommonConfig {
     setProperty("schema_memory_allocate_proportion", String.valueOf(schemaMemoryAllocate));
     return this;
   }
+
+  @Override
+  public CommonConfig setWriteMemoryProportion(String writeMemoryProportion) {
+    setProperty("write_memory_proportion", writeMemoryProportion);
+    return this;
+  }
 }
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
index ddaba5a331..275ed1f8f8 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/MppSharedCommonConfig.java
@@ -343,4 +343,11 @@ public class MppSharedCommonConfig implements CommonConfig {
     cnConfig.setSchemaMemoryAllocate(schemaMemoryAllocate);
     return this;
   }
+
+  @Override
+  public CommonConfig setWriteMemoryProportion(String writeMemoryProportion) {
+    dnConfig.setWriteMemoryProportion(writeMemoryProportion);
+    cnConfig.setWriteMemoryProportion(writeMemoryProportion);
+    return this;
+  }
 }
diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
index 440b38253e..526df5dbab 100644
--- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/RemoteCommonConfig.java
@@ -246,4 +246,9 @@ public class RemoteCommonConfig implements CommonConfig {
   public CommonConfig setSchemaMemoryAllocate(String schemaMemoryAllocate) {
     return this;
   }
+
+  @Override
+  public CommonConfig setWriteMemoryProportion(String writeMemoryProportion) {
+    return this;
+  }
 }
diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
index 59dcd778ab..4792b8b3e7 100644
--- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
+++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java
@@ -110,4 +110,6 @@ public interface CommonConfig {
   CommonConfig setSeriesSlotNum(int seriesSlotNum);
 
   CommonConfig setSchemaMemoryAllocate(String schemaMemoryAllocate);
+
+  CommonConfig setWriteMemoryProportion(String writeMemoryProportion);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 75a461b006..2cf7c2b9ec 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -517,7 +517,7 @@ public class IoTDBConfig {
   private long allocateMemoryForTimeIndex = allocateMemoryForRead * 200 / 1001;
 
   /** Memory allocated proportion for time partition info */
-  private long allocateMemoryForTimePartitionInfo = allocateMemoryForStorageEngine * 50 / 1001;
+  private long allocateMemoryForTimePartitionInfo = 0;
   /**
    * If true, we will estimate each query's possible memory footprint before executing it and deny
    * it if its estimated memory exceeds current free memory
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index a4c40333ab..f83968c736 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -1606,13 +1606,14 @@ public class IoTDBDescriptor {
 
     String allocationRatioForWrite = properties.getProperty("write_memory_proportion", "19:1");
     proportions = allocationRatioForWrite.split(":");
-    int proportionForMemTable = Integer.parseInt(proportions[0].trim());
+    int proportionForMemTable = Integer.parseInt(proportions[0].replace("\\", "").trim());
     int proportionForTimePartitionInfo = Integer.parseInt(proportions[1].trim());
+    logger.info("proportionForTimePartitionInfo = {}", proportionForTimePartitionInfo);
 
     double memtableProportionForWrite =
         ((double) (proportionForMemTable)
             / (double) (proportionForMemTable + proportionForTimePartitionInfo));
-    Double.parseDouble(properties.getProperty("flush_time_memory_proportion", "0.05"));
+
     double timePartitionInfoForWrite =
         ((double) (proportionForTimePartitionInfo)
             / (double) (proportionForMemTable + proportionForTimePartitionInfo));
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java
index 1f77b36b82..18e65b8996 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java
@@ -24,6 +24,9 @@ import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngine;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.HashMap;
 import java.util.Map;
 import java.util.TreeMap;
@@ -31,6 +34,7 @@ import java.util.TreeSet;
 
 /** Manage all the time partitions for all data regions and control the total memory of them */
 public class TimePartitionManager {
+  private static final Logger logger = LoggerFactory.getLogger(TimePartitionManager.class);
   final Map<DataRegionId, Map<Long, TimePartitionInfo>> timePartitionInfoMap;
 
   long memCost = 0;
@@ -74,6 +78,7 @@ public class TimePartitionManager {
         memCost += memSize - timePartitionInfo.memSize;
         timePartitionInfo.memSize = memSize;
         timePartitionInfo.isActive = isActive;
+        logger.error("{}", timePartitionInfoMemoryThreshold);
         if (memCost > timePartitionInfoMemoryThreshold) {
           evictOldPartition();
         }


[iotdb] 02/02: [IOTDB-5620] Fix cannot flush when there is a lot of time partitions in each DataRegion

Posted by ha...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 969eb03b16420a2584e89d27e46cca4848d846fa
Author: HTHou <hh...@outlook.com>
AuthorDate: Mon Mar 6 10:05:21 2023 +0800

    [IOTDB-5620] Fix cannot flush when there is a lot of time partitions in each DataRegion
---
 .../iotdb/db/it/IoTDBInsertMultiPartitionIT.java   | 71 ++++++++++++++++++++++
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  3 +-
 .../engine/storagegroup/TimePartitionManager.java  |  5 +-
 3 files changed, 76 insertions(+), 3 deletions(-)

diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBInsertMultiPartitionIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBInsertMultiPartitionIT.java
new file mode 100644
index 0000000000..fac1175f74
--- /dev/null
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBInsertMultiPartitionIT.java
@@ -0,0 +1,71 @@
+/*
+ * 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.it;
+
+import org.apache.iotdb.it.env.EnvFactory;
+import org.apache.iotdb.it.framework.IoTDBTestRunner;
+import org.apache.iotdb.itbase.category.ClusterIT;
+import org.apache.iotdb.itbase.category.LocalStandaloneIT;
+import org.apache.iotdb.itbase.category.RemoteIT;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.sql.Connection;
+import java.sql.Statement;
+
+import static org.junit.Assert.fail;
+
+@RunWith(IoTDBTestRunner.class)
+@Category({LocalStandaloneIT.class, ClusterIT.class, RemoteIT.class})
+public class IoTDBInsertMultiPartitionIT {
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvFactory.getEnv()
+        .getConfig()
+        .getDataNodeCommonConfig()
+        .setWriteMemoryProportion("10000000:1");
+    EnvFactory.getEnv().initClusterEnvironment();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    EnvFactory.getEnv().cleanClusterEnvironment();
+  }
+
+  @Test
+  public void testInsertMultiPartition() {
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      statement.execute("insert into root.sg.d1(time,s1) values(1,2)");
+      statement.execute("flush");
+      statement.execute("insert into root.sg.d1(time,s1) values(2,2)");
+      statement.execute("insert into root.sg.d1(time,s1) values(604800001,2)");
+      statement.execute("flush");
+    } catch (Exception e) {
+      fail(e.getMessage());
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 2cf7c2b9ec..70a0b0f3da 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -517,7 +517,7 @@ public class IoTDBConfig {
   private long allocateMemoryForTimeIndex = allocateMemoryForRead * 200 / 1001;
 
   /** Memory allocated proportion for time partition info */
-  private long allocateMemoryForTimePartitionInfo = 0;
+  private long allocateMemoryForTimePartitionInfo = allocateMemoryForStorageEngine * 50 / 1001;
   /**
    * If true, we will estimate each query's possible memory footprint before executing it and deny
    * it if its estimated memory exceeds current free memory
@@ -1785,7 +1785,6 @@ public class IoTDBConfig {
 
   public void setAllocateMemoryForStorageEngine(long allocateMemoryForStorageEngine) {
     this.allocateMemoryForStorageEngine = allocateMemoryForStorageEngine;
-    this.allocateMemoryForTimePartitionInfo = allocateMemoryForStorageEngine * 50 / 1001;
   }
 
   public long getAllocateMemoryForSchema() {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java
index 18e65b8996..ded5be3d28 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TimePartitionManager.java
@@ -107,7 +107,10 @@ public class TimePartitionManager {
       }
 
       while (memCost > timePartitionInfoMemoryThreshold) {
-        TimePartitionInfo timePartitionInfo = treeSet.first();
+        TimePartitionInfo timePartitionInfo = treeSet.pollFirst();
+        if (timePartitionInfo == null) {
+          return;
+        }
         memCost -= timePartitionInfo.memSize;
         DataRegion dataRegion =
             StorageEngine.getInstance().getDataRegion(timePartitionInfo.dataRegionId);