Details

    • Sub-task
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 1.5.0
    • 1.5.0
    • OM

    Description

      As fix done to simplify OMStateMachine with HDDS-10026, a issue induced.

      Found below log, that leader is selected but is not ready, and further some appendEntry is failing, so its not getting ready.

       

      2024-02-22 04:01:34,446 INFO [om1546336109-server-thread2]-org.apache.ratis.server.RaftServer$Division: om1546336109@group-5A2AC78077C7: change Leader from null to om1546336117 at term 2 for appendEntries, leader elected after 2618ms2024-02-22 04:01:34,454 INFO [om1546336109-server-thread1]-org.apache.ratis.server.RaftServer$Division: om1546336109@group-5A2AC78077C7: Failed appendEntries as the first entry (index 0) already exists (snapshotIndex: 0, commitIndex: 0)2024-02-22 04:01:34,484 INFO [om1546336109-server-thread1]-org.apache.ratis.server.RaftServer$Division: om1546336109@group-5A2AC78077C7: inconsistency entries. Reply:om1546336117<-om1546336109#1:FAIL-t2,INCONSISTENCY,nextIndex=1,followerCommit=0,matchIndex=-12024-02-22 04:01:34,523 INFO [om1546336109-server-thread2]-org.apache.ratis.server.RaftServer$Division: om1546336109@group-5A2AC78077C7: Failed appendEntries as previous log entry ((t:2, i:1)) is not found2024-02-22 04:01:34,523 INFO [om1546336109-server-thread2]-org.apache.ratis.server.RaftServer$Division: om1546336109@group-5A2AC78077C7: inconsistency entries. Reply:om1546336117<-om1546336109#2:FAIL-t2,INCONSISTENCY,nextIndex=1,followerCommit=0,matchIndex=-12024-02-22 04:01:34,531 INFO [om1546336109-server-thread2]-org.apache.ratis.server.RaftServer$Division: om1546336109@group-5A2AC78077C7: set configuration 1: peers:[om1546336109|rpc:ccycloud-2.quasar-xrskyv.root.comops.site:9872|admin:|client:|dataStream:|priority:0|startupRole:FOLLOWER, om1546336117|rpc:ccycloud-5.quasar-xrskyv.root.comops.site:9872|admin:|client:|dataStream:|priority:0|startupRole:FOLLOWER, om1546336113|rpc:ccycloud-7.quasar-xrskyv.root.comops.site:9872|admin:|client:|dataStream:|priority:0|startupRole:FOLLOWER]|listeners:[], old=null2024-02-22 04:01:34,538 INFO [om1546336109-server-thread2]-org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker: om1546336109@group-5A2AC78077C7-SegmentedRaftLogWorker: Rolling segment log-0_0 to index:0  

       

       

      Reason:

      Raft side, LeaderStateImpl.java, leader being ready depends on LastAppliedIndex to be updated.

       

      boolean isReady() {
        return server.getState().getLastAppliedIndex() >= placeHolderIndex;
      } 

       

       

      t with HDDS-10026, its getting updated only during takeSnapshot(). As analysis from logs,

       

      2024-03-17 16:33:22,573 INFO [om1546336117@group-350E5C994683-StateMachineUpdater]-org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine:  applied = (t:0, i:~)2024-03-17 16:33:22,573 INFO [om1546336117@group-350E5C994683-StateMachineUpdater]-org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine:  skipped = -12024-03-17 16:33:22,573 INFO [om1546336117@group-350E5C994683-StateMachineUpdater]-org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine: notified = (t:1, i:0)2024-03-17 16:33:22,573 INFO [om1546336117@group-350E5C994683-StateMachineUpdater]-org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine: snapshot = (t:1, i:0) 

      And later it just stop, no further update.

       

       

      With just updating lastAppliedTermIndex when notified with termIndex from ratis, it started working.

       

      So as solution,

      We need update logic,

      • keep updating lastAppliedTermIndex when notified termIndex is just ascending order (diff is "1")
      • keep last sequence of skipIndex to notifiedIndex, and update if dbUpdate index and skipIndex diff = 1.

       

      Attachments

        Issue Links

          Activity

            People

              sumitagrawl Sumit Agrawal
              ssulav Soumitra Sulav
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: