This is an automated email from the ASF dual-hosted git repository.
tanxinyu pushed a commit to branch rel/1.1
in repository https://gitbox.apache.org/repos/asf/iotdb.git
The following commit(s) were added to refs/heads/rel/1.1 by this push:
new a204f29a77 [To rel/1.1][IOTDB-5852] Add some IoTConsensus and
RatisConsensus properties (#9851)
a204f29a77 is described below
commit a204f29a77c4ea12b716cdb08035535e67512be5
Author: Xiangpeng Hu <[email protected]>
AuthorDate: Tue May 16 19:32:32 2023 +0800
[To rel/1.1][IOTDB-5852] Add some IoTConsensus and RatisConsensus
properties (#9851)
---
.../iotdb/confignode/conf/ConfigNodeConfig.java | 22 ++++++++
.../confignode/conf/ConfigNodeDescriptor.java | 16 ++++++
.../iotdb/confignode/manager/node/NodeManager.java | 3 ++
docs/UserGuide/Reference/Common-Config-Manual.md | 56 ++++++++++++++++++++
.../zh/UserGuide/Reference/Common-Config-Manual.md | 58 +++++++++++++++++++++
.../resources/conf/iotdb-common.properties | 22 ++++++++
.../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 60 ++++++++++++++++++++++
.../org/apache/iotdb/db/conf/IoTDBDescriptor.java | 35 +++++++++++++
.../db/consensus/DataRegionConsensusImpl.java | 9 ++++
.../src/main/thrift/confignode.thrift | 3 ++
10 files changed, 284 insertions(+)
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
index b4d0b77c7f..22f2e86a19 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
@@ -219,6 +219,8 @@ public class ConfigNodeConfig {
/** RatisConsensus protocol, allow flushing Raft Log asynchronously */
private boolean dataRegionRatisLogUnsafeFlushEnable = false;
+ private int dataRegionRatisLogForceSyncNum = 128;
+
private boolean configNodeRatisLogUnsafeFlushEnable = false;
private boolean schemaRegionRatisLogUnsafeFlushEnable = false;
@@ -232,6 +234,8 @@ public class ConfigNodeConfig {
/** RatisConsensus protocol, flow control window for ratis grpc log appender
*/
private long dataRegionRatisGrpcFlowControlWindow = 4 * 1024 * 1024L;
+ private int dataRegionRatisGrpcLeaderOutstandingAppendsMax = 128;
+
private long configNodeRatisGrpcFlowControlWindow = 4 * 1024 * 1024L;
private long schemaRegionRatisGrpcFlowControlWindow = 4 * 1024 * 1024L;
@@ -745,6 +749,14 @@ public class ConfigNodeConfig {
this.dataRegionRatisLogUnsafeFlushEnable =
dataRegionRatisLogUnsafeFlushEnable;
}
+ public int getDataRegionRatisLogForceSyncNum() {
+ return dataRegionRatisLogForceSyncNum;
+ }
+
+ public void setDataRegionRatisLogForceSyncNum(int
dataRegionRatisLogForceSyncNum) {
+ this.dataRegionRatisLogForceSyncNum = dataRegionRatisLogForceSyncNum;
+ }
+
public long getDataRegionRatisLogSegmentSizeMax() {
return dataRegionRatisLogSegmentSizeMax;
}
@@ -761,6 +773,16 @@ public class ConfigNodeConfig {
this.dataRegionRatisGrpcFlowControlWindow =
dataRegionRatisGrpcFlowControlWindow;
}
+ public int getDataRegionRatisGrpcLeaderOutstandingAppendsMax() {
+ return dataRegionRatisGrpcLeaderOutstandingAppendsMax;
+ }
+
+ public void setDataRegionRatisGrpcLeaderOutstandingAppendsMax(
+ int dataRegionRatisGrpcLeaderOutstandingAppendsMax) {
+ this.dataRegionRatisGrpcLeaderOutstandingAppendsMax =
+ dataRegionRatisGrpcLeaderOutstandingAppendsMax;
+ }
+
public long getDataRegionRatisRpcLeaderElectionTimeoutMinMs() {
return dataRegionRatisRpcLeaderElectionTimeoutMinMs;
}
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
index 5ff6618930..9f6f9d3abd 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
@@ -486,6 +486,14 @@ public class ConfigNodeDescriptor {
String.valueOf(conf.isDataRegionRatisLogUnsafeFlushEnable()))
.trim()));
+ conf.setDataRegionRatisLogForceSyncNum(
+ Integer.parseInt(
+ properties
+ .getProperty(
+ "data_region_ratis_log_force_sync_num",
+ String.valueOf(conf.getDataRegionRatisLogForceSyncNum()))
+ .trim()));
+
conf.setConfigNodeRatisLogUnsafeFlushEnable(
Boolean.parseBoolean(
properties
@@ -558,6 +566,14 @@ public class ConfigNodeDescriptor {
String.valueOf(conf.getSchemaRegionRatisGrpcFlowControlWindow()))
.trim()));
+ conf.setDataRegionRatisGrpcLeaderOutstandingAppendsMax(
+ Integer.parseInt(
+ properties
+ .getProperty(
+ "data_region_ratis_grpc_leader_outstanding_appends_max",
+
String.valueOf(conf.getDataRegionRatisGrpcLeaderOutstandingAppendsMax()))
+ .trim()));
+
conf.setDataRegionRatisRpcLeaderElectionTimeoutMinMs(
Long.parseLong(
properties
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
index 276b879d4d..1177282e3a 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
@@ -183,12 +183,15 @@ public class NodeManager {
ratisConfig.setDataLogUnsafeFlushEnable(conf.isDataRegionRatisLogUnsafeFlushEnable());
ratisConfig.setSchemaLogUnsafeFlushEnable(conf.isSchemaRegionRatisLogUnsafeFlushEnable());
+
ratisConfig.setDataRegionLogForceSyncNum(conf.getDataRegionRatisLogForceSyncNum());
ratisConfig.setDataLogSegmentSizeMax(conf.getDataRegionRatisLogSegmentSizeMax());
ratisConfig.setSchemaLogSegmentSizeMax(conf.getSchemaRegionRatisLogSegmentSizeMax());
ratisConfig.setDataGrpcFlowControlWindow(conf.getDataRegionRatisGrpcFlowControlWindow());
ratisConfig.setSchemaGrpcFlowControlWindow(conf.getSchemaRegionRatisGrpcFlowControlWindow());
+ ratisConfig.setDataRegionGrpcLeaderOutstandingAppendsMax(
+ conf.getDataRegionRatisGrpcLeaderOutstandingAppendsMax());
ratisConfig.setDataLeaderElectionTimeoutMin(
conf.getDataRegionRatisRpcLeaderElectionTimeoutMinMs());
diff --git a/docs/UserGuide/Reference/Common-Config-Manual.md
b/docs/UserGuide/Reference/Common-Config-Manual.md
index 33b977bd90..82c4a503e8 100644
--- a/docs/UserGuide/Reference/Common-Config-Manual.md
+++ b/docs/UserGuide/Reference/Common-Config-Manual.md
@@ -1487,6 +1487,44 @@ Different configuration parameters take effect in the
following three ways:
| Default | 5 |
| Effective | hot-load |
+### IOTConsensus Configuration
+
+* data_region_iot_max_log_entries_num_per_batch
+
+| Name | data_region_iot_max_log_entries_num_per_batch
|
+| :---------: | :------------------------------------------------ |
+| Description | The maximum log entries num in IoTConsensus Batch |
+| Type | int32 |
+| Default | 1024 |
+| Effective | After restarting system |
+
+* data_region_iot_max_size_per_batch
+
+| Name | data_region_iot_max_size_per_batch |
+| :---------: | :------------------------------------- |
+| Description | The maximum size in IoTConsensus Batch |
+| Type | int32 |
+| Default | 16MB |
+| Effective | After restarting system |
+
+* data_region_iot_max_pending_batches_num
+
+| Name | data_region_iot_max_pending_batches_num
|
+| :---------: | :---------------------------------------------- |
+| Description | The maximum pending batches num in IoTConsensus |
+| Type | int32 |
+| Default | 12 |
+| Effective | After restarting system |
+
+* data_region_iot_max_memory_ratio_for_queue
+
+| Name | data_region_iot_max_memory_ratio_for_queue
|
+| :---------: | :------------------------------------------------- |
+| Description | The maximum memory ratio for queue in IoTConsensus |
+| Type | double |
+| Default | 0.6 |
+| Effective | After restarting system |
+
### RatisConsensus Configuration
* config\_node\_ratis\_log\_appender\_buffer\_size\_max
@@ -1625,6 +1663,24 @@ Different configuration parameters take effect in the
following three ways:
| Default | 4MB |
| Effective | After restarting system |
+* data_region_ratis_grpc_leader_outstanding_appends_max
+
+| Name | data_region_ratis_grpc_leader_outstanding_appends_max |
+| :---------: | :---------------------------------------------------- |
+| Description | data region grpc pipeline concurrency threshold |
+| Type | int32 |
+| Default | 128 |
+| Effective | After restarting system |
+
+* data_region_ratis_log_force_sync_num
+
+| Name | data_region_ratis_log_force_sync_num |
+| :---------: | :----------------------------------- |
+| Description | data region fsync threshold |
+| Type | int32 |
+| Default | 128 |
+| Effective | After restarting system |
+
* config\_node\_ratis\_rpc\_leader\_election\_timeout\_min\_ms
| Name | config\_node\_ratis\_rpc\_leader\_election\_timeout\_min\_ms |
diff --git a/docs/zh/UserGuide/Reference/Common-Config-Manual.md
b/docs/zh/UserGuide/Reference/Common-Config-Manual.md
index 04cb4fced4..ba8b1c80b8 100644
--- a/docs/zh/UserGuide/Reference/Common-Config-Manual.md
+++ b/docs/zh/UserGuide/Reference/Common-Config-Manual.md
@@ -1524,6 +1524,46 @@ IoTDB ConfigNode 和 DataNode 的公共配置参数位于 `conf` 目录下。
| 默认值 | 5 |
| 改后生效方式 | 热加载 |
+#### IoT 共识协议配置
+
+当Region配置了IoTConsensus共识协议之后,下述的配置项才会生效
+
+* data_region_iot_max_log_entries_num_per_batch
+
+| 名字 | data_region_iot_max_log_entries_num_per_batch |
+| :----------: | :-------------------------------- |
+| 描述 | IoTConsensus batch 的最大日志条数 |
+| 类型 | int32 |
+| 默认值 | 1024 |
+| 改后生效方式 | 重启生效 |
+
+* data_region_iot_max_size_per_batch
+
+| 名字 | data_region_iot_max_size_per_batch |
+| :----------: | :---------------------------- |
+| 描述 | IoTConsensus batch 的最大大小 |
+| 类型 | int32 |
+| 默认值 | 16MB |
+| 改后生效方式 | 重启生效 |
+
+* data_region_iot_max_pending_batches_num
+
+| 名字 | data_region_iot_max_pending_batches_num |
+| :----------: | :---------------------------------- |
+| 描述 | IoTConsensus batch 的流水线并发阈值 |
+| 类型 | int32 |
+| 默认值 | 12 |
+| 改后生效方式 | 重启生效 |
+
+* data_region_iot_max_memory_ratio_for_queue
+
+| 名字 | data_region_iot_max_memory_ratio_for_queue |
+| :----------: | :---------------------------- |
+| 描述 | IoTConsensus 队列内存分配比例 |
+| 类型 | double |
+| 默认值 | 0.6 |
+| 改后生效方式 | 重启生效 |
+
#### Ratis 共识协议配置
当Region配置了RatisConsensus共识协议之后,下述的配置项才会生效
@@ -1663,6 +1703,24 @@ IoTDB ConfigNode 和 DataNode 的公共配置参数位于 `conf` 目录下。
| 默认值 | 4MB |
| 改后生效方式 | 重启生效 |
+* data_region_ratis_grpc_leader_outstanding_appends_max
+
+| 名字 | data_region_ratis_grpc_leader_outstanding_appends_max |
+| :----------: | :---------------------------------------------------- |
+| 描述 | data region grpc 流水线并发阈值 |
+| 类型 | int32 |
+| 默认值 | 128 |
+| 改后生效方式 | 重启生效 |
+
+* data_region_ratis_log_force_sync_num
+
+| 名字 | data_region_ratis_log_force_sync_num |
+| :----------: | :----------------------------------- |
+| 描述 | data region fsync 阈值 |
+| 类型 | int32 |
+| 默认值 | 128 |
+| 改后生效方式 | 重启生效 |
+
* config\_node\_ratis\_rpc\_leader\_election\_timeout\_min\_ms
| 名字 | config\_node\_ratis\_rpc\_leader\_election\_timeout\_min\_ms |
diff --git a/node-commons/src/assembly/resources/conf/iotdb-common.properties
b/node-commons/src/assembly/resources/conf/iotdb-common.properties
index 94746ce407..603016c6bf 100644
--- a/node-commons/src/assembly/resources/conf/iotdb-common.properties
+++ b/node-commons/src/assembly/resources/conf/iotdb-common.properties
@@ -737,6 +737,26 @@ cluster_name=defaultCluster
# Datatype: long
# iot_consensus_cache_window_time_in_ms=-1
+####################
+### IoTConsensus Configuration
+####################
+
+# The maximum log entries num in IoTConsensus Batch
+# Datatype: int
+# data_region_iot_max_log_entries_num_per_batch = 1024
+
+# The maximum size in IoTConsensus Batch
+# Datatype: int
+# data_region_iot_max_size_per_batch = 16777216
+
+# The maximum pending batches num in IoTConsensus
+# Datatype: int
+# data_region_iot_max_pending_batches_num = 12
+
+# The maximum memory ratio for queue in IoTConsensus
+# Datatype: double
+# data_region_iot_max_memory_ratio_for_queue = 0.6
+
####################
### TsFile Configurations
####################
@@ -966,6 +986,8 @@ cluster_name=defaultCluster
# config_node_ratis_grpc_flow_control_window=4194304
# schema_region_ratis_grpc_flow_control_window=4194304
# data_region_ratis_grpc_flow_control_window=4194304
+# data_region_ratis_grpc_leader_outstanding_appends_max=128
+# data_region_ratis_log_force_sync_num=128
# min election timeout for leader election
# config_node_ratis_rpc_leader_election_timeout_min_ms=2000
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 79a46cabd4..ee10b7eec5 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
@@ -1019,12 +1019,16 @@ public class IoTDBConfig {
private boolean dataRatisConsensusLogUnsafeFlushEnable = false;
private boolean schemaRatisConsensusLogUnsafeFlushEnable = false;
+ private int dataRatisConsensusLogForceSyncNum = 128;
+
private long dataRatisConsensusLogSegmentSizeMax = 24 * 1024 * 1024L;
private long schemaRatisConsensusLogSegmentSizeMax = 24 * 1024 * 1024L;
private long dataRatisConsensusGrpcFlowControlWindow = 4 * 1024 * 1024L;
private long schemaRatisConsensusGrpcFlowControlWindow = 4 * 1024 * 1024L;
+ private int dataRatisConsensusGrpcLeaderOutstandingAppendsMax = 128;
+
private long dataRatisConsensusLeaderElectionTimeoutMinMs = 2000L;
private long schemaRatisConsensusLeaderElectionTimeoutMinMs = 2000L;
@@ -1076,8 +1080,46 @@ public class IoTDBConfig {
// customizedProperties, this should be empty by default.
private Properties customizedProperties = new Properties();
+ // IoTConsensus Config
+ private int maxLogEntriesNumPerBatch = 1024;
+ private int maxSizePerBatch = 16 * 1024 * 1024;
+ private int maxPendingBatchesNum = 12;
+ private double maxMemoryRatioForQueue = 0.6;
+
IoTDBConfig() {}
+ public int getMaxLogEntriesNumPerBatch() {
+ return maxLogEntriesNumPerBatch;
+ }
+
+ public int getMaxSizePerBatch() {
+ return maxSizePerBatch;
+ }
+
+ public int getMaxPendingBatchesNum() {
+ return maxPendingBatchesNum;
+ }
+
+ public double getMaxMemoryRatioForQueue() {
+ return maxMemoryRatioForQueue;
+ }
+
+ public void setMaxLogEntriesNumPerBatch(int maxLogEntriesNumPerBatch) {
+ this.maxLogEntriesNumPerBatch = maxLogEntriesNumPerBatch;
+ }
+
+ public void setMaxSizePerBatch(int maxSizePerBatch) {
+ this.maxSizePerBatch = maxSizePerBatch;
+ }
+
+ public void setMaxPendingBatchesNum(int maxPendingBatchesNum) {
+ this.maxPendingBatchesNum = maxPendingBatchesNum;
+ }
+
+ public void setMaxMemoryRatioForQueue(double maxMemoryRatioForQueue) {
+ this.maxMemoryRatioForQueue = maxMemoryRatioForQueue;
+ }
+
public float getUdfMemoryBudgetInMB() {
return udfMemoryBudgetInMB;
}
@@ -3400,6 +3442,14 @@ public class IoTDBConfig {
this.dataRatisConsensusLogUnsafeFlushEnable =
dataRatisConsensusLogUnsafeFlushEnable;
}
+ public int getDataRatisConsensusLogForceSyncNum() {
+ return dataRatisConsensusLogForceSyncNum;
+ }
+
+ public void setDataRatisConsensusLogForceSyncNum(int
dataRatisConsensusLogForceSyncNum) {
+ this.dataRatisConsensusLogForceSyncNum = dataRatisConsensusLogForceSyncNum;
+ }
+
public long getDataRatisConsensusLogSegmentSizeMax() {
return dataRatisConsensusLogSegmentSizeMax;
}
@@ -3417,6 +3467,16 @@ public class IoTDBConfig {
this.dataRatisConsensusGrpcFlowControlWindow =
dataRatisConsensusGrpcFlowControlWindow;
}
+ public int getDataRatisConsensusGrpcLeaderOutstandingAppendsMax() {
+ return dataRatisConsensusGrpcLeaderOutstandingAppendsMax;
+ }
+
+ public void setDataRatisConsensusGrpcLeaderOutstandingAppendsMax(
+ int dataRatisConsensusGrpcLeaderOutstandingAppendsMax) {
+ this.dataRatisConsensusGrpcLeaderOutstandingAppendsMax =
+ dataRatisConsensusGrpcLeaderOutstandingAppendsMax;
+ }
+
public long getDataRatisConsensusLeaderElectionTimeoutMinMs() {
return dataRatisConsensusLeaderElectionTimeoutMinMs;
}
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 91a08093ce..5b9fbd0ac5 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
@@ -1040,6 +1040,38 @@ public class IoTDBDescriptor {
// author cache
loadAuthorCache(properties);
+
+ loadIoTConsensusProps(properties);
+ }
+
+ private void loadIoTConsensusProps(Properties properties) {
+ conf.setMaxLogEntriesNumPerBatch(
+ Integer.parseInt(
+ properties
+ .getProperty(
+ "data_region_iot_max_log_entries_num_per_batch",
+ String.valueOf(conf.getMaxLogEntriesNumPerBatch()))
+ .trim()));
+ conf.setMaxSizePerBatch(
+ Integer.parseInt(
+ properties
+ .getProperty(
+ "data_region_iot_max_size_per_batch",
String.valueOf(conf.getMaxSizePerBatch()))
+ .trim()));
+ conf.setMaxPendingBatchesNum(
+ Integer.parseInt(
+ properties
+ .getProperty(
+ "data_region_iot_max_pending_batches_num",
+ String.valueOf(conf.getMaxPendingBatchesNum()))
+ .trim()));
+ conf.setMaxMemoryRatioForQueue(
+ Double.parseDouble(
+ properties
+ .getProperty(
+ "data_region_iot_max_memory_ratio_for_queue",
+ String.valueOf(conf.getMaxMemoryRatioForQueue()))
+ .trim()));
}
private void loadAuthorCache(Properties properties) {
@@ -1968,12 +2000,15 @@ public class IoTDBDescriptor {
conf.setDataRatisConsensusLogUnsafeFlushEnable(ratisConfig.isDataLogUnsafeFlushEnable());
conf.setSchemaRatisConsensusLogUnsafeFlushEnable(ratisConfig.isSchemaLogUnsafeFlushEnable());
+
conf.setDataRatisConsensusLogForceSyncNum(ratisConfig.getDataRegionLogForceSyncNum());
conf.setDataRatisConsensusLogSegmentSizeMax(ratisConfig.getDataLogSegmentSizeMax());
conf.setSchemaRatisConsensusLogSegmentSizeMax(ratisConfig.getSchemaLogSegmentSizeMax());
conf.setDataRatisConsensusGrpcFlowControlWindow(ratisConfig.getDataGrpcFlowControlWindow());
conf.setSchemaRatisConsensusGrpcFlowControlWindow(ratisConfig.getSchemaGrpcFlowControlWindow());
+ conf.setDataRatisConsensusGrpcLeaderOutstandingAppendsMax(
+ ratisConfig.getDataRegionGrpcLeaderOutstandingAppendsMax());
conf.setDataRatisConsensusLeaderElectionTimeoutMinMs(
ratisConfig.getDataLeaderElectionTimeoutMin());
diff --git
a/server/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java
b/server/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java
index 905a8275c5..fd5976adc2 100644
---
a/server/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java
+++
b/server/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java
@@ -97,6 +97,11 @@ public class DataRegionConsensusImpl {
.setWalThrottleThreshold(conf.getThrottleThreshold())
.setAllocateMemoryForConsensus(
conf.getAllocateMemoryForConsensus())
+ .setMaxLogEntriesNumPerBatch(
+ conf.getMaxLogEntriesNumPerBatch())
+
.setMaxSizePerBatch(conf.getMaxSizePerBatch())
+
.setMaxPendingBatchesNum(conf.getMaxPendingBatchesNum())
+
.setMaxMemoryRatioForQueue(conf.getMaxMemoryRatioForQueue())
.build())
.build())
.setRatisConfig(
@@ -114,6 +119,7 @@ public class DataRegionConsensusImpl {
RatisConfig.Log.newBuilder()
.setUnsafeFlushEnabled(
conf.isDataRatisConsensusLogUnsafeFlushEnable())
+
.setForceSyncNum(conf.getDataRatisConsensusLogForceSyncNum())
.setSegmentSizeMax(
SizeInBytes.valueOf(
conf.getDataRatisConsensusLogSegmentSizeMax()))
@@ -125,6 +131,9 @@ public class DataRegionConsensusImpl {
.setFlowControlWindow(
SizeInBytes.valueOf(
conf.getDataRatisConsensusGrpcFlowControlWindow()))
+ .setLeaderOutstandingAppendsMax(
+ conf
+
.getDataRatisConsensusGrpcLeaderOutstandingAppendsMax())
.build())
.setRpc(
RatisConfig.Rpc.newBuilder()
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift
b/thrift-confignode/src/main/thrift/confignode.thrift
index ce33f24720..41f67a9c55 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -79,6 +79,9 @@ struct TRatisConfig {
27: required i64 schemaRegionRatisLogMax
28: required i64 dataRegionRatisLogMax
+
+ 29: required i32 dataRegionGrpcLeaderOutstandingAppendsMax
+ 30: required i32 dataRegionLogForceSyncNum
}
struct TCQConfig {