You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hu...@apache.org on 2022/07/07 15:48:19 UTC

[iotdb] 06/14: add system config

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

hui pushed a commit to branch lmh/aggrOpRefactor
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit e08b452659e072c2e20779d128eba96ab5509705
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Wed Jul 6 17:44:07 2022 +0800

    add system config
---
 docs/UserGuide/Reference/DataNode-Config-Manual.md | 28 +++++++++++++++++++++-
 .../UserGuide/Reference/DataNode-Config-Manual.md  | 28 +++++++++++++++++++++-
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 11 +++++++++
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  | 22 +++++++++++++++++
 .../mpp/execution/schedule/DriverTaskThread.java   |  6 ++++-
 .../iotdb/tsfile/common/conf/TSFileConfig.java     | 22 +++++++++++++++++
 .../tsfile/read/common/block/TsBlockBuilder.java   |  6 ++++-
 .../read/common/block/TsBlockBuilderStatus.java    |  4 +++-
 8 files changed, 122 insertions(+), 5 deletions(-)

diff --git a/docs/UserGuide/Reference/DataNode-Config-Manual.md b/docs/UserGuide/Reference/DataNode-Config-Manual.md
index 5e5c5ba473..55bb5bf3d4 100644
--- a/docs/UserGuide/Reference/DataNode-Config-Manual.md
+++ b/docs/UserGuide/Reference/DataNode-Config-Manual.md
@@ -199,7 +199,7 @@ The permission definitions are in ${IOTDB\_CONF}/conf/jmx.access.
 |Default| 5 |
 |Effective|After restarting system|
 
-* mpp\_data\_exchange\_core\_pool\_size=1
+* mpp\_data\_exchange\_keep\_alive\_time\_in\_ms
 
 |Name| mpp\_data\_exchange\_keep\_alive\_time\_in\_ms |
 |:---:|:---|
@@ -208,6 +208,32 @@ The permission definitions are in ${IOTDB\_CONF}/conf/jmx.access.
 |Default| 1000 |
 |Effective|After restarting system|
 
+* driver\_task\_execution\_time\_slice\_in\_ms
+
+|Name| driver\_task\_execution\_time\_slice\_in\_ms |
+|:---:|:---|
+|Description| Maximum execution time of a DriverTask |
+|Type| int |
+|Default| 100 |
+|Effective|After restarting system|
+
+* max\_tsblock\_size\_in\_bytes
+
+|Name| max\_tsblock\_size\_in\_bytes |
+|:---:|:---|
+|Description| Maximum capacity of a TsBlock |
+|Type| int |
+|Default| 1024 * 1024 (1 MB) |
+|Effective|After restarting system|
+
+* max\_tsblock\_line\_numbers
+
+|Name| max\_tsblock\_line\_numbers |
+|:---:|:---|
+|Description| Maximum number of lines in a single TsBlock |
+|Type| int |
+|Default| 1024 |
+|Effective|After restarting system|
 
 ### DataNode Internal Service
 
diff --git a/docs/zh/UserGuide/Reference/DataNode-Config-Manual.md b/docs/zh/UserGuide/Reference/DataNode-Config-Manual.md
index c6f8f36671..de10f78867 100644
--- a/docs/zh/UserGuide/Reference/DataNode-Config-Manual.md
+++ b/docs/zh/UserGuide/Reference/DataNode-Config-Manual.md
@@ -181,7 +181,7 @@ IoTDB DataNode 与 Standalone 模式共用一套配置文件,均位于 IoTDB 
 |默认值| 5 |
 |改后生效方式|重启服务生效|
 
-* mpp\_data\_exchange\_core\_pool\_size
+* mpp\_data\_exchange\_keep\_alive\_time\_in\_ms
 
 |名字| mpp\_data\_exchange\_keep\_alive\_time\_in\_ms |
 |:---:|:---|
@@ -190,6 +190,32 @@ IoTDB DataNode 与 Standalone 模式共用一套配置文件,均位于 IoTDB 
 |默认值| 1000 |
 |改后生效方式|重启服务生效|
 
+* driver\_task\_execution\_time\_slice\_in\_ms
+
+|名字| driver\_task\_execution\_time\_slice\_in\_ms |
+|:---:|:---|
+|描述| 单个 DriverTask 最长执行时间 |
+|类型| int |
+|默认值| 100 |
+|改后生效方式|重启服务生效|
+
+* max\_tsblock\_size\_in\_bytes
+
+|名字| max\_tsblock\_size\_in\_bytes |
+|:---:|:---|
+|描述| 单个 TsBlock 的最大容量 |
+|类型| int |
+|默认值| 1024 * 1024 (1 MB) |
+|改后生效方式|重启服务生效|
+
+* max\_tsblock\_line\_numbers
+
+|名字| max\_tsblock\_line\_numbers |
+|:---:|:---|
+|描述| 单个 TsBlock 的最大行数 |
+|类型| int |
+|默认值| 1024 |
+|改后生效方式|重启服务生效|
 
 ### DataNode 内部服务参数
 
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 a2c8ce8faa..bd8a13c914 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
@@ -948,6 +948,9 @@ public class IoTDBConfig {
   /** Memory allocated for LastCache */
   private long allocateMemoryForLastCache = allocateMemoryForSchema / 10;
 
+  /** Maximum execution time of a DriverTask */
+  private int driverTaskExecutionTimeSliceInMs = 100;
+
   IoTDBConfig() {}
 
   public float getUdfMemoryBudgetInMB() {
@@ -3009,4 +3012,12 @@ public class IoTDBConfig {
   public void setAllocateMemoryForLastCache(long allocateMemoryForLastCache) {
     this.allocateMemoryForLastCache = allocateMemoryForLastCache;
   }
+
+  public int getDriverTaskExecutionTimeSliceInMs() {
+    return driverTaskExecutionTimeSliceInMs;
+  }
+
+  public void setDriverTaskExecutionTimeSliceInMs(int driverTaskExecutionTimeSliceInMs) {
+    this.driverTaskExecutionTimeSliceInMs = driverTaskExecutionTimeSliceInMs;
+  }
 }
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 a25d6674d7..2c3a2a6ed0 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
@@ -1198,6 +1198,22 @@ public class IoTDBDescriptor {
                     "max_degree_of_index_node",
                     Integer.toString(
                         TSFileDescriptor.getInstance().getConfig().getMaxDegreeOfIndexNode()))));
+    TSFileDescriptor.getInstance()
+        .getConfig()
+        .setMaxTsBlockSizeInBytes(
+            Integer.parseInt(
+                properties.getProperty(
+                    "max_tsblock_size_in_bytes",
+                    Integer.toString(
+                        TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes()))));
+    TSFileDescriptor.getInstance()
+        .getConfig()
+        .setMaxTsBlockLineNumber(
+            Integer.parseInt(
+                properties.getProperty(
+                    "max_tsblock_line_number",
+                    Integer.toString(
+                        TSFileDescriptor.getInstance().getConfig().getMaxTsBlockLineNumber()))));
   }
 
   // Mqtt related
@@ -1723,6 +1739,12 @@ public class IoTDBDescriptor {
         Integer.parseInt(
             properties.getProperty(
                 "partition_cache_size", Integer.toString(conf.getPartitionCacheSize()))));
+
+    conf.setDriverTaskExecutionTimeSliceInMs(
+        Integer.parseInt(
+            properties.getProperty(
+                "driver_task_execution_time_slice_in_ms",
+                Integer.toString(conf.getDriverTaskExecutionTimeSliceInMs()))));
   }
 
   /** Get default encode algorithm by data type */
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskThread.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskThread.java
index 02a795cff3..f49806ac31 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskThread.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/schedule/DriverTaskThread.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.execution.schedule;
 
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.mpp.execution.driver.IDriver;
 import org.apache.iotdb.db.mpp.execution.schedule.queue.IndexedBlockingQueue;
 import org.apache.iotdb.db.mpp.execution.schedule.task.DriverTask;
@@ -34,7 +35,10 @@ import java.util.concurrent.TimeUnit;
 /** the worker thread of {@link DriverTask} */
 public class DriverTaskThread extends AbstractDriverThread {
 
-  public static final Duration EXECUTION_TIME_SLICE = new Duration(100, TimeUnit.MILLISECONDS);
+  public static final Duration EXECUTION_TIME_SLICE =
+      new Duration(
+          IoTDBDescriptor.getInstance().getConfig().getDriverTaskExecutionTimeSliceInMs(),
+          TimeUnit.MILLISECONDS);
 
   // As the callback is lightweight enough, there's no need to use another one thread to execute.
   private static final Executor listeningExecutor = MoreExecutors.directExecutor();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
index 5d36d6286b..0b991aed74 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
@@ -150,6 +150,12 @@ public class TSFileConfig implements Serializable {
   /** The amount of data iterate each time */
   private int batchSize = 1000;
 
+  /** Maximum capacity of a TsBlock */
+  private int maxTsBlockSizeInBytes = 1024 * 1024;
+
+  /** Maximum number of lines in a single TsBlock */
+  private int maxTsBlockLineNumber = 1024;
+
   public TSFileConfig() {}
 
   public int getGroupSizeInByte() {
@@ -431,4 +437,20 @@ public class TSFileConfig implements Serializable {
   public void setFreqEncodingBlockSize(int freqEncodingBlockSize) {
     this.freqEncodingBlockSize = freqEncodingBlockSize;
   }
+
+  public int getMaxTsBlockSizeInBytes() {
+    return maxTsBlockSizeInBytes;
+  }
+
+  public void setMaxTsBlockSizeInBytes(int maxTsBlockSizeInBytes) {
+    this.maxTsBlockSizeInBytes = maxTsBlockSizeInBytes;
+  }
+
+  public int getMaxTsBlockLineNumber() {
+    return maxTsBlockLineNumber;
+  }
+
+  public void setMaxTsBlockLineNumber(int maxTsBlockLineNumber) {
+    this.maxTsBlockLineNumber = maxTsBlockLineNumber;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilder.java
index b02e897235..b2b47bf43b 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilder.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.tsfile.read.common.block;
 
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.block.column.BinaryColumnBuilder;
 import org.apache.iotdb.tsfile.read.common.block.column.BooleanColumnBuilder;
@@ -44,6 +45,9 @@ public class TsBlockBuilder {
   // This could be any other small number.
   private static final int DEFAULT_INITIAL_EXPECTED_ENTRIES = 8;
 
+  private static final int MAX_LINE_NUMBER =
+      TSFileDescriptor.getInstance().getConfig().getMaxTsBlockLineNumber();
+
   private TimeColumnBuilder timeColumnBuilder;
   private ColumnBuilder[] valueColumnBuilders;
   private List<TSDataType> types;
@@ -254,7 +258,7 @@ public class TsBlockBuilder {
   }
 
   public boolean isFull() {
-    return declaredPositions == Integer.MAX_VALUE || tsBlockBuilderStatus.isFull();
+    return declaredPositions == MAX_LINE_NUMBER || tsBlockBuilderStatus.isFull();
   }
 
   public boolean isEmpty() {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilderStatus.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilderStatus.java
index a48e9cd0ff..15d585800c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilderStatus.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/block/TsBlockBuilderStatus.java
@@ -18,11 +18,13 @@
  */
 package org.apache.iotdb.tsfile.read.common.block;
 
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.read.common.block.column.ColumnBuilderStatus;
 
 public class TsBlockBuilderStatus {
 
-  public static final int DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES = 1024 * 1024;
+  public static final int DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES =
+      TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes();
 
   private final int maxTsBlockSizeInBytes;