Hi Tsz-Wo,

After reading tons of logs, I’ve come up with one possible events sequence:

1. One follower is under full gc for a relative long time (5 seconds).
2. Server keeps sending AE in the meanwhile. These AE may contain entries with 
index
(1-10), (11-20), (21-30) etc…
3. AE use gRPC async stub, the above requests maybe handled unordered. (21-30) 
is handled first, and be rejected. So is (11-20). Follower will reply 
“inconsistent AE” to these 2 requests and saying the nextIndex should be 1.
4. Follower accepts (1-10), nextIndex = 11;
5. Leader receives “inconsistent AE”, decrease nextIndex to 1 and retry. (This 
step happens twice since there are two “inconsistent AE” replies)
6. Leader sends (1-30) to follower. This time this AE is again rejected since 
(1-10) already exists. (This step also happens twice).
7. Leader set nextIndex to 11, and sends (11-30) to follower. (Happens twice)
8. Follower accepts first (11-30), reject the second.

This may cause the system sending a lot of extra requests. ( roughly *squared* 
to number of AEs in step 2).

I’m not sure whether grpc will handle AE in an unordered manner (as I assumed 
in step 3). Would you please help me check it out? Thanks in advance.

William


> 2022年8月17日 10:39,William Song <[email protected]> 写道:
> 
> Hi Tsz-Wo,
> 
> Thanks for checking this problem. I’ve posted leader’s log in 
> https://issues.apache.org/jira/browse/RATIS-1674 
> <https://issues.apache.org/jira/browse/RATIS-1674>. I observed a gap of 
> hundreds milliseconds between Follower first reply “inconsistent AE” with 
> leader first receive this “inconsistent AE” reply. During this gap leader 
> keeps sending AppendEntries. Please tell me if there’s anything I can check 
> on the existing log.
> 
> William
> 
>> 2022年8月17日 01:10,Tsz Wo Sze <[email protected]> 写道:
>> 
>> Hi William,
>> 
>> Thanks for reporting the problem.  I checked the log posted in
>> https://issues.apache.org/jira/browse/RATIS-1674 .  The Follower sent the
>> next index 1394 in the INCONSISTENCY reply.   However, the Leader somehow
>> did not use it to send the next appendEntries. It sent a smaller next index
>> (e.g. first entry index 1297) instead so that it was inconsistent again.
>> Could you also post the related log in the Leader?
>> 
>> Tsz-Wo
>> 
>> 
>> On Tue, Aug 16, 2022 at 7:28 AM William Song <[email protected]> wrote:
>> 
>>> Hi,
>>> 
>>> After grpc 1.48.1 release, we packaged ratis with master branch and
>>> thirdparty 1.0.2 and test whether out of direct memory issue is fixed.
>>> Unfortunately, this error is again reproduced.
>>> However, we discover some other abnormalities.
>>> 
>>> That is, one follower often report that it receives inconsistent
>>> AppendEntries from leader as the first entry in AE already exists locally.
>>> After some complaining on "inconsistent appendEntry", this follower can
>>> still write logs to RaftLog and update commitIndex. The "inconsistent AE"
>>> and "success log write" are interleaved. Meanwhile, the leader keeps
>>> reporting changing this follower's nextIndex. The detailed error stack and
>>> log please see issue https://issues.apache.org/jira/browse/RATIS-1674 <
>>> https://issues.apache.org/jira/browse/RATIS-1674>.
>>> 
>>> I looked back at previous directOOM logs and discovered this similar
>>> pattern. Since these "inconsistent AppendEntries" are sent from leader
>>> almost with 0 waiting time (according to follower reply timestamp), I guess
>>> directOOM may related to continuous Leader "inconsistent AppendEntries",
>>> which consumes lots of direct memory for network transimission.
>>> 
>>> Through, I'm confused why a follower will report inconsistent from time to
>>> time while still can successfully append logs between these "inconsistent
>>> AppendEntries”, why leader keeps sending “inconsistent AE" with zero
>>> waiting time.  There is no obvious network congestion.  However, I've set
>>> RaftLog forceFlush to false. I'm not sure whether this async disk flush
>>> strategy is the cause for above abnormalities.
>>> 
>>> I'll be grateful if anyone can help me with this issue. Thanks in advance!
>>> 
>>> 
>>> Regards,
>>> William
>>> 
>>> 
> 

Reply via email to