Hey Kevin, Harsha,

Thanks for the explanation! Now I agree with the motivation and the
addition of this metric.

Regards,
Dong

On Thu, Feb 28, 2019 at 12:30 PM Kevin Lu <lu.ke...@berkeley.edu> wrote:

> Hi Harsha, thanks for the context. It's very useful to see how this
> AtMinIsr metric will function under different environments.
>
> Hi Dong,
>
> Thanks for the response. Let me try to change your mind. :)
>
> I think it's important to relate this metric with the different
> scenarios/configurations that people use, so I have modified the KIP to
> include material from our discussions.
>
> Let's first define UnderReplicated and AtMinIsr that applies in all
> scenarios:
>
> *UnderReplicated:* A partition in which the isr_set_size is not equal to
> the replica_set_size (isr_set_size can be bigger or smaller than
> replica_replication_factor)
> *AtMinIsr: *A partition in which the isr_set_size is equal to the
> min_isr_size, which also means 1 more drop in isr_set_size will lead to at
> least producer (acks=ALL) failure
>
> We can see that there is some overlap between the two definitions,
> especially in the examples you have provided. In these cases, the AtMinIsr
> metric is the exact same as the UnderReplicated metric. However, here are a
> few scenarios in which AtMinIsr provides an improvement over
> UnderReplicated:
>
> *(1) Repartitioning*
>
> When an admin triggers a repartition, the ISR set is first expanded from
> [old_set] to [old_set + new_set], and then reduced to just the [new_set].
> In this case, UnderReplicated will be non-zero even when the ISR set is
> [old_set + new_set]. AtMinIsr will not be non-zero during [old_set +
> new_set] step unless something goes wrong during repartitioning and
> replicas are failing to fetch (reducing the isr_set_size to min_isr_size),
> but we want to know if this happens.
>
> *(2) min.insync.replicas = 1*
>
> The default value for this configuration is 1, and users can change this to
> provide higher durability guarantees. In the default scenario where
> min.insync.replicas = 1 and replication-factor = 3, the AtMinIsr metric
> will be non-zero when isr_set_size = 1, which tells us that 1 more drop in
> this set will lead to a completely unavailable partition. This is very
> powerful for users that have min.insync.replicas = 1 and replication-factor
> > 2.
>
> *(3) replication-factor - min.insync.replicas > 1*
>
> Kafka is built to be fault-tolerant, so we ideally want to be able to
> tolerate more than 1 failure which means we want the difference between
> replication-factor and min.insync.replicas to be > 1. If it is equal to 1,
> then we can only tolerate 1 failure otherwise acks=ALL producers will fail.
>
> We generally want isr_set_size to equal replica_replication_factor to have
> the best guarantees, but this is not always possible for all Kafka users
> depending on their environment and resources. In some situations, we can
> allow the isr_set_size to be reduced, especially if we can tolerate more
> than 1 failure (replication-factor - min.insync.replicas > 1). The only
> requirement is that the isr_set_size must be at least min_isr_size
> otherwise acks=ALL producers will fail.
>
> One example is if we have a cluster with massive load and we do not want to
> trigger a repartition to make isr_set_size = replica_replication_factor
> unless absolutely necessary as repartitioning introduces additional load
> which can impact clients. Maybe we also expect the failed broker to be
> restored soon so we don't want to do anything unless absolutely necessary.
> In these scenarios, the AtMinIsr metric will tell us when we absolutely
> need to *consider* repartitioning or some other action to restore the
> health of the cluster (false negative is still possible but it tells us
> that we could not tolerate any more failure at the time it was non-zero if
> we do not want acks=ALL producers to fail).
>
> In our Kafka environment, we do not even have alerts configured for
> UnderReplicated as it is too noisy for us and we can tolerate some
> failures. We run a periodic job to perform the same functionality as
> AtMinIsr, but it would be better to have it as a metric so we can configure
> an alert on it.
>
>
> The usage of the AtMinIsr metric is the same as UnderReplicated. If the
> user has alerts configured on UnderReplicated and they are using
> min_isr_size = replica_set_size - 1, then AtMinIsr will be the same as
> UnderReplicated. In the other scenarios listed above, AtMinIsr can be a
> more severe. If UnderReplicated is not too noisy for the user, then they
> can keep the UnderReplicated alert and set an AtMinIsr alert with higher
> severity.
>
>
> The way I see it is that the AtMinIsr metric is as good as the
> UnderReplicated metric, but better in some scenarios such as the ones
> listed above.
>
> Regards,
> Kevin
>
> On Thu, Feb 28, 2019 at 10:21 AM Harsha <ka...@harsha.io> wrote:
>
> > Hi Dong,
> >              I think AtMinIsr is still valuable to indicate cluster is at
> > a critical state and something needs to be done asap to restore.
> > To your example
> > " let's say min_isr = 1 and replica_set_size = 3, it is
> > > still possible that planned maintenance (e.g. one broker restart +
> > > partition reassignment) can cause isr size drop to 1. Since AtMinIsr
> can
> > > also cause fault positive (i.e. the fact that AtMinIsr > 0 does not
> > > necessarily need attention from user), "
> >
> > One broker restart shouldn't cause ISR to drop to 1 from 3 unless 2
> > partitions are co-located on the same broker.
> > This is still a valuable indicator to the admins that the partition
> > assignment needs to be moved.
> >
> > In our case, we run 4 replicas for critical topics with min.isr = 2 .
> URPs
> > are not really good indicator to take immediate action if one of the
> > replicas is down. If 2 replicas are down and we are at 2 alive replicas
> > this is stop everything to restore the cluster to a good state.
> >
> > Thanks,
> > Harsha
> >
> >
> >
> >
> >
> >
> > On Wed, Feb 27, 2019, at 11:17 PM, Dong Lin wrote:
> > > Hey Kevin,
> > >
> > > Thanks for the update.
> > >
> > > The KIP suggests that AtMinIsr is better than UnderReplicatedPartition
> as
> > > indicator for alerting. However, in most case where min_isr =
> > > replica_set_size - 1, these two metrics are exactly the same, where
> > planned
> > > maintenance can easily cause positive AtMinIsr value. In the other
> > > scenario, for example let's say min_isr = 1 and replica_set_size = 3,
> it
> > is
> > > still possible that planned maintenance (e.g. one broker restart +
> > > partition reassignment) can cause isr size drop to 1. Since AtMinIsr
> can
> > > also cause fault positive (i.e. the fact that AtMinIsr > 0 does not
> > > necessarily need attention from user), I am not sure it is worth to add
> > > this metric.
> > >
> > > In the Usage section, it is mentioned that user needs to manually check
> > > whether there is ongoing maintenance after AtMinIsr is triggered. Could
> > you
> > > explain how is this different from the current way where we use
> > > UnderReplicatedPartition to trigger alert? More specifically, can we
> just
> > > replace AtMinIsr with UnderReplicatedPartition in the Usage section?
> > >
> > > Thanks,
> > > Dong
> > >
> > >
> > > On Tue, Feb 26, 2019 at 6:49 PM Kevin Lu <lu.ke...@berkeley.edu>
> wrote:
> > >
> > > > Hi Dong!
> > > >
> > > > Thanks for the feedback!
> > > >
> > > > You bring up a good point in that the AtMinIsr metric cannot be used
> to
> > > > identify failure in the mentioned scenarios. I admit the motivation
> > section
> > > > placed too much emphasis on "identifying failure".
> > > >
> > > > I have modified the KIP to reflect the implementation as the AtMinIsr
> > > > metric is intended to serve as a warning as one more failure to a
> > partition
> > > > AtMinIsr will cause producers with acks=ALL configured to fail. It
> has
> > an
> > > > additional benefit when minIsr=1 as it will warn us that the entire
> > > > partition is at risk of going offline, but that is more of a side
> > effect
> > > > that only applies in that scenario (minIsr=1).
> > > >
> > > > Regards,
> > > > Kevin
> > > >
> > > > On Tue, Feb 26, 2019 at 5:11 PM Dong Lin <lindon...@gmail.com>
> wrote:
> > > >
> > > > > Hey Kevin,
> > > > >
> > > > > Thanks for the proposal!
> > > > >
> > > > > It seems that the proposed implementation does not match the
> > motivation.
> > > > > The motivation suggests that the operator wants to tell the planned
> > > > > maintenance (e.g. broker restart) from unplanned failure (e.g.
> > network
> > > > > failure). But the use of the metric AtMinIsr does not really
> > > > differentiate
> > > > > between these causes of the reduced number of ISR. For example, an
> > > > > unplanned failure can cause ISR to drop from 3 to 2 but it can
> still
> > be
> > > > > higher than the minIsr (say 1). And a planned maintenance can cause
> > ISR
> > > > to
> > > > > drop from 3 to 2, which trigger the AtMinIsr metric if minIsr=2.
> Can
> > you
> > > > > update the design doc to fix or explain this issue?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > > On Tue, Feb 12, 2019 at 9:02 AM Kevin Lu <lu.ke...@berkeley.edu>
> > wrote:
> > > > >
> > > > > > Hi All,
> > > > > >
> > > > > > Getting the discussion thread started for KIP-427 in case anyone
> is
> > > > free
> > > > > > right now.
> > > > > >
> > > > > > I’d like to propose a new category of topic partitions *AtMinIsr*
> > which
> > > > > are
> > > > > > partitions that only have the minimum number of in sync replicas
> > left
> > > > in
> > > > > > the ISR set (as configured by min.insync.replicas).
> > > > > >
> > > > > > This would add two new metrics
> > *ReplicaManager.AtMinIsrPartitionCount
> > > > *&
> > > > > > *Partition.AtMinIsr*, and a new TopicCommand option*
> > > > > > --at-min-isr-partitions* to help in monitoring and alerting.
> > > > > >
> > > > > > KIP link: KIP-427: Add AtMinIsr topic partition category (new
> > metric &
> > > > > > TopicCommand option)
> > > > > > <
> > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=103089398
> > > > > > >
> > > > > >
> > > > > > Please take a look and let me know what you think.
> > > > > >
> > > > > > Regards,
> > > > > > Kevin
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to