[ 
https://issues.apache.org/jira/browse/KAFKA-10853?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17250598#comment-17250598
 ] 

Kyle Ambroff-Kao commented on KAFKA-10853:
------------------------------------------

Thanks for the reply Jun!
{quote}As for your proposal of using largestAckedOffset, the challenge is that 
we allow producers with different ack mode even on the same topic.
{quote}
Hrm, yeah that's a great point. I think for our internal use case the 
largestAckedOffset change would be OK since a producer that doesn't use acks: 
all would be a configuration bug. But it's true that one mis-configured client 
would ruin it for everyone.

I guess my proposal would have to either require some strict ordering of 
acknowledgements to producers, blocking acknowledgment of a acks=1 
producerequest until a acks=all request that came before it completes. That's 
obviously terrible.

Or I guess it could be done with a new topic configuration that put the topic 
into durable mode, enforcing acks=all reguardless of what the ProduceRequest 
says. That sounds less terrible.
{quote}It sounds like you already have a separate mechanism to detect the 
slowness of a broker based on request.max.local.time.ms. So, another approach 
could be communicating this info to the controller so that it can choose to 
move the leader off the slow broker. It seems that this could solve the problem 
if request.max.local.time.ms < replica.lag.time.max.ms.
{quote}
Hrm, that's an interesting proposal. I like this idea. We could change our code 
to, instead of halt if request.max.local.time.ms is exceeded, just stop 
accepting client traffic (except for follower fetch requests), and signal to 
the controller that leadership should be shifted elsewhere. That sounds really 
nice.

I'm nervous about the idea of changing out replica.lag.time.max.ms though. We 
have had that set to 10 seconds for many years now and I'm nervous about 
increasing it. 10 seconds already feels like it is too long.

And decreasing replica.lag.time.max.ms seems scary too because we don't want 
false positives. We've set it to 60 seconds through experience and trial and 
error, since we generally don't see GC pauses that long, but we do see IOPS 
pause for that long when we have a real storage issue.

The other problem is that I think handling failures this way only works for 
certain failure modes, like a really slow or failing disk. It doesn't handle 
network partitions at all, since we may not be able to reach the controller. Or 
the pause may be caused by something else that prevents any thread from being 
scheduled.

I need to think about this for a bit.

> Replication protocol deficiencies with workloads requiring high durability 
> guarantees
> -------------------------------------------------------------------------------------
>
>                 Key: KAFKA-10853
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10853
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 2.4.0
>            Reporter: Kyle Ambroff-Kao
>            Priority: Major
>
> *tl;dr: The definition of ISR and the consistency model from the perspective 
> of the producer seem a bit out of sync*
> We have many systems in production that trade off availability in order to 
> provide stronger consistency guarantees. Most of these configurations look 
> like this:
> Topic configuration:
>  * replication factor 3
>  * min.insync.replicas=2
>  * unclean.leader.election.enable=false
> Producer configuration:
>  * acks=all
> Broker configuration:
>  * replica.lag.time.max.ms=10000
> So the goal here is to reduce the chance of ever dropping a message that the 
> leader has acknowledged to the producer.
> This works great, except that we've found some situations in production where 
> we are forced to enable unclean leader election to recover, which we never 
> want to do. These situations all seem totally avoidable with some small 
> tweaks to the replication protocol.
> *A scenario we've seen many times*
> The following sequence of events are in time order: A replica set for a 
> topic-partition TP with leader L and replicas R1 and R2. All three replicas 
> are in ISR.
>  # Producer sends ProduceRequest R with acks=all that contains a message 
> batch to the leader L.
>  # L receives R and appends the batch it contains to the active segment of TP 
> but does not ack to the producer yet because the request was acks=all
>  # A storage fault occurs on L which makes all IOPS take a long time but 
> doesn't cause a hard failure.
>  # R1 and R2 send follower fetch requests to L which are infinitely delayed 
> due to the storage fault on L.
>  # 10 seconds after appending the batch and appending it to the log, L 
> shrinks the ISR, removing R1 and R2. This is because ISR is defined as at 
> most replica.lag.time.max.ms milliseconds behind the log append time of the 
> leader end offset. The leader end offset is a message that has not been 
> replicated yet.
> The storage fault example in step 3 could easily be another kind of fault. 
> Say for example, L is partitioned from R1 and R2 but not from ZooKeeper or 
> the producer.
> The producer never receives acknowledgement of the ProduceRequest because the 
> min.insync.replicas constraint was never satisfied. So in terms of data 
> consistency, everything is working fine.
> The problem is recovering from this situation. If the fault on L is not a 
> temporary blip, then L needs to be replaced. But since L shrunk the ISR, the 
> only way that leadership can move to either R1 or R2 is to set 
> unclean.leader.election.enable=true.
> This works but it is a potentially unsafe way to recover and move leadership. 
> It would be better to have other options.
> *Recovery could be automatic in this scenario.*
> If you think about it, from the perspective of the producer, the write was 
> not acknowledged, and therefore, L, R1 and R2 are actually in-sync. So it 
> should actually be totally safe for leadership to transition to either R1 or 
> R2.
> It seems that the producer and the leader don't have fully compatible 
> definitions for what it means for the replica set to be in-sync. If the 
> leader L used different rules for defining ISR, it could allow self-healing 
> in this or similar scenarios, since the ISR would not shrink.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to