mirror of https://github.com/apache/kafka.git
KAFKA-16527; Implement request handling for updated KRaft RPCs (#16235)
Implement request handling for the updated versions of the KRaft RPCs (Fetch, FetchSnapshot, Vote, BeginQuorumEpoch and EndQuorumEpoch). This doesn't add support for KRaft replicas to send the new version of the KRaft RPCs. That will be implemented in KAFKA-16529. All of the RPCs responses were extended to include the leader's endpoint for the listener of the channel used in the request. EpochState was extended to include the leader's endpoint information but only the FollowerState and LeaderState know the leader id and its endpoint(s). For the Fetch request, the replica directory id was added. The leader now tracks the follower's log end offset using both the replica id and replica directory id. For the FetchSnapshot request, the replica directory id was added. This is not used by the KRaft leader and it is there for consistency with Fetch and for help debugging. For the Vote request, the replica key for both the voter (destination) and the candidate (source) were added. The voter key is checked for consistency. The candidate key is persisted when the vote is granted. For the BeginQuorumEpoch request, all of the leader's endpoints are included. This is needed so that the voters can return the leader's endpoint for all of the supported listeners. For the EndQuorumEpoch request, all of the leader's endpoints are included. This is needed so that the voters can return the leader's endpoint for all of the supported listeners. The successor list has been extended to include the directory id. Receiving voters can use the entire replica key when searching their position in the successor list. Updated the existing test in KafkaRaftClientTest and KafkaRaftClientSnapshotTest to execute using both the old version and new version of the RPCs. Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
This commit is contained in:
parent
5b0e96d785
commit
adee6f0cc1
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.common.errors;
|
||||
|
||||
public class InvalidVoterKeyException extends ApiException {
|
||||
|
||||
private static final long serialVersionUID = 1;
|
||||
|
||||
public InvalidVoterKeyException(String s) {
|
||||
super(s);
|
||||
}
|
||||
|
||||
public InvalidVoterKeyException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
}
|
|
@ -77,6 +77,7 @@ import org.apache.kafka.common.errors.InvalidTopicException;
|
|||
import org.apache.kafka.common.errors.InvalidTxnStateException;
|
||||
import org.apache.kafka.common.errors.InvalidTxnTimeoutException;
|
||||
import org.apache.kafka.common.errors.InvalidUpdateVersionException;
|
||||
import org.apache.kafka.common.errors.InvalidVoterKeyException;
|
||||
import org.apache.kafka.common.errors.KafkaStorageException;
|
||||
import org.apache.kafka.common.errors.LeaderNotAvailableException;
|
||||
import org.apache.kafka.common.errors.ListenerNotFoundException;
|
||||
|
@ -403,7 +404,8 @@ public enum Errors {
|
|||
INVALID_RECORD_STATE(121, "The record state is invalid. The acknowledgement of delivery could not be completed.", InvalidRecordStateException::new),
|
||||
SHARE_SESSION_NOT_FOUND(122, "The share session was not found.", ShareSessionNotFoundException::new),
|
||||
INVALID_SHARE_SESSION_EPOCH(123, "The share session epoch is invalid.", InvalidShareSessionEpochException::new),
|
||||
FENCED_STATE_EPOCH(124, "The share coordinator rejected the request because the share-group state epoch did not match.", FencedStateEpochException::new);
|
||||
FENCED_STATE_EPOCH(124, "The share coordinator rejected the request because the share-group state epoch did not match.", FencedStateEpochException::new),
|
||||
INVALID_VOTER_KEY(125, "The voter key doesn't match the receiving replica's key.", InvalidVoterKeyException::new);
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(Errors.class);
|
||||
|
||||
|
|
|
@ -68,16 +68,12 @@ public class BeginQuorumEpochRequest extends AbstractRequest {
|
|||
return new BeginQuorumEpochRequest(new BeginQuorumEpochRequestData(new ByteBufferAccessor(buffer), version), version);
|
||||
}
|
||||
|
||||
public static BeginQuorumEpochRequestData singletonRequest(TopicPartition topicPartition,
|
||||
int leaderEpoch,
|
||||
int leaderId) {
|
||||
return singletonRequest(topicPartition, null, leaderEpoch, leaderId);
|
||||
}
|
||||
|
||||
public static BeginQuorumEpochRequestData singletonRequest(TopicPartition topicPartition,
|
||||
String clusterId,
|
||||
int leaderEpoch,
|
||||
int leaderId) {
|
||||
public static BeginQuorumEpochRequestData singletonRequest(
|
||||
TopicPartition topicPartition,
|
||||
String clusterId,
|
||||
int leaderEpoch,
|
||||
int leaderId
|
||||
) {
|
||||
return new BeginQuorumEpochRequestData()
|
||||
.setClusterId(clusterId)
|
||||
.setTopics(Collections.singletonList(
|
||||
|
@ -90,5 +86,4 @@ public class BeginQuorumEpochRequest extends AbstractRequest {
|
|||
.setLeaderId(leaderId))))
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,14 +17,12 @@
|
|||
|
||||
package org.apache.kafka.common.requests;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -49,27 +47,6 @@ public class BeginQuorumEpochResponse extends AbstractResponse {
|
|||
this.data = data;
|
||||
}
|
||||
|
||||
public static BeginQuorumEpochResponseData singletonResponse(
|
||||
Errors topLevelError,
|
||||
TopicPartition topicPartition,
|
||||
Errors partitionLevelError,
|
||||
int leaderEpoch,
|
||||
int leaderId
|
||||
) {
|
||||
return new BeginQuorumEpochResponseData()
|
||||
.setErrorCode(topLevelError.code())
|
||||
.setTopics(Collections.singletonList(
|
||||
new BeginQuorumEpochResponseData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(Collections.singletonList(
|
||||
new BeginQuorumEpochResponseData.PartitionData()
|
||||
.setErrorCode(partitionLevelError.code())
|
||||
.setLeaderId(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
)))
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Errors, Integer> errorCounts() {
|
||||
Map<Errors, Integer> errors = new HashMap<>();
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.common.requests;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
|
@ -26,6 +27,7 @@ import org.apache.kafka.common.protocol.Errors;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class EndQuorumEpochRequest extends AbstractRequest {
|
||||
public static class Builder extends AbstractRequest.Builder<EndQuorumEpochRequest> {
|
||||
|
@ -95,4 +97,18 @@ public class EndQuorumEpochRequest extends AbstractRequest {
|
|||
);
|
||||
}
|
||||
|
||||
public static List<EndQuorumEpochRequestData.ReplicaInfo> preferredCandidates(EndQuorumEpochRequestData.PartitionData partition) {
|
||||
if (partition.preferredCandidates().isEmpty()) {
|
||||
return partition
|
||||
.preferredSuccessors()
|
||||
.stream()
|
||||
.map(id -> new EndQuorumEpochRequestData.ReplicaInfo()
|
||||
.setCandidateId(id)
|
||||
.setCandidateDirectoryId(Uuid.ZERO_UUID)
|
||||
)
|
||||
.collect(Collectors.toList());
|
||||
} else {
|
||||
return partition.preferredCandidates();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,14 +17,12 @@
|
|||
|
||||
package org.apache.kafka.common.requests;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -78,27 +76,6 @@ public class EndQuorumEpochResponse extends AbstractResponse {
|
|||
// Not supported by the response schema
|
||||
}
|
||||
|
||||
public static EndQuorumEpochResponseData singletonResponse(
|
||||
Errors topLevelError,
|
||||
TopicPartition topicPartition,
|
||||
Errors partitionLevelError,
|
||||
int leaderEpoch,
|
||||
int leaderId
|
||||
) {
|
||||
return new EndQuorumEpochResponseData()
|
||||
.setErrorCode(topLevelError.code())
|
||||
.setTopics(Collections.singletonList(
|
||||
new EndQuorumEpochResponseData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(Collections.singletonList(
|
||||
new EndQuorumEpochResponseData.PartitionData()
|
||||
.setErrorCode(partitionLevelError.code())
|
||||
.setLeaderId(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
)))
|
||||
);
|
||||
}
|
||||
|
||||
public static EndQuorumEpochResponse parse(ByteBuffer buffer, short version) {
|
||||
return new EndQuorumEpochResponse(new EndQuorumEpochResponseData(new ByteBufferAccessor(buffer), version));
|
||||
}
|
||||
|
|
|
@ -23,11 +23,9 @@ import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
|||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.UnaryOperator;
|
||||
|
||||
public final class FetchSnapshotResponse extends AbstractResponse {
|
||||
private final FetchSnapshotResponseData data;
|
||||
|
@ -81,33 +79,6 @@ public final class FetchSnapshotResponse extends AbstractResponse {
|
|||
return new FetchSnapshotResponseData().setErrorCode(error.code());
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a FetchSnapshotResponseData with a single PartitionSnapshot for the topic partition.
|
||||
*
|
||||
* The partition index will already be populated when calling operator.
|
||||
*
|
||||
* @param topicPartition the topic partition to include
|
||||
* @param operator unary operator responsible for populating all of the appropriate fields
|
||||
* @return the created fetch snapshot response data
|
||||
*/
|
||||
public static FetchSnapshotResponseData singleton(
|
||||
TopicPartition topicPartition,
|
||||
UnaryOperator<FetchSnapshotResponseData.PartitionSnapshot> operator
|
||||
) {
|
||||
FetchSnapshotResponseData.PartitionSnapshot partitionSnapshot = operator.apply(
|
||||
new FetchSnapshotResponseData.PartitionSnapshot().setIndex(topicPartition.partition())
|
||||
);
|
||||
|
||||
return new FetchSnapshotResponseData()
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new FetchSnapshotResponseData.TopicSnapshot()
|
||||
.setName(topicPartition.topic())
|
||||
.setPartitions(Collections.singletonList(partitionSnapshot))
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds the PartitionSnapshot for a given topic partition.
|
||||
*
|
||||
|
|
|
@ -69,19 +69,6 @@ public class VoteRequest extends AbstractRequest {
|
|||
return new VoteRequest(new VoteRequestData(new ByteBufferAccessor(buffer), version), version);
|
||||
}
|
||||
|
||||
public static VoteRequestData singletonRequest(TopicPartition topicPartition,
|
||||
int candidateEpoch,
|
||||
int candidateId,
|
||||
int lastEpoch,
|
||||
long lastEpochEndOffset) {
|
||||
return singletonRequest(topicPartition,
|
||||
null,
|
||||
candidateEpoch,
|
||||
candidateId,
|
||||
lastEpoch,
|
||||
lastEpochEndOffset);
|
||||
}
|
||||
|
||||
public static VoteRequestData singletonRequest(TopicPartition topicPartition,
|
||||
String clusterId,
|
||||
int candidateEpoch,
|
||||
|
@ -102,5 +89,4 @@ public class VoteRequest extends AbstractRequest {
|
|||
.setLastOffset(lastEpochEndOffset))
|
||||
)));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,14 +17,12 @@
|
|||
|
||||
package org.apache.kafka.common.requests;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.message.VoteResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -49,25 +47,6 @@ public class VoteResponse extends AbstractResponse {
|
|||
this.data = data;
|
||||
}
|
||||
|
||||
public static VoteResponseData singletonResponse(Errors topLevelError,
|
||||
TopicPartition topicPartition,
|
||||
Errors partitionLevelError,
|
||||
int leaderEpoch,
|
||||
int leaderId,
|
||||
boolean voteGranted) {
|
||||
return new VoteResponseData()
|
||||
.setErrorCode(topLevelError.code())
|
||||
.setTopics(Collections.singletonList(
|
||||
new VoteResponseData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(Collections.singletonList(
|
||||
new VoteResponseData.PartitionData()
|
||||
.setErrorCode(partitionLevelError.code())
|
||||
.setLeaderId(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
.setVoteGranted(voteGranted)))));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<Errors, Integer> errorCounts() {
|
||||
Map<Errors, Integer> errors = new HashMap<>();
|
||||
|
|
|
@ -18,24 +18,38 @@
|
|||
"type": "request",
|
||||
"listeners": ["controller"],
|
||||
"name": "BeginQuorumEpochRequest",
|
||||
"validVersions": "0",
|
||||
"flexibleVersions": "none",
|
||||
// Version 1 adds flexible versions, voter key and leader endpoints (KIP-853)
|
||||
"validVersions": "0-1",
|
||||
"flexibleVersions": "1+",
|
||||
"fields": [
|
||||
{ "name": "ClusterId", "type": "string", "versions": "0+",
|
||||
"nullableVersions": "0+", "default": "null"},
|
||||
{ "name": "VoterId", "type": "int32", "versions": "1+", "entityType": "brokerId", "ignorable": true,
|
||||
"about": "The voter ID of the receiving replica" },
|
||||
{ "name": "Topics", "type": "[]TopicData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The ID of the newly elected leader"},
|
||||
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The epoch of the newly elected leader"}
|
||||
]}
|
||||
]}
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name" },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index" },
|
||||
{ "name": "VoterDirectoryId", "type": "uuid", "versions": "1+", "ignorable": true,
|
||||
"about": "The directory id of the receiving replica" },
|
||||
{ "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The ID of the newly elected leader"},
|
||||
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The epoch of the newly elected leader"}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{ "name": "LeaderEndpoints", "type": "[]LeaderEndpoint", "versions": "1+", "ignorable": true,
|
||||
"about": "Endpoints for the leader", "fields": [
|
||||
{ "name": "Name", "type": "string", "versions": "1+", "mapKey": true, "about": "The name of the endpoint" },
|
||||
{ "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" },
|
||||
{ "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" }
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
|
|
|
@ -17,25 +17,36 @@
|
|||
"apiKey": 53,
|
||||
"type": "response",
|
||||
"name": "BeginQuorumEpochResponse",
|
||||
"validVersions": "0",
|
||||
"flexibleVersions": "none",
|
||||
// Version 1 adds flexible versions and leader endpoint (KIP-853)
|
||||
"validVersions": "0-1",
|
||||
"flexibleVersions": "1+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The top level error code."},
|
||||
{ "name": "Topics", "type": "[]TopicData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+"},
|
||||
{ "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": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+"},
|
||||
{ "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": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "1+", "taggedVersions": "1+", "tag": 0,
|
||||
"about": "Endpoints for all leaders enumerated in PartitionData", "fields": [
|
||||
{ "name": "NodeId", "type": "int32", "versions": "1+",
|
||||
"mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node" },
|
||||
{ "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" },
|
||||
{ "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" }
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
|
|
|
@ -18,26 +18,43 @@
|
|||
"type": "request",
|
||||
"listeners": ["controller"],
|
||||
"name": "EndQuorumEpochRequest",
|
||||
"validVersions": "0",
|
||||
"flexibleVersions": "none",
|
||||
// Version 1 adds flexible versions, replaces preferred successors with preferred candidates
|
||||
// and adds leader endpoints (KIP-853)
|
||||
"validVersions": "0-1",
|
||||
"flexibleVersions": "1+",
|
||||
"fields": [
|
||||
{ "name": "ClusterId", "type": "string", "versions": "0+",
|
||||
"nullableVersions": "0+", "default": "null"},
|
||||
{ "name": "Topics", "type": "[]TopicData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The current leader ID that is resigning"},
|
||||
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The current epoch"},
|
||||
{ "name": "PreferredSuccessors", "type": "[]int32", "versions": "0+",
|
||||
"about": "A sorted list of preferred successors to start the election"}
|
||||
]}
|
||||
]}
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The current leader ID that is resigning"},
|
||||
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The current epoch"},
|
||||
{ "name": "PreferredSuccessors", "type": "[]int32", "versions": "0", "ignorable": true,
|
||||
"about": "A sorted list of preferred successors to start the election" },
|
||||
{ "name": "PreferredCandidates", "type": "[]ReplicaInfo", "versions": "1+", "ignorable": true,
|
||||
"about": "A sorted list of preferred candidates to start the election", "fields": [
|
||||
{ "name": "CandidateId", "type": "int32", "versions": "1+", "entityType": "brokerId" },
|
||||
{ "name": "CandidateDirectoryId", "type": "uuid", "versions": "1+" }
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{ "name": "LeaderEndpoints", "type": "[]LeaderEndpoint", "versions": "1+", "ignorable": true,
|
||||
"about": "Endpoints for the leader", "fields": [
|
||||
{ "name": "Name", "type": "string", "versions": "1+", "mapKey": true, "about": "The name of the endpoint" },
|
||||
{ "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" },
|
||||
{ "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" }
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
|
|
|
@ -17,25 +17,36 @@
|
|||
"apiKey": 54,
|
||||
"type": "response",
|
||||
"name": "EndQuorumEpochResponse",
|
||||
"validVersions": "0",
|
||||
"flexibleVersions": "none",
|
||||
// Version 1 adds flexible versions and leader endpoint (KIP-853)
|
||||
"validVersions": "0-1",
|
||||
"flexibleVersions": "1+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The top level error code."},
|
||||
{ "name": "Topics", "type": "[]TopicData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+"},
|
||||
{ "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": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+"},
|
||||
{ "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": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "1+", "taggedVersions": "1+", "tag": 0,
|
||||
"about": "Endpoints for all leaders enumerated in PartitionData", "fields": [
|
||||
{ "name": "NodeId", "type": "int32", "versions": "1+",
|
||||
"mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node" },
|
||||
{ "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" },
|
||||
{ "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" }
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
|
|
|
@ -55,7 +55,9 @@
|
|||
// deprecate the old ReplicaId field and set its default value to -1. (KIP-903)
|
||||
//
|
||||
// Version 16 is the same as version 15 (KIP-951).
|
||||
"validVersions": "0-16",
|
||||
//
|
||||
// Version 17 adds directory id support from KIP-853
|
||||
"validVersions": "0-17",
|
||||
"deprecatedVersions": "0-3",
|
||||
"flexibleVersions": "12+",
|
||||
"fields": [
|
||||
|
@ -100,7 +102,9 @@
|
|||
{ "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true,
|
||||
"about": "The earliest available offset of the follower replica. The field is only used when the request is sent by the follower."},
|
||||
{ "name": "PartitionMaxBytes", "type": "int32", "versions": "0+",
|
||||
"about": "The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored." }
|
||||
"about": "The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored." },
|
||||
{ "name": "ReplicaDirectoryId", "type": "uuid", "versions": "17+", "taggedVersions": "17+", "tag": 0,
|
||||
"about": "The directory id of the follower fetching" }
|
||||
]}
|
||||
]},
|
||||
{ "name": "ForgottenTopicsData", "type": "[]ForgottenTopic", "versions": "7+", "ignorable": false,
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
//
|
||||
// Version 1 adds throttle time.
|
||||
//
|
||||
// Version 2 and 3 are the same as version 1.
|
||||
// Version 2 and 3 are the same as version 1.
|
||||
//
|
||||
// Version 4 adds features for transactional consumption.
|
||||
//
|
||||
|
@ -47,7 +47,9 @@
|
|||
// Version 15 is the same as version 14 (KIP-903).
|
||||
//
|
||||
// Version 16 adds the 'NodeEndpoints' field (KIP-951).
|
||||
"validVersions": "0-16",
|
||||
//
|
||||
// Version 17 no changes to the response (KIP-853).
|
||||
"validVersions": "0-17",
|
||||
"flexibleVersions": "12+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
|
||||
|
|
|
@ -18,7 +18,8 @@
|
|||
"type": "request",
|
||||
"listeners": ["controller"],
|
||||
"name": "FetchSnapshotRequest",
|
||||
"validVersions": "0",
|
||||
// Version 1 adds replica directory id (KIP-853)
|
||||
"validVersions": "0-1",
|
||||
"flexibleVersions": "0+",
|
||||
"fields": [
|
||||
{ "name": "ClusterId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "taggedVersions": "0+", "tag": 0,
|
||||
|
@ -29,23 +30,27 @@
|
|||
"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": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The name of the topic to fetch" },
|
||||
{ "name": "Partitions", "type": "[]PartitionSnapshot", "versions": "0+",
|
||||
"about": "The partitions to fetch", "fields": [
|
||||
{ "name": "Partition", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index" },
|
||||
{ "name": "CurrentLeaderEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The current leader epoch of the partition, -1 for unknown leader epoch" },
|
||||
{ "name": "SnapshotId", "type": "SnapshotId", "versions": "0+",
|
||||
"about": "The snapshot endOffset and epoch to fetch",
|
||||
"fields": [
|
||||
{ "name": "EndOffset", "type": "int64", "versions": "0+" },
|
||||
{ "name": "Epoch", "type": "int32", "versions": "0+" }
|
||||
]},
|
||||
{ "name": "Position", "type": "int64", "versions": "0+",
|
||||
"about": "The byte position within the snapshot to start fetching from" }
|
||||
]}
|
||||
]}
|
||||
{ "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The name of the topic to fetch" },
|
||||
{ "name": "Partitions", "type": "[]PartitionSnapshot", "versions": "0+",
|
||||
"about": "The partitions to fetch", "fields": [
|
||||
{ "name": "Partition", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index" },
|
||||
{ "name": "CurrentLeaderEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The current leader epoch of the partition, -1 for unknown leader epoch" },
|
||||
{ "name": "SnapshotId", "type": "SnapshotId", "versions": "0+",
|
||||
"about": "The snapshot endOffset and epoch to fetch", "fields": [
|
||||
{ "name": "EndOffset", "type": "int64", "versions": "0+" },
|
||||
{ "name": "Epoch", "type": "int32", "versions": "0+" }
|
||||
]
|
||||
},
|
||||
{ "name": "Position", "type": "int64", "versions": "0+",
|
||||
"about": "The byte position within the snapshot to start fetching from" },
|
||||
{ "name": "ReplicaDirectoryId", "type": "uuid", "versions": "1+", "taggedVersions": "1+", "tag": 0,
|
||||
"about": "The directory id of the follower fetching" }
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
|
|
|
@ -17,7 +17,8 @@
|
|||
"apiKey": 59,
|
||||
"type": "response",
|
||||
"name": "FetchSnapshotResponse",
|
||||
"validVersions": "0",
|
||||
"validVersions": "0-1",
|
||||
// Version 1 adds leader endpoint (KIP-853)
|
||||
"flexibleVersions": "0+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true,
|
||||
|
@ -26,34 +27,45 @@
|
|||
"about": "The top level response error code." },
|
||||
{ "name": "Topics", "type": "[]TopicSnapshot", "versions": "0+",
|
||||
"about": "The topics to fetch.", "fields": [
|
||||
{ "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The name of the topic to fetch." },
|
||||
{ "name": "Partitions", "type": "[]PartitionSnapshot", "versions": "0+",
|
||||
"about": "The partitions to fetch.", "fields": [
|
||||
{ "name": "Index", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"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": "LeaderIdAndEpoch",
|
||||
"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": "Size", "type": "int64", "versions": "0+",
|
||||
"about": "The total size of the snapshot." },
|
||||
{ "name": "Position", "type": "int64", "versions": "0+",
|
||||
"about": "The starting byte position within the snapshot included in the Bytes field." },
|
||||
{ "name": "UnalignedRecords", "type": "records", "versions": "0+",
|
||||
"about": "Snapshot data in records format which may not be aligned on an offset boundary" }
|
||||
]}
|
||||
]}
|
||||
{ "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The name of the topic to fetch." },
|
||||
{ "name": "Partitions", "type": "[]PartitionSnapshot", "versions": "0+",
|
||||
"about": "The partitions to fetch.", "fields": [
|
||||
{ "name": "Index", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"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": "LeaderIdAndEpoch",
|
||||
"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": "Size", "type": "int64", "versions": "0+",
|
||||
"about": "The total size of the snapshot." },
|
||||
{ "name": "Position", "type": "int64", "versions": "0+",
|
||||
"about": "The starting byte position within the snapshot included in the Bytes field." },
|
||||
{ "name": "UnalignedRecords", "type": "records", "versions": "0+",
|
||||
"about": "Snapshot data in records format which may not be aligned on an offset boundary" }
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{ "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "1+", "taggedVersions": "1+", "tag": 0,
|
||||
"about": "Endpoints for all current-leaders enumerated in PartitionSnapshot", "fields": [
|
||||
{ "name": "NodeId", "type": "int32", "versions": "1+",
|
||||
"mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node" },
|
||||
{ "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" },
|
||||
{ "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" }
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
|
|
|
@ -18,30 +18,37 @@
|
|||
"type": "request",
|
||||
"listeners": ["controller"],
|
||||
"name": "VoteRequest",
|
||||
"validVersions": "0",
|
||||
// Version 1 adds voter key and candidate directory id (KIP-853)
|
||||
"validVersions": "0-1",
|
||||
"flexibleVersions": "0+",
|
||||
"fields": [
|
||||
{ "name": "ClusterId", "type": "string", "versions": "0+",
|
||||
"nullableVersions": "0+", "default": "null"},
|
||||
{ "name": "VoterId", "type": "int32", "versions": "1+", "ignorable": true, "default": "-1", "entityType": "brokerId",
|
||||
"about": "The replica id of the voter receiving the request" },
|
||||
{ "name": "Topics", "type": "[]TopicData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "CandidateEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The bumped epoch of the candidate sending the request"},
|
||||
{ "name": "CandidateId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The ID of the voter sending the request"},
|
||||
{ "name": "LastOffsetEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The epoch of the last record written to the metadata log"},
|
||||
{ "name": "LastOffset", "type": "int64", "versions": "0+",
|
||||
"about": "The offset of the last record written to the metadata log"}
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "CandidateEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The bumped epoch of the candidate sending the request"},
|
||||
{ "name": "CandidateId", "type": "int32", "versions": "0+", "entityType": "brokerId",
|
||||
"about": "The replica id of the voter sending the request"},
|
||||
{ "name": "CandidateDirectoryId", "type": "uuid", "versions": "1+", "ignorable": true,
|
||||
"about": "The directory id of the voter sending the request" },
|
||||
{ "name": "VoterDirectoryId", "type": "uuid", "versions": "1+", "ignorable": true,
|
||||
"about": "The ID of the voter sending the request"},
|
||||
{ "name": "LastOffsetEpoch", "type": "int32", "versions": "0+",
|
||||
"about": "The epoch of the last record written to the metadata log"},
|
||||
{ "name": "LastOffset", "type": "int64", "versions": "0+",
|
||||
"about": "The offset of the last record written to the metadata log"}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
|
|
|
@ -17,29 +17,38 @@
|
|||
"apiKey": 52,
|
||||
"type": "response",
|
||||
"name": "VoteResponse",
|
||||
"validVersions": "0",
|
||||
// Version 1 adds leader endpoint (KIP-853)
|
||||
"validVersions": "0-1",
|
||||
"flexibleVersions": "0+",
|
||||
"fields": [
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The top level error code."},
|
||||
{ "name": "Topics", "type": "[]TopicData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+"},
|
||||
{ "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": "VoteGranted", "type": "bool", "versions": "0+",
|
||||
"about": "True if the vote was granted and false otherwise"}
|
||||
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
|
||||
"about": "The topic name." },
|
||||
{ "name": "Partitions", "type": "[]PartitionData",
|
||||
"versions": "0+", "fields": [
|
||||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+"},
|
||||
{ "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": "VoteGranted", "type": "bool", "versions": "0+",
|
||||
"about": "True if the vote was granted and false otherwise"}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{ "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "1+", "taggedVersions": "1+", "tag": 0,
|
||||
"about": "Endpoints for all current-leaders enumerated in PartitionData", "fields": [
|
||||
{ "name": "NodeId", "type": "int32", "versions": "1+",
|
||||
"mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node"},
|
||||
{ "name": "Host", "type": "string", "versions": "1+", "about": "The node's hostname" },
|
||||
{ "name": "Port", "type": "uint16", "versions": "1+", "about": "The node's port" }
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
|
|
|
@ -463,7 +463,7 @@ class RequestChannel(val queueSize: Int,
|
|||
}
|
||||
}
|
||||
|
||||
/** Get the next request or block until specified time has elapsed
|
||||
/** Get the next request or block until specified time has elapsed
|
||||
* Check the callback queue and execute first if present since these
|
||||
* requests have already waited in line. */
|
||||
def receiveRequest(timeout: Long): RequestChannel.BaseRequest = {
|
||||
|
|
|
@ -39,11 +39,12 @@ import org.apache.kafka.common.internals.Topic
|
|||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector}
|
||||
import org.apache.kafka.common.protocol.ApiMessage
|
||||
import org.apache.kafka.common.requests.RequestContext
|
||||
import org.apache.kafka.common.requests.RequestHeader
|
||||
import org.apache.kafka.common.security.JaasContext
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.{LogContext, Time, Utils}
|
||||
import org.apache.kafka.raft.{FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, QuorumConfig, RaftClient, ReplicatedLog}
|
||||
import org.apache.kafka.raft.{Endpoints, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, QuorumConfig, RaftClient, ReplicatedLog}
|
||||
import org.apache.kafka.server.ProcessRole
|
||||
import org.apache.kafka.server.common.serialization.RecordSerde
|
||||
import org.apache.kafka.server.util.KafkaScheduler
|
||||
|
@ -122,6 +123,7 @@ object KafkaRaftManager {
|
|||
|
||||
trait RaftManager[T] {
|
||||
def handleRequest(
|
||||
context: RequestContext,
|
||||
header: RequestHeader,
|
||||
request: ApiMessage,
|
||||
createdTimeMs: Long
|
||||
|
@ -152,6 +154,7 @@ class KafkaRaftManager[T](
|
|||
threadNamePrefixOpt: Option[String],
|
||||
val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]],
|
||||
bootstrapServers: JCollection[InetSocketAddress],
|
||||
controllerListeners: Endpoints,
|
||||
fatalFaultHandler: FaultHandler
|
||||
) extends RaftManager[T] with Logging {
|
||||
|
||||
|
@ -214,15 +217,16 @@ class KafkaRaftManager[T](
|
|||
}
|
||||
|
||||
override def handleRequest(
|
||||
context: RequestContext,
|
||||
header: RequestHeader,
|
||||
request: ApiMessage,
|
||||
createdTimeMs: Long
|
||||
): CompletableFuture[ApiMessage] = {
|
||||
clientDriver.handleRequest(header, request, createdTimeMs)
|
||||
clientDriver.handleRequest(context, header, request, createdTimeMs)
|
||||
}
|
||||
|
||||
private def buildRaftClient(): KafkaRaftClient[T] = {
|
||||
val client = new KafkaRaftClient(
|
||||
new KafkaRaftClient(
|
||||
OptionalInt.of(config.nodeId),
|
||||
metadataLogDirUuid,
|
||||
recordSerde,
|
||||
|
@ -233,9 +237,9 @@ class KafkaRaftManager[T](
|
|||
logContext,
|
||||
clusterId,
|
||||
bootstrapServers,
|
||||
controllerListeners,
|
||||
raftConfig
|
||||
)
|
||||
client
|
||||
}
|
||||
|
||||
private def buildNetworkChannel(): KafkaNetworkChannel = {
|
||||
|
|
|
@ -679,7 +679,7 @@ class ControllerApis(
|
|||
private def handleRaftRequest(request: RequestChannel.Request,
|
||||
buildResponse: ApiMessage => AbstractResponse): CompletableFuture[Unit] = {
|
||||
val requestBody = request.body[AbstractRequest]
|
||||
val future = raftManager.handleRequest(request.header, requestBody.data, time.milliseconds())
|
||||
val future = raftManager.handleRequest(request.context, request.header, requestBody.data, time.milliseconds())
|
||||
future.handle[Unit] { (responseData, exception) =>
|
||||
val response = if (exception != null) {
|
||||
requestBody.getErrorResponse(exception)
|
||||
|
@ -926,7 +926,7 @@ class ControllerApis(
|
|||
CreateDelegationTokenResponse.prepareResponse(request.context.requestVersion, requestThrottleMs,
|
||||
Errors.DELEGATION_TOKEN_REQUEST_NOT_ALLOWED, owner, requester))
|
||||
CompletableFuture.completedFuture[Unit](())
|
||||
} else if (!owner.equals(requester) &&
|
||||
} else if (!owner.equals(requester) &&
|
||||
!authHelper.authorize(request.context, CREATE_TOKENS, USER, owner.toString)) {
|
||||
// Requester is always allowed to create token for self
|
||||
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
|
||||
|
|
|
@ -187,9 +187,10 @@ class ControllerServer(
|
|||
credentialProvider,
|
||||
apiVersionManager)
|
||||
|
||||
val listenerInfo = ListenerInfo.create(config.controllerListeners.map(_.toJava).asJava).
|
||||
withWildcardHostnamesResolved().
|
||||
withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name)))
|
||||
val listenerInfo = ListenerInfo
|
||||
.create(config.effectiveAdvertisedControllerListeners.map(_.toJava).asJava)
|
||||
.withWildcardHostnamesResolved()
|
||||
.withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name)))
|
||||
socketServerFirstBoundPortFuture.complete(listenerInfo.firstListener().port())
|
||||
|
||||
val endpointReadyFutures = {
|
||||
|
@ -203,7 +204,7 @@ class ControllerServer(
|
|||
config.earlyStartListeners.map(_.value()).asJava))
|
||||
}
|
||||
|
||||
sharedServer.startForController()
|
||||
sharedServer.startForController(listenerInfo)
|
||||
|
||||
createTopicPolicy = Option(config.
|
||||
getConfiguredInstance(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[CreateTopicPolicy]))
|
||||
|
|
|
@ -788,13 +788,24 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
}
|
||||
}
|
||||
|
||||
// Use advertised listeners if defined, fallback to listeners otherwise
|
||||
def effectiveAdvertisedControllerListeners: Seq[EndPoint] = {
|
||||
// Only expose controller listeners
|
||||
advertisedListeners.filter(l => controllerListenerNames.contains(l.listenerName.value()))
|
||||
}
|
||||
|
||||
def effectiveAdvertisedListeners: Seq[EndPoint] = {
|
||||
// Only expose broker listeners
|
||||
advertisedListeners.filterNot(l => controllerListenerNames.contains(l.listenerName.value()))
|
||||
}
|
||||
|
||||
// Use advertised listeners if defined, fallback to listeners otherwise
|
||||
private def advertisedListeners: Seq[EndPoint] = {
|
||||
val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
|
||||
if (advertisedListenersProp != null)
|
||||
if (advertisedListenersProp != null) {
|
||||
CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false)
|
||||
else
|
||||
listeners.filterNot(l => controllerListenerNames.contains(l.listenerName.value()))
|
||||
} else {
|
||||
listeners
|
||||
}
|
||||
}
|
||||
|
||||
private def getInterBrokerListenerNameAndSecurityProtocol: (ListenerName, SecurityProtocol) = {
|
||||
|
@ -927,7 +938,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
s"${SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG} is not supported in KRaft mode.")
|
||||
}
|
||||
def validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker(): Unit = {
|
||||
require(!advertisedListenerNames.exists(aln => controllerListenerNames.contains(aln.value())),
|
||||
require(advertisedListenerNames.forall(aln => !controllerListenerNames.contains(aln.value())),
|
||||
s"The advertised.listeners config must not contain KRaft controller listeners from ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} when ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the broker role because Kafka clients that send requests via advertised listeners do not send requests to KRaft controllers -- they only send requests to KRaft brokers.")
|
||||
}
|
||||
def validateControllerQuorumVotersMustContainNodeIdForKRaftController(): Unit = {
|
||||
|
@ -979,11 +990,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
|||
// KRaft controller-only
|
||||
validateNonEmptyQuorumVotersForKRaft()
|
||||
validateControlPlaneListenerEmptyForKRaft()
|
||||
// advertised listeners must be empty when only the controller is configured
|
||||
require(
|
||||
getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) == null,
|
||||
s"The ${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG} config must be empty when ${KRaftConfigs.PROCESS_ROLES_CONFIG}=controller"
|
||||
)
|
||||
// listeners should only contain listeners also enumerated in the controller listener
|
||||
require(
|
||||
effectiveAdvertisedListeners.isEmpty,
|
||||
|
|
|
@ -93,6 +93,8 @@ class KafkaRaftServer(
|
|||
|
||||
override def startup(): Unit = {
|
||||
Mx4jLoader.maybeLoad()
|
||||
// Controller component must be started before the broker component so that
|
||||
// the controller endpoints are passed to the KRaft manager
|
||||
controller.foreach(_.startup())
|
||||
broker.foreach(_.startup())
|
||||
AppInfoParser.registerAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics, time.milliseconds())
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationF
|
|||
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble}
|
||||
import org.apache.kafka.metadata.{BrokerState, MetadataRecordSerde, VersionRange}
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
import org.apache.kafka.raft.Endpoints
|
||||
import org.apache.kafka.security.CredentialProvider
|
||||
import org.apache.kafka.server.NodeToControllerChannelManager
|
||||
import org.apache.kafka.server.authorizer.Authorizer
|
||||
|
@ -440,6 +441,9 @@ class KafkaServer(
|
|||
threadNamePrefix,
|
||||
CompletableFuture.completedFuture(quorumVoters),
|
||||
QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers),
|
||||
// Endpoint information is only needed for KRaft controllers (voters). ZK brokers
|
||||
// (observers) can never be KRaft controllers
|
||||
Endpoints.empty(),
|
||||
fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown())
|
||||
)
|
||||
quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config)
|
||||
|
|
|
@ -22,15 +22,18 @@ import kafka.server.Server.MetricsPrefix
|
|||
import kafka.server.metadata.BrokerServerMetrics
|
||||
import kafka.utils.{CoreUtils, Logging}
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time}
|
||||
import org.apache.kafka.controller.metrics.ControllerMetadataMetrics
|
||||
import org.apache.kafka.image.MetadataProvenance
|
||||
import org.apache.kafka.image.loader.MetadataLoader
|
||||
import org.apache.kafka.image.loader.metrics.MetadataLoaderMetrics
|
||||
import org.apache.kafka.image.publisher.{SnapshotEmitter, SnapshotGenerator}
|
||||
import org.apache.kafka.image.publisher.metrics.SnapshotEmitterMetrics
|
||||
import org.apache.kafka.image.publisher.{SnapshotEmitter, SnapshotGenerator}
|
||||
import org.apache.kafka.metadata.ListenerInfo
|
||||
import org.apache.kafka.metadata.MetadataRecordSerde
|
||||
import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble
|
||||
import org.apache.kafka.raft.Endpoints
|
||||
import org.apache.kafka.server.ProcessRole
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion
|
||||
import org.apache.kafka.server.fault.{FaultHandler, LoggingFaultHandler, ProcessTerminatingFaultHandler}
|
||||
|
@ -43,7 +46,7 @@ import java.util.concurrent.atomic.AtomicReference
|
|||
import java.util.concurrent.{CompletableFuture, TimeUnit}
|
||||
import java.util.{Collection => JCollection}
|
||||
import java.util.{Map => JMap}
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
/**
|
||||
* Creates a fault handler.
|
||||
|
@ -128,7 +131,7 @@ class SharedServer(
|
|||
*/
|
||||
def startForBroker(): Unit = synchronized {
|
||||
if (!isUsed()) {
|
||||
start()
|
||||
start(Endpoints.empty())
|
||||
}
|
||||
usedByBroker = true
|
||||
}
|
||||
|
@ -136,9 +139,22 @@ class SharedServer(
|
|||
/**
|
||||
* The start function called by the controller.
|
||||
*/
|
||||
def startForController(): Unit = synchronized {
|
||||
def startForController(listenerInfo: ListenerInfo): Unit = synchronized {
|
||||
if (!isUsed()) {
|
||||
start()
|
||||
val endpoints = Endpoints.fromInetSocketAddresses(
|
||||
listenerInfo
|
||||
.listeners()
|
||||
.asScala
|
||||
.map { case (listenerName, endpoint) =>
|
||||
(
|
||||
ListenerName.normalised(listenerName),
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
)
|
||||
}
|
||||
.toMap
|
||||
.asJava
|
||||
)
|
||||
start(endpoints)
|
||||
}
|
||||
usedByController = true
|
||||
}
|
||||
|
@ -237,7 +253,7 @@ class SharedServer(
|
|||
// Note: snapshot generation does not need to be disabled for a publishing fault.
|
||||
})
|
||||
|
||||
private def start(): Unit = synchronized {
|
||||
private def start(listenerEndpoints: Endpoints): Unit = synchronized {
|
||||
if (started) {
|
||||
debug("SharedServer has already been started.")
|
||||
} else {
|
||||
|
@ -256,6 +272,7 @@ class SharedServer(
|
|||
if (sharedServerConfig.processRoles.contains(ProcessRole.ControllerRole)) {
|
||||
controllerServerMetrics = new ControllerMetadataMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry()))
|
||||
}
|
||||
|
||||
val _raftManager = new KafkaRaftManager[ApiMessageAndVersion](
|
||||
clusterId,
|
||||
sharedServerConfig,
|
||||
|
@ -268,6 +285,7 @@ class SharedServer(
|
|||
Some(s"kafka-${sharedServerConfig.nodeId}-raft"), // No dash expected at the end
|
||||
controllerQuorumVotersFuture,
|
||||
bootstrapServers,
|
||||
listenerEndpoints,
|
||||
raftManagerFaultHandler
|
||||
)
|
||||
raftManager = _raftManager
|
||||
|
|
|
@ -94,6 +94,7 @@ class TestRaftRequestHandler(
|
|||
val requestBody = request.body[AbstractRequest]
|
||||
|
||||
val future = raftManager.handleRequest(
|
||||
request.context,
|
||||
request.header,
|
||||
requestBody.data,
|
||||
time.milliseconds()
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package kafka.tools
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.atomic.{AtomicInteger, AtomicLong}
|
||||
import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit}
|
||||
import joptsimple.{OptionException, OptionSpec}
|
||||
|
@ -35,7 +36,7 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok
|
|||
import org.apache.kafka.common.utils.{Time, Utils}
|
||||
import org.apache.kafka.common.{TopicPartition, Uuid, protocol}
|
||||
import org.apache.kafka.raft.errors.NotLeaderException
|
||||
import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient, QuorumConfig}
|
||||
import org.apache.kafka.raft.{Batch, BatchReader, Endpoints, LeaderAndEpoch, RaftClient, QuorumConfig}
|
||||
import org.apache.kafka.security.CredentialProvider
|
||||
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
|
||||
import org.apache.kafka.server.common.serialization.RecordSerde
|
||||
|
@ -84,6 +85,15 @@ class TestRaftServer(
|
|||
() => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION))
|
||||
socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager)
|
||||
|
||||
val endpoints = Endpoints.fromInetSocketAddresses(
|
||||
config.effectiveAdvertisedControllerListeners
|
||||
.map { endpoint =>
|
||||
(endpoint.listenerName, InetSocketAddress.createUnresolved(endpoint.host, endpoint.port))
|
||||
}
|
||||
.toMap
|
||||
.asJava
|
||||
)
|
||||
|
||||
raftManager = new KafkaRaftManager[Array[Byte]](
|
||||
Uuid.ZERO_UUID.toString,
|
||||
config,
|
||||
|
@ -96,6 +106,7 @@ class TestRaftServer(
|
|||
Some(threadNamePrefix),
|
||||
CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)),
|
||||
QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers),
|
||||
endpoints,
|
||||
new ProcessTerminatingFaultHandler.Builder().build()
|
||||
)
|
||||
|
||||
|
|
|
@ -439,8 +439,9 @@ class SocketServerTest {
|
|||
@Test
|
||||
def testDisabledRequestIsRejected(): Unit = {
|
||||
val correlationId = 57
|
||||
val header = new RequestHeader(ApiKeys.VOTE, 0, "", correlationId)
|
||||
val request = new VoteRequest.Builder(new VoteRequestData()).build()
|
||||
val version: Short = 0
|
||||
val header = new RequestHeader(ApiKeys.VOTE, version, "", correlationId)
|
||||
val request = new VoteRequest.Builder(new VoteRequestData()).build(version)
|
||||
val serializedBytes = Utils.toArray(request.serializeWithHeader(header))
|
||||
|
||||
val socket = connect()
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package kafka.raft
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.nio.channels.FileChannel
|
||||
import java.nio.channels.OverlappingFileLockException
|
||||
import java.nio.file.{Files, Path, StandardOpenOption}
|
||||
|
@ -23,25 +24,26 @@ import java.util.Properties
|
|||
import java.util.concurrent.CompletableFuture
|
||||
import kafka.log.LogManager
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.TestUtils
|
||||
import kafka.tools.TestRaftServer.ByteArraySerde
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.Uuid
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.raft.Endpoints
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.ProcessRole
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.fault.FaultHandler
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
import org.apache.kafka.server.fault.FaultHandler
|
||||
import org.mockito.Mockito._
|
||||
|
||||
import scala.util.Using
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
class RaftManagerTest {
|
||||
private def createZkBrokerConfig(
|
||||
|
@ -107,6 +109,15 @@ class RaftManagerTest {
|
|||
): KafkaRaftManager[Array[Byte]] = {
|
||||
val topicId = new Uuid(0L, 2L)
|
||||
|
||||
val endpoints = Endpoints.fromInetSocketAddresses(
|
||||
config.effectiveAdvertisedControllerListeners
|
||||
.map { endpoint =>
|
||||
(endpoint.listenerName, InetSocketAddress.createUnresolved(endpoint.host, endpoint.port))
|
||||
}
|
||||
.toMap
|
||||
.asJava
|
||||
)
|
||||
|
||||
new KafkaRaftManager[Array[Byte]](
|
||||
Uuid.randomUuid.toString,
|
||||
config,
|
||||
|
@ -119,6 +130,7 @@ class RaftManagerTest {
|
|||
Option.empty,
|
||||
CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)),
|
||||
QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers),
|
||||
endpoints,
|
||||
mock(classOf[FaultHandler])
|
||||
)
|
||||
}
|
||||
|
|
|
@ -219,6 +219,7 @@ class ControllerApisTest {
|
|||
def testFetchSentToKRaft(): Unit = {
|
||||
when(
|
||||
raftManager.handleRequest(
|
||||
any(classOf[RequestContext]),
|
||||
any(classOf[RequestHeader]),
|
||||
any(classOf[ApiMessage]),
|
||||
any(classOf[Long])
|
||||
|
@ -231,6 +232,7 @@ class ControllerApisTest {
|
|||
controllerApis.handleFetch(buildRequest(new FetchRequest(new FetchRequestData(), 12)))
|
||||
|
||||
verify(raftManager).handleRequest(
|
||||
ArgumentMatchers.any(),
|
||||
ArgumentMatchers.any(),
|
||||
ArgumentMatchers.any(),
|
||||
ArgumentMatchers.any()
|
||||
|
@ -245,6 +247,7 @@ class ControllerApisTest {
|
|||
|
||||
when(
|
||||
raftManager.handleRequest(
|
||||
any(classOf[RequestContext]),
|
||||
any(classOf[RequestHeader]),
|
||||
any(classOf[ApiMessage]),
|
||||
any(classOf[Long])
|
||||
|
@ -262,6 +265,7 @@ class ControllerApisTest {
|
|||
|
||||
|
||||
verify(raftManager).handleRequest(
|
||||
ArgumentMatchers.eq(request.context),
|
||||
ArgumentMatchers.eq(request.header),
|
||||
ArgumentMatchers.eq(fetchRequestData),
|
||||
ArgumentMatchers.eq(initialTimeMs)
|
||||
|
@ -285,6 +289,7 @@ class ControllerApisTest {
|
|||
def testFetchSnapshotSentToKRaft(): Unit = {
|
||||
when(
|
||||
raftManager.handleRequest(
|
||||
any(classOf[RequestContext]),
|
||||
any(classOf[RequestHeader]),
|
||||
any(classOf[ApiMessage]),
|
||||
any(classOf[Long])
|
||||
|
@ -297,6 +302,7 @@ class ControllerApisTest {
|
|||
controllerApis.handleFetchSnapshot(buildRequest(new FetchSnapshotRequest(new FetchSnapshotRequestData(), 0)))
|
||||
|
||||
verify(raftManager).handleRequest(
|
||||
ArgumentMatchers.any(),
|
||||
ArgumentMatchers.any(),
|
||||
ArgumentMatchers.any(),
|
||||
ArgumentMatchers.any()
|
||||
|
@ -1207,7 +1213,7 @@ class ControllerApisTest {
|
|||
val response = new FetchResponseData()
|
||||
val responseFuture = new CompletableFuture[ApiMessage]()
|
||||
val errorResponseFuture = new AtomicReference[AbstractResponse]()
|
||||
when(raftManager.handleRequest(any(), any(), any())).thenReturn(responseFuture)
|
||||
when(raftManager.handleRequest(any(), any(), any(), any())).thenReturn(responseFuture)
|
||||
when(requestChannel.sendResponse(any(), any(), any())).thenAnswer { _ =>
|
||||
// Simulate an encoding failure in the initial fetch response
|
||||
throw new UnsupportedVersionException("Something went wrong")
|
||||
|
|
|
@ -1294,7 +1294,7 @@ class KafkaConfigTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
def testControllerListenersCannotBeAdvertisedForKRaftBroker(): Unit = {
|
||||
def testControllerListenersCanBeAdvertisedForKRaftCombined(): Unit = {
|
||||
val props = new Properties()
|
||||
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
|
||||
val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094"
|
||||
|
@ -1304,11 +1304,8 @@ class KafkaConfigTest {
|
|||
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT,SSL")
|
||||
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
|
||||
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
|
||||
assertBadConfigContainingMessage(props,
|
||||
"The advertised.listeners config must not contain KRaft controller listeners from controller.listener.names when process.roles contains the broker role")
|
||||
|
||||
// Valid now
|
||||
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "SASL_SSL://C:9094")
|
||||
// Valid
|
||||
KafkaConfig.fromProps(props)
|
||||
|
||||
// Also valid if we allow advertised listeners to derive from listeners/controller.listener.names
|
||||
|
@ -1317,8 +1314,7 @@ class KafkaConfigTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
def testAdvertisedListenersDisallowedForKRaftControllerOnlyRole(): Unit = {
|
||||
// Test that advertised listeners cannot be set when KRaft and server is controller only.
|
||||
def testAdvertisedListenersAllowedForKRaftControllerOnlyRole(): Unit = {
|
||||
// Test that listeners must enumerate every controller listener
|
||||
// Test that controller listener must enumerate every listener
|
||||
val correctListeners = "PLAINTEXT://A:9092,SSL://B:9093"
|
||||
|
@ -1329,21 +1325,18 @@ class KafkaConfigTest {
|
|||
val props = new Properties()
|
||||
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
|
||||
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, correctListeners)
|
||||
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, incorrectListeners)
|
||||
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, correctListeners)
|
||||
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, correctControllerListenerNames)
|
||||
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
|
||||
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
|
||||
var expectedExceptionContainsText = "The advertised.listeners config must be empty when process.roles=controller"
|
||||
assertBadConfigContainingMessage(props, expectedExceptionContainsText)
|
||||
|
||||
// Invalid if advertised listeners is explicitly to the set
|
||||
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, correctListeners)
|
||||
assertBadConfigContainingMessage(props, expectedExceptionContainsText)
|
||||
// Valid
|
||||
KafkaConfig.fromProps(props)
|
||||
|
||||
// Invalid if listeners contains names not in controller.listener.names
|
||||
props.remove(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
|
||||
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, incorrectListeners)
|
||||
expectedExceptionContainsText = """The listeners config must only contain KRaft controller listeners from
|
||||
var expectedExceptionContainsText = """The listeners config must only contain KRaft controller listeners from
|
||||
|controller.listener.names when process.roles=controller""".stripMargin.replaceAll("\n", " ")
|
||||
assertBadConfigContainingMessage(props, expectedExceptionContainsText)
|
||||
|
||||
|
|
|
@ -121,15 +121,20 @@ class ReplicaFetcherThreadTest {
|
|||
|
||||
@Test
|
||||
def shouldSendLatestRequestVersionsByDefault(): Unit = {
|
||||
val props = TestUtils.createBrokerConfig(1, "localhost:1234")
|
||||
val config = KafkaConfig.fromProps(props)
|
||||
|
||||
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
|
||||
when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats]))
|
||||
|
||||
assertEquals(ApiKeys.FETCH.latestVersion, config.interBrokerProtocolVersion.fetchRequestVersion())
|
||||
assertEquals(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, config.interBrokerProtocolVersion.offsetForLeaderEpochRequestVersion)
|
||||
assertEquals(ApiKeys.LIST_OFFSETS.latestVersion, config.interBrokerProtocolVersion.listOffsetRequestVersion)
|
||||
// Check unstable versions
|
||||
val testingVersion = MetadataVersion.latestTesting
|
||||
assertEquals(
|
||||
ApiKeys.FETCH.latestVersion(true),
|
||||
testingVersion.fetchRequestVersion
|
||||
)
|
||||
assertEquals(
|
||||
ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion(true),
|
||||
testingVersion.offsetForLeaderEpochRequestVersion
|
||||
)
|
||||
assertEquals(
|
||||
ApiKeys.LIST_OFFSETS.latestVersion(true),
|
||||
testingVersion.listOffsetRequestVersion
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -638,10 +638,10 @@ class RequestQuotaTest extends BaseRequestTest {
|
|||
new AlterUserScramCredentialsRequest.Builder(new AlterUserScramCredentialsRequestData())
|
||||
|
||||
case ApiKeys.VOTE =>
|
||||
new VoteRequest.Builder(VoteRequest.singletonRequest(tp, 1, 2, 0, 10))
|
||||
new VoteRequest.Builder(VoteRequest.singletonRequest(tp, null, 1, 2, 0, 10))
|
||||
|
||||
case ApiKeys.BEGIN_QUORUM_EPOCH =>
|
||||
new BeginQuorumEpochRequest.Builder(BeginQuorumEpochRequest.singletonRequest(tp, 2, 5))
|
||||
new BeginQuorumEpochRequest.Builder(BeginQuorumEpochRequest.singletonRequest(tp, null, 2, 5))
|
||||
|
||||
case ApiKeys.END_QUORUM_EPOCH =>
|
||||
new EndQuorumEpochRequest.Builder(EndQuorumEpochRequest.singletonRequest(
|
||||
|
|
|
@ -64,7 +64,7 @@ public class CandidateState implements EpochState {
|
|||
int electionTimeoutMs,
|
||||
LogContext logContext
|
||||
) {
|
||||
if (!voters.isVoter(ReplicaKey.of(localId, Optional.of(localDirectoryId)))) {
|
||||
if (!voters.isVoter(ReplicaKey.of(localId, localDirectoryId))) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
"Local replica (%d, %s) must be in the set of voters %s",
|
||||
|
@ -247,7 +247,7 @@ public class CandidateState implements EpochState {
|
|||
public ElectionState election() {
|
||||
return ElectionState.withVotedCandidate(
|
||||
epoch,
|
||||
ReplicaKey.of(localId, Optional.of(localDirectoryId)),
|
||||
ReplicaKey.of(localId, localDirectoryId),
|
||||
voteStates.keySet()
|
||||
);
|
||||
}
|
||||
|
@ -257,6 +257,11 @@ public class CandidateState implements EpochState {
|
|||
return epoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoints leaderEndpoints() {
|
||||
return Endpoints.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<LogOffsetMetadata> highWatermark() {
|
||||
return highWatermark;
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.raft.generated.QuorumStateData;
|
||||
import org.apache.kafka.raft.internals.ReplicaKey;
|
||||
|
||||
|
@ -33,7 +32,6 @@ import java.util.stream.Collectors;
|
|||
public final class ElectionState {
|
||||
private static final int UNKNOWN_LEADER_ID = -1;
|
||||
private static final int NOT_VOTED = -1;
|
||||
private static final Uuid NO_VOTED_DIRECTORY_ID = Uuid.ZERO_UUID;
|
||||
|
||||
private final int epoch;
|
||||
private final OptionalInt leaderId;
|
||||
|
@ -136,7 +134,9 @@ public final class ElectionState {
|
|||
.collect(Collectors.toList());
|
||||
data.setCurrentVoters(dataVoters);
|
||||
} else if (version == 1) {
|
||||
data.setVotedDirectoryId(votedKey.flatMap(ReplicaKey::directoryId).orElse(NO_VOTED_DIRECTORY_ID));
|
||||
data.setVotedDirectoryId(
|
||||
votedKey.flatMap(ReplicaKey::directoryId).orElse(ReplicaKey.NO_DIRECTORY_ID)
|
||||
);
|
||||
} else {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
|
@ -199,13 +199,9 @@ public final class ElectionState {
|
|||
}
|
||||
|
||||
public static ElectionState fromQuorumStateData(QuorumStateData data) {
|
||||
Optional<Uuid> votedDirectoryId = data.votedDirectoryId().equals(NO_VOTED_DIRECTORY_ID) ?
|
||||
Optional.empty() :
|
||||
Optional.of(data.votedDirectoryId());
|
||||
|
||||
Optional<ReplicaKey> votedKey = data.votedId() == NOT_VOTED ?
|
||||
Optional.empty() :
|
||||
Optional.of(ReplicaKey.of(data.votedId(), votedDirectoryId));
|
||||
Optional.of(ReplicaKey.of(data.votedId(), data.votedDirectoryId()));
|
||||
|
||||
return new ElectionState(
|
||||
data.leaderEpoch(),
|
||||
|
|
|
@ -0,0 +1,219 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.message.FetchResponseData;
|
||||
import org.apache.kafka.common.message.FetchSnapshotResponseData;
|
||||
import org.apache.kafka.common.message.VoteResponseData;
|
||||
import org.apache.kafka.common.message.VotersRecord;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
public final class Endpoints {
|
||||
private final Map<ListenerName, InetSocketAddress> endpoints;
|
||||
|
||||
private Endpoints(Map<ListenerName, InetSocketAddress> endpoints) {
|
||||
this.endpoints = endpoints;
|
||||
}
|
||||
|
||||
public Optional<InetSocketAddress> address(ListenerName listener) {
|
||||
return Optional.ofNullable(endpoints.get(listener));
|
||||
}
|
||||
|
||||
public Iterator<VotersRecord.Endpoint> votersRecordEndpoints() {
|
||||
return endpoints.entrySet()
|
||||
.stream()
|
||||
.map(entry ->
|
||||
new VotersRecord.Endpoint()
|
||||
.setName(entry.getKey().value())
|
||||
.setHost(entry.getValue().getHostString())
|
||||
.setPort(entry.getValue().getPort())
|
||||
)
|
||||
.iterator();
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return endpoints.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
Endpoints that = (Endpoints) o;
|
||||
|
||||
return endpoints.equals(that.endpoints);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(endpoints);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("Endpoints(endpoints=%s)", endpoints);
|
||||
}
|
||||
|
||||
public BeginQuorumEpochRequestData.LeaderEndpointCollection toBeginQuorumEpochRequest() {
|
||||
BeginQuorumEpochRequestData.LeaderEndpointCollection leaderEndpoints =
|
||||
new BeginQuorumEpochRequestData.LeaderEndpointCollection(endpoints.size());
|
||||
for (Map.Entry<ListenerName, InetSocketAddress> entry : endpoints.entrySet()) {
|
||||
leaderEndpoints.add(
|
||||
new BeginQuorumEpochRequestData.LeaderEndpoint()
|
||||
.setName(entry.getKey().value())
|
||||
.setHost(entry.getValue().getHostString())
|
||||
.setPort(entry.getValue().getPort())
|
||||
);
|
||||
}
|
||||
|
||||
return leaderEndpoints;
|
||||
}
|
||||
|
||||
private static final Endpoints EMPTY = new Endpoints(Collections.emptyMap());
|
||||
public static Endpoints empty() {
|
||||
return EMPTY;
|
||||
}
|
||||
|
||||
public static Endpoints fromInetSocketAddresses(Map<ListenerName, InetSocketAddress> endpoints) {
|
||||
return new Endpoints(endpoints);
|
||||
}
|
||||
|
||||
public static Endpoints fromVotersRecordEndpoints(Collection<VotersRecord.Endpoint> endpoints) {
|
||||
Map<ListenerName, InetSocketAddress> listeners = new HashMap<>(endpoints.size());
|
||||
for (VotersRecord.Endpoint endpoint : endpoints) {
|
||||
listeners.put(
|
||||
ListenerName.normalised(endpoint.name()),
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
);
|
||||
}
|
||||
|
||||
return new Endpoints(listeners);
|
||||
}
|
||||
|
||||
public static Endpoints fromBeginQuorumEpochRequest(BeginQuorumEpochRequestData.LeaderEndpointCollection endpoints) {
|
||||
Map<ListenerName, InetSocketAddress> listeners = new HashMap<>(endpoints.size());
|
||||
for (BeginQuorumEpochRequestData.LeaderEndpoint endpoint : endpoints) {
|
||||
listeners.put(
|
||||
ListenerName.normalised(endpoint.name()),
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
);
|
||||
}
|
||||
|
||||
return new Endpoints(listeners);
|
||||
}
|
||||
|
||||
public static Endpoints fromBeginQuorumEpochResponse(
|
||||
ListenerName listenerName,
|
||||
BeginQuorumEpochResponseData.NodeEndpointCollection endpoints
|
||||
) {
|
||||
Map<ListenerName, InetSocketAddress> listeners = new HashMap<>(endpoints.size());
|
||||
for (BeginQuorumEpochResponseData.NodeEndpoint endpoint : endpoints) {
|
||||
listeners.put(
|
||||
listenerName,
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
);
|
||||
}
|
||||
|
||||
return new Endpoints(listeners);
|
||||
}
|
||||
|
||||
public static Endpoints fromEndQuorumEpochRequest(EndQuorumEpochRequestData.LeaderEndpointCollection endpoints) {
|
||||
Map<ListenerName, InetSocketAddress> listeners = new HashMap<>(endpoints.size());
|
||||
for (EndQuorumEpochRequestData.LeaderEndpoint endpoint : endpoints) {
|
||||
listeners.put(
|
||||
ListenerName.normalised(endpoint.name()),
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
);
|
||||
}
|
||||
|
||||
return new Endpoints(listeners);
|
||||
}
|
||||
|
||||
public static Endpoints fromEndQuorumEpochResponse(
|
||||
ListenerName listenerName,
|
||||
EndQuorumEpochResponseData.NodeEndpointCollection endpoints
|
||||
) {
|
||||
Map<ListenerName, InetSocketAddress> listeners = new HashMap<>(endpoints.size());
|
||||
for (EndQuorumEpochResponseData.NodeEndpoint endpoint : endpoints) {
|
||||
listeners.put(
|
||||
listenerName,
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
);
|
||||
}
|
||||
|
||||
return new Endpoints(listeners);
|
||||
}
|
||||
|
||||
public static Endpoints fromVoteResponse(
|
||||
ListenerName listenerName,
|
||||
VoteResponseData.NodeEndpointCollection endpoints
|
||||
) {
|
||||
Map<ListenerName, InetSocketAddress> listeners = new HashMap<>(endpoints.size());
|
||||
for (VoteResponseData.NodeEndpoint endpoint : endpoints) {
|
||||
listeners.put(
|
||||
listenerName,
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
);
|
||||
}
|
||||
|
||||
return new Endpoints(listeners);
|
||||
}
|
||||
|
||||
public static Endpoints fromFetchResponse(
|
||||
ListenerName listenerName,
|
||||
FetchResponseData.NodeEndpointCollection endpoints
|
||||
) {
|
||||
Map<ListenerName, InetSocketAddress> listeners = new HashMap<>(endpoints.size());
|
||||
for (FetchResponseData.NodeEndpoint endpoint : endpoints) {
|
||||
listeners.put(
|
||||
listenerName,
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
);
|
||||
}
|
||||
|
||||
return new Endpoints(listeners);
|
||||
}
|
||||
|
||||
public static Endpoints fromFetchSnapshotResponse(
|
||||
ListenerName listenerName,
|
||||
FetchSnapshotResponseData.NodeEndpointCollection endpoints
|
||||
) {
|
||||
Map<ListenerName, InetSocketAddress> listeners = new HashMap<>(endpoints.size());
|
||||
for (FetchSnapshotResponseData.NodeEndpoint endpoint : endpoints) {
|
||||
listeners.put(
|
||||
listenerName,
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
);
|
||||
}
|
||||
|
||||
return new Endpoints(listeners);
|
||||
}
|
||||
}
|
|
@ -49,6 +49,13 @@ public interface EpochState extends Closeable {
|
|||
*/
|
||||
int epoch();
|
||||
|
||||
/**
|
||||
* Returns the known endpoints for the leader.
|
||||
*
|
||||
* If the leader is not known then {@code Endpoints.empty()} is returned.
|
||||
*/
|
||||
Endpoints leaderEndpoints();
|
||||
|
||||
/**
|
||||
* User-friendly description of the state
|
||||
*/
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Timer;
|
||||
|
@ -32,7 +33,8 @@ import java.util.Set;
|
|||
public class FollowerState implements EpochState {
|
||||
private final int fetchTimeoutMs;
|
||||
private final int epoch;
|
||||
private final Node leader;
|
||||
private final int leaderId;
|
||||
private final Endpoints endpoints;
|
||||
private final Set<Integer> voters;
|
||||
// Used for tracking the expiration of both the Fetch and FetchSnapshot requests
|
||||
private final Timer fetchTimer;
|
||||
|
@ -47,7 +49,8 @@ public class FollowerState implements EpochState {
|
|||
public FollowerState(
|
||||
Time time,
|
||||
int epoch,
|
||||
Node leader,
|
||||
int leaderId,
|
||||
Endpoints endpoints,
|
||||
Set<Integer> voters,
|
||||
Optional<LogOffsetMetadata> highWatermark,
|
||||
int fetchTimeoutMs,
|
||||
|
@ -55,7 +58,8 @@ public class FollowerState implements EpochState {
|
|||
) {
|
||||
this.fetchTimeoutMs = fetchTimeoutMs;
|
||||
this.epoch = epoch;
|
||||
this.leader = leader;
|
||||
this.leaderId = leaderId;
|
||||
this.endpoints = endpoints;
|
||||
this.voters = voters;
|
||||
this.fetchTimer = time.timer(fetchTimeoutMs);
|
||||
this.highWatermark = highWatermark;
|
||||
|
@ -64,7 +68,7 @@ public class FollowerState implements EpochState {
|
|||
|
||||
@Override
|
||||
public ElectionState election() {
|
||||
return ElectionState.withElectedLeader(epoch, leader.id(), voters);
|
||||
return ElectionState.withElectedLeader(epoch, leaderId, voters);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -72,6 +76,11 @@ public class FollowerState implements EpochState {
|
|||
return epoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoints leaderEndpoints() {
|
||||
return endpoints;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return "Follower";
|
||||
|
@ -82,8 +91,28 @@ public class FollowerState implements EpochState {
|
|||
return fetchTimer.remainingMs();
|
||||
}
|
||||
|
||||
public Node leader() {
|
||||
return leader;
|
||||
public int leaderId() {
|
||||
return leaderId;
|
||||
}
|
||||
|
||||
public Node leaderNode(ListenerName listener) {
|
||||
/* KAFKA-16529 is going to change this so that the leader is not required to be in the set
|
||||
* of voters. In other words, don't throw an IllegalStateException if the leader is not in
|
||||
* the set of voters.
|
||||
*/
|
||||
return endpoints
|
||||
.address(listener)
|
||||
.map(address -> new Node(leaderId, address.getHostString(), address.getPort()))
|
||||
.orElseThrow(() ->
|
||||
new IllegalArgumentException(
|
||||
String.format(
|
||||
"Unknown endpoint for leader %d and listener %s, known endpoints are %s",
|
||||
leaderId,
|
||||
listener,
|
||||
endpoints
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public boolean hasFetchTimeoutExpired(long currentTimeMs) {
|
||||
|
@ -158,7 +187,7 @@ public class FollowerState implements EpochState {
|
|||
log.debug(
|
||||
"Rejecting vote request from candidate ({}) since we already have a leader {} in epoch {}",
|
||||
candidateKey,
|
||||
leader,
|
||||
leaderId,
|
||||
epoch
|
||||
);
|
||||
return false;
|
||||
|
@ -167,11 +196,12 @@ public class FollowerState implements EpochState {
|
|||
@Override
|
||||
public String toString() {
|
||||
return String.format(
|
||||
"FollowerState(fetchTimeoutMs=%d, epoch=%d, leader=%s voters=%s, highWatermark=%s, " +
|
||||
"FollowerState(fetchTimeoutMs=%d, epoch=%d, leader=%d, endpoints=%s, voters=%s, highWatermark=%s, " +
|
||||
"fetchingSnapshot=%s)",
|
||||
fetchTimeoutMs,
|
||||
epoch,
|
||||
leader,
|
||||
leaderId,
|
||||
endpoints,
|
||||
voters,
|
||||
highWatermark,
|
||||
fetchingSnapshot
|
||||
|
|
|
@ -46,17 +46,14 @@ import org.apache.kafka.common.record.Records;
|
|||
import org.apache.kafka.common.record.UnalignedMemoryRecords;
|
||||
import org.apache.kafka.common.record.UnalignedRecords;
|
||||
import org.apache.kafka.common.requests.BeginQuorumEpochRequest;
|
||||
import org.apache.kafka.common.requests.BeginQuorumEpochResponse;
|
||||
import org.apache.kafka.common.requests.DescribeQuorumRequest;
|
||||
import org.apache.kafka.common.requests.DescribeQuorumResponse;
|
||||
import org.apache.kafka.common.requests.EndQuorumEpochRequest;
|
||||
import org.apache.kafka.common.requests.EndQuorumEpochResponse;
|
||||
import org.apache.kafka.common.requests.FetchRequest;
|
||||
import org.apache.kafka.common.requests.FetchResponse;
|
||||
import org.apache.kafka.common.requests.FetchSnapshotRequest;
|
||||
import org.apache.kafka.common.requests.FetchSnapshotResponse;
|
||||
import org.apache.kafka.common.requests.VoteRequest;
|
||||
import org.apache.kafka.common.requests.VoteResponse;
|
||||
import org.apache.kafka.common.utils.BufferSupplier;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
|
@ -164,6 +161,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
private final Time time;
|
||||
private final int fetchMaxWaitMs;
|
||||
private final String clusterId;
|
||||
private final Endpoints localListeners;
|
||||
private final NetworkChannel channel;
|
||||
private final ReplicatedLog log;
|
||||
private final Random random;
|
||||
|
@ -214,6 +212,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
LogContext logContext,
|
||||
String clusterId,
|
||||
Collection<InetSocketAddress> bootstrapServers,
|
||||
Endpoints localListeners,
|
||||
QuorumConfig quorumConfig
|
||||
) {
|
||||
this(
|
||||
|
@ -229,6 +228,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
MAX_FETCH_WAIT_MS,
|
||||
clusterId,
|
||||
bootstrapServers,
|
||||
localListeners,
|
||||
logContext,
|
||||
new Random(),
|
||||
quorumConfig
|
||||
|
@ -248,6 +248,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
int fetchMaxWaitMs,
|
||||
String clusterId,
|
||||
Collection<InetSocketAddress> bootstrapServers,
|
||||
Endpoints localListeners,
|
||||
LogContext logContext,
|
||||
Random random,
|
||||
QuorumConfig quorumConfig
|
||||
|
@ -264,6 +265,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
this.appendPurgatory = new ThresholdPurgatory<>(expirationService);
|
||||
this.time = time;
|
||||
this.clusterId = clusterId;
|
||||
this.localListeners = localListeners;
|
||||
this.fetchMaxWaitMs = fetchMaxWaitMs;
|
||||
this.logger = logContext.logger(KafkaRaftClient.class);
|
||||
this.random = random;
|
||||
|
@ -315,7 +317,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
) {
|
||||
final LogOffsetMetadata endOffsetMetadata = log.endOffset();
|
||||
|
||||
if (state.updateLocalState(endOffsetMetadata, partitionState.lastVoterSet().voters())) {
|
||||
if (state.updateLocalState(endOffsetMetadata, partitionState.lastVoterSet())) {
|
||||
onUpdateLeaderHighWatermark(state, currentTimeMs);
|
||||
}
|
||||
|
||||
|
@ -452,9 +454,9 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
quorum = new QuorumState(
|
||||
nodeId,
|
||||
nodeDirectoryId,
|
||||
channel.listenerName(),
|
||||
partitionState::lastVoterSet,
|
||||
partitionState::lastKraftVersion,
|
||||
localListeners,
|
||||
quorumConfig.electionTimeoutMs(),
|
||||
quorumConfig.fetchTimeoutMs(),
|
||||
quorumStateStore,
|
||||
|
@ -613,22 +615,32 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
private void transitionToFollower(
|
||||
int epoch,
|
||||
Node leader,
|
||||
int leaderId,
|
||||
Endpoints endpoints,
|
||||
long currentTimeMs
|
||||
) {
|
||||
quorum.transitionToFollower(epoch, leader);
|
||||
quorum.transitionToFollower(epoch, leaderId, endpoints);
|
||||
maybeFireLeaderChange();
|
||||
onBecomeFollower(currentTimeMs);
|
||||
}
|
||||
|
||||
private VoteResponseData buildVoteResponse(Errors partitionLevelError, boolean voteGranted) {
|
||||
return VoteResponse.singletonResponse(
|
||||
private VoteResponseData buildVoteResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
Errors partitionLevelError,
|
||||
boolean voteGranted
|
||||
) {
|
||||
return RaftUtil.singletonVoteResponse(
|
||||
listenerName,
|
||||
apiVersion,
|
||||
Errors.NONE,
|
||||
log.topicPartition(),
|
||||
partitionLevelError,
|
||||
quorum.epoch(),
|
||||
quorum.leaderIdOrSentinel(),
|
||||
voteGranted);
|
||||
voteGranted,
|
||||
quorum.leaderEndpoints()
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -638,8 +650,6 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
* but different from this node
|
||||
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down
|
||||
* - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch
|
||||
* - {@link Errors#INCONSISTENT_VOTER_SET} if the request suggests inconsistent voter membership (e.g.
|
||||
* if this node or the sender is not one of the current known voters)
|
||||
* - {@link Errors#INVALID_REQUEST} if the last epoch or offset are invalid
|
||||
*/
|
||||
private VoteResponseData handleVoteRequest(
|
||||
|
@ -665,20 +675,52 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
int lastEpoch = partitionRequest.lastOffsetEpoch();
|
||||
long lastEpochEndOffset = partitionRequest.lastOffset();
|
||||
if (lastEpochEndOffset < 0 || lastEpoch < 0 || lastEpoch >= candidateEpoch) {
|
||||
return buildVoteResponse(Errors.INVALID_REQUEST, false);
|
||||
return buildVoteResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
Errors.INVALID_REQUEST,
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
Optional<Errors> errorOpt = validateVoterOnlyRequest(candidateId, candidateEpoch);
|
||||
if (errorOpt.isPresent()) {
|
||||
return buildVoteResponse(errorOpt.get(), false);
|
||||
return buildVoteResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
errorOpt.get(),
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
if (candidateEpoch > quorum.epoch()) {
|
||||
transitionToUnattached(candidateEpoch);
|
||||
}
|
||||
|
||||
// Check that the request was intended for this replica
|
||||
Optional<ReplicaKey> voterKey = RaftUtil.voteRequestVoterKey(request, partitionRequest);
|
||||
if (!isValidVoterKey(voterKey)) {
|
||||
logger.info(
|
||||
"Candidate sent a voter key ({}) in the VOTE request that doesn't match the " +
|
||||
"local key ({}, {}); rejecting the vote",
|
||||
voterKey,
|
||||
nodeId,
|
||||
nodeDirectoryId
|
||||
);
|
||||
// The request is not intended to this replica since the replica keys don't match
|
||||
return buildVoteResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
Errors.INVALID_VOTER_KEY,
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
OffsetAndEpoch lastEpochEndOffsetAndEpoch = new OffsetAndEpoch(lastEpochEndOffset, lastEpoch);
|
||||
ReplicaKey candidateKey = ReplicaKey.of(candidateId, Optional.empty());
|
||||
ReplicaKey candidateKey = ReplicaKey.of(
|
||||
candidateId,
|
||||
partitionRequest.candidateDirectoryId()
|
||||
);
|
||||
boolean voteGranted = quorum.canGrantVote(
|
||||
candidateKey,
|
||||
lastEpochEndOffsetAndEpoch.compareTo(endOffset()) >= 0
|
||||
|
@ -689,7 +731,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
logger.info("Vote request {} with epoch {} is {}", request, candidateEpoch, voteGranted ? "granted" : "rejected");
|
||||
return buildVoteResponse(Errors.NONE, voteGranted);
|
||||
return buildVoteResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
Errors.NONE,
|
||||
voteGranted
|
||||
);
|
||||
}
|
||||
|
||||
private boolean handleVoteResponse(
|
||||
|
@ -716,6 +763,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
Optional<Boolean> handled = maybeHandleCommonResponse(
|
||||
error, responseLeaderId, responseEpoch, currentTimeMs);
|
||||
// KAFKA-16529 will need to handle the INVALID_VOTER_KEY error when handling the response
|
||||
if (handled.isPresent()) {
|
||||
return handled.get();
|
||||
} else if (error == Errors.NONE) {
|
||||
|
@ -763,22 +811,31 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) {
|
||||
if (positionInSuccessors <= 0 || positionInSuccessors >= totalNumSuccessors) {
|
||||
throw new IllegalArgumentException("Position " + positionInSuccessors + " should be larger than zero" +
|
||||
" and smaller than total number of successors " + totalNumSuccessors);
|
||||
if (positionInSuccessors == 0) {
|
||||
return 0;
|
||||
} else if (positionInSuccessors < 0 || positionInSuccessors >= totalNumSuccessors) {
|
||||
return quorumConfig.electionBackoffMaxMs();
|
||||
}
|
||||
|
||||
int retryBackOffBaseMs = quorumConfig.electionBackoffMaxMs() >> (totalNumSuccessors - 1);
|
||||
return Math.min(quorumConfig.electionBackoffMaxMs(), retryBackOffBaseMs << (positionInSuccessors - 1));
|
||||
}
|
||||
|
||||
private BeginQuorumEpochResponseData buildBeginQuorumEpochResponse(Errors partitionLevelError) {
|
||||
return BeginQuorumEpochResponse.singletonResponse(
|
||||
private BeginQuorumEpochResponseData buildBeginQuorumEpochResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
Errors partitionLevelError
|
||||
) {
|
||||
return RaftUtil.singletonBeginQuorumEpochResponse(
|
||||
listenerName,
|
||||
apiVersion,
|
||||
Errors.NONE,
|
||||
log.topicPartition(),
|
||||
partitionLevelError,
|
||||
quorum.epoch(),
|
||||
quorum.leaderIdOrSentinel());
|
||||
quorum.leaderIdOrSentinel(),
|
||||
quorum.leaderEndpoints()
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -787,8 +844,6 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
* - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request
|
||||
* but different from this node
|
||||
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down
|
||||
* - {@link Errors#INCONSISTENT_VOTER_SET} if the request suggests inconsistent voter membership (e.g.
|
||||
* if this node or the sender is not one of the current known voters)
|
||||
* - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch
|
||||
*/
|
||||
private BeginQuorumEpochResponseData handleBeginQuorumEpochRequest(
|
||||
|
@ -814,15 +869,51 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
Optional<Errors> errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch);
|
||||
if (errorOpt.isPresent()) {
|
||||
return buildBeginQuorumEpochResponse(errorOpt.get());
|
||||
return buildBeginQuorumEpochResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
errorOpt.get()
|
||||
);
|
||||
}
|
||||
|
||||
// Compute the leader's endpoint from the request or the voter set
|
||||
final Endpoints leaderEndpoints;
|
||||
if (request.leaderEndpoints().isEmpty()) {
|
||||
leaderEndpoints = partitionState.lastVoterSet().listeners(requestLeaderId);
|
||||
} else {
|
||||
leaderEndpoints = Endpoints.fromBeginQuorumEpochRequest(request.leaderEndpoints());
|
||||
}
|
||||
|
||||
maybeTransition(
|
||||
partitionState.lastVoterSet().voterNode(requestLeaderId, channel.listenerName()),
|
||||
OptionalInt.of(requestLeaderId),
|
||||
requestEpoch,
|
||||
leaderEndpoints,
|
||||
currentTimeMs
|
||||
);
|
||||
return buildBeginQuorumEpochResponse(Errors.NONE);
|
||||
|
||||
// Check that the request was intended for this replica
|
||||
Optional<ReplicaKey> voterKey = RaftUtil.beginQuorumEpochRequestVoterKey(request, partitionRequest);
|
||||
if (!isValidVoterKey(voterKey)) {
|
||||
logger.info(
|
||||
"Leader sent a voter key ({}) in the BEGIN_QUORUM_EPOCH request that doesn't " +
|
||||
"match the local key ({}, {}); returning INVALID_VOTER_KEY",
|
||||
voterKey,
|
||||
nodeId,
|
||||
nodeDirectoryId
|
||||
);
|
||||
// The request is not intended to this replica since the replica keys don't match
|
||||
return buildBeginQuorumEpochResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
Errors.INVALID_VOTER_KEY
|
||||
);
|
||||
}
|
||||
|
||||
return buildBeginQuorumEpochResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
Errors.NONE
|
||||
);
|
||||
}
|
||||
|
||||
private boolean handleBeginQuorumEpochResponse(
|
||||
|
@ -849,6 +940,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
Optional<Boolean> handled = maybeHandleCommonResponse(
|
||||
partitionError, responseLeaderId, responseEpoch, currentTimeMs);
|
||||
// KAFKA-16529 will need to handle the INVALID_VOTER_KEY error when handling the response
|
||||
if (handled.isPresent()) {
|
||||
return handled.get();
|
||||
} else if (partitionError == Errors.NONE) {
|
||||
|
@ -865,13 +957,21 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
}
|
||||
|
||||
private EndQuorumEpochResponseData buildEndQuorumEpochResponse(Errors partitionLevelError) {
|
||||
return EndQuorumEpochResponse.singletonResponse(
|
||||
private EndQuorumEpochResponseData buildEndQuorumEpochResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
Errors partitionLevelError
|
||||
) {
|
||||
return RaftUtil.singletonEndQuorumEpochResponse(
|
||||
listenerName,
|
||||
apiVersion,
|
||||
Errors.NONE,
|
||||
log.topicPartition(),
|
||||
partitionLevelError,
|
||||
quorum.epoch(),
|
||||
quorum.leaderIdOrSentinel());
|
||||
quorum.leaderIdOrSentinel(),
|
||||
quorum.leaderEndpoints()
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -880,8 +980,6 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
* - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request
|
||||
* but different from this node
|
||||
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down
|
||||
* - {@link Errors#INCONSISTENT_VOTER_SET} if the request suggests inconsistent voter membership (e.g.
|
||||
* if this node or the sender is not one of the current known voters)
|
||||
* - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch
|
||||
*/
|
||||
private EndQuorumEpochResponseData handleEndQuorumEpochRequest(
|
||||
|
@ -907,39 +1005,72 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
Optional<Errors> errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch);
|
||||
if (errorOpt.isPresent()) {
|
||||
return buildEndQuorumEpochResponse(errorOpt.get());
|
||||
return buildEndQuorumEpochResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
errorOpt.get()
|
||||
);
|
||||
}
|
||||
|
||||
final Endpoints leaderEndpoints;
|
||||
if (request.leaderEndpoints().isEmpty()) {
|
||||
leaderEndpoints = partitionState.lastVoterSet().listeners(requestLeaderId);
|
||||
} else {
|
||||
leaderEndpoints = Endpoints.fromEndQuorumEpochRequest(request.leaderEndpoints());
|
||||
}
|
||||
|
||||
maybeTransition(
|
||||
partitionState.lastVoterSet().voterNode(requestLeaderId, channel.listenerName()),
|
||||
OptionalInt.of(requestLeaderId),
|
||||
requestEpoch,
|
||||
leaderEndpoints,
|
||||
currentTimeMs
|
||||
);
|
||||
|
||||
if (quorum.isFollower()) {
|
||||
FollowerState state = quorum.followerStateOrThrow();
|
||||
if (state.leader().id() == requestLeaderId) {
|
||||
List<Integer> preferredSuccessors = partitionRequest.preferredSuccessors();
|
||||
long electionBackoffMs = endEpochElectionBackoff(preferredSuccessors);
|
||||
logger.debug("Overriding follower fetch timeout to {} after receiving " +
|
||||
"EndQuorumEpoch request from leader {} in epoch {}", electionBackoffMs,
|
||||
requestLeaderId, requestEpoch);
|
||||
if (state.leaderId() == requestLeaderId) {
|
||||
List<ReplicaKey> preferredCandidates = EndQuorumEpochRequest
|
||||
.preferredCandidates(partitionRequest)
|
||||
.stream()
|
||||
.map(replica -> ReplicaKey.of(replica.candidateId(), replica.candidateDirectoryId()))
|
||||
.collect(Collectors.toList());
|
||||
long electionBackoffMs = endEpochElectionBackoff(preferredCandidates);
|
||||
logger.debug(
|
||||
"Overriding follower fetch timeout to {} after receiving EndQuorumEpoch " +
|
||||
"request from leader {} in epoch {}",
|
||||
electionBackoffMs,
|
||||
requestLeaderId,
|
||||
requestEpoch
|
||||
);
|
||||
state.overrideFetchTimeout(currentTimeMs, electionBackoffMs);
|
||||
}
|
||||
}
|
||||
return buildEndQuorumEpochResponse(Errors.NONE);
|
||||
return buildEndQuorumEpochResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
Errors.NONE
|
||||
);
|
||||
}
|
||||
|
||||
private long endEpochElectionBackoff(List<Integer> preferredSuccessors) {
|
||||
private long endEpochElectionBackoff(Collection<ReplicaKey> preferredCandidates) {
|
||||
// Based on the priority inside the preferred successors, choose the corresponding delayed
|
||||
// election backoff time based on strict exponential mechanism so that the most up-to-date
|
||||
// voter has a higher chance to be elected. If the node's priority is highest, become
|
||||
// candidate immediately instead of waiting for next poll.
|
||||
int position = preferredSuccessors.indexOf(quorum.localIdOrThrow());
|
||||
if (position <= 0) {
|
||||
return 0;
|
||||
} else {
|
||||
return strictExponentialElectionBackoffMs(position, preferredSuccessors.size());
|
||||
int position = 0;
|
||||
for (ReplicaKey candidate : preferredCandidates) {
|
||||
if (candidate.id() == quorum.localIdOrThrow()) {
|
||||
if (!candidate.directoryId().isPresent() ||
|
||||
candidate.directoryId().get().equals(quorum.localDirectoryId())
|
||||
) {
|
||||
// Found ourselves in the preferred candidate list
|
||||
break;
|
||||
}
|
||||
}
|
||||
position++;
|
||||
}
|
||||
|
||||
return strictExponentialElectionBackoffMs(position, preferredCandidates.size());
|
||||
}
|
||||
|
||||
private boolean handleEndQuorumEpochResponse(
|
||||
|
@ -977,45 +1108,60 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
private FetchResponseData buildFetchResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
Errors error,
|
||||
Records records,
|
||||
ValidOffsetAndEpoch validOffsetAndEpoch,
|
||||
Optional<LogOffsetMetadata> highWatermark
|
||||
) {
|
||||
return RaftUtil.singletonFetchResponse(log.topicPartition(), log.topicId(), Errors.NONE, partitionData -> {
|
||||
partitionData
|
||||
.setRecords(records)
|
||||
.setErrorCode(error.code())
|
||||
.setLogStartOffset(log.startOffset())
|
||||
.setHighWatermark(
|
||||
highWatermark.map(offsetMetadata -> offsetMetadata.offset).orElse(-1L)
|
||||
);
|
||||
return RaftUtil.singletonFetchResponse(
|
||||
listenerName,
|
||||
apiVersion,
|
||||
log.topicPartition(),
|
||||
log.topicId(),
|
||||
Errors.NONE,
|
||||
quorum.leaderIdOrSentinel(),
|
||||
quorum.leaderEndpoints(),
|
||||
partitionData -> {
|
||||
partitionData
|
||||
.setRecords(records)
|
||||
.setErrorCode(error.code())
|
||||
.setLogStartOffset(log.startOffset())
|
||||
.setHighWatermark(
|
||||
highWatermark.map(offsetMetadata -> offsetMetadata.offset).orElse(-1L)
|
||||
);
|
||||
|
||||
partitionData.currentLeader()
|
||||
.setLeaderEpoch(quorum.epoch())
|
||||
.setLeaderId(quorum.leaderIdOrSentinel());
|
||||
partitionData.currentLeader()
|
||||
.setLeaderEpoch(quorum.epoch())
|
||||
.setLeaderId(quorum.leaderIdOrSentinel());
|
||||
|
||||
switch (validOffsetAndEpoch.kind()) {
|
||||
case DIVERGING:
|
||||
partitionData.divergingEpoch()
|
||||
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch())
|
||||
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset());
|
||||
break;
|
||||
case SNAPSHOT:
|
||||
partitionData.snapshotId()
|
||||
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch())
|
||||
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset());
|
||||
break;
|
||||
default:
|
||||
switch (validOffsetAndEpoch.kind()) {
|
||||
case DIVERGING:
|
||||
partitionData.divergingEpoch()
|
||||
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch())
|
||||
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset());
|
||||
break;
|
||||
case SNAPSHOT:
|
||||
partitionData.snapshotId()
|
||||
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch())
|
||||
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset());
|
||||
break;
|
||||
default:
|
||||
}
|
||||
}
|
||||
});
|
||||
);
|
||||
}
|
||||
|
||||
private FetchResponseData buildEmptyFetchResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
Errors error,
|
||||
Optional<LogOffsetMetadata> highWatermark
|
||||
) {
|
||||
return buildFetchResponse(
|
||||
listenerName,
|
||||
apiVersion,
|
||||
error,
|
||||
MemoryRecords.EMPTY,
|
||||
ValidOffsetAndEpoch.valid(),
|
||||
|
@ -1069,12 +1215,26 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|| fetchPartition.lastFetchedEpoch() < 0
|
||||
|| fetchPartition.lastFetchedEpoch() > fetchPartition.currentLeaderEpoch()) {
|
||||
return completedFuture(
|
||||
buildEmptyFetchResponse(Errors.INVALID_REQUEST, Optional.empty())
|
||||
buildEmptyFetchResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
Errors.INVALID_REQUEST,
|
||||
Optional.empty()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
int replicaId = FetchRequest.replicaId(request);
|
||||
FetchResponseData response = tryCompleteFetchRequest(replicaId, fetchPartition, currentTimeMs);
|
||||
ReplicaKey replicaKey = ReplicaKey.of(
|
||||
FetchRequest.replicaId(request),
|
||||
fetchPartition.replicaDirectoryId()
|
||||
);
|
||||
FetchResponseData response = tryCompleteFetchRequest(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
replicaKey,
|
||||
fetchPartition,
|
||||
currentTimeMs
|
||||
);
|
||||
FetchResponseData.PartitionData partitionResponse =
|
||||
response.responses().get(0).partitions().get(0);
|
||||
|
||||
|
@ -1111,32 +1271,53 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
return response;
|
||||
} else {
|
||||
// If there was any error other than REQUEST_TIMED_OUT, return it.
|
||||
logger.info("Failed to handle fetch from {} at {} due to {}",
|
||||
replicaId, fetchPartition.fetchOffset(), error);
|
||||
return buildEmptyFetchResponse(error, Optional.empty());
|
||||
logger.info(
|
||||
"Failed to handle fetch from {} at {} due to {}",
|
||||
replicaKey,
|
||||
fetchPartition.fetchOffset(),
|
||||
error
|
||||
);
|
||||
return buildEmptyFetchResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
error,
|
||||
Optional.empty()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
// FIXME: `completionTimeMs`, which can be null
|
||||
logger.trace("Completing delayed fetch from {} starting at offset {} at {}",
|
||||
replicaId, fetchPartition.fetchOffset(), completionTimeMs);
|
||||
logger.trace(
|
||||
"Completing delayed fetch from {} starting at offset {} at {}",
|
||||
replicaKey,
|
||||
fetchPartition.fetchOffset(),
|
||||
completionTimeMs
|
||||
);
|
||||
|
||||
// It is safe to call tryCompleteFetchRequest because only the polling thread completes this
|
||||
// future successfully. This is true because only the polling thread appends record batches to
|
||||
// the log from maybeAppendBatches.
|
||||
return tryCompleteFetchRequest(replicaId, fetchPartition, time.milliseconds());
|
||||
return tryCompleteFetchRequest(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
replicaKey,
|
||||
fetchPartition,
|
||||
time.milliseconds()
|
||||
);
|
||||
});
|
||||
}
|
||||
|
||||
private FetchResponseData tryCompleteFetchRequest(
|
||||
int replicaId,
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
ReplicaKey replicaKey,
|
||||
FetchRequestData.FetchPartition request,
|
||||
long currentTimeMs
|
||||
) {
|
||||
try {
|
||||
Optional<Errors> errorOpt = validateLeaderOnlyRequest(request.currentLeaderEpoch());
|
||||
if (errorOpt.isPresent()) {
|
||||
return buildEmptyFetchResponse(errorOpt.get(), Optional.empty());
|
||||
return buildEmptyFetchResponse(listenerName, apiVersion, errorOpt.get(), Optional.empty());
|
||||
}
|
||||
|
||||
long fetchOffset = request.fetchOffset();
|
||||
|
@ -1157,7 +1338,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
if (validOffsetAndEpoch.kind() == ValidOffsetAndEpoch.Kind.VALID) {
|
||||
LogFetchInfo info = log.read(fetchOffset, Isolation.UNCOMMITTED);
|
||||
|
||||
if (state.updateReplicaState(replicaId, Uuid.ZERO_UUID, currentTimeMs, info.startOffsetMetadata)) {
|
||||
if (state.updateReplicaState(replicaKey, currentTimeMs, info.startOffsetMetadata)) {
|
||||
onUpdateLeaderHighWatermark(state, currentTimeMs);
|
||||
}
|
||||
|
||||
|
@ -1166,10 +1347,17 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
records = MemoryRecords.EMPTY;
|
||||
}
|
||||
|
||||
return buildFetchResponse(Errors.NONE, records, validOffsetAndEpoch, state.highWatermark());
|
||||
return buildFetchResponse(
|
||||
listenerName,
|
||||
apiVersion,
|
||||
Errors.NONE,
|
||||
records,
|
||||
validOffsetAndEpoch,
|
||||
state.highWatermark()
|
||||
);
|
||||
} catch (Exception e) {
|
||||
logger.error("Caught unexpected error in fetch completion of request {}", request, e);
|
||||
return buildEmptyFetchResponse(Errors.UNKNOWN_SERVER_ERROR, Optional.empty());
|
||||
return buildEmptyFetchResponse(listenerName, apiVersion, Errors.UNKNOWN_SERVER_ERROR, Optional.empty());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1352,12 +1540,11 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
LeaderState<T> leaderState = quorum.leaderStateOrThrow();
|
||||
return DescribeQuorumResponse.singletonResponse(
|
||||
return RaftUtil.singletonDescribeQuorumResponse(
|
||||
requestMetadata.apiVersion(),
|
||||
log.topicPartition(),
|
||||
leaderState.describeQuorum(currentTimeMs),
|
||||
requestMetadata.apiVersion() < DescribeQuorumResponseData.Node.LOWEST_SUPPORTED_VERSION
|
||||
? null
|
||||
: leaderState.nodes(currentTimeMs)
|
||||
leaderState.nodes(currentTimeMs)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -1399,8 +1586,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
data.topics().get(0).partitions().get(0).partition()
|
||||
);
|
||||
|
||||
return FetchSnapshotResponse.singleton(
|
||||
return RaftUtil.singletonFetchSnapshotResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
unknownTopicPartition,
|
||||
quorum.leaderIdOrSentinel(),
|
||||
quorum.leaderEndpoints(),
|
||||
responsePartitionSnapshot -> responsePartitionSnapshot
|
||||
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code())
|
||||
);
|
||||
|
@ -1411,8 +1602,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
partitionSnapshot.currentLeaderEpoch()
|
||||
);
|
||||
if (leaderValidation.isPresent()) {
|
||||
return FetchSnapshotResponse.singleton(
|
||||
return RaftUtil.singletonFetchSnapshotResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
log.topicPartition(),
|
||||
quorum.leaderIdOrSentinel(),
|
||||
quorum.leaderEndpoints(),
|
||||
responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot)
|
||||
.setErrorCode(leaderValidation.get().code())
|
||||
);
|
||||
|
@ -1424,8 +1619,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
);
|
||||
Optional<RawSnapshotReader> snapshotOpt = log.readSnapshot(snapshotId);
|
||||
if (!snapshotOpt.isPresent()) {
|
||||
return FetchSnapshotResponse.singleton(
|
||||
return RaftUtil.singletonFetchSnapshotResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
log.topicPartition(),
|
||||
quorum.leaderIdOrSentinel(),
|
||||
quorum.leaderEndpoints(),
|
||||
responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot)
|
||||
.setErrorCode(Errors.SNAPSHOT_NOT_FOUND.code())
|
||||
);
|
||||
|
@ -1434,8 +1633,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
RawSnapshotReader snapshot = snapshotOpt.get();
|
||||
long snapshotSize = snapshot.sizeInBytes();
|
||||
if (partitionSnapshot.position() < 0 || partitionSnapshot.position() >= snapshotSize) {
|
||||
return FetchSnapshotResponse.singleton(
|
||||
return RaftUtil.singletonFetchSnapshotResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
log.topicPartition(),
|
||||
quorum.leaderIdOrSentinel(),
|
||||
quorum.leaderEndpoints(),
|
||||
responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot)
|
||||
.setErrorCode(Errors.POSITION_OUT_OF_RANGE.code())
|
||||
);
|
||||
|
@ -1462,10 +1665,20 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
UnalignedRecords records = snapshot.slice(partitionSnapshot.position(), Math.min(data.maxBytes(), maxSnapshotSize));
|
||||
|
||||
LeaderState<T> state = quorum.leaderStateOrThrow();
|
||||
state.updateCheckQuorumForFollowingVoter(data.replicaId(), currentTimeMs);
|
||||
state.updateCheckQuorumForFollowingVoter(
|
||||
ReplicaKey.of(
|
||||
data.replicaId(),
|
||||
partitionSnapshot.replicaDirectoryId()
|
||||
),
|
||||
currentTimeMs
|
||||
);
|
||||
|
||||
return FetchSnapshotResponse.singleton(
|
||||
return RaftUtil.singletonFetchSnapshotResponse(
|
||||
requestMetadata.listenerName(),
|
||||
requestMetadata.apiVersion(),
|
||||
log.topicPartition(),
|
||||
quorum.leaderIdOrSentinel(),
|
||||
quorum.leaderEndpoints(),
|
||||
responsePartitionSnapshot -> {
|
||||
addQuorumLeader(responsePartitionSnapshot)
|
||||
.snapshotId()
|
||||
|
@ -1648,11 +1861,11 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
int epoch,
|
||||
long currentTimeMs
|
||||
) {
|
||||
Optional<Node> leader = leaderId.isPresent() ?
|
||||
partitionState
|
||||
Endpoints leaderEndpoints = leaderId.isPresent() ?
|
||||
leaderEndpoints = partitionState
|
||||
.lastVoterSet()
|
||||
.voterNode(leaderId.getAsInt(), channel.listenerName()) :
|
||||
Optional.empty();
|
||||
.listeners(leaderId.getAsInt()) :
|
||||
Endpoints.empty();
|
||||
|
||||
if (epoch < quorum.epoch() || error == Errors.UNKNOWN_LEADER_EPOCH) {
|
||||
// We have a larger epoch, so the response is no longer relevant
|
||||
|
@ -1663,10 +1876,10 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
// The response indicates that the request had a stale epoch, but we need
|
||||
// to validate the epoch from the response against our current state.
|
||||
maybeTransition(leader, epoch, currentTimeMs);
|
||||
maybeTransition(leaderId, epoch, leaderEndpoints, currentTimeMs);
|
||||
return Optional.of(true);
|
||||
} else if (epoch == quorum.epoch()
|
||||
&& leader.isPresent()
|
||||
&& leaderId.isPresent()
|
||||
&& !quorum.hasLeader()) {
|
||||
|
||||
// Since we are transitioning to Follower, we will only forward the
|
||||
|
@ -1674,7 +1887,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
// the request be retried immediately (if needed) after the transition.
|
||||
// This handling allows an observer to discover the leader and append
|
||||
// to the log in the same Fetch request.
|
||||
transitionToFollower(epoch, leader.get(), currentTimeMs);
|
||||
transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs);
|
||||
if (error == Errors.NONE) {
|
||||
return Optional.empty();
|
||||
} else {
|
||||
|
@ -1696,28 +1909,28 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
private void maybeTransition(
|
||||
Optional<Node> leader,
|
||||
OptionalInt leaderId,
|
||||
int epoch,
|
||||
Endpoints leaderEndpoints,
|
||||
long currentTimeMs
|
||||
) {
|
||||
OptionalInt leaderId = leader.isPresent() ?
|
||||
OptionalInt.of(leader.get().id()) :
|
||||
OptionalInt.empty();
|
||||
|
||||
if (!hasConsistentLeader(epoch, leaderId)) {
|
||||
throw new IllegalStateException("Received request or response with leader " + leader +
|
||||
throw new IllegalStateException("Received request or response with leader " + leaderId +
|
||||
" and epoch " + epoch + " which is inconsistent with current leader " +
|
||||
quorum.leaderId() + " and epoch " + quorum.epoch());
|
||||
} else if (epoch > quorum.epoch()) {
|
||||
if (leader.isPresent()) {
|
||||
transitionToFollower(epoch, leader.get(), currentTimeMs);
|
||||
if (leaderId.isPresent()) {
|
||||
transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs);
|
||||
} else {
|
||||
transitionToUnattached(epoch);
|
||||
}
|
||||
} else if (leader.isPresent() && !quorum.hasLeader()) {
|
||||
// The request or response indicates the leader of the current epoch,
|
||||
// which is currently unknown
|
||||
transitionToFollower(epoch, leader.get(), currentTimeMs);
|
||||
} else if (
|
||||
leaderId.isPresent() &&
|
||||
(!quorum.hasLeader() || leaderEndpoints.size() > quorum.leaderEndpoints().size())
|
||||
) {
|
||||
// The request or response indicates the leader of the current epoch
|
||||
// which are currently unknown or the replica has discovered more endpoints
|
||||
transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1791,6 +2004,17 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true if the voter key matches the local replica's key
|
||||
*/
|
||||
private boolean isValidVoterKey(Optional<ReplicaKey> voterKey) {
|
||||
return voterKey
|
||||
.map(key ->
|
||||
OptionalInt.of(key.id()).equals(nodeId) &&
|
||||
key.directoryId().equals(Optional.of(nodeDirectoryId))
|
||||
)
|
||||
.orElse(true);
|
||||
}
|
||||
/**
|
||||
* Validate a request which is intended for the current quorum leader.
|
||||
* If an error is present in the returned value, it should be returned
|
||||
|
@ -1847,10 +2071,8 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
responseFuture.whenComplete((response, exception) -> {
|
||||
final ApiMessage message;
|
||||
if (response != null) {
|
||||
message = response;
|
||||
} else {
|
||||
ApiMessage message = response;
|
||||
if (message == null) {
|
||||
message = RaftUtil.errorResponse(apiKey, Errors.forException(exception));
|
||||
}
|
||||
|
||||
|
@ -1898,7 +2120,6 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
RaftRequest.Outbound requestMessage = new RaftRequest.Outbound(
|
||||
correlationId,
|
||||
request.highestSupportedVersion(),
|
||||
request,
|
||||
destination,
|
||||
currentTimeMs
|
||||
|
@ -2231,11 +2452,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
// If the current leader is backing off due to some failure or if the
|
||||
// request has timed out, then we attempt to send the Fetch to another
|
||||
// voter in order to discover if there has been a leader change.
|
||||
if (requestManager.hasRequestTimedOut(state.leader(), currentTimeMs)) {
|
||||
Node leaderNode = state.leaderNode(channel.listenerName());
|
||||
if (requestManager.hasRequestTimedOut(leaderNode, currentTimeMs)) {
|
||||
// Once the request has timed out backoff the connection
|
||||
requestManager.reset(state.leader());
|
||||
requestManager.reset(leaderNode);
|
||||
backoffMs = maybeSendAnyVoterFetch(currentTimeMs);
|
||||
} else if (requestManager.isBackingOff(state.leader(), currentTimeMs)) {
|
||||
} else if (requestManager.isBackingOff(leaderNode, currentTimeMs)) {
|
||||
backoffMs = maybeSendAnyVoterFetch(currentTimeMs);
|
||||
} else if (!requestManager.hasAnyInflightRequest(currentTimeMs)) {
|
||||
backoffMs = maybeSendFetchOrFetchSnapshot(state, currentTimeMs);
|
||||
|
@ -2259,7 +2481,11 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
requestSupplier = this::buildFetchRequest;
|
||||
}
|
||||
|
||||
return maybeSendRequest(currentTimeMs, state.leader(), requestSupplier);
|
||||
return maybeSendRequest(
|
||||
currentTimeMs,
|
||||
state.leaderNode(channel.listenerName()),
|
||||
requestSupplier
|
||||
);
|
||||
}
|
||||
|
||||
private long pollVoted(long currentTimeMs) {
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.protocol.ApiMessage;
|
||||
import org.apache.kafka.common.requests.RequestContext;
|
||||
import org.apache.kafka.common.requests.RequestHeader;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.server.fault.FaultHandler;
|
||||
|
@ -101,11 +102,13 @@ public class KafkaRaftClientDriver<T> extends ShutdownableThread {
|
|||
}
|
||||
|
||||
public CompletableFuture<ApiMessage> handleRequest(
|
||||
RequestContext context,
|
||||
RequestHeader header,
|
||||
ApiMessage request,
|
||||
long createdTimeMs
|
||||
) {
|
||||
RaftRequest.Inbound inboundRequest = new RaftRequest.Inbound(
|
||||
context.listenerName,
|
||||
header.correlationId(),
|
||||
header.apiVersion(),
|
||||
request,
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.message.DescribeQuorumResponseData;
|
||||
import org.apache.kafka.common.message.LeaderChangeMessage;
|
||||
import org.apache.kafka.common.message.LeaderChangeMessage.Voter;
|
||||
|
@ -36,13 +35,13 @@ import java.util.Collection;
|
|||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* In the context of LeaderState, an acknowledged voter means one who has acknowledged the current leader by either
|
||||
|
@ -54,15 +53,15 @@ public class LeaderState<T> implements EpochState {
|
|||
static final long OBSERVER_SESSION_TIMEOUT_MS = 300_000L;
|
||||
static final double CHECK_QUORUM_TIMEOUT_FACTOR = 1.5;
|
||||
|
||||
private final int localId;
|
||||
private final Uuid localDirectoryId;
|
||||
private final ReplicaKey localReplicaKey;
|
||||
private final int epoch;
|
||||
private final long epochStartOffset;
|
||||
private final Set<Integer> grantingVoters;
|
||||
private final Endpoints endpoints;
|
||||
|
||||
private Optional<LogOffsetMetadata> highWatermark = Optional.empty();
|
||||
private final Map<Integer, ReplicaState> voterStates = new HashMap<>();
|
||||
private final Map<Integer, ReplicaState> observerStates = new HashMap<>();
|
||||
private Map<Integer, ReplicaState> voterStates = new HashMap<>();
|
||||
private final Map<ReplicaKey, ReplicaState> observerStates = new HashMap<>();
|
||||
private final Logger log;
|
||||
private final BatchAccumulator<T> accumulator;
|
||||
// The set includes all of the followers voters that FETCH or FETCH_SNAPSHOT during the current checkQuorumTimer interval.
|
||||
|
@ -75,24 +74,27 @@ public class LeaderState<T> implements EpochState {
|
|||
|
||||
protected LeaderState(
|
||||
Time time,
|
||||
int localId,
|
||||
Uuid localDirectoryId,
|
||||
ReplicaKey localReplicaKey,
|
||||
int epoch,
|
||||
long epochStartOffset,
|
||||
Map<Integer, VoterSet.VoterNode> voters,
|
||||
VoterSet voters,
|
||||
Set<Integer> grantingVoters,
|
||||
BatchAccumulator<T> accumulator,
|
||||
Endpoints endpoints,
|
||||
int fetchTimeoutMs,
|
||||
LogContext logContext
|
||||
) {
|
||||
this.localId = localId;
|
||||
this.localDirectoryId = localDirectoryId;
|
||||
this.localReplicaKey = localReplicaKey;
|
||||
this.epoch = epoch;
|
||||
this.epochStartOffset = epochStartOffset;
|
||||
this.endpoints = endpoints;
|
||||
|
||||
for (Map.Entry<Integer, VoterSet.VoterNode> voter : voters.entrySet()) {
|
||||
boolean hasAcknowledgedLeader = voter.getKey() == localId;
|
||||
this.voterStates.put(voter.getKey(), new ReplicaState(voter.getKey(), voter.getValue().voterKey().directoryId(), hasAcknowledgedLeader));
|
||||
for (VoterSet.VoterNode voterNode: voters.voterNodes()) {
|
||||
boolean hasAcknowledgedLeader = voterNode.isVoter(localReplicaKey);
|
||||
this.voterStates.put(
|
||||
voterNode.voterKey().id(),
|
||||
new ReplicaState(voterNode.voterKey(), hasAcknowledgedLeader)
|
||||
);
|
||||
}
|
||||
this.grantingVoters = Collections.unmodifiableSet(new HashSet<>(grantingVoters));
|
||||
this.log = logContext.logger(LeaderState.class);
|
||||
|
@ -104,7 +106,9 @@ public class LeaderState<T> implements EpochState {
|
|||
|
||||
/**
|
||||
* Get the remaining time in milliseconds until the checkQuorumTimer expires.
|
||||
* This will happen if we didn't receive a valid fetch/fetchSnapshot request from the majority of the voters within checkQuorumTimeoutMs.
|
||||
*
|
||||
* This will happen if we didn't receive a valid fetch/fetchSnapshot request from the majority
|
||||
* of the voters within checkQuorumTimeoutMs.
|
||||
*
|
||||
* @param currentTimeMs the current timestamp in millisecond
|
||||
* @return the remainingMs before the checkQuorumTimer expired
|
||||
|
@ -118,10 +122,12 @@ public class LeaderState<T> implements EpochState {
|
|||
long remainingMs = checkQuorumTimer.remainingMs();
|
||||
if (remainingMs == 0) {
|
||||
log.info(
|
||||
"Did not receive fetch request from the majority of the voters within {}ms. Current fetched voters are {}, and voters are {}",
|
||||
"Did not receive fetch request from the majority of the voters within {}ms. " +
|
||||
"Current fetched voters are {}, and voters are {}",
|
||||
checkQuorumTimeoutMs,
|
||||
fetchedVoters,
|
||||
voterStates.keySet());
|
||||
voterStates.values().stream().map(voter -> voter.replicaKey)
|
||||
);
|
||||
}
|
||||
return remainingMs;
|
||||
}
|
||||
|
@ -132,15 +138,15 @@ public class LeaderState<T> implements EpochState {
|
|||
* @param id the node id
|
||||
* @param currentTimeMs the current timestamp in millisecond
|
||||
*/
|
||||
public void updateCheckQuorumForFollowingVoter(int id, long currentTimeMs) {
|
||||
updateFetchedVoters(id);
|
||||
public void updateCheckQuorumForFollowingVoter(ReplicaKey replicaKey, long currentTimeMs) {
|
||||
updateFetchedVoters(replicaKey);
|
||||
// The majority number of the voters. Ex: 2 for 3 voters, 3 for 4 voters... etc.
|
||||
int majority = (voterStates.size() / 2) + 1;
|
||||
// If the leader is in the voter set, it should be implicitly counted as part of the
|
||||
// majority, but the leader will never be a member of the fetchedVoters.
|
||||
// If the leader is not in the voter set, it is not in the majority. Then, the
|
||||
// majority can only be composed of fetched voters.
|
||||
if (voterStates.containsKey(localId)) {
|
||||
if (voterStates.containsKey(localReplicaKey.id())) {
|
||||
majority = majority - 1;
|
||||
}
|
||||
|
||||
|
@ -151,13 +157,14 @@ public class LeaderState<T> implements EpochState {
|
|||
}
|
||||
}
|
||||
|
||||
private void updateFetchedVoters(int id) {
|
||||
if (id == localId) {
|
||||
private void updateFetchedVoters(ReplicaKey replicaKey) {
|
||||
if (replicaKey.id() == localReplicaKey.id()) {
|
||||
throw new IllegalArgumentException("Received a FETCH/FETCH_SNAPSHOT request from the leader itself.");
|
||||
}
|
||||
|
||||
if (isVoter(id)) {
|
||||
fetchedVoters.add(id);
|
||||
ReplicaState state = voterStates.get(replicaKey.id());
|
||||
if (state != null && state.matchesKey(replicaKey)) {
|
||||
fetchedVoters.add(replicaKey.id());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -200,7 +207,7 @@ public class LeaderState<T> implements EpochState {
|
|||
|
||||
@Override
|
||||
public ElectionState election() {
|
||||
return ElectionState.withElectedLeader(epoch, localId, voterStates.keySet());
|
||||
return ElectionState.withElectedLeader(epoch, localReplicaKey.id(), voterStates.keySet());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -208,30 +215,28 @@ public class LeaderState<T> implements EpochState {
|
|||
return epoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoints leaderEndpoints() {
|
||||
return endpoints;
|
||||
}
|
||||
|
||||
public Set<Integer> grantingVoters() {
|
||||
return this.grantingVoters;
|
||||
}
|
||||
|
||||
public int localId() {
|
||||
return localId;
|
||||
}
|
||||
|
||||
public Uuid localDirectoryId() {
|
||||
return localDirectoryId;
|
||||
}
|
||||
|
||||
public Set<Integer> nonAcknowledgingVoters() {
|
||||
Set<Integer> nonAcknowledging = new HashSet<>();
|
||||
for (ReplicaState state : voterStates.values()) {
|
||||
if (!state.hasAcknowledgedLeader)
|
||||
nonAcknowledging.add(state.nodeId);
|
||||
nonAcknowledging.add(state.replicaKey.id());
|
||||
}
|
||||
return nonAcknowledging;
|
||||
}
|
||||
|
||||
private boolean maybeUpdateHighWatermark() {
|
||||
// Find the largest offset which is replicated to a majority of replicas (the leader counts)
|
||||
List<ReplicaState> followersByDescendingFetchOffset = followersByDescendingFetchOffset();
|
||||
ArrayList<ReplicaState> followersByDescendingFetchOffset = followersByDescendingFetchOffset()
|
||||
.collect(Collectors.toCollection(ArrayList::new));
|
||||
|
||||
int indexOfHw = voterStates.size() / 2;
|
||||
Optional<LogOffsetMetadata> highWatermarkUpdateOpt = followersByDescendingFetchOffset.get(indexOfHw).endOffset;
|
||||
|
@ -323,17 +328,19 @@ public class LeaderState<T> implements EpochState {
|
|||
*/
|
||||
public boolean updateLocalState(
|
||||
LogOffsetMetadata endOffsetMetadata,
|
||||
Map<Integer, VoterSet.VoterNode> lastVoters
|
||||
VoterSet lastVoterSet
|
||||
) {
|
||||
ReplicaState state = getOrCreateReplicaState(localId, localDirectoryId);
|
||||
ReplicaState state = getOrCreateReplicaState(localReplicaKey);
|
||||
state.endOffset.ifPresent(currentEndOffset -> {
|
||||
if (currentEndOffset.offset > endOffsetMetadata.offset) {
|
||||
throw new IllegalStateException("Detected non-monotonic update of local " +
|
||||
"end offset: " + currentEndOffset.offset + " -> " + endOffsetMetadata.offset);
|
||||
}
|
||||
});
|
||||
|
||||
state.updateLeaderEndOffset(endOffsetMetadata);
|
||||
updateVoterAndObserverStates(lastVoters);
|
||||
updateVoterAndObserverStates(lastVoterSet);
|
||||
|
||||
return maybeUpdateHighWatermark();
|
||||
}
|
||||
|
||||
|
@ -347,50 +354,53 @@ public class LeaderState<T> implements EpochState {
|
|||
* @return true if the high watermark is updated as a result of this call
|
||||
*/
|
||||
public boolean updateReplicaState(
|
||||
int replicaId,
|
||||
Uuid replicaDirectoryId,
|
||||
ReplicaKey replicaKey,
|
||||
long currentTimeMs,
|
||||
LogOffsetMetadata fetchOffsetMetadata
|
||||
) {
|
||||
// Ignore fetches from negative replica id, as it indicates
|
||||
// the fetch is from non-replica. For example, a consumer.
|
||||
if (replicaId < 0) {
|
||||
if (replicaKey.id() < 0) {
|
||||
return false;
|
||||
} else if (replicaId == localId) {
|
||||
throw new IllegalStateException("Remote replica ID " + replicaId + " matches the local leader ID");
|
||||
} else if (replicaKey.id() == localReplicaKey.id()) {
|
||||
throw new IllegalStateException(
|
||||
String.format("Remote replica ID %s matches the local leader ID", replicaKey)
|
||||
);
|
||||
}
|
||||
|
||||
ReplicaState state = getOrCreateReplicaState(replicaId, replicaDirectoryId);
|
||||
ReplicaState state = getOrCreateReplicaState(replicaKey);
|
||||
|
||||
state.endOffset.ifPresent(currentEndOffset -> {
|
||||
if (currentEndOffset.offset > fetchOffsetMetadata.offset) {
|
||||
log.warn("Detected non-monotonic update of fetch offset from nodeId {}: {} -> {}",
|
||||
state.nodeId, currentEndOffset.offset, fetchOffsetMetadata.offset);
|
||||
state.replicaKey, currentEndOffset.offset, fetchOffsetMetadata.offset);
|
||||
}
|
||||
});
|
||||
Optional<LogOffsetMetadata> leaderEndOffsetOpt = getOrCreateReplicaState(localId, localDirectoryId).endOffset;
|
||||
|
||||
Optional<LogOffsetMetadata> leaderEndOffsetOpt = getOrCreateReplicaState(localReplicaKey).endOffset;
|
||||
|
||||
state.updateFollowerState(
|
||||
currentTimeMs,
|
||||
fetchOffsetMetadata,
|
||||
leaderEndOffsetOpt
|
||||
);
|
||||
updateCheckQuorumForFollowingVoter(replicaId, currentTimeMs);
|
||||
updateCheckQuorumForFollowingVoter(replicaKey, currentTimeMs);
|
||||
|
||||
return isVoter(state.nodeId) && maybeUpdateHighWatermark();
|
||||
return isVoter(state.replicaKey) && maybeUpdateHighWatermark();
|
||||
}
|
||||
|
||||
public List<Integer> nonLeaderVotersByDescendingFetchOffset() {
|
||||
return followersByDescendingFetchOffset().stream()
|
||||
.filter(state -> state.nodeId != localId)
|
||||
.map(state -> state.nodeId)
|
||||
return followersByDescendingFetchOffset()
|
||||
.filter(state -> !state.matchesKey(localReplicaKey))
|
||||
.map(state -> state.replicaKey.id())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private List<ReplicaState> followersByDescendingFetchOffset() {
|
||||
return new ArrayList<>(this.voterStates.values()).stream()
|
||||
.sorted()
|
||||
.collect(Collectors.toList());
|
||||
private Stream<ReplicaState> followersByDescendingFetchOffset() {
|
||||
return voterStates
|
||||
.values()
|
||||
.stream()
|
||||
.sorted();
|
||||
}
|
||||
|
||||
public void addAcknowledgementFrom(int remoteNodeId) {
|
||||
|
@ -400,8 +410,9 @@ public class LeaderState<T> implements EpochState {
|
|||
|
||||
private ReplicaState ensureValidVoter(int remoteNodeId) {
|
||||
ReplicaState state = voterStates.get(remoteNodeId);
|
||||
if (state == null)
|
||||
if (state == null) {
|
||||
throw new IllegalArgumentException("Unexpected acknowledgement from non-voter " + remoteNodeId);
|
||||
}
|
||||
return state;
|
||||
}
|
||||
|
||||
|
@ -409,55 +420,58 @@ public class LeaderState<T> implements EpochState {
|
|||
return epochStartOffset;
|
||||
}
|
||||
|
||||
private ReplicaState getOrCreateReplicaState(int remoteNodeId, Uuid remoteNodeDirectory) {
|
||||
ReplicaState state = voterStates.get(remoteNodeId);
|
||||
if (state == null) {
|
||||
observerStates.putIfAbsent(remoteNodeId, new ReplicaState(remoteNodeId, Optional.of(remoteNodeDirectory), false));
|
||||
return observerStates.get(remoteNodeId);
|
||||
private ReplicaState getOrCreateReplicaState(ReplicaKey replicaKey) {
|
||||
ReplicaState state = voterStates.get(replicaKey.id());
|
||||
if (state == null || !state.matchesKey(replicaKey)) {
|
||||
observerStates.putIfAbsent(replicaKey, new ReplicaState(replicaKey, false));
|
||||
return observerStates.get(replicaKey);
|
||||
}
|
||||
return state;
|
||||
}
|
||||
|
||||
private Optional<ReplicaState> getReplicaState(ReplicaKey replicaKey) {
|
||||
ReplicaState state = voterStates.get(replicaKey.id());
|
||||
if (state == null || !state.matchesKey(replicaKey)) {
|
||||
state = observerStates.get(replicaKey);
|
||||
}
|
||||
|
||||
return Optional.ofNullable(state);
|
||||
}
|
||||
|
||||
public DescribeQuorumResponseData.PartitionData describeQuorum(long currentTimeMs) {
|
||||
clearInactiveObservers(currentTimeMs);
|
||||
|
||||
return new DescribeQuorumResponseData.PartitionData()
|
||||
.setErrorCode(Errors.NONE.code())
|
||||
.setLeaderId(localId)
|
||||
.setLeaderId(localReplicaKey.id())
|
||||
.setLeaderEpoch(epoch)
|
||||
.setHighWatermark(highWatermark.map(offsetMetadata -> offsetMetadata.offset).orElse(-1L))
|
||||
.setCurrentVoters(describeReplicaStates(voterStates, currentTimeMs))
|
||||
.setObservers(describeReplicaStates(observerStates, currentTimeMs));
|
||||
.setCurrentVoters(describeReplicaStates(voterStates.values(), currentTimeMs))
|
||||
.setObservers(describeReplicaStates(observerStates.values(), currentTimeMs));
|
||||
}
|
||||
|
||||
public DescribeQuorumResponseData.NodeCollection nodes(long currentTimeMs) {
|
||||
clearInactiveObservers(currentTimeMs);
|
||||
|
||||
return nodes(voterStates.values(), observerStates.values());
|
||||
}
|
||||
DescribeQuorumResponseData.NodeCollection nodes = new DescribeQuorumResponseData.NodeCollection();
|
||||
|
||||
private static DescribeQuorumResponseData.NodeCollection nodes(Collection<ReplicaState> voters, Collection<ReplicaState> observers) {
|
||||
DescribeQuorumResponseData.NodeCollection res = new DescribeQuorumResponseData.NodeCollection();
|
||||
voterStates.values().forEach(replicaState -> {
|
||||
if (nodes.find(replicaState.replicaKey.id()) == null) {
|
||||
// KAFKA-16953 will add support for including the node listeners in the node
|
||||
// collection
|
||||
nodes.add(new DescribeQuorumResponseData.Node().setNodeId(replicaState.replicaKey.id()));
|
||||
}
|
||||
});
|
||||
|
||||
voters.forEach(replicaState -> node(res, replicaState));
|
||||
observers.forEach(replicaState -> node(res, replicaState));
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
private static void node(DescribeQuorumResponseData.NodeCollection res, ReplicaState replicaState) {
|
||||
if (res.find(replicaState.nodeId) != null) {
|
||||
return;
|
||||
}
|
||||
|
||||
res.add(new DescribeQuorumResponseData.Node().setNodeId(replicaState.nodeId));
|
||||
return nodes;
|
||||
}
|
||||
|
||||
private List<DescribeQuorumResponseData.ReplicaState> describeReplicaStates(
|
||||
Map<Integer, ReplicaState> state,
|
||||
Collection<ReplicaState> states,
|
||||
long currentTimeMs
|
||||
) {
|
||||
return state.values().stream()
|
||||
return states
|
||||
.stream()
|
||||
.map(replicaState -> describeReplicaState(replicaState, currentTimeMs))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
@ -468,7 +482,7 @@ public class LeaderState<T> implements EpochState {
|
|||
) {
|
||||
final long lastCaughtUpTimestamp;
|
||||
final long lastFetchTimestamp;
|
||||
if (replicaState.nodeId == localId) {
|
||||
if (replicaState.matchesKey(localReplicaKey)) {
|
||||
lastCaughtUpTimestamp = currentTimeMs;
|
||||
lastFetchTimestamp = currentTimeMs;
|
||||
} else {
|
||||
|
@ -476,8 +490,8 @@ public class LeaderState<T> implements EpochState {
|
|||
lastFetchTimestamp = replicaState.lastFetchTimestamp;
|
||||
}
|
||||
return new DescribeQuorumResponseData.ReplicaState()
|
||||
.setReplicaId(replicaState.nodeId)
|
||||
.setReplicaDirectoryId(replicaState.nodeDirectory.orElse(Uuid.ZERO_UUID))
|
||||
.setReplicaId(replicaState.replicaKey.id())
|
||||
.setReplicaDirectoryId(replicaState.replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID))
|
||||
.setLogEndOffset(replicaState.endOffset.map(md -> md.offset).orElse(-1L))
|
||||
.setLastCaughtUpTimestamp(lastCaughtUpTimestamp)
|
||||
.setLastFetchTimestamp(lastFetchTimestamp);
|
||||
|
@ -490,46 +504,50 @@ public class LeaderState<T> implements EpochState {
|
|||
private void clearInactiveObservers(final long currentTimeMs) {
|
||||
observerStates.entrySet().removeIf(integerReplicaStateEntry ->
|
||||
currentTimeMs - integerReplicaStateEntry.getValue().lastFetchTimestamp >= OBSERVER_SESSION_TIMEOUT_MS &&
|
||||
integerReplicaStateEntry.getKey() != localId
|
||||
!integerReplicaStateEntry.getKey().equals(localReplicaKey)
|
||||
);
|
||||
}
|
||||
|
||||
private boolean isVoter(int remoteNodeId) {
|
||||
return voterStates.containsKey(remoteNodeId);
|
||||
private boolean isVoter(ReplicaKey remoteReplicaKey) {
|
||||
ReplicaState state = voterStates.get(remoteReplicaKey.id());
|
||||
return state != null && state.matchesKey(remoteReplicaKey);
|
||||
}
|
||||
|
||||
private void updateVoterAndObserverStates(Map<Integer, VoterSet.VoterNode> lastVoters) {
|
||||
// Move any replica that is not in the last voter set from voterStates to observerStates
|
||||
for (Iterator<Map.Entry<Integer, ReplicaState>> iter = voterStates.entrySet().iterator(); iter.hasNext(); ) {
|
||||
Map.Entry<Integer, ReplicaState> replica = iter.next();
|
||||
if (!lastVoters.containsKey(replica.getKey())) {
|
||||
observerStates.put(replica.getKey(), replica.getValue());
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
private void updateVoterAndObserverStates(VoterSet lastVoterSet) {
|
||||
Map<Integer, ReplicaState> newVoterStates = new HashMap<>();
|
||||
Map<Integer, ReplicaState> oldVoterStates = new HashMap<>(voterStates);
|
||||
|
||||
// Add replicas that are in the last voter set and not in voterStates to voterStates (from observerStates
|
||||
// if they exist)
|
||||
for (Map.Entry<Integer, VoterSet.VoterNode> voter : lastVoters.entrySet()) {
|
||||
if (!voterStates.containsKey(voter.getKey())) {
|
||||
Optional<ReplicaState> existingObserverState = Optional.ofNullable(observerStates.remove(voter.getKey()));
|
||||
voterStates.put(voter.getKey(), existingObserverState.orElse(new ReplicaState(voter.getKey(), voter.getValue().voterKey().directoryId(), false)));
|
||||
}
|
||||
// Compute the new voter states map
|
||||
for (VoterSet.VoterNode voterNode : lastVoterSet.voterNodes()) {
|
||||
ReplicaState state = getReplicaState(voterNode.voterKey())
|
||||
.orElse(new ReplicaState(voterNode.voterKey(), false));
|
||||
|
||||
// Remove the voter from the previous data structures
|
||||
oldVoterStates.remove(voterNode.voterKey().id());
|
||||
observerStates.remove(voterNode.voterKey());
|
||||
|
||||
// Make sure that the replica key in the replica state matches the voter's
|
||||
state.setReplicaKey(voterNode.voterKey());
|
||||
newVoterStates.put(state.replicaKey.id(), state);
|
||||
}
|
||||
voterStates = newVoterStates;
|
||||
|
||||
// Move any of the remaining old voters to observerStates
|
||||
for (ReplicaState replicaStateEntry : oldVoterStates.values()) {
|
||||
observerStates.putIfAbsent(replicaStateEntry.replicaKey, replicaStateEntry);
|
||||
}
|
||||
}
|
||||
|
||||
private static class ReplicaState implements Comparable<ReplicaState> {
|
||||
final int nodeId;
|
||||
final Optional<Uuid> nodeDirectory;
|
||||
ReplicaKey replicaKey;
|
||||
Optional<LogOffsetMetadata> endOffset;
|
||||
long lastFetchTimestamp;
|
||||
long lastFetchLeaderLogEndOffset;
|
||||
long lastCaughtUpTimestamp;
|
||||
boolean hasAcknowledgedLeader;
|
||||
|
||||
public ReplicaState(int nodeId, Optional<Uuid> nodeDirectory, boolean hasAcknowledgedLeader) {
|
||||
this.nodeId = nodeId;
|
||||
this.nodeDirectory = nodeDirectory;
|
||||
public ReplicaState(ReplicaKey replicaKey, boolean hasAcknowledgedLeader) {
|
||||
this.replicaKey = replicaKey;
|
||||
this.endOffset = Optional.empty();
|
||||
this.lastFetchTimestamp = -1;
|
||||
this.lastFetchLeaderLogEndOffset = -1;
|
||||
|
@ -537,6 +555,41 @@ public class LeaderState<T> implements EpochState {
|
|||
this.hasAcknowledgedLeader = hasAcknowledgedLeader;
|
||||
}
|
||||
|
||||
void setReplicaKey(ReplicaKey replicaKey) {
|
||||
if (this.replicaKey.id() != replicaKey.id()) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
"Attempting to update the replica key %s with a different replica id %s",
|
||||
this.replicaKey,
|
||||
replicaKey
|
||||
)
|
||||
);
|
||||
} else if (this.replicaKey.directoryId().isPresent() &&
|
||||
!this.replicaKey.equals(replicaKey)
|
||||
) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
"Attempting to update an already set directory id %s with a different directory id %s",
|
||||
this.replicaKey,
|
||||
replicaKey
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
this.replicaKey = replicaKey;
|
||||
}
|
||||
|
||||
boolean matchesKey(ReplicaKey replicaKey) {
|
||||
if (this.replicaKey.id() != replicaKey.id()) return false;
|
||||
|
||||
if (this.replicaKey.directoryId().isPresent()) {
|
||||
return this.replicaKey.directoryId().equals(replicaKey.directoryId());
|
||||
} else {
|
||||
// it doesn't include a directory id so it matches as long as the ids match
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
void updateLeaderEndOffset(
|
||||
LogOffsetMetadata endOffsetMetadata
|
||||
) {
|
||||
|
@ -571,7 +624,7 @@ public class LeaderState<T> implements EpochState {
|
|||
@Override
|
||||
public int compareTo(ReplicaState that) {
|
||||
if (this.endOffset.equals(that.endOffset))
|
||||
return Integer.compare(this.nodeId, that.nodeId);
|
||||
return this.replicaKey.compareTo(that.replicaKey);
|
||||
else if (!this.endOffset.isPresent())
|
||||
return 1;
|
||||
else if (!that.endOffset.isPresent())
|
||||
|
@ -583,10 +636,9 @@ public class LeaderState<T> implements EpochState {
|
|||
@Override
|
||||
public String toString() {
|
||||
return String.format(
|
||||
"ReplicaState(nodeId=%d, nodeDirectoryId=%s, endOffset=%s, lastFetchTimestamp=%s, " +
|
||||
"lastCaughtUpTimestamp=%s, hasAcknowledgedLeader=%s)",
|
||||
nodeId,
|
||||
nodeDirectory,
|
||||
"ReplicaState(replicaKey=%s, endOffset=%s, lastFetchTimestamp=%s, " +
|
||||
"lastCaughtUpTimestamp=%s, hasAcknowledgedLeader=%s)",
|
||||
replicaKey,
|
||||
endOffset,
|
||||
lastFetchTimestamp,
|
||||
lastCaughtUpTimestamp,
|
||||
|
@ -608,9 +660,8 @@ public class LeaderState<T> implements EpochState {
|
|||
@Override
|
||||
public String toString() {
|
||||
return String.format(
|
||||
"Leader(localId=%d, localDirectoryId=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)",
|
||||
localId,
|
||||
localDirectoryId,
|
||||
"Leader(localReplicaKey=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)",
|
||||
localReplicaKey,
|
||||
epoch,
|
||||
epochStartOffset,
|
||||
highWatermark,
|
||||
|
@ -627,5 +678,4 @@ public class LeaderState<T> implements EpochState {
|
|||
public void close() {
|
||||
accumulator.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,9 +16,7 @@
|
|||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.raft.internals.BatchAccumulator;
|
||||
|
@ -84,9 +82,9 @@ public class QuorumState {
|
|||
private final Time time;
|
||||
private final Logger log;
|
||||
private final QuorumStateStore store;
|
||||
private final ListenerName listenerName;
|
||||
private final Supplier<VoterSet> latestVoterSet;
|
||||
private final Supplier<Short> latestKraftVersion;
|
||||
private final Endpoints localListeners;
|
||||
private final Random random;
|
||||
private final int electionTimeoutMs;
|
||||
private final int fetchTimeoutMs;
|
||||
|
@ -97,9 +95,9 @@ public class QuorumState {
|
|||
public QuorumState(
|
||||
OptionalInt localId,
|
||||
Uuid localDirectoryId,
|
||||
ListenerName listenerName,
|
||||
Supplier<VoterSet> latestVoterSet,
|
||||
Supplier<Short> latestKraftVersion,
|
||||
Endpoints localListeners,
|
||||
int electionTimeoutMs,
|
||||
int fetchTimeoutMs,
|
||||
QuorumStateStore store,
|
||||
|
@ -109,9 +107,9 @@ public class QuorumState {
|
|||
) {
|
||||
this.localId = localId;
|
||||
this.localDirectoryId = localDirectoryId;
|
||||
this.listenerName = listenerName;
|
||||
this.latestVoterSet = latestVoterSet;
|
||||
this.latestKraftVersion = latestKraftVersion;
|
||||
this.localListeners = localListeners;
|
||||
this.electionTimeoutMs = electionTimeoutMs;
|
||||
this.fetchTimeoutMs = fetchTimeoutMs;
|
||||
this.store = store;
|
||||
|
@ -174,11 +172,12 @@ public class QuorumState {
|
|||
latestVoterSet.get().voterIds(),
|
||||
randomElectionTimeoutMs(),
|
||||
Collections.emptyList(),
|
||||
localListeners,
|
||||
logContext
|
||||
);
|
||||
} else if (
|
||||
localId.isPresent() &&
|
||||
election.isVotedCandidate(ReplicaKey.of(localId.getAsInt(), Optional.of(localDirectoryId)))
|
||||
election.isVotedCandidate(ReplicaKey.of(localId.getAsInt(), localDirectoryId))
|
||||
) {
|
||||
initialState = new CandidateState(
|
||||
time,
|
||||
|
@ -202,27 +201,13 @@ public class QuorumState {
|
|||
logContext
|
||||
);
|
||||
} else if (election.hasLeader()) {
|
||||
/* KAFKA-16529 is going to change this so that the leader is not required to be in the set
|
||||
* of voters. In other words, don't throw an IllegalStateException if the leader is not in
|
||||
* the set of voters.
|
||||
*/
|
||||
Node leader = latestVoterSet
|
||||
.get()
|
||||
.voterNode(election.leaderId(), listenerName)
|
||||
.orElseThrow(() ->
|
||||
new IllegalStateException(
|
||||
String.format(
|
||||
"Leader %s must be in the voter set %s",
|
||||
election.leaderId(),
|
||||
latestVoterSet.get()
|
||||
)
|
||||
)
|
||||
);
|
||||
VoterSet voters = latestVoterSet.get();
|
||||
initialState = new FollowerState(
|
||||
time,
|
||||
election.epoch(),
|
||||
leader,
|
||||
latestVoterSet.get().voterIds(),
|
||||
election.leaderId(),
|
||||
voters.listeners(election.leaderId()),
|
||||
voters.voterIds(),
|
||||
Optional.empty(),
|
||||
fetchTimeoutMs,
|
||||
logContext
|
||||
|
@ -243,9 +228,7 @@ public class QuorumState {
|
|||
|
||||
public boolean isOnlyVoter() {
|
||||
return localId.isPresent() &&
|
||||
latestVoterSet.get().isOnlyVoter(
|
||||
ReplicaKey.of(localId.getAsInt(), Optional.of(localDirectoryId))
|
||||
);
|
||||
latestVoterSet.get().isOnlyVoter(ReplicaKey.of(localId.getAsInt(), localDirectoryId));
|
||||
}
|
||||
|
||||
public int localIdOrSentinel() {
|
||||
|
@ -277,7 +260,6 @@ public class QuorumState {
|
|||
}
|
||||
|
||||
public OptionalInt leaderId() {
|
||||
|
||||
ElectionState election = state.election();
|
||||
if (election.hasLeader())
|
||||
return OptionalInt.of(state.election().leaderId());
|
||||
|
@ -293,6 +275,10 @@ public class QuorumState {
|
|||
return hasLeader() && leaderIdOrSentinel() != localIdOrSentinel();
|
||||
}
|
||||
|
||||
public Endpoints leaderEndpoints() {
|
||||
return state.leaderEndpoints();
|
||||
}
|
||||
|
||||
public boolean isVoter() {
|
||||
if (!localId.isPresent()) {
|
||||
return false;
|
||||
|
@ -300,7 +286,7 @@ public class QuorumState {
|
|||
|
||||
return latestVoterSet
|
||||
.get()
|
||||
.isVoter(ReplicaKey.of(localId.getAsInt(), Optional.of(localDirectoryId)));
|
||||
.isVoter(ReplicaKey.of(localId.getAsInt(), localDirectoryId));
|
||||
}
|
||||
|
||||
public boolean isVoter(ReplicaKey nodeKey) {
|
||||
|
@ -327,6 +313,7 @@ public class QuorumState {
|
|||
latestVoterSet.get().voterIds(),
|
||||
randomElectionTimeoutMs(),
|
||||
preferredSuccessors,
|
||||
localListeners,
|
||||
logContext
|
||||
)
|
||||
);
|
||||
|
@ -427,16 +414,16 @@ public class QuorumState {
|
|||
/**
|
||||
* Become a follower of an elected leader so that we can begin fetching.
|
||||
*/
|
||||
public void transitionToFollower(int epoch, Node leader) {
|
||||
public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) {
|
||||
int currentEpoch = state.epoch();
|
||||
if (localId.isPresent() && leader.id() == localId.getAsInt()) {
|
||||
throw new IllegalStateException("Cannot transition to Follower with leader " + leader +
|
||||
if (localId.isPresent() && leaderId == localId.getAsInt()) {
|
||||
throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId +
|
||||
" and epoch " + epoch + " since it matches the local broker.id " + localId);
|
||||
} else if (epoch < currentEpoch) {
|
||||
throw new IllegalStateException("Cannot transition to Follower with leader " + leader +
|
||||
throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId +
|
||||
" and epoch " + epoch + " since the current epoch " + currentEpoch + " is larger");
|
||||
} else if (epoch == currentEpoch && (isFollower() || isLeader())) {
|
||||
throw new IllegalStateException("Cannot transition to Follower with leader " + leader +
|
||||
throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId +
|
||||
" and epoch " + epoch + " from state " + state);
|
||||
}
|
||||
|
||||
|
@ -444,7 +431,8 @@ public class QuorumState {
|
|||
new FollowerState(
|
||||
time,
|
||||
epoch,
|
||||
leader,
|
||||
leaderId,
|
||||
endpoints,
|
||||
latestVoterSet.get().voterIds(),
|
||||
state.highWatermark(),
|
||||
fetchTimeoutMs,
|
||||
|
@ -518,13 +506,13 @@ public class QuorumState {
|
|||
|
||||
LeaderState<T> state = new LeaderState<>(
|
||||
time,
|
||||
localIdOrThrow(),
|
||||
localDirectoryId(),
|
||||
ReplicaKey.of(localIdOrThrow(), localDirectoryId),
|
||||
epoch(),
|
||||
epochStartOffset,
|
||||
latestVoterSet.get().voters(),
|
||||
latestVoterSet.get(),
|
||||
candidateState.grantingVoters(),
|
||||
accumulator,
|
||||
localListeners,
|
||||
fetchTimeoutMs,
|
||||
logContext
|
||||
);
|
||||
|
|
|
@ -21,7 +21,5 @@ import org.apache.kafka.common.protocol.ApiMessage;
|
|||
public interface RaftMessage {
|
||||
int correlationId();
|
||||
|
||||
short apiVersion();
|
||||
|
||||
ApiMessage data();
|
||||
}
|
||||
|
|
|
@ -17,19 +17,18 @@
|
|||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.protocol.ApiMessage;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
public abstract class RaftRequest implements RaftMessage {
|
||||
private final int correlationId;
|
||||
private final short apiVersion;
|
||||
private final ApiMessage data;
|
||||
private final long createdTimeMs;
|
||||
|
||||
public RaftRequest(int correlationId, short apiVersion, ApiMessage data, long createdTimeMs) {
|
||||
public RaftRequest(int correlationId, ApiMessage data, long createdTimeMs) {
|
||||
this.correlationId = correlationId;
|
||||
this.apiVersion = apiVersion;
|
||||
this.data = data;
|
||||
this.createdTimeMs = createdTimeMs;
|
||||
}
|
||||
|
@ -39,11 +38,6 @@ public abstract class RaftRequest implements RaftMessage {
|
|||
return correlationId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short apiVersion() {
|
||||
return apiVersion;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApiMessage data() {
|
||||
return data;
|
||||
|
@ -54,17 +48,40 @@ public abstract class RaftRequest implements RaftMessage {
|
|||
}
|
||||
|
||||
public static final class Inbound extends RaftRequest {
|
||||
private final short apiVersion;
|
||||
private final ListenerName listenerName;
|
||||
|
||||
public final CompletableFuture<RaftResponse.Outbound> completion = new CompletableFuture<>();
|
||||
|
||||
public Inbound(int correlationId, short apiVersion, ApiMessage data, long createdTimeMs) {
|
||||
super(correlationId, apiVersion, data, createdTimeMs);
|
||||
public Inbound(
|
||||
ListenerName listenerName,
|
||||
int correlationId,
|
||||
short apiVersion,
|
||||
ApiMessage data,
|
||||
long createdTimeMs
|
||||
) {
|
||||
super(correlationId, data, createdTimeMs);
|
||||
|
||||
this.listenerName = listenerName;
|
||||
this.apiVersion = apiVersion;
|
||||
}
|
||||
|
||||
public short apiVersion() {
|
||||
return apiVersion;
|
||||
}
|
||||
|
||||
public ListenerName listenerName() {
|
||||
return listenerName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format(
|
||||
"InboundRequest(correlationId=%d, data=%s, createdTimeMs=%d)",
|
||||
"InboundRequest(listenerName=%s, correlationId=%d, apiVersion=%d, data=%s, " +
|
||||
"createdTimeMs=%d)",
|
||||
listenerName,
|
||||
correlationId(),
|
||||
apiVersion,
|
||||
data(),
|
||||
createdTimeMs()
|
||||
);
|
||||
|
@ -75,8 +92,8 @@ public abstract class RaftRequest implements RaftMessage {
|
|||
private final Node destination;
|
||||
public final CompletableFuture<RaftResponse.Inbound> completion = new CompletableFuture<>();
|
||||
|
||||
public Outbound(int correlationId, short apiVersion, ApiMessage data, Node destination, long createdTimeMs) {
|
||||
super(correlationId, apiVersion, data, createdTimeMs);
|
||||
public Outbound(int correlationId, ApiMessage data, Node destination, long createdTimeMs) {
|
||||
super(correlationId, data, createdTimeMs);
|
||||
this.destination = destination;
|
||||
}
|
||||
|
||||
|
|
|
@ -33,11 +33,6 @@ public abstract class RaftResponse implements RaftMessage {
|
|||
return correlationId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short apiVersion() {
|
||||
return data().highestSupportedVersion();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApiMessage data() {
|
||||
return data;
|
||||
|
|
|
@ -21,22 +21,30 @@ import org.apache.kafka.common.Uuid;
|
|||
import org.apache.kafka.common.message.BeginQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.message.DescribeQuorumRequestData;
|
||||
import org.apache.kafka.common.message.DescribeQuorumResponseData;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.message.FetchRequestData;
|
||||
import org.apache.kafka.common.message.FetchResponseData;
|
||||
import org.apache.kafka.common.message.FetchSnapshotRequestData;
|
||||
import org.apache.kafka.common.message.FetchSnapshotResponseData;
|
||||
import org.apache.kafka.common.message.VoteRequestData;
|
||||
import org.apache.kafka.common.message.VoteResponseData;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.ApiMessage;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.raft.internals.ReplicaKey;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.UnaryOperator;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static java.util.Collections.singletonList;
|
||||
|
||||
@SuppressWarnings({ "ClassDataAbstractionCoupling", "ClassFanOutComplexity" })
|
||||
public class RaftUtil {
|
||||
|
||||
public static ApiMessage errorResponse(ApiKeys apiKey, Errors error) {
|
||||
|
@ -70,16 +78,20 @@ public class RaftUtil {
|
|||
new FetchRequestData.FetchTopic()
|
||||
.setTopic(topicPartition.topic())
|
||||
.setTopicId(topicId)
|
||||
.setPartitions(singletonList(fetchPartition));
|
||||
.setPartitions(Collections.singletonList(fetchPartition));
|
||||
|
||||
return new FetchRequestData()
|
||||
.setTopics(singletonList(fetchTopic));
|
||||
.setTopics(Collections.singletonList(fetchTopic));
|
||||
}
|
||||
|
||||
public static FetchResponseData singletonFetchResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
TopicPartition topicPartition,
|
||||
Uuid topicId,
|
||||
Errors topLevelError,
|
||||
int leaderId,
|
||||
Endpoints endpoints,
|
||||
Consumer<FetchResponseData.PartitionData> partitionConsumer
|
||||
) {
|
||||
FetchResponseData.PartitionData fetchablePartition =
|
||||
|
@ -95,11 +107,421 @@ public class RaftUtil {
|
|||
.setTopicId(topicId)
|
||||
.setPartitions(Collections.singletonList(fetchablePartition));
|
||||
|
||||
return new FetchResponseData()
|
||||
FetchResponseData response = new FetchResponseData();
|
||||
|
||||
if (apiVersion >= 17) {
|
||||
Optional<InetSocketAddress> address = endpoints.address(listenerName);
|
||||
if (address.isPresent() && leaderId >= 0) {
|
||||
// Populate the node endpoints
|
||||
FetchResponseData.NodeEndpointCollection nodeEndpoints = new FetchResponseData.NodeEndpointCollection(1);
|
||||
nodeEndpoints.add(
|
||||
new FetchResponseData.NodeEndpoint()
|
||||
.setNodeId(leaderId)
|
||||
.setHost(address.get().getHostString())
|
||||
.setPort(address.get().getPort())
|
||||
);
|
||||
response.setNodeEndpoints(nodeEndpoints);
|
||||
}
|
||||
}
|
||||
|
||||
return response
|
||||
.setErrorCode(topLevelError.code())
|
||||
.setResponses(Collections.singletonList(fetchableTopic));
|
||||
}
|
||||
|
||||
public static VoteRequestData singletonVoteRequest(
|
||||
TopicPartition topicPartition,
|
||||
String clusterId,
|
||||
int candidateEpoch,
|
||||
ReplicaKey candidateKey,
|
||||
ReplicaKey voterKey,
|
||||
int lastEpoch,
|
||||
long lastEpochEndOffset
|
||||
) {
|
||||
return new VoteRequestData()
|
||||
.setClusterId(clusterId)
|
||||
.setVoterId(voterKey.id())
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new VoteRequestData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(
|
||||
Collections.singletonList(
|
||||
new VoteRequestData.PartitionData()
|
||||
.setPartitionIndex(topicPartition.partition())
|
||||
.setCandidateEpoch(candidateEpoch)
|
||||
.setCandidateId(candidateKey.id())
|
||||
.setCandidateDirectoryId(
|
||||
candidateKey
|
||||
.directoryId()
|
||||
.orElse(ReplicaKey.NO_DIRECTORY_ID)
|
||||
)
|
||||
.setVoterDirectoryId(
|
||||
voterKey
|
||||
.directoryId()
|
||||
.orElse(ReplicaKey.NO_DIRECTORY_ID)
|
||||
)
|
||||
.setLastOffsetEpoch(lastEpoch)
|
||||
.setLastOffset(lastEpochEndOffset)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public static VoteResponseData singletonVoteResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
Errors topLevelError,
|
||||
TopicPartition topicPartition,
|
||||
Errors partitionLevelError,
|
||||
int leaderEpoch,
|
||||
int leaderId,
|
||||
boolean voteGranted,
|
||||
Endpoints endpoints
|
||||
) {
|
||||
VoteResponseData response = new VoteResponseData()
|
||||
.setErrorCode(topLevelError.code())
|
||||
.setTopics(Collections.singletonList(
|
||||
new VoteResponseData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(Collections.singletonList(
|
||||
new VoteResponseData.PartitionData()
|
||||
.setErrorCode(partitionLevelError.code())
|
||||
.setLeaderId(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
.setVoteGranted(voteGranted)))));
|
||||
|
||||
if (apiVersion >= 1) {
|
||||
Optional<InetSocketAddress> address = endpoints.address(listenerName);
|
||||
if (address.isPresent() && leaderId >= 0) {
|
||||
// Populate the node endpoints
|
||||
VoteResponseData.NodeEndpointCollection nodeEndpoints = new VoteResponseData.NodeEndpointCollection(1);
|
||||
nodeEndpoints.add(
|
||||
new VoteResponseData.NodeEndpoint()
|
||||
.setNodeId(leaderId)
|
||||
.setHost(address.get().getHostString())
|
||||
.setPort(address.get().getPort())
|
||||
);
|
||||
response.setNodeEndpoints(nodeEndpoints);
|
||||
}
|
||||
}
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
public static FetchSnapshotRequestData singletonFetchSnapshotRequest(
|
||||
String clusterId,
|
||||
ReplicaKey replicaKey,
|
||||
TopicPartition topicPartition,
|
||||
int epoch,
|
||||
OffsetAndEpoch offsetAndEpoch,
|
||||
int maxBytes,
|
||||
long position
|
||||
) {
|
||||
FetchSnapshotRequestData.SnapshotId snapshotId = new FetchSnapshotRequestData.SnapshotId()
|
||||
.setEndOffset(offsetAndEpoch.offset())
|
||||
.setEpoch(offsetAndEpoch.epoch());
|
||||
|
||||
FetchSnapshotRequestData.PartitionSnapshot partitionSnapshot = new FetchSnapshotRequestData.PartitionSnapshot()
|
||||
.setPartition(topicPartition.partition())
|
||||
.setCurrentLeaderEpoch(epoch)
|
||||
.setSnapshotId(snapshotId)
|
||||
.setPosition(position)
|
||||
.setReplicaDirectoryId(replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID));
|
||||
|
||||
return new FetchSnapshotRequestData()
|
||||
.setClusterId(clusterId)
|
||||
.setReplicaId(replicaKey.id())
|
||||
.setMaxBytes(maxBytes)
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new FetchSnapshotRequestData.TopicSnapshot()
|
||||
.setName(topicPartition.topic())
|
||||
.setPartitions(Collections.singletonList(partitionSnapshot))
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a FetchSnapshotResponseData with a single PartitionSnapshot for the topic partition.
|
||||
*
|
||||
* The partition index will already be populated when calling operator.
|
||||
*
|
||||
* @param listenerName the listener used to accept the request
|
||||
* @param apiVersion the api version of the request
|
||||
* @param topicPartition the topic partition to include
|
||||
* @param leaderId the id of the leader
|
||||
* @param endpoints the endpoints of the leader
|
||||
* @param operator unary operator responsible for populating all of the appropriate fields
|
||||
* @return the created fetch snapshot response data
|
||||
*/
|
||||
public static FetchSnapshotResponseData singletonFetchSnapshotResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
TopicPartition topicPartition,
|
||||
int leaderId,
|
||||
Endpoints endpoints,
|
||||
UnaryOperator<FetchSnapshotResponseData.PartitionSnapshot> operator
|
||||
) {
|
||||
FetchSnapshotResponseData.PartitionSnapshot partitionSnapshot = operator.apply(
|
||||
new FetchSnapshotResponseData.PartitionSnapshot().setIndex(topicPartition.partition())
|
||||
);
|
||||
|
||||
FetchSnapshotResponseData response = new FetchSnapshotResponseData()
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new FetchSnapshotResponseData.TopicSnapshot()
|
||||
.setName(topicPartition.topic())
|
||||
.setPartitions(Collections.singletonList(partitionSnapshot))
|
||||
)
|
||||
);
|
||||
|
||||
if (apiVersion >= 1) {
|
||||
Optional<InetSocketAddress> address = endpoints.address(listenerName);
|
||||
if (address.isPresent() && leaderId >= 0) {
|
||||
// Populate the node endpoints
|
||||
FetchSnapshotResponseData.NodeEndpointCollection nodeEndpoints =
|
||||
new FetchSnapshotResponseData.NodeEndpointCollection(1);
|
||||
nodeEndpoints.add(
|
||||
new FetchSnapshotResponseData.NodeEndpoint()
|
||||
.setNodeId(leaderId)
|
||||
.setHost(address.get().getHostString())
|
||||
.setPort(address.get().getPort())
|
||||
);
|
||||
response.setNodeEndpoints(nodeEndpoints);
|
||||
}
|
||||
}
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
public static BeginQuorumEpochRequestData singletonBeginQuorumEpochRequest(
|
||||
TopicPartition topicPartition,
|
||||
String clusterId,
|
||||
int leaderEpoch,
|
||||
int leaderId,
|
||||
Endpoints leaderEndponts,
|
||||
ReplicaKey voterKey
|
||||
) {
|
||||
return new BeginQuorumEpochRequestData()
|
||||
.setClusterId(clusterId)
|
||||
.setVoterId(voterKey.id())
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new BeginQuorumEpochRequestData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(
|
||||
Collections.singletonList(
|
||||
new BeginQuorumEpochRequestData.PartitionData()
|
||||
.setPartitionIndex(topicPartition.partition())
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
.setLeaderId(leaderId)
|
||||
.setVoterDirectoryId(voterKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID))
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
.setLeaderEndpoints(leaderEndponts.toBeginQuorumEpochRequest());
|
||||
}
|
||||
|
||||
public static BeginQuorumEpochResponseData singletonBeginQuorumEpochResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
Errors topLevelError,
|
||||
TopicPartition topicPartition,
|
||||
Errors partitionLevelError,
|
||||
int leaderEpoch,
|
||||
int leaderId,
|
||||
Endpoints endpoints
|
||||
) {
|
||||
BeginQuorumEpochResponseData response = new BeginQuorumEpochResponseData()
|
||||
.setErrorCode(topLevelError.code())
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new BeginQuorumEpochResponseData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(
|
||||
Collections.singletonList(
|
||||
new BeginQuorumEpochResponseData.PartitionData()
|
||||
.setErrorCode(partitionLevelError.code())
|
||||
.setLeaderId(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
if (apiVersion >= 1) {
|
||||
Optional<InetSocketAddress> address = endpoints.address(listenerName);
|
||||
if (address.isPresent() && leaderId >= 0) {
|
||||
// Populate the node endpoints
|
||||
BeginQuorumEpochResponseData.NodeEndpointCollection nodeEndpoints =
|
||||
new BeginQuorumEpochResponseData.NodeEndpointCollection(1);
|
||||
nodeEndpoints.add(
|
||||
new BeginQuorumEpochResponseData.NodeEndpoint()
|
||||
.setNodeId(leaderId)
|
||||
.setHost(address.get().getHostString())
|
||||
.setPort(address.get().getPort())
|
||||
);
|
||||
response.setNodeEndpoints(nodeEndpoints);
|
||||
}
|
||||
}
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
public static EndQuorumEpochRequestData singletonEndQuorumEpochRequest(
|
||||
TopicPartition topicPartition,
|
||||
String clusterId,
|
||||
int leaderEpoch,
|
||||
int leaderId,
|
||||
List<ReplicaKey> preferredReplicaKeys
|
||||
) {
|
||||
List<Integer> preferredSuccessors = preferredReplicaKeys
|
||||
.stream()
|
||||
.map(ReplicaKey::id)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
List<EndQuorumEpochRequestData.ReplicaInfo> preferredCandidates = preferredReplicaKeys
|
||||
.stream()
|
||||
.map(replicaKey -> new EndQuorumEpochRequestData.ReplicaInfo()
|
||||
.setCandidateId(replicaKey.id())
|
||||
.setCandidateDirectoryId(replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID))
|
||||
)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return new EndQuorumEpochRequestData()
|
||||
.setClusterId(clusterId)
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new EndQuorumEpochRequestData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(
|
||||
Collections.singletonList(
|
||||
new EndQuorumEpochRequestData.PartitionData()
|
||||
.setPartitionIndex(topicPartition.partition())
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
.setLeaderId(leaderId)
|
||||
.setPreferredSuccessors(preferredSuccessors)
|
||||
.setPreferredCandidates(preferredCandidates)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
}
|
||||
|
||||
public static EndQuorumEpochResponseData singletonEndQuorumEpochResponse(
|
||||
ListenerName listenerName,
|
||||
short apiVersion,
|
||||
Errors topLevelError,
|
||||
TopicPartition topicPartition,
|
||||
Errors partitionLevelError,
|
||||
int leaderEpoch,
|
||||
int leaderId,
|
||||
Endpoints endpoints
|
||||
) {
|
||||
EndQuorumEpochResponseData response = new EndQuorumEpochResponseData()
|
||||
.setErrorCode(topLevelError.code())
|
||||
.setTopics(Collections.singletonList(
|
||||
new EndQuorumEpochResponseData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(Collections.singletonList(
|
||||
new EndQuorumEpochResponseData.PartitionData()
|
||||
.setErrorCode(partitionLevelError.code())
|
||||
.setLeaderId(leaderId)
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
)))
|
||||
);
|
||||
|
||||
if (apiVersion >= 1) {
|
||||
Optional<InetSocketAddress> address = endpoints.address(listenerName);
|
||||
if (address.isPresent() && leaderId >= 0) {
|
||||
// Populate the node endpoints
|
||||
EndQuorumEpochResponseData.NodeEndpointCollection nodeEndpoints =
|
||||
new EndQuorumEpochResponseData.NodeEndpointCollection(1);
|
||||
nodeEndpoints.add(
|
||||
new EndQuorumEpochResponseData.NodeEndpoint()
|
||||
.setNodeId(leaderId)
|
||||
.setHost(address.get().getHostString())
|
||||
.setPort(address.get().getPort())
|
||||
);
|
||||
response.setNodeEndpoints(nodeEndpoints);
|
||||
}
|
||||
}
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
|
||||
public static DescribeQuorumRequestData singletonDescribeQuorumRequest(
|
||||
TopicPartition topicPartition
|
||||
) {
|
||||
|
||||
return new DescribeQuorumRequestData()
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new DescribeQuorumRequestData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(
|
||||
Collections.singletonList(
|
||||
new DescribeQuorumRequestData.PartitionData()
|
||||
.setPartitionIndex(topicPartition.partition())
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public static DescribeQuorumResponseData singletonDescribeQuorumResponse(
|
||||
short apiVersion,
|
||||
TopicPartition topicPartition,
|
||||
DescribeQuorumResponseData.PartitionData partitionData,
|
||||
DescribeQuorumResponseData.NodeCollection nodes
|
||||
) {
|
||||
DescribeQuorumResponseData response = new DescribeQuorumResponseData()
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new DescribeQuorumResponseData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(
|
||||
Collections.singletonList(
|
||||
partitionData.setPartitionIndex(topicPartition.partition())
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
if (apiVersion >= 2) {
|
||||
response.setNodes(nodes);
|
||||
}
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
public static Optional<ReplicaKey> voteRequestVoterKey(
|
||||
VoteRequestData request,
|
||||
VoteRequestData.PartitionData partition
|
||||
) {
|
||||
if (request.voterId() < 0) {
|
||||
return Optional.empty();
|
||||
} else {
|
||||
return Optional.of(ReplicaKey.of(request.voterId(), partition.voterDirectoryId()));
|
||||
}
|
||||
}
|
||||
|
||||
public static Optional<ReplicaKey> beginQuorumEpochRequestVoterKey(
|
||||
BeginQuorumEpochRequestData request,
|
||||
BeginQuorumEpochRequestData.PartitionData partition
|
||||
) {
|
||||
if (request.voterId() < 0) {
|
||||
return Optional.empty();
|
||||
} else {
|
||||
return Optional.of(ReplicaKey.of(request.voterId(), partition.voterDirectoryId()));
|
||||
}
|
||||
}
|
||||
|
||||
static boolean hasValidTopicPartition(FetchRequestData data, TopicPartition topicPartition, Uuid topicId) {
|
||||
return data.topics().size() == 1 &&
|
||||
data.topics().get(0).topicId().equals(topicId) &&
|
||||
|
|
|
@ -133,7 +133,7 @@ public class RequestManager {
|
|||
* If there is a connection with a pending request it returns the amount of time to wait until
|
||||
* the request times out.
|
||||
*
|
||||
* Returns zero, if there are no pending request and at least one of the boorstrap servers is
|
||||
* Returns zero, if there are no pending requests and at least one of the boorstrap servers is
|
||||
* ready.
|
||||
*
|
||||
* If all of the bootstrap servers are backing off and there are no pending requests, return
|
||||
|
|
|
@ -44,6 +44,7 @@ import java.util.Set;
|
|||
public class ResignedState implements EpochState {
|
||||
private final int localId;
|
||||
private final int epoch;
|
||||
private final Endpoints endpoints;
|
||||
private final Set<Integer> voters;
|
||||
private final long electionTimeoutMs;
|
||||
private final Set<Integer> unackedVoters;
|
||||
|
@ -58,6 +59,7 @@ public class ResignedState implements EpochState {
|
|||
Set<Integer> voters,
|
||||
long electionTimeoutMs,
|
||||
List<Integer> preferredSuccessors,
|
||||
Endpoints endpoints,
|
||||
LogContext logContext
|
||||
) {
|
||||
this.localId = localId;
|
||||
|
@ -68,6 +70,7 @@ public class ResignedState implements EpochState {
|
|||
this.electionTimeoutMs = electionTimeoutMs;
|
||||
this.electionTimer = time.timer(electionTimeoutMs);
|
||||
this.preferredSuccessors = preferredSuccessors;
|
||||
this.endpoints = endpoints;
|
||||
this.log = logContext.logger(ResignedState.class);
|
||||
}
|
||||
|
||||
|
@ -81,6 +84,11 @@ public class ResignedState implements EpochState {
|
|||
return epoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoints leaderEndpoints() {
|
||||
return endpoints;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the set of voters which have yet to acknowledge the resignation.
|
||||
* This node will send `EndQuorumEpoch` requests to this set until these
|
||||
|
|
|
@ -65,6 +65,11 @@ public class UnattachedState implements EpochState {
|
|||
return epoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoints leaderEndpoints() {
|
||||
return Endpoints.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return "Unattached";
|
||||
|
|
|
@ -74,6 +74,11 @@ public class VotedState implements EpochState {
|
|||
return epoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoints leaderEndpoints() {
|
||||
return Endpoints.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return "Voted";
|
||||
|
|
|
@ -33,11 +33,6 @@ public class BlockingMessageQueue implements RaftMessageQueue {
|
|||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short apiVersion() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApiMessage data() {
|
||||
return null;
|
||||
|
|
|
@ -21,7 +21,9 @@ import org.apache.kafka.common.Uuid;
|
|||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
public final class ReplicaKey {
|
||||
public final class ReplicaKey implements Comparable<ReplicaKey> {
|
||||
public static final Uuid NO_DIRECTORY_ID = Uuid.ZERO_UUID;
|
||||
|
||||
private final int id;
|
||||
private final Optional<Uuid> directoryId;
|
||||
|
||||
|
@ -38,6 +40,18 @@ public final class ReplicaKey {
|
|||
return directoryId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ReplicaKey that) {
|
||||
int idComparison = Integer.compare(this.id, that.id);
|
||||
if (idComparison == 0) {
|
||||
return directoryId
|
||||
.orElse(NO_DIRECTORY_ID)
|
||||
.compareTo(that.directoryId.orElse(NO_DIRECTORY_ID));
|
||||
} else {
|
||||
return idComparison;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
|
@ -59,7 +73,10 @@ public final class ReplicaKey {
|
|||
return String.format("ReplicaKey(id=%d, directoryId=%s)", id, directoryId);
|
||||
}
|
||||
|
||||
public static ReplicaKey of(int id, Optional<Uuid> directoryId) {
|
||||
return new ReplicaKey(id, directoryId);
|
||||
public static ReplicaKey of(int id, Uuid directoryId) {
|
||||
return new ReplicaKey(
|
||||
id,
|
||||
directoryId.equals(NO_DIRECTORY_ID) ? Optional.empty() : Optional.of(directoryId)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.kafka.common.feature.SupportedVersionRange;
|
|||
import org.apache.kafka.common.message.VotersRecord;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.Endpoints;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Collections;
|
||||
|
@ -97,28 +98,19 @@ public final class VoterSet {
|
|||
/**
|
||||
* Returns if the node is a voter in the set of voters.
|
||||
*
|
||||
* If the voter set includes the directory id, the {@code nodeKey} directory id must match the
|
||||
* If the voter set includes the directory id, the {@code replicaKey} directory id must match the
|
||||
* directory id specified by the voter set.
|
||||
*
|
||||
* If the voter set doesn't include the directory id ({@code Optional.empty()}), a node is in
|
||||
* the voter set as long as the node id matches. The directory id is not checked.
|
||||
*
|
||||
* @param nodeKey the node's id and directory id
|
||||
* @param replicaKey the node's id and directory id
|
||||
* @return true if the node is a voter in the voter set, otherwise false
|
||||
*/
|
||||
public boolean isVoter(ReplicaKey nodeKey) {
|
||||
VoterNode node = voters.get(nodeKey.id());
|
||||
if (node != null) {
|
||||
if (node.voterKey().directoryId().isPresent()) {
|
||||
return node.voterKey().directoryId().equals(nodeKey.directoryId());
|
||||
} else {
|
||||
// configured voter set doesn't include a directory id so it is a voter as long as the node id
|
||||
// matches
|
||||
return true;
|
||||
}
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
public boolean isVoter(ReplicaKey replicaKey) {
|
||||
return Optional.ofNullable(voters.get(replicaKey.id()))
|
||||
.map(node -> node.isVoter(replicaKey))
|
||||
.orElse(false);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -138,8 +130,39 @@ public final class VoterSet {
|
|||
return voters.keySet();
|
||||
}
|
||||
|
||||
public Map<Integer, VoterNode> voters() {
|
||||
return voters;
|
||||
/**
|
||||
* Returns all of the voters.
|
||||
*/
|
||||
public Set<ReplicaKey> voterKeys() {
|
||||
return voters
|
||||
.values()
|
||||
.stream()
|
||||
.map(VoterNode::voterKey)
|
||||
.collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all of the voters.
|
||||
*/
|
||||
public Set<VoterNode> voterNodes() {
|
||||
return voters
|
||||
.values()
|
||||
.stream()
|
||||
.collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all of the endpoints for a voter id.
|
||||
*
|
||||
* {@code Endpoints.empty()} is returned if the id is not a voter.
|
||||
*
|
||||
* @param voterId the id of the voter
|
||||
* @return the endpoints for the voter
|
||||
*/
|
||||
public Endpoints listeners(int voterId) {
|
||||
return Optional.ofNullable(voters.get(voterId))
|
||||
.map(VoterNode::listeners)
|
||||
.orElse(Endpoints.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -194,15 +217,7 @@ public final class VoterSet {
|
|||
Function<VoterNode, VotersRecord.Voter> voterConvertor = voter -> {
|
||||
Iterator<VotersRecord.Endpoint> endpoints = voter
|
||||
.listeners()
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(entry ->
|
||||
new VotersRecord.Endpoint()
|
||||
.setName(entry.getKey().value())
|
||||
.setHost(entry.getValue().getHostString())
|
||||
.setPort(entry.getValue().getPort())
|
||||
)
|
||||
.iterator();
|
||||
.votersRecordEndpoints();
|
||||
|
||||
VotersRecord.KRaftVersionFeature kraftVersionFeature = new VotersRecord.KRaftVersionFeature()
|
||||
.setMinSupportedVersion(voter.supportedKRaftVersion().min())
|
||||
|
@ -239,17 +254,8 @@ public final class VoterSet {
|
|||
* @return true if they have an overlapping majority, false otherwise
|
||||
*/
|
||||
public boolean hasOverlappingMajority(VoterSet that) {
|
||||
Set<ReplicaKey> thisReplicaKeys = voters
|
||||
.values()
|
||||
.stream()
|
||||
.map(VoterNode::voterKey)
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
Set<ReplicaKey> thatReplicaKeys = that.voters
|
||||
.values()
|
||||
.stream()
|
||||
.map(VoterNode::voterKey)
|
||||
.collect(Collectors.toSet());
|
||||
Set<ReplicaKey> thisReplicaKeys = voterKeys();
|
||||
Set<ReplicaKey> thatReplicaKeys = that.voterKeys();
|
||||
|
||||
if (Utils.diff(HashSet::new, thisReplicaKeys, thatReplicaKeys).size() > 1) return false;
|
||||
return Utils.diff(HashSet::new, thatReplicaKeys, thisReplicaKeys).size() <= 1;
|
||||
|
@ -277,12 +283,12 @@ public final class VoterSet {
|
|||
|
||||
public static final class VoterNode {
|
||||
private final ReplicaKey voterKey;
|
||||
private final Map<ListenerName, InetSocketAddress> listeners;
|
||||
private final Endpoints listeners;
|
||||
private final SupportedVersionRange supportedKRaftVersion;
|
||||
|
||||
public VoterNode(
|
||||
VoterNode(
|
||||
ReplicaKey voterKey,
|
||||
Map<ListenerName, InetSocketAddress> listeners,
|
||||
Endpoints listeners,
|
||||
SupportedVersionRange supportedKRaftVersion
|
||||
) {
|
||||
this.voterKey = voterKey;
|
||||
|
@ -294,7 +300,31 @@ public final class VoterSet {
|
|||
return voterKey;
|
||||
}
|
||||
|
||||
Map<ListenerName, InetSocketAddress> listeners() {
|
||||
/**
|
||||
* Returns if the provided replica key matches this voter node.
|
||||
*
|
||||
* If the voter node includes the directory id, the {@code replicaKey} directory id must
|
||||
* match the directory id specified by the voter set.
|
||||
*
|
||||
* If the voter node doesn't include the directory id ({@code Optional.empty()}), a replica
|
||||
* is the voter as long as the node id matches. The directory id is not checked.
|
||||
*
|
||||
* @param replicaKey the replica key
|
||||
* @return true if the replica key is the voter, otherwise false
|
||||
*/
|
||||
public boolean isVoter(ReplicaKey replicaKey) {
|
||||
if (voterKey.id() != replicaKey.id()) return false;
|
||||
|
||||
if (voterKey.directoryId().isPresent()) {
|
||||
return voterKey.directoryId().equals(replicaKey.directoryId());
|
||||
} else {
|
||||
// configured voter set doesn't include a directory id so it is a voter as long as
|
||||
// the ids match
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
Endpoints listeners() {
|
||||
return listeners;
|
||||
}
|
||||
|
||||
|
@ -304,7 +334,7 @@ public final class VoterSet {
|
|||
|
||||
|
||||
Optional<InetSocketAddress> address(ListenerName listener) {
|
||||
return Optional.ofNullable(listeners.get(listener));
|
||||
return listeners.address(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -344,26 +374,11 @@ public final class VoterSet {
|
|||
public static VoterSet fromVotersRecord(VotersRecord voters) {
|
||||
HashMap<Integer, VoterNode> voterNodes = new HashMap<>(voters.voters().size());
|
||||
for (VotersRecord.Voter voter: voters.voters()) {
|
||||
final Optional<Uuid> directoryId;
|
||||
if (!voter.voterDirectoryId().equals(Uuid.ZERO_UUID)) {
|
||||
directoryId = Optional.of(voter.voterDirectoryId());
|
||||
} else {
|
||||
directoryId = Optional.empty();
|
||||
}
|
||||
|
||||
Map<ListenerName, InetSocketAddress> listeners = new HashMap<>(voter.endpoints().size());
|
||||
for (VotersRecord.Endpoint endpoint : voter.endpoints()) {
|
||||
listeners.put(
|
||||
ListenerName.normalised(endpoint.name()),
|
||||
InetSocketAddress.createUnresolved(endpoint.host(), endpoint.port())
|
||||
);
|
||||
}
|
||||
|
||||
voterNodes.put(
|
||||
voter.voterId(),
|
||||
new VoterNode(
|
||||
ReplicaKey.of(voter.voterId(), directoryId),
|
||||
listeners,
|
||||
ReplicaKey.of(voter.voterId(), voter.voterDirectoryId()),
|
||||
Endpoints.fromVotersRecordEndpoints(voter.endpoints()),
|
||||
new SupportedVersionRange(
|
||||
voter.kRaftVersionFeature().minSupportedVersion(),
|
||||
voter.kRaftVersionFeature().maxSupportedVersion()
|
||||
|
@ -390,8 +405,8 @@ public final class VoterSet {
|
|||
Collectors.toMap(
|
||||
Map.Entry::getKey,
|
||||
entry -> new VoterNode(
|
||||
ReplicaKey.of(entry.getKey(), Optional.empty()),
|
||||
Collections.singletonMap(listener, entry.getValue()),
|
||||
ReplicaKey.of(entry.getKey(), Uuid.ZERO_UUID),
|
||||
Endpoints.fromInetSocketAddresses(Collections.singletonMap(listener, entry.getValue())),
|
||||
new SupportedVersionRange((short) 0, (short) 0)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -204,10 +204,10 @@ public class CandidateStateTest {
|
|||
voterSetWithLocal(IntStream.of(1, 2, 3))
|
||||
);
|
||||
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(0, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(0, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -232,6 +232,15 @@ public class CandidateStateTest {
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testLeaderEndpoints() {
|
||||
CandidateState state = newCandidateState(
|
||||
voterSetWithLocal(IntStream.of(1, 2, 3))
|
||||
);
|
||||
|
||||
assertEquals(Endpoints.empty(), state.leaderEndpoints());
|
||||
}
|
||||
|
||||
private VoterSet voterSetWithLocal(IntStream remoteVoters) {
|
||||
Map<Integer, VoterSet.VoterNode> voterMap = VoterSetTest.voterMap(remoteVoters, true);
|
||||
voterMap.put(localNode.voterKey().id(), localNode);
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.junit.jupiter.params.provider.ValueSource;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -38,38 +37,38 @@ final class ElectionStateTest {
|
|||
@Test
|
||||
void testVotedCandidateWithoutVotedId() {
|
||||
ElectionState electionState = ElectionState.withUnknownLeader(5, Collections.emptySet());
|
||||
assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, Optional.empty())));
|
||||
assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testVotedCandidateWithoutVotedDirectoryId() {
|
||||
ElectionState electionState = ElectionState.withVotedCandidate(
|
||||
5,
|
||||
ReplicaKey.of(1, Optional.empty()),
|
||||
ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID),
|
||||
Collections.emptySet()
|
||||
);
|
||||
assertTrue(electionState.isVotedCandidate(ReplicaKey.of(1, Optional.empty())));
|
||||
assertTrue(electionState.isVotedCandidate(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertTrue(
|
||||
electionState.isVotedCandidate(ReplicaKey.of(1, Optional.of(Uuid.randomUuid())))
|
||||
electionState.isVotedCandidate(ReplicaKey.of(1, Uuid.randomUuid()))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testVotedCandidateWithVotedDirectoryId() {
|
||||
ReplicaKey votedKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey votedKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
ElectionState electionState = ElectionState.withVotedCandidate(
|
||||
5,
|
||||
votedKey,
|
||||
Collections.emptySet()
|
||||
);
|
||||
assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, Optional.empty())));
|
||||
assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertTrue(electionState.isVotedCandidate(votedKey));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
void testQuorumStateDataRoundTrip(short version) {
|
||||
ReplicaKey votedKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey votedKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
List<ElectionState> electionStates = Arrays.asList(
|
||||
ElectionState.withUnknownLeader(5, Utils.mkSet(1, 2, 3)),
|
||||
ElectionState.withElectedLeader(5, 1, Utils.mkSet(1, 2, 3)),
|
||||
|
@ -83,7 +82,7 @@ final class ElectionStateTest {
|
|||
ElectionState.withElectedLeader(5, 1, Utils.mkSet(1, 2, 3)),
|
||||
ElectionState.withVotedCandidate(
|
||||
5,
|
||||
ReplicaKey.of(1, Optional.empty()),
|
||||
ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID),
|
||||
Utils.mkSet(1, 2, 3)
|
||||
)
|
||||
);
|
||||
|
|
|
@ -84,8 +84,7 @@ public class FileQuorumStateStoreTest {
|
|||
|
||||
final int epoch = 2;
|
||||
final int voter1 = 1;
|
||||
final Optional<Uuid> voter1DirectoryId = Optional.of(Uuid.randomUuid());
|
||||
final ReplicaKey voter1Key = ReplicaKey.of(voter1, voter1DirectoryId);
|
||||
final ReplicaKey voter1Key = ReplicaKey.of(voter1, Uuid.randomUuid());
|
||||
final int voter2 = 2;
|
||||
final int voter3 = 3;
|
||||
Set<Integer> voters = Utils.mkSet(voter1, voter2, voter3);
|
||||
|
@ -108,7 +107,7 @@ public class FileQuorumStateStoreTest {
|
|||
expected = Optional.of(
|
||||
ElectionState.withVotedCandidate(
|
||||
epoch,
|
||||
ReplicaKey.of(voter1, Optional.empty()),
|
||||
ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID),
|
||||
voters
|
||||
)
|
||||
);
|
||||
|
|
|
@ -16,7 +16,7 @@
|
|||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
@ -26,6 +26,8 @@ import org.junit.jupiter.api.Test;
|
|||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Collections;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.Set;
|
||||
|
@ -40,7 +42,13 @@ public class FollowerStateTest {
|
|||
private final LogContext logContext = new LogContext();
|
||||
private final int epoch = 5;
|
||||
private final int fetchTimeoutMs = 15000;
|
||||
private final Node leader = new Node(3, "mock-host-3", 1234);
|
||||
private final int leaderId = 3;
|
||||
private final Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses(
|
||||
Collections.singletonMap(
|
||||
ListenerName.normalised("CONTROLLER"),
|
||||
InetSocketAddress.createUnresolved("mock-host-3", 1234)
|
||||
)
|
||||
);
|
||||
|
||||
private FollowerState newFollowerState(
|
||||
Set<Integer> voters,
|
||||
|
@ -49,7 +57,8 @@ public class FollowerStateTest {
|
|||
return new FollowerState(
|
||||
time,
|
||||
epoch,
|
||||
leader,
|
||||
leaderId,
|
||||
leaderEndpoints,
|
||||
voters,
|
||||
highWatermark,
|
||||
fetchTimeoutMs,
|
||||
|
@ -93,15 +102,16 @@ public class FollowerStateTest {
|
|||
Optional.empty()
|
||||
);
|
||||
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLeaderNode() {
|
||||
public void testLeaderIdAndEndpoint() {
|
||||
FollowerState state = newFollowerState(Utils.mkSet(0, 1, 2), Optional.empty());
|
||||
|
||||
assertEquals(leader, state.leader());
|
||||
assertEquals(leaderId, state.leaderId());
|
||||
assertEquals(leaderEndpoints, state.leaderEndpoints());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -233,7 +233,6 @@ public class KafkaNetworkChannelTest {
|
|||
ApiMessage apiRequest = buildTestRequest(apiKey);
|
||||
RaftRequest.Outbound request = new RaftRequest.Outbound(
|
||||
correlationId,
|
||||
apiRequest.highestSupportedVersion(),
|
||||
apiRequest,
|
||||
destination,
|
||||
createdTimeMs
|
||||
|
@ -322,7 +321,23 @@ public class KafkaNetworkChannelTest {
|
|||
case END_QUORUM_EPOCH:
|
||||
return new EndQuorumEpochResponseData().setErrorCode(error.code());
|
||||
case VOTE:
|
||||
return VoteResponse.singletonResponse(error, topicPartition, Errors.NONE, 1, 5, false);
|
||||
return new VoteResponseData()
|
||||
.setErrorCode(error.code())
|
||||
.setTopics(
|
||||
Collections.singletonList(
|
||||
new VoteResponseData.TopicData()
|
||||
.setTopicName(topicPartition.topic())
|
||||
.setPartitions(
|
||||
Collections.singletonList(
|
||||
new VoteResponseData.PartitionData()
|
||||
.setErrorCode(Errors.NONE.code())
|
||||
.setLeaderId(1)
|
||||
.setLeaderEpoch(5)
|
||||
.setVoteGranted(false)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
case FETCH:
|
||||
return new FetchResponseData().setErrorCode(error.code());
|
||||
case FETCH_SNAPSHOT:
|
||||
|
|
|
@ -25,8 +25,8 @@ import org.apache.kafka.common.protocol.Errors;
|
|||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.UnalignedMemoryRecords;
|
||||
import org.apache.kafka.common.requests.FetchSnapshotRequest;
|
||||
import org.apache.kafka.common.requests.FetchSnapshotResponse;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.internals.ReplicaKey;
|
||||
import org.apache.kafka.raft.internals.StringSerde;
|
||||
import org.apache.kafka.snapshot.RawSnapshotReader;
|
||||
import org.apache.kafka.snapshot.RawSnapshotWriter;
|
||||
|
@ -37,6 +37,7 @@ import org.apache.kafka.snapshot.SnapshotWriterReaderTest;
|
|||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.CsvSource;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -93,14 +94,15 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testLeaderListenerNotified(boolean entireLog) throws Exception {
|
||||
@CsvSource({"false,false", "false,true", "true,false", "true,true"})
|
||||
public void testLeaderListenerNotified(boolean entireLog, boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, false);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
|
||||
|
||||
RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
|
||||
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
|
||||
.withEmptySnapshot(snapshotId);
|
||||
|
@ -116,7 +118,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
// Advance the highWatermark
|
||||
long localLogEndOffset = context.log.endOffset().offset;
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0));
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, localLogEndOffset, epoch, 0));
|
||||
context.pollUntilResponse();
|
||||
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId));
|
||||
assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong());
|
||||
|
@ -216,14 +218,16 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testListenerRenotified() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testListenerRenotified(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
|
||||
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
|
||||
.appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i"))
|
||||
|
@ -239,7 +243,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
// Advance the highWatermark
|
||||
long localLogEndOffset = context.log.endOffset().offset;
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0));
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, localLogEndOffset, epoch, 0));
|
||||
context.pollUntilResponse();
|
||||
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId));
|
||||
assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong());
|
||||
|
@ -270,15 +274,17 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLeaderImmediatelySendsSnapshotId() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testLeaderImmediatelySendsSnapshotId(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 4);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(snapshotId.epoch())
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
|
||||
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
|
||||
.appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i"))
|
||||
|
@ -290,7 +296,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
int epoch = context.currentEpoch();
|
||||
|
||||
// Send a fetch request for an end offset and epoch which has been snapshotted
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 6, 2, 500));
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 6, 2, 500));
|
||||
context.client.poll();
|
||||
|
||||
// Expect that the leader replies immediately with a snapshot id
|
||||
|
@ -302,14 +308,16 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchRequestOffsetLessThanLogStart() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withAppendLingerMs(1)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -338,7 +346,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.client.poll();
|
||||
|
||||
// Send Fetch request less than start offset
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeId, snapshotId.offset() - 2, snapshotId.epoch(), 0));
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, snapshotId.offset() - 2, snapshotId.epoch(), 0));
|
||||
context.pollUntilResponse();
|
||||
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
|
||||
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
|
||||
|
@ -348,15 +356,17 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchRequestOffsetAtZero() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchRequestOffsetAtZero(boolean withKip853Rpc) throws Exception {
|
||||
// When the follower sends a FETCH request at offset 0, reply with snapshot id if it exists
|
||||
int localId = 0;
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withAppendLingerMs(1)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -384,7 +394,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
}
|
||||
|
||||
// Send Fetch request for offset 0
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 0, 0, 0));
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 0, 0, 0));
|
||||
context.pollUntilResponse();
|
||||
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
|
||||
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
|
||||
|
@ -394,11 +404,12 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchRequestWithLargerLastFetchedEpoch() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -406,6 +417,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c"))
|
||||
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f"))
|
||||
.withAppendLingerMs(1)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -427,17 +439,18 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.client.poll();
|
||||
|
||||
// It is an invalid request to send an last fetched epoch greater than the current epoch
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset() + 1, epoch + 1, 0));
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, oldestSnapshotId.offset() + 1, epoch + 1, 0));
|
||||
context.pollUntilResponse();
|
||||
context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchRequestTruncateToLogStart() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchRequestTruncateToLogStart(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = localId + 1;
|
||||
int syncNodeId = otherNodeId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId, syncNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int syncNodeId = otherNodeKey.id() + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -445,6 +458,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c"))
|
||||
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("d", "e", "f"))
|
||||
.withAppendLingerMs(1)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -463,7 +477,13 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
// This should truncate to the old snapshot
|
||||
context.deliverRequest(
|
||||
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset() + 1, oldestSnapshotId.epoch() + 1, 0)
|
||||
context.fetchRequest(
|
||||
epoch,
|
||||
otherNodeKey,
|
||||
oldestSnapshotId.offset() + 1,
|
||||
oldestSnapshotId.epoch() + 1,
|
||||
0
|
||||
)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
|
||||
|
@ -474,12 +494,13 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(oldestSnapshotId.offset(), partitionResponse.divergingEpoch().endOffset());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchRequestAtLogStartOffsetWithValidEpoch() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchRequestAtLogStartOffsetWithValidEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = localId + 1;
|
||||
int syncNodeId = otherNodeId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId, syncNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int syncNodeId = otherNodeKey.id() + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -488,6 +509,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f"))
|
||||
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i"))
|
||||
.withAppendLingerMs(1)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -506,18 +528,25 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
// Send fetch request at log start offset with valid last fetched epoch
|
||||
context.deliverRequest(
|
||||
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset(), oldestSnapshotId.epoch(), 0)
|
||||
context.fetchRequest(
|
||||
epoch,
|
||||
otherNodeKey,
|
||||
oldestSnapshotId.offset(),
|
||||
oldestSnapshotId.epoch(),
|
||||
0
|
||||
)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchRequestAtLogStartOffsetWithInvalidEpoch() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = localId + 1;
|
||||
int syncNodeId = otherNodeId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId, syncNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int syncNodeId = otherNodeKey.id() + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -526,6 +555,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f"))
|
||||
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i"))
|
||||
.withAppendLingerMs(1)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -545,7 +575,13 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
// Send fetch with log start offset and invalid last fetched epoch
|
||||
context.deliverRequest(
|
||||
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset(), oldestSnapshotId.epoch() + 1, 0)
|
||||
context.fetchRequest(
|
||||
epoch,
|
||||
otherNodeKey,
|
||||
oldestSnapshotId.offset(),
|
||||
oldestSnapshotId.epoch() + 1,
|
||||
0
|
||||
)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
|
||||
|
@ -556,12 +592,15 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot(
|
||||
boolean withKip853Rpc
|
||||
) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = localId + 1;
|
||||
int syncNodeId = otherNodeId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId, syncNodeId);
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int syncNodeId = otherNodeKey.id() + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -570,6 +609,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f"))
|
||||
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i"))
|
||||
.withAppendLingerMs(1)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -590,7 +630,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverRequest(
|
||||
context.fetchRequest(
|
||||
epoch,
|
||||
otherNodeId,
|
||||
otherNodeKey,
|
||||
context.log.endOffset().offset,
|
||||
oldestSnapshotId.epoch() - 1,
|
||||
0
|
||||
|
@ -605,13 +645,19 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchSnapshotRequestMissingSnapshot() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchSnapshotRequestMissingSnapshot(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
int epoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
|
||||
RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch);
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(3)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
int epoch = context.currentEpoch();
|
||||
|
||||
context.deliverRequest(
|
||||
fetchSnapshotRequest(
|
||||
|
@ -629,14 +675,20 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(Errors.SNAPSHOT_NOT_FOUND, Errors.forCode(response.errorCode()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchSnapshotRequestUnknownPartition() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchSnapshotRequestUnknownPartition(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
int epoch = 2;
|
||||
TopicPartition topicPartition = new TopicPartition("unknown", 0);
|
||||
|
||||
RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch);
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(3)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
int epoch = context.currentEpoch();
|
||||
|
||||
context.deliverRequest(
|
||||
fetchSnapshotRequest(
|
||||
|
@ -654,8 +706,9 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.forCode(response.errorCode()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchSnapshotRequestAsLeader() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1);
|
||||
|
@ -663,6 +716,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.appendToLog(snapshotId.epoch(), Collections.singletonList("a"))
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -703,18 +757,22 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(memoryRecords.buffer(), ((UnalignedMemoryRecords) response.unalignedRecords()).buffer());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajorityVoters() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajorityVoters(
|
||||
boolean withKip853Rpc
|
||||
) throws Exception {
|
||||
int localId = 0;
|
||||
int voter1 = 1;
|
||||
int voter2 = 2;
|
||||
int observerId3 = 3;
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter1, voter2);
|
||||
ReplicaKey voter1 = replicaKey(1, withKip853Rpc);
|
||||
ReplicaKey voter2 = replicaKey(2, withKip853Rpc);
|
||||
ReplicaKey observer3 = replicaKey(3, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter1.id(), voter2.id());
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1);
|
||||
List<String> records = Arrays.asList("foo", "bar");
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.appendToLog(snapshotId.epoch(), Collections.singletonList("a"))
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
int resignLeadershipTimeout = context.checkQuorumTimeoutMs;
|
||||
|
@ -743,7 +801,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
FetchSnapshotRequestData observerFetchSnapshotRequest = fetchSnapshotRequest(
|
||||
context.clusterId.toString(),
|
||||
observerId3,
|
||||
observer3,
|
||||
context.metadataPartition,
|
||||
epoch,
|
||||
snapshotId,
|
||||
|
@ -794,8 +852,9 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertTrue(context.client.quorum().isResigned());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPartialFetchSnapshotRequestAsLeader() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(2, 1);
|
||||
|
@ -803,6 +862,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.appendToLog(snapshotId.epoch(), records)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -873,8 +933,9 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(snapshotBuffer, responseBuffer.flip());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchSnapshotRequestAsFollower() throws IOException {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchSnapshotRequestAsFollower(boolean withKip853Rpc) throws IOException {
|
||||
int localId = 0;
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
|
@ -883,6 +944,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, leaderId)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.deliverRequest(
|
||||
|
@ -903,8 +965,9 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(leaderId, response.currentLeader().leaderId());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchSnapshotRequestWithInvalidPosition() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1);
|
||||
|
@ -912,6 +975,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.appendToLog(snapshotId.epoch(), Collections.singletonList("a"))
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
|
@ -961,14 +1025,20 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(localId, response.currentLeader().leaderId());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchSnapshotRequestWithOlderEpoch() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchSnapshotRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(0, 0);
|
||||
|
||||
RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch);
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(1)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
int epoch = context.currentEpoch();
|
||||
|
||||
context.deliverRequest(
|
||||
fetchSnapshotRequest(
|
||||
|
@ -988,14 +1058,20 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertEquals(localId, response.currentLeader().leaderId());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchSnapshotRequestWithNewerEpoch() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchSnapshotRequestWithNewerEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = 0;
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(0, 0);
|
||||
|
||||
RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch);
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(1)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
context.becomeLeader();
|
||||
int epoch = context.currentEpoch();
|
||||
|
||||
context.deliverRequest(
|
||||
fetchSnapshotRequest(
|
||||
|
@ -1036,7 +1112,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, invalidEpoch, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, invalidEpoch, 200L)
|
||||
);
|
||||
|
||||
// Handle the invalid response
|
||||
|
@ -1053,7 +1129,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, invalidEndOffset, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, invalidEndOffset, 200L)
|
||||
);
|
||||
|
||||
// Handle the invalid response
|
||||
|
@ -1095,7 +1171,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
|
@ -1121,7 +1197,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
fetchSnapshotResponse(
|
||||
context.metadataPartition,
|
||||
context,
|
||||
epoch,
|
||||
leaderId,
|
||||
snapshotId,
|
||||
|
@ -1166,7 +1242,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
|
@ -1195,7 +1271,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
fetchSnapshotResponse(
|
||||
context.metadataPartition,
|
||||
context,
|
||||
epoch,
|
||||
leaderId,
|
||||
snapshotId,
|
||||
|
@ -1224,7 +1300,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
fetchSnapshotResponse(
|
||||
context.metadataPartition,
|
||||
context,
|
||||
epoch,
|
||||
leaderId,
|
||||
snapshotId,
|
||||
|
@ -1269,7 +1345,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
|
@ -1288,8 +1364,8 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
FetchSnapshotResponse.singleton(
|
||||
context.metadataPartition,
|
||||
context.fetchSnapshotResponse(
|
||||
leaderId,
|
||||
responsePartitionSnapshot -> {
|
||||
responsePartitionSnapshot
|
||||
.currentLeader()
|
||||
|
@ -1327,7 +1403,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, firstLeaderId, snapshotId, 200L)
|
||||
context.snapshotFetchResponse(epoch, firstLeaderId, snapshotId, 200L)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
|
@ -1346,8 +1422,8 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
FetchSnapshotResponse.singleton(
|
||||
context.metadataPartition,
|
||||
context.fetchSnapshotResponse(
|
||||
secondLeaderId,
|
||||
responsePartitionSnapshot -> {
|
||||
responsePartitionSnapshot
|
||||
.currentLeader()
|
||||
|
@ -1384,7 +1460,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
|
@ -1403,8 +1479,8 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
FetchSnapshotResponse.singleton(
|
||||
context.metadataPartition,
|
||||
context.fetchSnapshotResponse(
|
||||
leaderId,
|
||||
responsePartitionSnapshot -> {
|
||||
responsePartitionSnapshot
|
||||
.currentLeader()
|
||||
|
@ -1441,7 +1517,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
|
@ -1460,8 +1536,8 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
FetchSnapshotResponse.singleton(
|
||||
context.metadataPartition,
|
||||
context.fetchSnapshotResponse(
|
||||
leaderId + 1,
|
||||
responsePartitionSnapshot -> {
|
||||
responsePartitionSnapshot
|
||||
.currentLeader()
|
||||
|
@ -1508,7 +1584,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
|
@ -1527,8 +1603,8 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
FetchSnapshotResponse.singleton(
|
||||
context.metadataPartition,
|
||||
context.fetchSnapshotResponse(
|
||||
leaderId,
|
||||
responsePartitionSnapshot -> {
|
||||
responsePartitionSnapshot
|
||||
.currentLeader()
|
||||
|
@ -1554,7 +1630,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
|
@ -1574,8 +1650,8 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
FetchSnapshotResponse.singleton(
|
||||
context.metadataPartition,
|
||||
context.fetchSnapshotResponse(
|
||||
leaderId,
|
||||
responsePartitionSnapshot -> {
|
||||
responsePartitionSnapshot
|
||||
.currentLeader()
|
||||
|
@ -1618,7 +1694,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L)
|
||||
context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
|
@ -1646,8 +1722,8 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverResponse(
|
||||
snapshotRequest.correlationId(),
|
||||
snapshotRequest.destination(),
|
||||
FetchSnapshotResponse.singleton(
|
||||
context.metadataPartition,
|
||||
context.fetchSnapshotResponse(
|
||||
leaderId,
|
||||
responsePartitionSnapshot -> {
|
||||
responsePartitionSnapshot
|
||||
.currentLeader()
|
||||
|
@ -1669,20 +1745,28 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.assertVotedCandidate(epoch + 1, localId);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchSnapshotRequestClusterIdValidation() throws Exception {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFetchSnapshotRequestClusterIdValidation(
|
||||
boolean withKip853Rpc
|
||||
) throws Exception {
|
||||
int localId = 0;
|
||||
int otherNodeId = 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
ReplicaKey otherNode = replicaKey(1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNode.id());
|
||||
|
||||
RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch);
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
||||
// null cluster id is accepted
|
||||
context.becomeLeader();
|
||||
int epoch = context.currentEpoch();
|
||||
|
||||
// valid cluster id is accepted
|
||||
context.deliverRequest(
|
||||
fetchSnapshotRequest(
|
||||
context.clusterId.toString(),
|
||||
otherNodeId,
|
||||
otherNode,
|
||||
context.metadataPartition,
|
||||
epoch,
|
||||
new OffsetAndEpoch(0, 0),
|
||||
|
@ -1697,7 +1781,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverRequest(
|
||||
fetchSnapshotRequest(
|
||||
null,
|
||||
otherNodeId,
|
||||
otherNode,
|
||||
context.metadataPartition,
|
||||
epoch,
|
||||
new OffsetAndEpoch(0, 0),
|
||||
|
@ -1712,7 +1796,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverRequest(
|
||||
fetchSnapshotRequest(
|
||||
"",
|
||||
otherNodeId,
|
||||
otherNode,
|
||||
context.metadataPartition,
|
||||
epoch,
|
||||
new OffsetAndEpoch(0, 0),
|
||||
|
@ -1727,7 +1811,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
context.deliverRequest(
|
||||
fetchSnapshotRequest(
|
||||
"invalid-uuid",
|
||||
otherNodeId,
|
||||
otherNode,
|
||||
context.metadataPartition,
|
||||
epoch,
|
||||
new OffsetAndEpoch(0, 0),
|
||||
|
@ -1852,6 +1936,11 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4, 0));
|
||||
}
|
||||
|
||||
private static ReplicaKey replicaKey(int id, boolean withDirectoryId) {
|
||||
Uuid directoryId = withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID;
|
||||
return ReplicaKey.of(id, directoryId);
|
||||
}
|
||||
|
||||
private static FetchSnapshotRequestData fetchSnapshotRequest(
|
||||
TopicPartition topicPartition,
|
||||
int epoch,
|
||||
|
@ -1859,37 +1948,39 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
int maxBytes,
|
||||
long position
|
||||
) {
|
||||
return fetchSnapshotRequest(null, -1, topicPartition, epoch, offsetAndEpoch, maxBytes, position);
|
||||
return RaftUtil.singletonFetchSnapshotRequest(
|
||||
null,
|
||||
ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID),
|
||||
topicPartition,
|
||||
epoch,
|
||||
offsetAndEpoch,
|
||||
maxBytes,
|
||||
position
|
||||
);
|
||||
}
|
||||
|
||||
private static FetchSnapshotRequestData fetchSnapshotRequest(
|
||||
String clusterId,
|
||||
int replicaId,
|
||||
ReplicaKey replicaKey,
|
||||
TopicPartition topicPartition,
|
||||
int epoch,
|
||||
OffsetAndEpoch offsetAndEpoch,
|
||||
int maxBytes,
|
||||
long position
|
||||
) {
|
||||
FetchSnapshotRequestData.SnapshotId snapshotId = new FetchSnapshotRequestData.SnapshotId()
|
||||
.setEndOffset(offsetAndEpoch.offset())
|
||||
.setEpoch(offsetAndEpoch.epoch());
|
||||
|
||||
FetchSnapshotRequestData request = FetchSnapshotRequest.singleton(
|
||||
return RaftUtil.singletonFetchSnapshotRequest(
|
||||
clusterId,
|
||||
replicaId,
|
||||
replicaKey,
|
||||
topicPartition,
|
||||
snapshotPartition -> snapshotPartition
|
||||
.setCurrentLeaderEpoch(epoch)
|
||||
.setSnapshotId(snapshotId)
|
||||
.setPosition(position)
|
||||
epoch,
|
||||
offsetAndEpoch,
|
||||
maxBytes,
|
||||
position
|
||||
);
|
||||
|
||||
return request.setMaxBytes(maxBytes);
|
||||
}
|
||||
|
||||
private static FetchSnapshotResponseData fetchSnapshotResponse(
|
||||
TopicPartition topicPartition,
|
||||
RaftClientTestContext context,
|
||||
int leaderEpoch,
|
||||
int leaderId,
|
||||
OffsetAndEpoch snapshotId,
|
||||
|
@ -1897,8 +1988,8 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
long position,
|
||||
ByteBuffer buffer
|
||||
) {
|
||||
return FetchSnapshotResponse.singleton(
|
||||
topicPartition,
|
||||
return context.fetchSnapshotResponse(
|
||||
leaderId,
|
||||
partitionSnapshot -> {
|
||||
partitionSnapshot.currentLeader()
|
||||
.setLeaderEpoch(leaderEpoch)
|
||||
|
@ -1916,27 +2007,6 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
);
|
||||
}
|
||||
|
||||
private static FetchResponseData snapshotFetchResponse(
|
||||
TopicPartition topicPartition,
|
||||
Uuid topicId,
|
||||
int epoch,
|
||||
int leaderId,
|
||||
OffsetAndEpoch snapshotId,
|
||||
long highWatermark
|
||||
) {
|
||||
return RaftUtil.singletonFetchResponse(topicPartition, topicId, Errors.NONE, partitionData -> {
|
||||
partitionData.setHighWatermark(highWatermark);
|
||||
|
||||
partitionData.currentLeader()
|
||||
.setLeaderEpoch(epoch)
|
||||
.setLeaderId(leaderId);
|
||||
|
||||
partitionData.snapshotId()
|
||||
.setEpoch(snapshotId.epoch())
|
||||
.setEndOffset(snapshotId.offset());
|
||||
});
|
||||
}
|
||||
|
||||
private static Optional<FetchSnapshotRequestData.PartitionSnapshot> assertFetchSnapshotRequest(
|
||||
RaftRequest.Outbound request,
|
||||
TopicPartition topicPartition,
|
||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
|
@ -31,6 +30,7 @@ import org.junit.jupiter.params.provider.ValueSource;
|
|||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.UncheckedIOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
@ -48,10 +48,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
|||
public class QuorumStateTest {
|
||||
private final int localId = 0;
|
||||
private final Uuid localDirectoryId = Uuid.randomUuid();
|
||||
private final ReplicaKey localVoterKey = ReplicaKey.of(
|
||||
localId,
|
||||
Optional.of(localDirectoryId)
|
||||
);
|
||||
private final ReplicaKey localVoterKey = ReplicaKey.of(localId, localDirectoryId);
|
||||
private final int logEndEpoch = 0;
|
||||
private final MockQuorumStateStore store = new MockQuorumStateStore();
|
||||
private final MockTime time = new MockTime();
|
||||
|
@ -68,9 +65,9 @@ public class QuorumStateTest {
|
|||
return new QuorumState(
|
||||
localId,
|
||||
localDirectoryId,
|
||||
VoterSetTest.DEFAULT_LISTENER_NAME,
|
||||
() -> voterSet,
|
||||
() -> kraftVersion,
|
||||
localId.isPresent() ? voterSet.listeners(localId.getAsInt()) : Endpoints.empty(),
|
||||
electionTimeoutMs,
|
||||
fetchTimeoutMs,
|
||||
store,
|
||||
|
@ -100,7 +97,7 @@ public class QuorumStateTest {
|
|||
return replicaKey;
|
||||
}
|
||||
|
||||
return ReplicaKey.of(replicaKey.id(), Optional.empty());
|
||||
return ReplicaKey.of(replicaKey.id(), ReplicaKey.NO_DIRECTORY_ID);
|
||||
}
|
||||
|
||||
private VoterSet localStandaloneVoterSet() {
|
||||
|
@ -112,7 +109,11 @@ public class QuorumStateTest {
|
|||
private VoterSet localWithRemoteVoterSet(IntStream remoteIds, short kraftVersion) {
|
||||
boolean withDirectoryId = kraftVersion > 0;
|
||||
Map<Integer, VoterSet.VoterNode> voters = VoterSetTest.voterMap(remoteIds, withDirectoryId);
|
||||
voters.put(localId, VoterSetTest.voterNode(localVoterKey));
|
||||
if (withDirectoryId) {
|
||||
voters.put(localId, VoterSetTest.voterNode(localVoterKey));
|
||||
} else {
|
||||
voters.put(localId, VoterSetTest.voterNode(ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
}
|
||||
|
||||
return VoterSetTest.voterSet(voters);
|
||||
}
|
||||
|
@ -170,15 +171,15 @@ public class QuorumStateTest {
|
|||
|
||||
FollowerState followerState = state.followerStateOrThrow();
|
||||
assertEquals(epoch, followerState.epoch());
|
||||
assertEquals(node1, followerState.leader().id());
|
||||
assertEquals(node1, followerState.leaderId());
|
||||
assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds()));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testInitializeAsVoted(short kraftVersion) {
|
||||
ReplicaKey nodeKey1 = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey nodeKey2 = ReplicaKey.of(2, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid());
|
||||
|
||||
int epoch = 5;
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2));
|
||||
|
@ -379,7 +380,7 @@ public class QuorumStateTest {
|
|||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToCandidate();
|
||||
|
||||
state.transitionToFollower(5, voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get());
|
||||
state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId));
|
||||
assertEquals(5, state.epoch());
|
||||
assertEquals(OptionalInt.of(otherNodeId), state.leaderId());
|
||||
assertEquals(
|
||||
|
@ -420,7 +421,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testCandidateToVoted(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -447,7 +448,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testCandidateToAnyStateLowerEpoch(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -459,7 +460,8 @@ public class QuorumStateTest {
|
|||
IllegalStateException.class,
|
||||
() -> state.transitionToFollower(
|
||||
4,
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeKey.id(),
|
||||
voters.listeners(otherNodeKey.id())
|
||||
)
|
||||
);
|
||||
assertEquals(6, state.epoch());
|
||||
|
@ -546,7 +548,7 @@ public class QuorumStateTest {
|
|||
state.transitionToCandidate();
|
||||
state.candidateStateOrThrow().recordGrantedVote(otherNodeId);
|
||||
state.transitionToLeader(0L, accumulator);
|
||||
state.transitionToFollower(5, voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get());
|
||||
state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId));
|
||||
|
||||
assertEquals(5, state.epoch());
|
||||
assertEquals(OptionalInt.of(otherNodeId), state.leaderId());
|
||||
|
@ -589,7 +591,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testLeaderToVoted(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -619,7 +621,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testLeaderToAnyStateLowerEpoch(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -633,7 +635,8 @@ public class QuorumStateTest {
|
|||
IllegalStateException.class,
|
||||
() -> state.transitionToFollower(
|
||||
4,
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeKey.id(),
|
||||
voters.listeners(otherNodeKey.id())
|
||||
)
|
||||
);
|
||||
assertEquals(6, state.epoch());
|
||||
|
@ -660,7 +663,8 @@ public class QuorumStateTest {
|
|||
IllegalStateException.class,
|
||||
() -> state.transitionToFollower(
|
||||
0,
|
||||
voters.voterNode(localId, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
localId,
|
||||
voters.listeners(localId)
|
||||
)
|
||||
);
|
||||
assertThrows(IllegalStateException.class, () -> state.transitionToVoted(0, localVoterKey));
|
||||
|
@ -669,7 +673,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testUnattachedToLeaderOrResigned(short kraftVersion) {
|
||||
ReplicaKey leaderKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey leaderKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
int epoch = 5;
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leaderKey));
|
||||
store.writeElectionState(
|
||||
|
@ -686,7 +690,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testUnattachedToVotedSameEpoch(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -719,7 +723,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testUnattachedToVotedHigherEpoch(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -745,7 +749,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testUnattachedToCandidate(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -765,7 +769,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testUnattachedToUnattached(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -786,7 +790,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testUnattachedToFollowerSameEpoch(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -794,14 +798,15 @@ public class QuorumStateTest {
|
|||
|
||||
state.transitionToFollower(
|
||||
5,
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeKey.id(),
|
||||
voters.listeners(otherNodeKey.id())
|
||||
);
|
||||
assertTrue(state.isFollower());
|
||||
FollowerState followerState = state.followerStateOrThrow();
|
||||
assertEquals(5, followerState.epoch());
|
||||
assertEquals(
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME),
|
||||
Optional.of(followerState.leader())
|
||||
voters.listeners(otherNodeKey.id()),
|
||||
followerState.leaderEndpoints()
|
||||
);
|
||||
assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds()));
|
||||
}
|
||||
|
@ -809,7 +814,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testUnattachedToFollowerHigherEpoch(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -817,14 +822,15 @@ public class QuorumStateTest {
|
|||
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeKey.id(),
|
||||
voters.listeners(otherNodeKey.id())
|
||||
);
|
||||
assertTrue(state.isFollower());
|
||||
FollowerState followerState = state.followerStateOrThrow();
|
||||
assertEquals(8, followerState.epoch());
|
||||
assertEquals(
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME),
|
||||
Optional.of(followerState.leader())
|
||||
voters.listeners(otherNodeKey.id()),
|
||||
followerState.leaderEndpoints()
|
||||
);
|
||||
assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds()));
|
||||
}
|
||||
|
@ -832,7 +838,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testUnattachedToAnyStateLowerEpoch(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -843,7 +849,8 @@ public class QuorumStateTest {
|
|||
IllegalStateException.class,
|
||||
() -> state.transitionToFollower(
|
||||
4,
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeKey.id(),
|
||||
voters.listeners(otherNodeKey.id())
|
||||
)
|
||||
);
|
||||
assertEquals(5, state.epoch());
|
||||
|
@ -866,7 +873,7 @@ public class QuorumStateTest {
|
|||
VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty()));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID));
|
||||
assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator));
|
||||
assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList()));
|
||||
}
|
||||
|
@ -879,7 +886,7 @@ public class QuorumStateTest {
|
|||
VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty()));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID));
|
||||
|
||||
int jitterMs = 2500;
|
||||
random.mockNextInt(electionTimeoutMs, jitterMs);
|
||||
|
@ -900,14 +907,14 @@ public class QuorumStateTest {
|
|||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToUnattached(5);
|
||||
state.transitionToVoted(8, ReplicaKey.of(node1, Optional.of(Uuid.randomUuid())));
|
||||
state.transitionToVoted(8, ReplicaKey.of(node1, Uuid.randomUuid()));
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(node1, Optional.empty()))
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))
|
||||
);
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(node2, Optional.empty()))
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -919,17 +926,18 @@ public class QuorumStateTest {
|
|||
VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty()));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID));
|
||||
state.transitionToFollower(
|
||||
5,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
);
|
||||
|
||||
FollowerState followerState = state.followerStateOrThrow();
|
||||
assertEquals(5, followerState.epoch());
|
||||
assertEquals(
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME),
|
||||
Optional.of(followerState.leader())
|
||||
voters.listeners(node2),
|
||||
followerState.leaderEndpoints()
|
||||
);
|
||||
assertEquals(
|
||||
Optional.of(
|
||||
|
@ -951,17 +959,18 @@ public class QuorumStateTest {
|
|||
VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty()));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID));
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
);
|
||||
|
||||
FollowerState followerState = state.followerStateOrThrow();
|
||||
assertEquals(8, followerState.epoch());
|
||||
assertEquals(
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get(),
|
||||
followerState.leader()
|
||||
voters.listeners(node2),
|
||||
followerState.leaderEndpoints()
|
||||
);
|
||||
assertEquals(
|
||||
Optional.of(
|
||||
|
@ -983,7 +992,7 @@ public class QuorumStateTest {
|
|||
VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, Optional.empty()));
|
||||
state.transitionToVoted(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID));
|
||||
assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5));
|
||||
}
|
||||
|
||||
|
@ -994,7 +1003,7 @@ public class QuorumStateTest {
|
|||
VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion);
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToVoted(5, ReplicaKey.of(otherNodeId, Optional.empty()));
|
||||
state.transitionToVoted(5, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID));
|
||||
|
||||
long remainingElectionTimeMs = state.votedStateOrThrow().remainingElectionTimeMs(time.milliseconds());
|
||||
time.sleep(1000);
|
||||
|
@ -1011,7 +1020,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testVotedToAnyStateLowerEpoch(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -1022,7 +1031,8 @@ public class QuorumStateTest {
|
|||
IllegalStateException.class,
|
||||
() -> state.transitionToFollower(
|
||||
4,
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeKey.id(),
|
||||
voters.listeners(otherNodeKey.id())
|
||||
)
|
||||
);
|
||||
assertEquals(5, state.epoch());
|
||||
|
@ -1048,28 +1058,31 @@ public class QuorumStateTest {
|
|||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
);
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node1,
|
||||
voters.listeners(node1)
|
||||
)
|
||||
);
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
)
|
||||
);
|
||||
|
||||
FollowerState followerState = state.followerStateOrThrow();
|
||||
assertEquals(8, followerState.epoch());
|
||||
assertEquals(
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME),
|
||||
Optional.of(followerState.leader())
|
||||
voters.listeners(node2),
|
||||
followerState.leaderEndpoints()
|
||||
);
|
||||
assertEquals(
|
||||
Optional.of(
|
||||
|
@ -1093,18 +1106,20 @@ public class QuorumStateTest {
|
|||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
);
|
||||
state.transitionToFollower(
|
||||
9,
|
||||
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node1,
|
||||
voters.listeners(node1)
|
||||
);
|
||||
|
||||
FollowerState followerState = state.followerStateOrThrow();
|
||||
assertEquals(9, followerState.epoch());
|
||||
assertEquals(
|
||||
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME),
|
||||
Optional.of(followerState.leader())
|
||||
voters.listeners(node1),
|
||||
followerState.leaderEndpoints()
|
||||
);
|
||||
assertEquals(
|
||||
Optional.of(
|
||||
|
@ -1128,7 +1143,8 @@ public class QuorumStateTest {
|
|||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
);
|
||||
assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator));
|
||||
assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList()));
|
||||
|
@ -1144,7 +1160,8 @@ public class QuorumStateTest {
|
|||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
);
|
||||
|
||||
int jitterMs = 2500;
|
||||
|
@ -1167,7 +1184,8 @@ public class QuorumStateTest {
|
|||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
);
|
||||
assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8));
|
||||
}
|
||||
|
@ -1182,7 +1200,8 @@ public class QuorumStateTest {
|
|||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
);
|
||||
|
||||
int jitterMs = 2500;
|
||||
|
@ -1205,35 +1224,37 @@ public class QuorumStateTest {
|
|||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node2,
|
||||
voters.listeners(node2)
|
||||
);
|
||||
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(node1, Optional.empty()))
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))
|
||||
);
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(localId, Optional.empty()))
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID))
|
||||
);
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(node2, Optional.empty()))
|
||||
() -> state.transitionToVoted(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID))
|
||||
);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testFollowerToVotedHigherEpoch(short kraftVersion) {
|
||||
ReplicaKey nodeKey1 = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey nodeKey2 = ReplicaKey.of(2, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid());
|
||||
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2));
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToFollower(
|
||||
8,
|
||||
voters.voterNode(nodeKey2.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
nodeKey2.id(),
|
||||
voters.listeners(nodeKey2.id())
|
||||
);
|
||||
|
||||
int jitterMs = 2500;
|
||||
|
@ -1259,18 +1280,20 @@ public class QuorumStateTest {
|
|||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
state.transitionToFollower(
|
||||
5,
|
||||
voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeId,
|
||||
voters.listeners(otherNodeId)
|
||||
);
|
||||
assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4));
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToVoted(4, ReplicaKey.of(otherNodeId, Optional.empty()))
|
||||
() -> state.transitionToVoted(4, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID))
|
||||
);
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToFollower(
|
||||
4,
|
||||
voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeId,
|
||||
voters.listeners(otherNodeId)
|
||||
)
|
||||
);
|
||||
assertEquals(5, state.epoch());
|
||||
|
@ -1290,7 +1313,7 @@ public class QuorumStateTest {
|
|||
@ValueSource(shorts = {0, 1})
|
||||
public void testCanBecomeFollowerOfNonVoter(short kraftVersion) {
|
||||
int otherNodeId = 1;
|
||||
ReplicaKey nonVoterKey = ReplicaKey.of(2, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey nonVoterKey = ReplicaKey.of(2, Uuid.randomUuid());
|
||||
VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion);
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
|
||||
|
@ -1304,8 +1327,16 @@ public class QuorumStateTest {
|
|||
assertEquals(nonVoterKey, votedState.votedKey());
|
||||
|
||||
// Transition to follower
|
||||
Node nonVoterNode = new Node(nonVoterKey.id(), "non-voter-host", 1234);
|
||||
state.transitionToFollower(4, nonVoterNode);
|
||||
state.transitionToFollower(
|
||||
4,
|
||||
nonVoterKey.id(),
|
||||
Endpoints.fromInetSocketAddresses(
|
||||
Collections.singletonMap(
|
||||
VoterSetTest.DEFAULT_LISTENER_NAME,
|
||||
InetSocketAddress.createUnresolved("non-voter-host", 1234)
|
||||
)
|
||||
)
|
||||
);
|
||||
assertEquals(
|
||||
new LeaderAndEpoch(OptionalInt.of(nonVoterKey.id()), 4),
|
||||
state.leaderAndEpoch()
|
||||
|
@ -1330,7 +1361,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testObserverWithIdCanVote(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(otherNodeKey));
|
||||
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
|
@ -1360,7 +1391,8 @@ public class QuorumStateTest {
|
|||
|
||||
state.transitionToFollower(
|
||||
2,
|
||||
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
node1,
|
||||
voters.listeners(node1)
|
||||
);
|
||||
state.transitionToUnattached(3);
|
||||
assertTrue(state.isUnattached());
|
||||
|
@ -1385,13 +1417,13 @@ public class QuorumStateTest {
|
|||
assertTrue(state.isObserver());
|
||||
|
||||
state.transitionToUnattached(2);
|
||||
state.transitionToFollower(3, voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get());
|
||||
state.transitionToFollower(3, node1, voters.listeners(node1));
|
||||
assertTrue(state.isFollower());
|
||||
FollowerState followerState = state.followerStateOrThrow();
|
||||
assertEquals(3, followerState.epoch());
|
||||
assertEquals(
|
||||
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME),
|
||||
Optional.of(followerState.leader())
|
||||
voters.listeners(node1),
|
||||
followerState.leaderEndpoints()
|
||||
);
|
||||
assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds()));
|
||||
}
|
||||
|
@ -1418,7 +1450,7 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testHasRemoteLeader(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
|
@ -1439,7 +1471,8 @@ public class QuorumStateTest {
|
|||
|
||||
state.transitionToFollower(
|
||||
state.epoch() + 1,
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeKey.id(),
|
||||
voters.listeners(otherNodeKey.id())
|
||||
);
|
||||
assertTrue(state.hasRemoteLeader());
|
||||
}
|
||||
|
@ -1447,13 +1480,14 @@ public class QuorumStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testHighWatermarkRetained(short kraftVersion) {
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
|
||||
ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
|
||||
|
||||
QuorumState state = initializeEmptyState(voters, kraftVersion);
|
||||
state.transitionToFollower(
|
||||
5,
|
||||
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get()
|
||||
otherNodeKey.id(),
|
||||
voters.listeners(otherNodeKey.id())
|
||||
);
|
||||
|
||||
FollowerState followerState = state.followerStateOrThrow();
|
||||
|
@ -1495,11 +1529,11 @@ public class QuorumStateTest {
|
|||
assertThrows(IllegalStateException.class, state::transitionToCandidate);
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> state.transitionToVoted(1, ReplicaKey.of(1, Optional.empty()))
|
||||
() -> state.transitionToVoted(1, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))
|
||||
);
|
||||
assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator));
|
||||
|
||||
state.transitionToFollower(1, voters.voterNode(1, VoterSetTest.DEFAULT_LISTENER_NAME).get());
|
||||
state.transitionToFollower(1, 1, voters.listeners(1));
|
||||
assertTrue(state.isFollower());
|
||||
|
||||
state.transitionToUnattached(2);
|
||||
|
@ -1519,7 +1553,7 @@ public class QuorumStateTest {
|
|||
store.writeElectionState(
|
||||
ElectionState.withVotedCandidate(
|
||||
epoch,
|
||||
ReplicaKey.of(votedId, Optional.empty()),
|
||||
ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID),
|
||||
voters.voterIds()
|
||||
),
|
||||
kraftVersion
|
||||
|
|
|
@ -23,12 +23,14 @@ import org.apache.kafka.common.compress.Compression;
|
|||
import org.apache.kafka.common.memory.MemoryPool;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.message.DescribeQuorumRequestData;
|
||||
import org.apache.kafka.common.message.DescribeQuorumResponseData;
|
||||
import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochRequestData;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochResponseData;
|
||||
import org.apache.kafka.common.message.FetchRequestData;
|
||||
import org.apache.kafka.common.message.FetchResponseData;
|
||||
import org.apache.kafka.common.message.FetchSnapshotRequestData;
|
||||
import org.apache.kafka.common.message.FetchSnapshotResponseData;
|
||||
import org.apache.kafka.common.message.LeaderChangeMessage;
|
||||
import org.apache.kafka.common.message.LeaderChangeMessage.Voter;
|
||||
|
@ -43,20 +45,15 @@ import org.apache.kafka.common.record.ControlRecordUtils;
|
|||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.Record;
|
||||
import org.apache.kafka.common.record.Records;
|
||||
import org.apache.kafka.common.requests.BeginQuorumEpochRequest;
|
||||
import org.apache.kafka.common.requests.BeginQuorumEpochResponse;
|
||||
import org.apache.kafka.common.requests.DescribeQuorumResponse;
|
||||
import org.apache.kafka.common.requests.EndQuorumEpochRequest;
|
||||
import org.apache.kafka.common.requests.EndQuorumEpochResponse;
|
||||
import org.apache.kafka.common.requests.FetchSnapshotResponse;
|
||||
import org.apache.kafka.common.requests.VoteRequest;
|
||||
import org.apache.kafka.common.requests.VoteResponse;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.internals.BatchBuilder;
|
||||
import org.apache.kafka.raft.internals.ReplicaKey;
|
||||
import org.apache.kafka.raft.internals.StringSerde;
|
||||
import org.apache.kafka.raft.internals.VoterSet;
|
||||
import org.apache.kafka.server.common.serialization.RecordSerde;
|
||||
import org.apache.kafka.snapshot.RawSnapshotWriter;
|
||||
import org.apache.kafka.snapshot.SnapshotReader;
|
||||
|
@ -79,6 +76,7 @@ import java.util.OptionalLong;
|
|||
import java.util.Set;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.UnaryOperator;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
|
@ -108,6 +106,7 @@ public final class RaftClientTestContext {
|
|||
private final QuorumStateStore quorumStateStore;
|
||||
final Uuid clusterId;
|
||||
private final OptionalInt localId;
|
||||
public final Uuid localDirectoryId;
|
||||
public final KafkaRaftClient<String> client;
|
||||
final Metrics metrics;
|
||||
public final MockLog log;
|
||||
|
@ -115,8 +114,10 @@ public final class RaftClientTestContext {
|
|||
final MockMessageQueue messageQueue;
|
||||
final MockTime time;
|
||||
final MockListener listener;
|
||||
final Set<Integer> voters;
|
||||
final VoterSet voters;
|
||||
final Set<Integer> bootstrapIds;
|
||||
// Used to determine which RPC request and response to construct
|
||||
final boolean kip853Rpc;
|
||||
|
||||
private final List<RaftResponse.Outbound> sentResponses = new ArrayList<>();
|
||||
|
||||
|
@ -142,7 +143,6 @@ public final class RaftClientTestContext {
|
|||
private final Uuid clusterId = Uuid.randomUuid();
|
||||
private final Set<Integer> voters;
|
||||
private final OptionalInt localId;
|
||||
private final Uuid localDirectoryId = Uuid.randomUuid();
|
||||
private final short kraftVersion = 0;
|
||||
|
||||
private int requestTimeoutMs = DEFAULT_REQUEST_TIMEOUT_MS;
|
||||
|
@ -150,6 +150,7 @@ public final class RaftClientTestContext {
|
|||
private int appendLingerMs = DEFAULT_APPEND_LINGER_MS;
|
||||
private MemoryPool memoryPool = MemoryPool.NONE;
|
||||
private List<InetSocketAddress> bootstrapServers = Collections.emptyList();
|
||||
private boolean kip853Rpc = false;
|
||||
|
||||
public Builder(int localId, Set<Integer> voters) {
|
||||
this(OptionalInt.of(localId), voters);
|
||||
|
@ -249,6 +250,11 @@ public final class RaftClientTestContext {
|
|||
return this;
|
||||
}
|
||||
|
||||
Builder withKip853Rpc(boolean kip853Rpc) {
|
||||
this.kip853Rpc = kip853Rpc;
|
||||
return this;
|
||||
}
|
||||
|
||||
public RaftClientTestContext build() throws IOException {
|
||||
Metrics metrics = new Metrics(time);
|
||||
MockNetworkChannel channel = new MockNetworkChannel();
|
||||
|
@ -257,6 +263,19 @@ public final class RaftClientTestContext {
|
|||
.stream()
|
||||
.collect(Collectors.toMap(Function.identity(), RaftClientTestContext::mockAddress));
|
||||
|
||||
VoterSet voterSet = VoterSet.fromInetSocketAddresses(
|
||||
channel.listenerName(),
|
||||
voterAddressMap
|
||||
);
|
||||
|
||||
// Compute the local listeners. Only potential voters/leader need to provide the local listeners
|
||||
// If the local id is not set (must be observer), the local listener can be empty.
|
||||
Endpoints localListeners = localId.isPresent() ?
|
||||
voterSet.listeners(localId.getAsInt()) :
|
||||
Endpoints.empty();
|
||||
|
||||
Uuid localDirectoryId = Uuid.randomUuid();
|
||||
|
||||
QuorumConfig quorumConfig = new QuorumConfig(
|
||||
requestTimeoutMs,
|
||||
RETRY_BACKOFF_MS,
|
||||
|
@ -279,6 +298,7 @@ public final class RaftClientTestContext {
|
|||
FETCH_MAX_WAIT_MS,
|
||||
clusterId.toString(),
|
||||
bootstrapServers,
|
||||
localListeners,
|
||||
logContext,
|
||||
random,
|
||||
quorumConfig
|
||||
|
@ -294,18 +314,20 @@ public final class RaftClientTestContext {
|
|||
RaftClientTestContext context = new RaftClientTestContext(
|
||||
clusterId,
|
||||
localId,
|
||||
localDirectoryId,
|
||||
client,
|
||||
log,
|
||||
channel,
|
||||
messageQueue,
|
||||
time,
|
||||
quorumStateStore,
|
||||
voters,
|
||||
voterSet,
|
||||
IntStream
|
||||
.iterate(-2, id -> id - 1)
|
||||
.limit(bootstrapServers.size())
|
||||
.boxed()
|
||||
.collect(Collectors.toSet()),
|
||||
kip853Rpc,
|
||||
metrics,
|
||||
listener
|
||||
);
|
||||
|
@ -318,22 +340,26 @@ public final class RaftClientTestContext {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("ParameterNumber")
|
||||
private RaftClientTestContext(
|
||||
Uuid clusterId,
|
||||
OptionalInt localId,
|
||||
Uuid localDirectoryId,
|
||||
KafkaRaftClient<String> client,
|
||||
MockLog log,
|
||||
MockNetworkChannel channel,
|
||||
MockMessageQueue messageQueue,
|
||||
MockTime time,
|
||||
QuorumStateStore quorumStateStore,
|
||||
Set<Integer> voters,
|
||||
VoterSet voters,
|
||||
Set<Integer> bootstrapIds,
|
||||
boolean kip853Rpc,
|
||||
Metrics metrics,
|
||||
MockListener listener
|
||||
) {
|
||||
this.clusterId = clusterId;
|
||||
this.localId = localId;
|
||||
this.localDirectoryId = localDirectoryId;
|
||||
this.client = client;
|
||||
this.log = log;
|
||||
this.channel = channel;
|
||||
|
@ -342,6 +368,7 @@ public final class RaftClientTestContext {
|
|||
this.quorumStateStore = quorumStateStore;
|
||||
this.voters = voters;
|
||||
this.bootstrapIds = bootstrapIds;
|
||||
this.kip853Rpc = kip853Rpc;
|
||||
this.metrics = metrics;
|
||||
this.listener = listener;
|
||||
}
|
||||
|
@ -444,6 +471,12 @@ public final class RaftClientTestContext {
|
|||
return localId.orElseThrow(() -> new AssertionError("Required local id is not defined"));
|
||||
}
|
||||
|
||||
public ReplicaKey localReplicaKey() {
|
||||
return kip853Rpc ?
|
||||
ReplicaKey.of(localIdOrThrow(), localDirectoryId) :
|
||||
ReplicaKey.of(localIdOrThrow(), ReplicaKey.NO_DIRECTORY_ID);
|
||||
}
|
||||
|
||||
private void expectBeginEpoch(int epoch) throws Exception {
|
||||
pollUntilRequest();
|
||||
for (RaftRequest.Outbound request : collectBeginEpochRequests(epoch)) {
|
||||
|
@ -472,8 +505,8 @@ public final class RaftClientTestContext {
|
|||
assertEquals(
|
||||
ElectionState.withVotedCandidate(
|
||||
epoch,
|
||||
ReplicaKey.of(candidateId, Optional.empty()),
|
||||
voters
|
||||
ReplicaKey.of(candidateId, ReplicaKey.NO_DIRECTORY_ID),
|
||||
voters.voterIds()
|
||||
),
|
||||
quorumStateStore.readElectionState().get()
|
||||
);
|
||||
|
@ -481,14 +514,14 @@ public final class RaftClientTestContext {
|
|||
|
||||
public void assertElectedLeader(int epoch, int leaderId) {
|
||||
assertEquals(
|
||||
ElectionState.withElectedLeader(epoch, leaderId, voters),
|
||||
ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()),
|
||||
quorumStateStore.readElectionState().get()
|
||||
);
|
||||
}
|
||||
|
||||
void assertUnknownLeader(int epoch) {
|
||||
assertEquals(
|
||||
ElectionState.withUnknownLeader(epoch, voters),
|
||||
ElectionState.withUnknownLeader(epoch, voters.voterIds()),
|
||||
quorumStateStore.readElectionState().get()
|
||||
);
|
||||
}
|
||||
|
@ -496,7 +529,7 @@ public final class RaftClientTestContext {
|
|||
void assertResignedLeader(int epoch, int leaderId) {
|
||||
assertTrue(client.quorum().isResigned());
|
||||
assertEquals(
|
||||
ElectionState.withElectedLeader(epoch, leaderId, voters),
|
||||
ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()),
|
||||
quorumStateStore.readElectionState().get()
|
||||
);
|
||||
}
|
||||
|
@ -530,17 +563,16 @@ public final class RaftClientTestContext {
|
|||
.setObservers(observerStates);
|
||||
|
||||
DescribeQuorumResponseData.NodeCollection nodes = new DescribeQuorumResponseData.NodeCollection();
|
||||
|
||||
Consumer<DescribeQuorumResponseData.ReplicaState> addToNodes = replicaState -> {
|
||||
if (nodes.find(replicaState.replicaId()) != null)
|
||||
return;
|
||||
|
||||
nodes.add(new DescribeQuorumResponseData.Node()
|
||||
.setNodeId(replicaState.replicaId()));
|
||||
};
|
||||
|
||||
voterStates.forEach(addToNodes);
|
||||
observerStates.forEach(addToNodes);
|
||||
voterStates.forEach(replicaState -> {
|
||||
if (kip853Rpc && nodes.find(replicaState.replicaId()) == null) {
|
||||
// KAFKA-16953 will add support for including the node listeners in the node
|
||||
// collection
|
||||
nodes.add(
|
||||
new DescribeQuorumResponseData.Node()
|
||||
.setNodeId(replicaState.replicaId())
|
||||
);
|
||||
}
|
||||
});
|
||||
|
||||
DescribeQuorumResponseData expectedResponse = DescribeQuorumResponse.singletonResponse(
|
||||
metadataPartition,
|
||||
|
@ -585,6 +617,15 @@ public final class RaftClientTestContext {
|
|||
assertEquals(error, Errors.forCode(partitionResponse.errorCode()));
|
||||
assertEquals(epoch, partitionResponse.leaderEpoch());
|
||||
assertEquals(leaderId.orElse(-1), partitionResponse.leaderId());
|
||||
|
||||
if (kip853Rpc && leaderId.isPresent()) {
|
||||
Endpoints expectedLeaderEndpoints = voters.listeners(leaderId.getAsInt());
|
||||
Endpoints responseEndpoints = Endpoints.fromVoteResponse(
|
||||
channel.listenerName(),
|
||||
response.nodeEndpoints()
|
||||
);
|
||||
assertEquals(expectedLeaderEndpoints, responseEndpoints);
|
||||
}
|
||||
}
|
||||
|
||||
List<RaftRequest.Outbound> collectVoteRequests(
|
||||
|
@ -610,7 +651,12 @@ public final class RaftClientTestContext {
|
|||
|
||||
void deliverRequest(ApiMessage request) {
|
||||
RaftRequest.Inbound inboundRequest = new RaftRequest.Inbound(
|
||||
channel.newCorrelationId(), request.highestSupportedVersion(), request, time.milliseconds());
|
||||
channel.listenerName(),
|
||||
channel.newCorrelationId(),
|
||||
raftRequestVersion(request),
|
||||
request,
|
||||
time.milliseconds()
|
||||
);
|
||||
inboundRequest.completion.whenComplete((response, exception) -> {
|
||||
if (exception != null) {
|
||||
throw new RuntimeException(exception);
|
||||
|
@ -655,6 +701,23 @@ public final class RaftClientTestContext {
|
|||
assertInstanceOf(BeginQuorumEpochResponseData.class, raftMessage.data());
|
||||
BeginQuorumEpochResponseData response = (BeginQuorumEpochResponseData) raftMessage.data();
|
||||
assertEquals(responseError, Errors.forCode(response.errorCode()));
|
||||
|
||||
if (!response.topics().isEmpty()) {
|
||||
BeginQuorumEpochResponseData.PartitionData partitionResponse = response
|
||||
.topics()
|
||||
.get(0)
|
||||
.partitions()
|
||||
.get(0);
|
||||
if (kip853Rpc && partitionResponse.leaderId() >= 0) {
|
||||
int leaderId = partitionResponse.leaderId();
|
||||
Endpoints expectedLeaderEndpoints = voters.listeners(leaderId);
|
||||
Endpoints responseEndpoints = Endpoints.fromBeginQuorumEpochResponse(
|
||||
channel.listenerName(),
|
||||
response.nodeEndpoints()
|
||||
);
|
||||
assertEquals(expectedLeaderEndpoints, responseEndpoints);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void assertSentBeginQuorumEpochResponse(
|
||||
|
@ -675,6 +738,15 @@ public final class RaftClientTestContext {
|
|||
assertEquals(epoch, partitionResponse.leaderEpoch());
|
||||
assertEquals(leaderId.orElse(-1), partitionResponse.leaderId());
|
||||
assertEquals(partitionError, Errors.forCode(partitionResponse.errorCode()));
|
||||
|
||||
if (kip853Rpc && leaderId.isPresent()) {
|
||||
Endpoints expectedLeaderEndpoints = voters.listeners(leaderId.getAsInt());
|
||||
Endpoints responseEndpoints = Endpoints.fromBeginQuorumEpochResponse(
|
||||
channel.listenerName(),
|
||||
response.nodeEndpoints()
|
||||
);
|
||||
assertEquals(expectedLeaderEndpoints, responseEndpoints);
|
||||
}
|
||||
}
|
||||
|
||||
RaftRequest.Outbound assertSentEndQuorumEpochRequest(int epoch, int destinationId) {
|
||||
|
@ -696,6 +768,23 @@ public final class RaftClientTestContext {
|
|||
assertInstanceOf(EndQuorumEpochResponseData.class, raftMessage.data());
|
||||
EndQuorumEpochResponseData response = (EndQuorumEpochResponseData) raftMessage.data();
|
||||
assertEquals(responseError, Errors.forCode(response.errorCode()));
|
||||
|
||||
if (!response.topics().isEmpty()) {
|
||||
EndQuorumEpochResponseData.PartitionData partitionResponse = response
|
||||
.topics()
|
||||
.get(0)
|
||||
.partitions()
|
||||
.get(0);
|
||||
if (kip853Rpc && partitionResponse.leaderId() >= 0) {
|
||||
int leaderId = partitionResponse.leaderId();
|
||||
Endpoints expectedLeaderEndpoints = voters.listeners(leaderId);
|
||||
Endpoints responseEndpoints = Endpoints.fromEndQuorumEpochResponse(
|
||||
channel.listenerName(),
|
||||
response.nodeEndpoints()
|
||||
);
|
||||
assertEquals(expectedLeaderEndpoints, responseEndpoints);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void assertSentEndQuorumEpochResponse(
|
||||
|
@ -716,6 +805,15 @@ public final class RaftClientTestContext {
|
|||
assertEquals(epoch, partitionResponse.leaderEpoch());
|
||||
assertEquals(leaderId.orElse(-1), partitionResponse.leaderId());
|
||||
assertEquals(partitionError, Errors.forCode(partitionResponse.errorCode()));
|
||||
|
||||
if (kip853Rpc && leaderId.isPresent()) {
|
||||
Endpoints expectedLeaderEndpoints = voters.listeners(leaderId.getAsInt());
|
||||
Endpoints responseEndpoints = Endpoints.fromEndQuorumEpochResponse(
|
||||
channel.listenerName(),
|
||||
response.nodeEndpoints()
|
||||
);
|
||||
assertEquals(expectedLeaderEndpoints, responseEndpoints);
|
||||
}
|
||||
}
|
||||
|
||||
RaftRequest.Outbound assertSentFetchRequest() {
|
||||
|
@ -749,7 +847,19 @@ public final class RaftClientTestContext {
|
|||
assertEquals(1, response.responses().size());
|
||||
assertEquals(metadataPartition.topic(), response.responses().get(0).topic());
|
||||
assertEquals(1, response.responses().get(0).partitions().size());
|
||||
return response.responses().get(0).partitions().get(0);
|
||||
|
||||
FetchResponseData.PartitionData partitionResponse = response.responses().get(0).partitions().get(0);
|
||||
if (kip853Rpc && partitionResponse.currentLeader().leaderId() >= 0) {
|
||||
int leaderId = partitionResponse.currentLeader().leaderId();
|
||||
Endpoints expectedLeaderEndpoints = voters.listeners(leaderId);
|
||||
Endpoints responseEndpoints = Endpoints.fromFetchResponse(
|
||||
channel.listenerName(),
|
||||
response.nodeEndpoints()
|
||||
);
|
||||
assertEquals(expectedLeaderEndpoints, responseEndpoints);
|
||||
}
|
||||
|
||||
return partitionResponse;
|
||||
}
|
||||
|
||||
void assertSentFetchPartitionResponse(Errors topLevelError) {
|
||||
|
@ -776,6 +886,7 @@ public final class RaftClientTestContext {
|
|||
assertEquals(-1, partitionResponse.divergingEpoch().epoch());
|
||||
assertEquals(-1, partitionResponse.snapshotId().endOffset());
|
||||
assertEquals(-1, partitionResponse.snapshotId().epoch());
|
||||
|
||||
return (MemoryRecords) partitionResponse.records();
|
||||
}
|
||||
|
||||
|
@ -802,6 +913,19 @@ public final class RaftClientTestContext {
|
|||
}
|
||||
|
||||
void assertSentFetchSnapshotResponse(Errors responseError) {
|
||||
assertSentFetchSnapshotResponse(responseError, metadataPartition);
|
||||
}
|
||||
|
||||
Optional<FetchSnapshotResponseData.PartitionSnapshot> assertSentFetchSnapshotResponse(
|
||||
TopicPartition topicPartition
|
||||
) {
|
||||
return assertSentFetchSnapshotResponse(Errors.NONE, topicPartition);
|
||||
}
|
||||
|
||||
Optional<FetchSnapshotResponseData.PartitionSnapshot> assertSentFetchSnapshotResponse(
|
||||
Errors responseError,
|
||||
TopicPartition topicPartition
|
||||
) {
|
||||
List<RaftResponse.Outbound> sentMessages = drainSentResponses(ApiKeys.FETCH_SNAPSHOT);
|
||||
assertEquals(1, sentMessages.size());
|
||||
|
||||
|
@ -810,19 +934,21 @@ public final class RaftClientTestContext {
|
|||
|
||||
FetchSnapshotResponseData response = (FetchSnapshotResponseData) message.data();
|
||||
assertEquals(responseError, Errors.forCode(response.errorCode()));
|
||||
}
|
||||
|
||||
Optional<FetchSnapshotResponseData.PartitionSnapshot> assertSentFetchSnapshotResponse(TopicPartition topicPartition) {
|
||||
List<RaftResponse.Outbound> sentMessages = drainSentResponses(ApiKeys.FETCH_SNAPSHOT);
|
||||
assertEquals(1, sentMessages.size());
|
||||
Optional<FetchSnapshotResponseData.PartitionSnapshot> result =
|
||||
FetchSnapshotResponse.forTopicPartition(response, topicPartition);
|
||||
|
||||
RaftMessage message = sentMessages.get(0);
|
||||
assertInstanceOf(FetchSnapshotResponseData.class, message.data());
|
||||
if (result.isPresent() && kip853Rpc && result.get().currentLeader().leaderId() >= 0) {
|
||||
int leaderId = result.get().currentLeader().leaderId();
|
||||
Endpoints expectedLeaderEndpoints = voters.listeners(leaderId);
|
||||
Endpoints responseEndpoints = Endpoints.fromFetchSnapshotResponse(
|
||||
channel.listenerName(),
|
||||
response.nodeEndpoints()
|
||||
);
|
||||
assertEquals(expectedLeaderEndpoints, responseEndpoints);
|
||||
}
|
||||
|
||||
FetchSnapshotResponseData response = (FetchSnapshotResponseData) message.data();
|
||||
assertEquals(Errors.NONE, Errors.forCode(response.errorCode()));
|
||||
|
||||
return FetchSnapshotResponse.forTopicPartition(response, topicPartition);
|
||||
return result;
|
||||
}
|
||||
|
||||
List<RaftRequest.Outbound> collectEndQuorumRequests(
|
||||
|
@ -861,7 +987,7 @@ public final class RaftClientTestContext {
|
|||
RaftRequest.Outbound fetchRequest = assertSentFetchRequest();
|
||||
int destinationId = fetchRequest.destination().id();
|
||||
assertTrue(
|
||||
voters.contains(destinationId) || bootstrapIds.contains(destinationId),
|
||||
voters.voterIds().contains(destinationId) || bootstrapIds.contains(destinationId),
|
||||
String.format("id %d is not in sets %s or %s", destinationId, voters, bootstrapIds)
|
||||
);
|
||||
assertFetchRequestData(fetchRequest, 0, 0L, 0);
|
||||
|
@ -892,32 +1018,35 @@ public final class RaftClientTestContext {
|
|||
}
|
||||
|
||||
public static InetSocketAddress mockAddress(int id) {
|
||||
return new InetSocketAddress("localhost", 9990 + id);
|
||||
return InetSocketAddress.createUnresolved("localhost", 9990 + id);
|
||||
}
|
||||
|
||||
EndQuorumEpochResponseData endEpochResponse(
|
||||
int epoch,
|
||||
OptionalInt leaderId
|
||||
) {
|
||||
return EndQuorumEpochResponse.singletonResponse(
|
||||
return RaftUtil.singletonEndQuorumEpochResponse(
|
||||
channel.listenerName(),
|
||||
endQuorumEpochRpcVersion(),
|
||||
Errors.NONE,
|
||||
metadataPartition,
|
||||
Errors.NONE,
|
||||
epoch,
|
||||
leaderId.orElse(-1)
|
||||
leaderId.orElse(-1),
|
||||
Endpoints.empty() // KAFKA-16529 will fix this
|
||||
);
|
||||
}
|
||||
|
||||
EndQuorumEpochRequestData endEpochRequest(
|
||||
int epoch,
|
||||
int leaderId,
|
||||
List<Integer> preferredSuccessors
|
||||
List<ReplicaKey> preferredCandidates
|
||||
) {
|
||||
return EndQuorumEpochRequest.singletonRequest(
|
||||
metadataPartition,
|
||||
return endEpochRequest(
|
||||
clusterId.toString(),
|
||||
epoch,
|
||||
leaderId,
|
||||
preferredSuccessors
|
||||
preferredCandidates
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -925,50 +1054,68 @@ public final class RaftClientTestContext {
|
|||
String clusterId,
|
||||
int epoch,
|
||||
int leaderId,
|
||||
List<Integer> preferredSuccessors
|
||||
List<ReplicaKey> preferredCandidates
|
||||
) {
|
||||
return EndQuorumEpochRequest.singletonRequest(
|
||||
return RaftUtil.singletonEndQuorumEpochRequest(
|
||||
metadataPartition,
|
||||
clusterId,
|
||||
epoch,
|
||||
leaderId,
|
||||
preferredSuccessors
|
||||
);
|
||||
}
|
||||
|
||||
BeginQuorumEpochRequestData beginEpochRequest(String clusterId, int epoch, int leaderId) {
|
||||
return BeginQuorumEpochRequest.singletonRequest(
|
||||
metadataPartition,
|
||||
clusterId,
|
||||
epoch,
|
||||
leaderId
|
||||
preferredCandidates
|
||||
);
|
||||
}
|
||||
|
||||
BeginQuorumEpochRequestData beginEpochRequest(int epoch, int leaderId) {
|
||||
return BeginQuorumEpochRequest.singletonRequest(
|
||||
return beginEpochRequest(clusterId.toString(), epoch, leaderId);
|
||||
}
|
||||
|
||||
BeginQuorumEpochRequestData beginEpochRequest(String clusterId, int epoch, int leaderId) {
|
||||
ReplicaKey localReplicaKey = kip853Rpc ?
|
||||
ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID) :
|
||||
ReplicaKey.of(localIdOrThrow(), localDirectoryId);
|
||||
|
||||
return beginEpochRequest(clusterId, epoch, leaderId, localReplicaKey);
|
||||
}
|
||||
|
||||
BeginQuorumEpochRequestData beginEpochRequest(
|
||||
String clusterId,
|
||||
int epoch,
|
||||
int leaderId,
|
||||
ReplicaKey voterKey
|
||||
) {
|
||||
return RaftUtil.singletonBeginQuorumEpochRequest(
|
||||
metadataPartition,
|
||||
clusterId,
|
||||
epoch,
|
||||
leaderId
|
||||
leaderId,
|
||||
voters.listeners(leaderId),
|
||||
voterKey
|
||||
);
|
||||
}
|
||||
|
||||
private BeginQuorumEpochResponseData beginEpochResponse(int epoch, int leaderId) {
|
||||
return BeginQuorumEpochResponse.singletonResponse(
|
||||
return RaftUtil.singletonBeginQuorumEpochResponse(
|
||||
channel.listenerName(),
|
||||
beginQuorumEpochRpcVersion(),
|
||||
Errors.NONE,
|
||||
metadataPartition,
|
||||
Errors.NONE,
|
||||
epoch,
|
||||
leaderId
|
||||
leaderId,
|
||||
Endpoints.empty() // KAFKA-16529 will fix this
|
||||
);
|
||||
}
|
||||
|
||||
VoteRequestData voteRequest(int epoch, int candidateId, int lastEpoch, long lastEpochOffset) {
|
||||
return VoteRequest.singletonRequest(
|
||||
metadataPartition,
|
||||
VoteRequestData voteRequest(
|
||||
int epoch,
|
||||
ReplicaKey candidateKey,
|
||||
int lastEpoch,
|
||||
long lastEpochOffset
|
||||
) {
|
||||
return voteRequest(
|
||||
clusterId.toString(),
|
||||
epoch,
|
||||
candidateId,
|
||||
candidateKey,
|
||||
lastEpoch,
|
||||
lastEpochOffset
|
||||
);
|
||||
|
@ -977,28 +1124,54 @@ public final class RaftClientTestContext {
|
|||
VoteRequestData voteRequest(
|
||||
String clusterId,
|
||||
int epoch,
|
||||
int candidateId,
|
||||
ReplicaKey candidateKey,
|
||||
int lastEpoch,
|
||||
long lastEpochOffset
|
||||
) {
|
||||
return VoteRequest.singletonRequest(
|
||||
ReplicaKey localReplicaKey = kip853Rpc ?
|
||||
ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID) :
|
||||
ReplicaKey.of(localIdOrThrow(), localDirectoryId);
|
||||
|
||||
return voteRequest(
|
||||
clusterId,
|
||||
epoch,
|
||||
candidateKey,
|
||||
localReplicaKey,
|
||||
lastEpoch,
|
||||
lastEpochOffset
|
||||
);
|
||||
}
|
||||
|
||||
VoteRequestData voteRequest(
|
||||
String clusterId,
|
||||
int epoch,
|
||||
ReplicaKey candidateKey,
|
||||
ReplicaKey voterKey,
|
||||
int lastEpoch,
|
||||
long lastEpochOffset
|
||||
) {
|
||||
return RaftUtil.singletonVoteRequest(
|
||||
metadataPartition,
|
||||
clusterId,
|
||||
epoch,
|
||||
candidateId,
|
||||
candidateKey,
|
||||
voterKey,
|
||||
lastEpoch,
|
||||
lastEpochOffset
|
||||
);
|
||||
}
|
||||
|
||||
VoteResponseData voteResponse(boolean voteGranted, Optional<Integer> leaderId, int epoch) {
|
||||
return VoteResponse.singletonResponse(
|
||||
return RaftUtil.singletonVoteResponse(
|
||||
channel.listenerName(),
|
||||
voteRpcVersion(),
|
||||
Errors.NONE,
|
||||
metadataPartition,
|
||||
Errors.NONE,
|
||||
epoch,
|
||||
leaderId.orElse(-1),
|
||||
voteGranted
|
||||
voteGranted,
|
||||
Endpoints.empty() // KAFKA-16529 will fix this
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -1062,7 +1235,7 @@ public final class RaftClientTestContext {
|
|||
assertEquals(localId.orElse(-1), request.replicaState().replicaId());
|
||||
|
||||
// Assert that voters have flushed up to the fetch offset
|
||||
if (localId.isPresent() && voters.contains(localId.getAsInt())) {
|
||||
if (localId.isPresent() && voters.voterIds().contains(localId.getAsInt())) {
|
||||
assertEquals(
|
||||
log.firstUnflushedOffset(),
|
||||
fetchOffset,
|
||||
|
@ -1079,7 +1252,7 @@ public final class RaftClientTestContext {
|
|||
|
||||
FetchRequestData fetchRequest(
|
||||
int epoch,
|
||||
int replicaId,
|
||||
ReplicaKey replicaKey,
|
||||
long fetchOffset,
|
||||
int lastFetchedEpoch,
|
||||
int maxWaitTimeMs
|
||||
|
@ -1087,7 +1260,7 @@ public final class RaftClientTestContext {
|
|||
return fetchRequest(
|
||||
epoch,
|
||||
clusterId.toString(),
|
||||
replicaId,
|
||||
replicaKey,
|
||||
fetchOffset,
|
||||
lastFetchedEpoch,
|
||||
maxWaitTimeMs
|
||||
|
@ -1097,21 +1270,28 @@ public final class RaftClientTestContext {
|
|||
FetchRequestData fetchRequest(
|
||||
int epoch,
|
||||
String clusterId,
|
||||
int replicaId,
|
||||
ReplicaKey replicaKey,
|
||||
long fetchOffset,
|
||||
int lastFetchedEpoch,
|
||||
int maxWaitTimeMs
|
||||
) {
|
||||
FetchRequestData request = RaftUtil.singletonFetchRequest(metadataPartition, metadataTopicId, fetchPartition -> {
|
||||
fetchPartition
|
||||
FetchRequestData request = RaftUtil.singletonFetchRequest(
|
||||
metadataPartition,
|
||||
metadataTopicId,
|
||||
fetchPartition -> fetchPartition
|
||||
.setCurrentLeaderEpoch(epoch)
|
||||
.setLastFetchedEpoch(lastFetchedEpoch)
|
||||
.setFetchOffset(fetchOffset);
|
||||
});
|
||||
.setFetchOffset(fetchOffset)
|
||||
.setReplicaDirectoryId(
|
||||
replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)
|
||||
)
|
||||
);
|
||||
return request
|
||||
.setMaxWaitMs(maxWaitTimeMs)
|
||||
.setClusterId(clusterId)
|
||||
.setReplicaState(new FetchRequestData.ReplicaState().setReplicaId(replicaId));
|
||||
.setReplicaState(
|
||||
new FetchRequestData.ReplicaState().setReplicaId(replicaKey.id())
|
||||
);
|
||||
}
|
||||
|
||||
FetchResponseData fetchResponse(
|
||||
|
@ -1121,16 +1301,25 @@ public final class RaftClientTestContext {
|
|||
long highWatermark,
|
||||
Errors error
|
||||
) {
|
||||
return RaftUtil.singletonFetchResponse(metadataPartition, metadataTopicId, Errors.NONE, partitionData -> {
|
||||
partitionData
|
||||
.setRecords(records)
|
||||
.setErrorCode(error.code())
|
||||
.setHighWatermark(highWatermark);
|
||||
return RaftUtil.singletonFetchResponse(
|
||||
channel.listenerName(),
|
||||
fetchRpcVersion(),
|
||||
metadataPartition,
|
||||
metadataTopicId,
|
||||
Errors.NONE,
|
||||
leaderId,
|
||||
Endpoints.empty(), // KAFKA-16529 will fix this
|
||||
partitionData -> {
|
||||
partitionData
|
||||
.setRecords(records)
|
||||
.setErrorCode(error.code())
|
||||
.setHighWatermark(highWatermark);
|
||||
|
||||
partitionData.currentLeader()
|
||||
.setLeaderEpoch(epoch)
|
||||
.setLeaderId(leaderId);
|
||||
});
|
||||
partitionData.currentLeader()
|
||||
.setLeaderEpoch(epoch)
|
||||
.setLeaderId(leaderId);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
FetchResponseData divergingFetchResponse(
|
||||
|
@ -1140,29 +1329,156 @@ public final class RaftClientTestContext {
|
|||
int divergingEpoch,
|
||||
long highWatermark
|
||||
) {
|
||||
return RaftUtil.singletonFetchResponse(metadataPartition, metadataTopicId, Errors.NONE, partitionData -> {
|
||||
partitionData.setHighWatermark(highWatermark);
|
||||
return RaftUtil.singletonFetchResponse(
|
||||
channel.listenerName(),
|
||||
fetchRpcVersion(),
|
||||
metadataPartition,
|
||||
metadataTopicId,
|
||||
Errors.NONE,
|
||||
leaderId,
|
||||
Endpoints.empty(), // KAFKA-16529 will fix this
|
||||
partitionData -> {
|
||||
partitionData.setHighWatermark(highWatermark);
|
||||
|
||||
partitionData.currentLeader()
|
||||
.setLeaderEpoch(epoch)
|
||||
.setLeaderId(leaderId);
|
||||
partitionData.currentLeader()
|
||||
.setLeaderEpoch(epoch)
|
||||
.setLeaderId(leaderId);
|
||||
|
||||
partitionData.divergingEpoch()
|
||||
.setEpoch(divergingEpoch)
|
||||
.setEndOffset(divergingEpochEndOffset);
|
||||
});
|
||||
partitionData.divergingEpoch()
|
||||
.setEpoch(divergingEpoch)
|
||||
.setEndOffset(divergingEpochEndOffset);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
FetchResponseData snapshotFetchResponse(
|
||||
int epoch,
|
||||
int leaderId,
|
||||
OffsetAndEpoch snapshotId,
|
||||
long highWatermark
|
||||
) {
|
||||
return RaftUtil.singletonFetchResponse(
|
||||
channel.listenerName(),
|
||||
fetchRpcVersion(),
|
||||
metadataPartition,
|
||||
metadataTopicId,
|
||||
Errors.NONE,
|
||||
leaderId,
|
||||
Endpoints.empty(), // KAFKA-16529 will fix this
|
||||
partitionData -> {
|
||||
partitionData.setHighWatermark(highWatermark);
|
||||
|
||||
partitionData.currentLeader()
|
||||
.setLeaderEpoch(epoch)
|
||||
.setLeaderId(leaderId);
|
||||
|
||||
partitionData.snapshotId()
|
||||
.setEpoch(snapshotId.epoch())
|
||||
.setEndOffset(snapshotId.offset());
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
FetchSnapshotResponseData fetchSnapshotResponse(
|
||||
int leaderId,
|
||||
UnaryOperator<FetchSnapshotResponseData.PartitionSnapshot> operator
|
||||
) {
|
||||
return RaftUtil.singletonFetchSnapshotResponse(
|
||||
channel.listenerName(),
|
||||
fetchSnapshotRpcVersion(),
|
||||
metadataPartition,
|
||||
leaderId,
|
||||
Endpoints.empty(), // KAFKA-16529 will fix this
|
||||
operator
|
||||
);
|
||||
}
|
||||
|
||||
DescribeQuorumRequestData describeQuorumRequest() {
|
||||
return RaftUtil.singletonDescribeQuorumRequest(metadataPartition);
|
||||
}
|
||||
|
||||
private short fetchRpcVersion() {
|
||||
if (kip853Rpc) {
|
||||
return 17;
|
||||
} else {
|
||||
return 16;
|
||||
}
|
||||
}
|
||||
|
||||
private short fetchSnapshotRpcVersion() {
|
||||
if (kip853Rpc) {
|
||||
return 1;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private short voteRpcVersion() {
|
||||
if (kip853Rpc) {
|
||||
return 1;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private short beginQuorumEpochRpcVersion() {
|
||||
if (kip853Rpc) {
|
||||
return 1;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private short endQuorumEpochRpcVersion() {
|
||||
if (kip853Rpc) {
|
||||
return 1;
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private short describeQuorumRpcVersion() {
|
||||
if (kip853Rpc) {
|
||||
return 2;
|
||||
} else {
|
||||
return 1;
|
||||
}
|
||||
}
|
||||
|
||||
private short raftRequestVersion(ApiMessage request) {
|
||||
if (request instanceof FetchRequestData) {
|
||||
return fetchRpcVersion();
|
||||
} else if (request instanceof FetchSnapshotRequestData) {
|
||||
return fetchSnapshotRpcVersion();
|
||||
} else if (request instanceof VoteRequestData) {
|
||||
return voteRpcVersion();
|
||||
} else if (request instanceof BeginQuorumEpochRequestData) {
|
||||
return beginQuorumEpochRpcVersion();
|
||||
} else if (request instanceof EndQuorumEpochRequestData) {
|
||||
return endQuorumEpochRpcVersion();
|
||||
} else if (request instanceof DescribeQuorumRequestData) {
|
||||
return describeQuorumRpcVersion();
|
||||
} else {
|
||||
throw new IllegalArgumentException(String.format("Request %s is not a raft request", request));
|
||||
}
|
||||
}
|
||||
|
||||
public void advanceLocalLeaderHighWatermarkToLogEndOffset() throws InterruptedException {
|
||||
assertEquals(localId, currentLeader());
|
||||
long localLogEndOffset = log.endOffset().offset;
|
||||
Set<Integer> followers = voters.stream().filter(voter -> voter != localId.getAsInt()).collect(Collectors.toSet());
|
||||
|
||||
// Send a request from every follower
|
||||
for (int follower : followers) {
|
||||
Iterable<ReplicaKey> followers = () -> voters
|
||||
.voterKeys()
|
||||
.stream()
|
||||
.filter(voterKey -> voterKey.id() != localId.getAsInt())
|
||||
.iterator();
|
||||
|
||||
// Send a request from every voter
|
||||
for (ReplicaKey follower : followers) {
|
||||
deliverRequest(
|
||||
fetchRequest(currentEpoch(), follower, localLogEndOffset, currentEpoch(), 0)
|
||||
);
|
||||
|
||||
pollUntilResponse();
|
||||
assertSentFetchPartitionResponse(Errors.NONE, currentEpoch(), localId);
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ import org.apache.kafka.common.Node;
|
|||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.memory.MemoryPool;
|
||||
import org.apache.kafka.common.message.ApiMessageType;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.protocol.ObjectSerializationCache;
|
||||
import org.apache.kafka.common.protocol.Readable;
|
||||
|
@ -772,6 +773,7 @@ public class RaftEventSimulationTest {
|
|||
FETCH_MAX_WAIT_MS,
|
||||
clusterId.toString(),
|
||||
Collections.emptyList(),
|
||||
Endpoints.empty(), // KAFKA-16529 will fix this
|
||||
logContext,
|
||||
random,
|
||||
quorumConfig
|
||||
|
@ -1236,8 +1238,20 @@ public class RaftEventSimulationTest {
|
|||
|
||||
int correlationId = outbound.correlationId();
|
||||
Node destination = outbound.destination();
|
||||
RaftRequest.Inbound inbound = new RaftRequest.Inbound(correlationId, outbound.apiVersion(), outbound.data(),
|
||||
cluster.time.milliseconds());
|
||||
RaftRequest.Inbound inbound = cluster
|
||||
.nodeIfRunning(senderId)
|
||||
.map(node ->
|
||||
new RaftRequest.Inbound(
|
||||
node.channel.listenerName(),
|
||||
correlationId,
|
||||
ApiMessageType
|
||||
.fromApiKey(outbound.data().apiKey())
|
||||
.highestSupportedVersion(true),
|
||||
outbound.data(),
|
||||
cluster.time.milliseconds()
|
||||
)
|
||||
)
|
||||
.get();
|
||||
|
||||
if (!filters.get(destination.id()).acceptInbound(inbound))
|
||||
return;
|
||||
|
|
|
@ -20,18 +20,20 @@ import org.apache.kafka.common.utils.LogContext;
|
|||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.internals.ReplicaKey;
|
||||
import org.apache.kafka.raft.internals.VoterSetTest;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
|
@ -42,6 +44,12 @@ class ResignedStateTest {
|
|||
int electionTimeoutMs = 5000;
|
||||
int localId = 0;
|
||||
int epoch = 5;
|
||||
Endpoints localEndpoints = Endpoints.fromInetSocketAddresses(
|
||||
Collections.singletonMap(
|
||||
VoterSetTest.DEFAULT_LISTENER_NAME,
|
||||
InetSocketAddress.createUnresolved("localhost", 1234)
|
||||
)
|
||||
);
|
||||
|
||||
private ResignedState newResignedState(
|
||||
Set<Integer> voters,
|
||||
|
@ -54,6 +62,7 @@ class ResignedStateTest {
|
|||
voters,
|
||||
electionTimeoutMs,
|
||||
preferredSuccessors,
|
||||
localEndpoints,
|
||||
logContext
|
||||
);
|
||||
}
|
||||
|
@ -90,9 +99,9 @@ class ResignedStateTest {
|
|||
Collections.emptyList()
|
||||
);
|
||||
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -107,4 +116,15 @@ class ResignedStateTest {
|
|||
// try non-existed voter must throw an exception
|
||||
assertThrows(IllegalArgumentException.class, () -> state.acknowledgeResignation(10));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testLeaderEndpoints() {
|
||||
ResignedState state = newResignedState(
|
||||
Utils.mkSet(1, 2, 3),
|
||||
Collections.emptyList()
|
||||
);
|
||||
|
||||
assertEquals(localEndpoints, state.leaderEndpoints());
|
||||
assertNotEquals(Endpoints.empty(), state.leaderEndpoints());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -78,21 +78,31 @@ public class UnattachedStateTest {
|
|||
@ValueSource(booleans = {true, false})
|
||||
public void testGrantVote(boolean isLogUpToDate) {
|
||||
UnattachedState state = newUnattachedState(
|
||||
Utils.mkSet(1, 2, 3),
|
||||
Optional.empty()
|
||||
Utils.mkSet(1, 2, 3),
|
||||
Optional.empty()
|
||||
);
|
||||
|
||||
assertEquals(
|
||||
isLogUpToDate,
|
||||
state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate)
|
||||
state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)
|
||||
);
|
||||
assertEquals(
|
||||
isLogUpToDate,
|
||||
state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate)
|
||||
state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)
|
||||
);
|
||||
assertEquals(
|
||||
isLogUpToDate,
|
||||
state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate)
|
||||
state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testLeaderEndpoints() {
|
||||
UnattachedState state = newUnattachedState(
|
||||
Utils.mkSet(1, 2, 3),
|
||||
Optional.empty()
|
||||
);
|
||||
|
||||
assertEquals(Endpoints.empty(), state.leaderEndpoints());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,15 +41,14 @@ class VotedStateTest {
|
|||
private final int electionTimeoutMs = 10000;
|
||||
|
||||
private VotedState newVotedState(
|
||||
Optional<Uuid> votedDirectoryId,
|
||||
Optional<LogOffsetMetadata> highWatermark
|
||||
Uuid votedDirectoryId
|
||||
) {
|
||||
return new VotedState(
|
||||
time,
|
||||
epoch,
|
||||
ReplicaKey.of(votedId, votedDirectoryId),
|
||||
Collections.emptySet(),
|
||||
highWatermark,
|
||||
Optional.empty(),
|
||||
electionTimeoutMs,
|
||||
logContext
|
||||
);
|
||||
|
@ -57,8 +56,8 @@ class VotedStateTest {
|
|||
|
||||
@Test
|
||||
public void testElectionTimeout() {
|
||||
VotedState state = newVotedState(Optional.empty(), Optional.empty());
|
||||
ReplicaKey votedKey = ReplicaKey.of(votedId, Optional.empty());
|
||||
VotedState state = newVotedState(ReplicaKey.NO_DIRECTORY_ID);
|
||||
ReplicaKey votedKey = ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID);
|
||||
|
||||
assertEquals(epoch, state.epoch());
|
||||
assertEquals(votedKey, state.votedKey());
|
||||
|
@ -81,36 +80,44 @@ class VotedStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) {
|
||||
VotedState state = newVotedState(Optional.empty(), Optional.empty());
|
||||
VotedState state = newVotedState(ReplicaKey.NO_DIRECTORY_ID);
|
||||
|
||||
assertTrue(
|
||||
state.canGrantVote(ReplicaKey.of(votedId, Optional.empty()), isLogUpToDate)
|
||||
state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)
|
||||
);
|
||||
assertTrue(
|
||||
state.canGrantVote(
|
||||
ReplicaKey.of(votedId, Optional.of(Uuid.randomUuid())),
|
||||
ReplicaKey.of(votedId, Uuid.randomUuid()),
|
||||
isLogUpToDate
|
||||
)
|
||||
);
|
||||
|
||||
assertFalse(
|
||||
state.canGrantVote(ReplicaKey.of(votedId + 1, Optional.empty()), isLogUpToDate)
|
||||
state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testCanGrantVoteWithDirectoryId() {
|
||||
Optional<Uuid> votedDirectoryId = Optional.of(Uuid.randomUuid());
|
||||
VotedState state = newVotedState(votedDirectoryId, Optional.empty());
|
||||
Uuid votedDirectoryId = Uuid.randomUuid();
|
||||
VotedState state = newVotedState(votedDirectoryId);
|
||||
|
||||
assertTrue(state.canGrantVote(ReplicaKey.of(votedId, votedDirectoryId), false));
|
||||
|
||||
assertFalse(
|
||||
state.canGrantVote(ReplicaKey.of(votedId, Optional.of(Uuid.randomUuid())), false)
|
||||
state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), false)
|
||||
);
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(votedId, Optional.empty()), false));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), false));
|
||||
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), false));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, Optional.empty()), false));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), false));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testLeaderEndpoints() {
|
||||
Uuid votedDirectoryId = Uuid.randomUuid();
|
||||
VotedState state = newVotedState(votedDirectoryId);
|
||||
|
||||
assertEquals(Endpoints.empty(), state.leaderEndpoints());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.mockito.Mockito;
|
|||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.Random;
|
||||
|
@ -68,9 +67,9 @@ public class KafkaRaftMetricsTest {
|
|||
return new QuorumState(
|
||||
OptionalInt.of(localId),
|
||||
localDirectoryId,
|
||||
VoterSetTest.DEFAULT_LISTENER_NAME,
|
||||
() -> voterSet,
|
||||
() -> kraftVersion,
|
||||
voterSet.listeners(localId),
|
||||
electionTimeoutMs,
|
||||
fetchTimeoutMs,
|
||||
new MockQuorumStateStore(),
|
||||
|
@ -88,7 +87,7 @@ public class KafkaRaftMetricsTest {
|
|||
VoterSetTest.voterNode(
|
||||
ReplicaKey.of(
|
||||
localId,
|
||||
withDirectoryId ? Optional.of(localDirectoryId) : Optional.empty()
|
||||
withDirectoryId ? localDirectoryId : ReplicaKey.NO_DIRECTORY_ID
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -105,7 +104,7 @@ public class KafkaRaftMetricsTest {
|
|||
VoterSetTest.voterNode(
|
||||
ReplicaKey.of(
|
||||
localId,
|
||||
withDirectoryId ? Optional.of(localDirectoryId) : Optional.empty()
|
||||
withDirectoryId ? localDirectoryId : ReplicaKey.NO_DIRECTORY_ID
|
||||
)
|
||||
)
|
||||
);
|
||||
|
@ -148,11 +147,15 @@ public class KafkaRaftMetricsTest {
|
|||
assertEquals((double) 1, getMetric(metrics, "current-epoch").metricValue());
|
||||
assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue());
|
||||
|
||||
state.leaderStateOrThrow().updateLocalState(new LogOffsetMetadata(5L), voters.voters());
|
||||
state.leaderStateOrThrow().updateReplicaState(1, Uuid.randomUuid(), 0, new LogOffsetMetadata(5L));
|
||||
state.leaderStateOrThrow().updateLocalState(new LogOffsetMetadata(5L), voters);
|
||||
state.leaderStateOrThrow().updateReplicaState(
|
||||
voterMap.get(1).voterKey(),
|
||||
0,
|
||||
new LogOffsetMetadata(5L)
|
||||
);
|
||||
assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue());
|
||||
|
||||
state.transitionToFollower(2, voters.voterNode(1, VoterSetTest.DEFAULT_LISTENER_NAME).get());
|
||||
state.transitionToFollower(2, 1, voters.listeners(1));
|
||||
assertEquals("follower", getMetric(metrics, "current-state").metricValue());
|
||||
assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue());
|
||||
assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue());
|
||||
|
@ -166,7 +169,7 @@ public class KafkaRaftMetricsTest {
|
|||
state.followerStateOrThrow().updateHighWatermark(OptionalLong.of(10L));
|
||||
assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue());
|
||||
|
||||
state.transitionToVoted(3, ReplicaKey.of(2, Optional.empty()));
|
||||
state.transitionToVoted(3, ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID));
|
||||
assertEquals("voted", getMetric(metrics, "current-state").metricValue());
|
||||
assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue());
|
||||
assertEquals((double) 2, getMetric(metrics, "current-vote").metricValue());
|
||||
|
@ -210,7 +213,7 @@ public class KafkaRaftMetricsTest {
|
|||
assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue());
|
||||
assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue());
|
||||
|
||||
state.transitionToFollower(2, voters.voterNode(1, VoterSetTest.DEFAULT_LISTENER_NAME).get());
|
||||
state.transitionToFollower(2, 1, voters.listeners(1));
|
||||
assertEquals("observer", getMetric(metrics, "current-state").metricValue());
|
||||
assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue());
|
||||
assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue());
|
||||
|
|
|
@ -21,8 +21,11 @@ import org.apache.kafka.common.Uuid;
|
|||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.Endpoints;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Arrays;
|
||||
|
@ -38,6 +41,7 @@ import java.util.stream.Stream;
|
|||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
|
@ -101,8 +105,8 @@ public final class VoterSetTest {
|
|||
Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(IntStream.of(1, 2, 3), true);
|
||||
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
|
||||
|
||||
assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Optional.empty())));
|
||||
assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Optional.of(Uuid.randomUuid()))));
|
||||
assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Uuid.randomUuid())));
|
||||
|
||||
VoterSet.VoterNode voter3 = aVoterMap.remove(3);
|
||||
assertEquals(
|
||||
|
@ -117,15 +121,15 @@ public final class VoterSetTest {
|
|||
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
|
||||
|
||||
assertTrue(voterSet.isVoter(aVoterMap.get(1).voterKey()));
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid()))));
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(1, Optional.empty())));
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(1, Uuid.randomUuid())));
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertFalse(
|
||||
voterSet.isVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId()))
|
||||
voterSet.isVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId().get()))
|
||||
);
|
||||
assertFalse(
|
||||
voterSet.isVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId()))
|
||||
voterSet.isVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId().get()))
|
||||
);
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.empty())));
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -133,10 +137,46 @@ public final class VoterSetTest {
|
|||
Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(IntStream.of(1, 2, 3), false);
|
||||
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
|
||||
|
||||
assertTrue(voterSet.isVoter(ReplicaKey.of(1, Optional.empty())));
|
||||
assertTrue(voterSet.isVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid()))));
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.of(Uuid.randomUuid()))));
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.empty())));
|
||||
assertTrue(voterSet.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertTrue(voterSet.isVoter(ReplicaKey.of(1, Uuid.randomUuid())));
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(4, Uuid.randomUuid())));
|
||||
assertFalse(voterSet.isVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testVoterNodeIsVoterWithDirectoryId() {
|
||||
VoterSet.VoterNode voterNode = voterNode(1, true);
|
||||
|
||||
assertTrue(voterNode.isVoter(voterNode.voterKey()));
|
||||
assertFalse(voterNode.isVoter(ReplicaKey.of(1, Uuid.randomUuid())));
|
||||
assertFalse(voterNode.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertFalse(voterNode.isVoter(ReplicaKey.of(2, Uuid.randomUuid())));
|
||||
assertFalse(voterNode.isVoter(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertFalse(voterNode.isVoter(ReplicaKey.of(2, voterNode.voterKey().directoryId().get())));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testVoterNodeIsVoterWithoutDirectoryId() {
|
||||
VoterSet.VoterNode voterNode = voterNode(1, false);
|
||||
|
||||
assertTrue(voterNode.isVoter(voterNode.voterKey()));
|
||||
assertTrue(voterNode.isVoter(ReplicaKey.of(1, Uuid.randomUuid())));
|
||||
assertTrue(voterNode.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertTrue(voterNode.isVoter(ReplicaKey.of(1, Uuid.randomUuid())));
|
||||
assertFalse(voterNode.isVoter(ReplicaKey.of(2, Uuid.randomUuid())));
|
||||
assertFalse(voterNode.isVoter(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = { true, false })
|
||||
void testEndpoints(boolean withDirectoryId) {
|
||||
Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(IntStream.of(1, 2, 3), withDirectoryId);
|
||||
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
|
||||
|
||||
assertNotEquals(Endpoints.empty(), voterSet.listeners(1));
|
||||
assertNotEquals(Endpoints.empty(), voterSet.listeners(2));
|
||||
assertNotEquals(Endpoints.empty(), voterSet.listeners(3));
|
||||
assertEquals(Endpoints.empty(), voterSet.listeners(4));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -145,12 +185,12 @@ public final class VoterSetTest {
|
|||
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
|
||||
|
||||
assertTrue(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey()));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid()))));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.empty())));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Uuid.randomUuid())));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertFalse(
|
||||
voterSet.isOnlyVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId()))
|
||||
voterSet.isOnlyVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId().get()))
|
||||
);
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(4, Optional.empty())));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -159,15 +199,15 @@ public final class VoterSetTest {
|
|||
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
|
||||
|
||||
assertFalse(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey()));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid()))));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.empty())));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Uuid.randomUuid())));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
assertFalse(
|
||||
voterSet.isOnlyVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId()))
|
||||
voterSet.isOnlyVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId().get()))
|
||||
);
|
||||
assertFalse(
|
||||
voterSet.isOnlyVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId()))
|
||||
voterSet.isOnlyVoter(ReplicaKey.of(4, aVoterMap.get(1).voterKey().directoryId().get()))
|
||||
);
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(4, Optional.empty())));
|
||||
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID)));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -272,7 +312,7 @@ public final class VoterSetTest {
|
|||
return voterNode(
|
||||
ReplicaKey.of(
|
||||
id,
|
||||
withDirectoryId ? Optional.of(Uuid.randomUuid()) : Optional.empty()
|
||||
withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -280,11 +320,13 @@ public final class VoterSetTest {
|
|||
public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) {
|
||||
return new VoterSet.VoterNode(
|
||||
replicaKey,
|
||||
Collections.singletonMap(
|
||||
DEFAULT_LISTENER_NAME,
|
||||
InetSocketAddress.createUnresolved(
|
||||
String.format("replica-%d", replicaKey.id()),
|
||||
1234
|
||||
Endpoints.fromInetSocketAddresses(
|
||||
Collections.singletonMap(
|
||||
DEFAULT_LISTENER_NAME,
|
||||
InetSocketAddress.createUnresolved(
|
||||
String.format("replica-%d", replicaKey.id()),
|
||||
1234
|
||||
)
|
||||
)
|
||||
),
|
||||
new SupportedVersionRange((short) 0, (short) 0)
|
||||
|
|
|
@ -408,7 +408,10 @@ public enum MetadataVersion {
|
|||
}
|
||||
|
||||
public short fetchRequestVersion() {
|
||||
if (this.isAtLeast(IBP_3_7_IV4)) {
|
||||
if (this.isAtLeast(IBP_4_0_IV0)) {
|
||||
// KAFKA-17018 will change this to IBP_3_9_IVX when it is available
|
||||
return 17;
|
||||
} else if (this.isAtLeast(IBP_3_7_IV4)) {
|
||||
return 16;
|
||||
} else if (this.isAtLeast(IBP_3_5_IV1)) {
|
||||
return 15;
|
||||
|
|
Loading…
Reference in New Issue