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:
José Armando García Sancio 2024-06-25 16:45:15 -04:00 committed by GitHub
parent 5b0e96d785
commit adee6f0cc1
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
72 changed files with 4097 additions and 1885 deletions

View File

@ -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);
}
}

View File

@ -77,6 +77,7 @@ import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.InvalidTxnStateException; import org.apache.kafka.common.errors.InvalidTxnStateException;
import org.apache.kafka.common.errors.InvalidTxnTimeoutException; import org.apache.kafka.common.errors.InvalidTxnTimeoutException;
import org.apache.kafka.common.errors.InvalidUpdateVersionException; 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.KafkaStorageException;
import org.apache.kafka.common.errors.LeaderNotAvailableException; import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.errors.ListenerNotFoundException; 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), 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), 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), 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); private static final Logger log = LoggerFactory.getLogger(Errors.class);

View File

@ -68,16 +68,12 @@ public class BeginQuorumEpochRequest extends AbstractRequest {
return new BeginQuorumEpochRequest(new BeginQuorumEpochRequestData(new ByteBufferAccessor(buffer), version), version); return new BeginQuorumEpochRequest(new BeginQuorumEpochRequestData(new ByteBufferAccessor(buffer), version), version);
} }
public static BeginQuorumEpochRequestData singletonRequest(TopicPartition topicPartition, public static BeginQuorumEpochRequestData singletonRequest(
int leaderEpoch, TopicPartition topicPartition,
int leaderId) {
return singletonRequest(topicPartition, null, leaderEpoch, leaderId);
}
public static BeginQuorumEpochRequestData singletonRequest(TopicPartition topicPartition,
String clusterId, String clusterId,
int leaderEpoch, int leaderEpoch,
int leaderId) { int leaderId
) {
return new BeginQuorumEpochRequestData() return new BeginQuorumEpochRequestData()
.setClusterId(clusterId) .setClusterId(clusterId)
.setTopics(Collections.singletonList( .setTopics(Collections.singletonList(
@ -90,5 +86,4 @@ public class BeginQuorumEpochRequest extends AbstractRequest {
.setLeaderId(leaderId)))) .setLeaderId(leaderId))))
); );
} }
} }

View File

@ -17,14 +17,12 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.BeginQuorumEpochResponseData; import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
@ -49,27 +47,6 @@ public class BeginQuorumEpochResponse extends AbstractResponse {
this.data = data; 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 @Override
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> errors = new HashMap<>(); Map<Errors, Integer> errors = new HashMap<>();

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition; 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.EndQuorumEpochRequestData;
import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.EndQuorumEpochResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
@ -26,6 +27,7 @@ import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.stream.Collectors;
public class EndQuorumEpochRequest extends AbstractRequest { public class EndQuorumEpochRequest extends AbstractRequest {
public static class Builder extends AbstractRequest.Builder<EndQuorumEpochRequest> { 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();
}
}
} }

View File

@ -17,14 +17,12 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.EndQuorumEpochResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
@ -78,27 +76,6 @@ public class EndQuorumEpochResponse extends AbstractResponse {
// Not supported by the response schema // 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) { public static EndQuorumEpochResponse parse(ByteBuffer buffer, short version) {
return new EndQuorumEpochResponse(new EndQuorumEpochResponseData(new ByteBufferAccessor(buffer), version)); return new EndQuorumEpochResponse(new EndQuorumEpochResponseData(new ByteBufferAccessor(buffer), version));
} }

View File

@ -23,11 +23,9 @@ import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.function.UnaryOperator;
public final class FetchSnapshotResponse extends AbstractResponse { public final class FetchSnapshotResponse extends AbstractResponse {
private final FetchSnapshotResponseData data; private final FetchSnapshotResponseData data;
@ -81,33 +79,6 @@ public final class FetchSnapshotResponse extends AbstractResponse {
return new FetchSnapshotResponseData().setErrorCode(error.code()); 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. * Finds the PartitionSnapshot for a given topic partition.
* *

View File

@ -69,19 +69,6 @@ public class VoteRequest extends AbstractRequest {
return new VoteRequest(new VoteRequestData(new ByteBufferAccessor(buffer), version), version); 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, public static VoteRequestData singletonRequest(TopicPartition topicPartition,
String clusterId, String clusterId,
int candidateEpoch, int candidateEpoch,
@ -102,5 +89,4 @@ public class VoteRequest extends AbstractRequest {
.setLastOffset(lastEpochEndOffset)) .setLastOffset(lastEpochEndOffset))
))); )));
} }
} }

View File

@ -17,14 +17,12 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.message.VoteResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
@ -49,25 +47,6 @@ public class VoteResponse extends AbstractResponse {
this.data = data; 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 @Override
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> errors = new HashMap<>(); Map<Errors, Integer> errors = new HashMap<>();

View File

@ -18,24 +18,38 @@
"type": "request", "type": "request",
"listeners": ["controller"], "listeners": ["controller"],
"name": "BeginQuorumEpochRequest", "name": "BeginQuorumEpochRequest",
"validVersions": "0", // Version 1 adds flexible versions, voter key and leader endpoints (KIP-853)
"flexibleVersions": "none", "validVersions": "0-1",
"flexibleVersions": "1+",
"fields": [ "fields": [
{ "name": "ClusterId", "type": "string", "versions": "0+", { "name": "ClusterId", "type": "string", "versions": "0+",
"nullableVersions": "0+", "default": "null"}, "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", { "name": "Topics", "type": "[]TopicData",
"versions": "0+", "fields": [ "versions": "0+", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
"about": "The topic name." }, "about": "The topic name" },
{ "name": "Partitions", "type": "[]PartitionData", { "name": "Partitions", "type": "[]PartitionData",
"versions": "0+", "fields": [ "versions": "0+", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+", { "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." }, "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", { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
"about": "The ID of the newly elected leader"}, "about": "The ID of the newly elected leader"},
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+", { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
"about": "The epoch of the newly elected leader"} "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" }
]
}
] ]
} }

View File

@ -17,8 +17,9 @@
"apiKey": 53, "apiKey": 53,
"type": "response", "type": "response",
"name": "BeginQuorumEpochResponse", "name": "BeginQuorumEpochResponse",
"validVersions": "0", // Version 1 adds flexible versions and leader endpoint (KIP-853)
"flexibleVersions": "none", "validVersions": "0-1",
"flexibleVersions": "1+",
"fields": [ "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The top level error code."}, "about": "The top level error code."},
@ -35,7 +36,17 @@
"about": "The ID of the current leader or -1 if the leader is unknown."}, "about": "The ID of the current leader or -1 if the leader is unknown."},
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+", { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
"about": "The latest known leader epoch"} "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" }
]
}
] ]
} }

View File

@ -18,8 +18,10 @@
"type": "request", "type": "request",
"listeners": ["controller"], "listeners": ["controller"],
"name": "EndQuorumEpochRequest", "name": "EndQuorumEpochRequest",
"validVersions": "0", // Version 1 adds flexible versions, replaces preferred successors with preferred candidates
"flexibleVersions": "none", // and adds leader endpoints (KIP-853)
"validVersions": "0-1",
"flexibleVersions": "1+",
"fields": [ "fields": [
{ "name": "ClusterId", "type": "string", "versions": "0+", { "name": "ClusterId", "type": "string", "versions": "0+",
"nullableVersions": "0+", "default": "null"}, "nullableVersions": "0+", "default": "null"},
@ -35,9 +37,24 @@
"about": "The current leader ID that is resigning"}, "about": "The current leader ID that is resigning"},
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+", { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
"about": "The current epoch"}, "about": "The current epoch"},
{ "name": "PreferredSuccessors", "type": "[]int32", "versions": "0+", { "name": "PreferredSuccessors", "type": "[]int32", "versions": "0", "ignorable": true,
"about": "A sorted list of preferred successors to start the election"} "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" }
]
}
] ]
} }

View File

@ -17,8 +17,9 @@
"apiKey": 54, "apiKey": 54,
"type": "response", "type": "response",
"name": "EndQuorumEpochResponse", "name": "EndQuorumEpochResponse",
"validVersions": "0", // Version 1 adds flexible versions and leader endpoint (KIP-853)
"flexibleVersions": "none", "validVersions": "0-1",
"flexibleVersions": "1+",
"fields": [ "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The top level error code."}, "about": "The top level error code."},
@ -35,7 +36,17 @@
"about": "The ID of the current leader or -1 if the leader is unknown."}, "about": "The ID of the current leader or -1 if the leader is unknown."},
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+", { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
"about": "The latest known leader epoch"} "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" }
]
}
] ]
} }

View File

@ -55,7 +55,9 @@
// deprecate the old ReplicaId field and set its default value to -1. (KIP-903) // deprecate the old ReplicaId field and set its default value to -1. (KIP-903)
// //
// Version 16 is the same as version 15 (KIP-951). // 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", "deprecatedVersions": "0-3",
"flexibleVersions": "12+", "flexibleVersions": "12+",
"fields": [ "fields": [
@ -100,7 +102,9 @@
{ "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true, { "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."}, "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+", { "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, { "name": "ForgottenTopicsData", "type": "[]ForgottenTopic", "versions": "7+", "ignorable": false,

View File

@ -47,7 +47,9 @@
// Version 15 is the same as version 14 (KIP-903). // Version 15 is the same as version 14 (KIP-903).
// //
// Version 16 adds the 'NodeEndpoints' field (KIP-951). // 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+", "flexibleVersions": "12+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,

View File

@ -18,7 +18,8 @@
"type": "request", "type": "request",
"listeners": ["controller"], "listeners": ["controller"],
"name": "FetchSnapshotRequest", "name": "FetchSnapshotRequest",
"validVersions": "0", // Version 1 adds replica directory id (KIP-853)
"validVersions": "0-1",
"flexibleVersions": "0+", "flexibleVersions": "0+",
"fields": [ "fields": [
{ "name": "ClusterId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "taggedVersions": "0+", "tag": 0, { "name": "ClusterId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "taggedVersions": "0+", "tag": 0,
@ -38,14 +39,18 @@
{ "name": "CurrentLeaderEpoch", "type": "int32", "versions": "0+", { "name": "CurrentLeaderEpoch", "type": "int32", "versions": "0+",
"about": "The current leader epoch of the partition, -1 for unknown leader epoch" }, "about": "The current leader epoch of the partition, -1 for unknown leader epoch" },
{ "name": "SnapshotId", "type": "SnapshotId", "versions": "0+", { "name": "SnapshotId", "type": "SnapshotId", "versions": "0+",
"about": "The snapshot endOffset and epoch to fetch", "about": "The snapshot endOffset and epoch to fetch", "fields": [
"fields": [
{ "name": "EndOffset", "type": "int64", "versions": "0+" }, { "name": "EndOffset", "type": "int64", "versions": "0+" },
{ "name": "Epoch", "type": "int32", "versions": "0+" } { "name": "Epoch", "type": "int32", "versions": "0+" }
]}, ]
},
{ "name": "Position", "type": "int64", "versions": "0+", { "name": "Position", "type": "int64", "versions": "0+",
"about": "The byte position within the snapshot to start fetching from" } "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" }
]
}
]
}
] ]
} }

View File

@ -17,7 +17,8 @@
"apiKey": 59, "apiKey": 59,
"type": "response", "type": "response",
"name": "FetchSnapshotResponse", "name": "FetchSnapshotResponse",
"validVersions": "0", "validVersions": "0-1",
// Version 1 adds leader endpoint (KIP-853)
"flexibleVersions": "0+", "flexibleVersions": "0+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true,
@ -35,25 +36,36 @@
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no fetch error." }, "about": "The error code, or 0 if there was no fetch error." },
{ "name": "SnapshotId", "type": "SnapshotId", "versions": "0+", { "name": "SnapshotId", "type": "SnapshotId", "versions": "0+",
"about": "The snapshot endOffset and epoch fetched", "about": "The snapshot endOffset and epoch fetched", "fields": [
"fields": [
{ "name": "EndOffset", "type": "int64", "versions": "0+" }, { "name": "EndOffset", "type": "int64", "versions": "0+" },
{ "name": "Epoch", "type": "int32", "versions": "0+" } { "name": "Epoch", "type": "int32", "versions": "0+" }
]}, ]
},
{ "name": "CurrentLeader", "type": "LeaderIdAndEpoch", { "name": "CurrentLeader", "type": "LeaderIdAndEpoch",
"versions": "0+", "taggedVersions": "0+", "tag": 0, "fields": [ "versions": "0+", "taggedVersions": "0+", "tag": 0, "fields": [
{ "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId",
"about": "The ID of the current leader or -1 if the leader is unknown."}, "about": "The ID of the current leader or -1 if the leader is unknown."},
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+", { "name": "LeaderEpoch", "type": "int32", "versions": "0+",
"about": "The latest known leader epoch"} "about": "The latest known leader epoch"}
]}, ]
},
{ "name": "Size", "type": "int64", "versions": "0+", { "name": "Size", "type": "int64", "versions": "0+",
"about": "The total size of the snapshot." }, "about": "The total size of the snapshot." },
{ "name": "Position", "type": "int64", "versions": "0+", { "name": "Position", "type": "int64", "versions": "0+",
"about": "The starting byte position within the snapshot included in the Bytes field." }, "about": "The starting byte position within the snapshot included in the Bytes field." },
{ "name": "UnalignedRecords", "type": "records", "versions": "0+", { "name": "UnalignedRecords", "type": "records", "versions": "0+",
"about": "Snapshot data in records format which may not be aligned on an offset boundary" } "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" }
]
}
] ]
} }

View File

@ -18,11 +18,14 @@
"type": "request", "type": "request",
"listeners": ["controller"], "listeners": ["controller"],
"name": "VoteRequest", "name": "VoteRequest",
"validVersions": "0", // Version 1 adds voter key and candidate directory id (KIP-853)
"validVersions": "0-1",
"flexibleVersions": "0+", "flexibleVersions": "0+",
"fields": [ "fields": [
{ "name": "ClusterId", "type": "string", "versions": "0+", { "name": "ClusterId", "type": "string", "versions": "0+",
"nullableVersions": "0+", "default": "null"}, "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", { "name": "Topics", "type": "[]TopicData",
"versions": "0+", "fields": [ "versions": "0+", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName", { "name": "TopicName", "type": "string", "versions": "0+", "entityType": "topicName",
@ -34,6 +37,10 @@
{ "name": "CandidateEpoch", "type": "int32", "versions": "0+", { "name": "CandidateEpoch", "type": "int32", "versions": "0+",
"about": "The bumped epoch of the candidate sending the request"}, "about": "The bumped epoch of the candidate sending the request"},
{ "name": "CandidateId", "type": "int32", "versions": "0+", "entityType": "brokerId", { "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"}, "about": "The ID of the voter sending the request"},
{ "name": "LastOffsetEpoch", "type": "int32", "versions": "0+", { "name": "LastOffsetEpoch", "type": "int32", "versions": "0+",
"about": "The epoch of the last record written to the metadata log"}, "about": "The epoch of the last record written to the metadata log"},

View File

@ -17,7 +17,8 @@
"apiKey": 52, "apiKey": 52,
"type": "response", "type": "response",
"name": "VoteResponse", "name": "VoteResponse",
"validVersions": "0", // Version 1 adds leader endpoint (KIP-853)
"validVersions": "0-1",
"flexibleVersions": "0+", "flexibleVersions": "0+",
"fields": [ "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",
@ -40,6 +41,14 @@
] ]
} }
] ]
},
{ "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" }
]
} }
] ]
} }

View File

@ -39,11 +39,12 @@ import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector} import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector}
import org.apache.kafka.common.protocol.ApiMessage 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.requests.RequestHeader
import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.JaasContext
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.{LogContext, Time, Utils} 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.ProcessRole
import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.common.serialization.RecordSerde
import org.apache.kafka.server.util.KafkaScheduler import org.apache.kafka.server.util.KafkaScheduler
@ -122,6 +123,7 @@ object KafkaRaftManager {
trait RaftManager[T] { trait RaftManager[T] {
def handleRequest( def handleRequest(
context: RequestContext,
header: RequestHeader, header: RequestHeader,
request: ApiMessage, request: ApiMessage,
createdTimeMs: Long createdTimeMs: Long
@ -152,6 +154,7 @@ class KafkaRaftManager[T](
threadNamePrefixOpt: Option[String], threadNamePrefixOpt: Option[String],
val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]], val controllerQuorumVotersFuture: CompletableFuture[JMap[Integer, InetSocketAddress]],
bootstrapServers: JCollection[InetSocketAddress], bootstrapServers: JCollection[InetSocketAddress],
controllerListeners: Endpoints,
fatalFaultHandler: FaultHandler fatalFaultHandler: FaultHandler
) extends RaftManager[T] with Logging { ) extends RaftManager[T] with Logging {
@ -214,15 +217,16 @@ class KafkaRaftManager[T](
} }
override def handleRequest( override def handleRequest(
context: RequestContext,
header: RequestHeader, header: RequestHeader,
request: ApiMessage, request: ApiMessage,
createdTimeMs: Long createdTimeMs: Long
): CompletableFuture[ApiMessage] = { ): CompletableFuture[ApiMessage] = {
clientDriver.handleRequest(header, request, createdTimeMs) clientDriver.handleRequest(context, header, request, createdTimeMs)
} }
private def buildRaftClient(): KafkaRaftClient[T] = { private def buildRaftClient(): KafkaRaftClient[T] = {
val client = new KafkaRaftClient( new KafkaRaftClient(
OptionalInt.of(config.nodeId), OptionalInt.of(config.nodeId),
metadataLogDirUuid, metadataLogDirUuid,
recordSerde, recordSerde,
@ -233,9 +237,9 @@ class KafkaRaftManager[T](
logContext, logContext,
clusterId, clusterId,
bootstrapServers, bootstrapServers,
controllerListeners,
raftConfig raftConfig
) )
client
} }
private def buildNetworkChannel(): KafkaNetworkChannel = { private def buildNetworkChannel(): KafkaNetworkChannel = {

View File

@ -679,7 +679,7 @@ class ControllerApis(
private def handleRaftRequest(request: RequestChannel.Request, private def handleRaftRequest(request: RequestChannel.Request,
buildResponse: ApiMessage => AbstractResponse): CompletableFuture[Unit] = { buildResponse: ApiMessage => AbstractResponse): CompletableFuture[Unit] = {
val requestBody = request.body[AbstractRequest] 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) => future.handle[Unit] { (responseData, exception) =>
val response = if (exception != null) { val response = if (exception != null) {
requestBody.getErrorResponse(exception) requestBody.getErrorResponse(exception)

View File

@ -187,9 +187,10 @@ class ControllerServer(
credentialProvider, credentialProvider,
apiVersionManager) apiVersionManager)
val listenerInfo = ListenerInfo.create(config.controllerListeners.map(_.toJava).asJava). val listenerInfo = ListenerInfo
withWildcardHostnamesResolved(). .create(config.effectiveAdvertisedControllerListeners.map(_.toJava).asJava)
withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name))) .withWildcardHostnamesResolved()
.withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name)))
socketServerFirstBoundPortFuture.complete(listenerInfo.firstListener().port()) socketServerFirstBoundPortFuture.complete(listenerInfo.firstListener().port())
val endpointReadyFutures = { val endpointReadyFutures = {
@ -203,7 +204,7 @@ class ControllerServer(
config.earlyStartListeners.map(_.value()).asJava)) config.earlyStartListeners.map(_.value()).asJava))
} }
sharedServer.startForController() sharedServer.startForController(listenerInfo)
createTopicPolicy = Option(config. createTopicPolicy = Option(config.
getConfiguredInstance(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[CreateTopicPolicy])) getConfiguredInstance(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[CreateTopicPolicy]))

View File

@ -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] = { 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) val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
if (advertisedListenersProp != null) if (advertisedListenersProp != null) {
CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false)
else } else {
listeners.filterNot(l => controllerListenerNames.contains(l.listenerName.value())) listeners
}
} }
private def getInterBrokerListenerNameAndSecurityProtocol: (ListenerName, SecurityProtocol) = { 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.") s"${SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG} is not supported in KRaft mode.")
} }
def validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker(): Unit = { 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.") 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 = { def validateControllerQuorumVotersMustContainNodeIdForKRaftController(): Unit = {
@ -979,11 +990,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
// KRaft controller-only // KRaft controller-only
validateNonEmptyQuorumVotersForKRaft() validateNonEmptyQuorumVotersForKRaft()
validateControlPlaneListenerEmptyForKRaft() 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 // listeners should only contain listeners also enumerated in the controller listener
require( require(
effectiveAdvertisedListeners.isEmpty, effectiveAdvertisedListeners.isEmpty,

View File

@ -93,6 +93,8 @@ class KafkaRaftServer(
override def startup(): Unit = { override def startup(): Unit = {
Mx4jLoader.maybeLoad() 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()) controller.foreach(_.startup())
broker.foreach(_.startup()) broker.foreach(_.startup())
AppInfoParser.registerAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics, time.milliseconds()) AppInfoParser.registerAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics, time.milliseconds())

View File

@ -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.properties.{MetaProperties, MetaPropertiesEnsemble}
import org.apache.kafka.metadata.{BrokerState, MetadataRecordSerde, VersionRange} import org.apache.kafka.metadata.{BrokerState, MetadataRecordSerde, VersionRange}
import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.raft.Endpoints
import org.apache.kafka.security.CredentialProvider import org.apache.kafka.security.CredentialProvider
import org.apache.kafka.server.NodeToControllerChannelManager import org.apache.kafka.server.NodeToControllerChannelManager
import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.authorizer.Authorizer
@ -440,6 +441,9 @@ class KafkaServer(
threadNamePrefix, threadNamePrefix,
CompletableFuture.completedFuture(quorumVoters), CompletableFuture.completedFuture(quorumVoters),
QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), 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()) fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown())
) )
quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config) quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config)

View File

@ -22,15 +22,18 @@ import kafka.server.Server.MetricsPrefix
import kafka.server.metadata.BrokerServerMetrics import kafka.server.metadata.BrokerServerMetrics
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import org.apache.kafka.common.metrics.Metrics 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.common.utils.{AppInfoParser, LogContext, Time}
import org.apache.kafka.controller.metrics.ControllerMetadataMetrics import org.apache.kafka.controller.metrics.ControllerMetadataMetrics
import org.apache.kafka.image.MetadataProvenance import org.apache.kafka.image.MetadataProvenance
import org.apache.kafka.image.loader.MetadataLoader import org.apache.kafka.image.loader.MetadataLoader
import org.apache.kafka.image.loader.metrics.MetadataLoaderMetrics 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.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.MetadataRecordSerde
import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble
import org.apache.kafka.raft.Endpoints
import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.server.fault.{FaultHandler, LoggingFaultHandler, ProcessTerminatingFaultHandler} 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.concurrent.{CompletableFuture, TimeUnit}
import java.util.{Collection => JCollection} import java.util.{Collection => JCollection}
import java.util.{Map => JMap} import java.util.{Map => JMap}
import scala.jdk.CollectionConverters._
/** /**
* Creates a fault handler. * Creates a fault handler.
@ -128,7 +131,7 @@ class SharedServer(
*/ */
def startForBroker(): Unit = synchronized { def startForBroker(): Unit = synchronized {
if (!isUsed()) { if (!isUsed()) {
start() start(Endpoints.empty())
} }
usedByBroker = true usedByBroker = true
} }
@ -136,9 +139,22 @@ class SharedServer(
/** /**
* The start function called by the controller. * The start function called by the controller.
*/ */
def startForController(): Unit = synchronized { def startForController(listenerInfo: ListenerInfo): Unit = synchronized {
if (!isUsed()) { 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 usedByController = true
} }
@ -237,7 +253,7 @@ class SharedServer(
// Note: snapshot generation does not need to be disabled for a publishing fault. // 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) { if (started) {
debug("SharedServer has already been started.") debug("SharedServer has already been started.")
} else { } else {
@ -256,6 +272,7 @@ class SharedServer(
if (sharedServerConfig.processRoles.contains(ProcessRole.ControllerRole)) { if (sharedServerConfig.processRoles.contains(ProcessRole.ControllerRole)) {
controllerServerMetrics = new ControllerMetadataMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry())) controllerServerMetrics = new ControllerMetadataMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry()))
} }
val _raftManager = new KafkaRaftManager[ApiMessageAndVersion]( val _raftManager = new KafkaRaftManager[ApiMessageAndVersion](
clusterId, clusterId,
sharedServerConfig, sharedServerConfig,
@ -268,6 +285,7 @@ class SharedServer(
Some(s"kafka-${sharedServerConfig.nodeId}-raft"), // No dash expected at the end Some(s"kafka-${sharedServerConfig.nodeId}-raft"), // No dash expected at the end
controllerQuorumVotersFuture, controllerQuorumVotersFuture,
bootstrapServers, bootstrapServers,
listenerEndpoints,
raftManagerFaultHandler raftManagerFaultHandler
) )
raftManager = _raftManager raftManager = _raftManager

View File

@ -94,6 +94,7 @@ class TestRaftRequestHandler(
val requestBody = request.body[AbstractRequest] val requestBody = request.body[AbstractRequest]
val future = raftManager.handleRequest( val future = raftManager.handleRequest(
request.context,
request.header, request.header,
requestBody.data, requestBody.data,
time.milliseconds() time.milliseconds()

View File

@ -17,6 +17,7 @@
package kafka.tools package kafka.tools
import java.net.InetSocketAddress
import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import java.util.concurrent.atomic.{AtomicInteger, AtomicLong}
import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit} import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit}
import joptsimple.{OptionException, OptionSpec} 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.utils.{Time, Utils}
import org.apache.kafka.common.{TopicPartition, Uuid, protocol} import org.apache.kafka.common.{TopicPartition, Uuid, protocol}
import org.apache.kafka.raft.errors.NotLeaderException 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.security.CredentialProvider
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.common.serialization.RecordSerde
@ -84,6 +85,15 @@ class TestRaftServer(
() => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION)) () => FinalizedFeatures.fromKRaftVersion(MetadataVersion.MINIMUM_KRAFT_VERSION))
socketServer = new SocketServer(config, metrics, time, credentialProvider, apiVersionManager) 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]]( raftManager = new KafkaRaftManager[Array[Byte]](
Uuid.ZERO_UUID.toString, Uuid.ZERO_UUID.toString,
config, config,
@ -96,6 +106,7 @@ class TestRaftServer(
Some(threadNamePrefix), Some(threadNamePrefix),
CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)),
QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers),
endpoints,
new ProcessTerminatingFaultHandler.Builder().build() new ProcessTerminatingFaultHandler.Builder().build()
) )

View File

@ -439,8 +439,9 @@ class SocketServerTest {
@Test @Test
def testDisabledRequestIsRejected(): Unit = { def testDisabledRequestIsRejected(): Unit = {
val correlationId = 57 val correlationId = 57
val header = new RequestHeader(ApiKeys.VOTE, 0, "", correlationId) val version: Short = 0
val request = new VoteRequest.Builder(new VoteRequestData()).build() 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 serializedBytes = Utils.toArray(request.serializeWithHeader(header))
val socket = connect() val socket = connect()

View File

@ -16,6 +16,7 @@
*/ */
package kafka.raft package kafka.raft
import java.net.InetSocketAddress
import java.nio.channels.FileChannel import java.nio.channels.FileChannel
import java.nio.channels.OverlappingFileLockException import java.nio.channels.OverlappingFileLockException
import java.nio.file.{Files, Path, StandardOpenOption} import java.nio.file.{Files, Path, StandardOpenOption}
@ -23,25 +24,26 @@ import java.util.Properties
import java.util.concurrent.CompletableFuture import java.util.concurrent.CompletableFuture
import kafka.log.LogManager import kafka.log.LogManager
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import kafka.tools.TestRaftServer.ByteArraySerde import kafka.tools.TestRaftServer.ByteArraySerde
import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.Uuid import org.apache.kafka.common.Uuid
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.Endpoints
import org.apache.kafka.raft.QuorumConfig 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.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.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource import org.junit.jupiter.params.provider.ValueSource
import org.apache.kafka.server.fault.FaultHandler
import org.mockito.Mockito._ import org.mockito.Mockito._
import scala.util.Using import scala.util.Using
import scala.jdk.CollectionConverters._
class RaftManagerTest { class RaftManagerTest {
private def createZkBrokerConfig( private def createZkBrokerConfig(
@ -107,6 +109,15 @@ class RaftManagerTest {
): KafkaRaftManager[Array[Byte]] = { ): KafkaRaftManager[Array[Byte]] = {
val topicId = new Uuid(0L, 2L) 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]]( new KafkaRaftManager[Array[Byte]](
Uuid.randomUuid.toString, Uuid.randomUuid.toString,
config, config,
@ -119,6 +130,7 @@ class RaftManagerTest {
Option.empty, Option.empty,
CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)), CompletableFuture.completedFuture(QuorumConfig.parseVoterConnections(config.quorumVoters)),
QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers), QuorumConfig.parseBootstrapServers(config.quorumBootstrapServers),
endpoints,
mock(classOf[FaultHandler]) mock(classOf[FaultHandler])
) )
} }

View File

@ -219,6 +219,7 @@ class ControllerApisTest {
def testFetchSentToKRaft(): Unit = { def testFetchSentToKRaft(): Unit = {
when( when(
raftManager.handleRequest( raftManager.handleRequest(
any(classOf[RequestContext]),
any(classOf[RequestHeader]), any(classOf[RequestHeader]),
any(classOf[ApiMessage]), any(classOf[ApiMessage]),
any(classOf[Long]) any(classOf[Long])
@ -231,6 +232,7 @@ class ControllerApisTest {
controllerApis.handleFetch(buildRequest(new FetchRequest(new FetchRequestData(), 12))) controllerApis.handleFetch(buildRequest(new FetchRequest(new FetchRequestData(), 12)))
verify(raftManager).handleRequest( verify(raftManager).handleRequest(
ArgumentMatchers.any(),
ArgumentMatchers.any(), ArgumentMatchers.any(),
ArgumentMatchers.any(), ArgumentMatchers.any(),
ArgumentMatchers.any() ArgumentMatchers.any()
@ -245,6 +247,7 @@ class ControllerApisTest {
when( when(
raftManager.handleRequest( raftManager.handleRequest(
any(classOf[RequestContext]),
any(classOf[RequestHeader]), any(classOf[RequestHeader]),
any(classOf[ApiMessage]), any(classOf[ApiMessage]),
any(classOf[Long]) any(classOf[Long])
@ -262,6 +265,7 @@ class ControllerApisTest {
verify(raftManager).handleRequest( verify(raftManager).handleRequest(
ArgumentMatchers.eq(request.context),
ArgumentMatchers.eq(request.header), ArgumentMatchers.eq(request.header),
ArgumentMatchers.eq(fetchRequestData), ArgumentMatchers.eq(fetchRequestData),
ArgumentMatchers.eq(initialTimeMs) ArgumentMatchers.eq(initialTimeMs)
@ -285,6 +289,7 @@ class ControllerApisTest {
def testFetchSnapshotSentToKRaft(): Unit = { def testFetchSnapshotSentToKRaft(): Unit = {
when( when(
raftManager.handleRequest( raftManager.handleRequest(
any(classOf[RequestContext]),
any(classOf[RequestHeader]), any(classOf[RequestHeader]),
any(classOf[ApiMessage]), any(classOf[ApiMessage]),
any(classOf[Long]) any(classOf[Long])
@ -297,6 +302,7 @@ class ControllerApisTest {
controllerApis.handleFetchSnapshot(buildRequest(new FetchSnapshotRequest(new FetchSnapshotRequestData(), 0))) controllerApis.handleFetchSnapshot(buildRequest(new FetchSnapshotRequest(new FetchSnapshotRequestData(), 0)))
verify(raftManager).handleRequest( verify(raftManager).handleRequest(
ArgumentMatchers.any(),
ArgumentMatchers.any(), ArgumentMatchers.any(),
ArgumentMatchers.any(), ArgumentMatchers.any(),
ArgumentMatchers.any() ArgumentMatchers.any()
@ -1207,7 +1213,7 @@ class ControllerApisTest {
val response = new FetchResponseData() val response = new FetchResponseData()
val responseFuture = new CompletableFuture[ApiMessage]() val responseFuture = new CompletableFuture[ApiMessage]()
val errorResponseFuture = new AtomicReference[AbstractResponse]() 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 { _ => when(requestChannel.sendResponse(any(), any(), any())).thenAnswer { _ =>
// Simulate an encoding failure in the initial fetch response // Simulate an encoding failure in the initial fetch response
throw new UnsupportedVersionException("Something went wrong") throw new UnsupportedVersionException("Something went wrong")

View File

@ -1294,7 +1294,7 @@ class KafkaConfigTest {
} }
@Test @Test
def testControllerListenersCannotBeAdvertisedForKRaftBroker(): Unit = { def testControllerListenersCanBeAdvertisedForKRaftCombined(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" 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.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT,SSL")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092") 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 // Valid
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "SASL_SSL://C:9094")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
// Also valid if we allow advertised listeners to derive from listeners/controller.listener.names // Also valid if we allow advertised listeners to derive from listeners/controller.listener.names
@ -1317,8 +1314,7 @@ class KafkaConfigTest {
} }
@Test @Test
def testAdvertisedListenersDisallowedForKRaftControllerOnlyRole(): Unit = { def testAdvertisedListenersAllowedForKRaftControllerOnlyRole(): Unit = {
// Test that advertised listeners cannot be set when KRaft and server is controller only.
// Test that listeners must enumerate every controller listener // Test that listeners must enumerate every controller listener
// Test that controller listener must enumerate every listener // Test that controller listener must enumerate every listener
val correctListeners = "PLAINTEXT://A:9092,SSL://B:9093" val correctListeners = "PLAINTEXT://A:9092,SSL://B:9093"
@ -1329,21 +1325,18 @@ class KafkaConfigTest {
val props = new Properties() val props = new Properties()
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, correctListeners) 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.CONTROLLER_LISTENER_NAMES_CONFIG, correctControllerListenerNames)
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092") 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 // Valid
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, correctListeners) KafkaConfig.fromProps(props)
assertBadConfigContainingMessage(props, expectedExceptionContainsText)
// Invalid if listeners contains names not in controller.listener.names // Invalid if listeners contains names not in controller.listener.names
props.remove(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) props.remove(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, incorrectListeners) 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", " ") |controller.listener.names when process.roles=controller""".stripMargin.replaceAll("\n", " ")
assertBadConfigContainingMessage(props, expectedExceptionContainsText) assertBadConfigContainingMessage(props, expectedExceptionContainsText)

View File

@ -121,15 +121,20 @@ class ReplicaFetcherThreadTest {
@Test @Test
def shouldSendLatestRequestVersionsByDefault(): Unit = { def shouldSendLatestRequestVersionsByDefault(): Unit = {
val props = TestUtils.createBrokerConfig(1, "localhost:1234") // Check unstable versions
val config = KafkaConfig.fromProps(props) val testingVersion = MetadataVersion.latestTesting
assertEquals(
val replicaManager: ReplicaManager = mock(classOf[ReplicaManager]) ApiKeys.FETCH.latestVersion(true),
when(replicaManager.brokerTopicStats).thenReturn(mock(classOf[BrokerTopicStats])) testingVersion.fetchRequestVersion
)
assertEquals(ApiKeys.FETCH.latestVersion, config.interBrokerProtocolVersion.fetchRequestVersion()) assertEquals(
assertEquals(ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion, config.interBrokerProtocolVersion.offsetForLeaderEpochRequestVersion) ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion(true),
assertEquals(ApiKeys.LIST_OFFSETS.latestVersion, config.interBrokerProtocolVersion.listOffsetRequestVersion) testingVersion.offsetForLeaderEpochRequestVersion
)
assertEquals(
ApiKeys.LIST_OFFSETS.latestVersion(true),
testingVersion.listOffsetRequestVersion
)
} }
@Test @Test

View File

@ -638,10 +638,10 @@ class RequestQuotaTest extends BaseRequestTest {
new AlterUserScramCredentialsRequest.Builder(new AlterUserScramCredentialsRequestData()) new AlterUserScramCredentialsRequest.Builder(new AlterUserScramCredentialsRequestData())
case ApiKeys.VOTE => 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 => 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 => case ApiKeys.END_QUORUM_EPOCH =>
new EndQuorumEpochRequest.Builder(EndQuorumEpochRequest.singletonRequest( new EndQuorumEpochRequest.Builder(EndQuorumEpochRequest.singletonRequest(

View File

@ -64,7 +64,7 @@ public class CandidateState implements EpochState {
int electionTimeoutMs, int electionTimeoutMs,
LogContext logContext LogContext logContext
) { ) {
if (!voters.isVoter(ReplicaKey.of(localId, Optional.of(localDirectoryId)))) { if (!voters.isVoter(ReplicaKey.of(localId, localDirectoryId))) {
throw new IllegalArgumentException( throw new IllegalArgumentException(
String.format( String.format(
"Local replica (%d, %s) must be in the set of voters %s", "Local replica (%d, %s) must be in the set of voters %s",
@ -247,7 +247,7 @@ public class CandidateState implements EpochState {
public ElectionState election() { public ElectionState election() {
return ElectionState.withVotedCandidate( return ElectionState.withVotedCandidate(
epoch, epoch,
ReplicaKey.of(localId, Optional.of(localDirectoryId)), ReplicaKey.of(localId, localDirectoryId),
voteStates.keySet() voteStates.keySet()
); );
} }
@ -257,6 +257,11 @@ public class CandidateState implements EpochState {
return epoch; return epoch;
} }
@Override
public Endpoints leaderEndpoints() {
return Endpoints.empty();
}
@Override @Override
public Optional<LogOffsetMetadata> highWatermark() { public Optional<LogOffsetMetadata> highWatermark() {
return highWatermark; return highWatermark;

View File

@ -16,7 +16,6 @@
*/ */
package org.apache.kafka.raft; package org.apache.kafka.raft;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.raft.generated.QuorumStateData; import org.apache.kafka.raft.generated.QuorumStateData;
import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.ReplicaKey;
@ -33,7 +32,6 @@ import java.util.stream.Collectors;
public final class ElectionState { public final class ElectionState {
private static final int UNKNOWN_LEADER_ID = -1; private static final int UNKNOWN_LEADER_ID = -1;
private static final int NOT_VOTED = -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 int epoch;
private final OptionalInt leaderId; private final OptionalInt leaderId;
@ -136,7 +134,9 @@ public final class ElectionState {
.collect(Collectors.toList()); .collect(Collectors.toList());
data.setCurrentVoters(dataVoters); data.setCurrentVoters(dataVoters);
} else if (version == 1) { } 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 { } else {
throw new IllegalStateException( throw new IllegalStateException(
String.format( String.format(
@ -199,13 +199,9 @@ public final class ElectionState {
} }
public static ElectionState fromQuorumStateData(QuorumStateData data) { 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<ReplicaKey> votedKey = data.votedId() == NOT_VOTED ?
Optional.empty() : Optional.empty() :
Optional.of(ReplicaKey.of(data.votedId(), votedDirectoryId)); Optional.of(ReplicaKey.of(data.votedId(), data.votedDirectoryId()));
return new ElectionState( return new ElectionState(
data.leaderEpoch(), data.leaderEpoch(),

View File

@ -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);
}
}

View File

@ -49,6 +49,13 @@ public interface EpochState extends Closeable {
*/ */
int epoch(); 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 * User-friendly description of the state
*/ */

View File

@ -17,6 +17,7 @@
package org.apache.kafka.raft; package org.apache.kafka.raft;
import org.apache.kafka.common.Node; 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.LogContext;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Timer;
@ -32,7 +33,8 @@ import java.util.Set;
public class FollowerState implements EpochState { public class FollowerState implements EpochState {
private final int fetchTimeoutMs; private final int fetchTimeoutMs;
private final int epoch; private final int epoch;
private final Node leader; private final int leaderId;
private final Endpoints endpoints;
private final Set<Integer> voters; private final Set<Integer> voters;
// Used for tracking the expiration of both the Fetch and FetchSnapshot requests // Used for tracking the expiration of both the Fetch and FetchSnapshot requests
private final Timer fetchTimer; private final Timer fetchTimer;
@ -47,7 +49,8 @@ public class FollowerState implements EpochState {
public FollowerState( public FollowerState(
Time time, Time time,
int epoch, int epoch,
Node leader, int leaderId,
Endpoints endpoints,
Set<Integer> voters, Set<Integer> voters,
Optional<LogOffsetMetadata> highWatermark, Optional<LogOffsetMetadata> highWatermark,
int fetchTimeoutMs, int fetchTimeoutMs,
@ -55,7 +58,8 @@ public class FollowerState implements EpochState {
) { ) {
this.fetchTimeoutMs = fetchTimeoutMs; this.fetchTimeoutMs = fetchTimeoutMs;
this.epoch = epoch; this.epoch = epoch;
this.leader = leader; this.leaderId = leaderId;
this.endpoints = endpoints;
this.voters = voters; this.voters = voters;
this.fetchTimer = time.timer(fetchTimeoutMs); this.fetchTimer = time.timer(fetchTimeoutMs);
this.highWatermark = highWatermark; this.highWatermark = highWatermark;
@ -64,7 +68,7 @@ public class FollowerState implements EpochState {
@Override @Override
public ElectionState election() { public ElectionState election() {
return ElectionState.withElectedLeader(epoch, leader.id(), voters); return ElectionState.withElectedLeader(epoch, leaderId, voters);
} }
@Override @Override
@ -72,6 +76,11 @@ public class FollowerState implements EpochState {
return epoch; return epoch;
} }
@Override
public Endpoints leaderEndpoints() {
return endpoints;
}
@Override @Override
public String name() { public String name() {
return "Follower"; return "Follower";
@ -82,8 +91,28 @@ public class FollowerState implements EpochState {
return fetchTimer.remainingMs(); return fetchTimer.remainingMs();
} }
public Node leader() { public int leaderId() {
return leader; 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) { public boolean hasFetchTimeoutExpired(long currentTimeMs) {
@ -158,7 +187,7 @@ public class FollowerState implements EpochState {
log.debug( log.debug(
"Rejecting vote request from candidate ({}) since we already have a leader {} in epoch {}", "Rejecting vote request from candidate ({}) since we already have a leader {} in epoch {}",
candidateKey, candidateKey,
leader, leaderId,
epoch epoch
); );
return false; return false;
@ -167,11 +196,12 @@ public class FollowerState implements EpochState {
@Override @Override
public String toString() { public String toString() {
return String.format( 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)", "fetchingSnapshot=%s)",
fetchTimeoutMs, fetchTimeoutMs,
epoch, epoch,
leader, leaderId,
endpoints,
voters, voters,
highWatermark, highWatermark,
fetchingSnapshot fetchingSnapshot

View File

@ -46,17 +46,14 @@ import org.apache.kafka.common.record.Records;
import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.record.UnalignedMemoryRecords;
import org.apache.kafka.common.record.UnalignedRecords; import org.apache.kafka.common.record.UnalignedRecords;
import org.apache.kafka.common.requests.BeginQuorumEpochRequest; 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.DescribeQuorumRequest;
import org.apache.kafka.common.requests.DescribeQuorumResponse; import org.apache.kafka.common.requests.DescribeQuorumResponse;
import org.apache.kafka.common.requests.EndQuorumEpochRequest; 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.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.requests.FetchSnapshotRequest; import org.apache.kafka.common.requests.FetchSnapshotRequest;
import org.apache.kafka.common.requests.FetchSnapshotResponse; import org.apache.kafka.common.requests.FetchSnapshotResponse;
import org.apache.kafka.common.requests.VoteRequest; 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.BufferSupplier;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time; 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 Time time;
private final int fetchMaxWaitMs; private final int fetchMaxWaitMs;
private final String clusterId; private final String clusterId;
private final Endpoints localListeners;
private final NetworkChannel channel; private final NetworkChannel channel;
private final ReplicatedLog log; private final ReplicatedLog log;
private final Random random; private final Random random;
@ -214,6 +212,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
LogContext logContext, LogContext logContext,
String clusterId, String clusterId,
Collection<InetSocketAddress> bootstrapServers, Collection<InetSocketAddress> bootstrapServers,
Endpoints localListeners,
QuorumConfig quorumConfig QuorumConfig quorumConfig
) { ) {
this( this(
@ -229,6 +228,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
MAX_FETCH_WAIT_MS, MAX_FETCH_WAIT_MS,
clusterId, clusterId,
bootstrapServers, bootstrapServers,
localListeners,
logContext, logContext,
new Random(), new Random(),
quorumConfig quorumConfig
@ -248,6 +248,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
int fetchMaxWaitMs, int fetchMaxWaitMs,
String clusterId, String clusterId,
Collection<InetSocketAddress> bootstrapServers, Collection<InetSocketAddress> bootstrapServers,
Endpoints localListeners,
LogContext logContext, LogContext logContext,
Random random, Random random,
QuorumConfig quorumConfig QuorumConfig quorumConfig
@ -264,6 +265,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
this.appendPurgatory = new ThresholdPurgatory<>(expirationService); this.appendPurgatory = new ThresholdPurgatory<>(expirationService);
this.time = time; this.time = time;
this.clusterId = clusterId; this.clusterId = clusterId;
this.localListeners = localListeners;
this.fetchMaxWaitMs = fetchMaxWaitMs; this.fetchMaxWaitMs = fetchMaxWaitMs;
this.logger = logContext.logger(KafkaRaftClient.class); this.logger = logContext.logger(KafkaRaftClient.class);
this.random = random; this.random = random;
@ -315,7 +317,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
) { ) {
final LogOffsetMetadata endOffsetMetadata = log.endOffset(); final LogOffsetMetadata endOffsetMetadata = log.endOffset();
if (state.updateLocalState(endOffsetMetadata, partitionState.lastVoterSet().voters())) { if (state.updateLocalState(endOffsetMetadata, partitionState.lastVoterSet())) {
onUpdateLeaderHighWatermark(state, currentTimeMs); onUpdateLeaderHighWatermark(state, currentTimeMs);
} }
@ -452,9 +454,9 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
quorum = new QuorumState( quorum = new QuorumState(
nodeId, nodeId,
nodeDirectoryId, nodeDirectoryId,
channel.listenerName(),
partitionState::lastVoterSet, partitionState::lastVoterSet,
partitionState::lastKraftVersion, partitionState::lastKraftVersion,
localListeners,
quorumConfig.electionTimeoutMs(), quorumConfig.electionTimeoutMs(),
quorumConfig.fetchTimeoutMs(), quorumConfig.fetchTimeoutMs(),
quorumStateStore, quorumStateStore,
@ -613,22 +615,32 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
private void transitionToFollower( private void transitionToFollower(
int epoch, int epoch,
Node leader, int leaderId,
Endpoints endpoints,
long currentTimeMs long currentTimeMs
) { ) {
quorum.transitionToFollower(epoch, leader); quorum.transitionToFollower(epoch, leaderId, endpoints);
maybeFireLeaderChange(); maybeFireLeaderChange();
onBecomeFollower(currentTimeMs); onBecomeFollower(currentTimeMs);
} }
private VoteResponseData buildVoteResponse(Errors partitionLevelError, boolean voteGranted) { private VoteResponseData buildVoteResponse(
return VoteResponse.singletonResponse( ListenerName listenerName,
short apiVersion,
Errors partitionLevelError,
boolean voteGranted
) {
return RaftUtil.singletonVoteResponse(
listenerName,
apiVersion,
Errors.NONE, Errors.NONE,
log.topicPartition(), log.topicPartition(),
partitionLevelError, partitionLevelError,
quorum.epoch(), quorum.epoch(),
quorum.leaderIdOrSentinel(), quorum.leaderIdOrSentinel(),
voteGranted); voteGranted,
quorum.leaderEndpoints()
);
} }
/** /**
@ -638,8 +650,6 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
* but different from this node * but different from this node
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down * - {@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#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 * - {@link Errors#INVALID_REQUEST} if the last epoch or offset are invalid
*/ */
private VoteResponseData handleVoteRequest( private VoteResponseData handleVoteRequest(
@ -665,20 +675,52 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
int lastEpoch = partitionRequest.lastOffsetEpoch(); int lastEpoch = partitionRequest.lastOffsetEpoch();
long lastEpochEndOffset = partitionRequest.lastOffset(); long lastEpochEndOffset = partitionRequest.lastOffset();
if (lastEpochEndOffset < 0 || lastEpoch < 0 || lastEpoch >= candidateEpoch) { 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); Optional<Errors> errorOpt = validateVoterOnlyRequest(candidateId, candidateEpoch);
if (errorOpt.isPresent()) { if (errorOpt.isPresent()) {
return buildVoteResponse(errorOpt.get(), false); return buildVoteResponse(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
errorOpt.get(),
false
);
} }
if (candidateEpoch > quorum.epoch()) { if (candidateEpoch > quorum.epoch()) {
transitionToUnattached(candidateEpoch); 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); OffsetAndEpoch lastEpochEndOffsetAndEpoch = new OffsetAndEpoch(lastEpochEndOffset, lastEpoch);
ReplicaKey candidateKey = ReplicaKey.of(candidateId, Optional.empty()); ReplicaKey candidateKey = ReplicaKey.of(
candidateId,
partitionRequest.candidateDirectoryId()
);
boolean voteGranted = quorum.canGrantVote( boolean voteGranted = quorum.canGrantVote(
candidateKey, candidateKey,
lastEpochEndOffsetAndEpoch.compareTo(endOffset()) >= 0 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"); 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( private boolean handleVoteResponse(
@ -716,6 +763,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
Optional<Boolean> handled = maybeHandleCommonResponse( Optional<Boolean> handled = maybeHandleCommonResponse(
error, responseLeaderId, responseEpoch, currentTimeMs); error, responseLeaderId, responseEpoch, currentTimeMs);
// KAFKA-16529 will need to handle the INVALID_VOTER_KEY error when handling the response
if (handled.isPresent()) { if (handled.isPresent()) {
return handled.get(); return handled.get();
} else if (error == Errors.NONE) { } else if (error == Errors.NONE) {
@ -763,22 +811,31 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
} }
private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) { private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) {
if (positionInSuccessors <= 0 || positionInSuccessors >= totalNumSuccessors) { if (positionInSuccessors == 0) {
throw new IllegalArgumentException("Position " + positionInSuccessors + " should be larger than zero" + return 0;
" and smaller than total number of successors " + totalNumSuccessors); } else if (positionInSuccessors < 0 || positionInSuccessors >= totalNumSuccessors) {
return quorumConfig.electionBackoffMaxMs();
} }
int retryBackOffBaseMs = quorumConfig.electionBackoffMaxMs() >> (totalNumSuccessors - 1); int retryBackOffBaseMs = quorumConfig.electionBackoffMaxMs() >> (totalNumSuccessors - 1);
return Math.min(quorumConfig.electionBackoffMaxMs(), retryBackOffBaseMs << (positionInSuccessors - 1)); return Math.min(quorumConfig.electionBackoffMaxMs(), retryBackOffBaseMs << (positionInSuccessors - 1));
} }
private BeginQuorumEpochResponseData buildBeginQuorumEpochResponse(Errors partitionLevelError) { private BeginQuorumEpochResponseData buildBeginQuorumEpochResponse(
return BeginQuorumEpochResponse.singletonResponse( ListenerName listenerName,
short apiVersion,
Errors partitionLevelError
) {
return RaftUtil.singletonBeginQuorumEpochResponse(
listenerName,
apiVersion,
Errors.NONE, Errors.NONE,
log.topicPartition(), log.topicPartition(),
partitionLevelError, partitionLevelError,
quorum.epoch(), 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 * - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request
* but different from this node * but different from this node
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down * - {@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 * - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch
*/ */
private BeginQuorumEpochResponseData handleBeginQuorumEpochRequest( private BeginQuorumEpochResponseData handleBeginQuorumEpochRequest(
@ -814,15 +869,51 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
Optional<Errors> errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch); Optional<Errors> errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch);
if (errorOpt.isPresent()) { 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( maybeTransition(
partitionState.lastVoterSet().voterNode(requestLeaderId, channel.listenerName()), OptionalInt.of(requestLeaderId),
requestEpoch, requestEpoch,
leaderEndpoints,
currentTimeMs 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( private boolean handleBeginQuorumEpochResponse(
@ -849,6 +940,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
Optional<Boolean> handled = maybeHandleCommonResponse( Optional<Boolean> handled = maybeHandleCommonResponse(
partitionError, responseLeaderId, responseEpoch, currentTimeMs); partitionError, responseLeaderId, responseEpoch, currentTimeMs);
// KAFKA-16529 will need to handle the INVALID_VOTER_KEY error when handling the response
if (handled.isPresent()) { if (handled.isPresent()) {
return handled.get(); return handled.get();
} else if (partitionError == Errors.NONE) { } else if (partitionError == Errors.NONE) {
@ -865,13 +957,21 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
} }
} }
private EndQuorumEpochResponseData buildEndQuorumEpochResponse(Errors partitionLevelError) { private EndQuorumEpochResponseData buildEndQuorumEpochResponse(
return EndQuorumEpochResponse.singletonResponse( ListenerName listenerName,
short apiVersion,
Errors partitionLevelError
) {
return RaftUtil.singletonEndQuorumEpochResponse(
listenerName,
apiVersion,
Errors.NONE, Errors.NONE,
log.topicPartition(), log.topicPartition(),
partitionLevelError, partitionLevelError,
quorum.epoch(), 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 * - {@link Errors#INCONSISTENT_CLUSTER_ID} if the cluster id is presented in request
* but different from this node * but different from this node
* - {@link Errors#BROKER_NOT_AVAILABLE} if this node is currently shutting down * - {@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 * - {@link Errors#FENCED_LEADER_EPOCH} if the epoch is smaller than this node's epoch
*/ */
private EndQuorumEpochResponseData handleEndQuorumEpochRequest( private EndQuorumEpochResponseData handleEndQuorumEpochRequest(
@ -907,40 +1005,73 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
Optional<Errors> errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch); Optional<Errors> errorOpt = validateVoterOnlyRequest(requestLeaderId, requestEpoch);
if (errorOpt.isPresent()) { 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( maybeTransition(
partitionState.lastVoterSet().voterNode(requestLeaderId, channel.listenerName()), OptionalInt.of(requestLeaderId),
requestEpoch, requestEpoch,
leaderEndpoints,
currentTimeMs currentTimeMs
); );
if (quorum.isFollower()) { if (quorum.isFollower()) {
FollowerState state = quorum.followerStateOrThrow(); FollowerState state = quorum.followerStateOrThrow();
if (state.leader().id() == requestLeaderId) { if (state.leaderId() == requestLeaderId) {
List<Integer> preferredSuccessors = partitionRequest.preferredSuccessors(); List<ReplicaKey> preferredCandidates = EndQuorumEpochRequest
long electionBackoffMs = endEpochElectionBackoff(preferredSuccessors); .preferredCandidates(partitionRequest)
logger.debug("Overriding follower fetch timeout to {} after receiving " + .stream()
"EndQuorumEpoch request from leader {} in epoch {}", electionBackoffMs, .map(replica -> ReplicaKey.of(replica.candidateId(), replica.candidateDirectoryId()))
requestLeaderId, requestEpoch); .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); 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 // 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 // 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 // voter has a higher chance to be elected. If the node's priority is highest, become
// candidate immediately instead of waiting for next poll. // candidate immediately instead of waiting for next poll.
int position = preferredSuccessors.indexOf(quorum.localIdOrThrow()); int position = 0;
if (position <= 0) { for (ReplicaKey candidate : preferredCandidates) {
return 0; if (candidate.id() == quorum.localIdOrThrow()) {
} else { if (!candidate.directoryId().isPresent() ||
return strictExponentialElectionBackoffMs(position, preferredSuccessors.size()); candidate.directoryId().get().equals(quorum.localDirectoryId())
) {
// Found ourselves in the preferred candidate list
break;
} }
} }
position++;
}
return strictExponentialElectionBackoffMs(position, preferredCandidates.size());
}
private boolean handleEndQuorumEpochResponse( private boolean handleEndQuorumEpochResponse(
RaftResponse.Inbound responseMetadata, RaftResponse.Inbound responseMetadata,
@ -977,12 +1108,22 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
} }
private FetchResponseData buildFetchResponse( private FetchResponseData buildFetchResponse(
ListenerName listenerName,
short apiVersion,
Errors error, Errors error,
Records records, Records records,
ValidOffsetAndEpoch validOffsetAndEpoch, ValidOffsetAndEpoch validOffsetAndEpoch,
Optional<LogOffsetMetadata> highWatermark Optional<LogOffsetMetadata> highWatermark
) { ) {
return RaftUtil.singletonFetchResponse(log.topicPartition(), log.topicId(), Errors.NONE, partitionData -> { return RaftUtil.singletonFetchResponse(
listenerName,
apiVersion,
log.topicPartition(),
log.topicId(),
Errors.NONE,
quorum.leaderIdOrSentinel(),
quorum.leaderEndpoints(),
partitionData -> {
partitionData partitionData
.setRecords(records) .setRecords(records)
.setErrorCode(error.code()) .setErrorCode(error.code())
@ -1008,14 +1149,19 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
break; break;
default: default:
} }
}); }
);
} }
private FetchResponseData buildEmptyFetchResponse( private FetchResponseData buildEmptyFetchResponse(
ListenerName listenerName,
short apiVersion,
Errors error, Errors error,
Optional<LogOffsetMetadata> highWatermark Optional<LogOffsetMetadata> highWatermark
) { ) {
return buildFetchResponse( return buildFetchResponse(
listenerName,
apiVersion,
error, error,
MemoryRecords.EMPTY, MemoryRecords.EMPTY,
ValidOffsetAndEpoch.valid(), ValidOffsetAndEpoch.valid(),
@ -1069,12 +1215,26 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|| fetchPartition.lastFetchedEpoch() < 0 || fetchPartition.lastFetchedEpoch() < 0
|| fetchPartition.lastFetchedEpoch() > fetchPartition.currentLeaderEpoch()) { || fetchPartition.lastFetchedEpoch() > fetchPartition.currentLeaderEpoch()) {
return completedFuture( return completedFuture(
buildEmptyFetchResponse(Errors.INVALID_REQUEST, Optional.empty()) buildEmptyFetchResponse(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
Errors.INVALID_REQUEST,
Optional.empty()
)
); );
} }
int replicaId = FetchRequest.replicaId(request); ReplicaKey replicaKey = ReplicaKey.of(
FetchResponseData response = tryCompleteFetchRequest(replicaId, fetchPartition, currentTimeMs); FetchRequest.replicaId(request),
fetchPartition.replicaDirectoryId()
);
FetchResponseData response = tryCompleteFetchRequest(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
replicaKey,
fetchPartition,
currentTimeMs
);
FetchResponseData.PartitionData partitionResponse = FetchResponseData.PartitionData partitionResponse =
response.responses().get(0).partitions().get(0); response.responses().get(0).partitions().get(0);
@ -1111,32 +1271,53 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
return response; return response;
} else { } else {
// If there was any error other than REQUEST_TIMED_OUT, return it. // If there was any error other than REQUEST_TIMED_OUT, return it.
logger.info("Failed to handle fetch from {} at {} due to {}", logger.info(
replicaId, fetchPartition.fetchOffset(), error); "Failed to handle fetch from {} at {} due to {}",
return buildEmptyFetchResponse(error, Optional.empty()); replicaKey,
fetchPartition.fetchOffset(),
error
);
return buildEmptyFetchResponse(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
error,
Optional.empty()
);
} }
} }
// FIXME: `completionTimeMs`, which can be null // FIXME: `completionTimeMs`, which can be null
logger.trace("Completing delayed fetch from {} starting at offset {} at {}", logger.trace(
replicaId, fetchPartition.fetchOffset(), completionTimeMs); "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 // 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 // future successfully. This is true because only the polling thread appends record batches to
// the log from maybeAppendBatches. // the log from maybeAppendBatches.
return tryCompleteFetchRequest(replicaId, fetchPartition, time.milliseconds()); return tryCompleteFetchRequest(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
replicaKey,
fetchPartition,
time.milliseconds()
);
}); });
} }
private FetchResponseData tryCompleteFetchRequest( private FetchResponseData tryCompleteFetchRequest(
int replicaId, ListenerName listenerName,
short apiVersion,
ReplicaKey replicaKey,
FetchRequestData.FetchPartition request, FetchRequestData.FetchPartition request,
long currentTimeMs long currentTimeMs
) { ) {
try { try {
Optional<Errors> errorOpt = validateLeaderOnlyRequest(request.currentLeaderEpoch()); Optional<Errors> errorOpt = validateLeaderOnlyRequest(request.currentLeaderEpoch());
if (errorOpt.isPresent()) { if (errorOpt.isPresent()) {
return buildEmptyFetchResponse(errorOpt.get(), Optional.empty()); return buildEmptyFetchResponse(listenerName, apiVersion, errorOpt.get(), Optional.empty());
} }
long fetchOffset = request.fetchOffset(); long fetchOffset = request.fetchOffset();
@ -1157,7 +1338,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
if (validOffsetAndEpoch.kind() == ValidOffsetAndEpoch.Kind.VALID) { if (validOffsetAndEpoch.kind() == ValidOffsetAndEpoch.Kind.VALID) {
LogFetchInfo info = log.read(fetchOffset, Isolation.UNCOMMITTED); 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); onUpdateLeaderHighWatermark(state, currentTimeMs);
} }
@ -1166,10 +1347,17 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
records = MemoryRecords.EMPTY; records = MemoryRecords.EMPTY;
} }
return buildFetchResponse(Errors.NONE, records, validOffsetAndEpoch, state.highWatermark()); return buildFetchResponse(
listenerName,
apiVersion,
Errors.NONE,
records,
validOffsetAndEpoch,
state.highWatermark()
);
} catch (Exception e) { } catch (Exception e) {
logger.error("Caught unexpected error in fetch completion of request {}", request, 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(); LeaderState<T> leaderState = quorum.leaderStateOrThrow();
return DescribeQuorumResponse.singletonResponse( return RaftUtil.singletonDescribeQuorumResponse(
requestMetadata.apiVersion(),
log.topicPartition(), log.topicPartition(),
leaderState.describeQuorum(currentTimeMs), leaderState.describeQuorum(currentTimeMs),
requestMetadata.apiVersion() < DescribeQuorumResponseData.Node.LOWEST_SUPPORTED_VERSION leaderState.nodes(currentTimeMs)
? null
: leaderState.nodes(currentTimeMs)
); );
} }
@ -1399,8 +1586,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
data.topics().get(0).partitions().get(0).partition() data.topics().get(0).partitions().get(0).partition()
); );
return FetchSnapshotResponse.singleton( return RaftUtil.singletonFetchSnapshotResponse(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
unknownTopicPartition, unknownTopicPartition,
quorum.leaderIdOrSentinel(),
quorum.leaderEndpoints(),
responsePartitionSnapshot -> responsePartitionSnapshot responsePartitionSnapshot -> responsePartitionSnapshot
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code())
); );
@ -1411,8 +1602,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
partitionSnapshot.currentLeaderEpoch() partitionSnapshot.currentLeaderEpoch()
); );
if (leaderValidation.isPresent()) { if (leaderValidation.isPresent()) {
return FetchSnapshotResponse.singleton( return RaftUtil.singletonFetchSnapshotResponse(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
log.topicPartition(), log.topicPartition(),
quorum.leaderIdOrSentinel(),
quorum.leaderEndpoints(),
responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot) responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot)
.setErrorCode(leaderValidation.get().code()) .setErrorCode(leaderValidation.get().code())
); );
@ -1424,8 +1619,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
); );
Optional<RawSnapshotReader> snapshotOpt = log.readSnapshot(snapshotId); Optional<RawSnapshotReader> snapshotOpt = log.readSnapshot(snapshotId);
if (!snapshotOpt.isPresent()) { if (!snapshotOpt.isPresent()) {
return FetchSnapshotResponse.singleton( return RaftUtil.singletonFetchSnapshotResponse(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
log.topicPartition(), log.topicPartition(),
quorum.leaderIdOrSentinel(),
quorum.leaderEndpoints(),
responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot) responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot)
.setErrorCode(Errors.SNAPSHOT_NOT_FOUND.code()) .setErrorCode(Errors.SNAPSHOT_NOT_FOUND.code())
); );
@ -1434,8 +1633,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
RawSnapshotReader snapshot = snapshotOpt.get(); RawSnapshotReader snapshot = snapshotOpt.get();
long snapshotSize = snapshot.sizeInBytes(); long snapshotSize = snapshot.sizeInBytes();
if (partitionSnapshot.position() < 0 || partitionSnapshot.position() >= snapshotSize) { if (partitionSnapshot.position() < 0 || partitionSnapshot.position() >= snapshotSize) {
return FetchSnapshotResponse.singleton( return RaftUtil.singletonFetchSnapshotResponse(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
log.topicPartition(), log.topicPartition(),
quorum.leaderIdOrSentinel(),
quorum.leaderEndpoints(),
responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot) responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot)
.setErrorCode(Errors.POSITION_OUT_OF_RANGE.code()) .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)); UnalignedRecords records = snapshot.slice(partitionSnapshot.position(), Math.min(data.maxBytes(), maxSnapshotSize));
LeaderState<T> state = quorum.leaderStateOrThrow(); 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(), log.topicPartition(),
quorum.leaderIdOrSentinel(),
quorum.leaderEndpoints(),
responsePartitionSnapshot -> { responsePartitionSnapshot -> {
addQuorumLeader(responsePartitionSnapshot) addQuorumLeader(responsePartitionSnapshot)
.snapshotId() .snapshotId()
@ -1648,11 +1861,11 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
int epoch, int epoch,
long currentTimeMs long currentTimeMs
) { ) {
Optional<Node> leader = leaderId.isPresent() ? Endpoints leaderEndpoints = leaderId.isPresent() ?
partitionState leaderEndpoints = partitionState
.lastVoterSet() .lastVoterSet()
.voterNode(leaderId.getAsInt(), channel.listenerName()) : .listeners(leaderId.getAsInt()) :
Optional.empty(); Endpoints.empty();
if (epoch < quorum.epoch() || error == Errors.UNKNOWN_LEADER_EPOCH) { if (epoch < quorum.epoch() || error == Errors.UNKNOWN_LEADER_EPOCH) {
// We have a larger epoch, so the response is no longer relevant // 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 // The response indicates that the request had a stale epoch, but we need
// to validate the epoch from the response against our current state. // to validate the epoch from the response against our current state.
maybeTransition(leader, epoch, currentTimeMs); maybeTransition(leaderId, epoch, leaderEndpoints, currentTimeMs);
return Optional.of(true); return Optional.of(true);
} else if (epoch == quorum.epoch() } else if (epoch == quorum.epoch()
&& leader.isPresent() && leaderId.isPresent()
&& !quorum.hasLeader()) { && !quorum.hasLeader()) {
// Since we are transitioning to Follower, we will only forward the // 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. // the request be retried immediately (if needed) after the transition.
// This handling allows an observer to discover the leader and append // This handling allows an observer to discover the leader and append
// to the log in the same Fetch request. // to the log in the same Fetch request.
transitionToFollower(epoch, leader.get(), currentTimeMs); transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs);
if (error == Errors.NONE) { if (error == Errors.NONE) {
return Optional.empty(); return Optional.empty();
} else { } else {
@ -1696,28 +1909,28 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
} }
private void maybeTransition( private void maybeTransition(
Optional<Node> leader, OptionalInt leaderId,
int epoch, int epoch,
Endpoints leaderEndpoints,
long currentTimeMs long currentTimeMs
) { ) {
OptionalInt leaderId = leader.isPresent() ?
OptionalInt.of(leader.get().id()) :
OptionalInt.empty();
if (!hasConsistentLeader(epoch, leaderId)) { 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 " + " and epoch " + epoch + " which is inconsistent with current leader " +
quorum.leaderId() + " and epoch " + quorum.epoch()); quorum.leaderId() + " and epoch " + quorum.epoch());
} else if (epoch > quorum.epoch()) { } else if (epoch > quorum.epoch()) {
if (leader.isPresent()) { if (leaderId.isPresent()) {
transitionToFollower(epoch, leader.get(), currentTimeMs); transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs);
} else { } else {
transitionToUnattached(epoch); transitionToUnattached(epoch);
} }
} else if (leader.isPresent() && !quorum.hasLeader()) { } else if (
// The request or response indicates the leader of the current epoch, leaderId.isPresent() &&
// which is currently unknown (!quorum.hasLeader() || leaderEndpoints.size() > quorum.leaderEndpoints().size())
transitionToFollower(epoch, leader.get(), currentTimeMs); ) {
// 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. * Validate a request which is intended for the current quorum leader.
* If an error is present in the returned value, it should be returned * 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) -> { responseFuture.whenComplete((response, exception) -> {
final ApiMessage message; ApiMessage message = response;
if (response != null) { if (message == null) {
message = response;
} else {
message = RaftUtil.errorResponse(apiKey, Errors.forException(exception)); 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( RaftRequest.Outbound requestMessage = new RaftRequest.Outbound(
correlationId, correlationId,
request.highestSupportedVersion(),
request, request,
destination, destination,
currentTimeMs 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 // 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 // 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. // 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 // Once the request has timed out backoff the connection
requestManager.reset(state.leader()); requestManager.reset(leaderNode);
backoffMs = maybeSendAnyVoterFetch(currentTimeMs); backoffMs = maybeSendAnyVoterFetch(currentTimeMs);
} else if (requestManager.isBackingOff(state.leader(), currentTimeMs)) { } else if (requestManager.isBackingOff(leaderNode, currentTimeMs)) {
backoffMs = maybeSendAnyVoterFetch(currentTimeMs); backoffMs = maybeSendAnyVoterFetch(currentTimeMs);
} else if (!requestManager.hasAnyInflightRequest(currentTimeMs)) { } else if (!requestManager.hasAnyInflightRequest(currentTimeMs)) {
backoffMs = maybeSendFetchOrFetchSnapshot(state, currentTimeMs); backoffMs = maybeSendFetchOrFetchSnapshot(state, currentTimeMs);
@ -2259,7 +2481,11 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
requestSupplier = this::buildFetchRequest; requestSupplier = this::buildFetchRequest;
} }
return maybeSendRequest(currentTimeMs, state.leader(), requestSupplier); return maybeSendRequest(
currentTimeMs,
state.leaderNode(channel.listenerName()),
requestSupplier
);
} }
private long pollVoted(long currentTimeMs) { private long pollVoted(long currentTimeMs) {

View File

@ -17,6 +17,7 @@
package org.apache.kafka.raft; package org.apache.kafka.raft;
import org.apache.kafka.common.protocol.ApiMessage; 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.requests.RequestHeader;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.FaultHandler;
@ -101,11 +102,13 @@ public class KafkaRaftClientDriver<T> extends ShutdownableThread {
} }
public CompletableFuture<ApiMessage> handleRequest( public CompletableFuture<ApiMessage> handleRequest(
RequestContext context,
RequestHeader header, RequestHeader header,
ApiMessage request, ApiMessage request,
long createdTimeMs long createdTimeMs
) { ) {
RaftRequest.Inbound inboundRequest = new RaftRequest.Inbound( RaftRequest.Inbound inboundRequest = new RaftRequest.Inbound(
context.listenerName,
header.correlationId(), header.correlationId(),
header.apiVersion(), header.apiVersion(),
request, request,

View File

@ -16,7 +16,6 @@
*/ */
package org.apache.kafka.raft; package org.apache.kafka.raft;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.DescribeQuorumResponseData; import org.apache.kafka.common.message.DescribeQuorumResponseData;
import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage;
import org.apache.kafka.common.message.LeaderChangeMessage.Voter; import org.apache.kafka.common.message.LeaderChangeMessage.Voter;
@ -36,13 +35,13 @@ import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Optional; import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; 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 * 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 long OBSERVER_SESSION_TIMEOUT_MS = 300_000L;
static final double CHECK_QUORUM_TIMEOUT_FACTOR = 1.5; static final double CHECK_QUORUM_TIMEOUT_FACTOR = 1.5;
private final int localId; private final ReplicaKey localReplicaKey;
private final Uuid localDirectoryId;
private final int epoch; private final int epoch;
private final long epochStartOffset; private final long epochStartOffset;
private final Set<Integer> grantingVoters; private final Set<Integer> grantingVoters;
private final Endpoints endpoints;
private Optional<LogOffsetMetadata> highWatermark = Optional.empty(); private Optional<LogOffsetMetadata> highWatermark = Optional.empty();
private final Map<Integer, ReplicaState> voterStates = new HashMap<>(); private Map<Integer, ReplicaState> voterStates = new HashMap<>();
private final Map<Integer, ReplicaState> observerStates = new HashMap<>(); private final Map<ReplicaKey, ReplicaState> observerStates = new HashMap<>();
private final Logger log; private final Logger log;
private final BatchAccumulator<T> accumulator; private final BatchAccumulator<T> accumulator;
// The set includes all of the followers voters that FETCH or FETCH_SNAPSHOT during the current checkQuorumTimer interval. // 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( protected LeaderState(
Time time, Time time,
int localId, ReplicaKey localReplicaKey,
Uuid localDirectoryId,
int epoch, int epoch,
long epochStartOffset, long epochStartOffset,
Map<Integer, VoterSet.VoterNode> voters, VoterSet voters,
Set<Integer> grantingVoters, Set<Integer> grantingVoters,
BatchAccumulator<T> accumulator, BatchAccumulator<T> accumulator,
Endpoints endpoints,
int fetchTimeoutMs, int fetchTimeoutMs,
LogContext logContext LogContext logContext
) { ) {
this.localId = localId; this.localReplicaKey = localReplicaKey;
this.localDirectoryId = localDirectoryId;
this.epoch = epoch; this.epoch = epoch;
this.epochStartOffset = epochStartOffset; this.epochStartOffset = epochStartOffset;
this.endpoints = endpoints;
for (Map.Entry<Integer, VoterSet.VoterNode> voter : voters.entrySet()) { for (VoterSet.VoterNode voterNode: voters.voterNodes()) {
boolean hasAcknowledgedLeader = voter.getKey() == localId; boolean hasAcknowledgedLeader = voterNode.isVoter(localReplicaKey);
this.voterStates.put(voter.getKey(), new ReplicaState(voter.getKey(), voter.getValue().voterKey().directoryId(), hasAcknowledgedLeader)); this.voterStates.put(
voterNode.voterKey().id(),
new ReplicaState(voterNode.voterKey(), hasAcknowledgedLeader)
);
} }
this.grantingVoters = Collections.unmodifiableSet(new HashSet<>(grantingVoters)); this.grantingVoters = Collections.unmodifiableSet(new HashSet<>(grantingVoters));
this.log = logContext.logger(LeaderState.class); 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. * 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 * @param currentTimeMs the current timestamp in millisecond
* @return the remainingMs before the checkQuorumTimer expired * @return the remainingMs before the checkQuorumTimer expired
@ -118,10 +122,12 @@ public class LeaderState<T> implements EpochState {
long remainingMs = checkQuorumTimer.remainingMs(); long remainingMs = checkQuorumTimer.remainingMs();
if (remainingMs == 0) { if (remainingMs == 0) {
log.info( 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, checkQuorumTimeoutMs,
fetchedVoters, fetchedVoters,
voterStates.keySet()); voterStates.values().stream().map(voter -> voter.replicaKey)
);
} }
return remainingMs; return remainingMs;
} }
@ -132,15 +138,15 @@ public class LeaderState<T> implements EpochState {
* @param id the node id * @param id the node id
* @param currentTimeMs the current timestamp in millisecond * @param currentTimeMs the current timestamp in millisecond
*/ */
public void updateCheckQuorumForFollowingVoter(int id, long currentTimeMs) { public void updateCheckQuorumForFollowingVoter(ReplicaKey replicaKey, long currentTimeMs) {
updateFetchedVoters(id); updateFetchedVoters(replicaKey);
// The majority number of the voters. Ex: 2 for 3 voters, 3 for 4 voters... etc. // The majority number of the voters. Ex: 2 for 3 voters, 3 for 4 voters... etc.
int majority = (voterStates.size() / 2) + 1; int majority = (voterStates.size() / 2) + 1;
// If the leader is in the voter set, it should be implicitly counted as part of the // 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. // 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 // 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. // majority can only be composed of fetched voters.
if (voterStates.containsKey(localId)) { if (voterStates.containsKey(localReplicaKey.id())) {
majority = majority - 1; majority = majority - 1;
} }
@ -151,13 +157,14 @@ public class LeaderState<T> implements EpochState {
} }
} }
private void updateFetchedVoters(int id) { private void updateFetchedVoters(ReplicaKey replicaKey) {
if (id == localId) { if (replicaKey.id() == localReplicaKey.id()) {
throw new IllegalArgumentException("Received a FETCH/FETCH_SNAPSHOT request from the leader itself."); throw new IllegalArgumentException("Received a FETCH/FETCH_SNAPSHOT request from the leader itself.");
} }
if (isVoter(id)) { ReplicaState state = voterStates.get(replicaKey.id());
fetchedVoters.add(id); if (state != null && state.matchesKey(replicaKey)) {
fetchedVoters.add(replicaKey.id());
} }
} }
@ -200,7 +207,7 @@ public class LeaderState<T> implements EpochState {
@Override @Override
public ElectionState election() { public ElectionState election() {
return ElectionState.withElectedLeader(epoch, localId, voterStates.keySet()); return ElectionState.withElectedLeader(epoch, localReplicaKey.id(), voterStates.keySet());
} }
@Override @Override
@ -208,30 +215,28 @@ public class LeaderState<T> implements EpochState {
return epoch; return epoch;
} }
@Override
public Endpoints leaderEndpoints() {
return endpoints;
}
public Set<Integer> grantingVoters() { public Set<Integer> grantingVoters() {
return this.grantingVoters; return this.grantingVoters;
} }
public int localId() {
return localId;
}
public Uuid localDirectoryId() {
return localDirectoryId;
}
public Set<Integer> nonAcknowledgingVoters() { public Set<Integer> nonAcknowledgingVoters() {
Set<Integer> nonAcknowledging = new HashSet<>(); Set<Integer> nonAcknowledging = new HashSet<>();
for (ReplicaState state : voterStates.values()) { for (ReplicaState state : voterStates.values()) {
if (!state.hasAcknowledgedLeader) if (!state.hasAcknowledgedLeader)
nonAcknowledging.add(state.nodeId); nonAcknowledging.add(state.replicaKey.id());
} }
return nonAcknowledging; return nonAcknowledging;
} }
private boolean maybeUpdateHighWatermark() { private boolean maybeUpdateHighWatermark() {
// Find the largest offset which is replicated to a majority of replicas (the leader counts) // 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; int indexOfHw = voterStates.size() / 2;
Optional<LogOffsetMetadata> highWatermarkUpdateOpt = followersByDescendingFetchOffset.get(indexOfHw).endOffset; Optional<LogOffsetMetadata> highWatermarkUpdateOpt = followersByDescendingFetchOffset.get(indexOfHw).endOffset;
@ -323,17 +328,19 @@ public class LeaderState<T> implements EpochState {
*/ */
public boolean updateLocalState( public boolean updateLocalState(
LogOffsetMetadata endOffsetMetadata, LogOffsetMetadata endOffsetMetadata,
Map<Integer, VoterSet.VoterNode> lastVoters VoterSet lastVoterSet
) { ) {
ReplicaState state = getOrCreateReplicaState(localId, localDirectoryId); ReplicaState state = getOrCreateReplicaState(localReplicaKey);
state.endOffset.ifPresent(currentEndOffset -> { state.endOffset.ifPresent(currentEndOffset -> {
if (currentEndOffset.offset > endOffsetMetadata.offset) { if (currentEndOffset.offset > endOffsetMetadata.offset) {
throw new IllegalStateException("Detected non-monotonic update of local " + throw new IllegalStateException("Detected non-monotonic update of local " +
"end offset: " + currentEndOffset.offset + " -> " + endOffsetMetadata.offset); "end offset: " + currentEndOffset.offset + " -> " + endOffsetMetadata.offset);
} }
}); });
state.updateLeaderEndOffset(endOffsetMetadata); state.updateLeaderEndOffset(endOffsetMetadata);
updateVoterAndObserverStates(lastVoters); updateVoterAndObserverStates(lastVoterSet);
return maybeUpdateHighWatermark(); 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 * @return true if the high watermark is updated as a result of this call
*/ */
public boolean updateReplicaState( public boolean updateReplicaState(
int replicaId, ReplicaKey replicaKey,
Uuid replicaDirectoryId,
long currentTimeMs, long currentTimeMs,
LogOffsetMetadata fetchOffsetMetadata LogOffsetMetadata fetchOffsetMetadata
) { ) {
// Ignore fetches from negative replica id, as it indicates // Ignore fetches from negative replica id, as it indicates
// the fetch is from non-replica. For example, a consumer. // the fetch is from non-replica. For example, a consumer.
if (replicaId < 0) { if (replicaKey.id() < 0) {
return false; return false;
} else if (replicaId == localId) { } else if (replicaKey.id() == localReplicaKey.id()) {
throw new IllegalStateException("Remote replica ID " + replicaId + " matches the local leader 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 -> { state.endOffset.ifPresent(currentEndOffset -> {
if (currentEndOffset.offset > fetchOffsetMetadata.offset) { if (currentEndOffset.offset > fetchOffsetMetadata.offset) {
log.warn("Detected non-monotonic update of fetch offset from nodeId {}: {} -> {}", 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( state.updateFollowerState(
currentTimeMs, currentTimeMs,
fetchOffsetMetadata, fetchOffsetMetadata,
leaderEndOffsetOpt leaderEndOffsetOpt
); );
updateCheckQuorumForFollowingVoter(replicaId, currentTimeMs); updateCheckQuorumForFollowingVoter(replicaKey, currentTimeMs);
return isVoter(state.nodeId) && maybeUpdateHighWatermark(); return isVoter(state.replicaKey) && maybeUpdateHighWatermark();
} }
public List<Integer> nonLeaderVotersByDescendingFetchOffset() { public List<Integer> nonLeaderVotersByDescendingFetchOffset() {
return followersByDescendingFetchOffset().stream() return followersByDescendingFetchOffset()
.filter(state -> state.nodeId != localId) .filter(state -> !state.matchesKey(localReplicaKey))
.map(state -> state.nodeId) .map(state -> state.replicaKey.id())
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
private List<ReplicaState> followersByDescendingFetchOffset() { private Stream<ReplicaState> followersByDescendingFetchOffset() {
return new ArrayList<>(this.voterStates.values()).stream() return voterStates
.sorted() .values()
.collect(Collectors.toList()); .stream()
.sorted();
} }
public void addAcknowledgementFrom(int remoteNodeId) { public void addAcknowledgementFrom(int remoteNodeId) {
@ -400,8 +410,9 @@ public class LeaderState<T> implements EpochState {
private ReplicaState ensureValidVoter(int remoteNodeId) { private ReplicaState ensureValidVoter(int remoteNodeId) {
ReplicaState state = voterStates.get(remoteNodeId); ReplicaState state = voterStates.get(remoteNodeId);
if (state == null) if (state == null) {
throw new IllegalArgumentException("Unexpected acknowledgement from non-voter " + remoteNodeId); throw new IllegalArgumentException("Unexpected acknowledgement from non-voter " + remoteNodeId);
}
return state; return state;
} }
@ -409,55 +420,58 @@ public class LeaderState<T> implements EpochState {
return epochStartOffset; return epochStartOffset;
} }
private ReplicaState getOrCreateReplicaState(int remoteNodeId, Uuid remoteNodeDirectory) { private ReplicaState getOrCreateReplicaState(ReplicaKey replicaKey) {
ReplicaState state = voterStates.get(remoteNodeId); ReplicaState state = voterStates.get(replicaKey.id());
if (state == null) { if (state == null || !state.matchesKey(replicaKey)) {
observerStates.putIfAbsent(remoteNodeId, new ReplicaState(remoteNodeId, Optional.of(remoteNodeDirectory), false)); observerStates.putIfAbsent(replicaKey, new ReplicaState(replicaKey, false));
return observerStates.get(remoteNodeId); return observerStates.get(replicaKey);
} }
return state; 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) { public DescribeQuorumResponseData.PartitionData describeQuorum(long currentTimeMs) {
clearInactiveObservers(currentTimeMs); clearInactiveObservers(currentTimeMs);
return new DescribeQuorumResponseData.PartitionData() return new DescribeQuorumResponseData.PartitionData()
.setErrorCode(Errors.NONE.code()) .setErrorCode(Errors.NONE.code())
.setLeaderId(localId) .setLeaderId(localReplicaKey.id())
.setLeaderEpoch(epoch) .setLeaderEpoch(epoch)
.setHighWatermark(highWatermark.map(offsetMetadata -> offsetMetadata.offset).orElse(-1L)) .setHighWatermark(highWatermark.map(offsetMetadata -> offsetMetadata.offset).orElse(-1L))
.setCurrentVoters(describeReplicaStates(voterStates, currentTimeMs)) .setCurrentVoters(describeReplicaStates(voterStates.values(), currentTimeMs))
.setObservers(describeReplicaStates(observerStates, currentTimeMs)); .setObservers(describeReplicaStates(observerStates.values(), currentTimeMs));
} }
public DescribeQuorumResponseData.NodeCollection nodes(long currentTimeMs) { public DescribeQuorumResponseData.NodeCollection nodes(long currentTimeMs) {
clearInactiveObservers(currentTimeMs); clearInactiveObservers(currentTimeMs);
return nodes(voterStates.values(), observerStates.values()); DescribeQuorumResponseData.NodeCollection nodes = 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()));
} }
});
private static DescribeQuorumResponseData.NodeCollection nodes(Collection<ReplicaState> voters, Collection<ReplicaState> observers) { return nodes;
DescribeQuorumResponseData.NodeCollection res = new DescribeQuorumResponseData.NodeCollection();
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));
} }
private List<DescribeQuorumResponseData.ReplicaState> describeReplicaStates( private List<DescribeQuorumResponseData.ReplicaState> describeReplicaStates(
Map<Integer, ReplicaState> state, Collection<ReplicaState> states,
long currentTimeMs long currentTimeMs
) { ) {
return state.values().stream() return states
.stream()
.map(replicaState -> describeReplicaState(replicaState, currentTimeMs)) .map(replicaState -> describeReplicaState(replicaState, currentTimeMs))
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
@ -468,7 +482,7 @@ public class LeaderState<T> implements EpochState {
) { ) {
final long lastCaughtUpTimestamp; final long lastCaughtUpTimestamp;
final long lastFetchTimestamp; final long lastFetchTimestamp;
if (replicaState.nodeId == localId) { if (replicaState.matchesKey(localReplicaKey)) {
lastCaughtUpTimestamp = currentTimeMs; lastCaughtUpTimestamp = currentTimeMs;
lastFetchTimestamp = currentTimeMs; lastFetchTimestamp = currentTimeMs;
} else { } else {
@ -476,8 +490,8 @@ public class LeaderState<T> implements EpochState {
lastFetchTimestamp = replicaState.lastFetchTimestamp; lastFetchTimestamp = replicaState.lastFetchTimestamp;
} }
return new DescribeQuorumResponseData.ReplicaState() return new DescribeQuorumResponseData.ReplicaState()
.setReplicaId(replicaState.nodeId) .setReplicaId(replicaState.replicaKey.id())
.setReplicaDirectoryId(replicaState.nodeDirectory.orElse(Uuid.ZERO_UUID)) .setReplicaDirectoryId(replicaState.replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID))
.setLogEndOffset(replicaState.endOffset.map(md -> md.offset).orElse(-1L)) .setLogEndOffset(replicaState.endOffset.map(md -> md.offset).orElse(-1L))
.setLastCaughtUpTimestamp(lastCaughtUpTimestamp) .setLastCaughtUpTimestamp(lastCaughtUpTimestamp)
.setLastFetchTimestamp(lastFetchTimestamp); .setLastFetchTimestamp(lastFetchTimestamp);
@ -490,46 +504,50 @@ public class LeaderState<T> implements EpochState {
private void clearInactiveObservers(final long currentTimeMs) { private void clearInactiveObservers(final long currentTimeMs) {
observerStates.entrySet().removeIf(integerReplicaStateEntry -> observerStates.entrySet().removeIf(integerReplicaStateEntry ->
currentTimeMs - integerReplicaStateEntry.getValue().lastFetchTimestamp >= OBSERVER_SESSION_TIMEOUT_MS && currentTimeMs - integerReplicaStateEntry.getValue().lastFetchTimestamp >= OBSERVER_SESSION_TIMEOUT_MS &&
integerReplicaStateEntry.getKey() != localId !integerReplicaStateEntry.getKey().equals(localReplicaKey)
); );
} }
private boolean isVoter(int remoteNodeId) { private boolean isVoter(ReplicaKey remoteReplicaKey) {
return voterStates.containsKey(remoteNodeId); ReplicaState state = voterStates.get(remoteReplicaKey.id());
return state != null && state.matchesKey(remoteReplicaKey);
} }
private void updateVoterAndObserverStates(Map<Integer, VoterSet.VoterNode> lastVoters) { private void updateVoterAndObserverStates(VoterSet lastVoterSet) {
// Move any replica that is not in the last voter set from voterStates to observerStates Map<Integer, ReplicaState> newVoterStates = new HashMap<>();
for (Iterator<Map.Entry<Integer, ReplicaState>> iter = voterStates.entrySet().iterator(); iter.hasNext(); ) { Map<Integer, ReplicaState> oldVoterStates = new HashMap<>(voterStates);
Map.Entry<Integer, ReplicaState> replica = iter.next();
if (!lastVoters.containsKey(replica.getKey())) {
observerStates.put(replica.getKey(), replica.getValue());
iter.remove();
}
}
// Add replicas that are in the last voter set and not in voterStates to voterStates (from observerStates // Compute the new voter states map
// if they exist) for (VoterSet.VoterNode voterNode : lastVoterSet.voterNodes()) {
for (Map.Entry<Integer, VoterSet.VoterNode> voter : lastVoters.entrySet()) { ReplicaState state = getReplicaState(voterNode.voterKey())
if (!voterStates.containsKey(voter.getKey())) { .orElse(new ReplicaState(voterNode.voterKey(), false));
Optional<ReplicaState> existingObserverState = Optional.ofNullable(observerStates.remove(voter.getKey()));
voterStates.put(voter.getKey(), existingObserverState.orElse(new ReplicaState(voter.getKey(), voter.getValue().voterKey().directoryId(), 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> { private static class ReplicaState implements Comparable<ReplicaState> {
final int nodeId; ReplicaKey replicaKey;
final Optional<Uuid> nodeDirectory;
Optional<LogOffsetMetadata> endOffset; Optional<LogOffsetMetadata> endOffset;
long lastFetchTimestamp; long lastFetchTimestamp;
long lastFetchLeaderLogEndOffset; long lastFetchLeaderLogEndOffset;
long lastCaughtUpTimestamp; long lastCaughtUpTimestamp;
boolean hasAcknowledgedLeader; boolean hasAcknowledgedLeader;
public ReplicaState(int nodeId, Optional<Uuid> nodeDirectory, boolean hasAcknowledgedLeader) { public ReplicaState(ReplicaKey replicaKey, boolean hasAcknowledgedLeader) {
this.nodeId = nodeId; this.replicaKey = replicaKey;
this.nodeDirectory = nodeDirectory;
this.endOffset = Optional.empty(); this.endOffset = Optional.empty();
this.lastFetchTimestamp = -1; this.lastFetchTimestamp = -1;
this.lastFetchLeaderLogEndOffset = -1; this.lastFetchLeaderLogEndOffset = -1;
@ -537,6 +555,41 @@ public class LeaderState<T> implements EpochState {
this.hasAcknowledgedLeader = hasAcknowledgedLeader; 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( void updateLeaderEndOffset(
LogOffsetMetadata endOffsetMetadata LogOffsetMetadata endOffsetMetadata
) { ) {
@ -571,7 +624,7 @@ public class LeaderState<T> implements EpochState {
@Override @Override
public int compareTo(ReplicaState that) { public int compareTo(ReplicaState that) {
if (this.endOffset.equals(that.endOffset)) if (this.endOffset.equals(that.endOffset))
return Integer.compare(this.nodeId, that.nodeId); return this.replicaKey.compareTo(that.replicaKey);
else if (!this.endOffset.isPresent()) else if (!this.endOffset.isPresent())
return 1; return 1;
else if (!that.endOffset.isPresent()) else if (!that.endOffset.isPresent())
@ -583,10 +636,9 @@ public class LeaderState<T> implements EpochState {
@Override @Override
public String toString() { public String toString() {
return String.format( return String.format(
"ReplicaState(nodeId=%d, nodeDirectoryId=%s, endOffset=%s, lastFetchTimestamp=%s, " + "ReplicaState(replicaKey=%s, endOffset=%s, lastFetchTimestamp=%s, " +
"lastCaughtUpTimestamp=%s, hasAcknowledgedLeader=%s)", "lastCaughtUpTimestamp=%s, hasAcknowledgedLeader=%s)",
nodeId, replicaKey,
nodeDirectory,
endOffset, endOffset,
lastFetchTimestamp, lastFetchTimestamp,
lastCaughtUpTimestamp, lastCaughtUpTimestamp,
@ -608,9 +660,8 @@ public class LeaderState<T> implements EpochState {
@Override @Override
public String toString() { public String toString() {
return String.format( return String.format(
"Leader(localId=%d, localDirectoryId=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)", "Leader(localReplicaKey=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)",
localId, localReplicaKey,
localDirectoryId,
epoch, epoch,
epochStartOffset, epochStartOffset,
highWatermark, highWatermark,
@ -627,5 +678,4 @@ public class LeaderState<T> implements EpochState {
public void close() { public void close() {
accumulator.close(); accumulator.close();
} }
} }

View File

@ -16,9 +16,7 @@
*/ */
package org.apache.kafka.raft; package org.apache.kafka.raft;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.Uuid; 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.LogContext;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.BatchAccumulator;
@ -84,9 +82,9 @@ public class QuorumState {
private final Time time; private final Time time;
private final Logger log; private final Logger log;
private final QuorumStateStore store; private final QuorumStateStore store;
private final ListenerName listenerName;
private final Supplier<VoterSet> latestVoterSet; private final Supplier<VoterSet> latestVoterSet;
private final Supplier<Short> latestKraftVersion; private final Supplier<Short> latestKraftVersion;
private final Endpoints localListeners;
private final Random random; private final Random random;
private final int electionTimeoutMs; private final int electionTimeoutMs;
private final int fetchTimeoutMs; private final int fetchTimeoutMs;
@ -97,9 +95,9 @@ public class QuorumState {
public QuorumState( public QuorumState(
OptionalInt localId, OptionalInt localId,
Uuid localDirectoryId, Uuid localDirectoryId,
ListenerName listenerName,
Supplier<VoterSet> latestVoterSet, Supplier<VoterSet> latestVoterSet,
Supplier<Short> latestKraftVersion, Supplier<Short> latestKraftVersion,
Endpoints localListeners,
int electionTimeoutMs, int electionTimeoutMs,
int fetchTimeoutMs, int fetchTimeoutMs,
QuorumStateStore store, QuorumStateStore store,
@ -109,9 +107,9 @@ public class QuorumState {
) { ) {
this.localId = localId; this.localId = localId;
this.localDirectoryId = localDirectoryId; this.localDirectoryId = localDirectoryId;
this.listenerName = listenerName;
this.latestVoterSet = latestVoterSet; this.latestVoterSet = latestVoterSet;
this.latestKraftVersion = latestKraftVersion; this.latestKraftVersion = latestKraftVersion;
this.localListeners = localListeners;
this.electionTimeoutMs = electionTimeoutMs; this.electionTimeoutMs = electionTimeoutMs;
this.fetchTimeoutMs = fetchTimeoutMs; this.fetchTimeoutMs = fetchTimeoutMs;
this.store = store; this.store = store;
@ -174,11 +172,12 @@ public class QuorumState {
latestVoterSet.get().voterIds(), latestVoterSet.get().voterIds(),
randomElectionTimeoutMs(), randomElectionTimeoutMs(),
Collections.emptyList(), Collections.emptyList(),
localListeners,
logContext logContext
); );
} else if ( } else if (
localId.isPresent() && localId.isPresent() &&
election.isVotedCandidate(ReplicaKey.of(localId.getAsInt(), Optional.of(localDirectoryId))) election.isVotedCandidate(ReplicaKey.of(localId.getAsInt(), localDirectoryId))
) { ) {
initialState = new CandidateState( initialState = new CandidateState(
time, time,
@ -202,27 +201,13 @@ public class QuorumState {
logContext logContext
); );
} else if (election.hasLeader()) { } else if (election.hasLeader()) {
/* KAFKA-16529 is going to change this so that the leader is not required to be in the set VoterSet voters = latestVoterSet.get();
* 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()
)
)
);
initialState = new FollowerState( initialState = new FollowerState(
time, time,
election.epoch(), election.epoch(),
leader, election.leaderId(),
latestVoterSet.get().voterIds(), voters.listeners(election.leaderId()),
voters.voterIds(),
Optional.empty(), Optional.empty(),
fetchTimeoutMs, fetchTimeoutMs,
logContext logContext
@ -243,9 +228,7 @@ public class QuorumState {
public boolean isOnlyVoter() { public boolean isOnlyVoter() {
return localId.isPresent() && return localId.isPresent() &&
latestVoterSet.get().isOnlyVoter( latestVoterSet.get().isOnlyVoter(ReplicaKey.of(localId.getAsInt(), localDirectoryId));
ReplicaKey.of(localId.getAsInt(), Optional.of(localDirectoryId))
);
} }
public int localIdOrSentinel() { public int localIdOrSentinel() {
@ -277,7 +260,6 @@ public class QuorumState {
} }
public OptionalInt leaderId() { public OptionalInt leaderId() {
ElectionState election = state.election(); ElectionState election = state.election();
if (election.hasLeader()) if (election.hasLeader())
return OptionalInt.of(state.election().leaderId()); return OptionalInt.of(state.election().leaderId());
@ -293,6 +275,10 @@ public class QuorumState {
return hasLeader() && leaderIdOrSentinel() != localIdOrSentinel(); return hasLeader() && leaderIdOrSentinel() != localIdOrSentinel();
} }
public Endpoints leaderEndpoints() {
return state.leaderEndpoints();
}
public boolean isVoter() { public boolean isVoter() {
if (!localId.isPresent()) { if (!localId.isPresent()) {
return false; return false;
@ -300,7 +286,7 @@ public class QuorumState {
return latestVoterSet return latestVoterSet
.get() .get()
.isVoter(ReplicaKey.of(localId.getAsInt(), Optional.of(localDirectoryId))); .isVoter(ReplicaKey.of(localId.getAsInt(), localDirectoryId));
} }
public boolean isVoter(ReplicaKey nodeKey) { public boolean isVoter(ReplicaKey nodeKey) {
@ -327,6 +313,7 @@ public class QuorumState {
latestVoterSet.get().voterIds(), latestVoterSet.get().voterIds(),
randomElectionTimeoutMs(), randomElectionTimeoutMs(),
preferredSuccessors, preferredSuccessors,
localListeners,
logContext logContext
) )
); );
@ -427,16 +414,16 @@ public class QuorumState {
/** /**
* Become a follower of an elected leader so that we can begin fetching. * 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(); int currentEpoch = state.epoch();
if (localId.isPresent() && leader.id() == localId.getAsInt()) { if (localId.isPresent() && leaderId == localId.getAsInt()) {
throw new IllegalStateException("Cannot transition to Follower with leader " + leader + throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId +
" and epoch " + epoch + " since it matches the local broker.id " + localId); " and epoch " + epoch + " since it matches the local broker.id " + localId);
} else if (epoch < currentEpoch) { } 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"); " and epoch " + epoch + " since the current epoch " + currentEpoch + " is larger");
} else if (epoch == currentEpoch && (isFollower() || isLeader())) { } 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); " and epoch " + epoch + " from state " + state);
} }
@ -444,7 +431,8 @@ public class QuorumState {
new FollowerState( new FollowerState(
time, time,
epoch, epoch,
leader, leaderId,
endpoints,
latestVoterSet.get().voterIds(), latestVoterSet.get().voterIds(),
state.highWatermark(), state.highWatermark(),
fetchTimeoutMs, fetchTimeoutMs,
@ -518,13 +506,13 @@ public class QuorumState {
LeaderState<T> state = new LeaderState<>( LeaderState<T> state = new LeaderState<>(
time, time,
localIdOrThrow(), ReplicaKey.of(localIdOrThrow(), localDirectoryId),
localDirectoryId(),
epoch(), epoch(),
epochStartOffset, epochStartOffset,
latestVoterSet.get().voters(), latestVoterSet.get(),
candidateState.grantingVoters(), candidateState.grantingVoters(),
accumulator, accumulator,
localListeners,
fetchTimeoutMs, fetchTimeoutMs,
logContext logContext
); );

View File

@ -21,7 +21,5 @@ import org.apache.kafka.common.protocol.ApiMessage;
public interface RaftMessage { public interface RaftMessage {
int correlationId(); int correlationId();
short apiVersion();
ApiMessage data(); ApiMessage data();
} }

View File

@ -17,19 +17,18 @@
package org.apache.kafka.raft; package org.apache.kafka.raft;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.ApiMessage;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
public abstract class RaftRequest implements RaftMessage { public abstract class RaftRequest implements RaftMessage {
private final int correlationId; private final int correlationId;
private final short apiVersion;
private final ApiMessage data; private final ApiMessage data;
private final long createdTimeMs; 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.correlationId = correlationId;
this.apiVersion = apiVersion;
this.data = data; this.data = data;
this.createdTimeMs = createdTimeMs; this.createdTimeMs = createdTimeMs;
} }
@ -39,11 +38,6 @@ public abstract class RaftRequest implements RaftMessage {
return correlationId; return correlationId;
} }
@Override
public short apiVersion() {
return apiVersion;
}
@Override @Override
public ApiMessage data() { public ApiMessage data() {
return data; return data;
@ -54,17 +48,40 @@ public abstract class RaftRequest implements RaftMessage {
} }
public static final class Inbound extends RaftRequest { public static final class Inbound extends RaftRequest {
private final short apiVersion;
private final ListenerName listenerName;
public final CompletableFuture<RaftResponse.Outbound> completion = new CompletableFuture<>(); public final CompletableFuture<RaftResponse.Outbound> completion = new CompletableFuture<>();
public Inbound(int correlationId, short apiVersion, ApiMessage data, long createdTimeMs) { public Inbound(
super(correlationId, apiVersion, data, createdTimeMs); 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 @Override
public String toString() { public String toString() {
return String.format( return String.format(
"InboundRequest(correlationId=%d, data=%s, createdTimeMs=%d)", "InboundRequest(listenerName=%s, correlationId=%d, apiVersion=%d, data=%s, " +
"createdTimeMs=%d)",
listenerName,
correlationId(), correlationId(),
apiVersion,
data(), data(),
createdTimeMs() createdTimeMs()
); );
@ -75,8 +92,8 @@ public abstract class RaftRequest implements RaftMessage {
private final Node destination; private final Node destination;
public final CompletableFuture<RaftResponse.Inbound> completion = new CompletableFuture<>(); public final CompletableFuture<RaftResponse.Inbound> completion = new CompletableFuture<>();
public Outbound(int correlationId, short apiVersion, ApiMessage data, Node destination, long createdTimeMs) { public Outbound(int correlationId, ApiMessage data, Node destination, long createdTimeMs) {
super(correlationId, apiVersion, data, createdTimeMs); super(correlationId, data, createdTimeMs);
this.destination = destination; this.destination = destination;
} }

View File

@ -33,11 +33,6 @@ public abstract class RaftResponse implements RaftMessage {
return correlationId; return correlationId;
} }
@Override
public short apiVersion() {
return data().highestSupportedVersion();
}
@Override @Override
public ApiMessage data() { public ApiMessage data() {
return data; return data;

View File

@ -21,22 +21,30 @@ import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.BeginQuorumEpochRequestData; import org.apache.kafka.common.message.BeginQuorumEpochRequestData;
import org.apache.kafka.common.message.BeginQuorumEpochResponseData; import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
import org.apache.kafka.common.message.DescribeQuorumRequestData; 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.EndQuorumEpochRequestData;
import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.EndQuorumEpochResponseData;
import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchRequestData;
import org.apache.kafka.common.message.FetchResponseData; 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.FetchSnapshotResponseData;
import org.apache.kafka.common.message.VoteRequestData; import org.apache.kafka.common.message.VoteRequestData;
import org.apache.kafka.common.message.VoteResponseData; 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.ApiKeys;
import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.ApiMessage;
import org.apache.kafka.common.protocol.Errors; 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.Collections;
import java.util.List;
import java.util.Optional;
import java.util.function.Consumer; 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 class RaftUtil {
public static ApiMessage errorResponse(ApiKeys apiKey, Errors error) { public static ApiMessage errorResponse(ApiKeys apiKey, Errors error) {
@ -70,16 +78,20 @@ public class RaftUtil {
new FetchRequestData.FetchTopic() new FetchRequestData.FetchTopic()
.setTopic(topicPartition.topic()) .setTopic(topicPartition.topic())
.setTopicId(topicId) .setTopicId(topicId)
.setPartitions(singletonList(fetchPartition)); .setPartitions(Collections.singletonList(fetchPartition));
return new FetchRequestData() return new FetchRequestData()
.setTopics(singletonList(fetchTopic)); .setTopics(Collections.singletonList(fetchTopic));
} }
public static FetchResponseData singletonFetchResponse( public static FetchResponseData singletonFetchResponse(
ListenerName listenerName,
short apiVersion,
TopicPartition topicPartition, TopicPartition topicPartition,
Uuid topicId, Uuid topicId,
Errors topLevelError, Errors topLevelError,
int leaderId,
Endpoints endpoints,
Consumer<FetchResponseData.PartitionData> partitionConsumer Consumer<FetchResponseData.PartitionData> partitionConsumer
) { ) {
FetchResponseData.PartitionData fetchablePartition = FetchResponseData.PartitionData fetchablePartition =
@ -95,11 +107,421 @@ public class RaftUtil {
.setTopicId(topicId) .setTopicId(topicId)
.setPartitions(Collections.singletonList(fetchablePartition)); .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()) .setErrorCode(topLevelError.code())
.setResponses(Collections.singletonList(fetchableTopic)); .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) { static boolean hasValidTopicPartition(FetchRequestData data, TopicPartition topicPartition, Uuid topicId) {
return data.topics().size() == 1 && return data.topics().size() == 1 &&
data.topics().get(0).topicId().equals(topicId) && data.topics().get(0).topicId().equals(topicId) &&

View File

@ -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 * If there is a connection with a pending request it returns the amount of time to wait until
* the request times out. * 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. * ready.
* *
* If all of the bootstrap servers are backing off and there are no pending requests, return * If all of the bootstrap servers are backing off and there are no pending requests, return

View File

@ -44,6 +44,7 @@ import java.util.Set;
public class ResignedState implements EpochState { public class ResignedState implements EpochState {
private final int localId; private final int localId;
private final int epoch; private final int epoch;
private final Endpoints endpoints;
private final Set<Integer> voters; private final Set<Integer> voters;
private final long electionTimeoutMs; private final long electionTimeoutMs;
private final Set<Integer> unackedVoters; private final Set<Integer> unackedVoters;
@ -58,6 +59,7 @@ public class ResignedState implements EpochState {
Set<Integer> voters, Set<Integer> voters,
long electionTimeoutMs, long electionTimeoutMs,
List<Integer> preferredSuccessors, List<Integer> preferredSuccessors,
Endpoints endpoints,
LogContext logContext LogContext logContext
) { ) {
this.localId = localId; this.localId = localId;
@ -68,6 +70,7 @@ public class ResignedState implements EpochState {
this.electionTimeoutMs = electionTimeoutMs; this.electionTimeoutMs = electionTimeoutMs;
this.electionTimer = time.timer(electionTimeoutMs); this.electionTimer = time.timer(electionTimeoutMs);
this.preferredSuccessors = preferredSuccessors; this.preferredSuccessors = preferredSuccessors;
this.endpoints = endpoints;
this.log = logContext.logger(ResignedState.class); this.log = logContext.logger(ResignedState.class);
} }
@ -81,6 +84,11 @@ public class ResignedState implements EpochState {
return epoch; return epoch;
} }
@Override
public Endpoints leaderEndpoints() {
return endpoints;
}
/** /**
* Get the set of voters which have yet to acknowledge the resignation. * Get the set of voters which have yet to acknowledge the resignation.
* This node will send `EndQuorumEpoch` requests to this set until these * This node will send `EndQuorumEpoch` requests to this set until these

View File

@ -65,6 +65,11 @@ public class UnattachedState implements EpochState {
return epoch; return epoch;
} }
@Override
public Endpoints leaderEndpoints() {
return Endpoints.empty();
}
@Override @Override
public String name() { public String name() {
return "Unattached"; return "Unattached";

View File

@ -74,6 +74,11 @@ public class VotedState implements EpochState {
return epoch; return epoch;
} }
@Override
public Endpoints leaderEndpoints() {
return Endpoints.empty();
}
@Override @Override
public String name() { public String name() {
return "Voted"; return "Voted";

View File

@ -33,11 +33,6 @@ public class BlockingMessageQueue implements RaftMessageQueue {
return 0; return 0;
} }
@Override
public short apiVersion() {
return 0;
}
@Override @Override
public ApiMessage data() { public ApiMessage data() {
return null; return null;

View File

@ -21,7 +21,9 @@ import org.apache.kafka.common.Uuid;
import java.util.Objects; import java.util.Objects;
import java.util.Optional; 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 int id;
private final Optional<Uuid> directoryId; private final Optional<Uuid> directoryId;
@ -38,6 +40,18 @@ public final class ReplicaKey {
return directoryId; 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 @Override
public boolean equals(Object o) { public boolean equals(Object o) {
if (this == o) return true; if (this == o) return true;
@ -59,7 +73,10 @@ public final class ReplicaKey {
return String.format("ReplicaKey(id=%d, directoryId=%s)", id, directoryId); return String.format("ReplicaKey(id=%d, directoryId=%s)", id, directoryId);
} }
public static ReplicaKey of(int id, Optional<Uuid> directoryId) { public static ReplicaKey of(int id, Uuid directoryId) {
return new ReplicaKey(id, directoryId); return new ReplicaKey(
id,
directoryId.equals(NO_DIRECTORY_ID) ? Optional.empty() : Optional.of(directoryId)
);
} }
} }

View File

@ -22,6 +22,7 @@ import org.apache.kafka.common.feature.SupportedVersionRange;
import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.message.VotersRecord;
import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.raft.Endpoints;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.Collections; import java.util.Collections;
@ -97,28 +98,19 @@ public final class VoterSet {
/** /**
* Returns if the node is a voter in the set of voters. * 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. * directory id specified by the voter set.
* *
* If the voter set doesn't include the directory id ({@code Optional.empty()}), a node is in * 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. * 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 * @return true if the node is a voter in the voter set, otherwise false
*/ */
public boolean isVoter(ReplicaKey nodeKey) { public boolean isVoter(ReplicaKey replicaKey) {
VoterNode node = voters.get(nodeKey.id()); return Optional.ofNullable(voters.get(replicaKey.id()))
if (node != null) { .map(node -> node.isVoter(replicaKey))
if (node.voterKey().directoryId().isPresent()) { .orElse(false);
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;
}
} }
/** /**
@ -138,8 +130,39 @@ public final class VoterSet {
return voters.keySet(); 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 -> { Function<VoterNode, VotersRecord.Voter> voterConvertor = voter -> {
Iterator<VotersRecord.Endpoint> endpoints = voter Iterator<VotersRecord.Endpoint> endpoints = voter
.listeners() .listeners()
.entrySet() .votersRecordEndpoints();
.stream()
.map(entry ->
new VotersRecord.Endpoint()
.setName(entry.getKey().value())
.setHost(entry.getValue().getHostString())
.setPort(entry.getValue().getPort())
)
.iterator();
VotersRecord.KRaftVersionFeature kraftVersionFeature = new VotersRecord.KRaftVersionFeature() VotersRecord.KRaftVersionFeature kraftVersionFeature = new VotersRecord.KRaftVersionFeature()
.setMinSupportedVersion(voter.supportedKRaftVersion().min()) .setMinSupportedVersion(voter.supportedKRaftVersion().min())
@ -239,17 +254,8 @@ public final class VoterSet {
* @return true if they have an overlapping majority, false otherwise * @return true if they have an overlapping majority, false otherwise
*/ */
public boolean hasOverlappingMajority(VoterSet that) { public boolean hasOverlappingMajority(VoterSet that) {
Set<ReplicaKey> thisReplicaKeys = voters Set<ReplicaKey> thisReplicaKeys = voterKeys();
.values() Set<ReplicaKey> thatReplicaKeys = that.voterKeys();
.stream()
.map(VoterNode::voterKey)
.collect(Collectors.toSet());
Set<ReplicaKey> thatReplicaKeys = that.voters
.values()
.stream()
.map(VoterNode::voterKey)
.collect(Collectors.toSet());
if (Utils.diff(HashSet::new, thisReplicaKeys, thatReplicaKeys).size() > 1) return false; if (Utils.diff(HashSet::new, thisReplicaKeys, thatReplicaKeys).size() > 1) return false;
return Utils.diff(HashSet::new, thatReplicaKeys, thisReplicaKeys).size() <= 1; return Utils.diff(HashSet::new, thatReplicaKeys, thisReplicaKeys).size() <= 1;
@ -277,12 +283,12 @@ public final class VoterSet {
public static final class VoterNode { public static final class VoterNode {
private final ReplicaKey voterKey; private final ReplicaKey voterKey;
private final Map<ListenerName, InetSocketAddress> listeners; private final Endpoints listeners;
private final SupportedVersionRange supportedKRaftVersion; private final SupportedVersionRange supportedKRaftVersion;
public VoterNode( VoterNode(
ReplicaKey voterKey, ReplicaKey voterKey,
Map<ListenerName, InetSocketAddress> listeners, Endpoints listeners,
SupportedVersionRange supportedKRaftVersion SupportedVersionRange supportedKRaftVersion
) { ) {
this.voterKey = voterKey; this.voterKey = voterKey;
@ -294,7 +300,31 @@ public final class VoterSet {
return voterKey; 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; return listeners;
} }
@ -304,7 +334,7 @@ public final class VoterSet {
Optional<InetSocketAddress> address(ListenerName listener) { Optional<InetSocketAddress> address(ListenerName listener) {
return Optional.ofNullable(listeners.get(listener)); return listeners.address(listener);
} }
@Override @Override
@ -344,26 +374,11 @@ public final class VoterSet {
public static VoterSet fromVotersRecord(VotersRecord voters) { public static VoterSet fromVotersRecord(VotersRecord voters) {
HashMap<Integer, VoterNode> voterNodes = new HashMap<>(voters.voters().size()); HashMap<Integer, VoterNode> voterNodes = new HashMap<>(voters.voters().size());
for (VotersRecord.Voter voter: voters.voters()) { 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( voterNodes.put(
voter.voterId(), voter.voterId(),
new VoterNode( new VoterNode(
ReplicaKey.of(voter.voterId(), directoryId), ReplicaKey.of(voter.voterId(), voter.voterDirectoryId()),
listeners, Endpoints.fromVotersRecordEndpoints(voter.endpoints()),
new SupportedVersionRange( new SupportedVersionRange(
voter.kRaftVersionFeature().minSupportedVersion(), voter.kRaftVersionFeature().minSupportedVersion(),
voter.kRaftVersionFeature().maxSupportedVersion() voter.kRaftVersionFeature().maxSupportedVersion()
@ -390,8 +405,8 @@ public final class VoterSet {
Collectors.toMap( Collectors.toMap(
Map.Entry::getKey, Map.Entry::getKey,
entry -> new VoterNode( entry -> new VoterNode(
ReplicaKey.of(entry.getKey(), Optional.empty()), ReplicaKey.of(entry.getKey(), Uuid.ZERO_UUID),
Collections.singletonMap(listener, entry.getValue()), Endpoints.fromInetSocketAddresses(Collections.singletonMap(listener, entry.getValue())),
new SupportedVersionRange((short) 0, (short) 0) new SupportedVersionRange((short) 0, (short) 0)
) )
) )

View File

@ -204,10 +204,10 @@ public class CandidateStateTest {
voterSetWithLocal(IntStream.of(1, 2, 3)) voterSetWithLocal(IntStream.of(1, 2, 3))
); );
assertFalse(state.canGrantVote(ReplicaKey.of(0, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(0, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
} }
@Test @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) { private VoterSet voterSetWithLocal(IntStream remoteVoters) {
Map<Integer, VoterSet.VoterNode> voterMap = VoterSetTest.voterMap(remoteVoters, true); Map<Integer, VoterSet.VoterNode> voterMap = VoterSetTest.voterMap(remoteVoters, true);
voterMap.put(localNode.voterKey().id(), localNode); voterMap.put(localNode.voterKey().id(), localNode);

View File

@ -28,7 +28,6 @@ import org.junit.jupiter.params.provider.ValueSource;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertFalse;
@ -38,38 +37,38 @@ final class ElectionStateTest {
@Test @Test
void testVotedCandidateWithoutVotedId() { void testVotedCandidateWithoutVotedId() {
ElectionState electionState = ElectionState.withUnknownLeader(5, Collections.emptySet()); 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 @Test
void testVotedCandidateWithoutVotedDirectoryId() { void testVotedCandidateWithoutVotedDirectoryId() {
ElectionState electionState = ElectionState.withVotedCandidate( ElectionState electionState = ElectionState.withVotedCandidate(
5, 5,
ReplicaKey.of(1, Optional.empty()), ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID),
Collections.emptySet() Collections.emptySet()
); );
assertTrue(electionState.isVotedCandidate(ReplicaKey.of(1, Optional.empty()))); assertTrue(electionState.isVotedCandidate(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
assertTrue( assertTrue(
electionState.isVotedCandidate(ReplicaKey.of(1, Optional.of(Uuid.randomUuid()))) electionState.isVotedCandidate(ReplicaKey.of(1, Uuid.randomUuid()))
); );
} }
@Test @Test
void testVotedCandidateWithVotedDirectoryId() { void testVotedCandidateWithVotedDirectoryId() {
ReplicaKey votedKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); ReplicaKey votedKey = ReplicaKey.of(1, Uuid.randomUuid());
ElectionState electionState = ElectionState.withVotedCandidate( ElectionState electionState = ElectionState.withVotedCandidate(
5, 5,
votedKey, votedKey,
Collections.emptySet() Collections.emptySet()
); );
assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, Optional.empty()))); assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
assertTrue(electionState.isVotedCandidate(votedKey)); assertTrue(electionState.isVotedCandidate(votedKey));
} }
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
void testQuorumStateDataRoundTrip(short version) { 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( List<ElectionState> electionStates = Arrays.asList(
ElectionState.withUnknownLeader(5, Utils.mkSet(1, 2, 3)), ElectionState.withUnknownLeader(5, Utils.mkSet(1, 2, 3)),
ElectionState.withElectedLeader(5, 1, 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.withElectedLeader(5, 1, Utils.mkSet(1, 2, 3)),
ElectionState.withVotedCandidate( ElectionState.withVotedCandidate(
5, 5,
ReplicaKey.of(1, Optional.empty()), ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID),
Utils.mkSet(1, 2, 3) Utils.mkSet(1, 2, 3)
) )
); );

View File

@ -84,8 +84,7 @@ public class FileQuorumStateStoreTest {
final int epoch = 2; final int epoch = 2;
final int voter1 = 1; final int voter1 = 1;
final Optional<Uuid> voter1DirectoryId = Optional.of(Uuid.randomUuid()); final ReplicaKey voter1Key = ReplicaKey.of(voter1, Uuid.randomUuid());
final ReplicaKey voter1Key = ReplicaKey.of(voter1, voter1DirectoryId);
final int voter2 = 2; final int voter2 = 2;
final int voter3 = 3; final int voter3 = 3;
Set<Integer> voters = Utils.mkSet(voter1, voter2, voter3); Set<Integer> voters = Utils.mkSet(voter1, voter2, voter3);
@ -108,7 +107,7 @@ public class FileQuorumStateStoreTest {
expected = Optional.of( expected = Optional.of(
ElectionState.withVotedCandidate( ElectionState.withVotedCandidate(
epoch, epoch,
ReplicaKey.of(voter1, Optional.empty()), ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID),
voters voters
) )
); );

View File

@ -16,7 +16,7 @@
*/ */
package org.apache.kafka.raft; 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.LogContext;
import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils; 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.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource; import org.junit.jupiter.params.provider.ValueSource;
import java.net.InetSocketAddress;
import java.util.Collections;
import java.util.Optional; import java.util.Optional;
import java.util.OptionalLong; import java.util.OptionalLong;
import java.util.Set; import java.util.Set;
@ -40,7 +42,13 @@ public class FollowerStateTest {
private final LogContext logContext = new LogContext(); private final LogContext logContext = new LogContext();
private final int epoch = 5; private final int epoch = 5;
private final int fetchTimeoutMs = 15000; 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( private FollowerState newFollowerState(
Set<Integer> voters, Set<Integer> voters,
@ -49,7 +57,8 @@ public class FollowerStateTest {
return new FollowerState( return new FollowerState(
time, time,
epoch, epoch,
leader, leaderId,
leaderEndpoints,
voters, voters,
highWatermark, highWatermark,
fetchTimeoutMs, fetchTimeoutMs,
@ -93,15 +102,16 @@ public class FollowerStateTest {
Optional.empty() Optional.empty()
); );
assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
} }
@Test @Test
public void testLeaderNode() { public void testLeaderIdAndEndpoint() {
FollowerState state = newFollowerState(Utils.mkSet(0, 1, 2), Optional.empty()); FollowerState state = newFollowerState(Utils.mkSet(0, 1, 2), Optional.empty());
assertEquals(leader, state.leader()); assertEquals(leaderId, state.leaderId());
assertEquals(leaderEndpoints, state.leaderEndpoints());
} }
} }

View File

@ -233,7 +233,6 @@ public class KafkaNetworkChannelTest {
ApiMessage apiRequest = buildTestRequest(apiKey); ApiMessage apiRequest = buildTestRequest(apiKey);
RaftRequest.Outbound request = new RaftRequest.Outbound( RaftRequest.Outbound request = new RaftRequest.Outbound(
correlationId, correlationId,
apiRequest.highestSupportedVersion(),
apiRequest, apiRequest,
destination, destination,
createdTimeMs createdTimeMs
@ -322,7 +321,23 @@ public class KafkaNetworkChannelTest {
case END_QUORUM_EPOCH: case END_QUORUM_EPOCH:
return new EndQuorumEpochResponseData().setErrorCode(error.code()); return new EndQuorumEpochResponseData().setErrorCode(error.code());
case VOTE: 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: case FETCH:
return new FetchResponseData().setErrorCode(error.code()); return new FetchResponseData().setErrorCode(error.code());
case FETCH_SNAPSHOT: case FETCH_SNAPSHOT:

View File

@ -25,8 +25,8 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.record.UnalignedMemoryRecords;
import org.apache.kafka.common.requests.FetchSnapshotRequest; 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.common.utils.Utils;
import org.apache.kafka.raft.internals.ReplicaKey;
import org.apache.kafka.raft.internals.StringSerde; import org.apache.kafka.raft.internals.StringSerde;
import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotReader;
import org.apache.kafka.snapshot.RawSnapshotWriter; 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.api.Test;
import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource;
import org.junit.jupiter.params.provider.ValueSource; import org.junit.jupiter.params.provider.ValueSource;
import java.io.IOException; import java.io.IOException;
@ -93,14 +94,15 @@ public final class KafkaRaftClientSnapshotTest {
} }
@ParameterizedTest @ParameterizedTest
@ValueSource(booleans = {true, false}) @CsvSource({"false,false", "false,true", "true,false", "true,true"})
public void testLeaderListenerNotified(boolean entireLog) throws Exception { public void testLeaderListenerNotified(boolean entireLog, boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = localId + 1; ReplicaKey otherNodeKey = replicaKey(localId + 1, false);
Set<Integer> voters = Utils.mkSet(localId, otherNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters)
.withKip853Rpc(withKip853Rpc)
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
.withEmptySnapshot(snapshotId); .withEmptySnapshot(snapshotId);
@ -116,7 +118,7 @@ public final class KafkaRaftClientSnapshotTest {
// Advance the highWatermark // Advance the highWatermark
long localLogEndOffset = context.log.endOffset().offset; 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.pollUntilResponse();
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId));
assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong());
@ -216,14 +218,16 @@ public final class KafkaRaftClientSnapshotTest {
} }
} }
@Test @ParameterizedTest
public void testListenerRenotified() throws Exception { @ValueSource(booleans = {false, true})
public void testListenerRenotified(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = localId + 1; ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
Set<Integer> voters = Utils.mkSet(localId, otherNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.withKip853Rpc(withKip853Rpc)
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
.appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i")) .appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i"))
@ -239,7 +243,7 @@ public final class KafkaRaftClientSnapshotTest {
// Advance the highWatermark // Advance the highWatermark
long localLogEndOffset = context.log.endOffset().offset; 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.pollUntilResponse();
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId));
assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong());
@ -270,15 +274,17 @@ public final class KafkaRaftClientSnapshotTest {
} }
} }
@Test @ParameterizedTest
public void testLeaderImmediatelySendsSnapshotId() throws Exception { @ValueSource(booleans = {false, true})
public void testLeaderImmediatelySendsSnapshotId(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = 1; ReplicaKey otherNodeKey = replicaKey(1, withKip853Rpc);
Set<Integer> voters = Utils.mkSet(localId, otherNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 4); OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 4);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.withUnknownLeader(snapshotId.epoch()) .withUnknownLeader(snapshotId.epoch())
.withKip853Rpc(withKip853Rpc)
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c")) .appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
.appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i")) .appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i"))
@ -290,7 +296,7 @@ public final class KafkaRaftClientSnapshotTest {
int epoch = context.currentEpoch(); int epoch = context.currentEpoch();
// Send a fetch request for an end offset and epoch which has been snapshotted // 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(); context.client.poll();
// Expect that the leader replies immediately with a snapshot id // Expect that the leader replies immediately with a snapshot id
@ -302,14 +308,16 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset()); assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset());
} }
@Test @ParameterizedTest
public void testFetchRequestOffsetLessThanLogStart() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = localId + 1; ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
Set<Integer> voters = Utils.mkSet(localId, otherNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.withAppendLingerMs(1) .withAppendLingerMs(1)
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -338,7 +346,7 @@ public final class KafkaRaftClientSnapshotTest {
context.client.poll(); context.client.poll();
// Send Fetch request less than start offset // 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(); context.pollUntilResponse();
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
@ -348,15 +356,17 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset()); assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset());
} }
@Test @ParameterizedTest
public void testFetchRequestOffsetAtZero() throws Exception { @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 // When the follower sends a FETCH request at offset 0, reply with snapshot id if it exists
int localId = 0; int localId = 0;
int otherNodeId = localId + 1; ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
Set<Integer> voters = Utils.mkSet(localId, otherNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.withAppendLingerMs(1) .withAppendLingerMs(1)
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -384,7 +394,7 @@ public final class KafkaRaftClientSnapshotTest {
} }
// Send Fetch request for offset 0 // 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(); context.pollUntilResponse();
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
@ -394,11 +404,12 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset()); assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset());
} }
@Test @ParameterizedTest
public void testFetchRequestWithLargerLastFetchedEpoch() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = localId + 1; ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
Set<Integer> voters = Utils.mkSet(localId, otherNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); 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("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f"))
.withAppendLingerMs(1) .withAppendLingerMs(1)
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -427,17 +439,18 @@ public final class KafkaRaftClientSnapshotTest {
context.client.poll(); context.client.poll();
// It is an invalid request to send an last fetched epoch greater than the current epoch // 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.pollUntilResponse();
context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId)); context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId));
} }
@Test @ParameterizedTest
public void testFetchRequestTruncateToLogStart() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchRequestTruncateToLogStart(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = localId + 1; ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
int syncNodeId = otherNodeId + 1; int syncNodeId = otherNodeKey.id() + 1;
Set<Integer> voters = Utils.mkSet(localId, otherNodeId, syncNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); 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(), Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("d", "e", "f")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("d", "e", "f"))
.withAppendLingerMs(1) .withAppendLingerMs(1)
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -463,7 +477,13 @@ public final class KafkaRaftClientSnapshotTest {
// This should truncate to the old snapshot // This should truncate to the old snapshot
context.deliverRequest( 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(); context.pollUntilResponse();
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
@ -474,12 +494,13 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(oldestSnapshotId.offset(), partitionResponse.divergingEpoch().endOffset()); assertEquals(oldestSnapshotId.offset(), partitionResponse.divergingEpoch().endOffset());
} }
@Test @ParameterizedTest
public void testFetchRequestAtLogStartOffsetWithValidEpoch() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchRequestAtLogStartOffsetWithValidEpoch(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = localId + 1; ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
int syncNodeId = otherNodeId + 1; int syncNodeId = otherNodeKey.id() + 1;
Set<Integer> voters = Utils.mkSet(localId, otherNodeId, syncNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); 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(), Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i"))
.withAppendLingerMs(1) .withAppendLingerMs(1)
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -506,18 +528,25 @@ public final class KafkaRaftClientSnapshotTest {
// Send fetch request at log start offset with valid last fetched epoch // Send fetch request at log start offset with valid last fetched epoch
context.deliverRequest( context.deliverRequest(
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset(), oldestSnapshotId.epoch(), 0) context.fetchRequest(
epoch,
otherNodeKey,
oldestSnapshotId.offset(),
oldestSnapshotId.epoch(),
0
)
); );
context.pollUntilResponse(); context.pollUntilResponse();
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId));
} }
@Test @ParameterizedTest
public void testFetchRequestAtLogStartOffsetWithInvalidEpoch() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = localId + 1; ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
int syncNodeId = otherNodeId + 1; int syncNodeId = otherNodeKey.id() + 1;
Set<Integer> voters = Utils.mkSet(localId, otherNodeId, syncNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); 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(), Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i"))
.withAppendLingerMs(1) .withAppendLingerMs(1)
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -545,7 +575,13 @@ public final class KafkaRaftClientSnapshotTest {
// Send fetch with log start offset and invalid last fetched epoch // Send fetch with log start offset and invalid last fetched epoch
context.deliverRequest( context.deliverRequest(
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset(), oldestSnapshotId.epoch() + 1, 0) context.fetchRequest(
epoch,
otherNodeKey,
oldestSnapshotId.offset(),
oldestSnapshotId.epoch() + 1,
0
)
); );
context.pollUntilResponse(); context.pollUntilResponse();
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
@ -556,12 +592,15 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset()); assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset());
} }
@Test @ParameterizedTest
public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot(
boolean withKip853Rpc
) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = localId + 1; ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
int syncNodeId = otherNodeId + 1; int syncNodeId = otherNodeKey.id() + 1;
Set<Integer> voters = Utils.mkSet(localId, otherNodeId, syncNodeId); Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2); 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(), Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i")) .appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i"))
.withAppendLingerMs(1) .withAppendLingerMs(1)
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -590,7 +630,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverRequest( context.deliverRequest(
context.fetchRequest( context.fetchRequest(
epoch, epoch,
otherNodeId, otherNodeKey,
context.log.endOffset().offset, context.log.endOffset().offset,
oldestSnapshotId.epoch() - 1, oldestSnapshotId.epoch() - 1,
0 0
@ -605,13 +645,19 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset()); assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset());
} }
@Test @ParameterizedTest
public void testFetchSnapshotRequestMissingSnapshot() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchSnapshotRequestMissingSnapshot(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
int epoch = 2;
Set<Integer> voters = Utils.mkSet(localId, localId + 1); 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( context.deliverRequest(
fetchSnapshotRequest( fetchSnapshotRequest(
@ -629,14 +675,20 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(Errors.SNAPSHOT_NOT_FOUND, Errors.forCode(response.errorCode())); assertEquals(Errors.SNAPSHOT_NOT_FOUND, Errors.forCode(response.errorCode()));
} }
@Test @ParameterizedTest
public void testFetchSnapshotRequestUnknownPartition() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchSnapshotRequestUnknownPartition(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
Set<Integer> voters = Utils.mkSet(localId, localId + 1); Set<Integer> voters = Utils.mkSet(localId, localId + 1);
int epoch = 2;
TopicPartition topicPartition = new TopicPartition("unknown", 0); 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( context.deliverRequest(
fetchSnapshotRequest( fetchSnapshotRequest(
@ -654,8 +706,9 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.forCode(response.errorCode())); assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.forCode(response.errorCode()));
} }
@Test @ParameterizedTest
public void testFetchSnapshotRequestAsLeader() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
Set<Integer> voters = Utils.mkSet(localId, localId + 1); Set<Integer> voters = Utils.mkSet(localId, localId + 1);
OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1);
@ -663,6 +716,7 @@ public final class KafkaRaftClientSnapshotTest {
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch(), Collections.singletonList("a")) .appendToLog(snapshotId.epoch(), Collections.singletonList("a"))
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -703,18 +757,22 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(memoryRecords.buffer(), ((UnalignedMemoryRecords) response.unalignedRecords()).buffer()); assertEquals(memoryRecords.buffer(), ((UnalignedMemoryRecords) response.unalignedRecords()).buffer());
} }
@Test @ParameterizedTest
public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajorityVoters() throws Exception { @ValueSource(booleans = {false, true})
public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajorityVoters(
boolean withKip853Rpc
) throws Exception {
int localId = 0; int localId = 0;
int voter1 = 1; ReplicaKey voter1 = replicaKey(1, withKip853Rpc);
int voter2 = 2; ReplicaKey voter2 = replicaKey(2, withKip853Rpc);
int observerId3 = 3; ReplicaKey observer3 = replicaKey(3, withKip853Rpc);
Set<Integer> voters = Utils.mkSet(localId, voter1, voter2); Set<Integer> voters = Utils.mkSet(localId, voter1.id(), voter2.id());
OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1);
List<String> records = Arrays.asList("foo", "bar"); List<String> records = Arrays.asList("foo", "bar");
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch(), Collections.singletonList("a")) .appendToLog(snapshotId.epoch(), Collections.singletonList("a"))
.withKip853Rpc(withKip853Rpc)
.build(); .build();
int resignLeadershipTimeout = context.checkQuorumTimeoutMs; int resignLeadershipTimeout = context.checkQuorumTimeoutMs;
@ -743,7 +801,7 @@ public final class KafkaRaftClientSnapshotTest {
FetchSnapshotRequestData observerFetchSnapshotRequest = fetchSnapshotRequest( FetchSnapshotRequestData observerFetchSnapshotRequest = fetchSnapshotRequest(
context.clusterId.toString(), context.clusterId.toString(),
observerId3, observer3,
context.metadataPartition, context.metadataPartition,
epoch, epoch,
snapshotId, snapshotId,
@ -794,8 +852,9 @@ public final class KafkaRaftClientSnapshotTest {
assertTrue(context.client.quorum().isResigned()); assertTrue(context.client.quorum().isResigned());
} }
@Test @ParameterizedTest
public void testPartialFetchSnapshotRequestAsLeader() throws Exception { @ValueSource(booleans = {false, true})
public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
Set<Integer> voters = Utils.mkSet(localId, localId + 1); Set<Integer> voters = Utils.mkSet(localId, localId + 1);
OffsetAndEpoch snapshotId = new OffsetAndEpoch(2, 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(2, 1);
@ -803,6 +862,7 @@ public final class KafkaRaftClientSnapshotTest {
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch(), records) .appendToLog(snapshotId.epoch(), records)
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -873,8 +933,9 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(snapshotBuffer, responseBuffer.flip()); assertEquals(snapshotBuffer, responseBuffer.flip());
} }
@Test @ParameterizedTest
public void testFetchSnapshotRequestAsFollower() throws IOException { @ValueSource(booleans = {false, true})
public void testFetchSnapshotRequestAsFollower(boolean withKip853Rpc) throws IOException {
int localId = 0; int localId = 0;
int leaderId = localId + 1; int leaderId = localId + 1;
Set<Integer> voters = Utils.mkSet(localId, leaderId); Set<Integer> voters = Utils.mkSet(localId, leaderId);
@ -883,6 +944,7 @@ public final class KafkaRaftClientSnapshotTest {
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.withElectedLeader(epoch, leaderId) .withElectedLeader(epoch, leaderId)
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.deliverRequest( context.deliverRequest(
@ -903,8 +965,9 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(leaderId, response.currentLeader().leaderId()); assertEquals(leaderId, response.currentLeader().leaderId());
} }
@Test @ParameterizedTest
public void testFetchSnapshotRequestWithInvalidPosition() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
Set<Integer> voters = Utils.mkSet(localId, localId + 1); Set<Integer> voters = Utils.mkSet(localId, localId + 1);
OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1); OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1);
@ -912,6 +975,7 @@ public final class KafkaRaftClientSnapshotTest {
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch(), Collections.singletonList("a")) .appendToLog(snapshotId.epoch(), Collections.singletonList("a"))
.withKip853Rpc(withKip853Rpc)
.build(); .build();
context.becomeLeader(); context.becomeLeader();
@ -961,14 +1025,20 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(localId, response.currentLeader().leaderId()); assertEquals(localId, response.currentLeader().leaderId());
} }
@Test @ParameterizedTest
public void testFetchSnapshotRequestWithOlderEpoch() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchSnapshotRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
Set<Integer> voters = Utils.mkSet(localId, localId + 1); Set<Integer> voters = Utils.mkSet(localId, localId + 1);
int epoch = 2;
OffsetAndEpoch snapshotId = new OffsetAndEpoch(0, 0); 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( context.deliverRequest(
fetchSnapshotRequest( fetchSnapshotRequest(
@ -988,14 +1058,20 @@ public final class KafkaRaftClientSnapshotTest {
assertEquals(localId, response.currentLeader().leaderId()); assertEquals(localId, response.currentLeader().leaderId());
} }
@Test @ParameterizedTest
public void testFetchSnapshotRequestWithNewerEpoch() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchSnapshotRequestWithNewerEpoch(boolean withKip853Rpc) throws Exception {
int localId = 0; int localId = 0;
Set<Integer> voters = Utils.mkSet(localId, localId + 1); Set<Integer> voters = Utils.mkSet(localId, localId + 1);
int epoch = 2;
OffsetAndEpoch snapshotId = new OffsetAndEpoch(0, 0); 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( context.deliverRequest(
fetchSnapshotRequest( fetchSnapshotRequest(
@ -1036,7 +1112,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, invalidEpoch, 200L) context.snapshotFetchResponse(epoch, leaderId, invalidEpoch, 200L)
); );
// Handle the invalid response // Handle the invalid response
@ -1053,7 +1129,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, invalidEndOffset, 200L) context.snapshotFetchResponse(epoch, leaderId, invalidEndOffset, 200L)
); );
// Handle the invalid response // Handle the invalid response
@ -1095,7 +1171,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
); );
context.pollUntilRequest(); context.pollUntilRequest();
@ -1121,7 +1197,7 @@ public final class KafkaRaftClientSnapshotTest {
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
fetchSnapshotResponse( fetchSnapshotResponse(
context.metadataPartition, context,
epoch, epoch,
leaderId, leaderId,
snapshotId, snapshotId,
@ -1166,7 +1242,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
); );
context.pollUntilRequest(); context.pollUntilRequest();
@ -1195,7 +1271,7 @@ public final class KafkaRaftClientSnapshotTest {
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
fetchSnapshotResponse( fetchSnapshotResponse(
context.metadataPartition, context,
epoch, epoch,
leaderId, leaderId,
snapshotId, snapshotId,
@ -1224,7 +1300,7 @@ public final class KafkaRaftClientSnapshotTest {
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
fetchSnapshotResponse( fetchSnapshotResponse(
context.metadataPartition, context,
epoch, epoch,
leaderId, leaderId,
snapshotId, snapshotId,
@ -1269,7 +1345,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
); );
context.pollUntilRequest(); context.pollUntilRequest();
@ -1288,8 +1364,8 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
FetchSnapshotResponse.singleton( context.fetchSnapshotResponse(
context.metadataPartition, leaderId,
responsePartitionSnapshot -> { responsePartitionSnapshot -> {
responsePartitionSnapshot responsePartitionSnapshot
.currentLeader() .currentLeader()
@ -1327,7 +1403,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, firstLeaderId, snapshotId, 200L) context.snapshotFetchResponse(epoch, firstLeaderId, snapshotId, 200L)
); );
context.pollUntilRequest(); context.pollUntilRequest();
@ -1346,8 +1422,8 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
FetchSnapshotResponse.singleton( context.fetchSnapshotResponse(
context.metadataPartition, secondLeaderId,
responsePartitionSnapshot -> { responsePartitionSnapshot -> {
responsePartitionSnapshot responsePartitionSnapshot
.currentLeader() .currentLeader()
@ -1384,7 +1460,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
); );
context.pollUntilRequest(); context.pollUntilRequest();
@ -1403,8 +1479,8 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
FetchSnapshotResponse.singleton( context.fetchSnapshotResponse(
context.metadataPartition, leaderId,
responsePartitionSnapshot -> { responsePartitionSnapshot -> {
responsePartitionSnapshot responsePartitionSnapshot
.currentLeader() .currentLeader()
@ -1441,7 +1517,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
); );
context.pollUntilRequest(); context.pollUntilRequest();
@ -1460,8 +1536,8 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
FetchSnapshotResponse.singleton( context.fetchSnapshotResponse(
context.metadataPartition, leaderId + 1,
responsePartitionSnapshot -> { responsePartitionSnapshot -> {
responsePartitionSnapshot responsePartitionSnapshot
.currentLeader() .currentLeader()
@ -1508,7 +1584,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
); );
context.pollUntilRequest(); context.pollUntilRequest();
@ -1527,8 +1603,8 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
FetchSnapshotResponse.singleton( context.fetchSnapshotResponse(
context.metadataPartition, leaderId,
responsePartitionSnapshot -> { responsePartitionSnapshot -> {
responsePartitionSnapshot responsePartitionSnapshot
.currentLeader() .currentLeader()
@ -1554,7 +1630,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
); );
context.pollUntilRequest(); context.pollUntilRequest();
@ -1574,8 +1650,8 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
FetchSnapshotResponse.singleton( context.fetchSnapshotResponse(
context.metadataPartition, leaderId,
responsePartitionSnapshot -> { responsePartitionSnapshot -> {
responsePartitionSnapshot responsePartitionSnapshot
.currentLeader() .currentLeader()
@ -1618,7 +1694,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
fetchRequest.correlationId(), fetchRequest.correlationId(),
fetchRequest.destination(), fetchRequest.destination(),
snapshotFetchResponse(context.metadataPartition, context.metadataTopicId, epoch, leaderId, snapshotId, 200L) context.snapshotFetchResponse(epoch, leaderId, snapshotId, 200L)
); );
context.pollUntilRequest(); context.pollUntilRequest();
@ -1646,8 +1722,8 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverResponse( context.deliverResponse(
snapshotRequest.correlationId(), snapshotRequest.correlationId(),
snapshotRequest.destination(), snapshotRequest.destination(),
FetchSnapshotResponse.singleton( context.fetchSnapshotResponse(
context.metadataPartition, leaderId,
responsePartitionSnapshot -> { responsePartitionSnapshot -> {
responsePartitionSnapshot responsePartitionSnapshot
.currentLeader() .currentLeader()
@ -1669,20 +1745,28 @@ public final class KafkaRaftClientSnapshotTest {
context.assertVotedCandidate(epoch + 1, localId); context.assertVotedCandidate(epoch + 1, localId);
} }
@Test @ParameterizedTest
public void testFetchSnapshotRequestClusterIdValidation() throws Exception { @ValueSource(booleans = {false, true})
public void testFetchSnapshotRequestClusterIdValidation(
boolean withKip853Rpc
) throws Exception {
int localId = 0; int localId = 0;
int otherNodeId = 1; ReplicaKey otherNode = replicaKey(1, withKip853Rpc);
int epoch = 5; Set<Integer> voters = Utils.mkSet(localId, otherNode.id());
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
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( context.deliverRequest(
fetchSnapshotRequest( fetchSnapshotRequest(
context.clusterId.toString(), context.clusterId.toString(),
otherNodeId, otherNode,
context.metadataPartition, context.metadataPartition,
epoch, epoch,
new OffsetAndEpoch(0, 0), new OffsetAndEpoch(0, 0),
@ -1697,7 +1781,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverRequest( context.deliverRequest(
fetchSnapshotRequest( fetchSnapshotRequest(
null, null,
otherNodeId, otherNode,
context.metadataPartition, context.metadataPartition,
epoch, epoch,
new OffsetAndEpoch(0, 0), new OffsetAndEpoch(0, 0),
@ -1712,7 +1796,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverRequest( context.deliverRequest(
fetchSnapshotRequest( fetchSnapshotRequest(
"", "",
otherNodeId, otherNode,
context.metadataPartition, context.metadataPartition,
epoch, epoch,
new OffsetAndEpoch(0, 0), new OffsetAndEpoch(0, 0),
@ -1727,7 +1811,7 @@ public final class KafkaRaftClientSnapshotTest {
context.deliverRequest( context.deliverRequest(
fetchSnapshotRequest( fetchSnapshotRequest(
"invalid-uuid", "invalid-uuid",
otherNodeId, otherNode,
context.metadataPartition, context.metadataPartition,
epoch, epoch,
new OffsetAndEpoch(0, 0), new OffsetAndEpoch(0, 0),
@ -1852,6 +1936,11 @@ public final class KafkaRaftClientSnapshotTest {
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4, 0)); 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( private static FetchSnapshotRequestData fetchSnapshotRequest(
TopicPartition topicPartition, TopicPartition topicPartition,
int epoch, int epoch,
@ -1859,37 +1948,39 @@ public final class KafkaRaftClientSnapshotTest {
int maxBytes, int maxBytes,
long position 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( private static FetchSnapshotRequestData fetchSnapshotRequest(
String clusterId, String clusterId,
int replicaId, ReplicaKey replicaKey,
TopicPartition topicPartition, TopicPartition topicPartition,
int epoch, int epoch,
OffsetAndEpoch offsetAndEpoch, OffsetAndEpoch offsetAndEpoch,
int maxBytes, int maxBytes,
long position long position
) { ) {
FetchSnapshotRequestData.SnapshotId snapshotId = new FetchSnapshotRequestData.SnapshotId() return RaftUtil.singletonFetchSnapshotRequest(
.setEndOffset(offsetAndEpoch.offset())
.setEpoch(offsetAndEpoch.epoch());
FetchSnapshotRequestData request = FetchSnapshotRequest.singleton(
clusterId, clusterId,
replicaId, replicaKey,
topicPartition, topicPartition,
snapshotPartition -> snapshotPartition epoch,
.setCurrentLeaderEpoch(epoch) offsetAndEpoch,
.setSnapshotId(snapshotId) maxBytes,
.setPosition(position) position
); );
return request.setMaxBytes(maxBytes);
} }
private static FetchSnapshotResponseData fetchSnapshotResponse( private static FetchSnapshotResponseData fetchSnapshotResponse(
TopicPartition topicPartition, RaftClientTestContext context,
int leaderEpoch, int leaderEpoch,
int leaderId, int leaderId,
OffsetAndEpoch snapshotId, OffsetAndEpoch snapshotId,
@ -1897,8 +1988,8 @@ public final class KafkaRaftClientSnapshotTest {
long position, long position,
ByteBuffer buffer ByteBuffer buffer
) { ) {
return FetchSnapshotResponse.singleton( return context.fetchSnapshotResponse(
topicPartition, leaderId,
partitionSnapshot -> { partitionSnapshot -> {
partitionSnapshot.currentLeader() partitionSnapshot.currentLeader()
.setLeaderEpoch(leaderEpoch) .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( private static Optional<FetchSnapshotRequestData.PartitionSnapshot> assertFetchSnapshotRequest(
RaftRequest.Outbound request, RaftRequest.Outbound request,
TopicPartition topicPartition, TopicPartition topicPartition,

File diff suppressed because it is too large Load Diff

View File

@ -16,7 +16,6 @@
*/ */
package org.apache.kafka.raft; package org.apache.kafka.raft;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.MockTime;
@ -31,6 +30,7 @@ import org.junit.jupiter.params.provider.ValueSource;
import org.mockito.Mockito; import org.mockito.Mockito;
import java.io.UncheckedIOException; import java.io.UncheckedIOException;
import java.net.InetSocketAddress;
import java.util.Collections; import java.util.Collections;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
@ -48,10 +48,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class QuorumStateTest { public class QuorumStateTest {
private final int localId = 0; private final int localId = 0;
private final Uuid localDirectoryId = Uuid.randomUuid(); private final Uuid localDirectoryId = Uuid.randomUuid();
private final ReplicaKey localVoterKey = ReplicaKey.of( private final ReplicaKey localVoterKey = ReplicaKey.of(localId, localDirectoryId);
localId,
Optional.of(localDirectoryId)
);
private final int logEndEpoch = 0; private final int logEndEpoch = 0;
private final MockQuorumStateStore store = new MockQuorumStateStore(); private final MockQuorumStateStore store = new MockQuorumStateStore();
private final MockTime time = new MockTime(); private final MockTime time = new MockTime();
@ -68,9 +65,9 @@ public class QuorumStateTest {
return new QuorumState( return new QuorumState(
localId, localId,
localDirectoryId, localDirectoryId,
VoterSetTest.DEFAULT_LISTENER_NAME,
() -> voterSet, () -> voterSet,
() -> kraftVersion, () -> kraftVersion,
localId.isPresent() ? voterSet.listeners(localId.getAsInt()) : Endpoints.empty(),
electionTimeoutMs, electionTimeoutMs,
fetchTimeoutMs, fetchTimeoutMs,
store, store,
@ -100,7 +97,7 @@ public class QuorumStateTest {
return replicaKey; return replicaKey;
} }
return ReplicaKey.of(replicaKey.id(), Optional.empty()); return ReplicaKey.of(replicaKey.id(), ReplicaKey.NO_DIRECTORY_ID);
} }
private VoterSet localStandaloneVoterSet() { private VoterSet localStandaloneVoterSet() {
@ -112,7 +109,11 @@ public class QuorumStateTest {
private VoterSet localWithRemoteVoterSet(IntStream remoteIds, short kraftVersion) { private VoterSet localWithRemoteVoterSet(IntStream remoteIds, short kraftVersion) {
boolean withDirectoryId = kraftVersion > 0; boolean withDirectoryId = kraftVersion > 0;
Map<Integer, VoterSet.VoterNode> voters = VoterSetTest.voterMap(remoteIds, withDirectoryId); Map<Integer, VoterSet.VoterNode> voters = VoterSetTest.voterMap(remoteIds, withDirectoryId);
if (withDirectoryId) {
voters.put(localId, VoterSetTest.voterNode(localVoterKey)); voters.put(localId, VoterSetTest.voterNode(localVoterKey));
} else {
voters.put(localId, VoterSetTest.voterNode(ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)));
}
return VoterSetTest.voterSet(voters); return VoterSetTest.voterSet(voters);
} }
@ -170,15 +171,15 @@ public class QuorumStateTest {
FollowerState followerState = state.followerStateOrThrow(); FollowerState followerState = state.followerStateOrThrow();
assertEquals(epoch, followerState.epoch()); assertEquals(epoch, followerState.epoch());
assertEquals(node1, followerState.leader().id()); assertEquals(node1, followerState.leaderId());
assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds()));
} }
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testInitializeAsVoted(short kraftVersion) { public void testInitializeAsVoted(short kraftVersion) {
ReplicaKey nodeKey1 = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid());
ReplicaKey nodeKey2 = ReplicaKey.of(2, Optional.of(Uuid.randomUuid())); ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid());
int epoch = 5; int epoch = 5;
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2));
@ -379,7 +380,7 @@ public class QuorumStateTest {
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToCandidate(); 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(5, state.epoch());
assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals(OptionalInt.of(otherNodeId), state.leaderId());
assertEquals( assertEquals(
@ -420,7 +421,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testCandidateToVoted(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -447,7 +448,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testCandidateToAnyStateLowerEpoch(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -459,7 +460,8 @@ public class QuorumStateTest {
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToFollower( () -> state.transitionToFollower(
4, 4,
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeKey.id(),
voters.listeners(otherNodeKey.id())
) )
); );
assertEquals(6, state.epoch()); assertEquals(6, state.epoch());
@ -546,7 +548,7 @@ public class QuorumStateTest {
state.transitionToCandidate(); state.transitionToCandidate();
state.candidateStateOrThrow().recordGrantedVote(otherNodeId); state.candidateStateOrThrow().recordGrantedVote(otherNodeId);
state.transitionToLeader(0L, accumulator); 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(5, state.epoch());
assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals(OptionalInt.of(otherNodeId), state.leaderId());
@ -589,7 +591,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testLeaderToVoted(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -619,7 +621,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testLeaderToAnyStateLowerEpoch(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -633,7 +635,8 @@ public class QuorumStateTest {
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToFollower( () -> state.transitionToFollower(
4, 4,
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeKey.id(),
voters.listeners(otherNodeKey.id())
) )
); );
assertEquals(6, state.epoch()); assertEquals(6, state.epoch());
@ -660,7 +663,8 @@ public class QuorumStateTest {
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToFollower( () -> state.transitionToFollower(
0, 0,
voters.voterNode(localId, VoterSetTest.DEFAULT_LISTENER_NAME).get() localId,
voters.listeners(localId)
) )
); );
assertThrows(IllegalStateException.class, () -> state.transitionToVoted(0, localVoterKey)); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(0, localVoterKey));
@ -669,7 +673,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testUnattachedToLeaderOrResigned(short kraftVersion) { public void testUnattachedToLeaderOrResigned(short kraftVersion) {
ReplicaKey leaderKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); ReplicaKey leaderKey = ReplicaKey.of(1, Uuid.randomUuid());
int epoch = 5; int epoch = 5;
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leaderKey)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leaderKey));
store.writeElectionState( store.writeElectionState(
@ -686,7 +690,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testUnattachedToVotedSameEpoch(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -719,7 +723,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testUnattachedToVotedHigherEpoch(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -745,7 +749,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testUnattachedToCandidate(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -765,7 +769,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testUnattachedToUnattached(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -786,7 +790,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testUnattachedToFollowerSameEpoch(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -794,14 +798,15 @@ public class QuorumStateTest {
state.transitionToFollower( state.transitionToFollower(
5, 5,
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeKey.id(),
voters.listeners(otherNodeKey.id())
); );
assertTrue(state.isFollower()); assertTrue(state.isFollower());
FollowerState followerState = state.followerStateOrThrow(); FollowerState followerState = state.followerStateOrThrow();
assertEquals(5, followerState.epoch()); assertEquals(5, followerState.epoch());
assertEquals( assertEquals(
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME), voters.listeners(otherNodeKey.id()),
Optional.of(followerState.leader()) followerState.leaderEndpoints()
); );
assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds()));
} }
@ -809,7 +814,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testUnattachedToFollowerHigherEpoch(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -817,14 +822,15 @@ public class QuorumStateTest {
state.transitionToFollower( state.transitionToFollower(
8, 8,
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeKey.id(),
voters.listeners(otherNodeKey.id())
); );
assertTrue(state.isFollower()); assertTrue(state.isFollower());
FollowerState followerState = state.followerStateOrThrow(); FollowerState followerState = state.followerStateOrThrow();
assertEquals(8, followerState.epoch()); assertEquals(8, followerState.epoch());
assertEquals( assertEquals(
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME), voters.listeners(otherNodeKey.id()),
Optional.of(followerState.leader()) followerState.leaderEndpoints()
); );
assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds()));
} }
@ -832,7 +838,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testUnattachedToAnyStateLowerEpoch(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -843,7 +849,8 @@ public class QuorumStateTest {
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToFollower( () -> state.transitionToFollower(
4, 4,
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeKey.id(),
voters.listeners(otherNodeKey.id())
) )
); );
assertEquals(5, state.epoch()); assertEquals(5, state.epoch());
@ -866,7 +873,7 @@ public class QuorumStateTest {
VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); 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.transitionToLeader(0, accumulator));
assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList()));
} }
@ -879,7 +886,7 @@ public class QuorumStateTest {
VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); 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; int jitterMs = 2500;
random.mockNextInt(electionTimeoutMs, jitterMs); random.mockNextInt(electionTimeoutMs, jitterMs);
@ -900,14 +907,14 @@ public class QuorumStateTest {
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToUnattached(5); state.transitionToUnattached(5);
state.transitionToVoted(8, ReplicaKey.of(node1, Optional.of(Uuid.randomUuid()))); state.transitionToVoted(8, ReplicaKey.of(node1, Uuid.randomUuid()));
assertThrows( assertThrows(
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToVoted(8, ReplicaKey.of(node1, Optional.empty())) () -> state.transitionToVoted(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))
); );
assertThrows( assertThrows(
IllegalStateException.class, 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); VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); 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( state.transitionToFollower(
5, 5,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
); );
FollowerState followerState = state.followerStateOrThrow(); FollowerState followerState = state.followerStateOrThrow();
assertEquals(5, followerState.epoch()); assertEquals(5, followerState.epoch());
assertEquals( assertEquals(
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME), voters.listeners(node2),
Optional.of(followerState.leader()) followerState.leaderEndpoints()
); );
assertEquals( assertEquals(
Optional.of( Optional.of(
@ -951,17 +959,18 @@ public class QuorumStateTest {
VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); 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( state.transitionToFollower(
8, 8,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
); );
FollowerState followerState = state.followerStateOrThrow(); FollowerState followerState = state.followerStateOrThrow();
assertEquals(8, followerState.epoch()); assertEquals(8, followerState.epoch());
assertEquals( assertEquals(
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get(), voters.listeners(node2),
followerState.leader() followerState.leaderEndpoints()
); );
assertEquals( assertEquals(
Optional.of( Optional.of(
@ -983,7 +992,7 @@ public class QuorumStateTest {
VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion);
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); 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)); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5));
} }
@ -994,7 +1003,7 @@ public class QuorumStateTest {
VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion);
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); 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()); long remainingElectionTimeMs = state.votedStateOrThrow().remainingElectionTimeMs(time.milliseconds());
time.sleep(1000); time.sleep(1000);
@ -1011,7 +1020,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testVotedToAnyStateLowerEpoch(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -1022,7 +1031,8 @@ public class QuorumStateTest {
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToFollower( () -> state.transitionToFollower(
4, 4,
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeKey.id(),
voters.listeners(otherNodeKey.id())
) )
); );
assertEquals(5, state.epoch()); assertEquals(5, state.epoch());
@ -1048,28 +1058,31 @@ public class QuorumStateTest {
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToFollower( state.transitionToFollower(
8, 8,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
); );
assertThrows( assertThrows(
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToFollower( () -> state.transitionToFollower(
8, 8,
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get() node1,
voters.listeners(node1)
) )
); );
assertThrows( assertThrows(
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToFollower( () -> state.transitionToFollower(
8, 8,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
) )
); );
FollowerState followerState = state.followerStateOrThrow(); FollowerState followerState = state.followerStateOrThrow();
assertEquals(8, followerState.epoch()); assertEquals(8, followerState.epoch());
assertEquals( assertEquals(
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME), voters.listeners(node2),
Optional.of(followerState.leader()) followerState.leaderEndpoints()
); );
assertEquals( assertEquals(
Optional.of( Optional.of(
@ -1093,18 +1106,20 @@ public class QuorumStateTest {
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToFollower( state.transitionToFollower(
8, 8,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
); );
state.transitionToFollower( state.transitionToFollower(
9, 9,
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get() node1,
voters.listeners(node1)
); );
FollowerState followerState = state.followerStateOrThrow(); FollowerState followerState = state.followerStateOrThrow();
assertEquals(9, followerState.epoch()); assertEquals(9, followerState.epoch());
assertEquals( assertEquals(
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME), voters.listeners(node1),
Optional.of(followerState.leader()) followerState.leaderEndpoints()
); );
assertEquals( assertEquals(
Optional.of( Optional.of(
@ -1128,7 +1143,8 @@ public class QuorumStateTest {
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToFollower( state.transitionToFollower(
8, 8,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
); );
assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator));
assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList()));
@ -1144,7 +1160,8 @@ public class QuorumStateTest {
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToFollower( state.transitionToFollower(
8, 8,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
); );
int jitterMs = 2500; int jitterMs = 2500;
@ -1167,7 +1184,8 @@ public class QuorumStateTest {
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToFollower( state.transitionToFollower(
8, 8,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
); );
assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8)); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8));
} }
@ -1182,7 +1200,8 @@ public class QuorumStateTest {
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToFollower( state.transitionToFollower(
8, 8,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
); );
int jitterMs = 2500; int jitterMs = 2500;
@ -1205,35 +1224,37 @@ public class QuorumStateTest {
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToFollower( state.transitionToFollower(
8, 8,
voters.voterNode(node2, VoterSetTest.DEFAULT_LISTENER_NAME).get() node2,
voters.listeners(node2)
); );
assertThrows( assertThrows(
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToVoted(8, ReplicaKey.of(node1, Optional.empty())) () -> state.transitionToVoted(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))
); );
assertThrows( assertThrows(
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToVoted(8, ReplicaKey.of(localId, Optional.empty())) () -> state.transitionToVoted(8, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID))
); );
assertThrows( assertThrows(
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToVoted(8, ReplicaKey.of(node2, Optional.empty())) () -> state.transitionToVoted(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID))
); );
} }
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testFollowerToVotedHigherEpoch(short kraftVersion) { public void testFollowerToVotedHigherEpoch(short kraftVersion) {
ReplicaKey nodeKey1 = ReplicaKey.of(1, Optional.of(Uuid.randomUuid())); ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid());
ReplicaKey nodeKey2 = ReplicaKey.of(2, Optional.of(Uuid.randomUuid())); ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid());
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToFollower( state.transitionToFollower(
8, 8,
voters.voterNode(nodeKey2.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() nodeKey2.id(),
voters.listeners(nodeKey2.id())
); );
int jitterMs = 2500; int jitterMs = 2500;
@ -1259,18 +1280,20 @@ public class QuorumStateTest {
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
state.transitionToFollower( state.transitionToFollower(
5, 5,
voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeId,
voters.listeners(otherNodeId)
); );
assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4));
assertThrows( assertThrows(
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToVoted(4, ReplicaKey.of(otherNodeId, Optional.empty())) () -> state.transitionToVoted(4, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID))
); );
assertThrows( assertThrows(
IllegalStateException.class, IllegalStateException.class,
() -> state.transitionToFollower( () -> state.transitionToFollower(
4, 4,
voters.voterNode(otherNodeId, VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeId,
voters.listeners(otherNodeId)
) )
); );
assertEquals(5, state.epoch()); assertEquals(5, state.epoch());
@ -1290,7 +1313,7 @@ public class QuorumStateTest {
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testCanBecomeFollowerOfNonVoter(short kraftVersion) { public void testCanBecomeFollowerOfNonVoter(short kraftVersion) {
int otherNodeId = 1; 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); VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion);
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.initialize(new OffsetAndEpoch(0L, logEndEpoch));
@ -1304,8 +1327,16 @@ public class QuorumStateTest {
assertEquals(nonVoterKey, votedState.votedKey()); assertEquals(nonVoterKey, votedState.votedKey());
// Transition to follower // Transition to follower
Node nonVoterNode = new Node(nonVoterKey.id(), "non-voter-host", 1234); state.transitionToFollower(
state.transitionToFollower(4, nonVoterNode); 4,
nonVoterKey.id(),
Endpoints.fromInetSocketAddresses(
Collections.singletonMap(
VoterSetTest.DEFAULT_LISTENER_NAME,
InetSocketAddress.createUnresolved("non-voter-host", 1234)
)
)
);
assertEquals( assertEquals(
new LeaderAndEpoch(OptionalInt.of(nonVoterKey.id()), 4), new LeaderAndEpoch(OptionalInt.of(nonVoterKey.id()), 4),
state.leaderAndEpoch() state.leaderAndEpoch()
@ -1330,7 +1361,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testObserverWithIdCanVote(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
@ -1360,7 +1391,8 @@ public class QuorumStateTest {
state.transitionToFollower( state.transitionToFollower(
2, 2,
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get() node1,
voters.listeners(node1)
); );
state.transitionToUnattached(3); state.transitionToUnattached(3);
assertTrue(state.isUnattached()); assertTrue(state.isUnattached());
@ -1385,13 +1417,13 @@ public class QuorumStateTest {
assertTrue(state.isObserver()); assertTrue(state.isObserver());
state.transitionToUnattached(2); state.transitionToUnattached(2);
state.transitionToFollower(3, voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME).get()); state.transitionToFollower(3, node1, voters.listeners(node1));
assertTrue(state.isFollower()); assertTrue(state.isFollower());
FollowerState followerState = state.followerStateOrThrow(); FollowerState followerState = state.followerStateOrThrow();
assertEquals(3, followerState.epoch()); assertEquals(3, followerState.epoch());
assertEquals( assertEquals(
voters.voterNode(node1, VoterSetTest.DEFAULT_LISTENER_NAME), voters.listeners(node1),
Optional.of(followerState.leader()) followerState.leaderEndpoints()
); );
assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds()));
} }
@ -1418,7 +1450,7 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testHasRemoteLeader(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
@ -1439,7 +1471,8 @@ public class QuorumStateTest {
state.transitionToFollower( state.transitionToFollower(
state.epoch() + 1, state.epoch() + 1,
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeKey.id(),
voters.listeners(otherNodeKey.id())
); );
assertTrue(state.hasRemoteLeader()); assertTrue(state.hasRemoteLeader());
} }
@ -1447,13 +1480,14 @@ public class QuorumStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(shorts = {0, 1}) @ValueSource(shorts = {0, 1})
public void testHighWatermarkRetained(short kraftVersion) { 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)); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey));
QuorumState state = initializeEmptyState(voters, kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion);
state.transitionToFollower( state.transitionToFollower(
5, 5,
voters.voterNode(otherNodeKey.id(), VoterSetTest.DEFAULT_LISTENER_NAME).get() otherNodeKey.id(),
voters.listeners(otherNodeKey.id())
); );
FollowerState followerState = state.followerStateOrThrow(); FollowerState followerState = state.followerStateOrThrow();
@ -1495,11 +1529,11 @@ public class QuorumStateTest {
assertThrows(IllegalStateException.class, state::transitionToCandidate); assertThrows(IllegalStateException.class, state::transitionToCandidate);
assertThrows( assertThrows(
IllegalStateException.class, 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)); 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()); assertTrue(state.isFollower());
state.transitionToUnattached(2); state.transitionToUnattached(2);
@ -1519,7 +1553,7 @@ public class QuorumStateTest {
store.writeElectionState( store.writeElectionState(
ElectionState.withVotedCandidate( ElectionState.withVotedCandidate(
epoch, epoch,
ReplicaKey.of(votedId, Optional.empty()), ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID),
voters.voterIds() voters.voterIds()
), ),
kraftVersion kraftVersion

View File

@ -23,12 +23,14 @@ import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.memory.MemoryPool;
import org.apache.kafka.common.message.BeginQuorumEpochRequestData; import org.apache.kafka.common.message.BeginQuorumEpochRequestData;
import org.apache.kafka.common.message.BeginQuorumEpochResponseData; 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;
import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState; import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState;
import org.apache.kafka.common.message.EndQuorumEpochRequestData; import org.apache.kafka.common.message.EndQuorumEpochRequestData;
import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.EndQuorumEpochResponseData;
import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchRequestData;
import org.apache.kafka.common.message.FetchResponseData; 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.FetchSnapshotResponseData;
import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage;
import org.apache.kafka.common.message.LeaderChangeMessage.Voter; 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.MemoryRecords;
import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.Records; 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.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.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.LogContext;
import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.raft.internals.BatchBuilder; import org.apache.kafka.raft.internals.BatchBuilder;
import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.ReplicaKey;
import org.apache.kafka.raft.internals.StringSerde; 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.server.common.serialization.RecordSerde;
import org.apache.kafka.snapshot.RawSnapshotWriter; import org.apache.kafka.snapshot.RawSnapshotWriter;
import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotReader;
@ -79,6 +76,7 @@ import java.util.OptionalLong;
import java.util.Set; import java.util.Set;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.function.Function; import java.util.function.Function;
import java.util.function.UnaryOperator;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
@ -108,6 +106,7 @@ public final class RaftClientTestContext {
private final QuorumStateStore quorumStateStore; private final QuorumStateStore quorumStateStore;
final Uuid clusterId; final Uuid clusterId;
private final OptionalInt localId; private final OptionalInt localId;
public final Uuid localDirectoryId;
public final KafkaRaftClient<String> client; public final KafkaRaftClient<String> client;
final Metrics metrics; final Metrics metrics;
public final MockLog log; public final MockLog log;
@ -115,8 +114,10 @@ public final class RaftClientTestContext {
final MockMessageQueue messageQueue; final MockMessageQueue messageQueue;
final MockTime time; final MockTime time;
final MockListener listener; final MockListener listener;
final Set<Integer> voters; final VoterSet voters;
final Set<Integer> bootstrapIds; 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<>(); private final List<RaftResponse.Outbound> sentResponses = new ArrayList<>();
@ -142,7 +143,6 @@ public final class RaftClientTestContext {
private final Uuid clusterId = Uuid.randomUuid(); private final Uuid clusterId = Uuid.randomUuid();
private final Set<Integer> voters; private final Set<Integer> voters;
private final OptionalInt localId; private final OptionalInt localId;
private final Uuid localDirectoryId = Uuid.randomUuid();
private final short kraftVersion = 0; private final short kraftVersion = 0;
private int requestTimeoutMs = DEFAULT_REQUEST_TIMEOUT_MS; private int requestTimeoutMs = DEFAULT_REQUEST_TIMEOUT_MS;
@ -150,6 +150,7 @@ public final class RaftClientTestContext {
private int appendLingerMs = DEFAULT_APPEND_LINGER_MS; private int appendLingerMs = DEFAULT_APPEND_LINGER_MS;
private MemoryPool memoryPool = MemoryPool.NONE; private MemoryPool memoryPool = MemoryPool.NONE;
private List<InetSocketAddress> bootstrapServers = Collections.emptyList(); private List<InetSocketAddress> bootstrapServers = Collections.emptyList();
private boolean kip853Rpc = false;
public Builder(int localId, Set<Integer> voters) { public Builder(int localId, Set<Integer> voters) {
this(OptionalInt.of(localId), voters); this(OptionalInt.of(localId), voters);
@ -249,6 +250,11 @@ public final class RaftClientTestContext {
return this; return this;
} }
Builder withKip853Rpc(boolean kip853Rpc) {
this.kip853Rpc = kip853Rpc;
return this;
}
public RaftClientTestContext build() throws IOException { public RaftClientTestContext build() throws IOException {
Metrics metrics = new Metrics(time); Metrics metrics = new Metrics(time);
MockNetworkChannel channel = new MockNetworkChannel(); MockNetworkChannel channel = new MockNetworkChannel();
@ -257,6 +263,19 @@ public final class RaftClientTestContext {
.stream() .stream()
.collect(Collectors.toMap(Function.identity(), RaftClientTestContext::mockAddress)); .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( QuorumConfig quorumConfig = new QuorumConfig(
requestTimeoutMs, requestTimeoutMs,
RETRY_BACKOFF_MS, RETRY_BACKOFF_MS,
@ -279,6 +298,7 @@ public final class RaftClientTestContext {
FETCH_MAX_WAIT_MS, FETCH_MAX_WAIT_MS,
clusterId.toString(), clusterId.toString(),
bootstrapServers, bootstrapServers,
localListeners,
logContext, logContext,
random, random,
quorumConfig quorumConfig
@ -294,18 +314,20 @@ public final class RaftClientTestContext {
RaftClientTestContext context = new RaftClientTestContext( RaftClientTestContext context = new RaftClientTestContext(
clusterId, clusterId,
localId, localId,
localDirectoryId,
client, client,
log, log,
channel, channel,
messageQueue, messageQueue,
time, time,
quorumStateStore, quorumStateStore,
voters, voterSet,
IntStream IntStream
.iterate(-2, id -> id - 1) .iterate(-2, id -> id - 1)
.limit(bootstrapServers.size()) .limit(bootstrapServers.size())
.boxed() .boxed()
.collect(Collectors.toSet()), .collect(Collectors.toSet()),
kip853Rpc,
metrics, metrics,
listener listener
); );
@ -318,22 +340,26 @@ public final class RaftClientTestContext {
} }
} }
@SuppressWarnings("ParameterNumber")
private RaftClientTestContext( private RaftClientTestContext(
Uuid clusterId, Uuid clusterId,
OptionalInt localId, OptionalInt localId,
Uuid localDirectoryId,
KafkaRaftClient<String> client, KafkaRaftClient<String> client,
MockLog log, MockLog log,
MockNetworkChannel channel, MockNetworkChannel channel,
MockMessageQueue messageQueue, MockMessageQueue messageQueue,
MockTime time, MockTime time,
QuorumStateStore quorumStateStore, QuorumStateStore quorumStateStore,
Set<Integer> voters, VoterSet voters,
Set<Integer> bootstrapIds, Set<Integer> bootstrapIds,
boolean kip853Rpc,
Metrics metrics, Metrics metrics,
MockListener listener MockListener listener
) { ) {
this.clusterId = clusterId; this.clusterId = clusterId;
this.localId = localId; this.localId = localId;
this.localDirectoryId = localDirectoryId;
this.client = client; this.client = client;
this.log = log; this.log = log;
this.channel = channel; this.channel = channel;
@ -342,6 +368,7 @@ public final class RaftClientTestContext {
this.quorumStateStore = quorumStateStore; this.quorumStateStore = quorumStateStore;
this.voters = voters; this.voters = voters;
this.bootstrapIds = bootstrapIds; this.bootstrapIds = bootstrapIds;
this.kip853Rpc = kip853Rpc;
this.metrics = metrics; this.metrics = metrics;
this.listener = listener; this.listener = listener;
} }
@ -444,6 +471,12 @@ public final class RaftClientTestContext {
return localId.orElseThrow(() -> new AssertionError("Required local id is not defined")); 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 { private void expectBeginEpoch(int epoch) throws Exception {
pollUntilRequest(); pollUntilRequest();
for (RaftRequest.Outbound request : collectBeginEpochRequests(epoch)) { for (RaftRequest.Outbound request : collectBeginEpochRequests(epoch)) {
@ -472,8 +505,8 @@ public final class RaftClientTestContext {
assertEquals( assertEquals(
ElectionState.withVotedCandidate( ElectionState.withVotedCandidate(
epoch, epoch,
ReplicaKey.of(candidateId, Optional.empty()), ReplicaKey.of(candidateId, ReplicaKey.NO_DIRECTORY_ID),
voters voters.voterIds()
), ),
quorumStateStore.readElectionState().get() quorumStateStore.readElectionState().get()
); );
@ -481,14 +514,14 @@ public final class RaftClientTestContext {
public void assertElectedLeader(int epoch, int leaderId) { public void assertElectedLeader(int epoch, int leaderId) {
assertEquals( assertEquals(
ElectionState.withElectedLeader(epoch, leaderId, voters), ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()),
quorumStateStore.readElectionState().get() quorumStateStore.readElectionState().get()
); );
} }
void assertUnknownLeader(int epoch) { void assertUnknownLeader(int epoch) {
assertEquals( assertEquals(
ElectionState.withUnknownLeader(epoch, voters), ElectionState.withUnknownLeader(epoch, voters.voterIds()),
quorumStateStore.readElectionState().get() quorumStateStore.readElectionState().get()
); );
} }
@ -496,7 +529,7 @@ public final class RaftClientTestContext {
void assertResignedLeader(int epoch, int leaderId) { void assertResignedLeader(int epoch, int leaderId) {
assertTrue(client.quorum().isResigned()); assertTrue(client.quorum().isResigned());
assertEquals( assertEquals(
ElectionState.withElectedLeader(epoch, leaderId, voters), ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()),
quorumStateStore.readElectionState().get() quorumStateStore.readElectionState().get()
); );
} }
@ -530,17 +563,16 @@ public final class RaftClientTestContext {
.setObservers(observerStates); .setObservers(observerStates);
DescribeQuorumResponseData.NodeCollection nodes = new DescribeQuorumResponseData.NodeCollection(); DescribeQuorumResponseData.NodeCollection nodes = new DescribeQuorumResponseData.NodeCollection();
voterStates.forEach(replicaState -> {
Consumer<DescribeQuorumResponseData.ReplicaState> addToNodes = replicaState -> { if (kip853Rpc && nodes.find(replicaState.replicaId()) == null) {
if (nodes.find(replicaState.replicaId()) != null) // KAFKA-16953 will add support for including the node listeners in the node
return; // collection
nodes.add(
nodes.add(new DescribeQuorumResponseData.Node() new DescribeQuorumResponseData.Node()
.setNodeId(replicaState.replicaId())); .setNodeId(replicaState.replicaId())
}; );
}
voterStates.forEach(addToNodes); });
observerStates.forEach(addToNodes);
DescribeQuorumResponseData expectedResponse = DescribeQuorumResponse.singletonResponse( DescribeQuorumResponseData expectedResponse = DescribeQuorumResponse.singletonResponse(
metadataPartition, metadataPartition,
@ -585,6 +617,15 @@ public final class RaftClientTestContext {
assertEquals(error, Errors.forCode(partitionResponse.errorCode())); assertEquals(error, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.leaderEpoch()); assertEquals(epoch, partitionResponse.leaderEpoch());
assertEquals(leaderId.orElse(-1), partitionResponse.leaderId()); 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( List<RaftRequest.Outbound> collectVoteRequests(
@ -610,7 +651,12 @@ public final class RaftClientTestContext {
void deliverRequest(ApiMessage request) { void deliverRequest(ApiMessage request) {
RaftRequest.Inbound inboundRequest = new RaftRequest.Inbound( 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) -> { inboundRequest.completion.whenComplete((response, exception) -> {
if (exception != null) { if (exception != null) {
throw new RuntimeException(exception); throw new RuntimeException(exception);
@ -655,6 +701,23 @@ public final class RaftClientTestContext {
assertInstanceOf(BeginQuorumEpochResponseData.class, raftMessage.data()); assertInstanceOf(BeginQuorumEpochResponseData.class, raftMessage.data());
BeginQuorumEpochResponseData response = (BeginQuorumEpochResponseData) raftMessage.data(); BeginQuorumEpochResponseData response = (BeginQuorumEpochResponseData) raftMessage.data();
assertEquals(responseError, Errors.forCode(response.errorCode())); 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( void assertSentBeginQuorumEpochResponse(
@ -675,6 +738,15 @@ public final class RaftClientTestContext {
assertEquals(epoch, partitionResponse.leaderEpoch()); assertEquals(epoch, partitionResponse.leaderEpoch());
assertEquals(leaderId.orElse(-1), partitionResponse.leaderId()); assertEquals(leaderId.orElse(-1), partitionResponse.leaderId());
assertEquals(partitionError, Errors.forCode(partitionResponse.errorCode())); 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) { RaftRequest.Outbound assertSentEndQuorumEpochRequest(int epoch, int destinationId) {
@ -696,6 +768,23 @@ public final class RaftClientTestContext {
assertInstanceOf(EndQuorumEpochResponseData.class, raftMessage.data()); assertInstanceOf(EndQuorumEpochResponseData.class, raftMessage.data());
EndQuorumEpochResponseData response = (EndQuorumEpochResponseData) raftMessage.data(); EndQuorumEpochResponseData response = (EndQuorumEpochResponseData) raftMessage.data();
assertEquals(responseError, Errors.forCode(response.errorCode())); 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( void assertSentEndQuorumEpochResponse(
@ -716,6 +805,15 @@ public final class RaftClientTestContext {
assertEquals(epoch, partitionResponse.leaderEpoch()); assertEquals(epoch, partitionResponse.leaderEpoch());
assertEquals(leaderId.orElse(-1), partitionResponse.leaderId()); assertEquals(leaderId.orElse(-1), partitionResponse.leaderId());
assertEquals(partitionError, Errors.forCode(partitionResponse.errorCode())); 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() { RaftRequest.Outbound assertSentFetchRequest() {
@ -749,7 +847,19 @@ public final class RaftClientTestContext {
assertEquals(1, response.responses().size()); assertEquals(1, response.responses().size());
assertEquals(metadataPartition.topic(), response.responses().get(0).topic()); assertEquals(metadataPartition.topic(), response.responses().get(0).topic());
assertEquals(1, response.responses().get(0).partitions().size()); 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) { void assertSentFetchPartitionResponse(Errors topLevelError) {
@ -776,6 +886,7 @@ public final class RaftClientTestContext {
assertEquals(-1, partitionResponse.divergingEpoch().epoch()); assertEquals(-1, partitionResponse.divergingEpoch().epoch());
assertEquals(-1, partitionResponse.snapshotId().endOffset()); assertEquals(-1, partitionResponse.snapshotId().endOffset());
assertEquals(-1, partitionResponse.snapshotId().epoch()); assertEquals(-1, partitionResponse.snapshotId().epoch());
return (MemoryRecords) partitionResponse.records(); return (MemoryRecords) partitionResponse.records();
} }
@ -802,6 +913,19 @@ public final class RaftClientTestContext {
} }
void assertSentFetchSnapshotResponse(Errors responseError) { 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); List<RaftResponse.Outbound> sentMessages = drainSentResponses(ApiKeys.FETCH_SNAPSHOT);
assertEquals(1, sentMessages.size()); assertEquals(1, sentMessages.size());
@ -810,19 +934,21 @@ public final class RaftClientTestContext {
FetchSnapshotResponseData response = (FetchSnapshotResponseData) message.data(); FetchSnapshotResponseData response = (FetchSnapshotResponseData) message.data();
assertEquals(responseError, Errors.forCode(response.errorCode())); assertEquals(responseError, Errors.forCode(response.errorCode()));
Optional<FetchSnapshotResponseData.PartitionSnapshot> result =
FetchSnapshotResponse.forTopicPartition(response, topicPartition);
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);
} }
Optional<FetchSnapshotResponseData.PartitionSnapshot> assertSentFetchSnapshotResponse(TopicPartition topicPartition) { return result;
List<RaftResponse.Outbound> sentMessages = drainSentResponses(ApiKeys.FETCH_SNAPSHOT);
assertEquals(1, sentMessages.size());
RaftMessage message = sentMessages.get(0);
assertInstanceOf(FetchSnapshotResponseData.class, message.data());
FetchSnapshotResponseData response = (FetchSnapshotResponseData) message.data();
assertEquals(Errors.NONE, Errors.forCode(response.errorCode()));
return FetchSnapshotResponse.forTopicPartition(response, topicPartition);
} }
List<RaftRequest.Outbound> collectEndQuorumRequests( List<RaftRequest.Outbound> collectEndQuorumRequests(
@ -861,7 +987,7 @@ public final class RaftClientTestContext {
RaftRequest.Outbound fetchRequest = assertSentFetchRequest(); RaftRequest.Outbound fetchRequest = assertSentFetchRequest();
int destinationId = fetchRequest.destination().id(); int destinationId = fetchRequest.destination().id();
assertTrue( 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) String.format("id %d is not in sets %s or %s", destinationId, voters, bootstrapIds)
); );
assertFetchRequestData(fetchRequest, 0, 0L, 0); assertFetchRequestData(fetchRequest, 0, 0L, 0);
@ -892,32 +1018,35 @@ public final class RaftClientTestContext {
} }
public static InetSocketAddress mockAddress(int id) { public static InetSocketAddress mockAddress(int id) {
return new InetSocketAddress("localhost", 9990 + id); return InetSocketAddress.createUnresolved("localhost", 9990 + id);
} }
EndQuorumEpochResponseData endEpochResponse( EndQuorumEpochResponseData endEpochResponse(
int epoch, int epoch,
OptionalInt leaderId OptionalInt leaderId
) { ) {
return EndQuorumEpochResponse.singletonResponse( return RaftUtil.singletonEndQuorumEpochResponse(
channel.listenerName(),
endQuorumEpochRpcVersion(),
Errors.NONE, Errors.NONE,
metadataPartition, metadataPartition,
Errors.NONE, Errors.NONE,
epoch, epoch,
leaderId.orElse(-1) leaderId.orElse(-1),
Endpoints.empty() // KAFKA-16529 will fix this
); );
} }
EndQuorumEpochRequestData endEpochRequest( EndQuorumEpochRequestData endEpochRequest(
int epoch, int epoch,
int leaderId, int leaderId,
List<Integer> preferredSuccessors List<ReplicaKey> preferredCandidates
) { ) {
return EndQuorumEpochRequest.singletonRequest( return endEpochRequest(
metadataPartition, clusterId.toString(),
epoch, epoch,
leaderId, leaderId,
preferredSuccessors preferredCandidates
); );
} }
@ -925,50 +1054,68 @@ public final class RaftClientTestContext {
String clusterId, String clusterId,
int epoch, int epoch,
int leaderId, int leaderId,
List<Integer> preferredSuccessors List<ReplicaKey> preferredCandidates
) { ) {
return EndQuorumEpochRequest.singletonRequest( return RaftUtil.singletonEndQuorumEpochRequest(
metadataPartition, metadataPartition,
clusterId, clusterId,
epoch, epoch,
leaderId, leaderId,
preferredSuccessors preferredCandidates
);
}
BeginQuorumEpochRequestData beginEpochRequest(String clusterId, int epoch, int leaderId) {
return BeginQuorumEpochRequest.singletonRequest(
metadataPartition,
clusterId,
epoch,
leaderId
); );
} }
BeginQuorumEpochRequestData beginEpochRequest(int epoch, int leaderId) { 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, metadataPartition,
clusterId,
epoch, epoch,
leaderId leaderId,
voters.listeners(leaderId),
voterKey
); );
} }
private BeginQuorumEpochResponseData beginEpochResponse(int epoch, int leaderId) { private BeginQuorumEpochResponseData beginEpochResponse(int epoch, int leaderId) {
return BeginQuorumEpochResponse.singletonResponse( return RaftUtil.singletonBeginQuorumEpochResponse(
channel.listenerName(),
beginQuorumEpochRpcVersion(),
Errors.NONE, Errors.NONE,
metadataPartition, metadataPartition,
Errors.NONE, Errors.NONE,
epoch, epoch,
leaderId leaderId,
Endpoints.empty() // KAFKA-16529 will fix this
); );
} }
VoteRequestData voteRequest(int epoch, int candidateId, int lastEpoch, long lastEpochOffset) { VoteRequestData voteRequest(
return VoteRequest.singletonRequest( int epoch,
metadataPartition, ReplicaKey candidateKey,
int lastEpoch,
long lastEpochOffset
) {
return voteRequest(
clusterId.toString(), clusterId.toString(),
epoch, epoch,
candidateId, candidateKey,
lastEpoch, lastEpoch,
lastEpochOffset lastEpochOffset
); );
@ -977,28 +1124,54 @@ public final class RaftClientTestContext {
VoteRequestData voteRequest( VoteRequestData voteRequest(
String clusterId, String clusterId,
int epoch, int epoch,
int candidateId, ReplicaKey candidateKey,
int lastEpoch, int lastEpoch,
long lastEpochOffset 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, metadataPartition,
clusterId, clusterId,
epoch, epoch,
candidateId, candidateKey,
voterKey,
lastEpoch, lastEpoch,
lastEpochOffset lastEpochOffset
); );
} }
VoteResponseData voteResponse(boolean voteGranted, Optional<Integer> leaderId, int epoch) { VoteResponseData voteResponse(boolean voteGranted, Optional<Integer> leaderId, int epoch) {
return VoteResponse.singletonResponse( return RaftUtil.singletonVoteResponse(
channel.listenerName(),
voteRpcVersion(),
Errors.NONE, Errors.NONE,
metadataPartition, metadataPartition,
Errors.NONE, Errors.NONE,
epoch, epoch,
leaderId.orElse(-1), 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()); assertEquals(localId.orElse(-1), request.replicaState().replicaId());
// Assert that voters have flushed up to the fetch offset // 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( assertEquals(
log.firstUnflushedOffset(), log.firstUnflushedOffset(),
fetchOffset, fetchOffset,
@ -1079,7 +1252,7 @@ public final class RaftClientTestContext {
FetchRequestData fetchRequest( FetchRequestData fetchRequest(
int epoch, int epoch,
int replicaId, ReplicaKey replicaKey,
long fetchOffset, long fetchOffset,
int lastFetchedEpoch, int lastFetchedEpoch,
int maxWaitTimeMs int maxWaitTimeMs
@ -1087,7 +1260,7 @@ public final class RaftClientTestContext {
return fetchRequest( return fetchRequest(
epoch, epoch,
clusterId.toString(), clusterId.toString(),
replicaId, replicaKey,
fetchOffset, fetchOffset,
lastFetchedEpoch, lastFetchedEpoch,
maxWaitTimeMs maxWaitTimeMs
@ -1097,21 +1270,28 @@ public final class RaftClientTestContext {
FetchRequestData fetchRequest( FetchRequestData fetchRequest(
int epoch, int epoch,
String clusterId, String clusterId,
int replicaId, ReplicaKey replicaKey,
long fetchOffset, long fetchOffset,
int lastFetchedEpoch, int lastFetchedEpoch,
int maxWaitTimeMs int maxWaitTimeMs
) { ) {
FetchRequestData request = RaftUtil.singletonFetchRequest(metadataPartition, metadataTopicId, fetchPartition -> { FetchRequestData request = RaftUtil.singletonFetchRequest(
fetchPartition metadataPartition,
metadataTopicId,
fetchPartition -> fetchPartition
.setCurrentLeaderEpoch(epoch) .setCurrentLeaderEpoch(epoch)
.setLastFetchedEpoch(lastFetchedEpoch) .setLastFetchedEpoch(lastFetchedEpoch)
.setFetchOffset(fetchOffset); .setFetchOffset(fetchOffset)
}); .setReplicaDirectoryId(
replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)
)
);
return request return request
.setMaxWaitMs(maxWaitTimeMs) .setMaxWaitMs(maxWaitTimeMs)
.setClusterId(clusterId) .setClusterId(clusterId)
.setReplicaState(new FetchRequestData.ReplicaState().setReplicaId(replicaId)); .setReplicaState(
new FetchRequestData.ReplicaState().setReplicaId(replicaKey.id())
);
} }
FetchResponseData fetchResponse( FetchResponseData fetchResponse(
@ -1121,7 +1301,15 @@ public final class RaftClientTestContext {
long highWatermark, long highWatermark,
Errors error Errors error
) { ) {
return RaftUtil.singletonFetchResponse(metadataPartition, metadataTopicId, Errors.NONE, partitionData -> { return RaftUtil.singletonFetchResponse(
channel.listenerName(),
fetchRpcVersion(),
metadataPartition,
metadataTopicId,
Errors.NONE,
leaderId,
Endpoints.empty(), // KAFKA-16529 will fix this
partitionData -> {
partitionData partitionData
.setRecords(records) .setRecords(records)
.setErrorCode(error.code()) .setErrorCode(error.code())
@ -1130,7 +1318,8 @@ public final class RaftClientTestContext {
partitionData.currentLeader() partitionData.currentLeader()
.setLeaderEpoch(epoch) .setLeaderEpoch(epoch)
.setLeaderId(leaderId); .setLeaderId(leaderId);
}); }
);
} }
FetchResponseData divergingFetchResponse( FetchResponseData divergingFetchResponse(
@ -1140,7 +1329,15 @@ public final class RaftClientTestContext {
int divergingEpoch, int divergingEpoch,
long highWatermark long highWatermark
) { ) {
return RaftUtil.singletonFetchResponse(metadataPartition, metadataTopicId, Errors.NONE, partitionData -> { return RaftUtil.singletonFetchResponse(
channel.listenerName(),
fetchRpcVersion(),
metadataPartition,
metadataTopicId,
Errors.NONE,
leaderId,
Endpoints.empty(), // KAFKA-16529 will fix this
partitionData -> {
partitionData.setHighWatermark(highWatermark); partitionData.setHighWatermark(highWatermark);
partitionData.currentLeader() partitionData.currentLeader()
@ -1150,19 +1347,138 @@ public final class RaftClientTestContext {
partitionData.divergingEpoch() partitionData.divergingEpoch()
.setEpoch(divergingEpoch) .setEpoch(divergingEpoch)
.setEndOffset(divergingEpochEndOffset); .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 { public void advanceLocalLeaderHighWatermarkToLogEndOffset() throws InterruptedException {
assertEquals(localId, currentLeader()); assertEquals(localId, currentLeader());
long localLogEndOffset = log.endOffset().offset; long localLogEndOffset = log.endOffset().offset;
Set<Integer> followers = voters.stream().filter(voter -> voter != localId.getAsInt()).collect(Collectors.toSet());
// Send a request from every follower Iterable<ReplicaKey> followers = () -> voters
for (int follower : followers) { .voterKeys()
.stream()
.filter(voterKey -> voterKey.id() != localId.getAsInt())
.iterator();
// Send a request from every voter
for (ReplicaKey follower : followers) {
deliverRequest( deliverRequest(
fetchRequest(currentEpoch(), follower, localLogEndOffset, currentEpoch(), 0) fetchRequest(currentEpoch(), follower, localLogEndOffset, currentEpoch(), 0)
); );
pollUntilResponse(); pollUntilResponse();
assertSentFetchPartitionResponse(Errors.NONE, currentEpoch(), localId); assertSentFetchPartitionResponse(Errors.NONE, currentEpoch(), localId);
} }

View File

@ -20,6 +20,7 @@ import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.memory.MemoryPool; 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.metrics.Metrics;
import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.protocol.Readable;
@ -772,6 +773,7 @@ public class RaftEventSimulationTest {
FETCH_MAX_WAIT_MS, FETCH_MAX_WAIT_MS,
clusterId.toString(), clusterId.toString(),
Collections.emptyList(), Collections.emptyList(),
Endpoints.empty(), // KAFKA-16529 will fix this
logContext, logContext,
random, random,
quorumConfig quorumConfig
@ -1236,8 +1238,20 @@ public class RaftEventSimulationTest {
int correlationId = outbound.correlationId(); int correlationId = outbound.correlationId();
Node destination = outbound.destination(); Node destination = outbound.destination();
RaftRequest.Inbound inbound = new RaftRequest.Inbound(correlationId, outbound.apiVersion(), outbound.data(), RaftRequest.Inbound inbound = cluster
cluster.time.milliseconds()); .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)) if (!filters.get(destination.id()).acceptInbound(inbound))
return; return;

View File

@ -20,18 +20,20 @@ import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.ReplicaKey;
import org.apache.kafka.raft.internals.VoterSetTest;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource; import org.junit.jupiter.params.provider.ValueSource;
import java.net.InetSocketAddress;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; 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.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
@ -42,6 +44,12 @@ class ResignedStateTest {
int electionTimeoutMs = 5000; int electionTimeoutMs = 5000;
int localId = 0; int localId = 0;
int epoch = 5; int epoch = 5;
Endpoints localEndpoints = Endpoints.fromInetSocketAddresses(
Collections.singletonMap(
VoterSetTest.DEFAULT_LISTENER_NAME,
InetSocketAddress.createUnresolved("localhost", 1234)
)
);
private ResignedState newResignedState( private ResignedState newResignedState(
Set<Integer> voters, Set<Integer> voters,
@ -54,6 +62,7 @@ class ResignedStateTest {
voters, voters,
electionTimeoutMs, electionTimeoutMs,
preferredSuccessors, preferredSuccessors,
localEndpoints,
logContext logContext
); );
} }
@ -90,9 +99,9 @@ class ResignedStateTest {
Collections.emptyList() Collections.emptyList()
); );
assertFalse(state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
assertFalse(state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
assertFalse(state.canGrantVote(ReplicaKey.of(3, Optional.empty()), isLogUpToDate)); assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
} }
@Test @Test
@ -107,4 +116,15 @@ class ResignedStateTest {
// try non-existed voter must throw an exception // try non-existed voter must throw an exception
assertThrows(IllegalArgumentException.class, () -> state.acknowledgeResignation(10)); 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());
}
} }

View File

@ -84,15 +84,25 @@ public class UnattachedStateTest {
assertEquals( assertEquals(
isLogUpToDate, isLogUpToDate,
state.canGrantVote(ReplicaKey.of(1, Optional.empty()), isLogUpToDate) state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)
); );
assertEquals( assertEquals(
isLogUpToDate, isLogUpToDate,
state.canGrantVote(ReplicaKey.of(2, Optional.empty()), isLogUpToDate) state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)
); );
assertEquals( assertEquals(
isLogUpToDate, 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());
}
} }

View File

@ -41,15 +41,14 @@ class VotedStateTest {
private final int electionTimeoutMs = 10000; private final int electionTimeoutMs = 10000;
private VotedState newVotedState( private VotedState newVotedState(
Optional<Uuid> votedDirectoryId, Uuid votedDirectoryId
Optional<LogOffsetMetadata> highWatermark
) { ) {
return new VotedState( return new VotedState(
time, time,
epoch, epoch,
ReplicaKey.of(votedId, votedDirectoryId), ReplicaKey.of(votedId, votedDirectoryId),
Collections.emptySet(), Collections.emptySet(),
highWatermark, Optional.empty(),
electionTimeoutMs, electionTimeoutMs,
logContext logContext
); );
@ -57,8 +56,8 @@ class VotedStateTest {
@Test @Test
public void testElectionTimeout() { public void testElectionTimeout() {
VotedState state = newVotedState(Optional.empty(), Optional.empty()); VotedState state = newVotedState(ReplicaKey.NO_DIRECTORY_ID);
ReplicaKey votedKey = ReplicaKey.of(votedId, Optional.empty()); ReplicaKey votedKey = ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID);
assertEquals(epoch, state.epoch()); assertEquals(epoch, state.epoch());
assertEquals(votedKey, state.votedKey()); assertEquals(votedKey, state.votedKey());
@ -81,36 +80,44 @@ class VotedStateTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(booleans = {true, false}) @ValueSource(booleans = {true, false})
public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) {
VotedState state = newVotedState(Optional.empty(), Optional.empty()); VotedState state = newVotedState(ReplicaKey.NO_DIRECTORY_ID);
assertTrue( assertTrue(
state.canGrantVote(ReplicaKey.of(votedId, Optional.empty()), isLogUpToDate) state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)
); );
assertTrue( assertTrue(
state.canGrantVote( state.canGrantVote(
ReplicaKey.of(votedId, Optional.of(Uuid.randomUuid())), ReplicaKey.of(votedId, Uuid.randomUuid()),
isLogUpToDate isLogUpToDate
) )
); );
assertFalse( assertFalse(
state.canGrantVote(ReplicaKey.of(votedId + 1, Optional.empty()), isLogUpToDate) state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)
); );
} }
@Test @Test
void testCanGrantVoteWithDirectoryId() { void testCanGrantVoteWithDirectoryId() {
Optional<Uuid> votedDirectoryId = Optional.of(Uuid.randomUuid()); Uuid votedDirectoryId = Uuid.randomUuid();
VotedState state = newVotedState(votedDirectoryId, Optional.empty()); VotedState state = newVotedState(votedDirectoryId);
assertTrue(state.canGrantVote(ReplicaKey.of(votedId, votedDirectoryId), false)); assertTrue(state.canGrantVote(ReplicaKey.of(votedId, votedDirectoryId), false));
assertFalse( 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, 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());
} }
} }

View File

@ -34,7 +34,6 @@ import org.mockito.Mockito;
import java.util.Collections; import java.util.Collections;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.OptionalInt; import java.util.OptionalInt;
import java.util.OptionalLong; import java.util.OptionalLong;
import java.util.Random; import java.util.Random;
@ -68,9 +67,9 @@ public class KafkaRaftMetricsTest {
return new QuorumState( return new QuorumState(
OptionalInt.of(localId), OptionalInt.of(localId),
localDirectoryId, localDirectoryId,
VoterSetTest.DEFAULT_LISTENER_NAME,
() -> voterSet, () -> voterSet,
() -> kraftVersion, () -> kraftVersion,
voterSet.listeners(localId),
electionTimeoutMs, electionTimeoutMs,
fetchTimeoutMs, fetchTimeoutMs,
new MockQuorumStateStore(), new MockQuorumStateStore(),
@ -88,7 +87,7 @@ public class KafkaRaftMetricsTest {
VoterSetTest.voterNode( VoterSetTest.voterNode(
ReplicaKey.of( ReplicaKey.of(
localId, localId,
withDirectoryId ? Optional.of(localDirectoryId) : Optional.empty() withDirectoryId ? localDirectoryId : ReplicaKey.NO_DIRECTORY_ID
) )
) )
) )
@ -105,7 +104,7 @@ public class KafkaRaftMetricsTest {
VoterSetTest.voterNode( VoterSetTest.voterNode(
ReplicaKey.of( ReplicaKey.of(
localId, 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) 1, getMetric(metrics, "current-epoch").metricValue());
assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue());
state.leaderStateOrThrow().updateLocalState(new LogOffsetMetadata(5L), voters.voters()); state.leaderStateOrThrow().updateLocalState(new LogOffsetMetadata(5L), voters);
state.leaderStateOrThrow().updateReplicaState(1, Uuid.randomUuid(), 0, new LogOffsetMetadata(5L)); state.leaderStateOrThrow().updateReplicaState(
voterMap.get(1).voterKey(),
0,
new LogOffsetMetadata(5L)
);
assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); 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("follower", getMetric(metrics, "current-state").metricValue());
assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue());
assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue());
@ -166,7 +169,7 @@ public class KafkaRaftMetricsTest {
state.followerStateOrThrow().updateHighWatermark(OptionalLong.of(10L)); state.followerStateOrThrow().updateHighWatermark(OptionalLong.of(10L));
assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); 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("voted", getMetric(metrics, "current-state").metricValue());
assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue());
assertEquals((double) 2, getMetric(metrics, "current-vote").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) 0, getMetric(metrics, "current-epoch").metricValue());
assertEquals((double) -1L, getMetric(metrics, "high-watermark").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("observer", getMetric(metrics, "current-state").metricValue());
assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue());
assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue());

View File

@ -21,8 +21,11 @@ import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.feature.SupportedVersionRange;
import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.raft.Endpoints;
import org.junit.jupiter.api.Test; 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.net.InetSocketAddress;
import java.util.Arrays; 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.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; 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.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; 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); Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(IntStream.of(1, 2, 3), true);
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); 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, ReplicaKey.NO_DIRECTORY_ID)));
assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Optional.of(Uuid.randomUuid())))); assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Uuid.randomUuid())));
VoterSet.VoterNode voter3 = aVoterMap.remove(3); VoterSet.VoterNode voter3 = aVoterMap.remove(3);
assertEquals( assertEquals(
@ -117,15 +121,15 @@ public final class VoterSetTest {
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
assertTrue(voterSet.isVoter(aVoterMap.get(1).voterKey())); assertTrue(voterSet.isVoter(aVoterMap.get(1).voterKey()));
assertFalse(voterSet.isVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid())))); assertFalse(voterSet.isVoter(ReplicaKey.of(1, Uuid.randomUuid())));
assertFalse(voterSet.isVoter(ReplicaKey.of(1, Optional.empty()))); assertFalse(voterSet.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
assertFalse( assertFalse(
voterSet.isVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId())) voterSet.isVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId().get()))
); );
assertFalse( 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 @Test
@ -133,10 +137,46 @@ public final class VoterSetTest {
Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(IntStream.of(1, 2, 3), false); Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(IntStream.of(1, 2, 3), false);
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
assertTrue(voterSet.isVoter(ReplicaKey.of(1, Optional.empty()))); assertTrue(voterSet.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
assertTrue(voterSet.isVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid())))); assertTrue(voterSet.isVoter(ReplicaKey.of(1, Uuid.randomUuid())));
assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.of(Uuid.randomUuid())))); assertFalse(voterSet.isVoter(ReplicaKey.of(4, Uuid.randomUuid())));
assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.empty()))); 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 @Test
@ -145,12 +185,12 @@ public final class VoterSetTest {
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
assertTrue(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey())); assertTrue(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey()));
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid())))); assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Uuid.randomUuid())));
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.empty()))); assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
assertFalse( 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 @Test
@ -159,15 +199,15 @@ public final class VoterSetTest {
VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
assertFalse(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey())); assertFalse(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey()));
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.of(Uuid.randomUuid())))); assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Uuid.randomUuid())));
assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.empty()))); assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)));
assertFalse( assertFalse(
voterSet.isOnlyVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId())) voterSet.isOnlyVoter(ReplicaKey.of(2, aVoterMap.get(1).voterKey().directoryId().get()))
); );
assertFalse( 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 @Test
@ -272,7 +312,7 @@ public final class VoterSetTest {
return voterNode( return voterNode(
ReplicaKey.of( ReplicaKey.of(
id, id,
withDirectoryId ? Optional.of(Uuid.randomUuid()) : Optional.empty() withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID
) )
); );
} }
@ -280,12 +320,14 @@ public final class VoterSetTest {
public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) { public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) {
return new VoterSet.VoterNode( return new VoterSet.VoterNode(
replicaKey, replicaKey,
Endpoints.fromInetSocketAddresses(
Collections.singletonMap( Collections.singletonMap(
DEFAULT_LISTENER_NAME, DEFAULT_LISTENER_NAME,
InetSocketAddress.createUnresolved( InetSocketAddress.createUnresolved(
String.format("replica-%d", replicaKey.id()), String.format("replica-%d", replicaKey.id()),
1234 1234
) )
)
), ),
new SupportedVersionRange((short) 0, (short) 0) new SupportedVersionRange((short) 0, (short) 0)
); );

View File

@ -408,7 +408,10 @@ public enum MetadataVersion {
} }
public short fetchRequestVersion() { 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; return 16;
} else if (this.isAtLeast(IBP_3_5_IV1)) { } else if (this.isAtLeast(IBP_3_5_IV1)) {
return 15; return 15;