[ 
https://issues.apache.org/jira/browse/RATIS-1534?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Junhua Zhang updated RATIS-1534:
--------------------------------
    Description: 
In our previous test([see streaming test 
doc|https://docs.google.com/document/d/1mS3GqovQ3D1b7V0L3--VF9xhl5jdId1mSL0cQNb7uHo/edit]),
 we found that SegmentedRaftLogWorker writing RaftLog seriously affected the 
performance of streaming. Mainly because the flush action was very frequent.

So we can change flush to async. This will greatly improve performance. Of 
course we'd better make this configurable, which is disable by default.

!screenshot-1.png!

We previously tried to reduce disk IO by introducing a minimum interval between 
flushes, the relevant pr is here: [https://github.com/apache/ratis/pull/611]

However, after subsequent tests, we found that the raft performance has 
degraded,  because {color:#FF0000}flushIfNecessary(){color} function does 
{color:#FF0000}more operations{color} than just flushing the stream:

!image-2022-03-03-16-23-32-929.png|width=741,height=593!

Since we enforced the minimum time for each flush, the raft log to can't be 
committed in flush intervals, which reduces the performance of raft.

So we want to separate out.flush()  from flushIfNecessary() and execute it 
asynchronously, so that the process of writing to disk will not block the 
operation of other threads.

  was:
In our previous test([see streaming test 
doc|https://docs.google.com/document/d/1mS3GqovQ3D1b7V0L3--VF9xhl5jdId1mSL0cQNb7uHo/edit]),
 we found that SegmentedRaftLogWorker writing RaftLog seriously affected the 
performance of streaming. Mainly because the flush action was very frequent.

So we can change flush to async. This will greatly improve performance. Of 
course we'd better make this configurable, which is disable by default.

 !screenshot-1.png! 


> SegmentedRaftLogWorker should enforce a minimum time interval between flush 
> calls
> ---------------------------------------------------------------------------------
>
>                 Key: RATIS-1534
>                 URL: https://issues.apache.org/jira/browse/RATIS-1534
>             Project: Ratis
>          Issue Type: Improvement
>          Components: server
>            Reporter: mingchao zhao
>            Assignee: Junhua Zhang
>            Priority: Major
>             Fix For: 2.3.0
>
>         Attachments: image-2022-03-03-16-23-32-929.png, screenshot-1.png
>
>          Time Spent: 1h 50m
>  Remaining Estimate: 0h
>
> In our previous test([see streaming test 
> doc|https://docs.google.com/document/d/1mS3GqovQ3D1b7V0L3--VF9xhl5jdId1mSL0cQNb7uHo/edit]),
>  we found that SegmentedRaftLogWorker writing RaftLog seriously affected the 
> performance of streaming. Mainly because the flush action was very frequent.
> So we can change flush to async. This will greatly improve performance. Of 
> course we'd better make this configurable, which is disable by default.
> !screenshot-1.png!
> We previously tried to reduce disk IO by introducing a minimum interval 
> between flushes, the relevant pr is here: 
> [https://github.com/apache/ratis/pull/611]
> However, after subsequent tests, we found that the raft performance has 
> degraded,  because {color:#FF0000}flushIfNecessary(){color} function does 
> {color:#FF0000}more operations{color} than just flushing the stream:
> !image-2022-03-03-16-23-32-929.png|width=741,height=593!
> Since we enforced the minimum time for each flush, the raft log to can't be 
> committed in flush intervals, which reduces the performance of raft.
> So we want to separate out.flush()  from flushIfNecessary() and execute it 
> asynchronously, so that the process of writing to disk will not block the 
> operation of other threads.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to