[IOTDB-4545] Seperate ratis configs for data/schema/partition region (#7461)
diff --git a/confignode/src/assembly/resources/conf/iotdb-confignode.properties b/confignode/src/assembly/resources/conf/iotdb-confignode.properties
index 43f6f79..4206734 100644
--- a/confignode/src/assembly/resources/conf/iotdb-confignode.properties
+++ b/confignode/src/assembly/resources/conf/iotdb-confignode.properties
@@ -342,25 +342,39 @@
####################
# max payload size for a single log-sync-RPC from leader to follower
-# ratis_log_appender_buffer_size_max = 4194304
+# partition_region_ratis_log_appender_buffer_size_max = 4194304
+# schema_region_ratis_log_appender_buffer_size_max = 4194304
+# data_region_ratis_log_appender_buffer_size_max = 4194304
# trigger a snapshot when ratis_snapshot_trigger_threshold logs are written
-# ratis_snapshot_trigger_threshold = 400000
+# partition_region_ratis_snapshot_trigger_threshold = 400000
+# schema_region_ratis_snapshot_trigger_threshold = 400000
+# data_region_ratis_snapshot_trigger_threshold = 400000
# allow flushing Raft Log asynchronously
-# ratis_log_unsafe_flush_enable = false
+# partition_region_ratis_log_unsafe_flush_enable = false
+# schema_region_ratis_log_unsafe_flush_enable = false
+# data_region_ratis_log_unsafe_flush_enable = false
# max capacity of a single Raft Log segment (by default 24MB)
-# ratis_log_segment_size_max = 25165824
+# partition_region_ratis_log_segment_size_max = 25165824
+# schema_region_ratis_log_segment_size_max = 25165824
+# data_region_ratis_log_segment_size_max = 25165824
# flow control window for ratis grpc log appender
-# ratis_grpc_flow_control_window = 4194304
+# partition_region_ratis_grpc_flow_control_window = 4194304
+# schema_region_ratis_grpc_flow_control_window = 4194304
+# data_region_ratis_grpc_flow_control_window = 4194304
# min election timeout for leader election
-# ratis_rpc_leader_election_timeout_min_ms = 2000
+# partition_region_ratis_rpc_leader_election_timeout_min_ms = 2000
+# schema_region_ratis_rpc_leader_election_timeout_min_ms = 2000
+# data_region_ratis_rpc_leader_election_timeout_min_ms = 2000
# max election timeout for leader election
-# ratis_rpc_leader_election_timeout_max_ms = 4000
+# partition_region_ratis_rpc_leader_election_timeout_max_ms = 4000
+# schema_region_ratis_rpc_leader_election_timeout_max_ms = 4000
+# data_region_ratis_rpc_leader_election_timeout_max_ms = 4000
####################
### Disk Monitor
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 ad56aad..af1b577 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
@@ -132,28 +132,49 @@
private String readConsistencyLevel = "strong";
/** RatisConsensus protocol, Max size for a single log append request from leader */
- private long ratisConsensusLogAppenderBufferSize = 4 * 1024 * 1024L;
+ private long dataRegionRatisConsensusLogAppenderBufferSize = 4 * 1024 * 1024L;
+
+ private long partitionRegionRatisConsensusLogAppenderBufferSize = 4 * 1024 * 1024L;
+ private long schemaRegionRatisConsensusLogAppenderBufferSize = 4 * 1024 * 1024L;
/**
* RatisConsensus protocol, trigger a snapshot when ratis_snapshot_trigger_threshold logs are
* written
*/
- private long ratisSnapshotTriggerThreshold = 400000L;
+ private long dataRegionRatisSnapshotTriggerThreshold = 400000L;
+
+ private long partitionRegionRatisSnapshotTriggerThreshold = 400000L;
+ private long schemaRegionRatisSnapshotTriggerThreshold = 400000L;
/** RatisConsensus protocol, allow flushing Raft Log asynchronously */
- private boolean ratisLogUnsafeFlushEnable = false;
+ private boolean dataRegionRatisLogUnsafeFlushEnable = false;
+
+ private boolean partitionRegionRatisLogUnsafeFlushEnable = false;
+ private boolean schemaRegionRatisLogUnsafeFlushEnable = false;
/** RatisConsensus protocol, max capacity of a single Raft Log segment */
- private long ratisLogSegmentSizeMax = 24 * 1024 * 1024L;
+ private long dataRegionRatisLogSegmentSizeMax = 24 * 1024 * 1024L;
+
+ private long partitionRegionRatisLogSegmentSizeMax = 24 * 1024 * 1024L;
+ private long schemaRegionRatisLogSegmentSizeMax = 24 * 1024 * 1024L;
/** RatisConsensus protocol, flow control window for ratis grpc log appender */
- private long ratisGrpcFlowControlWindow = 4 * 1024 * 1024L;
+ private long dataRegionRatisGrpcFlowControlWindow = 4 * 1024 * 1024L;
+
+ private long partitionRegionRatisGrpcFlowControlWindow = 4 * 1024 * 1024L;
+ private long schemaRegionRatisGrpcFlowControlWindow = 4 * 1024 * 1024L;
/** RatisConsensus protocol, min election timeout for leader election */
- private long ratisRpcLeaderElectionTimeoutMinMs = 2000L;
+ private long dataRegionRatisRpcLeaderElectionTimeoutMinMs = 2000L;
+
+ private long partitionRegionRatisRpcLeaderElectionTimeoutMinMs = 2000L;
+ private long schemaRegionRatisRpcLeaderElectionTimeoutMinMs = 2000L;
/** RatisConsensus protocol, max election timeout for leader election */
- private long ratisRpcLeaderElectionTimeoutMaxMs = 4000L;
+ private long dataRegionRatisRpcLeaderElectionTimeoutMaxMs = 4000L;
+
+ private long partitionRegionRatisRpcLeaderElectionTimeoutMaxMs = 4000L;
+ private long schemaRegionRatisRpcLeaderElectionTimeoutMaxMs = 4000L;
public ConfigNodeConfig() {
// empty constructor
@@ -436,59 +457,197 @@
this.readConsistencyLevel = readConsistencyLevel;
}
- public long getRatisConsensusLogAppenderBufferSize() {
- return ratisConsensusLogAppenderBufferSize;
+ public long getDataRegionRatisConsensusLogAppenderBufferSize() {
+ return dataRegionRatisConsensusLogAppenderBufferSize;
}
- public void setRatisConsensusLogAppenderBufferSize(long ratisConsensusLogAppenderBufferSize) {
- this.ratisConsensusLogAppenderBufferSize = ratisConsensusLogAppenderBufferSize;
+ public void setDataRegionRatisConsensusLogAppenderBufferSize(
+ long dataRegionRatisConsensusLogAppenderBufferSize) {
+ this.dataRegionRatisConsensusLogAppenderBufferSize =
+ dataRegionRatisConsensusLogAppenderBufferSize;
}
- public long getRatisSnapshotTriggerThreshold() {
- return ratisSnapshotTriggerThreshold;
+ public long getDataRegionRatisSnapshotTriggerThreshold() {
+ return dataRegionRatisSnapshotTriggerThreshold;
}
- public void setRatisSnapshotTriggerThreshold(long ratisSnapshotTriggerThreshold) {
- this.ratisSnapshotTriggerThreshold = ratisSnapshotTriggerThreshold;
+ public void setDataRegionRatisSnapshotTriggerThreshold(
+ long dataRegionRatisSnapshotTriggerThreshold) {
+ this.dataRegionRatisSnapshotTriggerThreshold = dataRegionRatisSnapshotTriggerThreshold;
}
- public boolean isRatisLogUnsafeFlushEnable() {
- return ratisLogUnsafeFlushEnable;
+ public boolean isDataRegionRatisLogUnsafeFlushEnable() {
+ return dataRegionRatisLogUnsafeFlushEnable;
}
- public void setRatisLogUnsafeFlushEnable(boolean ratisLogUnsafeFlushEnable) {
- this.ratisLogUnsafeFlushEnable = ratisLogUnsafeFlushEnable;
+ public void setDataRegionRatisLogUnsafeFlushEnable(boolean dataRegionRatisLogUnsafeFlushEnable) {
+ this.dataRegionRatisLogUnsafeFlushEnable = dataRegionRatisLogUnsafeFlushEnable;
}
- public long getRatisLogSegmentSizeMax() {
- return ratisLogSegmentSizeMax;
+ public long getDataRegionRatisLogSegmentSizeMax() {
+ return dataRegionRatisLogSegmentSizeMax;
}
- public void setRatisLogSegmentSizeMax(long ratisLogSegmentSizeMax) {
- this.ratisLogSegmentSizeMax = ratisLogSegmentSizeMax;
+ public void setDataRegionRatisLogSegmentSizeMax(long dataRegionRatisLogSegmentSizeMax) {
+ this.dataRegionRatisLogSegmentSizeMax = dataRegionRatisLogSegmentSizeMax;
}
- public long getRatisGrpcFlowControlWindow() {
- return ratisGrpcFlowControlWindow;
+ public long getDataRegionRatisGrpcFlowControlWindow() {
+ return dataRegionRatisGrpcFlowControlWindow;
}
- public void setRatisGrpcFlowControlWindow(long ratisGrpcFlowControlWindow) {
- this.ratisGrpcFlowControlWindow = ratisGrpcFlowControlWindow;
+ public void setDataRegionRatisGrpcFlowControlWindow(long dataRegionRatisGrpcFlowControlWindow) {
+ this.dataRegionRatisGrpcFlowControlWindow = dataRegionRatisGrpcFlowControlWindow;
}
- public long getRatisRpcLeaderElectionTimeoutMinMs() {
- return ratisRpcLeaderElectionTimeoutMinMs;
+ public long getDataRegionRatisRpcLeaderElectionTimeoutMinMs() {
+ return dataRegionRatisRpcLeaderElectionTimeoutMinMs;
}
- public void setRatisRpcLeaderElectionTimeoutMinMs(long ratisRpcLeaderElectionTimeoutMinMs) {
- this.ratisRpcLeaderElectionTimeoutMinMs = ratisRpcLeaderElectionTimeoutMinMs;
+ public void setDataRegionRatisRpcLeaderElectionTimeoutMinMs(
+ long dataRegionRatisRpcLeaderElectionTimeoutMinMs) {
+ this.dataRegionRatisRpcLeaderElectionTimeoutMinMs =
+ dataRegionRatisRpcLeaderElectionTimeoutMinMs;
}
- public long getRatisRpcLeaderElectionTimeoutMaxMs() {
- return ratisRpcLeaderElectionTimeoutMaxMs;
+ public long getDataRegionRatisRpcLeaderElectionTimeoutMaxMs() {
+ return dataRegionRatisRpcLeaderElectionTimeoutMaxMs;
}
- public void setRatisRpcLeaderElectionTimeoutMaxMs(long ratisRpcLeaderElectionTimeoutMaxMs) {
- this.ratisRpcLeaderElectionTimeoutMaxMs = ratisRpcLeaderElectionTimeoutMaxMs;
+ public void setDataRegionRatisRpcLeaderElectionTimeoutMaxMs(
+ long dataRegionRatisRpcLeaderElectionTimeoutMaxMs) {
+ this.dataRegionRatisRpcLeaderElectionTimeoutMaxMs =
+ dataRegionRatisRpcLeaderElectionTimeoutMaxMs;
+ }
+
+ public long getPartitionRegionRatisConsensusLogAppenderBufferSize() {
+ return partitionRegionRatisConsensusLogAppenderBufferSize;
+ }
+
+ public void setPartitionRegionRatisConsensusLogAppenderBufferSize(
+ long partitionRegionRatisConsensusLogAppenderBufferSize) {
+ this.partitionRegionRatisConsensusLogAppenderBufferSize =
+ partitionRegionRatisConsensusLogAppenderBufferSize;
+ }
+
+ public long getPartitionRegionRatisSnapshotTriggerThreshold() {
+ return partitionRegionRatisSnapshotTriggerThreshold;
+ }
+
+ public void setPartitionRegionRatisSnapshotTriggerThreshold(
+ long partitionRegionRatisSnapshotTriggerThreshold) {
+ this.partitionRegionRatisSnapshotTriggerThreshold =
+ partitionRegionRatisSnapshotTriggerThreshold;
+ }
+
+ public boolean isPartitionRegionRatisLogUnsafeFlushEnable() {
+ return partitionRegionRatisLogUnsafeFlushEnable;
+ }
+
+ public void setPartitionRegionRatisLogUnsafeFlushEnable(
+ boolean partitionRegionRatisLogUnsafeFlushEnable) {
+ this.partitionRegionRatisLogUnsafeFlushEnable = partitionRegionRatisLogUnsafeFlushEnable;
+ }
+
+ public long getPartitionRegionRatisLogSegmentSizeMax() {
+ return partitionRegionRatisLogSegmentSizeMax;
+ }
+
+ public void setPartitionRegionRatisLogSegmentSizeMax(long partitionRegionRatisLogSegmentSizeMax) {
+ this.partitionRegionRatisLogSegmentSizeMax = partitionRegionRatisLogSegmentSizeMax;
+ }
+
+ public long getPartitionRegionRatisGrpcFlowControlWindow() {
+ return partitionRegionRatisGrpcFlowControlWindow;
+ }
+
+ public void setPartitionRegionRatisGrpcFlowControlWindow(
+ long partitionRegionRatisGrpcFlowControlWindow) {
+ this.partitionRegionRatisGrpcFlowControlWindow = partitionRegionRatisGrpcFlowControlWindow;
+ }
+
+ public long getPartitionRegionRatisRpcLeaderElectionTimeoutMinMs() {
+ return partitionRegionRatisRpcLeaderElectionTimeoutMinMs;
+ }
+
+ public void setPartitionRegionRatisRpcLeaderElectionTimeoutMinMs(
+ long partitionRegionRatisRpcLeaderElectionTimeoutMinMs) {
+ this.partitionRegionRatisRpcLeaderElectionTimeoutMinMs =
+ partitionRegionRatisRpcLeaderElectionTimeoutMinMs;
+ }
+
+ public long getPartitionRegionRatisRpcLeaderElectionTimeoutMaxMs() {
+ return partitionRegionRatisRpcLeaderElectionTimeoutMaxMs;
+ }
+
+ public void setPartitionRegionRatisRpcLeaderElectionTimeoutMaxMs(
+ long partitionRegionRatisRpcLeaderElectionTimeoutMaxMs) {
+ this.partitionRegionRatisRpcLeaderElectionTimeoutMaxMs =
+ partitionRegionRatisRpcLeaderElectionTimeoutMaxMs;
+ }
+
+ public long getSchemaRegionRatisConsensusLogAppenderBufferSize() {
+ return schemaRegionRatisConsensusLogAppenderBufferSize;
+ }
+
+ public void setSchemaRegionRatisConsensusLogAppenderBufferSize(
+ long schemaRegionRatisConsensusLogAppenderBufferSize) {
+ this.schemaRegionRatisConsensusLogAppenderBufferSize =
+ schemaRegionRatisConsensusLogAppenderBufferSize;
+ }
+
+ public long getSchemaRegionRatisSnapshotTriggerThreshold() {
+ return schemaRegionRatisSnapshotTriggerThreshold;
+ }
+
+ public void setSchemaRegionRatisSnapshotTriggerThreshold(
+ long schemaRegionRatisSnapshotTriggerThreshold) {
+ this.schemaRegionRatisSnapshotTriggerThreshold = schemaRegionRatisSnapshotTriggerThreshold;
+ }
+
+ public boolean isSchemaRegionRatisLogUnsafeFlushEnable() {
+ return schemaRegionRatisLogUnsafeFlushEnable;
+ }
+
+ public void setSchemaRegionRatisLogUnsafeFlushEnable(
+ boolean schemaRegionRatisLogUnsafeFlushEnable) {
+ this.schemaRegionRatisLogUnsafeFlushEnable = schemaRegionRatisLogUnsafeFlushEnable;
+ }
+
+ public long getSchemaRegionRatisLogSegmentSizeMax() {
+ return schemaRegionRatisLogSegmentSizeMax;
+ }
+
+ public void setSchemaRegionRatisLogSegmentSizeMax(long schemaRegionRatisLogSegmentSizeMax) {
+ this.schemaRegionRatisLogSegmentSizeMax = schemaRegionRatisLogSegmentSizeMax;
+ }
+
+ public long getSchemaRegionRatisGrpcFlowControlWindow() {
+ return schemaRegionRatisGrpcFlowControlWindow;
+ }
+
+ public void setSchemaRegionRatisGrpcFlowControlWindow(
+ long schemaRegionRatisGrpcFlowControlWindow) {
+ this.schemaRegionRatisGrpcFlowControlWindow = schemaRegionRatisGrpcFlowControlWindow;
+ }
+
+ public long getSchemaRegionRatisRpcLeaderElectionTimeoutMinMs() {
+ return schemaRegionRatisRpcLeaderElectionTimeoutMinMs;
+ }
+
+ public void setSchemaRegionRatisRpcLeaderElectionTimeoutMinMs(
+ long schemaRegionRatisRpcLeaderElectionTimeoutMinMs) {
+ this.schemaRegionRatisRpcLeaderElectionTimeoutMinMs =
+ schemaRegionRatisRpcLeaderElectionTimeoutMinMs;
+ }
+
+ public long getSchemaRegionRatisRpcLeaderElectionTimeoutMaxMs() {
+ return schemaRegionRatisRpcLeaderElectionTimeoutMaxMs;
+ }
+
+ public void setSchemaRegionRatisRpcLeaderElectionTimeoutMaxMs(
+ long schemaRegionRatisRpcLeaderElectionTimeoutMaxMs) {
+ this.schemaRegionRatisRpcLeaderElectionTimeoutMaxMs =
+ schemaRegionRatisRpcLeaderElectionTimeoutMaxMs;
}
}
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 c4e81b1..2167564 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
@@ -282,46 +282,131 @@
}
private void loadRatisConsensusConfig(Properties properties) {
- conf.setRatisConsensusLogAppenderBufferSize(
+ conf.setDataRegionRatisConsensusLogAppenderBufferSize(
Long.parseLong(
properties.getProperty(
- "ratis_log_appender_buffer_size_max",
- String.valueOf(conf.getRatisConsensusLogAppenderBufferSize()))));
+ "data_region_ratis_log_appender_buffer_size_max",
+ String.valueOf(conf.getDataRegionRatisConsensusLogAppenderBufferSize()))));
- conf.setRatisSnapshotTriggerThreshold(
+ conf.setPartitionRegionRatisConsensusLogAppenderBufferSize(
Long.parseLong(
properties.getProperty(
- "ratis_snapshot_trigger_threshold",
- String.valueOf(conf.getRatisConsensusLogAppenderBufferSize()))));
+ "partition_region_ratis_log_appender_buffer_size_max",
+ String.valueOf(conf.getPartitionRegionRatisConsensusLogAppenderBufferSize()))));
- conf.setRatisLogUnsafeFlushEnable(
+ conf.setSchemaRegionRatisConsensusLogAppenderBufferSize(
+ Long.parseLong(
+ properties.getProperty(
+ "schema_region_ratis_log_appender_buffer_size_max",
+ String.valueOf(conf.getSchemaRegionRatisConsensusLogAppenderBufferSize()))));
+
+ conf.setDataRegionRatisSnapshotTriggerThreshold(
+ Long.parseLong(
+ properties.getProperty(
+ "data_region_ratis_snapshot_trigger_threshold",
+ String.valueOf(conf.getDataRegionRatisConsensusLogAppenderBufferSize()))));
+
+ conf.setPartitionRegionRatisSnapshotTriggerThreshold(
+ Long.parseLong(
+ properties.getProperty(
+ "partition_region_ratis_snapshot_trigger_threshold",
+ String.valueOf(conf.getPartitionRegionRatisSnapshotTriggerThreshold()))));
+
+ conf.setSchemaRegionRatisSnapshotTriggerThreshold(
+ Long.parseLong(
+ properties.getProperty(
+ "schema_region_ratis_snapshot_trigger_threshold",
+ String.valueOf(conf.getSchemaRegionRatisSnapshotTriggerThreshold()))));
+
+ conf.setDataRegionRatisLogUnsafeFlushEnable(
Boolean.parseBoolean(
properties.getProperty(
- "ratis_log_unsafe_flush_enable",
- String.valueOf(conf.isRatisLogUnsafeFlushEnable()))));
+ "data_region_ratis_log_unsafe_flush_enable",
+ String.valueOf(conf.isDataRegionRatisLogUnsafeFlushEnable()))));
- conf.setRatisLogSegmentSizeMax(
+ conf.setPartitionRegionRatisLogUnsafeFlushEnable(
+ Boolean.parseBoolean(
+ properties.getProperty(
+ "partition_region_ratis_log_unsafe_flush_enable",
+ String.valueOf(conf.isPartitionRegionRatisLogUnsafeFlushEnable()))));
+
+ conf.setSchemaRegionRatisLogUnsafeFlushEnable(
+ Boolean.parseBoolean(
+ properties.getProperty(
+ "schema_region_ratis_log_unsafe_flush_enable",
+ String.valueOf(conf.isSchemaRegionRatisLogUnsafeFlushEnable()))));
+
+ conf.setDataRegionRatisLogSegmentSizeMax(
Long.parseLong(
properties.getProperty(
- "ratis_log_segment_size_max", String.valueOf(conf.getRatisLogSegmentSizeMax()))));
+ "data_region_ratis_log_segment_size_max",
+ String.valueOf(conf.getDataRegionRatisLogSegmentSizeMax()))));
- conf.setRatisGrpcFlowControlWindow(
+ conf.setPartitionRegionRatisLogSegmentSizeMax(
Long.parseLong(
properties.getProperty(
- "ratis_grpc_flow_control_window",
- String.valueOf(conf.getRatisGrpcFlowControlWindow()))));
+ "partition_region_ratis_log_segment_size_max",
+ String.valueOf(conf.getPartitionRegionRatisLogSegmentSizeMax()))));
- conf.setRatisRpcLeaderElectionTimeoutMinMs(
+ conf.setSchemaRegionRatisLogSegmentSizeMax(
Long.parseLong(
properties.getProperty(
- "ratis_rpc_leader_election_timeout_min_ms",
- String.valueOf(conf.getRatisRpcLeaderElectionTimeoutMinMs()))));
+ "schema_region_ratis_log_segment_size_max",
+ String.valueOf(conf.getSchemaRegionRatisLogSegmentSizeMax()))));
- conf.setRatisRpcLeaderElectionTimeoutMinMs(
+ conf.setDataRegionRatisGrpcFlowControlWindow(
Long.parseLong(
properties.getProperty(
- "ratis_rpc_leader_election_timeout_max_ms",
- String.valueOf(conf.getRatisRpcLeaderElectionTimeoutMaxMs()))));
+ "data_region_ratis_grpc_flow_control_window",
+ String.valueOf(conf.getDataRegionRatisGrpcFlowControlWindow()))));
+
+ conf.setPartitionRegionRatisGrpcFlowControlWindow(
+ Long.parseLong(
+ properties.getProperty(
+ "partition_region_ratis_grpc_flow_control_window",
+ String.valueOf(conf.getPartitionRegionRatisGrpcFlowControlWindow()))));
+
+ conf.setSchemaRegionRatisGrpcFlowControlWindow(
+ Long.parseLong(
+ properties.getProperty(
+ "schema_region_ratis_grpc_flow_control_window",
+ String.valueOf(conf.getSchemaRegionRatisGrpcFlowControlWindow()))));
+
+ conf.setDataRegionRatisRpcLeaderElectionTimeoutMinMs(
+ Long.parseLong(
+ properties.getProperty(
+ "data_region_ratis_rpc_leader_election_timeout_min_ms",
+ String.valueOf(conf.getDataRegionRatisRpcLeaderElectionTimeoutMinMs()))));
+
+ conf.setPartitionRegionRatisRpcLeaderElectionTimeoutMinMs(
+ Long.parseLong(
+ properties.getProperty(
+ "partition_region_ratis_rpc_leader_election_timeout_min_ms",
+ String.valueOf(conf.getPartitionRegionRatisRpcLeaderElectionTimeoutMinMs()))));
+
+ conf.setSchemaRegionRatisRpcLeaderElectionTimeoutMinMs(
+ Long.parseLong(
+ properties.getProperty(
+ "schema_region_ratis_rpc_leader_election_timeout_min_ms",
+ String.valueOf(conf.getSchemaRegionRatisRpcLeaderElectionTimeoutMinMs()))));
+
+ conf.setDataRegionRatisRpcLeaderElectionTimeoutMinMs(
+ Long.parseLong(
+ properties.getProperty(
+ "data_region_ratis_rpc_leader_election_timeout_max_ms",
+ String.valueOf(conf.getDataRegionRatisRpcLeaderElectionTimeoutMaxMs()))));
+
+ conf.setPartitionRegionRatisRpcLeaderElectionTimeoutMaxMs(
+ Long.parseLong(
+ properties.getProperty(
+ "partition_region_ratis_rpc_leader_election_timeout_max_ms",
+ String.valueOf(conf.getPartitionRegionRatisRpcLeaderElectionTimeoutMaxMs()))));
+
+ conf.setSchemaRegionRatisRpcLeaderElectionTimeoutMaxMs(
+ Long.parseLong(
+ properties.getProperty(
+ "schema_region_ratis_rpc_leader_election_timeout_max_ms",
+ String.valueOf(conf.getSchemaRegionRatisRpcLeaderElectionTimeoutMaxMs()))));
}
/**
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
index 6f7eb10..0b35146 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
@@ -38,8 +38,11 @@
import org.apache.iotdb.consensus.common.response.ConsensusReadResponse;
import org.apache.iotdb.consensus.common.response.ConsensusWriteResponse;
import org.apache.iotdb.consensus.config.ConsensusConfig;
+import org.apache.iotdb.consensus.config.RatisConfig;
import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.ratis.util.SizeInBytes;
+import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -82,6 +85,47 @@
CONF.getConfigNodeConsensusProtocolClass(),
ConsensusConfig.newBuilder()
.setThisNode(new TEndPoint(CONF.getInternalAddress(), CONF.getConsensusPort()))
+ .setRatisConfig(
+ RatisConfig.newBuilder()
+ .setLeaderLogAppender(
+ RatisConfig.LeaderLogAppender.newBuilder()
+ .setBufferByteLimit(
+ CONF
+ .getPartitionRegionRatisConsensusLogAppenderBufferSize())
+ .build())
+ .setSnapshot(
+ RatisConfig.Snapshot.newBuilder()
+ .setAutoTriggerThreshold(
+ CONF.getPartitionRegionRatisSnapshotTriggerThreshold())
+ .build())
+ .setLog(
+ RatisConfig.Log.newBuilder()
+ .setUnsafeFlushEnabled(
+ CONF.isPartitionRegionRatisLogUnsafeFlushEnable())
+ .setSegmentCacheSizeMax(
+ SizeInBytes.valueOf(
+ CONF.getPartitionRegionRatisLogSegmentSizeMax()))
+ .build())
+ .setGrpc(
+ RatisConfig.Grpc.newBuilder()
+ .setFlowControlWindow(
+ SizeInBytes.valueOf(
+ CONF.getPartitionRegionRatisGrpcFlowControlWindow()))
+ .build())
+ .setRpc(
+ RatisConfig.Rpc.newBuilder()
+ .setTimeoutMin(
+ TimeDuration.valueOf(
+ CONF
+ .getPartitionRegionRatisRpcLeaderElectionTimeoutMinMs(),
+ TimeUnit.MILLISECONDS))
+ .setTimeoutMax(
+ TimeDuration.valueOf(
+ CONF
+ .getPartitionRegionRatisRpcLeaderElectionTimeoutMaxMs(),
+ TimeUnit.MILLISECONDS))
+ .build())
+ .build())
.setStorageDir(CONF.getConsensusDir())
.build(),
gid -> stateMachine)
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 2f2f658..f974a1d 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
@@ -142,13 +142,34 @@
private void setRatisConfig(DataNodeRegisterResp dataSet) {
final ConfigNodeConfig conf = ConfigNodeDescriptor.getInstance().getConf();
TRatisConfig ratisConfig = new TRatisConfig();
- ratisConfig.setAppenderBufferSize(conf.getRatisConsensusLogAppenderBufferSize());
- ratisConfig.setSnapshotTriggerThreshold(conf.getRatisSnapshotTriggerThreshold());
- ratisConfig.setLogUnsafeFlushEnable(conf.isRatisLogUnsafeFlushEnable());
- ratisConfig.setLogSegmentSizeMax(conf.getRatisLogSegmentSizeMax());
- ratisConfig.setGrpcFlowControlWindow(conf.getRatisGrpcFlowControlWindow());
- ratisConfig.setLeaderElectionTimeoutMin(conf.getRatisRpcLeaderElectionTimeoutMinMs());
- ratisConfig.setLeaderElectionTimeoutMax(conf.getRatisRpcLeaderElectionTimeoutMaxMs());
+
+ ratisConfig.setDataAppenderBufferSize(conf.getDataRegionRatisConsensusLogAppenderBufferSize());
+ ratisConfig.setSchemaAppenderBufferSize(
+ conf.getSchemaRegionRatisConsensusLogAppenderBufferSize());
+
+ ratisConfig.setDataSnapshotTriggerThreshold(conf.getDataRegionRatisSnapshotTriggerThreshold());
+ ratisConfig.setSchemaSnapshotTriggerThreshold(
+ conf.getSchemaRegionRatisSnapshotTriggerThreshold());
+
+ ratisConfig.setDataLogUnsafeFlushEnable(conf.isDataRegionRatisLogUnsafeFlushEnable());
+ ratisConfig.setSchemaLogUnsafeFlushEnable(conf.isSchemaRegionRatisLogUnsafeFlushEnable());
+
+ ratisConfig.setDataLogSegmentSizeMax(conf.getDataRegionRatisLogSegmentSizeMax());
+ ratisConfig.setSchemaLogSegmentSizeMax(conf.getSchemaRegionRatisLogSegmentSizeMax());
+
+ ratisConfig.setDataGrpcFlowControlWindow(conf.getDataRegionRatisGrpcFlowControlWindow());
+ ratisConfig.setSchemaGrpcFlowControlWindow(conf.getSchemaRegionRatisGrpcFlowControlWindow());
+
+ ratisConfig.setDataLeaderElectionTimeoutMin(
+ conf.getDataRegionRatisRpcLeaderElectionTimeoutMinMs());
+ ratisConfig.setSchemaLeaderElectionTimeoutMin(
+ conf.getSchemaRegionRatisRpcLeaderElectionTimeoutMinMs());
+
+ ratisConfig.setDataLeaderElectionTimeoutMax(
+ conf.getDataRegionRatisRpcLeaderElectionTimeoutMaxMs());
+ ratisConfig.setSchemaLeaderElectionTimeoutMax(
+ conf.getSchemaRegionRatisRpcLeaderElectionTimeoutMaxMs());
+
dataSet.setRatisConfig(ratisConfig);
}
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 8f7787c..035a4dc 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,19 +1019,26 @@
/** Maximum wait time of write cache in MultiLeader consensus. Unit: ms */
private long cacheWindowTimeInMs = 60 * 1000;
- private long ratisConsensusLogAppenderBufferSizeMax = 4 * 1024 * 1024L;
+ private long dataRatisConsensusLogAppenderBufferSizeMax = 4 * 1024 * 1024L;
+ private long schemaRatisConsensusLogAppenderBufferSizeMax = 4 * 1024 * 1024L;
- private long ratisConsensusSnapshotTriggerThreshold = 400000L;
+ private long dataRatisConsensusSnapshotTriggerThreshold = 400000L;
+ private long schemaRatisConsensusSnapshotTriggerThreshold = 400000L;
- private boolean ratisConsensusLogUnsafeFlushEnable = false;
+ private boolean dataRatisConsensusLogUnsafeFlushEnable = false;
+ private boolean schemaRatisConsensusLogUnsafeFlushEnable = false;
- private long ratisConsensusLogSegmentSizeMax = 24 * 1024 * 1024L;
+ private long dataRatisConsensusLogSegmentSizeMax = 24 * 1024 * 1024L;
+ private long schemaRatisConsensusLogSegmentSizeMax = 24 * 1024 * 1024L;
- private long ratisConsensusGrpcFlowControlWindow = 4 * 1024 * 1024L;
+ private long dataRatisConsensusGrpcFlowControlWindow = 4 * 1024 * 1024L;
+ private long schemaRatisConsensusGrpcFlowControlWindow = 4 * 1024 * 1024L;
- private long ratisConsensusLeaderElectionTimeoutMinMs = 2000L;
+ private long dataRatisConsensusLeaderElectionTimeoutMinMs = 2000L;
+ private long schemaRatisConsensusLeaderElectionTimeoutMinMs = 2000L;
- private long RatisConsensusLeaderElectionTimeoutMaxMs = 4000L;
+ private long dataRatisConsensusLeaderElectionTimeoutMaxMs = 4000L;
+ private long schemaRatisConsensusLeaderElectionTimeoutMaxMs = 4000L;
IoTDBConfig() {}
@@ -3273,13 +3280,13 @@
this.cacheWindowTimeInMs = cacheWindowTimeInMs;
}
- public long getRatisConsensusLogAppenderBufferSizeMax() {
- return ratisConsensusLogAppenderBufferSizeMax;
+ public long getDataRatisConsensusLogAppenderBufferSizeMax() {
+ return dataRatisConsensusLogAppenderBufferSizeMax;
}
- public void setRatisConsensusLogAppenderBufferSizeMax(
- long ratisConsensusLogAppenderBufferSizeMax) {
- this.ratisConsensusLogAppenderBufferSizeMax = ratisConsensusLogAppenderBufferSizeMax;
+ public void setDataRatisConsensusLogAppenderBufferSizeMax(
+ long dataRatisConsensusLogAppenderBufferSizeMax) {
+ this.dataRatisConsensusLogAppenderBufferSizeMax = dataRatisConsensusLogAppenderBufferSizeMax;
}
public String getConfigMessage() {
@@ -3315,54 +3322,124 @@
return configMessage;
}
- public long getRatisConsensusSnapshotTriggerThreshold() {
- return ratisConsensusSnapshotTriggerThreshold;
+ public long getDataRatisConsensusSnapshotTriggerThreshold() {
+ return dataRatisConsensusSnapshotTriggerThreshold;
}
- public void setRatisConsensusSnapshotTriggerThreshold(
- long ratisConsensusSnapshotTriggerThreshold) {
- this.ratisConsensusSnapshotTriggerThreshold = ratisConsensusSnapshotTriggerThreshold;
+ public void setDataRatisConsensusSnapshotTriggerThreshold(
+ long dataRatisConsensusSnapshotTriggerThreshold) {
+ this.dataRatisConsensusSnapshotTriggerThreshold = dataRatisConsensusSnapshotTriggerThreshold;
}
- public boolean isRatisConsensusLogUnsafeFlushEnable() {
- return ratisConsensusLogUnsafeFlushEnable;
+ public boolean isDataRatisConsensusLogUnsafeFlushEnable() {
+ return dataRatisConsensusLogUnsafeFlushEnable;
}
- public void setRatisConsensusLogUnsafeFlushEnable(boolean ratisConsensusLogUnsafeFlushEnable) {
- this.ratisConsensusLogUnsafeFlushEnable = ratisConsensusLogUnsafeFlushEnable;
+ public void setDataRatisConsensusLogUnsafeFlushEnable(
+ boolean dataRatisConsensusLogUnsafeFlushEnable) {
+ this.dataRatisConsensusLogUnsafeFlushEnable = dataRatisConsensusLogUnsafeFlushEnable;
}
- public long getRatisConsensusLogSegmentSizeMax() {
- return ratisConsensusLogSegmentSizeMax;
+ public long getDataRatisConsensusLogSegmentSizeMax() {
+ return dataRatisConsensusLogSegmentSizeMax;
}
- public void setRatisConsensusLogSegmentSizeMax(long ratisConsensusLogSegmentSizeMax) {
- this.ratisConsensusLogSegmentSizeMax = ratisConsensusLogSegmentSizeMax;
+ public void setDataRatisConsensusLogSegmentSizeMax(long dataRatisConsensusLogSegmentSizeMax) {
+ this.dataRatisConsensusLogSegmentSizeMax = dataRatisConsensusLogSegmentSizeMax;
}
- public long getRatisConsensusGrpcFlowControlWindow() {
- return ratisConsensusGrpcFlowControlWindow;
+ public long getDataRatisConsensusGrpcFlowControlWindow() {
+ return dataRatisConsensusGrpcFlowControlWindow;
}
- public void setRatisConsensusGrpcFlowControlWindow(long ratisConsensusGrpcFlowControlWindow) {
- this.ratisConsensusGrpcFlowControlWindow = ratisConsensusGrpcFlowControlWindow;
+ public void setDataRatisConsensusGrpcFlowControlWindow(
+ long dataRatisConsensusGrpcFlowControlWindow) {
+ this.dataRatisConsensusGrpcFlowControlWindow = dataRatisConsensusGrpcFlowControlWindow;
}
- public long getRatisConsensusLeaderElectionTimeoutMinMs() {
- return ratisConsensusLeaderElectionTimeoutMinMs;
+ public long getDataRatisConsensusLeaderElectionTimeoutMinMs() {
+ return dataRatisConsensusLeaderElectionTimeoutMinMs;
}
- public void setRatisConsensusLeaderElectionTimeoutMinMs(
- long ratisConsensusLeaderElectionTimeoutMinMs) {
- this.ratisConsensusLeaderElectionTimeoutMinMs = ratisConsensusLeaderElectionTimeoutMinMs;
+ public void setDataRatisConsensusLeaderElectionTimeoutMinMs(
+ long dataRatisConsensusLeaderElectionTimeoutMinMs) {
+ this.dataRatisConsensusLeaderElectionTimeoutMinMs =
+ dataRatisConsensusLeaderElectionTimeoutMinMs;
}
- public long getRatisConsensusLeaderElectionTimeoutMaxMs() {
- return RatisConsensusLeaderElectionTimeoutMaxMs;
+ public long getDataRatisConsensusLeaderElectionTimeoutMaxMs() {
+ return dataRatisConsensusLeaderElectionTimeoutMaxMs;
}
- public void setRatisConsensusLeaderElectionTimeoutMaxMs(
- long ratisConsensusLeaderElectionTimeoutMaxMs) {
- RatisConsensusLeaderElectionTimeoutMaxMs = ratisConsensusLeaderElectionTimeoutMaxMs;
+ public void setDataRatisConsensusLeaderElectionTimeoutMaxMs(
+ long dataRatisConsensusLeaderElectionTimeoutMaxMs) {
+ this.dataRatisConsensusLeaderElectionTimeoutMaxMs =
+ dataRatisConsensusLeaderElectionTimeoutMaxMs;
+ }
+
+ public long getSchemaRatisConsensusLogAppenderBufferSizeMax() {
+ return schemaRatisConsensusLogAppenderBufferSizeMax;
+ }
+
+ public void setSchemaRatisConsensusLogAppenderBufferSizeMax(
+ long schemaRatisConsensusLogAppenderBufferSizeMax) {
+ this.schemaRatisConsensusLogAppenderBufferSizeMax =
+ schemaRatisConsensusLogAppenderBufferSizeMax;
+ }
+
+ public long getSchemaRatisConsensusSnapshotTriggerThreshold() {
+ return schemaRatisConsensusSnapshotTriggerThreshold;
+ }
+
+ public void setSchemaRatisConsensusSnapshotTriggerThreshold(
+ long schemaRatisConsensusSnapshotTriggerThreshold) {
+ this.schemaRatisConsensusSnapshotTriggerThreshold =
+ schemaRatisConsensusSnapshotTriggerThreshold;
+ }
+
+ public boolean isSchemaRatisConsensusLogUnsafeFlushEnable() {
+ return schemaRatisConsensusLogUnsafeFlushEnable;
+ }
+
+ public void setSchemaRatisConsensusLogUnsafeFlushEnable(
+ boolean schemaRatisConsensusLogUnsafeFlushEnable) {
+ this.schemaRatisConsensusLogUnsafeFlushEnable = schemaRatisConsensusLogUnsafeFlushEnable;
+ }
+
+ public long getSchemaRatisConsensusLogSegmentSizeMax() {
+ return schemaRatisConsensusLogSegmentSizeMax;
+ }
+
+ public void setSchemaRatisConsensusLogSegmentSizeMax(long schemaRatisConsensusLogSegmentSizeMax) {
+ this.schemaRatisConsensusLogSegmentSizeMax = schemaRatisConsensusLogSegmentSizeMax;
+ }
+
+ public long getSchemaRatisConsensusGrpcFlowControlWindow() {
+ return schemaRatisConsensusGrpcFlowControlWindow;
+ }
+
+ public void setSchemaRatisConsensusGrpcFlowControlWindow(
+ long schemaRatisConsensusGrpcFlowControlWindow) {
+ this.schemaRatisConsensusGrpcFlowControlWindow = schemaRatisConsensusGrpcFlowControlWindow;
+ }
+
+ public long getSchemaRatisConsensusLeaderElectionTimeoutMinMs() {
+ return schemaRatisConsensusLeaderElectionTimeoutMinMs;
+ }
+
+ public void setSchemaRatisConsensusLeaderElectionTimeoutMinMs(
+ long schemaRatisConsensusLeaderElectionTimeoutMinMs) {
+ this.schemaRatisConsensusLeaderElectionTimeoutMinMs =
+ schemaRatisConsensusLeaderElectionTimeoutMinMs;
+ }
+
+ public long getSchemaRatisConsensusLeaderElectionTimeoutMaxMs() {
+ return schemaRatisConsensusLeaderElectionTimeoutMaxMs;
+ }
+
+ public void setSchemaRatisConsensusLeaderElectionTimeoutMaxMs(
+ long schemaRatisConsensusLeaderElectionTimeoutMaxMs) {
+ this.schemaRatisConsensusLeaderElectionTimeoutMaxMs =
+ schemaRatisConsensusLeaderElectionTimeoutMaxMs;
}
}
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 57ed301..3ee6ee8 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
@@ -1926,13 +1926,32 @@
}
public void loadRatisConfig(TRatisConfig ratisConfig) {
- conf.setRatisConsensusLogAppenderBufferSizeMax(ratisConfig.getAppenderBufferSize());
- conf.setRatisConsensusSnapshotTriggerThreshold(ratisConfig.getSnapshotTriggerThreshold());
- conf.setRatisConsensusLogUnsafeFlushEnable(ratisConfig.isLogUnsafeFlushEnable());
- conf.setRatisConsensusLogSegmentSizeMax(ratisConfig.getLogSegmentSizeMax());
- conf.setRatisConsensusGrpcFlowControlWindow(ratisConfig.getGrpcFlowControlWindow());
- conf.setRatisConsensusLeaderElectionTimeoutMinMs(ratisConfig.getLeaderElectionTimeoutMin());
- conf.setRatisConsensusLeaderElectionTimeoutMaxMs(ratisConfig.getLeaderElectionTimeoutMax());
+ conf.setDataRatisConsensusLogAppenderBufferSizeMax(ratisConfig.getDataAppenderBufferSize());
+ conf.setSchemaRatisConsensusLogAppenderBufferSizeMax(ratisConfig.getSchemaAppenderBufferSize());
+
+ conf.setDataRatisConsensusSnapshotTriggerThreshold(
+ ratisConfig.getDataSnapshotTriggerThreshold());
+ conf.setSchemaRatisConsensusSnapshotTriggerThreshold(
+ ratisConfig.getSchemaSnapshotTriggerThreshold());
+
+ conf.setDataRatisConsensusLogUnsafeFlushEnable(ratisConfig.isDataLogUnsafeFlushEnable());
+ conf.setSchemaRatisConsensusLogUnsafeFlushEnable(ratisConfig.isSchemaLogUnsafeFlushEnable());
+
+ conf.setDataRatisConsensusLogSegmentSizeMax(ratisConfig.getDataLogSegmentSizeMax());
+ conf.setSchemaRatisConsensusLogSegmentSizeMax(ratisConfig.getSchemaLogSegmentSizeMax());
+
+ conf.setDataRatisConsensusGrpcFlowControlWindow(ratisConfig.getDataGrpcFlowControlWindow());
+ conf.setSchemaRatisConsensusGrpcFlowControlWindow(ratisConfig.getSchemaGrpcFlowControlWindow());
+
+ conf.setDataRatisConsensusLeaderElectionTimeoutMinMs(
+ ratisConfig.getDataLeaderElectionTimeoutMin());
+ conf.setSchemaRatisConsensusLeaderElectionTimeoutMinMs(
+ ratisConfig.getSchemaLeaderElectionTimeoutMin());
+
+ conf.setDataRatisConsensusLeaderElectionTimeoutMaxMs(
+ ratisConfig.getDataLeaderElectionTimeoutMax());
+ conf.setSchemaRatisConsensusLeaderElectionTimeoutMaxMs(
+ ratisConfig.getSchemaLeaderElectionTimeoutMax());
}
public void initClusterSchemaMemoryAllocate() {
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 0e493c2..df0e98b 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
@@ -96,37 +96,39 @@
Snapshot.newBuilder()
.setCreationGap(1)
.setAutoTriggerThreshold(
- conf.getRatisConsensusSnapshotTriggerThreshold())
+ conf.getDataRatisConsensusSnapshotTriggerThreshold())
.build())
.setLog(
RatisConfig.Log.newBuilder()
.setUnsafeFlushEnabled(
- conf.isRatisConsensusLogUnsafeFlushEnable())
- .setSegmentCacheSizeMax(
+ conf.isDataRatisConsensusLogUnsafeFlushEnable())
+ .setSegmentSizeMax(
SizeInBytes.valueOf(
- conf.getRatisConsensusLogSegmentSizeMax()))
+ conf.getDataRatisConsensusLogSegmentSizeMax()))
.build())
.setGrpc(
RatisConfig.Grpc.newBuilder()
.setFlowControlWindow(
SizeInBytes.valueOf(
- conf.getRatisConsensusGrpcFlowControlWindow()))
+ conf.getDataRatisConsensusGrpcFlowControlWindow()))
.build())
.setRpc(
RatisConfig.Rpc.newBuilder()
.setTimeoutMin(
TimeDuration.valueOf(
- conf.getRatisConsensusLeaderElectionTimeoutMinMs(),
+ conf
+ .getDataRatisConsensusLeaderElectionTimeoutMinMs(),
TimeUnit.MILLISECONDS))
.setTimeoutMax(
TimeDuration.valueOf(
- conf.getRatisConsensusLeaderElectionTimeoutMaxMs(),
+ conf
+ .getDataRatisConsensusLeaderElectionTimeoutMaxMs(),
TimeUnit.MILLISECONDS))
.build())
.setLeaderLogAppender(
RatisConfig.LeaderLogAppender.newBuilder()
.setBufferByteLimit(
- conf.getRatisConsensusLogAppenderBufferSizeMax())
+ conf.getDataRatisConsensusLogAppenderBufferSizeMax())
.build())
.build())
.build(),
diff --git a/server/src/main/java/org/apache/iotdb/db/consensus/SchemaRegionConsensusImpl.java b/server/src/main/java/org/apache/iotdb/db/consensus/SchemaRegionConsensusImpl.java
index af95eb6f..4ae4b63 100644
--- a/server/src/main/java/org/apache/iotdb/db/consensus/SchemaRegionConsensusImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/consensus/SchemaRegionConsensusImpl.java
@@ -24,11 +24,17 @@
import org.apache.iotdb.consensus.ConsensusFactory;
import org.apache.iotdb.consensus.IConsensus;
import org.apache.iotdb.consensus.config.ConsensusConfig;
+import org.apache.iotdb.consensus.config.RatisConfig;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.consensus.statemachine.SchemaRegionStateMachine;
import org.apache.iotdb.db.metadata.schemaregion.SchemaEngine;
+import org.apache.ratis.util.SizeInBytes;
+import org.apache.ratis.util.TimeDuration;
+
+import java.util.concurrent.TimeUnit;
+
/**
* We can use SchemaRegionConsensusImpl.getInstance() to obtain a consensus layer reference for
* schemaRegion's reading and writing
@@ -55,6 +61,46 @@
.setThisNode(
new TEndPoint(
conf.getInternalAddress(), conf.getSchemaRegionConsensusPort()))
+ .setRatisConfig(
+ RatisConfig.newBuilder()
+ .setSnapshot(
+ RatisConfig.Snapshot.newBuilder()
+ .setAutoTriggerThreshold(
+ conf.getSchemaRatisConsensusSnapshotTriggerThreshold())
+ .build())
+ .setLog(
+ RatisConfig.Log.newBuilder()
+ .setUnsafeFlushEnabled(
+ conf.isSchemaRatisConsensusLogUnsafeFlushEnable())
+ .setSegmentSizeMax(
+ SizeInBytes.valueOf(
+ conf.getSchemaRatisConsensusLogSegmentSizeMax()))
+ .build())
+ .setGrpc(
+ RatisConfig.Grpc.newBuilder()
+ .setFlowControlWindow(
+ SizeInBytes.valueOf(
+ conf.getSchemaRatisConsensusGrpcFlowControlWindow()))
+ .build())
+ .setRpc(
+ RatisConfig.Rpc.newBuilder()
+ .setTimeoutMin(
+ TimeDuration.valueOf(
+ conf
+ .getSchemaRatisConsensusLeaderElectionTimeoutMinMs(),
+ TimeUnit.MILLISECONDS))
+ .setTimeoutMax(
+ TimeDuration.valueOf(
+ conf
+ .getSchemaRatisConsensusLeaderElectionTimeoutMaxMs(),
+ TimeUnit.MILLISECONDS))
+ .build())
+ .setLeaderLogAppender(
+ RatisConfig.LeaderLogAppender.newBuilder()
+ .setBufferByteLimit(
+ conf.getSchemaRatisConsensusLogAppenderBufferSizeMax())
+ .build())
+ .build())
.setStorageDir(conf.getSchemaRegionConsensusDir())
.build(),
gid ->
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index b956c66..9d95e3d 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -49,13 +49,26 @@
}
struct TRatisConfig {
- 1: optional i64 appenderBufferSize
- 2: optional i64 snapshotTriggerThreshold
- 3: optional bool logUnsafeFlushEnable
- 4: optional i64 logSegmentSizeMax
- 5: optional i64 grpcFlowControlWindow
- 6: optional i64 leaderElectionTimeoutMin
- 7: optional i64 leaderElectionTimeoutMax
+ 1: optional i64 schemaAppenderBufferSize
+ 2: optional i64 dataAppenderBufferSize
+
+ 3: optional i64 schemaSnapshotTriggerThreshold
+ 4: optional i64 dataSnapshotTriggerThreshold
+
+ 5: optional bool schemaLogUnsafeFlushEnable
+ 6: optional bool dataLogUnsafeFlushEnable
+
+ 7: optional i64 schemaLogSegmentSizeMax
+ 8: optional i64 dataLogSegmentSizeMax
+
+ 9: optional i64 schemaGrpcFlowControlWindow
+ 10: optional i64 dataGrpcFlowControlWindow
+
+ 11: optional i64 schemaLeaderElectionTimeoutMin
+ 12: optional i64 dataLeaderElectionTimeoutMin
+
+ 13: optional i64 schemaLeaderElectionTimeoutMax
+ 14: optional i64 dataLeaderElectionTimeoutMax
}
struct TDataNodeRemoveReq {