Uploaded image for project: 'Ratis'
  1. Ratis
  2. RATIS-2045

SnapshotInstallationHandler doesn't notify follower when snapshotIndex is -1 and firstAvailableLogIndex is 0

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.1.0
    • server
    • None

    Description

      Follower logs:

      2024-03-11 22:03:29,047 INFO [grpc-default-executor-0]-org.apache.ratis.server.impl.SnapshotInstallationHandler: om39@group-ACB3E04B8AE9: receive installSnapshot: om32->om39#0-t6,notify:(t:1, i:0)
      2024-03-11 22:03:29,057 INFO [grpc-default-executor-0]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: change Leader from null to om32 at term 6 for installSnapshot, leader elected after 6686ms
      2024-03-11 22:03:29,062 INFO [grpc-default-executor-0]-org.apache.ratis.server.impl.SnapshotInstallationHandler: om39@group-ACB3E04B8AE9: Received notification to install snapshot at index 0
      2024-03-11 22:03:29,064 INFO [grpc-default-executor-0]-org.apache.ratis.server.impl.SnapshotInstallationHandler: om39@group-ACB3E04B8AE9: InstallSnapshot notification result: ALREADY_INSTALLED, current snapshot index: -1
      2024-03-11 22:03:29,328 INFO [grpc-default-executor-0]-org.apache.ratis.server.impl.SnapshotInstallationHandler: om39@group-ACB3E04B8AE9: set new configuration index: 63986
      configurationEntry {
        peers {
          id: "om32"
          address: "ccycloud-3.heku-719-3.root.comops.site:9872"
          startupRole: FOLLOWER
        }
        peers {
          id: "om30"
          address: "ccycloud-2.heku-719-3.root.comops.site:9872"
          startupRole: FOLLOWER
        }
      }
       from snapshot
      2024-03-11 22:03:29,332 INFO [grpc-default-executor-0]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: set configuration 63986: peers:[om32|rpc:ccycloud-3.heku-719-3.root.comops.site:9872|admin:|client:|dataStream:|priority:0|startupRole:FOLLOWER, om30|rpc:ccycloud-2.heku-719-3.root.comops.site:
      9872|admin:|client:|dataStream:|priority:0|startupRole:FOLLOWER]|listeners:[], old=null
      2024-03-11 22:03:29,344 INFO [grpc-default-executor-0]-org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine: Received Configuration change notification from Ratis. New Peer list:
      [id: "om32"
      address: "ccycloud-3.heku-719-3.root.comops.site:9872"
      startupRole: FOLLOWER
      , id: "om30"
      address: "ccycloud-2.heku-719-3.root.comops.site:9872"
      startupRole: FOLLOWER
      ]
      2024-03-11 22:03:29,346 INFO [grpc-default-executor-0]-org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer: Added OM om32 to Ratis Peers list.
      2024-03-11 22:03:29,347 INFO [grpc-default-executor-0]-org.apache.hadoop.ozone.om.OzoneManager: Added OM om32 to the Peer list.
      2024-03-11 22:03:29,347 INFO [grpc-default-executor-0]-org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer: Added OM om30 to Ratis Peers list.
      2024-03-11 22:03:29,348 INFO [grpc-default-executor-0]-org.apache.hadoop.ozone.om.OzoneManager: Added OM om30 to the Peer list.
      2024-03-11 22:03:29,348 INFO [grpc-default-executor-0]-org.apache.ratis.server.impl.SnapshotInstallationHandler: om39@group-ACB3E04B8AE9: reply installSnapshot: om32<-om39#0:OK-t0,ALREADY_INSTALLED
      2024-03-11 22:03:29,362 INFO [grpc-default-executor-0]-org.apache.ratis.grpc.server.GrpcServerProtocolService: om39: Completed INSTALL_SNAPSHOT, lastRequest: om32->om39#0-t6,notify:(t:1, i:0)
      2024-03-11 22:03:29,363 INFO [grpc-default-executor-0]-org.apache.ratis.grpc.server.GrpcServerProtocolService: om39: Completed INSTALL_SNAPSHOT, lastReply: null
      2024-03-11 22:03:29,446 INFO [om39-server-thread1]-org.apache.ratis.server.impl.RoleInfo: om39: start om39@group-ACB3E04B8AE9-FollowerState
      2024-03-11 22:03:29,447 INFO [om39@group-ACB3E04B8AE9-FollowerState]-org.apache.ratis.server.RaftServerConfigKeys: raft.server.rpc.first-election.timeout.min = 5s (fallback to raft.server.rpc.timeout.min)
      2024-03-11 22:03:29,447 INFO [om39@group-ACB3E04B8AE9-FollowerState]-org.apache.ratis.server.RaftServerConfigKeys: raft.server.rpc.first-election.timeout.max = 5200ms (fallback to raft.server.rpc.timeout.max)
      2024-03-11 22:03:29,449 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: Failed appendEntries as previous log entry ((t:1, i:0)) is not found
      2024-03-11 22:03:29,455 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: inconsistency entries. Reply:om32<-om39#0:FAIL-t6,INCONSISTENCY,nextIndex=0,followerCommit=-1,matchIndex=-1
      2024-03-11 22:03:29,488 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: Failed appendEntries as previous log entry ((t:1, i:0)) is not found
      2024-03-11 22:03:29,488 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: inconsistency entries. Reply:om32<-om39#1:FAIL-t6,INCONSISTENCY,nextIndex=0,followerCommit=-1,matchIndex=-1
      2024-03-11 22:03:29,511 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: Failed appendEntries as previous log entry ((t:1, i:1024)) is not found
      2024-03-11 22:03:29,511 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: inconsistency entries. Reply:om32<-om39#2:FAIL-t6,INCONSISTENCY,nextIndex=0,followerCommit=-1,matchIndex=-1
      2024-03-11 22:03:29,526 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: Failed appendEntries as previous log entry ((t:1, i:2048)) is not found
      2024-03-11 22:03:29,526 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: inconsistency entries. Reply:om32<-om39#3:FAIL-t6,INCONSISTENCY,nextIndex=0,followerCommit=-1,matchIndex=-1
      2024-03-11 22:03:29,543 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: Failed appendEntries as previous log entry ((t:1, i:3072)) is not found
      2024-03-11 22:03:29,543 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: inconsistency entries. Reply:om32<-om39#4:FAIL-t6,INCONSISTENCY,nextIndex=0,followerCommit=-1,matchIndex=-1
      2024-03-11 22:03:29,558 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: set configuration 0: peers:[om31|rpc:ccycloud-1.heku-719-3.root.comops.site:9872|admin:|client:|dataStream:|priority:0|startupRole:FOLLOWER, om32|rpc:ccycloud-3.heku-719-3.root.comops.site:9872|adm
      in:|client:|dataStream:|priority:0|startupRole:FOLLOWER, om30|rpc:ccycloud-2.heku-719-3.root.comops.site:9872|admin:|client:|dataStream:|priority:0|startupRole:FOLLOWER]|listeners:[], old=null
      2024-03-11 22:03:29,567 INFO [om39-server-thread1]-org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker: om39@group-ACB3E04B8AE9-SegmentedRaftLogWorker: Starting segment from index:0
      2024-03-11 22:03:29,793 INFO [om39@group-ACB3E04B8AE9-SegmentedRaftLogWorker]-org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker: om39@group-ACB3E04B8AE9-SegmentedRaftLogWorker: created new log segment /var/lib/hadoop-ozone/om/ratis/a655b65f-764f-3cab-b5c3-acb3e04b8ae9/current/log_inprogress_0
      2024-03-11 22:03:29,841 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: set configuration 0: peers:[om31|rpc:ccycloud-1.heku-719-3.root.comops.site:9872|admin:|client:|dataStream:|priority:0|startupRole:FOLLOWER, om32|rpc:ccycloud-3.heku-719-3.root.comops.site:9872|adm
      in:|client:|dataStream:|priority:0|startupRole:FOLLOWER, om30|rpc:ccycloud-2.heku-719-3.root.comops.site:9872|admin:|client:|dataStream:|priority:0|startupRole:FOLLOWER]|listeners:[], old=null
      2024-03-11 22:03:29,852 INFO [om39@group-ACB3E04B8AE9-StateMachineUpdater]-org.apache.hadoop.ozone.om.ratis.OzoneManagerStateMachine: Received Configuration change notification from Ratis. New Peer list:
      [id: "om31"
      address: "ccycloud-1.heku-719-3.root.comops.site:9872"
      startupRole: FOLLOWER
      , id: "om32"
      address: "ccycloud-3.heku-719-3.root.comops.site:9872"
      startupRole: FOLLOWER
      , id: "om30"
      address: "ccycloud-2.heku-719-3.root.comops.site:9872"
      startupRole: FOLLOWER
      ]
      2024-03-11 22:03:29,863 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: Failed appendEntries as the first entry (index 0) already exists (snapshotIndex: -1, commitIndex: 127)
      2024-03-11 22:03:29,863 INFO [om39-server-thread1]-org.apache.ratis.server.RaftServer$Division: om39@group-ACB3E04B8AE9: inconsistency entries. Reply:om32<-om39#9:FAIL-t6,INCONSISTENCY,nextIndex=128,followerCommit=127,matchIndex=-1
      2024-03-11 22:03:29,887 ERROR [om39@group-ACB3E04B8AE9-StateMachineUpdater]-org.apache.hadoop.ozone.om.OzoneManager: There is no OM configuration for node ID om31 in ozone-site.xml.
      2024-03-11 22:03:30,091 INFO [OM StateMachine ApplyTransaction Thread - 0]-org.apache.hadoop.ozone.om.request.bucket.OMBucketCreateRequest: created bucket: cloudera-health-monitoring-ozone-basic-canary-bucket of layout FILE_SYSTEM_OPTIMIZED in volume: s3v
      2024-03-11 22:03:30,313 INFO [om39-server-thread2]-org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker: om39@group-ACB3E04B8AE9-SegmentedRaftLogWorker: Rolling segment log-0_8572 to index:8572
      

      Leader logs:

      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.server.RaftServerConfigKeys: raft.server.log.appender.snapshot.chunk.size.max = 16MB (=16777216) (default)
      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.server.RaftServerConfigKeys: raft.server.log.appender.buffer.byte-limit = 33554432 (custom)
      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.server.RaftServerConfigKeys: raft.server.log.appender.buffer.element-limit = 1024 (custom)
      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.server.RaftServerConfigKeys: raft.server.log.appender.wait-time.min = 0ms (custom)
      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.grpc.GrpcConfigKeys: raft.grpc.server.leader.outstanding.appends.max = 128 (default)
      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.server.RaftServerConfigKeys: raft.server.rpc.request.timeout = 3000ms (default)
      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.grpc.GrpcConfigKeys: raft.grpc.server.install_snapshot.request.element-limit = 8 (default)
      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.grpc.GrpcConfigKeys: raft.grpc.server.install_snapshot.request.timeout = 3000ms (default)
      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.server.RaftServerConfigKeys: raft.server.log.appender.install.snapshot.enabled = false (custom)
      2024-03-11 22:03:28,633 INFO [IPC Server handler 4 on 9862]-org.apache.ratis.grpc.GrpcConfigKeys: raft.grpc.server.heartbeat.channel = true (default)
      2024-03-11 22:03:28,634 INFO [om32@group-ACB3E04B8AE9->om39-GrpcLogAppender-LogAppenderDaemon]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-GrpcLogAppender: followerNextIndex = 0 but logStartIndex = 0, notify follower to install snapshot-(t:1, i:0)
      2024-03-11 22:03:28,641 INFO [om32@group-ACB3E04B8AE9->om39-GrpcLogAppender-LogAppenderDaemon]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-GrpcLogAppender: send om32->om39#0-t6,notify:(t:1, i:0)
      2024-03-11 22:03:28,641 INFO [om32@group-ACB3E04B8AE9->om39-GrpcLogAppender-LogAppenderDaemon]-org.apache.ratis.grpc.server.GrpcServerProtocolClient: Build channel for om39
      2024-03-11 22:03:28,693 INFO [Socket Reader #1 for port 9862]-SecurityLogger.org.apache.hadoop.ipc.Server: Auth successful for hue/ccycloud-1.heku-719-3.root.comops.site@ROOT.COMOPS.SITE (auth:KERBEROS)
      2024-03-11 22:03:28,696 INFO [Socket Reader #1 for port 9862]-SecurityLogger.org.apache.hadoop.security.authorize.ServiceAuthorizationManager: Authorization successful for hue/ccycloud-1.heku-719-3.root.comops.site@ROOT.COMOPS.SITE (auth:KERBEROS) for protocol=interface org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol
      2024-03-11 22:03:29,375 INFO [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-InstallSnapshotResponseHandler: received the first reply om32<-om39#0:OK-t0,ALREADY_INSTALLED
      2024-03-11 22:03:29,376 INFO [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-InstallSnapshotResponseHandler: Follower snapshot is already at index 0.
      2024-03-11 22:03:29,376 INFO [grpc-default-executor-1]-org.apache.ratis.server.leader.FollowerInfo: om32@group-ACB3E04B8AE9->om39: matchIndex: setUnconditionally -1 -> 0
      2024-03-11 22:03:29,376 INFO [grpc-default-executor-1]-org.apache.ratis.server.leader.FollowerInfo: om32@group-ACB3E04B8AE9->om39: nextIndex: setUnconditionally 0 -> 1
      2024-03-11 22:03:29,376 INFO [grpc-default-executor-1]-org.apache.ratis.server.leader.FollowerInfo: Follower om32@group-ACB3E04B8AE9->om39 acknowledged installing snapshot
      2024-03-11 22:03:29,376 INFO [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-GrpcLogAppender: updateNextIndex 1 for ALREADY_INSTALLED
      2024-03-11 22:03:29,459 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 0
      2024-03-11 22:03:29,461 INFO [grpc-default-executor-1]-org.apache.ratis.server.leader.FollowerInfo: om32@group-ACB3E04B8AE9->om39: setNextIndex nextIndex: updateUnconditionally 4097 -> 0
      2024-03-11 22:03:29,490 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 0
      2024-03-11 22:03:29,513 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 0
      2024-03-11 22:03:29,513 INFO [grpc-default-executor-1]-org.apache.ratis.server.leader.FollowerInfo: om32@group-ACB3E04B8AE9->om39: setNextIndex nextIndex: updateUnconditionally 2048 -> 0
      2024-03-11 22:03:29,528 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 0
      2024-03-11 22:03:29,544 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 0
      2024-03-11 22:03:29,545 INFO [grpc-default-executor-1]-org.apache.ratis.server.leader.FollowerInfo: om32@group-ACB3E04B8AE9->om39: setNextIndex nextIndex: updateUnconditionally 2048 -> 0
      2024-03-11 22:03:30,112 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 128
      2024-03-11 22:03:30,112 INFO [grpc-default-executor-1]-org.apache.ratis.server.leader.FollowerInfo: om32@group-ACB3E04B8AE9->om39: setNextIndex nextIndex: updateUnconditionally 10240 -> 128
      2024-03-11 22:03:30,701 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 6144
      2024-03-11 22:03:30,701 INFO [grpc-default-executor-1]-org.apache.ratis.server.leader.FollowerInfo: om32@group-ACB3E04B8AE9->om39: setNextIndex nextIndex: updateUnconditionally 18560 -> 6144
      2024-03-11 22:03:30,701 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 6144
      2024-03-11 22:03:30,701 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 6144
      2024-03-11 22:03:30,704 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 6144
      2024-03-11 22:03:30,704 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 6144
      2024-03-11 22:03:30,705 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 6144
      2024-03-11 22:03:30,705 WARN [grpc-default-executor-1]-org.apache.ratis.grpc.server.GrpcLogAppender: om32@group-ACB3E04B8AE9->om39-AppendLogResponseHandler: received INCONSISTENCY reply with nextIndex 7168
      2024-03-11 22:03:30,705 INFO [grpc-default-executor-1]-org.apache.ratis.server.leader.FollowerInfo: om32@group-ACB3E04B8AE9->om39: setNextIndex nextIndex: updateUnconditionally 6144 -> 7168
      

      Attachments

        Issue Links

          Activity

            People

              hemantk Hemant Kumar
              hemantk Hemant Kumar
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 1h 20m
                  1h 20m