Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Code Block
{
  "apiKey": 77,
  "type": "response",
  "name": "UpdateVoterResponse",
  "validVersions": "0",
  "flexibleVersions": "0+",
  "fields": [
    { "name": "ErrorCode", "type": "int16", "versions": "0+",
      "about": "The error code, or 0 if there was no error" },
    { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", "versions": "0+", "taggedVersions": "0+", "tag": 0, "fields": [
      { "name": "LeaderId", "type": "int32", "versions": "0+", "default": "-1", "entityType" : "brokerId",
        "about": "The replica id of the current leader or -1 if the leader is unknown" },
      { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "default": "-1",
        "about": "The latest known leader epoch" }
    ]},
    { "name": "NodeEndpoint", "type": "NodeEndpoint", "versions": "0+", "taggedVersions": "0+", "tag": 1,
      "about": "Endpoint for current leader of the topic partition", "fields": [
      { "name": "Host", "type": "string", "versions": "0+", "about": "The node's hostname" },
      { "name": "Port", "type": "int32", "versions": "0+", "about": "The node's port" }
    ]}
  ]
}

UpdateFeatures

TODO: Fill this out

Handling

Request

Response

Fetch

Request

Version 14 adds the field ReplicaUuid to the FetchPartition. This field is populated with the replica generated UUID. If the ReplicaUuid and the ReplicaId fields are populated, the topic partition leader can assume that the replica supports becoming a voter.

Vote

Handling

Since the set of voters can change and not all replicas know the latest voter set, handling of Vote request needs to be relaxed from what was defined and implemented for KIP-595.

KRaft replicas will accept Vote requests from all replicas. Candidate replicas don't need to be in the voters' voter set to receive a vote. This is needed to be able to elect a leader from the new voter set even though the new voter set hasn't been replicated to all of its voters. 

The voter will persist both the candidate ID and UUID in the quorum state so that it only votes for at most one candidate for a given epoch.

The replica will return the following new errors:

  1. INVALID_REQUEST - when the voter ID and UUID doesn't match the local ID and UUID.
  2. UNSUPPORTED_VERSION - when a non-empty candidate UUID is specified but the voter doesn't support kraft.version 1.

Request

Code Block
languagejs
{
git  "apiKey": 1,
  "type": "request",
  "listeners": ["zkBroker", "broker", "controller"],
 diff upstream/trunk clients/src/main/resources/common/message/VoteRequest.json
diff --git a/clients/src/main/resources/common/message/VoteRequest.json b/clients/src/main/resources/common/message/VoteRequest.json
index 35583a790b..ff808cbafe 100644
--- a/clients/src/main/resources/common/message/VoteRequest.json
+++ b/clients/src/main/resources/common/message/VoteRequest.json
@@ -18,11 +18,13 @@
   "type": "request",
   "listeners": ["controller"],
   "name": "FetchRequest"VoteRequest",
-  "validVersions": "0",
+  "validVersions": "0-141",
   "flexibleVersions": "120+",
   "fields": [
     { "name": "ClusterId", "type": "string", "versions": "120+",
       "nullableVersions": "120+", "default": "null"},
+ "taggedVersions   { "name": "VoterId", "type": "12+int32", "tagversions": 0"1+", "ignorable": true,
      "aboutdefault": "-1"The clusterId if known. This is used to validate metadata fetches prior to broker registration." },
    { "name": "ReplicaId", "type": "int32", "versions": "0+", ", "entityType": "brokerId",
+      "about": "The replica IDid of the follower,voter of -1 if this request is from a consumer.receiving the request" },
    ...
    { "name": "Topics", "type": "[]FetchTopicTopicData", "versions": "0+",

       "aboutversions": "The topics to fetch.0+", "fields": [
       { "name": "TopicTopicName", "type": "string", "versions": "0-12+", "entityType": "topicName", "ignorable": true,

@@ -34,14 +36,16 @@
         { "aboutname": "CandidateEpoch"The name of the topic to fetch." },
      { "name": "TopicId", ", "type": "uuidint32", "versions": "130+",
 "ignorable": true,
          "about": "The unique topic ID bumped epoch of the candidate sending the request"},
         { "name": "PartitionsCandidateId", "type": "[]FetchPartitionint32", "versions": "0+", "entityType": "brokerId",
-          "about": "The partitions to fetch.", "fields": [
ID of the voter sending the request"},
+          "about": "The replica id of the voter sending the request"},
+        { "name": "PartitionCandidateUuid", "type": "int32uuid", "versions": "01+",
+          "about": "The partitiondirectory index." },
        id of the voter sending the request" },
+        { "name": "ReplicaUuidVoterUuid", "type": "uuid", "versions": "141+", "nullableVersions": "14+", "default": "null",

+          "about": "The replicadirectory generatedid UUID.of nullthe otherwise." },
        ...
      ]}
    ]}
  ]
}

Response

Version 14 rename LeaderIdAndEpoch to CurrentLeader and adds Endpoint to CurrentLeader.

Code Block
{
  "apiKey": 1,
  "type": "response",
 voter receiving the request to vote, empty uuid if unknown" },
         { "name": "FetchResponseLastOffsetEpoch",
  "validVersionstype": "0-14int32",
  "flexibleVersionsversions": "120+",
  "fields": [
    ...
    { "nameabout": "Responses", "type": "[]FetchableTopicResponse", "versions": "0+",
      "about": "The response topics.", "fields": [
      The epoch of the last record written to the metadata log"},
         { "name": "TopicLastOffset", "type": "stringint64", "versions": "0-12+", "ignorable": true, "entityType": "topicName",

           "about": "The topicoffset nameof the last record written to the metadata log"}
       ]}
     ]}
   ]
 }

Response

Code Block
languagejs
git diff upstream/trunk clients/src/main/resources/common/message/VoteResponse.json
diff --git a/clients/src/main/resources/common/message/VoteResponse.json b/clients/src/main/resources/common/message/VoteResponse.json
index b92d0070c1..21d0aa5312 100644
--- a/clients/src/main/resources/common/message/VoteResponse.json
+++ b/clients/src/main/resources/common/message/VoteResponse.json
@@ -17,7 +17,7 @@
   "apiKey": 52,
   "type": "response",
   "name": "VoteResponse",
-  "validVersions": "0",
+  "validVersions": "0-1",
   "flexibleVersions": "0+",
  ." },
      { "name": "TopicId", "type": "uuid", "versions": "13+", "ignorable": true, "about": "The unique topic ID"},
      { "name": "Partitions", "type": "[]PartitionData", "versions": "0+",
        "about": "The topic partitions.", "fields": [
        ...
        { "name": "CurrentLeader", "type": "CurrentLeader",
          "versions": "12+", "taggedVersions": "12+", "tag": 1, "fields": [
           { "name": "LeaderIdErrorCode", "type": "int32int16", "versions": "120+", "default": "-1", "entityType": "brokerId",
 
@@ -37,9 +37,14 @@
           "about": "The IDlatest of theknown current leader or -1 if the leader is unknown.epoch"},
          { "name": "LeaderEpochVoteGranted", "type": "int32bool", "versions": "12+", "default": "-10+",
            "about": "TheTrue latestif known leader epoch"},
          the vote was granted and false otherwise"}
       ]}
     ]},
+    { "name": "EndPointNodeEndpoints", "type": "Endpoint[]NodeEndpoint", "versions": "141+",
 "taggedVersions": "1+", "tag": 0,
+        "about": "TheEndpoints endpointfor thatall cancurrent-leaders beenumerated used to communicate with the leaderin PartitionData", "fields": [
+            { "name": "HostNodeId", "type": "stringint32", "versions": "141+",
+        "mapKey": true,    "entityType": "brokerId", "about": "The hostname." ID of the associated node"},
      +      { "name": "PortHost", "type": "uint16string", "versions": "141+",
 "about": "The node's hostname" },
+      {   "aboutname": "Port"The, port."type" }
          ]}
    : "int32", "versions": "1+", "about": "The node's port" }
+    ]},
        ...
      ]}
    ]}
  ]
}

Handling

Replica that support becoming voters will send both the replica ID and UUID in the Fetch request. The leader will assume that replicas that report both fields are voters or are able to become voters.

There are a few changes to the leader request handling described in KIP-595. The leaders will track the fetch offset for the replica tuple (ID, UUID). This means that replica are unique identified by their ID and UUID. So their state will be tracking using the ID and UUID.

When removing the leader from the voter set, it will remain the leader for that epoch until the RemoveVoterRecord gets committed. This means that the leader needs to allow replicas (voters and observers) to fetch from the leader even if it is not part of the voter set. This also means that if the leader is not part of the voter set it should not include itself when computing the committed offset (also known as the high-watermark).

FetchSnapshot

Request

Version 1 adds the field ReplicaUuid to PartitionSnapshot. If the ReplicaUuid and the ReplicaId fields are populated, the topic partition leader can assume that the replica supports becoming a voter.

]
 }

UpdateFeatures

TODO: Fill this out

Handling

Request

Response

Fetch

Request

Version 14 adds the field ReplicaUuid to the FetchPartition. This field is populated with the replica generated UUID. If the ReplicaUuid and the ReplicaId fields are populated, the topic partition leader can assume that the replica supports becoming a voter.

Code Block
languagejs
{
  "apiKey": 1,
  "type": "request",
  "listeners": ["zkBroker", "broker", "controller"],
  "name": "FetchRequest",
  "validVersions": "0-14",
  "flexibleVersions": "12+",
  "fields": [
    { "name": "ClusterId", "type": "string", "versions": "12+", "nullableVersions": "12+", "default": "null", "taggedVersions": "12+", "tag": 0, "ignorable": true,
Code Block
languagejs
{
  "apiKey": 59,
  "type": "request",
  "listeners": ["controller"],
  "name": "FetchSnapshotRequest",
  "validVersions": "0-1",
  "flexibleVersions": "0+",
  "fields": [
    { "name": "ClusterId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "taggedVersions": "0+", "tag": 0,
      "about": "The clusterId if known,. thisThis is used to validate metadata fetches prior to broker
   registration." },
    { "name": "ReplicaId", "type": "int32", "versions": "0+", "default": "-1", "entityType": "brokerId",
      "about": "The replica ID of the follower, of -1 if this request is from a consumer." },
    ...
    { "name": "MaxBytesTopics", "type": "int32[]FetchTopic", "versions": "0+", "default": "0x7fffffff",
      "about": "The maximumtopics bytes to fetch from all of the snapshots" },
.", "fields": [
      { "name": "TopicsTopic", "type": "[]TopicSnapshotstring", "versions": "0+"-12", "entityType": "topicName", "ignorable": true,
        "about": "The topics name of the topic to fetch.", "fields": [ },
      { "name": "NameTopicId", "type": "stringuuid", "versions": "013+", "entityTypeignorable": "topicName"true,
        "about": "The name ofunique the topic to fetchID" },
      { "name": "Partitions", "type": "[]PartitionSnapshotFetchPartition", "versions": "0+",
        "about": "The partitions to fetch.", "fields": [
        { "name": "Partition", "type": "int32", "versions": "0+",
          "about": "The partition index." },
         { "name": "ReplicaUuid", "type": "uuid", "versions": "14+", "nullableVersions": "114+", "default": "null",
           "about": "The replica generated UUID. of the followernull otherwise." },
        ...
      ]}
    ]}
  ]
}

Response

Version 14 rename LeaderIdAndEpoch to CurrentLeader and adds Endpoint to CurrentLeader.

Code Block
{
  "nameapiKey": "CurrentLeaderEpoch"1,
  "type": "int32response",
  "versionsname": "0+FetchResponse",
          "about"validVersions": "0-14",
  "flexibleVersions": "The current leader epoch of the partition, -1 for unknown leader epoch" },12+",
  "fields": [
    ...
    { "name": "SnapshotIdResponses", "type": "SnapshotId[]FetchableTopicResponse", "versions": "0+",
          "about": "The snapshot endOffset and epoch to fetchresponse topics.", "fields": [
          { "name": "EndOffsetTopic", "type": "int64string", "versions": "0+" },
          -12", "ignorable": true, "entityType": "topicName",
        "about": "The topic name." },
      { "name": "EpochTopicId", "type": "int32uuid", "versions": "013+" }
        ], "ignorable": true, "about": "The unique topic ID"},
        { "name": "PositionPartitions", "type": "int64[]PartitionData", "versions": "0+",
          ""about": "The byte position within the snapshot to start fetching from" }
      ]}
    ]}
  ]
}

Response

Version 1 renames LeaderIdAndEpoch to CurrentLeader and adds Endpoint to CurrentLeader.

Code Block
languagejs
{
  "apiKey": 59,
  topic partitions.", "fields": [
        ...
        { "name": "CurrentLeader", "type": "responseCurrentLeader",
          "nameversions": "FetchSnapshotResponse12+",
  "validVersionstaggedVersions": "0-112+",
  "flexibleVersionstag": "0+",
 1, "fields": [
           { "name": "ThrottleTimeMsLeaderId", "type": "int32", "versions": "012+", "ignorabledefault": "-1", "entityType": true"brokerId",
            "about": "The durationID inof millisecondsthe forcurrent whichleader theor request-1 wasif throttledthe dueleader to a quota violation, or zero if the request did not violate any quota." },
    is unknown."},
          { "name": "ErrorCodeLeaderEpoch", "type": "int16int32", "versions": "012+", "ignorabledefault": false"-1",
            "about": "The toplatest levelknown response error code." leader epoch"},
          { "name": "TopicsEndPoint", "type": "[]TopicSnapshotEndpoint", "versions": "014+",
            "about": "The topics endpoint that can be used to fetch. communicate with the leader", "fields": [
            { "name": "NameHost", "type": "string", "versions": "014+", "entityType": "topicName",

              "about": "The name of the topic to fetch." },
hostname." },
            { "name": "PartitionsPort", "type": "[]PartitionSnapshotuint16", "versions": "014+",
              "about": "The partitions to fetch.", "fields": [port." }
        { "name": "Index", "type": "int32", "versions": "0+", ]}
          "about": "The partition index." ]},
        { "name": "ErrorCode", "type": "int16", "versions": "0+",...
      ]}
    ]}
  ]
}

Handling

Replica that support becoming voters will send both the replica ID and UUID in the Fetch request. The leader will assume that replicas that report both fields are voters or are able to become voters.

There are a few changes to the leader request handling described in KIP-595. The leaders will track the fetch offset for the replica tuple (ID, UUID). This means that replica are unique identified by their ID and UUID. So their state will be tracking using the ID and UUID.

When removing the leader from the voter set, it will remain the leader for that epoch until the RemoveVoterRecord gets committed. This means that the leader needs to allow replicas (voters and observers) to fetch from the leader even if it is not part of the voter set. This also means that if the leader is not part of the voter set it should not include itself when computing the committed offset (also known as the high-watermark).

FetchSnapshot

Request

Version 1 adds the field ReplicaUuid to PartitionSnapshot. If the ReplicaUuid and the ReplicaId fields are populated, the topic partition leader can assume that the replica supports becoming a voter.

Code Block
languagejs
{
  "apiKey": 59,
  "type": "request",
  "listeners": ["controller"],
  "name": "FetchSnapshotRequest",
  "validVersions": "0-1",
  "flexibleVersions": "0+",
  "fields": [
      "about": "The error code, or 0 if there was no fetch error." },
        { "name": "SnapshotId", "type": "SnapshotId", "versions": "0+",
          "about": "The snapshot endOffset and epoch fetched",
          "fields": [
          { "name": "EndOffset", "type": "int64", "versions": "0+" },
          { "name": "Epoch", "type": "int32", "versions": "0+" }
        ]},
        { "name": "CurrentLeader", "type": "CurrentLeader",
          "versions": "0+", "taggedVersions": "0+", "tag": 0, "fields": [
          { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
            "about": "The ID of the current leader or -1 if the leader is unknown."},
          { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
            "about": "The latest known leader epoch"},
          { "name": "EndPoint", "type": "Endpoint", "versions": "1+",
            "about": "The endpoint that can be used to communicate with the leader", "fields": [
            { "name": "Host", "type": "string", "versions": "1+",
              "about": "The hostname." },
            { "name": "Port", "type": "uint16", "versions": "1+",
              "about": "The port." }
          ]},
        { "name": "SizeClusterId", "type": "int64string", "versions": "0+",
          "about"nullableVersions": "The total size of the snapshot." },
        { "name0+", "default": "Positionnull", "typetaggedVersions": "int640+", "versionstag": "0+",
          "about": "The startingclusterId byteif positionknown, withinthis theis snapshotused includedto invalidate themetadata Bytesfetches field."prior to  },broker
   registration" },
    { "name": "UnalignedRecordsReplicaId", "type": "recordsint32", "versions": "0+",
 "default": "-1", "entityType": "brokerId",
      "about": "SnapshotThe datareplica inID recordsof format which may not be aligned on an offset boundarythe follower" },
    {  ]}
    ]}
  ]
}

Handling

Similar to Fetch, replica that support becoming voters will send both the replica ID and UUID in the FetchSnapshot request. The leader will assume that replicas that report both fields are voters or are able to become voters.

There are a few changes to the leader request handling described in KIP-630. The leaders will track the fetch offset for the replica tuple (ID, UUID). This means that replica are unique identified by their ID and UUID. So their state will be tracking using the ID and UUID.

When removing the leader from the voter set, it will remain the leader for that epoch until the RemoveVoterRecord gets committed. This means that the leader needs to allow replicas (voters and observers) to fetch snapshots from the leader even if it is not part of the voter set.

Vote

Request

Changes:

  1. Candidate Id was moved out of the topic partition maps
  2. Candidate Uuid was added to the PartitionData
  3. VoterId was added to the top level
  4. VoterUuId was added to PartitionData
Code Block
languagejs
{
  "apiKey": 52,
 "name": "MaxBytes", "type": "int32", "versions": "0+", "default": "0x7fffffff",
      "about": "The maximum bytes to fetch from all of the snapshots" },
    { "name": "Topics", "type": "[]TopicSnapshot", "versions": "0+",
      "about": "The topics to fetch", "fields": [
      { "name": "Name", "type": "requeststring",
 "versions": "listeners0+", "entityType": ["controllertopicName"],
        "nameabout": "VoteRequest",
  "validVersionsThe name of the topic to fetch" },
      { "name": "0-1Partitions",
 "type": "flexibleVersions[]PartitionSnapshot", "versions": "0+",
        "about": "The partitions to fetch", "fields": [
        { "name": "ClusterIdPartition", "type": "stringint32", "versions": "0+",
          "nullableVersionsabout": "0+", "default": "null"The partition index" },
        { "name": "CandidateIdReplicaUuid", "type": "int32uuid", "versions": "1+", "entityTypedefault": "brokerIdnull",
           "about": "The IDreplica UUID of the voter sending the requestfollower" }, 
        { "name": "VoterIdCurrentLeaderEpoch", "type": "int32", "versions": "10+", "entityType": "brokerId",

          "about": "The current leader IDepoch of the replicapartition, receiving-1 thefor requestunknown toleader vote.epoch" },
        { "name": "TopicsSnapshotId", "type": "[]TopicDataSnapshotId", "versions": "0+",
          "about": "The snapshot endOffset and epoch to fetch", "fields": [
          { "name": "TopicNameEndOffset", "type": "stringint64", "versions": "0+" },
  "entityType": "topicName",
       { "aboutname": "The topic name." },
      { "nameEpoch", "type": "Partitionsint32", "typeversions": "[]PartitionData",
 0+" }
       "versions": "0+", "fields": [ ]},
        { "name": "PartitionIndexPosition", "type": "int32int64", "versions": "0+",
          "about": "The partition index.byte position within the snapshot to start fetching from" },
      ]}
    ]}
  ]
}

Response

Version 1 renames LeaderIdAndEpoch to CurrentLeader and adds Endpoint to CurrentLeader.

Code Block
languagejs
{
  "nameapiKey": "CandidateEpoch"59,
  "type": "int32response",
  "versionsname": "0+FetchSnapshotResponse",
          "about"validVersions": "0-1",
  "flexibleVersions": "The bumped epoch of the candidate sending the request"}0+",
  "fields": [
     { "name": "CandidateIdThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true,
      "entityTypeabout": "brokerId",
          "about": "The ID of the voter sending the request"},
        The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
    { "name": "CandidateUuidErrorCode", "type": "uuidint16", "versions": "10+",
 "ignorable": false,
        "about": "The candidatetop generatedlevel UUID,response nullerror otherwisecode." },
        { "name": "VoterUuidTopics", "type": "uuid[]TopicSnapshot", "versions": "10+", "nullableVersions": "1+", "default": "null" }
          "about": "The replica generated UUID of the replica receiving the request to vote, null otherwise" }, 
  topics to fetch.", "fields": [
      { "name": "LastOffsetEpochName", "type": "int32string", "versions": "0++", "entityType": "topicName",
          "about": "The epochname of the lasttopic record written to the metadata log"fetch." },
        { "name": "LastOffsetPartitions", "type": "int64[]PartitionSnapshot", "versions": "0+",
          "about": "The offset of the last record written to the metadata log"}
      ]}
partitions to fetch.", "fields": [
      ]}
  ]
}

Response

Version 1 addes the tagged field LeaderEndpoint to PartitionData.

Code Block
languagejs
{
  "apiKeyname": 52"Index",
  "type": "responseint32",
  "nameversions": "VoteResponse0+",
          "validVersionsabout": "0-1",
  "flexibleVersions": "0+",
  "fields": [
The partition index." },
        { "name": "ErrorCode", "type": "int16", "versions": "0+",
          "about": "The top level error code." error code, or 0 if there was no fetch error." },
        { "name": "TopicsSnapshotId", "type": "[]TopicDataSnapshotId",
      "versions": "0+",
 "fields": [
        { "nameabout": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
The snapshot endOffset and epoch fetched",
          "aboutfields": "The[
  topic name." },
      { "name": "PartitionsEndOffset", "type": "[]PartitionDataint64",
        "versions": "0+" },
 "fields": [
        { "name": "PartitionIndexEpoch", "type": "int32", "versions": "0+" }
        ]},
        {  "aboutname": "CurrentLeader"The partition index." }, "type": "CurrentLeader",
        {  "nameversions": "ErrorCode0+", "typetaggedVersions": "int160+", "versionstag": 0, "0+"},
fields": [
          { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
            "about": "The ID of the current leader or -1 if the leader is unknown."},
          { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
            "about": "The latest known leader epoch"},
            { "name": "VoteGrantedEndPoint", "type": "boolEndpoint", "versions": "01+",
            "about": "True if the vote was granted and false otherwise"},
The endpoint that can be used to communicate with the leader", "fields": [
            { "name": "VoterUuidHost", "type": "boolstring", "versions": "1+",
              "about": "The replica generated uuid for the replica casting a vote." } hostname." },
            { "name": "Port", "type": "uint16", "versions": "1+",
        { "name           "about": "LeaderEndPoint", "typeThe port." }
          ]},
        { "name": "EndpointSize", "versionstype": "1+int64", "taggedVersionsversions": "0+1", "tag": 0,
          "about": "The endpointtotal that can be used to communicate withsize of the leadersnapshot.", "fields": [
   },
        { "name": "HostPosition", "type": "stringint64", "versions": "10+",
            "about": "The hostname."  starting byte position within the snapshot included in the Bytes field."   },
          { "name": "PortUnalignedRecords", "type": "uint16records", "versions": "10+",
            "about": "The port." }
        ]Snapshot data in records format which may not be aligned on an offset boundary" }
      ]}
    ]}
  ]
}

Handling

The handling of the Vote request will from that described in KIP-595 in that all replicas are allowed to vote event if they are not voters for the quorum. The voter will persist in the quorum state both the candidate ID and UUID so that it only votes for at most one candidate for a given epoch.

The replica will return the following errors:

...


Handling

Similar to Fetch, replica that support becoming voters will send both the replica ID and UUID in the FetchSnapshot request. The leader will assume that replicas that report both fields are voters or are able to become voters.

There are a few changes to the leader request handling described in KIP-630. The leaders will track the fetch offset for the replica tuple (ID, UUID). This means that replica are unique identified by their ID and UUID. So their state will be tracking using the ID and UUID.

When removing the leader from the voter set, it will remain the leader for that epoch until the RemoveVoterRecord gets committed. This means that the leader needs to allow replicas (voters and observers) to fetch snapshots from the leader even if it is not part of the voter set

...

.

BeginQuorumEpoch

The version of the response is increase and the fields remain unchanged.

...