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

qiaojialin pushed a commit to branch revert-10267-iot_consensus
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 595c52c1933ad221e157228160f5810e5a407fb4
Author: Jialin Qiao <[email protected]>
AuthorDate: Sun Jun 25 11:04:56 2023 +0800

    Revert "[To rel/1.2][IOTDB-6022] Modify the default parameters of 
iotconsensus (#10267)"
    
    This reverts commit db4cfc5488a843699da44399fc5fabdaa20b4636.
---
 .../iotdb/consensus/config/IoTConsensusConfig.java     | 18 +-----------------
 .../consensus/iot/logdispatcher/LogDispatcher.java     |  5 -----
 .../assembly/resources/conf/iotdb-common.properties    |  2 +-
 .../apache/iotdb/commons/concurrent/ThreadName.java    |  7 -------
 4 files changed, 2 insertions(+), 30 deletions(-)

diff --git 
a/consensus/src/main/java/org/apache/iotdb/consensus/config/IoTConsensusConfig.java
 
b/consensus/src/main/java/org/apache/iotdb/consensus/config/IoTConsensusConfig.java
index 91b4fb262ac..a4300d68b5a 100644
--- 
a/consensus/src/main/java/org/apache/iotdb/consensus/config/IoTConsensusConfig.java
+++ 
b/consensus/src/main/java/org/apache/iotdb/consensus/config/IoTConsensusConfig.java
@@ -256,7 +256,6 @@ public class IoTConsensusConfig {
 
     private final int maxQueueLength;
     private final long maxWaitingTimeForWaitBatchInMs;
-    private final int maxWaitingTimeForAccumulatingBatchInMs;
     private final long basicRetryWaitTimeMs;
     private final long maxRetryWaitTimeMs;
     private final long walThrottleThreshold;
@@ -271,7 +270,6 @@ public class IoTConsensusConfig {
         int maxPendingBatchesNum,
         int maxQueueLength,
         long maxWaitingTimeForWaitBatchInMs,
-        int maxWaitingTimeForAccumulatingBatchInMs,
         long basicRetryWaitTimeMs,
         long maxRetryWaitTimeMs,
         long walThrottleThreshold,
@@ -284,7 +282,6 @@ public class IoTConsensusConfig {
       this.maxPendingBatchesNum = maxPendingBatchesNum;
       this.maxQueueLength = maxQueueLength;
       this.maxWaitingTimeForWaitBatchInMs = maxWaitingTimeForWaitBatchInMs;
-      this.maxWaitingTimeForAccumulatingBatchInMs = 
maxWaitingTimeForAccumulatingBatchInMs;
       this.basicRetryWaitTimeMs = basicRetryWaitTimeMs;
       this.maxRetryWaitTimeMs = maxRetryWaitTimeMs;
       this.walThrottleThreshold = walThrottleThreshold;
@@ -314,10 +311,6 @@ public class IoTConsensusConfig {
       return maxWaitingTimeForWaitBatchInMs;
     }
 
-    public int getMaxWaitingTimeForAccumulatingBatchInMs() {
-      return maxWaitingTimeForAccumulatingBatchInMs;
-    }
-
     public long getBasicRetryWaitTimeMs() {
       return basicRetryWaitTimeMs;
     }
@@ -354,11 +347,9 @@ public class IoTConsensusConfig {
 
       private int maxLogEntriesNumPerBatch = 1024;
       private int maxSizePerBatch = 16 * 1024 * 1024;
-      private int maxPendingBatchesNum = 5;
+      private int maxPendingBatchesNum = 12;
       private int maxQueueLength = 4096;
       private long maxWaitingTimeForWaitBatchInMs = 10 * 1000L;
-
-      private int maxWaitingTimeForAccumulatingBatchInMs = 500;
       private long basicRetryWaitTimeMs = TimeUnit.MILLISECONDS.toMillis(100);
       private long maxRetryWaitTimeMs = TimeUnit.SECONDS.toMillis(20);
       private long walThrottleThreshold = 50 * 1024 * 1024 * 1024L;
@@ -393,12 +384,6 @@ public class IoTConsensusConfig {
         return this;
       }
 
-      public Builder setMaxWaitingTimeForAccumulatingBatchInMs(
-          int maxWaitingTimeForAccumulatingBatchInMs) {
-        this.maxWaitingTimeForAccumulatingBatchInMs = 
maxWaitingTimeForAccumulatingBatchInMs;
-        return this;
-      }
-
       public Replication.Builder setBasicRetryWaitTimeMs(long 
basicRetryWaitTimeMs) {
         this.basicRetryWaitTimeMs = basicRetryWaitTimeMs;
         return this;
@@ -441,7 +426,6 @@ public class IoTConsensusConfig {
             maxPendingBatchesNum,
             maxQueueLength,
             maxWaitingTimeForWaitBatchInMs,
-            maxWaitingTimeForAccumulatingBatchInMs,
             basicRetryWaitTimeMs,
             maxRetryWaitTimeMs,
             walThrottleThreshold,
diff --git 
a/consensus/src/main/java/org/apache/iotdb/consensus/iot/logdispatcher/LogDispatcher.java
 
b/consensus/src/main/java/org/apache/iotdb/consensus/iot/logdispatcher/LogDispatcher.java
index be7a5d83a1f..dbf9336043c 100644
--- 
a/consensus/src/main/java/org/apache/iotdb/consensus/iot/logdispatcher/LogDispatcher.java
+++ 
b/consensus/src/main/java/org/apache/iotdb/consensus/iot/logdispatcher/LogDispatcher.java
@@ -316,11 +316,6 @@ public class LogDispatcher {
                 pendingEntries.poll(PENDING_REQUEST_TAKING_TIME_OUT_IN_SEC, 
TimeUnit.SECONDS);
             if (request != null) {
               bufferedEntries.add(request);
-              // If write pressure is low, we simply sleep a little to reduce 
the number of RPC
-              if (pendingEntries.size() <= 
config.getReplication().getMaxLogEntriesNumPerBatch()
-                  && bufferedEntries.isEmpty()) {
-                
Thread.sleep(config.getReplication().getMaxWaitingTimeForAccumulatingBatchInMs());
-              }
             }
           }
           logDispatcherThreadMetrics.recordConstructBatchTime(
diff --git a/node-commons/src/assembly/resources/conf/iotdb-common.properties 
b/node-commons/src/assembly/resources/conf/iotdb-common.properties
index 0bd89a4a617..96cf45f7731 100644
--- a/node-commons/src/assembly/resources/conf/iotdb-common.properties
+++ b/node-commons/src/assembly/resources/conf/iotdb-common.properties
@@ -761,7 +761,7 @@ cluster_name=defaultCluster
 
 # The maximum pending batches num in IoTConsensus
 # Datatype: int
-# data_region_iot_max_pending_batches_num = 5
+# data_region_iot_max_pending_batches_num = 12
 
 # The maximum memory ratio for queue in IoTConsensus
 # Datatype: double
diff --git 
a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
 
b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
index e28887791b4..51783d4c281 100644
--- 
a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
+++ 
b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
@@ -18,9 +18,6 @@
  */
 package org.apache.iotdb.commons.concurrent;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
@@ -135,7 +132,6 @@ public enum ThreadName {
   SYSTEM_SCHEDULE_METRICS("SystemScheduleMetrics"),
   RESOURCE_CONTROL_DISK_STATISTIC("ResourceControl-DataRegionDiskStatistics"),
   PROMETHEUS_REACTOR_HTTP_NIO("reactor-http-nio"),
-  PROMETHEUS_REACTOR_HTTP_EPOLL("reactor-http-epoll"),
   PROMETHEUS_BOUNDED_ELASTIC("boundedElastic-evictor"),
   // -------------------------- Other --------------------------
   TTL_CHECK("TTL-CHECK"),
@@ -150,7 +146,6 @@ public enum ThreadName {
   UNKOWN("UNKNOWN");
 
   private final String name;
-  private static final Logger log = LoggerFactory.getLogger(ThreadName.class);
   private static Set<ThreadName> queryThreadNames =
       new HashSet<>(
           Arrays.asList(
@@ -262,7 +257,6 @@ public enum ThreadName {
               SYSTEM_SCHEDULE_METRICS,
               RESOURCE_CONTROL_DISK_STATISTIC,
               PROMETHEUS_REACTOR_HTTP_NIO,
-              PROMETHEUS_REACTOR_HTTP_EPOLL,
               PROMETHEUS_BOUNDED_ELASTIC));
   private static Set<ThreadName> otherThreadNames =
       new HashSet<>(
@@ -357,7 +351,6 @@ public enum ThreadName {
         }
       }
     }
-    log.debug("Unknown thread name: {}", givenThreadName);
     return ThreadName.UNKOWN;
   }
 }

Reply via email to