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