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

Mirza Aliev updated IGNITE-27501:
---------------------------------
    Description: 
h3. Motivation
The following code leads to OOM locally, 


{code:java}
    @Test
    public void testAppendEntriesWhenFollowerIsInErrorState() throws Exception {
        List<TestPeer> peers = TestUtils.generatePeers(testInfo, 3);

        cluster = new TestCluster("unitest", dataPath, peers, 
ELECTION_TIMEOUT_MILLIS, testInfo);

        for (TestPeer peer : peers)
            assertTrue(cluster.start(peer));

        Node oldLeader = cluster.waitAndGetLeader();
        assertNotNull(oldLeader);

        List<Node> followers = cluster.getFollowers();
        assertEquals(2, followers.size());
        Node errorNode = followers.get(0);
        Node errorNode2 = followers.get(1);

        cluster.stop(errorNode.getNodeId().getPeerId());
        cluster.stop(errorNode2.getNodeId().getPeerId());

        sendTestTaskAndWait(oldLeader, 0, 1_000_000, RaftError.SUCCESS);
    }

    void sendTestTaskAndWait(Node node, int start, int amount,
                                     RaftError err) throws InterruptedException 
{
        for (int i = start; i < start + amount; i++) {
            byte[] bytes = new byte[154216]; // ~150 KB
            ByteBuffer data = ByteBuffer.wrap(bytes);
            Task task = new Task(data, null);
            node.apply(task);
        }
}
{code}

 !screenshot-1.png! 

The reason is because {{LogManagerImpl#logsInMemory}} get full of {{LogEntry}} 
and this {{SegmentList}} is not cleaned up because majority is lost. The other 
reason that when `org.apache.ignite.raft.jraft.core.NodeImpl#apply` and 
{{applyQueue.tryPublishEvent(translator)}} we put to {{applyQueue}} too much 
tasks with entires that occupies all heap.

The idea is to add some throttling to the logic of 
{{org.apache.ignite.raft.jraft.core.NodeImpl#apply}} so we can control how much 
data is applied to a {{applyQueue}}. We want to have some tracker for the size 
of applied task, and if the new task when applied occupies more than 
preconfigured value,  {{org.apache.ignite.raft.jraft.core.NodeImpl#apply}} will 
throw 
{{org.apache.ignite.raft.jraft.error.OverloadException#OverloadException(java.lang.String)}}.


The next tasks is to retry such {{NodeImpl#apply}} if {{OverloadException}} is 
thrown, but this will be done in the next ticket.

Also in the next task we will implement setting value of the tracker through 
the configuration mechanism, so user could control the memory size of the apply 
queue. 
 

  was:
h3. Motivation
The following code leads to OOM locally, 


{code:java}
    @Test
    public void testAppendEntriesWhenFollowerIsInErrorState() throws Exception {
        List<TestPeer> peers = TestUtils.generatePeers(testInfo, 3);

        cluster = new TestCluster("unitest", dataPath, peers, 
ELECTION_TIMEOUT_MILLIS, testInfo);

        for (TestPeer peer : peers)
            assertTrue(cluster.start(peer));

        Node oldLeader = cluster.waitAndGetLeader();
        assertNotNull(oldLeader);

        List<Node> followers = cluster.getFollowers();
        assertEquals(2, followers.size());
        Node errorNode = followers.get(0);
        Node errorNode2 = followers.get(1);

        cluster.stop(errorNode.getNodeId().getPeerId());
        cluster.stop(errorNode2.getNodeId().getPeerId());

        sendTestTaskAndWait(oldLeader, 0, 1_000_000, RaftError.SUCCESS);
    }

    void sendTestTaskAndWait(Node node, int start, int amount,
                                     RaftError err) throws InterruptedException 
{
        for (int i = start; i < start + amount; i++) {
            byte[] bytes = new byte[154216]; // ~150 KB
            ByteBuffer data = ByteBuffer.wrap(bytes);
            Task task = new Task(data, null);
            node.apply(task);
        }
}
{code}

 !screenshot-1.png! 

The reason is because {{LogManagerImpl#logsInMemory}} get full of {{LogEntry}} 
and this {{SegmentList}} is not cleaned up because majority is lost. The other 
reason that when `org.apache.ignite.raft.jraft.core.NodeImpl#apply` and 
{{applyQueue.tryPublishEvent(translator)}} we put to {{applyQueue}} too much 
tasks with entires that occupies all heap.

The idea is to add some throttling to the logic of 
{{org.apache.ignite.raft.jraft.core.NodeImpl#apply}} so we can control how much 
data is applied to a {{applyQueue}}. We want to have some tracker for the size 
of applied task, and if the new task when applied occupies more than 
preconfigured value,  {{org.apache.ignite.raft.jraft.core.NodeImpl#apply}} will 
throw 
{{org.apache.ignite.raft.jraft.error.OverloadException#OverloadException(java.lang.String)}}.

The value of the tracker must be configureable through the configuration 
mechanism and it must be possible to configure it through the config. 

The next task is to retry such {{NodeImpl#apply}} if {{OverloadException}} is 
thrown, but this will be done in the next ticket.



> Raft group under the load and lost majority could die with OOM
> --------------------------------------------------------------
>
>                 Key: IGNITE-27501
>                 URL: https://issues.apache.org/jira/browse/IGNITE-27501
>             Project: Ignite
>          Issue Type: Bug
>            Reporter: Mirza Aliev
>            Assignee: Mirza Aliev
>            Priority: Major
>              Labels: ignite-3
>         Attachments: screenshot-1.png
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> h3. Motivation
> The following code leads to OOM locally, 
> {code:java}
>     @Test
>     public void testAppendEntriesWhenFollowerIsInErrorState() throws 
> Exception {
>         List<TestPeer> peers = TestUtils.generatePeers(testInfo, 3);
>         cluster = new TestCluster("unitest", dataPath, peers, 
> ELECTION_TIMEOUT_MILLIS, testInfo);
>         for (TestPeer peer : peers)
>             assertTrue(cluster.start(peer));
>         Node oldLeader = cluster.waitAndGetLeader();
>         assertNotNull(oldLeader);
>         List<Node> followers = cluster.getFollowers();
>         assertEquals(2, followers.size());
>         Node errorNode = followers.get(0);
>         Node errorNode2 = followers.get(1);
>         cluster.stop(errorNode.getNodeId().getPeerId());
>         cluster.stop(errorNode2.getNodeId().getPeerId());
>         sendTestTaskAndWait(oldLeader, 0, 1_000_000, RaftError.SUCCESS);
>     }
>     void sendTestTaskAndWait(Node node, int start, int amount,
>                                      RaftError err) throws 
> InterruptedException {
>         for (int i = start; i < start + amount; i++) {
>             byte[] bytes = new byte[154216]; // ~150 KB
>             ByteBuffer data = ByteBuffer.wrap(bytes);
>             Task task = new Task(data, null);
>             node.apply(task);
>         }
> }
> {code}
>  !screenshot-1.png! 
> The reason is because {{LogManagerImpl#logsInMemory}} get full of 
> {{LogEntry}} and this {{SegmentList}} is not cleaned up because majority is 
> lost. The other reason that when 
> `org.apache.ignite.raft.jraft.core.NodeImpl#apply` and 
> {{applyQueue.tryPublishEvent(translator)}} we put to {{applyQueue}} too much 
> tasks with entires that occupies all heap.
> The idea is to add some throttling to the logic of 
> {{org.apache.ignite.raft.jraft.core.NodeImpl#apply}} so we can control how 
> much data is applied to a {{applyQueue}}. We want to have some tracker for 
> the size of applied task, and if the new task when applied occupies more than 
> preconfigured value,  {{org.apache.ignite.raft.jraft.core.NodeImpl#apply}} 
> will throw 
> {{org.apache.ignite.raft.jraft.error.OverloadException#OverloadException(java.lang.String)}}.
> The next tasks is to retry such {{NodeImpl#apply}} if {{OverloadException}} 
> is thrown, but this will be done in the next ticket.
> Also in the next task we will implement setting value of the tracker through 
> the configuration mechanism, so user could control the memory size of the 
> apply queue. 
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to