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

cmccabe pushed a commit to branch 2.8
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.8 by this push:
     new a6a68bc  MINOR: tune KIP-631 configurations (#10179)
a6a68bc is described below

commit a6a68bc22176cc6193cbcbf25637ea6782e40a73
Author: Colin Patrick McCabe <[email protected]>
AuthorDate: Thu Feb 25 17:16:37 2021 -0800

    MINOR: tune KIP-631 configurations (#10179)
    
    Since we expect KIP-631 controller fail-overs to be fairly cheap, tune
    the default raft configuration parameters so that we detect node
    failures more quickly.
    
    Reduce the broker session timeout as well so that broker failures are
    detected more quickly.
    
    Reviewers: Jason Gustafson <[email protected]>, Alok Nikhil 
<[email protected]>
---
 core/src/main/scala/kafka/server/KafkaConfig.scala        |  2 +-
 .../main/java/org/apache/kafka/raft/KafkaRaftClient.java  |  2 +-
 raft/src/main/java/org/apache/kafka/raft/RaftConfig.java  | 15 ++++++++++-----
 3 files changed, 12 insertions(+), 7 deletions(-)

diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala 
b/core/src/main/scala/kafka/server/KafkaConfig.scala
index e01bf60..d2e3414 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -72,7 +72,7 @@ object Defaults {
   val QueuedMaxRequestBytes = -1
   val InitialBrokerRegistrationTimeoutMs = 60000
   val BrokerHeartbeatIntervalMs = 2000
-  val BrokerSessionTimeoutMs = 18000
+  val BrokerSessionTimeoutMs = 9000
 
   /** KIP-500 Configuration */
   val EmptyNodeId: Int = -1
diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java 
b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
index 3aa9f90..672b967 100644
--- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
+++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
@@ -133,7 +133,7 @@ import static 
org.apache.kafka.raft.RaftUtil.hasValidTopicPartition;
  */
 public class KafkaRaftClient<T> implements RaftClient<T> {
     private static final int RETRY_BACKOFF_BASE_MS = 100;
-    private static final int FETCH_MAX_WAIT_MS = 1000;
+    private static final int FETCH_MAX_WAIT_MS = 500;
     static final int MAX_BATCH_SIZE = 8 * 1024 * 1024;
 
     private final AtomicReference<GracefulShutdown> shutdown = new 
AtomicReference<>();
diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java 
b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java
index 13dd879..358d5f3 100644
--- a/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java
+++ b/raft/src/main/java/org/apache/kafka/raft/RaftConfig.java
@@ -40,6 +40,11 @@ import java.util.stream.Collectors;
  * For example: `[email protected]:0,[email protected]:0,[email protected]:0`
  * This will assign an {@link UnknownAddressSpec} to the voter entries
  *
+ * The default raft timeouts are relatively low compared to some other 
timeouts such as
+ * request.timeout.ms. This is part of a general design philosophy where we 
see changing
+ * the leader of a Raft cluster as a relatively quick operation. For example, 
the KIP-631
+ * controller should be able to transition from standby to active without 
reloading all of
+ * the metadata. The standby is a "hot" standby, not a "cold" one.
  */
 public class RaftConfig {
 
@@ -58,18 +63,18 @@ public class RaftConfig {
     public static final String QUORUM_ELECTION_TIMEOUT_MS_CONFIG = 
QUORUM_PREFIX + "election.timeout.ms";
     public static final String QUORUM_ELECTION_TIMEOUT_MS_DOC = "Maximum time 
in milliseconds to wait " +
         "without being able to fetch from the leader before triggering a new 
election";
-    public static final int DEFAULT_QUORUM_ELECTION_TIMEOUT_MS = 5_000;
+    public static final int DEFAULT_QUORUM_ELECTION_TIMEOUT_MS = 1_000;
 
     public static final String QUORUM_FETCH_TIMEOUT_MS_CONFIG = QUORUM_PREFIX 
+ "fetch.timeout.ms";
     public static final String QUORUM_FETCH_TIMEOUT_MS_DOC = "Maximum time 
without a successful fetch from " +
         "the current leader before becoming a candidate and triggering a 
election for voters; Maximum time without " +
         "receiving fetch from a majority of the quorum before asking around to 
see if there's a new epoch for leader";
-    public static final int DEFAULT_QUORUM_FETCH_TIMEOUT_MS = 15_000;
+    public static final int DEFAULT_QUORUM_FETCH_TIMEOUT_MS = 2_000;
 
     public static final String QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG = 
QUORUM_PREFIX + "election.backoff.max.ms";
     public static final String QUORUM_ELECTION_BACKOFF_MAX_MS_DOC = "Maximum 
time in milliseconds before starting new elections. " +
         "This is used in the binary exponential backoff mechanism that helps 
prevent gridlocked elections";
-    public static final int DEFAULT_QUORUM_ELECTION_BACKOFF_MAX_MS = 5_000;
+    public static final int DEFAULT_QUORUM_ELECTION_BACKOFF_MAX_MS = 1_000;
 
     public static final String QUORUM_LINGER_MS_CONFIG = QUORUM_PREFIX + 
"append.linger.ms";
     public static final String QUORUM_LINGER_MS_DOC = "The duration in 
milliseconds that the leader will " +
@@ -79,12 +84,12 @@ public class RaftConfig {
     public static final String QUORUM_REQUEST_TIMEOUT_MS_CONFIG = 
QUORUM_PREFIX +
         CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG;
     public static final String QUORUM_REQUEST_TIMEOUT_MS_DOC = 
CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC;
-    public static final int DEFAULT_QUORUM_REQUEST_TIMEOUT_MS = 20_000;
+    public static final int DEFAULT_QUORUM_REQUEST_TIMEOUT_MS = 2_000;
 
     public static final String QUORUM_RETRY_BACKOFF_MS_CONFIG = QUORUM_PREFIX +
         CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG;
     public static final String QUORUM_RETRY_BACKOFF_MS_DOC = 
CommonClientConfigs.RETRY_BACKOFF_MS_DOC;
-    public static final int DEFAULT_QUORUM_RETRY_BACKOFF_MS = 100;
+    public static final int DEFAULT_QUORUM_RETRY_BACKOFF_MS = 20;
 
     private final int requestTimeoutMs;
     private final int retryBackoffMs;

Reply via email to