Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: update on metadata and leader timeout

...

To resolve this issue, we will piggy-back on the "quorum.fetch.timeout.ms" config, such that if the leader did not receive FetchQuorumRecords requests from a majority of the quorum for that amount of time, it would start sending DiscoverBrokerssending Metadata request to random nodes in the cluster to understand the latest quorum. If it couldn't connect to any known voter, the old leader shall reset the connection information and send out DiscoverBrokers. And if the returned response includes a newer epoch leader, this zombie leader would step down and becomes an observer; and if it realized that it is still within the current quorum's voter list, it would start fetching from that leader. Note that the node will remain a leader until it finds that it has been supplanted by another voter.

...

  1. Upon starting up, brokers always try to bootstrap its knowledge of the quorum by first reading the quorum-state file and then scanning forward from AppliedOffset to the end of the log to see if there are any changes to the quorum state. For newly started brokers, the log / file would all be empty so no previous knowledge can be restored.
  2. If after step 1), there's some known quorum state along with a leader / epoch already, the broker would:
    1. Promote itself from observer to voter if it finds out that it's a voter for the epoch.
    2. Start sending FetchQuorumRecords request to the current leader it knows (it may not be the latest epoch's leader actually).
  3. Otherwise, it will try to learn the quorum state by sending DiscoverBrokers to any other brokers inside the cluster via boostrap.servers as the second option of quorum state discovery.
    1. As long as a broker does not know all the current quorum voter's connections, it should continue periodically ask other brokers via DiscoverBrokers.
  4. Send out MetadataRequest to the discovered brokers to find the current metadata partition leader.
    1. As long as a broker does not know the current quorum (including the leader and the voters), it should continue periodically ask other brokersvia Metadata.
  5. If even step 3) 4) cannot find any quorum information – e.g. when there's no other brokers in the cluster, or there's a network partition preventing this broker to talk to others in the cluster – fallback to the third option of quorum state discover by checking if it is among the brokers listed in quorum.voters.
    1. If so, then it will promote to voter state and add its own connection information to the cached quorum state and return that in the DiscoverBrokers responses it answers to other brokers; otherwise stays in observer state.
    2. In either case, it continues to try to send DiscoverBrokers to all other brokers in the cluster via boostrap.servers.
  6. For any voter, after it has learned a majority number of voters in the expected quorum from DiscoverBrokers responses, it will begin a vote.

...