You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Luis Alves <lm...@gmail.com> on 2023/03/06 14:11:15 UTC

Kafka cluster rolling restart

Hello,

I'm doing some tests with rolling restarts in a Kafka cluster and I have a
couple of questions related to the impact of rolling restarts on Kafka
consumers/producers and on the overall process.

First, some context on my setup:

   - Kafka cluster with 3 nodes.
   - Topic replication factor of 3 with minISR of 2.
   - All topics have a single partition (I intend to increase the
   partitioning factor in the future, but for now it's just 1 for testing
   purposes).
   - Kafka version is 3.2.3.
   - I have two systems that communicate via these Kafka topics. The
   high-level flow is:
      1. System A sends a message to a Kafka topic (at a rate of ~10
      events/sec).
      2. System B consumes the message.
      3. System B sends a reply to a Kafka topic.
      4. System A consumes the reply.
   - When the system is stable, I see end-to-end latencies (measured on
   System A) around 10ms in the 99th percentile.
   - System A is using Kafka client 3.3.1, and System B is using Kafka
   client 3.4.0.
   - Kafka consumers and producers on both systems are with the default
   configurations, except that the Kafka consumers have auto-commits disabled.
   - All Kafka brokers are configured with controlled.shutdown.enable set
   to true.
   - The Kafka cluster is running in Kubernetes and deployed using Strimzi
   (this is just for awareness).
   - The rolling restart process is the following (when using Strimzi to
   manage it, and when we try to do it manually):
      1. Restart each broker, one at a time, by sending a SIGTERM to the
      broker process. The controller broker is the last one to be restarted.
      2. Only restart the next broker when the current broker reports the
      broker state as RUNNING. Note: when we do this manually (without
Strimzi),
      we wait to see the end-to-end latencies stabilize before moving
to the next
      broker.

Now, my questions:

   1. When we do this process with Strimzi (waits for the broker state to
   be RUNNING before moving to the next one), we've seen end-to-end latencies
   growing up to 1-2 minutes (System A is not even able to send events to the
   Kafka topic). This is unexpected because AFAIK the configurations that we
   are using are the ones recommended for high availability during rolling
   restarts. My question is: is it enough to wait for the broker state to be
   RUNNING to move on to the next broker?
   2. When we do this process manually (we wait for end-to-end latencies to
   stabilize and only then move to the next broker), we've seen end-to-end
   latencies growing up to 1 second. While this is much better than what we
   see in 1., my question is whether this latency increase is expected or not.

Thanks in advance,
Luís Alves

Re: Kafka cluster rolling restart

Posted by Divij Vaidya <di...@gmail.com>.
I concur with what Peter mentioned. You should wait for the
under-replicated partition count to be zero.

The increase in latency could be expected. Let's try to take a deeper look
at what happens during a rolling restart.

When you perform a controlled shutdown of a node, it will tell the
controller that it is ready to shutdown and in response the controller will
take the leadership away from that node and distribute it amongst others
(uniformly assuming uniform distribution of partitions in the first place).
Hence, compared to a steady state, during a rolling restart, you can expect
the controller to handle more requests and produce more LISR requests due
to leadership redistribution. While the leadership is being redistributed,
the producer requests will fail and a backpressure will be built up on the
producer. The producer should be configured to handle this backpressure
else your producer may block/degrade traffic for partitions even if they
are not hosted on the broker being restarted. The configuration is done by
setting an appropriate value of "buffer.memory" and "max.block.ms" on the
producer. On the brokers which are not being restarted, you will get an
increased number of LISR requests and they may become new leaders & group
coordinators. The additional work of being a leader & group
coordinator will increase the number of requests they have to handle and if
they don't have appropriate headroom (in terms of CPU, memory, num io &
network threads etc.), their request processing will slow down leading to
increased latency.

Could you analyze your cluster based on the above explanation and let us
know if you are facing one of the bottlenecks described above?

--
Divij Vaidya



On Mon, Mar 6, 2023 at 7:18 PM Peter Bukowinski <pm...@gmail.com> wrote:

> When doing rolling restarts, I always wait until the under-replicated
> partition count returns to zero before restarting the next broker. This
> state is achieved AFTER the last restarted broker returns to a running
> state. If you just wait for the running state, you risk restarting the next
> broker before all partitions have returned to healthy, and then you’ll have
> offline partitions because your minISR is 2.
>
> --
> Peter Bukowinski
>
> > On Mar 6, 2023, at 7:04 AM, Luis Alves <lm...@gmail.com> wrote:
> >
> > Hello,
> >
> > I'm doing some tests with rolling restarts in a Kafka cluster and I have
> a
> > couple of questions related to the impact of rolling restarts on Kafka
> > consumers/producers and on the overall process.
> >
> > First, some context on my setup:
> >
> >   - Kafka cluster with 3 nodes.
> >   - Topic replication factor of 3 with minISR of 2.
> >   - All topics have a single partition (I intend to increase the
> >   partitioning factor in the future, but for now it's just 1 for testing
> >   purposes).
> >   - Kafka version is 3.2.3.
> >   - I have two systems that communicate via these Kafka topics. The
> >   high-level flow is:
> >      1. System A sends a message to a Kafka topic (at a rate of ~10
> >      events/sec).
> >      2. System B consumes the message.
> >      3. System B sends a reply to a Kafka topic.
> >      4. System A consumes the reply.
> >   - When the system is stable, I see end-to-end latencies (measured on
> >   System A) around 10ms in the 99th percentile.
> >   - System A is using Kafka client 3.3.1, and System B is using Kafka
> >   client 3.4.0.
> >   - Kafka consumers and producers on both systems are with the default
> >   configurations, except that the Kafka consumers have auto-commits
> disabled.
> >   - All Kafka brokers are configured with controlled.shutdown.enable set
> >   to true.
> >   - The Kafka cluster is running in Kubernetes and deployed using Strimzi
> >   (this is just for awareness).
> >   - The rolling restart process is the following (when using Strimzi to
> >   manage it, and when we try to do it manually):
> >      1. Restart each broker, one at a time, by sending a SIGTERM to the
> >      broker process. The controller broker is the last one to be
> restarted.
> >      2. Only restart the next broker when the current broker reports the
> >      broker state as RUNNING. Note: when we do this manually (without
> > Strimzi),
> >      we wait to see the end-to-end latencies stabilize before moving
> > to the next
> >      broker.
> >
> > Now, my questions:
> >
> >   1. When we do this process with Strimzi (waits for the broker state to
> >   be RUNNING before moving to the next one), we've seen end-to-end
> latencies
> >   growing up to 1-2 minutes (System A is not even able to send events to
> the
> >   Kafka topic). This is unexpected because AFAIK the configurations that
> we
> >   are using are the ones recommended for high availability during rolling
> >   restarts. My question is: is it enough to wait for the broker state to
> be
> >   RUNNING to move on to the next broker?
> >   2. When we do this process manually (we wait for end-to-end latencies
> to
> >   stabilize and only then move to the next broker), we've seen end-to-end
> >   latencies growing up to 1 second. While this is much better than what
> we
> >   see in 1., my question is whether this latency increase is expected or
> not.
> >
> > Thanks in advance,
> > Luís Alves
>

Re: Kafka cluster rolling restart

Posted by Peter Bukowinski <pm...@gmail.com>.
When doing rolling restarts, I always wait until the under-replicated partition count returns to zero before restarting the next broker. This state is achieved AFTER the last restarted broker returns to a running state. If you just wait for the running state, you risk restarting the next broker before all partitions have returned to healthy, and then you’ll have offline partitions because your minISR is 2.

--
Peter Bukowinski

> On Mar 6, 2023, at 7:04 AM, Luis Alves <lm...@gmail.com> wrote:
> 
> Hello,
> 
> I'm doing some tests with rolling restarts in a Kafka cluster and I have a
> couple of questions related to the impact of rolling restarts on Kafka
> consumers/producers and on the overall process.
> 
> First, some context on my setup:
> 
>   - Kafka cluster with 3 nodes.
>   - Topic replication factor of 3 with minISR of 2.
>   - All topics have a single partition (I intend to increase the
>   partitioning factor in the future, but for now it's just 1 for testing
>   purposes).
>   - Kafka version is 3.2.3.
>   - I have two systems that communicate via these Kafka topics. The
>   high-level flow is:
>      1. System A sends a message to a Kafka topic (at a rate of ~10
>      events/sec).
>      2. System B consumes the message.
>      3. System B sends a reply to a Kafka topic.
>      4. System A consumes the reply.
>   - When the system is stable, I see end-to-end latencies (measured on
>   System A) around 10ms in the 99th percentile.
>   - System A is using Kafka client 3.3.1, and System B is using Kafka
>   client 3.4.0.
>   - Kafka consumers and producers on both systems are with the default
>   configurations, except that the Kafka consumers have auto-commits disabled.
>   - All Kafka brokers are configured with controlled.shutdown.enable set
>   to true.
>   - The Kafka cluster is running in Kubernetes and deployed using Strimzi
>   (this is just for awareness).
>   - The rolling restart process is the following (when using Strimzi to
>   manage it, and when we try to do it manually):
>      1. Restart each broker, one at a time, by sending a SIGTERM to the
>      broker process. The controller broker is the last one to be restarted.
>      2. Only restart the next broker when the current broker reports the
>      broker state as RUNNING. Note: when we do this manually (without
> Strimzi),
>      we wait to see the end-to-end latencies stabilize before moving
> to the next
>      broker.
> 
> Now, my questions:
> 
>   1. When we do this process with Strimzi (waits for the broker state to
>   be RUNNING before moving to the next one), we've seen end-to-end latencies
>   growing up to 1-2 minutes (System A is not even able to send events to the
>   Kafka topic). This is unexpected because AFAIK the configurations that we
>   are using are the ones recommended for high availability during rolling
>   restarts. My question is: is it enough to wait for the broker state to be
>   RUNNING to move on to the next broker?
>   2. When we do this process manually (we wait for end-to-end latencies to
>   stabilize and only then move to the next broker), we've seen end-to-end
>   latencies growing up to 1 second. While this is much better than what we
>   see in 1., my question is whether this latency increase is expected or not.
> 
> Thanks in advance,
> Luís Alves